diff --git a/akka-actor/src/main/scala/akka/util/WallClock.scala b/akka-actor/src/main/scala/akka/util/WallClock.scala new file mode 100644 index 0000000000..3254f52e31 --- /dev/null +++ b/akka-actor/src/main/scala/akka/util/WallClock.scala @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 + } + } + } + } +} diff --git a/akka-cluster-sharding-typed/src/main/mima-filters/2.6.8.backwards.excludes/replicated-event-sourcing.excludes b/akka-cluster-sharding-typed/src/main/mima-filters/2.6.8.backwards.excludes/replicated-event-sourcing.excludes new file mode 100644 index 0000000000..4e794fd578 --- /dev/null +++ b/akka-cluster-sharding-typed/src/main/mima-filters/2.6.8.backwards.excludes/replicated-event-sourcing.excludes @@ -0,0 +1,2 @@ +# new private method on type that is do not inherit +ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.sharding.typed.scaladsl.EntityRef.asJava") \ No newline at end of file diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/Murmur2MessageExtractor.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/Murmur2MessageExtractor.scala index 166a620218..631191270c 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/Murmur2MessageExtractor.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/Murmur2MessageExtractor.scala @@ -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. diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedEntityProvider.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedEntityProvider.scala new file mode 100644 index 0000000000..4a02335d49 --- /dev/null +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedEntityProvider.scala @@ -0,0 +1,185 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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]]) diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingExtension.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingExtension.scala new file mode 100644 index 0000000000..af35f26cc7 --- /dev/null +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingExtension.scala @@ -0,0 +1,76 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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]] +} diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingDirectReplication.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingDirectReplication.scala new file mode 100644 index 0000000000..1b87757542 --- /dev/null +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingDirectReplication.scala @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 + } + } + +} diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ClusterShardingImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ClusterShardingImpl.scala index ad30aeb6dd..f7a7138b02 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ClusterShardingImpl.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ClusterShardingImpl.scala @@ -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 } /** diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ReplicatedShardingExtensionImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ReplicatedShardingExtensionImpl.scala new file mode 100644 index 0000000000..57ef3965e1 --- /dev/null +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ReplicatedShardingExtensionImpl.scala @@ -0,0 +1,100 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 + +} diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/testkit/TestEntityRefImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/testkit/TestEntityRefImpl.scala index 2c11f5bb8f..08a8d82217 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/testkit/TestEntityRefImpl.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/testkit/TestEntityRefImpl.scala @@ -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 } diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/javadsl/ClusterSharding.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/javadsl/ClusterSharding.scala index 16e7c4af96..140631eddd 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/javadsl/ClusterSharding.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/javadsl/ClusterSharding.scala @@ -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] => /** diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ClusterSharding.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ClusterSharding.scala index 4aa97ff6a0..5bd2287750 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ClusterSharding.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/scaladsl/ClusterSharding.scala @@ -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 { diff --git a/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala new file mode 100644 index 0000000000..3d39c47991 --- /dev/null +++ b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala @@ -0,0 +1,189 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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") + } + } +} diff --git a/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ReplicatedShardingTest.java b/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ReplicatedShardingTest.java new file mode 100644 index 0000000000..da813a698b --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ReplicatedShardingTest.java @@ -0,0 +1,256 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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> { + 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 replyTo; + + public GetTexts(ActorRef replyTo) { + this.replyTo = replyTo; + } + } + + static class Texts { + public final Set texts; + + public Texts(Set texts) { + this.texts = texts; + } + } + + static Behavior create(ReplicationId replicationId) { + return ReplicatedEventSourcing.commonJournalConfig( + replicationId, + ALL_REPLICAS, + PersistenceTestKitReadJournal.Identifier(), + MyReplicatedStringSet::new); + } + + private MyReplicatedStringSet(ReplicationContext replicationContext) { + super(replicationContext); + } + + @Override + public Set emptyState() { + return new HashSet<>(); + } + + @Override + public CommandHandler> 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, String> eventHandler() { + return newEventHandlerBuilder() + .forAnyState() + .onAnyEvent( + (state, text) -> { + state.add(text); + return state; + }); + } + } + + public static class ProxyActor extends AbstractBehavior { + 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 create() { + return Behaviors.setup(ProxyActor::new); + } + + public static final Set ALL_REPLICAS = + Collections.unmodifiableSet( + new HashSet<>( + Arrays.asList( + new ReplicaId("DC-A"), new ReplicaId("DC-B"), new ReplicaId("DC-C")))); + + private final ReplicatedSharding replicatedSharding; + + private ProxyActor(ActorContext context) { + super(context); + + // #bootstrap + ReplicatedEntityProvider 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 replicatedSharding = + extension.init(replicatedEntityProvider); + // #bootstrap + + this.replicatedSharding = replicatedSharding; + } + + @Override + public Receive createReceive() { + return newReceiveBuilder() + .onMessage(ForwardToRandom.class, this::onForwardToRandom) + .onMessage(ForwardToAll.class, this::onForwardToAll) + .build(); + } + + private Behavior onForwardToRandom(ForwardToRandom forwardToRandom) { + Map> 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 onForwardToAll(ForwardToAll forwardToAll) { + // #all-entity-refs + Map> 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 testProbe = testKit.createTestProbe(); + testProbe.awaitAssert( + () -> { + assertEquals(MemberStatus.up(), node.selfMember().status()); + return null; + }); + + // forward messages to replicas + ActorRef 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 responseProbe = testKit.createTestProbe(); + proxy.tell( + new ProxyActor.ForwardToAll( + "id1", new MyReplicatedStringSet.GetTexts(responseProbe.ref()))); + List responses = responseProbe.receiveSeveralMessages(3); + Set uniqueTexts = + responses.stream().flatMap(res -> res.texts.stream()).collect(Collectors.toSet()); + assertEquals(new HashSet<>(Arrays.asList("to-all", "to-random")), uniqueTexts); + return null; + }); + } +} diff --git a/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java new file mode 100644 index 0000000000..afc488bf64 --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java @@ -0,0 +1,99 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 myEventSourcedBehavior(ReplicationId replicationId) { + return null; + } + + public static final Set ALL_REPLICAS = + Collections.unmodifiableSet( + new HashSet<>( + Arrays.asList(new ReplicaId("DC-A"), new ReplicaId("DC-B"), new ReplicaId("DC-C")))); + + public static ReplicatedEntityProvider 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 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 replicatedSharding = extension.init(provider()); + + Map> myEntityId = + replicatedSharding.getEntityRefsFor("myEntityId"); + // #sending-messages + + } +} diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingDirectReplicationSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingDirectReplicationSpec.scala new file mode 100644 index 0000000000..ae19cb35ad --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingDirectReplicationSpec.scala @@ -0,0 +1,54 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 + } + + } + +} diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala new file mode 100644 index 0000000000..6fd6d0925a --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala @@ -0,0 +1,294 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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)) + } + } + } + +} diff --git a/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.scala b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.scala new file mode 100644 index 0000000000..3954ea139c --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.scala @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 +} diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterSharding.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterSharding.scala index c9be7625d0..b21b890397 100755 --- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterSharding.scala +++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/ClusterSharding.scala @@ -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 diff --git a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala index c9d8c9388b..b2d2aea3e4 100644 --- a/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala +++ b/akka-cluster-sharding/src/main/scala/akka/cluster/sharding/Shard.scala @@ -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 = { diff --git a/akka-cluster/src/main/scala/akka/cluster/Member.scala b/akka-cluster/src/main/scala/akka/cluster/Member.scala index 6af9765a09..1f01f53cbe 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Member.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Member.scala @@ -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) diff --git a/akka-docs/src/main/paradox/persistence.md b/akka-docs/src/main/paradox/persistence.md index 1aa95c1858..6c58f2f6dd 100644 --- a/akka-docs/src/main/paradox/persistence.md +++ b/akka-docs/src/main/paradox/persistence.md @@ -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. diff --git a/akka-docs/src/main/paradox/project/examples.md b/akka-docs/src/main/paradox/project/examples.md index af9e879ffe..d108262df8 100644 --- a/akka-docs/src/main/paradox/project/examples.md +++ b/akka-docs/src/main/paradox/project/examples.md @@ -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 diff --git a/akka-docs/src/main/paradox/serialization-jackson.md b/akka-docs/src/main/paradox/serialization-jackson.md index 21b3aa1496..c2e387b06f 100644 --- a/akka-docs/src/main/paradox/serialization-jackson.md +++ b/akka-docs/src/main/paradox/serialization-jackson.md @@ -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 diff --git a/akka-docs/src/main/paradox/typed/cluster-dc.md b/akka-docs/src/main/paradox/typed/cluster-dc.md index ebe05ed07e..6cf331b541 100644 --- a/akka-docs/src/main/paradox/typed/cluster-dc.md +++ b/akka-docs/src/main/paradox/typed/cluster-dc.md @@ -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 diff --git a/akka-docs/src/main/paradox/typed/guide/modules.md b/akka-docs/src/main/paradox/typed/guide/modules.md index 18c703640c..8580d49a99 100644 --- a/akka-docs/src/main/paradox/typed/guide/modules.md +++ b/akka-docs/src/main/paradox/typed/guide/modules.md @@ -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. diff --git a/akka-docs/src/main/paradox/typed/images/causality.png b/akka-docs/src/main/paradox/typed/images/causality.png new file mode 100644 index 0000000000..cc19c839aa Binary files /dev/null and b/akka-docs/src/main/paradox/typed/images/causality.png differ diff --git a/akka-docs/src/main/paradox/typed/images/lww.png b/akka-docs/src/main/paradox/typed/images/lww.png new file mode 100644 index 0000000000..c6e9f20480 Binary files /dev/null and b/akka-docs/src/main/paradox/typed/images/lww.png differ diff --git a/akka-docs/src/main/paradox/typed/images/replicated-events1.png b/akka-docs/src/main/paradox/typed/images/replicated-events1.png new file mode 100644 index 0000000000..9688dae998 Binary files /dev/null and b/akka-docs/src/main/paradox/typed/images/replicated-events1.png differ diff --git a/akka-docs/src/main/paradox/typed/images/replicated-events2.png b/akka-docs/src/main/paradox/typed/images/replicated-events2.png new file mode 100644 index 0000000000..ae4882dd68 Binary files /dev/null and b/akka-docs/src/main/paradox/typed/images/replicated-events2.png differ diff --git a/akka-docs/src/main/paradox/typed/index-persistence.md b/akka-docs/src/main/paradox/typed/index-persistence.md index 1604b1aece..739a8d2349 100644 --- a/akka-docs/src/main/paradox/typed/index-persistence.md +++ b/akka-docs/src/main/paradox/typed/index-persistence.md @@ -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) @@@ diff --git a/akka-docs/src/main/paradox/typed/persistence.md b/akka-docs/src/main/paradox/typed/persistence.md index eae94d72e7..c55aa91aaa 100644 --- a/akka-docs/src/main/paradox/typed/persistence.md +++ b/akka-docs/src/main/paradox/typed/persistence.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. diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing-auction.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-auction.md new file mode 100644 index 0000000000..a10bc9ba9f --- /dev/null +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-auction.md @@ -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. + + + diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing-cart.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-cart.md new file mode 100644 index 0000000000..09da74c292 --- /dev/null +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-cart.md @@ -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. diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md new file mode 100644 index 0000000000..e0f17977d1 --- /dev/null +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md @@ -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) + +@@@ + diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md new file mode 100644 index 0000000000..52770e4b95 --- /dev/null +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -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. + +![images/replicated-events1.png](images/replicated-events1.png) + +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. + +![images/replicated-events2.png](images/replicated-events2.png) + +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. + +![images/lww.png](images/lww.png) + +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). + +![images/causality.png](images/causality.png) + +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. diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/EventEnvelope.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/EventEnvelope.scala index 684d7b9324..ac9ed891b4 100644 --- a/akka-persistence-query/src/main/scala/akka/persistence/query/EventEnvelope.scala +++ b/akka-persistence-query/src/main/scala/akka/persistence/query/EventEnvelope.scala @@ -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" diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala index 70aebd84b8..dcfc755322 100644 --- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala +++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByPersistenceIdStage.scala @@ -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 diff --git a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala index 610071f827..85d3558a5e 100644 --- a/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala +++ b/akka-persistence-query/src/main/scala/akka/persistence/query/journal/leveldb/EventsByTagStage.scala @@ -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 diff --git a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/TestActor.scala b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/TestActor.scala index 06c3f6ff57..1befbc27fe 100644 --- a/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/TestActor.scala +++ b/akka-persistence-query/src/test/scala/akka/persistence/query/journal/leveldb/TestActor.scala @@ -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" } } diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala b/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala index 24f5d91119..26b4fbc700 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala @@ -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 diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala index 854b7faa70..b34fcfa708 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/journal/JournalSpec.scala @@ -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)) + + } + } } } diff --git a/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala b/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala index 702bedb37f..520c1fa6d0 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/snapshot/SnapshotStoreSpec.scala @@ -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)) + } + } + } } } diff --git a/akka-persistence-testkit/src/main/resources/reference.conf b/akka-persistence-testkit/src/main/resources/reference.conf index f4d10a05f7..223191e06b 100644 --- a/akka-persistence-testkit/src/main/resources/reference.conf +++ b/akka-persistence-testkit/src/main/resources/reference.conf @@ -28,3 +28,7 @@ akka.persistence.testkit { } } + +akka.persistence.testkit.query { + class = "akka.persistence.testkit.query.PersistenceTestKitReadJournalProvider" +} diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/EventStorage.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/EventStorage.scala index 070e39901f..d7b6ac0727 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/EventStorage.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/EventStorage.scala @@ -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] - } /** diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/PersistenceTestKitPlugin.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/PersistenceTestKitPlugin.scala index 9730e8a918..f0f67418c4 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/PersistenceTestKitPlugin.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/PersistenceTestKitPlugin.scala @@ -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) + } /** diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/ProcessingPolicy.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/ProcessingPolicy.scala index 119f1b9163..0871bf56bb 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/ProcessingPolicy.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/ProcessingPolicy.scala @@ -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 { diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/SnapshotStorage.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/SnapshotStorage.scala index 1c66d012b8..1ddeeb5df5 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/SnapshotStorage.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/SnapshotStorage.scala @@ -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._ diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/InMemStorageExtension.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/InMemStorageExtension.scala index 83652f3b56..8454e74143 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/InMemStorageExtension.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/InMemStorageExtension.scala @@ -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 + } + }) + +} diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SerializedEventStorageImpl.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SerializedEventStorageImpl.scala index 9d25479dba..8b4970bd5e 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SerializedEventStorageImpl.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SerializedEventStorageImpl.scala @@ -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)) + } } diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SnapshotStorageEmulatorExtension.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SnapshotStorageEmulatorExtension.scala index 56e5b86153..7de3f8e9d3 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SnapshotStorageEmulatorExtension.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SnapshotStorageEmulatorExtension.scala @@ -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 diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/TestKitStorage.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/TestKitStorage.scala index 0b3fd01981..a4ea67c145 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/TestKitStorage.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/TestKitStorage.scala @@ -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] diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/PersistenceTestKitReadJournalProvider.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/PersistenceTestKitReadJournalProvider.scala new file mode 100644 index 0000000000..cd0644590d --- /dev/null +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/PersistenceTestKitReadJournalProvider.scala @@ -0,0 +1,18 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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()) +} diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/internal/EventsByPersistenceIdStage.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/internal/EventsByPersistenceIdStage.scala new file mode 100644 index 0000000000..5d82e6bb1e --- /dev/null +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/internal/EventsByPersistenceIdStage.scala @@ -0,0 +1,78 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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) + } + + } + +} diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/javadsl/PersistenceTestKitReadJournal.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/javadsl/PersistenceTestKitReadJournal.scala new file mode 100644 index 0000000000..2b96285343 --- /dev/null +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/javadsl/PersistenceTestKitReadJournal.scala @@ -0,0 +1,43 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 + +} diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/scaladsl/PersistenceTestKitReadJournal.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/scaladsl/PersistenceTestKitReadJournal.scala new file mode 100644 index 0000000000..491032f04f --- /dev/null +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/scaladsl/PersistenceTestKitReadJournal.scala @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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) + } + } +} diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceTestKit.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceTestKit.scala index aae6f4de07..49b90c8b1c 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceTestKit.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/scaladsl/PersistenceTestKit.scala @@ -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 diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/query/CurrentEventsByTagSpec.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/query/CurrentEventsByTagSpec.scala new file mode 100644 index 0000000000..0f38947075 --- /dev/null +++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/query/CurrentEventsByTagSpec.scala @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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")) + } + } + +} diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/query/EventsByPersistenceIdSpec.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/query/EventsByPersistenceIdSpec.scala new file mode 100644 index 0000000000..a642a79788 --- /dev/null +++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/query/EventsByPersistenceIdSpec.scala @@ -0,0 +1,137 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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() + } + } +} diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKitSpec.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKitSpec.scala index 39869b475d..38ce74b8ea 100644 --- a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKitSpec.scala +++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/EventSourcedBehaviorTestKitSpec.scala @@ -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]) diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/MultipleJournalsSpec.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/MultipleJournalsSpec.scala new file mode 100644 index 0000000000..093ebb35a8 --- /dev/null +++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/MultipleJournalsSpec.scala @@ -0,0 +1,115 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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")) + } + + } + +} diff --git a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/PersistenceTestKitJournalCompatSpec.scala b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/PersistenceTestKitJournalCompatSpec.scala index c2ea678c62..30e2996fab 100644 --- a/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/PersistenceTestKitJournalCompatSpec.scala +++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/PersistenceTestKitJournalCompatSpec.scala @@ -37,6 +37,7 @@ class PersistenceTestKitJournalCompatSpec extends JournalSpec(config = Persisten } override protected def supportsRejectingNonSerializableObjects: CapabilityFlag = true + override protected def supportsMetadata: CapabilityFlag = true } class PersistenceTestKitSnapshotStoreCompatSpec diff --git a/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ReplicatedEventSourcingTest.java b/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ReplicatedEventSourcingTest.java new file mode 100644 index 0000000000..8badd779ad --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ReplicatedEventSourcingTest.java @@ -0,0 +1,206 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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> { + interface Command {} + + static final class GetState implements Command { + final ActorRef replyTo; + + public GetState(ActorRef replyTo) { + this.replyTo = replyTo; + } + } + + static final class StoreMe implements Command { + final String text; + final ActorRef replyTo; + + public StoreMe(String text, ActorRef replyTo) { + this.text = text; + this.replyTo = replyTo; + } + } + + static final class StoreUs implements Command { + final List texts; + final ActorRef replyTo; + + public StoreUs(List texts, ActorRef replyTo) { + this.texts = texts; + this.replyTo = replyTo; + } + } + + static final class GetReplica implements Command { + final ActorRef replyTo; + + public GetReplica(ActorRef replyTo) { + this.replyTo = replyTo; + } + } + + static final class State { + final Set texts; + + public State(Set texts) { + this.texts = texts; + } + } + + enum Stop implements Command { + INSTANCE + } + + public static Behavior create( + String entityId, ReplicaId replicaId, Set 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 emptyState() { + return Collections.emptySet(); + } + + @Override + public CommandHandler> 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, 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 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 allReplicas = new HashSet<>(Arrays.asList(dcA, dcB, dcC)); + + ActorRef replicaA = + testKit.spawn(TestBehavior.create("id1", dcA, allReplicas)); + ActorRef replicaB = + testKit.spawn(TestBehavior.create("id1", dcB, allReplicas)); + ActorRef replicaC = + testKit.spawn(TestBehavior.create("id1", dcC, allReplicas)); + + TestProbe 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(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(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(Arrays.asList("stored-to-a", "stored-to-b", "stored-to-c"))); + return null; + }); + } +} diff --git a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java new file mode 100644 index 0000000000..25166dccd7 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java @@ -0,0 +1,75 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 ALL_REPLICAS = + Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DCA, DCB))); + // #replicas + + // #factory-shared + public static Behavior 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 create(String entityId, ReplicaId replicaId) { + Map 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 commandHandler() { + throw new UnsupportedOperationException("dummy for example"); + } + + @Override + public EventHandler eventHandler() { + throw new UnsupportedOperationException("dummy for example"); + } +} diff --git a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java new file mode 100644 index 0000000000..2401d1bfb9 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java @@ -0,0 +1,445 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 replicaA = + testKit.spawn(AuctionEntity.create(R1, auctionName, initialBid, closeAt, true)); + ActorRef 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 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 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 ALL_REPLICAS = new HashSet<>(Arrays.asList(R1, R2)); + + private final ActorContext context; + private final TimerScheduler 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 replyTo; + + public GetHighestBid(ActorRef replyTo) { + this.replyTo = replyTo; + } + } + + public static final class IsClosed implements Command { + public final ActorRef replyTo; + + public IsClosed(ActorRef 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 finishedAtDc; + + AuctionState( + boolean stillRunning, Bid highestBid, int highestCounterOffer, Set 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 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 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 context, + ReplicationContext replicationContext, + TimerScheduler 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 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 commandHandler() { + + CommandHandlerBuilder 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 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 diff --git a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedBlogExample.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedBlogExample.java new file mode 100644 index 0000000000..fb550b04fe --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedBlogExample.java @@ -0,0 +1,291 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 context; + + interface Command { + String getPostId(); + } + + static final class AddPost implements Command { + final String postId; + final PostContent content; + final ActorRef replyTo; + + public AddPost(String postId, PostContent content, ActorRef 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 replyTo; + + public GetPost(String postId, ActorRef 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 replyTo; + + public ChangeBody(String postId, PostContent newContent, ActorRef 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 replyTo; + + public Publish(String postId, ActorRef 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 content; + final LwwTime contentTimestamp; + final boolean published; + + public BlogState(Optional 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 create( + String entityId, ReplicaId replicaId, Set allReplicas) { + return Behaviors.setup( + context -> + ReplicatedEventSourcing.commonJournalConfig( + new ReplicationId("blog", entityId, replicaId), + allReplicas, + PersistenceTestKitReadJournal.Identifier(), + replicationContext -> new BlogEntity(context, replicationContext))); + } + + private BlogEntity(ActorContext context, ReplicationContext replicationContext) { + super(replicationContext); + this.context = context; + } + + @Override + public BlogState emptyState() { + return BlogState.EMPTY; + } + + // #command-handler + @Override + public CommandHandler 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 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 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 onPublish(BlogState state, Publish command) { + Published evt = new Published(getReplicationContext().entityId()); + return Effect().persist(evt).thenRun(() -> command.replyTo.tell(Done.getInstance())); + } + + private Effect 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 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 + + } +} diff --git a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedMovieExample.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedMovieExample.java new file mode 100644 index 0000000000..da80971b95 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedMovieExample.java @@ -0,0 +1,104 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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> { + + 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 replyTo; + + public GetMovieList(ActorRef replyTo) { + this.replyTo = replyTo; + } + } + + public static class MovieList { + public final Set movieIds; + + public MovieList(Set movieIds) { + this.movieIds = Collections.unmodifiableSet(movieIds); + } + } + + public static Behavior create( + String entityId, ReplicaId replicaId, Set allReplicas) { + return ReplicatedEventSourcing.commonJournalConfig( + new ReplicationId("movies", entityId, replicaId), + allReplicas, + PersistenceTestKitReadJournal.Identifier(), + MovieWatchList::new); + } + + private MovieWatchList(ReplicationContext replicationContext) { + super(replicationContext); + } + + @Override + public ORSet emptyState() { + return ORSet.empty(getReplicationContext().replicaId()); + } + + @Override + public CommandHandler> 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.DeltaOp> eventHandler() { + return newEventHandlerBuilder().forAnyState().onAnyEvent(ORSet::applyOperation); + } + } + // #movie-entity +} diff --git a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedShoppingCartExample.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedShoppingCartExample.java new file mode 100644 index 0000000000..069f0982aa --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedShoppingCartExample.java @@ -0,0 +1,158 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 replyTo; + + public GetCartItems(ActorRef replyTo) { + this.replyTo = replyTo; + } + } + + public static final class CartItems { + public final Map items; + + public CartItems(Map items) { + this.items = items; + } + } + + public static final class State { + public final Map items = new HashMap<>(); + } + + public static Behavior create( + String entityId, ReplicaId replicaId, Set 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 commandHandler() { + return newCommandHandlerBuilder() + .forAnyState() + .onCommand(AddItem.class, this::onAddItem) + .onCommand(RemoveItem.class, this::onRemoveItem) + .onCommand(GetCartItems.class, this::onGetCartItems) + .build(); + } + + private Effect onAddItem(State state, AddItem command) { + return Effect() + .persist(new ItemUpdated(command.productId, new Counter.Updated(command.count))); + } + + private Effect onRemoveItem(State state, RemoveItem command) { + return Effect() + .persist(new ItemUpdated(command.productId, new Counter.Updated(-command.count))); + } + + private Effect onGetCartItems(State state, GetCartItems command) { + command.replyTo.tell(new CartItems(filterEmptyAndNegative(state.items))); + return Effect().none(); + } + + private Map filterEmptyAndNegative(Map cart) { + Map result = new HashMap<>(); + for (Map.Entry entry : cart.entrySet()) { + int count = entry.getValue().value().intValue(); + if (count > 0) result.put(entry.getKey(), count); + } + return Collections.unmodifiableMap(result); + } + + @Override + public EventHandler 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 +} diff --git a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedStringSet.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedStringSet.java new file mode 100644 index 0000000000..011108922f --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedStringSet.java @@ -0,0 +1,86 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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> { + interface Command {} + + public static final class AddString implements Command { + final String string; + + public AddString(String string) { + this.string = string; + } + } + + public static Behavior create( + String entityId, ReplicaId replicaId, Set allReplicas) { + return ReplicatedEventSourcing.commonJournalConfig( + new ReplicationId("StringSet", entityId, replicaId), + allReplicas, + PersistenceTestKitReadJournal.Identifier(), + ReplicatedStringSet::new); + } + + private ReplicatedStringSet(ReplicationContext replicationContext) { + super(replicationContext); + } + + @Override + public Set emptyState() { + return new HashSet<>(); + } + + @Override + public CommandHandler> 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, String> eventHandler() { + return newEventHandlerBuilder() + .forAnyState() + .onAnyEvent( + (set, string) -> { + HashSet newState = new HashSet<>(set); + newState.add(string); + return newState; + }); + } + + // #tagging + @Override + public Set 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 tags = new HashSet<>(); + tags.add("strings"); + if (event.length() > 10) tags.add("long-strings"); + return tags; + } + } + // #tagging +} diff --git a/akka-persistence-typed-tests/src/test/resources/logback-test.xml b/akka-persistence-typed-tests/src/test/resources/logback-test.xml new file mode 100644 index 0000000000..df3cd05c69 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/resources/logback-test.xml @@ -0,0 +1,32 @@ + + + + + + + + %date{ISO8601} [%-5level] [%logger] [%marker] [%X{akkaSource}] [%X{persistencePhase}] [%X{persistenceId}] - %msg %n + + + + + + + + + + + + + + + + diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/EventPublishingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/EventPublishingSpec.scala new file mode 100644 index 0000000000..62cab82ff4 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/EventPublishingSpec.scala @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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")) + } + + } + +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalReplicationSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalReplicationSpec.scala new file mode 100644 index 0000000000..cc7867f5dd --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalReplicationSpec.scala @@ -0,0 +1,117 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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")) + + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventPublishingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventPublishingSpec.scala new file mode 100644 index 0000000000..eb8e701238 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventPublishingSpec.scala @@ -0,0 +1,240 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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")) + } + + } + +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSpec.scala new file mode 100644 index 0000000000..919bda434b --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSpec.scala @@ -0,0 +1,429 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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" + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingTaggingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingTaggingSpec.scala new file mode 100644 index 0000000000..f5a7e61601 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingTaggingSpec.scala @@ -0,0 +1,117 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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) + + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationBaseSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationBaseSpec.scala new file mode 100644 index 0000000000..ec8a9487a7 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationBaseSpec.scala @@ -0,0 +1,30 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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()}" + +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationIllegalAccessSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationIllegalAccessSpec.scala new file mode 100644 index 0000000000..cf457d5e4d --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationIllegalAccessSpec.scala @@ -0,0 +1,102 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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") + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationSnapshotSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationSnapshotSpec.scala new file mode 100644 index 0000000000..620c8ffbb4 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationSnapshotSpec.scala @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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"))) + } + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/CounterSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/CounterSpec.scala new file mode 100644 index 0000000000..54f4e7ab7d --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/CounterSpec.scala @@ -0,0 +1,132 @@ +/* + * Copyright (C) 2017-2020 Lightbend Inc. + */ + +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() + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/LwwSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/LwwSpec.scala new file mode 100644 index 0000000000..f7a1cfbc73 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/LwwSpec.scala @@ -0,0 +1,110 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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))) + } + } + } + +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/ORSetSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/ORSetSpec.scala new file mode 100644 index 0000000000..6f9a384ad8 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/ORSetSpec.scala @@ -0,0 +1,102 @@ +/* + * Copyright (C) 2017-2020 Lightbend Inc. + */ + +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) + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/jackson/ReplicatedEventSourcingJacksonSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/jackson/ReplicatedEventSourcingJacksonSpec.scala new file mode 100644 index 0000000000..df992e26c3 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/jackson/ReplicatedEventSourcingJacksonSpec.scala @@ -0,0 +1,48 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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) + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala new file mode 100644 index 0000000000..71d868a6a1 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala @@ -0,0 +1,337 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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) + } + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala new file mode 100644 index 0000000000..cba33aee8e --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala @@ -0,0 +1,170 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 + } + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala new file mode 100644 index 0000000000..8ab4ebec6e --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 + +} diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedMovieWatchListExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedMovieWatchListExampleSpec.scala new file mode 100644 index 0000000000..53a726bb3a --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedMovieWatchListExampleSpec.scala @@ -0,0 +1,104 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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"))) + } + } + + } + +} diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedShoppingCartExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedShoppingCartExampleSpec.scala new file mode 100644 index 0000000000..53a9fd2106 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedShoppingCartExampleSpec.scala @@ -0,0 +1,121 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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))) + } + } + } +} diff --git a/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ReplicatedEventSourcing.java b/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ReplicatedEventSourcing.java new file mode 100644 index 0000000000..c1fd5de3d4 --- /dev/null +++ b/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ReplicatedEventSourcing.java @@ -0,0 +1,12487 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: ReplicatedEventSourcing.proto + +package akka.persistence.typed.serialization; + +public final class ReplicatedEventSourcing { + private ReplicatedEventSourcing() {} + + public static void registerAllExtensions( + akka.protobufv3.internal.ExtensionRegistryLite registry) {} + + public static void registerAllExtensions(akka.protobufv3.internal.ExtensionRegistry registry) { + registerAllExtensions((akka.protobufv3.internal.ExtensionRegistryLite) registry); + } + /** Protobuf enum {@code ORSetDeltaOp} */ + public enum ORSetDeltaOp implements akka.protobufv3.internal.ProtocolMessageEnum { + /** Add = 0; */ + Add(0), + /** Remove = 1; */ + Remove(1), + /** Full = 2; */ + Full(2), + ; + + /** Add = 0; */ + public static final int Add_VALUE = 0; + /** Remove = 1; */ + public static final int Remove_VALUE = 1; + /** Full = 2; */ + public static final int Full_VALUE = 2; + + public final int getNumber() { + return value; + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + * @deprecated Use {@link #forNumber(int)} instead. + */ + @java.lang.Deprecated + public static ORSetDeltaOp valueOf(int value) { + return forNumber(value); + } + + /** + * @param value The numeric wire value of the corresponding enum entry. + * @return The enum associated with the given numeric wire value. + */ + public static ORSetDeltaOp forNumber(int value) { + switch (value) { + case 0: + return Add; + case 1: + return Remove; + case 2: + return Full; + default: + return null; + } + } + + public static akka.protobufv3.internal.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + + private static final akka.protobufv3.internal.Internal.EnumLiteMap + internalValueMap = + new akka.protobufv3.internal.Internal.EnumLiteMap() { + public ORSetDeltaOp findValueByNumber(int number) { + return ORSetDeltaOp.forNumber(number); + } + }; + + public final akka.protobufv3.internal.Descriptors.EnumValueDescriptor getValueDescriptor() { + return getDescriptor().getValues().get(ordinal()); + } + + public final akka.protobufv3.internal.Descriptors.EnumDescriptor getDescriptorForType() { + return getDescriptor(); + } + + public static final akka.protobufv3.internal.Descriptors.EnumDescriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.getDescriptor() + .getEnumTypes() + .get(0); + } + + private static final ORSetDeltaOp[] VALUES = values(); + + public static ORSetDeltaOp valueOf( + akka.protobufv3.internal.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException("EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int value; + + private ORSetDeltaOp(int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:ORSetDeltaOp) + } + + public interface CounterOrBuilder + extends + // @@protoc_insertion_point(interface_extends:Counter) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * required bytes value = 1; + * + * @return Whether the value field is set. + */ + boolean hasValue(); + /** + * required bytes value = 1; + * + * @return The value. + */ + akka.protobufv3.internal.ByteString getValue(); + } + /** Protobuf type {@code Counter} */ + public static final class Counter extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:Counter) + CounterOrBuilder { + private static final long serialVersionUID = 0L; + // Use Counter.newBuilder() to construct. + private Counter(akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Counter() { + value_ = akka.protobufv3.internal.ByteString.EMPTY; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new Counter(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Counter( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + bitField0_ |= 0x00000001; + value_ = input.readBytes(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_Counter_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_Counter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.Builder.class); + } + + private int bitField0_; + public static final int VALUE_FIELD_NUMBER = 1; + private akka.protobufv3.internal.ByteString value_; + /** + * required bytes value = 1; + * + * @return Whether the value field is set. + */ + public boolean hasValue() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required bytes value = 1; + * + * @return The value. + */ + public akka.protobufv3.internal.ByteString getValue() { + return value_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasValue()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeBytes(1, value_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeBytesSize(1, value_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter) obj; + + if (hasValue() != other.hasValue()) return false; + if (hasValue()) { + if (!getValue().equals(other.getValue())) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasValue()) { + hash = (37 * hash) + VALUE_FIELD_NUMBER; + hash = (53 * hash) + getValue().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + java.nio.ByteBuffer data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code Counter} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:Counter) + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_Counter_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_Counter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + value_ = akka.protobufv3.internal.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_Counter_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + to_bitField0_ |= 0x00000001; + } + result.value_ = value_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + .getDefaultInstance()) return this; + if (other.hasValue()) { + setValue(other.getValue()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (!hasValue()) { + return false; + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private akka.protobufv3.internal.ByteString value_ = + akka.protobufv3.internal.ByteString.EMPTY; + /** + * required bytes value = 1; + * + * @return Whether the value field is set. + */ + public boolean hasValue() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required bytes value = 1; + * + * @return The value. + */ + public akka.protobufv3.internal.ByteString getValue() { + return value_; + } + /** + * required bytes value = 1; + * + * @param value The value to set. + * @return This builder for chaining. + */ + public Builder setValue(akka.protobufv3.internal.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + value_ = value; + onChanged(); + return this; + } + /** + * required bytes value = 1; + * + * @return This builder for chaining. + */ + public Builder clearValue() { + bitField0_ = (bitField0_ & ~0x00000001); + value_ = getDefaultInstance().getValue(); + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:Counter) + } + + // @@protoc_insertion_point(class_scope:Counter) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public Counter parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new Counter(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface CounterUpdateOrBuilder + extends + // @@protoc_insertion_point(interface_extends:CounterUpdate) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * required bytes delta = 1; + * + * @return Whether the delta field is set. + */ + boolean hasDelta(); + /** + * required bytes delta = 1; + * + * @return The delta. + */ + akka.protobufv3.internal.ByteString getDelta(); + } + /** Protobuf type {@code CounterUpdate} */ + public static final class CounterUpdate extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:CounterUpdate) + CounterUpdateOrBuilder { + private static final long serialVersionUID = 0L; + // Use CounterUpdate.newBuilder() to construct. + private CounterUpdate(akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private CounterUpdate() { + delta_ = akka.protobufv3.internal.ByteString.EMPTY; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new CounterUpdate(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private CounterUpdate( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + bitField0_ |= 0x00000001; + delta_ = input.readBytes(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_CounterUpdate_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_CounterUpdate_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.Builder + .class); + } + + private int bitField0_; + public static final int DELTA_FIELD_NUMBER = 1; + private akka.protobufv3.internal.ByteString delta_; + /** + * required bytes delta = 1; + * + * @return Whether the delta field is set. + */ + public boolean hasDelta() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required bytes delta = 1; + * + * @return The delta. + */ + public akka.protobufv3.internal.ByteString getDelta() { + return delta_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasDelta()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeBytes(1, delta_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeBytesSize(1, delta_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate) obj; + + if (hasDelta() != other.hasDelta()) return false; + if (hasDelta()) { + if (!getDelta().equals(other.getDelta())) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasDelta()) { + hash = (37 * hash) + DELTA_FIELD_NUMBER; + hash = (53 * hash) + getDelta().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code CounterUpdate} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:CounterUpdate) + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdateOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_CounterUpdate_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_CounterUpdate_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.Builder + .class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + delta_ = akka.protobufv3.internal.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_CounterUpdate_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + to_bitField0_ |= 0x00000001; + } + result.delta_ = delta_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + .getDefaultInstance()) return this; + if (other.hasDelta()) { + setDelta(other.getDelta()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (!hasDelta()) { + return false; + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private akka.protobufv3.internal.ByteString delta_ = + akka.protobufv3.internal.ByteString.EMPTY; + /** + * required bytes delta = 1; + * + * @return Whether the delta field is set. + */ + public boolean hasDelta() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required bytes delta = 1; + * + * @return The delta. + */ + public akka.protobufv3.internal.ByteString getDelta() { + return delta_; + } + /** + * required bytes delta = 1; + * + * @param value The delta to set. + * @return This builder for chaining. + */ + public Builder setDelta(akka.protobufv3.internal.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + delta_ = value; + onChanged(); + return this; + } + /** + * required bytes delta = 1; + * + * @return This builder for chaining. + */ + public Builder clearDelta() { + bitField0_ = (bitField0_ & ~0x00000001); + delta_ = getDefaultInstance().getDelta(); + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:CounterUpdate) + } + + // @@protoc_insertion_point(class_scope:CounterUpdate) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public CounterUpdate parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new CounterUpdate(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface ORSetOrBuilder + extends + // @@protoc_insertion_point(interface_extends:ORSet) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * required string originDc = 1; + * + * @return Whether the originDc field is set. + */ + boolean hasOriginDc(); + /** + * required string originDc = 1; + * + * @return The originDc. + */ + java.lang.String getOriginDc(); + /** + * required string originDc = 1; + * + * @return The bytes for originDc. + */ + akka.protobufv3.internal.ByteString getOriginDcBytes(); + + /** + * required .VersionVector vvector = 2; + * + * @return Whether the vvector field is set. + */ + boolean hasVvector(); + /** + * required .VersionVector vvector = 2; + * + * @return The vvector. + */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVvector(); + /** required .VersionVector vvector = 2; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVvectorOrBuilder(); + + /** repeated .VersionVector dots = 3; */ + java.util.List + getDotsList(); + /** repeated .VersionVector dots = 3; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getDots(int index); + /** repeated .VersionVector dots = 3; */ + int getDotsCount(); + /** repeated .VersionVector dots = 3; */ + java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + getDotsOrBuilderList(); + /** repeated .VersionVector dots = 3; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getDotsOrBuilder(int index); + + /** + * repeated string stringElements = 4; + * + * @return A list containing the stringElements. + */ + java.util.List getStringElementsList(); + /** + * repeated string stringElements = 4; + * + * @return The count of stringElements. + */ + int getStringElementsCount(); + /** + * repeated string stringElements = 4; + * + * @param index The index of the element to return. + * @return The stringElements at the given index. + */ + java.lang.String getStringElements(int index); + /** + * repeated string stringElements = 4; + * + * @param index The index of the value to return. + * @return The bytes of the stringElements at the given index. + */ + akka.protobufv3.internal.ByteString getStringElementsBytes(int index); + + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @return A list containing the intElements. + */ + java.util.List getIntElementsList(); + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @return The count of intElements. + */ + int getIntElementsCount(); + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @param index The index of the element to return. + * @return The intElements at the given index. + */ + int getIntElements(int index); + + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @return A list containing the longElements. + */ + java.util.List getLongElementsList(); + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @return The count of longElements. + */ + int getLongElementsCount(); + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @param index The index of the element to return. + * @return The longElements at the given index. + */ + long getLongElements(int index); + + /** repeated .Payload otherElements = 7; */ + java.util.List getOtherElementsList(); + /** repeated .Payload otherElements = 7; */ + akka.remote.ContainerFormats.Payload getOtherElements(int index); + /** repeated .Payload otherElements = 7; */ + int getOtherElementsCount(); + /** repeated .Payload otherElements = 7; */ + java.util.List + getOtherElementsOrBuilderList(); + /** repeated .Payload otherElements = 7; */ + akka.remote.ContainerFormats.PayloadOrBuilder getOtherElementsOrBuilder(int index); + } + /** Protobuf type {@code ORSet} */ + public static final class ORSet extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:ORSet) + ORSetOrBuilder { + private static final long serialVersionUID = 0L; + // Use ORSet.newBuilder() to construct. + private ORSet(akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ORSet() { + originDc_ = ""; + dots_ = java.util.Collections.emptyList(); + stringElements_ = akka.protobufv3.internal.LazyStringArrayList.EMPTY; + intElements_ = emptyIntList(); + longElements_ = emptyLongList(); + otherElements_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new ORSet(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ORSet( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + akka.protobufv3.internal.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + originDc_ = bs; + break; + } + case 18: + { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + subBuilder = null; + if (((bitField0_ & 0x00000002) != 0)) { + subBuilder = vvector_.toBuilder(); + } + vvector_ = + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(vvector_); + vvector_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + case 26: + { + if (!((mutable_bitField0_ & 0x00000004) != 0)) { + dots_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVector>(); + mutable_bitField0_ |= 0x00000004; + } + dots_.add( + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .PARSER, + extensionRegistry)); + break; + } + case 34: + { + akka.protobufv3.internal.ByteString bs = input.readBytes(); + if (!((mutable_bitField0_ & 0x00000008) != 0)) { + stringElements_ = new akka.protobufv3.internal.LazyStringArrayList(); + mutable_bitField0_ |= 0x00000008; + } + stringElements_.add(bs); + break; + } + case 40: + { + if (!((mutable_bitField0_ & 0x00000010) != 0)) { + intElements_ = newIntList(); + mutable_bitField0_ |= 0x00000010; + } + intElements_.addInt(input.readSInt32()); + break; + } + case 42: + { + int length = input.readRawVarint32(); + int limit = input.pushLimit(length); + if (!((mutable_bitField0_ & 0x00000010) != 0) && input.getBytesUntilLimit() > 0) { + intElements_ = newIntList(); + mutable_bitField0_ |= 0x00000010; + } + while (input.getBytesUntilLimit() > 0) { + intElements_.addInt(input.readSInt32()); + } + input.popLimit(limit); + break; + } + case 48: + { + if (!((mutable_bitField0_ & 0x00000020) != 0)) { + longElements_ = newLongList(); + mutable_bitField0_ |= 0x00000020; + } + longElements_.addLong(input.readSInt64()); + break; + } + case 50: + { + int length = input.readRawVarint32(); + int limit = input.pushLimit(length); + if (!((mutable_bitField0_ & 0x00000020) != 0) && input.getBytesUntilLimit() > 0) { + longElements_ = newLongList(); + mutable_bitField0_ |= 0x00000020; + } + while (input.getBytesUntilLimit() > 0) { + longElements_.addLong(input.readSInt64()); + } + input.popLimit(limit); + break; + } + case 58: + { + if (!((mutable_bitField0_ & 0x00000040) != 0)) { + otherElements_ = new java.util.ArrayList(); + mutable_bitField0_ |= 0x00000040; + } + otherElements_.add( + input.readMessage( + akka.remote.ContainerFormats.Payload.PARSER, extensionRegistry)); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000004) != 0)) { + dots_ = java.util.Collections.unmodifiableList(dots_); + } + if (((mutable_bitField0_ & 0x00000008) != 0)) { + stringElements_ = stringElements_.getUnmodifiableView(); + } + if (((mutable_bitField0_ & 0x00000010) != 0)) { + intElements_.makeImmutable(); // C + } + if (((mutable_bitField0_ & 0x00000020) != 0)) { + longElements_.makeImmutable(); // C + } + if (((mutable_bitField0_ & 0x00000040) != 0)) { + otherElements_ = java.util.Collections.unmodifiableList(otherElements_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSet_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSet_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder.class); + } + + private int bitField0_; + public static final int ORIGINDC_FIELD_NUMBER = 1; + private volatile java.lang.Object originDc_; + /** + * required string originDc = 1; + * + * @return Whether the originDc field is set. + */ + public boolean hasOriginDc() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required string originDc = 1; + * + * @return The originDc. + */ + public java.lang.String getOriginDc() { + java.lang.Object ref = originDc_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + originDc_ = s; + } + return s; + } + } + /** + * required string originDc = 1; + * + * @return The bytes for originDc. + */ + public akka.protobufv3.internal.ByteString getOriginDcBytes() { + java.lang.Object ref = originDc_; + if (ref instanceof java.lang.String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + originDc_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + + public static final int VVECTOR_FIELD_NUMBER = 2; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector vvector_; + /** + * required .VersionVector vvector = 2; + * + * @return Whether the vvector field is set. + */ + public boolean hasVvector() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required .VersionVector vvector = 2; + * + * @return The vvector. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVvector() { + return vvector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : vvector_; + } + /** required .VersionVector vvector = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVvectorOrBuilder() { + return vvector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : vvector_; + } + + public static final int DOTS_FIELD_NUMBER = 3; + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector> + dots_; + /** repeated .VersionVector dots = 3; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector> + getDotsList() { + return dots_; + } + /** repeated .VersionVector dots = 3; */ + public java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + getDotsOrBuilderList() { + return dots_; + } + /** repeated .VersionVector dots = 3; */ + public int getDotsCount() { + return dots_.size(); + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getDots( + int index) { + return dots_.get(index); + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getDotsOrBuilder(int index) { + return dots_.get(index); + } + + public static final int STRINGELEMENTS_FIELD_NUMBER = 4; + private akka.protobufv3.internal.LazyStringList stringElements_; + /** + * repeated string stringElements = 4; + * + * @return A list containing the stringElements. + */ + public akka.protobufv3.internal.ProtocolStringList getStringElementsList() { + return stringElements_; + } + /** + * repeated string stringElements = 4; + * + * @return The count of stringElements. + */ + public int getStringElementsCount() { + return stringElements_.size(); + } + /** + * repeated string stringElements = 4; + * + * @param index The index of the element to return. + * @return The stringElements at the given index. + */ + public java.lang.String getStringElements(int index) { + return stringElements_.get(index); + } + /** + * repeated string stringElements = 4; + * + * @param index The index of the value to return. + * @return The bytes of the stringElements at the given index. + */ + public akka.protobufv3.internal.ByteString getStringElementsBytes(int index) { + return stringElements_.getByteString(index); + } + + public static final int INTELEMENTS_FIELD_NUMBER = 5; + private akka.protobufv3.internal.Internal.IntList intElements_; + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @return A list containing the intElements. + */ + public java.util.List getIntElementsList() { + return intElements_; + } + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @return The count of intElements. + */ + public int getIntElementsCount() { + return intElements_.size(); + } + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @param index The index of the element to return. + * @return The intElements at the given index. + */ + public int getIntElements(int index) { + return intElements_.getInt(index); + } + + private int intElementsMemoizedSerializedSize = -1; + + public static final int LONGELEMENTS_FIELD_NUMBER = 6; + private akka.protobufv3.internal.Internal.LongList longElements_; + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @return A list containing the longElements. + */ + public java.util.List getLongElementsList() { + return longElements_; + } + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @return The count of longElements. + */ + public int getLongElementsCount() { + return longElements_.size(); + } + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @param index The index of the element to return. + * @return The longElements at the given index. + */ + public long getLongElements(int index) { + return longElements_.getLong(index); + } + + private int longElementsMemoizedSerializedSize = -1; + + public static final int OTHERELEMENTS_FIELD_NUMBER = 7; + private java.util.List otherElements_; + /** repeated .Payload otherElements = 7; */ + public java.util.List getOtherElementsList() { + return otherElements_; + } + /** repeated .Payload otherElements = 7; */ + public java.util.List + getOtherElementsOrBuilderList() { + return otherElements_; + } + /** repeated .Payload otherElements = 7; */ + public int getOtherElementsCount() { + return otherElements_.size(); + } + /** repeated .Payload otherElements = 7; */ + public akka.remote.ContainerFormats.Payload getOtherElements(int index) { + return otherElements_.get(index); + } + /** repeated .Payload otherElements = 7; */ + public akka.remote.ContainerFormats.PayloadOrBuilder getOtherElementsOrBuilder(int index) { + return otherElements_.get(index); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasOriginDc()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasVvector()) { + memoizedIsInitialized = 0; + return false; + } + if (!getVvector().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getDotsCount(); i++) { + if (!getDots(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + for (int i = 0; i < getOtherElementsCount(); i++) { + if (!getOtherElements(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) != 0)) { + akka.protobufv3.internal.GeneratedMessageV3.writeString(output, 1, originDc_); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getVvector()); + } + for (int i = 0; i < dots_.size(); i++) { + output.writeMessage(3, dots_.get(i)); + } + for (int i = 0; i < stringElements_.size(); i++) { + akka.protobufv3.internal.GeneratedMessageV3.writeString( + output, 4, stringElements_.getRaw(i)); + } + if (getIntElementsList().size() > 0) { + output.writeUInt32NoTag(42); + output.writeUInt32NoTag(intElementsMemoizedSerializedSize); + } + for (int i = 0; i < intElements_.size(); i++) { + output.writeSInt32NoTag(intElements_.getInt(i)); + } + if (getLongElementsList().size() > 0) { + output.writeUInt32NoTag(50); + output.writeUInt32NoTag(longElementsMemoizedSerializedSize); + } + for (int i = 0; i < longElements_.size(); i++) { + output.writeSInt64NoTag(longElements_.getLong(i)); + } + for (int i = 0; i < otherElements_.size(); i++) { + output.writeMessage(7, otherElements_.get(i)); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.GeneratedMessageV3.computeStringSize(1, originDc_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeMessageSize(2, getVvector()); + } + for (int i = 0; i < dots_.size(); i++) { + size += akka.protobufv3.internal.CodedOutputStream.computeMessageSize(3, dots_.get(i)); + } + { + int dataSize = 0; + for (int i = 0; i < stringElements_.size(); i++) { + dataSize += computeStringSizeNoTag(stringElements_.getRaw(i)); + } + size += dataSize; + size += 1 * getStringElementsList().size(); + } + { + int dataSize = 0; + for (int i = 0; i < intElements_.size(); i++) { + dataSize += + akka.protobufv3.internal.CodedOutputStream.computeSInt32SizeNoTag( + intElements_.getInt(i)); + } + size += dataSize; + if (!getIntElementsList().isEmpty()) { + size += 1; + size += akka.protobufv3.internal.CodedOutputStream.computeInt32SizeNoTag(dataSize); + } + intElementsMemoizedSerializedSize = dataSize; + } + { + int dataSize = 0; + for (int i = 0; i < longElements_.size(); i++) { + dataSize += + akka.protobufv3.internal.CodedOutputStream.computeSInt64SizeNoTag( + longElements_.getLong(i)); + } + size += dataSize; + if (!getLongElementsList().isEmpty()) { + size += 1; + size += akka.protobufv3.internal.CodedOutputStream.computeInt32SizeNoTag(dataSize); + } + longElementsMemoizedSerializedSize = dataSize; + } + for (int i = 0; i < otherElements_.size(); i++) { + size += + akka.protobufv3.internal.CodedOutputStream.computeMessageSize(7, otherElements_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet) obj; + + if (hasOriginDc() != other.hasOriginDc()) return false; + if (hasOriginDc()) { + if (!getOriginDc().equals(other.getOriginDc())) return false; + } + if (hasVvector() != other.hasVvector()) return false; + if (hasVvector()) { + if (!getVvector().equals(other.getVvector())) return false; + } + if (!getDotsList().equals(other.getDotsList())) return false; + if (!getStringElementsList().equals(other.getStringElementsList())) return false; + if (!getIntElementsList().equals(other.getIntElementsList())) return false; + if (!getLongElementsList().equals(other.getLongElementsList())) return false; + if (!getOtherElementsList().equals(other.getOtherElementsList())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasOriginDc()) { + hash = (37 * hash) + ORIGINDC_FIELD_NUMBER; + hash = (53 * hash) + getOriginDc().hashCode(); + } + if (hasVvector()) { + hash = (37 * hash) + VVECTOR_FIELD_NUMBER; + hash = (53 * hash) + getVvector().hashCode(); + } + if (getDotsCount() > 0) { + hash = (37 * hash) + DOTS_FIELD_NUMBER; + hash = (53 * hash) + getDotsList().hashCode(); + } + if (getStringElementsCount() > 0) { + hash = (37 * hash) + STRINGELEMENTS_FIELD_NUMBER; + hash = (53 * hash) + getStringElementsList().hashCode(); + } + if (getIntElementsCount() > 0) { + hash = (37 * hash) + INTELEMENTS_FIELD_NUMBER; + hash = (53 * hash) + getIntElementsList().hashCode(); + } + if (getLongElementsCount() > 0) { + hash = (37 * hash) + LONGELEMENTS_FIELD_NUMBER; + hash = (53 * hash) + getLongElementsList().hashCode(); + } + if (getOtherElementsCount() > 0) { + hash = (37 * hash) + OTHERELEMENTS_FIELD_NUMBER; + hash = (53 * hash) + getOtherElementsList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + java.nio.ByteBuffer data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code ORSet} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:ORSet) + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSet_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSet_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) { + getVvectorFieldBuilder(); + getDotsFieldBuilder(); + getOtherElementsFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + originDc_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + if (vvectorBuilder_ == null) { + vvector_ = null; + } else { + vvectorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + if (dotsBuilder_ == null) { + dots_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + } else { + dotsBuilder_.clear(); + } + stringElements_ = akka.protobufv3.internal.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + intElements_ = emptyIntList(); + bitField0_ = (bitField0_ & ~0x00000010); + longElements_ = emptyLongList(); + bitField0_ = (bitField0_ & ~0x00000020); + if (otherElementsBuilder_ == null) { + otherElements_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000040); + } else { + otherElementsBuilder_.clear(); + } + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSet_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + to_bitField0_ |= 0x00000001; + } + result.originDc_ = originDc_; + if (((from_bitField0_ & 0x00000002) != 0)) { + if (vvectorBuilder_ == null) { + result.vvector_ = vvector_; + } else { + result.vvector_ = vvectorBuilder_.build(); + } + to_bitField0_ |= 0x00000002; + } + if (dotsBuilder_ == null) { + if (((bitField0_ & 0x00000004) != 0)) { + dots_ = java.util.Collections.unmodifiableList(dots_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.dots_ = dots_; + } else { + result.dots_ = dotsBuilder_.build(); + } + if (((bitField0_ & 0x00000008) != 0)) { + stringElements_ = stringElements_.getUnmodifiableView(); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.stringElements_ = stringElements_; + if (((bitField0_ & 0x00000010) != 0)) { + intElements_.makeImmutable(); + bitField0_ = (bitField0_ & ~0x00000010); + } + result.intElements_ = intElements_; + if (((bitField0_ & 0x00000020) != 0)) { + longElements_.makeImmutable(); + bitField0_ = (bitField0_ & ~0x00000020); + } + result.longElements_ = longElements_; + if (otherElementsBuilder_ == null) { + if (((bitField0_ & 0x00000040) != 0)) { + otherElements_ = java.util.Collections.unmodifiableList(otherElements_); + bitField0_ = (bitField0_ & ~0x00000040); + } + result.otherElements_ = otherElements_; + } else { + result.otherElements_ = otherElementsBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance()) return this; + if (other.hasOriginDc()) { + bitField0_ |= 0x00000001; + originDc_ = other.originDc_; + onChanged(); + } + if (other.hasVvector()) { + mergeVvector(other.getVvector()); + } + if (dotsBuilder_ == null) { + if (!other.dots_.isEmpty()) { + if (dots_.isEmpty()) { + dots_ = other.dots_; + bitField0_ = (bitField0_ & ~0x00000004); + } else { + ensureDotsIsMutable(); + dots_.addAll(other.dots_); + } + onChanged(); + } + } else { + if (!other.dots_.isEmpty()) { + if (dotsBuilder_.isEmpty()) { + dotsBuilder_.dispose(); + dotsBuilder_ = null; + dots_ = other.dots_; + bitField0_ = (bitField0_ & ~0x00000004); + dotsBuilder_ = + akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders + ? getDotsFieldBuilder() + : null; + } else { + dotsBuilder_.addAllMessages(other.dots_); + } + } + } + if (!other.stringElements_.isEmpty()) { + if (stringElements_.isEmpty()) { + stringElements_ = other.stringElements_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensureStringElementsIsMutable(); + stringElements_.addAll(other.stringElements_); + } + onChanged(); + } + if (!other.intElements_.isEmpty()) { + if (intElements_.isEmpty()) { + intElements_ = other.intElements_; + bitField0_ = (bitField0_ & ~0x00000010); + } else { + ensureIntElementsIsMutable(); + intElements_.addAll(other.intElements_); + } + onChanged(); + } + if (!other.longElements_.isEmpty()) { + if (longElements_.isEmpty()) { + longElements_ = other.longElements_; + bitField0_ = (bitField0_ & ~0x00000020); + } else { + ensureLongElementsIsMutable(); + longElements_.addAll(other.longElements_); + } + onChanged(); + } + if (otherElementsBuilder_ == null) { + if (!other.otherElements_.isEmpty()) { + if (otherElements_.isEmpty()) { + otherElements_ = other.otherElements_; + bitField0_ = (bitField0_ & ~0x00000040); + } else { + ensureOtherElementsIsMutable(); + otherElements_.addAll(other.otherElements_); + } + onChanged(); + } + } else { + if (!other.otherElements_.isEmpty()) { + if (otherElementsBuilder_.isEmpty()) { + otherElementsBuilder_.dispose(); + otherElementsBuilder_ = null; + otherElements_ = other.otherElements_; + bitField0_ = (bitField0_ & ~0x00000040); + otherElementsBuilder_ = + akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders + ? getOtherElementsFieldBuilder() + : null; + } else { + otherElementsBuilder_.addAllMessages(other.otherElements_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (!hasOriginDc()) { + return false; + } + if (!hasVvector()) { + return false; + } + if (!getVvector().isInitialized()) { + return false; + } + for (int i = 0; i < getDotsCount(); i++) { + if (!getDots(i).isInitialized()) { + return false; + } + } + for (int i = 0; i < getOtherElementsCount(); i++) { + if (!getOtherElements(i).isInitialized()) { + return false; + } + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.lang.Object originDc_ = ""; + /** + * required string originDc = 1; + * + * @return Whether the originDc field is set. + */ + public boolean hasOriginDc() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required string originDc = 1; + * + * @return The originDc. + */ + public java.lang.String getOriginDc() { + java.lang.Object ref = originDc_; + if (!(ref instanceof java.lang.String)) { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + originDc_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string originDc = 1; + * + * @return The bytes for originDc. + */ + public akka.protobufv3.internal.ByteString getOriginDcBytes() { + java.lang.Object ref = originDc_; + if (ref instanceof String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + originDc_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + /** + * required string originDc = 1; + * + * @param value The originDc to set. + * @return This builder for chaining. + */ + public Builder setOriginDc(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + originDc_ = value; + onChanged(); + return this; + } + /** + * required string originDc = 1; + * + * @return This builder for chaining. + */ + public Builder clearOriginDc() { + bitField0_ = (bitField0_ & ~0x00000001); + originDc_ = getDefaultInstance().getOriginDc(); + onChanged(); + return this; + } + /** + * required string originDc = 1; + * + * @param value The bytes for originDc to set. + * @return This builder for chaining. + */ + public Builder setOriginDcBytes(akka.protobufv3.internal.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + originDc_ = value; + onChanged(); + return this; + } + + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector vvector_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + vvectorBuilder_; + /** + * required .VersionVector vvector = 2; + * + * @return Whether the vvector field is set. + */ + public boolean hasVvector() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required .VersionVector vvector = 2; + * + * @return The vvector. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVvector() { + if (vvectorBuilder_ == null) { + return vvector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : vvector_; + } else { + return vvectorBuilder_.getMessage(); + } + } + /** required .VersionVector vvector = 2; */ + public Builder setVvector( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (vvectorBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + vvector_ = value; + onChanged(); + } else { + vvectorBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** required .VersionVector vvector = 2; */ + public Builder setVvector( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { + if (vvectorBuilder_ == null) { + vvector_ = builderForValue.build(); + onChanged(); + } else { + vvectorBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** required .VersionVector vvector = 2; */ + public Builder mergeVvector( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (vvectorBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) + && vvector_ != null + && vvector_ + != akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance()) { + vvector_ = + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .newBuilder(vvector_) + .mergeFrom(value) + .buildPartial(); + } else { + vvector_ = value; + } + onChanged(); + } else { + vvectorBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** required .VersionVector vvector = 2; */ + public Builder clearVvector() { + if (vvectorBuilder_ == null) { + vvector_ = null; + onChanged(); + } else { + vvectorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** required .VersionVector vvector = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + getVvectorBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getVvectorFieldBuilder().getBuilder(); + } + /** required .VersionVector vvector = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVvectorOrBuilder() { + if (vvectorBuilder_ != null) { + return vvectorBuilder_.getMessageOrBuilder(); + } else { + return vvector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : vvector_; + } + } + /** required .VersionVector vvector = 2; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + getVvectorFieldBuilder() { + if (vvectorBuilder_ == null) { + vvectorBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVectorOrBuilder>(getVvector(), getParentForChildren(), isClean()); + vvector_ = null; + } + return vvectorBuilder_; + } + + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector> + dots_ = java.util.Collections.emptyList(); + + private void ensureDotsIsMutable() { + if (!((bitField0_ & 0x00000004) != 0)) { + dots_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector>( + dots_); + bitField0_ |= 0x00000004; + } + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + dotsBuilder_; + + /** repeated .VersionVector dots = 3; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector> + getDotsList() { + if (dotsBuilder_ == null) { + return java.util.Collections.unmodifiableList(dots_); + } else { + return dotsBuilder_.getMessageList(); + } + } + /** repeated .VersionVector dots = 3; */ + public int getDotsCount() { + if (dotsBuilder_ == null) { + return dots_.size(); + } else { + return dotsBuilder_.getCount(); + } + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getDots( + int index) { + if (dotsBuilder_ == null) { + return dots_.get(index); + } else { + return dotsBuilder_.getMessage(index); + } + } + /** repeated .VersionVector dots = 3; */ + public Builder setDots( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (dotsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureDotsIsMutable(); + dots_.set(index, value); + onChanged(); + } else { + dotsBuilder_.setMessage(index, value); + } + return this; + } + /** repeated .VersionVector dots = 3; */ + public Builder setDots( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { + if (dotsBuilder_ == null) { + ensureDotsIsMutable(); + dots_.set(index, builderForValue.build()); + onChanged(); + } else { + dotsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .VersionVector dots = 3; */ + public Builder addDots( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (dotsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureDotsIsMutable(); + dots_.add(value); + onChanged(); + } else { + dotsBuilder_.addMessage(value); + } + return this; + } + /** repeated .VersionVector dots = 3; */ + public Builder addDots( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (dotsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureDotsIsMutable(); + dots_.add(index, value); + onChanged(); + } else { + dotsBuilder_.addMessage(index, value); + } + return this; + } + /** repeated .VersionVector dots = 3; */ + public Builder addDots( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { + if (dotsBuilder_ == null) { + ensureDotsIsMutable(); + dots_.add(builderForValue.build()); + onChanged(); + } else { + dotsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** repeated .VersionVector dots = 3; */ + public Builder addDots( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { + if (dotsBuilder_ == null) { + ensureDotsIsMutable(); + dots_.add(index, builderForValue.build()); + onChanged(); + } else { + dotsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .VersionVector dots = 3; */ + public Builder addAllDots( + java.lang.Iterable< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector> + values) { + if (dotsBuilder_ == null) { + ensureDotsIsMutable(); + akka.protobufv3.internal.AbstractMessageLite.Builder.addAll(values, dots_); + onChanged(); + } else { + dotsBuilder_.addAllMessages(values); + } + return this; + } + /** repeated .VersionVector dots = 3; */ + public Builder clearDots() { + if (dotsBuilder_ == null) { + dots_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + onChanged(); + } else { + dotsBuilder_.clear(); + } + return this; + } + /** repeated .VersionVector dots = 3; */ + public Builder removeDots(int index) { + if (dotsBuilder_ == null) { + ensureDotsIsMutable(); + dots_.remove(index); + onChanged(); + } else { + dotsBuilder_.remove(index); + } + return this; + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + getDotsBuilder(int index) { + return getDotsFieldBuilder().getBuilder(index); + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getDotsOrBuilder(int index) { + if (dotsBuilder_ == null) { + return dots_.get(index); + } else { + return dotsBuilder_.getMessageOrBuilder(index); + } + } + /** repeated .VersionVector dots = 3; */ + public java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVectorOrBuilder> + getDotsOrBuilderList() { + if (dotsBuilder_ != null) { + return dotsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(dots_); + } + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + addDotsBuilder() { + return getDotsFieldBuilder() + .addBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance()); + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + addDotsBuilder(int index) { + return getDotsFieldBuilder() + .addBuilder( + index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance()); + } + /** repeated .VersionVector dots = 3; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder> + getDotsBuilderList() { + return getDotsFieldBuilder().getBuilderList(); + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + getDotsFieldBuilder() { + if (dotsBuilder_ == null) { + dotsBuilder_ = + new akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVectorOrBuilder>( + dots_, ((bitField0_ & 0x00000004) != 0), getParentForChildren(), isClean()); + dots_ = null; + } + return dotsBuilder_; + } + + private akka.protobufv3.internal.LazyStringList stringElements_ = + akka.protobufv3.internal.LazyStringArrayList.EMPTY; + + private void ensureStringElementsIsMutable() { + if (!((bitField0_ & 0x00000008) != 0)) { + stringElements_ = new akka.protobufv3.internal.LazyStringArrayList(stringElements_); + bitField0_ |= 0x00000008; + } + } + /** + * repeated string stringElements = 4; + * + * @return A list containing the stringElements. + */ + public akka.protobufv3.internal.ProtocolStringList getStringElementsList() { + return stringElements_.getUnmodifiableView(); + } + /** + * repeated string stringElements = 4; + * + * @return The count of stringElements. + */ + public int getStringElementsCount() { + return stringElements_.size(); + } + /** + * repeated string stringElements = 4; + * + * @param index The index of the element to return. + * @return The stringElements at the given index. + */ + public java.lang.String getStringElements(int index) { + return stringElements_.get(index); + } + /** + * repeated string stringElements = 4; + * + * @param index The index of the value to return. + * @return The bytes of the stringElements at the given index. + */ + public akka.protobufv3.internal.ByteString getStringElementsBytes(int index) { + return stringElements_.getByteString(index); + } + /** + * repeated string stringElements = 4; + * + * @param index The index to set the value at. + * @param value The stringElements to set. + * @return This builder for chaining. + */ + public Builder setStringElements(int index, java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureStringElementsIsMutable(); + stringElements_.set(index, value); + onChanged(); + return this; + } + /** + * repeated string stringElements = 4; + * + * @param value The stringElements to add. + * @return This builder for chaining. + */ + public Builder addStringElements(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureStringElementsIsMutable(); + stringElements_.add(value); + onChanged(); + return this; + } + /** + * repeated string stringElements = 4; + * + * @param values The stringElements to add. + * @return This builder for chaining. + */ + public Builder addAllStringElements(java.lang.Iterable values) { + ensureStringElementsIsMutable(); + akka.protobufv3.internal.AbstractMessageLite.Builder.addAll(values, stringElements_); + onChanged(); + return this; + } + /** + * repeated string stringElements = 4; + * + * @return This builder for chaining. + */ + public Builder clearStringElements() { + stringElements_ = akka.protobufv3.internal.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + onChanged(); + return this; + } + /** + * repeated string stringElements = 4; + * + * @param value The bytes of the stringElements to add. + * @return This builder for chaining. + */ + public Builder addStringElementsBytes(akka.protobufv3.internal.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + ensureStringElementsIsMutable(); + stringElements_.add(value); + onChanged(); + return this; + } + + private akka.protobufv3.internal.Internal.IntList intElements_ = emptyIntList(); + + private void ensureIntElementsIsMutable() { + if (!((bitField0_ & 0x00000010) != 0)) { + intElements_ = mutableCopy(intElements_); + bitField0_ |= 0x00000010; + } + } + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @return A list containing the intElements. + */ + public java.util.List getIntElementsList() { + return ((bitField0_ & 0x00000010) != 0) + ? java.util.Collections.unmodifiableList(intElements_) + : intElements_; + } + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @return The count of intElements. + */ + public int getIntElementsCount() { + return intElements_.size(); + } + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @param index The index of the element to return. + * @return The intElements at the given index. + */ + public int getIntElements(int index) { + return intElements_.getInt(index); + } + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @param index The index to set the value at. + * @param value The intElements to set. + * @return This builder for chaining. + */ + public Builder setIntElements(int index, int value) { + ensureIntElementsIsMutable(); + intElements_.setInt(index, value); + onChanged(); + return this; + } + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @param value The intElements to add. + * @return This builder for chaining. + */ + public Builder addIntElements(int value) { + ensureIntElementsIsMutable(); + intElements_.addInt(value); + onChanged(); + return this; + } + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @param values The intElements to add. + * @return This builder for chaining. + */ + public Builder addAllIntElements(java.lang.Iterable values) { + ensureIntElementsIsMutable(); + akka.protobufv3.internal.AbstractMessageLite.Builder.addAll(values, intElements_); + onChanged(); + return this; + } + /** + * repeated sint32 intElements = 5 [packed = true]; + * + * @return This builder for chaining. + */ + public Builder clearIntElements() { + intElements_ = emptyIntList(); + bitField0_ = (bitField0_ & ~0x00000010); + onChanged(); + return this; + } + + private akka.protobufv3.internal.Internal.LongList longElements_ = emptyLongList(); + + private void ensureLongElementsIsMutable() { + if (!((bitField0_ & 0x00000020) != 0)) { + longElements_ = mutableCopy(longElements_); + bitField0_ |= 0x00000020; + } + } + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @return A list containing the longElements. + */ + public java.util.List getLongElementsList() { + return ((bitField0_ & 0x00000020) != 0) + ? java.util.Collections.unmodifiableList(longElements_) + : longElements_; + } + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @return The count of longElements. + */ + public int getLongElementsCount() { + return longElements_.size(); + } + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @param index The index of the element to return. + * @return The longElements at the given index. + */ + public long getLongElements(int index) { + return longElements_.getLong(index); + } + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @param index The index to set the value at. + * @param value The longElements to set. + * @return This builder for chaining. + */ + public Builder setLongElements(int index, long value) { + ensureLongElementsIsMutable(); + longElements_.setLong(index, value); + onChanged(); + return this; + } + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @param value The longElements to add. + * @return This builder for chaining. + */ + public Builder addLongElements(long value) { + ensureLongElementsIsMutable(); + longElements_.addLong(value); + onChanged(); + return this; + } + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @param values The longElements to add. + * @return This builder for chaining. + */ + public Builder addAllLongElements(java.lang.Iterable values) { + ensureLongElementsIsMutable(); + akka.protobufv3.internal.AbstractMessageLite.Builder.addAll(values, longElements_); + onChanged(); + return this; + } + /** + * repeated sint64 longElements = 6 [packed = true]; + * + * @return This builder for chaining. + */ + public Builder clearLongElements() { + longElements_ = emptyLongList(); + bitField0_ = (bitField0_ & ~0x00000020); + onChanged(); + return this; + } + + private java.util.List otherElements_ = + java.util.Collections.emptyList(); + + private void ensureOtherElementsIsMutable() { + if (!((bitField0_ & 0x00000040) != 0)) { + otherElements_ = + new java.util.ArrayList(otherElements_); + bitField0_ |= 0x00000040; + } + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.remote.ContainerFormats.Payload, + akka.remote.ContainerFormats.Payload.Builder, + akka.remote.ContainerFormats.PayloadOrBuilder> + otherElementsBuilder_; + + /** repeated .Payload otherElements = 7; */ + public java.util.List getOtherElementsList() { + if (otherElementsBuilder_ == null) { + return java.util.Collections.unmodifiableList(otherElements_); + } else { + return otherElementsBuilder_.getMessageList(); + } + } + /** repeated .Payload otherElements = 7; */ + public int getOtherElementsCount() { + if (otherElementsBuilder_ == null) { + return otherElements_.size(); + } else { + return otherElementsBuilder_.getCount(); + } + } + /** repeated .Payload otherElements = 7; */ + public akka.remote.ContainerFormats.Payload getOtherElements(int index) { + if (otherElementsBuilder_ == null) { + return otherElements_.get(index); + } else { + return otherElementsBuilder_.getMessage(index); + } + } + /** repeated .Payload otherElements = 7; */ + public Builder setOtherElements(int index, akka.remote.ContainerFormats.Payload value) { + if (otherElementsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureOtherElementsIsMutable(); + otherElements_.set(index, value); + onChanged(); + } else { + otherElementsBuilder_.setMessage(index, value); + } + return this; + } + /** repeated .Payload otherElements = 7; */ + public Builder setOtherElements( + int index, akka.remote.ContainerFormats.Payload.Builder builderForValue) { + if (otherElementsBuilder_ == null) { + ensureOtherElementsIsMutable(); + otherElements_.set(index, builderForValue.build()); + onChanged(); + } else { + otherElementsBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .Payload otherElements = 7; */ + public Builder addOtherElements(akka.remote.ContainerFormats.Payload value) { + if (otherElementsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureOtherElementsIsMutable(); + otherElements_.add(value); + onChanged(); + } else { + otherElementsBuilder_.addMessage(value); + } + return this; + } + /** repeated .Payload otherElements = 7; */ + public Builder addOtherElements(int index, akka.remote.ContainerFormats.Payload value) { + if (otherElementsBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureOtherElementsIsMutable(); + otherElements_.add(index, value); + onChanged(); + } else { + otherElementsBuilder_.addMessage(index, value); + } + return this; + } + /** repeated .Payload otherElements = 7; */ + public Builder addOtherElements( + akka.remote.ContainerFormats.Payload.Builder builderForValue) { + if (otherElementsBuilder_ == null) { + ensureOtherElementsIsMutable(); + otherElements_.add(builderForValue.build()); + onChanged(); + } else { + otherElementsBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** repeated .Payload otherElements = 7; */ + public Builder addOtherElements( + int index, akka.remote.ContainerFormats.Payload.Builder builderForValue) { + if (otherElementsBuilder_ == null) { + ensureOtherElementsIsMutable(); + otherElements_.add(index, builderForValue.build()); + onChanged(); + } else { + otherElementsBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .Payload otherElements = 7; */ + public Builder addAllOtherElements( + java.lang.Iterable values) { + if (otherElementsBuilder_ == null) { + ensureOtherElementsIsMutable(); + akka.protobufv3.internal.AbstractMessageLite.Builder.addAll(values, otherElements_); + onChanged(); + } else { + otherElementsBuilder_.addAllMessages(values); + } + return this; + } + /** repeated .Payload otherElements = 7; */ + public Builder clearOtherElements() { + if (otherElementsBuilder_ == null) { + otherElements_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000040); + onChanged(); + } else { + otherElementsBuilder_.clear(); + } + return this; + } + /** repeated .Payload otherElements = 7; */ + public Builder removeOtherElements(int index) { + if (otherElementsBuilder_ == null) { + ensureOtherElementsIsMutable(); + otherElements_.remove(index); + onChanged(); + } else { + otherElementsBuilder_.remove(index); + } + return this; + } + /** repeated .Payload otherElements = 7; */ + public akka.remote.ContainerFormats.Payload.Builder getOtherElementsBuilder(int index) { + return getOtherElementsFieldBuilder().getBuilder(index); + } + /** repeated .Payload otherElements = 7; */ + public akka.remote.ContainerFormats.PayloadOrBuilder getOtherElementsOrBuilder(int index) { + if (otherElementsBuilder_ == null) { + return otherElements_.get(index); + } else { + return otherElementsBuilder_.getMessageOrBuilder(index); + } + } + /** repeated .Payload otherElements = 7; */ + public java.util.List + getOtherElementsOrBuilderList() { + if (otherElementsBuilder_ != null) { + return otherElementsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(otherElements_); + } + } + /** repeated .Payload otherElements = 7; */ + public akka.remote.ContainerFormats.Payload.Builder addOtherElementsBuilder() { + return getOtherElementsFieldBuilder() + .addBuilder(akka.remote.ContainerFormats.Payload.getDefaultInstance()); + } + /** repeated .Payload otherElements = 7; */ + public akka.remote.ContainerFormats.Payload.Builder addOtherElementsBuilder(int index) { + return getOtherElementsFieldBuilder() + .addBuilder(index, akka.remote.ContainerFormats.Payload.getDefaultInstance()); + } + /** repeated .Payload otherElements = 7; */ + public java.util.List + getOtherElementsBuilderList() { + return getOtherElementsFieldBuilder().getBuilderList(); + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.remote.ContainerFormats.Payload, + akka.remote.ContainerFormats.Payload.Builder, + akka.remote.ContainerFormats.PayloadOrBuilder> + getOtherElementsFieldBuilder() { + if (otherElementsBuilder_ == null) { + otherElementsBuilder_ = + new akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.remote.ContainerFormats.Payload, + akka.remote.ContainerFormats.Payload.Builder, + akka.remote.ContainerFormats.PayloadOrBuilder>( + otherElements_, + ((bitField0_ & 0x00000040) != 0), + getParentForChildren(), + isClean()); + otherElements_ = null; + } + return otherElementsBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:ORSet) + } + + // @@protoc_insertion_point(class_scope:ORSet) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public ORSet parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new ORSet(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface ORSetDeltaGroupOrBuilder + extends + // @@protoc_insertion_point(interface_extends:ORSetDeltaGroup) + akka.protobufv3.internal.MessageOrBuilder { + + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry> + getEntriesList(); + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry getEntries( + int index); + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + int getEntriesCount(); + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder> + getEntriesOrBuilderList(); + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.EntryOrBuilder + getEntriesOrBuilder(int index); + } + /** Protobuf type {@code ORSetDeltaGroup} */ + public static final class ORSetDeltaGroup extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:ORSetDeltaGroup) + ORSetDeltaGroupOrBuilder { + private static final long serialVersionUID = 0L; + // Use ORSetDeltaGroup.newBuilder() to construct. + private ORSetDeltaGroup(akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ORSetDeltaGroup() { + entries_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new ORSetDeltaGroup(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ORSetDeltaGroup( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + if (!((mutable_bitField0_ & 0x00000001) != 0)) { + entries_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ORSetDeltaGroup.Entry>(); + mutable_bitField0_ |= 0x00000001; + } + entries_.add( + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry.PARSER, + extensionRegistry)); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) != 0)) { + entries_ = java.util.Collections.unmodifiableList(entries_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Builder + .class); + } + + public interface EntryOrBuilder + extends + // @@protoc_insertion_point(interface_extends:ORSetDeltaGroup.Entry) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * required .ORSetDeltaOp operation = 1; + * + * @return Whether the operation field is set. + */ + boolean hasOperation(); + /** + * required .ORSetDeltaOp operation = 1; + * + * @return The operation. + */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp getOperation(); + + /** + * required .ORSet underlying = 2; + * + * @return Whether the underlying field is set. + */ + boolean hasUnderlying(); + /** + * required .ORSet underlying = 2; + * + * @return The underlying. + */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet getUnderlying(); + /** required .ORSet underlying = 2; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder + getUnderlyingOrBuilder(); + } + /** Protobuf type {@code ORSetDeltaGroup.Entry} */ + public static final class Entry extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:ORSetDeltaGroup.Entry) + EntryOrBuilder { + private static final long serialVersionUID = 0L; + // Use Entry.newBuilder() to construct. + private Entry(akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Entry() { + operation_ = 0; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new Entry(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Entry( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 8: + { + int rawValue = input.readEnum(); + @SuppressWarnings("deprecation") + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp value = + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp + .valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(1, rawValue); + } else { + bitField0_ |= 0x00000001; + operation_ = rawValue; + } + break; + } + case 18: + { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder + subBuilder = null; + if (((bitField0_ & 0x00000002) != 0)) { + subBuilder = underlying_.toBuilder(); + } + underlying_ = + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(underlying_); + underlying_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_Entry_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder.class); + } + + private int bitField0_; + public static final int OPERATION_FIELD_NUMBER = 1; + private int operation_; + /** + * required .ORSetDeltaOp operation = 1; + * + * @return Whether the operation field is set. + */ + public boolean hasOperation() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required .ORSetDeltaOp operation = 1; + * + * @return The operation. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp + getOperation() { + @SuppressWarnings("deprecation") + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp result = + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp.valueOf( + operation_); + return result == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp.Add + : result; + } + + public static final int UNDERLYING_FIELD_NUMBER = 2; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet underlying_; + /** + * required .ORSet underlying = 2; + * + * @return Whether the underlying field is set. + */ + public boolean hasUnderlying() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required .ORSet underlying = 2; + * + * @return The underlying. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet getUnderlying() { + return underlying_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance() + : underlying_; + } + /** required .ORSet underlying = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder + getUnderlyingOrBuilder() { + return underlying_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance() + : underlying_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasOperation()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasUnderlying()) { + memoizedIsInitialized = 0; + return false; + } + if (!getUnderlying().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeEnum(1, operation_); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getUnderlying()); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeEnumSize(1, operation_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeMessageSize(2, getUnderlying()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry) + obj; + + if (hasOperation() != other.hasOperation()) return false; + if (hasOperation()) { + if (operation_ != other.operation_) return false; + } + if (hasUnderlying() != other.hasUnderlying()) return false; + if (hasUnderlying()) { + if (!getUnderlying().equals(other.getUnderlying())) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasOperation()) { + hash = (37 * hash) + OPERATION_FIELD_NUMBER; + hash = (53 * hash) + operation_; + } + if (hasUnderlying()) { + hash = (37 * hash) + UNDERLYING_FIELD_NUMBER; + hash = (53 * hash) + getUnderlying().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code ORSetDeltaGroup.Entry} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:ORSetDeltaGroup.Entry) + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_Entry_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) { + getUnderlyingFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + operation_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + if (underlyingBuilder_ == null) { + underlying_ = null; + } else { + underlyingBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_Entry_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + to_bitField0_ |= 0x00000001; + } + result.operation_ = operation_; + if (((from_bitField0_ & 0x00000002) != 0)) { + if (underlyingBuilder_ == null) { + result.underlying_ = underlying_; + } else { + result.underlying_ = underlyingBuilder_.build(); + } + to_bitField0_ |= 0x00000002; + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .getDefaultInstance()) return this; + if (other.hasOperation()) { + setOperation(other.getOperation()); + } + if (other.hasUnderlying()) { + mergeUnderlying(other.getUnderlying()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (!hasOperation()) { + return false; + } + if (!hasUnderlying()) { + return false; + } + if (!getUnderlying().isInitialized()) { + return false; + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private int operation_ = 0; + /** + * required .ORSetDeltaOp operation = 1; + * + * @return Whether the operation field is set. + */ + public boolean hasOperation() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required .ORSetDeltaOp operation = 1; + * + * @return The operation. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp + getOperation() { + @SuppressWarnings("deprecation") + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp result = + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp.valueOf( + operation_); + return result == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp.Add + : result; + } + /** + * required .ORSetDeltaOp operation = 1; + * + * @param value The operation to set. + * @return This builder for chaining. + */ + public Builder setOperation( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + operation_ = value.getNumber(); + onChanged(); + return this; + } + /** + * required .ORSetDeltaOp operation = 1; + * + * @return This builder for chaining. + */ + public Builder clearOperation() { + bitField0_ = (bitField0_ & ~0x00000001); + operation_ = 0; + onChanged(); + return this; + } + + private akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet underlying_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder> + underlyingBuilder_; + /** + * required .ORSet underlying = 2; + * + * @return Whether the underlying field is set. + */ + public boolean hasUnderlying() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required .ORSet underlying = 2; + * + * @return The underlying. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet getUnderlying() { + if (underlyingBuilder_ == null) { + return underlying_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance() + : underlying_; + } else { + return underlyingBuilder_.getMessage(); + } + } + /** required .ORSet underlying = 2; */ + public Builder setUnderlying( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet value) { + if (underlyingBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + underlying_ = value; + onChanged(); + } else { + underlyingBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** required .ORSet underlying = 2; */ + public Builder setUnderlying( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder + builderForValue) { + if (underlyingBuilder_ == null) { + underlying_ = builderForValue.build(); + onChanged(); + } else { + underlyingBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** required .ORSet underlying = 2; */ + public Builder mergeUnderlying( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet value) { + if (underlyingBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) + && underlying_ != null + && underlying_ + != akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance()) { + underlying_ = + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.newBuilder( + underlying_) + .mergeFrom(value) + .buildPartial(); + } else { + underlying_ = value; + } + onChanged(); + } else { + underlyingBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** required .ORSet underlying = 2; */ + public Builder clearUnderlying() { + if (underlyingBuilder_ == null) { + underlying_ = null; + onChanged(); + } else { + underlyingBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** required .ORSet underlying = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder + getUnderlyingBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getUnderlyingFieldBuilder().getBuilder(); + } + /** required .ORSet underlying = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder + getUnderlyingOrBuilder() { + if (underlyingBuilder_ != null) { + return underlyingBuilder_.getMessageOrBuilder(); + } else { + return underlying_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance() + : underlying_; + } + } + /** required .ORSet underlying = 2; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder> + getUnderlyingFieldBuilder() { + if (underlyingBuilder_ == null) { + underlyingBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder>( + getUnderlying(), getParentForChildren(), isClean()); + underlying_ = null; + } + return underlyingBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:ORSetDeltaGroup.Entry) + } + + // @@protoc_insertion_point(class_scope:ORSetDeltaGroup.Entry) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .ORSetDeltaGroup.Entry + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public Entry parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new Entry(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public static final int ENTRIES_FIELD_NUMBER = 1; + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry> + entries_; + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry> + getEntriesList() { + return entries_; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder> + getEntriesOrBuilderList() { + return entries_; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public int getEntriesCount() { + return entries_.size(); + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + getEntries(int index) { + return entries_.get(index); + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder + getEntriesOrBuilder(int index) { + return entries_.get(index); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + for (int i = 0; i < getEntriesCount(); i++) { + if (!getEntries(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + for (int i = 0; i < entries_.size(); i++) { + output.writeMessage(1, entries_.get(i)); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < entries_.size(); i++) { + size += akka.protobufv3.internal.CodedOutputStream.computeMessageSize(1, entries_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup) obj; + + if (!getEntriesList().equals(other.getEntriesList())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (getEntriesCount() > 0) { + hash = (37 * hash) + ENTRIES_FIELD_NUMBER; + hash = (53 * hash) + getEntriesList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code ORSetDeltaGroup} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:ORSetDeltaGroup) + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroupOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Builder + .class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) { + getEntriesFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + if (entriesBuilder_ == null) { + entries_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + entriesBuilder_.clear(); + } + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSetDeltaGroup_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup(this); + int from_bitField0_ = bitField0_; + if (entriesBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0)) { + entries_ = java.util.Collections.unmodifiableList(entries_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.entries_ = entries_; + } else { + result.entries_ = entriesBuilder_.build(); + } + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .getDefaultInstance()) return this; + if (entriesBuilder_ == null) { + if (!other.entries_.isEmpty()) { + if (entries_.isEmpty()) { + entries_ = other.entries_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureEntriesIsMutable(); + entries_.addAll(other.entries_); + } + onChanged(); + } + } else { + if (!other.entries_.isEmpty()) { + if (entriesBuilder_.isEmpty()) { + entriesBuilder_.dispose(); + entriesBuilder_ = null; + entries_ = other.entries_; + bitField0_ = (bitField0_ & ~0x00000001); + entriesBuilder_ = + akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders + ? getEntriesFieldBuilder() + : null; + } else { + entriesBuilder_.addAllMessages(other.entries_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + for (int i = 0; i < getEntriesCount(); i++) { + if (!getEntries(i).isInitialized()) { + return false; + } + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry> + entries_ = java.util.Collections.emptyList(); + + private void ensureEntriesIsMutable() { + if (!((bitField0_ & 0x00000001) != 0)) { + entries_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry>(entries_); + bitField0_ |= 0x00000001; + } + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder> + entriesBuilder_; + + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry> + getEntriesList() { + if (entriesBuilder_ == null) { + return java.util.Collections.unmodifiableList(entries_); + } else { + return entriesBuilder_.getMessageList(); + } + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public int getEntriesCount() { + if (entriesBuilder_ == null) { + return entries_.size(); + } else { + return entriesBuilder_.getCount(); + } + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + getEntries(int index) { + if (entriesBuilder_ == null) { + return entries_.get(index); + } else { + return entriesBuilder_.getMessage(index); + } + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public Builder setEntries( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + value) { + if (entriesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureEntriesIsMutable(); + entries_.set(index, value); + onChanged(); + } else { + entriesBuilder_.setMessage(index, value); + } + return this; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public Builder setEntries( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry.Builder + builderForValue) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + entries_.set(index, builderForValue.build()); + onChanged(); + } else { + entriesBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public Builder addEntries( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + value) { + if (entriesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureEntriesIsMutable(); + entries_.add(value); + onChanged(); + } else { + entriesBuilder_.addMessage(value); + } + return this; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public Builder addEntries( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + value) { + if (entriesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureEntriesIsMutable(); + entries_.add(index, value); + onChanged(); + } else { + entriesBuilder_.addMessage(index, value); + } + return this; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public Builder addEntries( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry.Builder + builderForValue) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + entries_.add(builderForValue.build()); + onChanged(); + } else { + entriesBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public Builder addEntries( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry.Builder + builderForValue) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + entries_.add(index, builderForValue.build()); + onChanged(); + } else { + entriesBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public Builder addAllEntries( + java.lang.Iterable< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry> + values) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + akka.protobufv3.internal.AbstractMessageLite.Builder.addAll(values, entries_); + onChanged(); + } else { + entriesBuilder_.addAllMessages(values); + } + return this; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public Builder clearEntries() { + if (entriesBuilder_ == null) { + entries_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + entriesBuilder_.clear(); + } + return this; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public Builder removeEntries(int index) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + entries_.remove(index); + onChanged(); + } else { + entriesBuilder_.remove(index); + } + return this; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder + getEntriesBuilder(int index) { + return getEntriesFieldBuilder().getBuilder(index); + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder + getEntriesOrBuilder(int index) { + if (entriesBuilder_ == null) { + return entries_.get(index); + } else { + return entriesBuilder_.getMessageOrBuilder(index); + } + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder> + getEntriesOrBuilderList() { + if (entriesBuilder_ != null) { + return entriesBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(entries_); + } + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder + addEntriesBuilder() { + return getEntriesFieldBuilder() + .addBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .getDefaultInstance()); + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder + addEntriesBuilder(int index) { + return getEntriesFieldBuilder() + .addBuilder( + index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .getDefaultInstance()); + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder> + getEntriesBuilderList() { + return getEntriesFieldBuilder().getBuilderList(); + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder> + getEntriesFieldBuilder() { + if (entriesBuilder_ == null) { + entriesBuilder_ = + new akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder>( + entries_, ((bitField0_ & 0x00000001) != 0), getParentForChildren(), isClean()); + entries_ = null; + } + return entriesBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:ORSetDeltaGroup) + } + + // @@protoc_insertion_point(class_scope:ORSetDeltaGroup) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .ORSetDeltaGroup + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public ORSetDeltaGroup parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new ORSetDeltaGroup(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface VersionVectorOrBuilder + extends + // @@protoc_insertion_point(interface_extends:VersionVector) + akka.protobufv3.internal.MessageOrBuilder { + + /** repeated .VersionVector.Entry entries = 1; */ + java.util.List + getEntriesList(); + /** repeated .VersionVector.Entry entries = 1; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry getEntries( + int index); + /** repeated .VersionVector.Entry entries = 1; */ + int getEntriesCount(); + /** repeated .VersionVector.Entry entries = 1; */ + java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder> + getEntriesOrBuilderList(); + /** repeated .VersionVector.Entry entries = 1; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.EntryOrBuilder + getEntriesOrBuilder(int index); + } + /** Protobuf type {@code VersionVector} */ + public static final class VersionVector extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:VersionVector) + VersionVectorOrBuilder { + private static final long serialVersionUID = 0L; + // Use VersionVector.newBuilder() to construct. + private VersionVector(akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private VersionVector() { + entries_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new VersionVector(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private VersionVector( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + if (!((mutable_bitField0_ & 0x00000001) != 0)) { + entries_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Entry>(); + mutable_bitField0_ |= 0x00000001; + } + entries_.add( + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Entry.PARSER, + extensionRegistry)); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000001) != 0)) { + entries_ = java.util.Collections.unmodifiableList(entries_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + .class); + } + + public interface EntryOrBuilder + extends + // @@protoc_insertion_point(interface_extends:VersionVector.Entry) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * required string key = 1; + * + * @return Whether the key field is set. + */ + boolean hasKey(); + /** + * required string key = 1; + * + * @return The key. + */ + java.lang.String getKey(); + /** + * required string key = 1; + * + * @return The bytes for key. + */ + akka.protobufv3.internal.ByteString getKeyBytes(); + + /** + * required int64 version = 2; + * + * @return Whether the version field is set. + */ + boolean hasVersion(); + /** + * required int64 version = 2; + * + * @return The version. + */ + long getVersion(); + } + /** Protobuf type {@code VersionVector.Entry} */ + public static final class Entry extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:VersionVector.Entry) + EntryOrBuilder { + private static final long serialVersionUID = 0L; + // Use Entry.newBuilder() to construct. + private Entry(akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Entry() { + key_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new Entry(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Entry( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + akka.protobufv3.internal.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + key_ = bs; + break; + } + case 16: + { + bitField0_ |= 0x00000002; + version_ = input.readInt64(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_Entry_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_Entry_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder.class); + } + + private int bitField0_; + public static final int KEY_FIELD_NUMBER = 1; + private volatile java.lang.Object key_; + /** + * required string key = 1; + * + * @return Whether the key field is set. + */ + public boolean hasKey() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required string key = 1; + * + * @return The key. + */ + public java.lang.String getKey() { + java.lang.Object ref = key_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + key_ = s; + } + return s; + } + } + /** + * required string key = 1; + * + * @return The bytes for key. + */ + public akka.protobufv3.internal.ByteString getKeyBytes() { + java.lang.Object ref = key_; + if (ref instanceof java.lang.String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + key_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + + public static final int VERSION_FIELD_NUMBER = 2; + private long version_; + /** + * required int64 version = 2; + * + * @return Whether the version field is set. + */ + public boolean hasVersion() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required int64 version = 2; + * + * @return The version. + */ + public long getVersion() { + return version_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasKey()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasVersion()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + akka.protobufv3.internal.GeneratedMessageV3.writeString(output, 1, key_); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeInt64(2, version_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.GeneratedMessageV3.computeStringSize(1, key_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeInt64Size(2, version_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry) obj; + + if (hasKey() != other.hasKey()) return false; + if (hasKey()) { + if (!getKey().equals(other.getKey())) return false; + } + if (hasVersion() != other.hasVersion()) return false; + if (hasVersion()) { + if (getVersion() != other.getVersion()) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasKey()) { + hash = (37 * hash) + KEY_FIELD_NUMBER; + hash = (53 * hash) + getKey().hashCode(); + } + if (hasVersion()) { + hash = (37 * hash) + VERSION_FIELD_NUMBER; + hash = (53 * hash) + akka.protobufv3.internal.Internal.hashLong(getVersion()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code VersionVector.Entry} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:VersionVector.Entry) + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_Entry_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_Entry_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + key_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + version_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_Entry_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry( + this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + to_bitField0_ |= 0x00000001; + } + result.key_ = key_; + if (((from_bitField0_ & 0x00000002) != 0)) { + result.version_ = version_; + to_bitField0_ |= 0x00000002; + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .getDefaultInstance()) return this; + if (other.hasKey()) { + bitField0_ |= 0x00000001; + key_ = other.key_; + onChanged(); + } + if (other.hasVersion()) { + setVersion(other.getVersion()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (!hasKey()) { + return false; + } + if (!hasVersion()) { + return false; + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.lang.Object key_ = ""; + /** + * required string key = 1; + * + * @return Whether the key field is set. + */ + public boolean hasKey() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required string key = 1; + * + * @return The key. + */ + public java.lang.String getKey() { + java.lang.Object ref = key_; + if (!(ref instanceof java.lang.String)) { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + key_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string key = 1; + * + * @return The bytes for key. + */ + public akka.protobufv3.internal.ByteString getKeyBytes() { + java.lang.Object ref = key_; + if (ref instanceof String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + key_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + /** + * required string key = 1; + * + * @param value The key to set. + * @return This builder for chaining. + */ + public Builder setKey(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + key_ = value; + onChanged(); + return this; + } + /** + * required string key = 1; + * + * @return This builder for chaining. + */ + public Builder clearKey() { + bitField0_ = (bitField0_ & ~0x00000001); + key_ = getDefaultInstance().getKey(); + onChanged(); + return this; + } + /** + * required string key = 1; + * + * @param value The bytes for key to set. + * @return This builder for chaining. + */ + public Builder setKeyBytes(akka.protobufv3.internal.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + key_ = value; + onChanged(); + return this; + } + + private long version_; + /** + * required int64 version = 2; + * + * @return Whether the version field is set. + */ + public boolean hasVersion() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required int64 version = 2; + * + * @return The version. + */ + public long getVersion() { + return version_; + } + /** + * required int64 version = 2; + * + * @param value The version to set. + * @return This builder for chaining. + */ + public Builder setVersion(long value) { + bitField0_ |= 0x00000002; + version_ = value; + onChanged(); + return this; + } + /** + * required int64 version = 2; + * + * @return This builder for chaining. + */ + public Builder clearVersion() { + bitField0_ = (bitField0_ & ~0x00000002); + version_ = 0L; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:VersionVector.Entry) + } + + // @@protoc_insertion_point(class_scope:VersionVector.Entry) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVector.Entry + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public Entry parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new Entry(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public static final int ENTRIES_FIELD_NUMBER = 1; + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry> + entries_; + /** repeated .VersionVector.Entry entries = 1; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry> + getEntriesList() { + return entries_; + } + /** repeated .VersionVector.Entry entries = 1; */ + public java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder> + getEntriesOrBuilderList() { + return entries_; + } + /** repeated .VersionVector.Entry entries = 1; */ + public int getEntriesCount() { + return entries_.size(); + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + getEntries(int index) { + return entries_.get(index); + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.EntryOrBuilder + getEntriesOrBuilder(int index) { + return entries_.get(index); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + for (int i = 0; i < getEntriesCount(); i++) { + if (!getEntries(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + for (int i = 0; i < entries_.size(); i++) { + output.writeMessage(1, entries_.get(i)); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < entries_.size(); i++) { + size += akka.protobufv3.internal.CodedOutputStream.computeMessageSize(1, entries_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector) obj; + + if (!getEntriesList().equals(other.getEntriesList())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (getEntriesCount() > 0) { + hash = (37 * hash) + ENTRIES_FIELD_NUMBER; + hash = (53 * hash) + getEntriesList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code VersionVector} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:VersionVector) + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + .class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) { + getEntriesFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + if (entriesBuilder_ == null) { + entries_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + entriesBuilder_.clear(); + } + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_VersionVector_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector(this); + int from_bitField0_ = bitField0_; + if (entriesBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0)) { + entries_ = java.util.Collections.unmodifiableList(entries_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.entries_ = entries_; + } else { + result.entries_ = entriesBuilder_.build(); + } + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance()) return this; + if (entriesBuilder_ == null) { + if (!other.entries_.isEmpty()) { + if (entries_.isEmpty()) { + entries_ = other.entries_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureEntriesIsMutable(); + entries_.addAll(other.entries_); + } + onChanged(); + } + } else { + if (!other.entries_.isEmpty()) { + if (entriesBuilder_.isEmpty()) { + entriesBuilder_.dispose(); + entriesBuilder_ = null; + entries_ = other.entries_; + bitField0_ = (bitField0_ & ~0x00000001); + entriesBuilder_ = + akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders + ? getEntriesFieldBuilder() + : null; + } else { + entriesBuilder_.addAllMessages(other.entries_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + for (int i = 0; i < getEntriesCount(); i++) { + if (!getEntries(i).isInitialized()) { + return false; + } + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry> + entries_ = java.util.Collections.emptyList(); + + private void ensureEntriesIsMutable() { + if (!((bitField0_ & 0x00000001) != 0)) { + entries_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry>( + entries_); + bitField0_ |= 0x00000001; + } + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder> + entriesBuilder_; + + /** repeated .VersionVector.Entry entries = 1; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry> + getEntriesList() { + if (entriesBuilder_ == null) { + return java.util.Collections.unmodifiableList(entries_); + } else { + return entriesBuilder_.getMessageList(); + } + } + /** repeated .VersionVector.Entry entries = 1; */ + public int getEntriesCount() { + if (entriesBuilder_ == null) { + return entries_.size(); + } else { + return entriesBuilder_.getCount(); + } + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + getEntries(int index) { + if (entriesBuilder_ == null) { + return entries_.get(index); + } else { + return entriesBuilder_.getMessage(index); + } + } + /** repeated .VersionVector.Entry entries = 1; */ + public Builder setEntries( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry value) { + if (entriesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureEntriesIsMutable(); + entries_.set(index, value); + onChanged(); + } else { + entriesBuilder_.setMessage(index, value); + } + return this; + } + /** repeated .VersionVector.Entry entries = 1; */ + public Builder setEntries( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry.Builder + builderForValue) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + entries_.set(index, builderForValue.build()); + onChanged(); + } else { + entriesBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .VersionVector.Entry entries = 1; */ + public Builder addEntries( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry value) { + if (entriesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureEntriesIsMutable(); + entries_.add(value); + onChanged(); + } else { + entriesBuilder_.addMessage(value); + } + return this; + } + /** repeated .VersionVector.Entry entries = 1; */ + public Builder addEntries( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry value) { + if (entriesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureEntriesIsMutable(); + entries_.add(index, value); + onChanged(); + } else { + entriesBuilder_.addMessage(index, value); + } + return this; + } + /** repeated .VersionVector.Entry entries = 1; */ + public Builder addEntries( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry.Builder + builderForValue) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + entries_.add(builderForValue.build()); + onChanged(); + } else { + entriesBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** repeated .VersionVector.Entry entries = 1; */ + public Builder addEntries( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry.Builder + builderForValue) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + entries_.add(index, builderForValue.build()); + onChanged(); + } else { + entriesBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .VersionVector.Entry entries = 1; */ + public Builder addAllEntries( + java.lang.Iterable< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Entry> + values) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + akka.protobufv3.internal.AbstractMessageLite.Builder.addAll(values, entries_); + onChanged(); + } else { + entriesBuilder_.addAllMessages(values); + } + return this; + } + /** repeated .VersionVector.Entry entries = 1; */ + public Builder clearEntries() { + if (entriesBuilder_ == null) { + entries_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + entriesBuilder_.clear(); + } + return this; + } + /** repeated .VersionVector.Entry entries = 1; */ + public Builder removeEntries(int index) { + if (entriesBuilder_ == null) { + ensureEntriesIsMutable(); + entries_.remove(index); + onChanged(); + } else { + entriesBuilder_.remove(index); + } + return this; + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder + getEntriesBuilder(int index) { + return getEntriesFieldBuilder().getBuilder(index); + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder + getEntriesOrBuilder(int index) { + if (entriesBuilder_ == null) { + return entries_.get(index); + } else { + return entriesBuilder_.getMessageOrBuilder(index); + } + } + /** repeated .VersionVector.Entry entries = 1; */ + public java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder> + getEntriesOrBuilderList() { + if (entriesBuilder_ != null) { + return entriesBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(entries_); + } + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder + addEntriesBuilder() { + return getEntriesFieldBuilder() + .addBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .getDefaultInstance()); + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder + addEntriesBuilder(int index) { + return getEntriesFieldBuilder() + .addBuilder( + index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .getDefaultInstance()); + } + /** repeated .VersionVector.Entry entries = 1; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder> + getEntriesBuilderList() { + return getEntriesFieldBuilder().getBuilderList(); + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder> + getEntriesFieldBuilder() { + if (entriesBuilder_ == null) { + entriesBuilder_ = + new akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder>( + entries_, ((bitField0_ & 0x00000001) != 0), getParentForChildren(), isClean()); + entries_ = null; + } + return entriesBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:VersionVector) + } + + // @@protoc_insertion_point(class_scope:VersionVector) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public VersionVector parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new VersionVector(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface ReplicatedEventMetadataOrBuilder + extends + // @@protoc_insertion_point(interface_extends:ReplicatedEventMetadata) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * required string originReplica = 1; + * + * @return Whether the originReplica field is set. + */ + boolean hasOriginReplica(); + /** + * required string originReplica = 1; + * + * @return The originReplica. + */ + java.lang.String getOriginReplica(); + /** + * required string originReplica = 1; + * + * @return The bytes for originReplica. + */ + akka.protobufv3.internal.ByteString getOriginReplicaBytes(); + + /** + * required int64 originSequenceNr = 2; + * + * @return Whether the originSequenceNr field is set. + */ + boolean hasOriginSequenceNr(); + /** + * required int64 originSequenceNr = 2; + * + * @return The originSequenceNr. + */ + long getOriginSequenceNr(); + + /** + * required .VersionVector versionVector = 3; + * + * @return Whether the versionVector field is set. + */ + boolean hasVersionVector(); + /** + * required .VersionVector versionVector = 3; + * + * @return The versionVector. + */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVersionVector(); + /** required .VersionVector versionVector = 3; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionVectorOrBuilder(); + + /** + * required bool concurrent = 4; + * + * @return Whether the concurrent field is set. + */ + boolean hasConcurrent(); + /** + * required bool concurrent = 4; + * + * @return The concurrent. + */ + boolean getConcurrent(); + } + /** Protobuf type {@code ReplicatedEventMetadata} */ + public static final class ReplicatedEventMetadata + extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:ReplicatedEventMetadata) + ReplicatedEventMetadataOrBuilder { + private static final long serialVersionUID = 0L; + // Use ReplicatedEventMetadata.newBuilder() to construct. + private ReplicatedEventMetadata( + akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ReplicatedEventMetadata() { + originReplica_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new ReplicatedEventMetadata(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ReplicatedEventMetadata( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + akka.protobufv3.internal.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + originReplica_ = bs; + break; + } + case 16: + { + bitField0_ |= 0x00000002; + originSequenceNr_ = input.readInt64(); + break; + } + case 26: + { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + subBuilder = null; + if (((bitField0_ & 0x00000004) != 0)) { + subBuilder = versionVector_.toBuilder(); + } + versionVector_ = + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(versionVector_); + versionVector_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000004; + break; + } + case 32: + { + bitField0_ |= 0x00000008; + concurrent_ = input.readBool(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedEventMetadata_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedEventMetadata_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .Builder.class); + } + + private int bitField0_; + public static final int ORIGINREPLICA_FIELD_NUMBER = 1; + private volatile java.lang.Object originReplica_; + /** + * required string originReplica = 1; + * + * @return Whether the originReplica field is set. + */ + public boolean hasOriginReplica() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required string originReplica = 1; + * + * @return The originReplica. + */ + public java.lang.String getOriginReplica() { + java.lang.Object ref = originReplica_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + originReplica_ = s; + } + return s; + } + } + /** + * required string originReplica = 1; + * + * @return The bytes for originReplica. + */ + public akka.protobufv3.internal.ByteString getOriginReplicaBytes() { + java.lang.Object ref = originReplica_; + if (ref instanceof java.lang.String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + originReplica_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + + public static final int ORIGINSEQUENCENR_FIELD_NUMBER = 2; + private long originSequenceNr_; + /** + * required int64 originSequenceNr = 2; + * + * @return Whether the originSequenceNr field is set. + */ + public boolean hasOriginSequenceNr() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required int64 originSequenceNr = 2; + * + * @return The originSequenceNr. + */ + public long getOriginSequenceNr() { + return originSequenceNr_; + } + + public static final int VERSIONVECTOR_FIELD_NUMBER = 3; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + versionVector_; + /** + * required .VersionVector versionVector = 3; + * + * @return Whether the versionVector field is set. + */ + public boolean hasVersionVector() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * required .VersionVector versionVector = 3; + * + * @return The versionVector. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVersionVector() { + return versionVector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : versionVector_; + } + /** required .VersionVector versionVector = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionVectorOrBuilder() { + return versionVector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : versionVector_; + } + + public static final int CONCURRENT_FIELD_NUMBER = 4; + private boolean concurrent_; + /** + * required bool concurrent = 4; + * + * @return Whether the concurrent field is set. + */ + public boolean hasConcurrent() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * required bool concurrent = 4; + * + * @return The concurrent. + */ + public boolean getConcurrent() { + return concurrent_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasOriginReplica()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasOriginSequenceNr()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasVersionVector()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasConcurrent()) { + memoizedIsInitialized = 0; + return false; + } + if (!getVersionVector().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + akka.protobufv3.internal.GeneratedMessageV3.writeString(output, 1, originReplica_); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeInt64(2, originSequenceNr_); + } + if (((bitField0_ & 0x00000004) != 0)) { + output.writeMessage(3, getVersionVector()); + } + if (((bitField0_ & 0x00000008) != 0)) { + output.writeBool(4, concurrent_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.GeneratedMessageV3.computeStringSize(1, originReplica_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeInt64Size(2, originSequenceNr_); + } + if (((bitField0_ & 0x00000004) != 0)) { + size += + akka.protobufv3.internal.CodedOutputStream.computeMessageSize(3, getVersionVector()); + } + if (((bitField0_ & 0x00000008) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeBoolSize(4, concurrent_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata) + obj; + + if (hasOriginReplica() != other.hasOriginReplica()) return false; + if (hasOriginReplica()) { + if (!getOriginReplica().equals(other.getOriginReplica())) return false; + } + if (hasOriginSequenceNr() != other.hasOriginSequenceNr()) return false; + if (hasOriginSequenceNr()) { + if (getOriginSequenceNr() != other.getOriginSequenceNr()) return false; + } + if (hasVersionVector() != other.hasVersionVector()) return false; + if (hasVersionVector()) { + if (!getVersionVector().equals(other.getVersionVector())) return false; + } + if (hasConcurrent() != other.hasConcurrent()) return false; + if (hasConcurrent()) { + if (getConcurrent() != other.getConcurrent()) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasOriginReplica()) { + hash = (37 * hash) + ORIGINREPLICA_FIELD_NUMBER; + hash = (53 * hash) + getOriginReplica().hashCode(); + } + if (hasOriginSequenceNr()) { + hash = (37 * hash) + ORIGINSEQUENCENR_FIELD_NUMBER; + hash = (53 * hash) + akka.protobufv3.internal.Internal.hashLong(getOriginSequenceNr()); + } + if (hasVersionVector()) { + hash = (37 * hash) + VERSIONVECTOR_FIELD_NUMBER; + hash = (53 * hash) + getVersionVector().hashCode(); + } + if (hasConcurrent()) { + hash = (37 * hash) + CONCURRENT_FIELD_NUMBER; + hash = (53 * hash) + akka.protobufv3.internal.Internal.hashBoolean(getConcurrent()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code ReplicatedEventMetadata} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:ReplicatedEventMetadata) + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadataOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedEventMetadata_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedEventMetadata_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) { + getVersionVectorFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + originReplica_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + originSequenceNr_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + if (versionVectorBuilder_ == null) { + versionVector_ = null; + } else { + versionVectorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + concurrent_ = false; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedEventMetadata_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + to_bitField0_ |= 0x00000001; + } + result.originReplica_ = originReplica_; + if (((from_bitField0_ & 0x00000002) != 0)) { + result.originSequenceNr_ = originSequenceNr_; + to_bitField0_ |= 0x00000002; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + if (versionVectorBuilder_ == null) { + result.versionVector_ = versionVector_; + } else { + result.versionVector_ = versionVectorBuilder_.build(); + } + to_bitField0_ |= 0x00000004; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.concurrent_ = concurrent_; + to_bitField0_ |= 0x00000008; + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .getDefaultInstance()) return this; + if (other.hasOriginReplica()) { + bitField0_ |= 0x00000001; + originReplica_ = other.originReplica_; + onChanged(); + } + if (other.hasOriginSequenceNr()) { + setOriginSequenceNr(other.getOriginSequenceNr()); + } + if (other.hasVersionVector()) { + mergeVersionVector(other.getVersionVector()); + } + if (other.hasConcurrent()) { + setConcurrent(other.getConcurrent()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (!hasOriginReplica()) { + return false; + } + if (!hasOriginSequenceNr()) { + return false; + } + if (!hasVersionVector()) { + return false; + } + if (!hasConcurrent()) { + return false; + } + if (!getVersionVector().isInitialized()) { + return false; + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.lang.Object originReplica_ = ""; + /** + * required string originReplica = 1; + * + * @return Whether the originReplica field is set. + */ + public boolean hasOriginReplica() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required string originReplica = 1; + * + * @return The originReplica. + */ + public java.lang.String getOriginReplica() { + java.lang.Object ref = originReplica_; + if (!(ref instanceof java.lang.String)) { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + originReplica_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string originReplica = 1; + * + * @return The bytes for originReplica. + */ + public akka.protobufv3.internal.ByteString getOriginReplicaBytes() { + java.lang.Object ref = originReplica_; + if (ref instanceof String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + originReplica_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + /** + * required string originReplica = 1; + * + * @param value The originReplica to set. + * @return This builder for chaining. + */ + public Builder setOriginReplica(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + originReplica_ = value; + onChanged(); + return this; + } + /** + * required string originReplica = 1; + * + * @return This builder for chaining. + */ + public Builder clearOriginReplica() { + bitField0_ = (bitField0_ & ~0x00000001); + originReplica_ = getDefaultInstance().getOriginReplica(); + onChanged(); + return this; + } + /** + * required string originReplica = 1; + * + * @param value The bytes for originReplica to set. + * @return This builder for chaining. + */ + public Builder setOriginReplicaBytes(akka.protobufv3.internal.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + originReplica_ = value; + onChanged(); + return this; + } + + private long originSequenceNr_; + /** + * required int64 originSequenceNr = 2; + * + * @return Whether the originSequenceNr field is set. + */ + public boolean hasOriginSequenceNr() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required int64 originSequenceNr = 2; + * + * @return The originSequenceNr. + */ + public long getOriginSequenceNr() { + return originSequenceNr_; + } + /** + * required int64 originSequenceNr = 2; + * + * @param value The originSequenceNr to set. + * @return This builder for chaining. + */ + public Builder setOriginSequenceNr(long value) { + bitField0_ |= 0x00000002; + originSequenceNr_ = value; + onChanged(); + return this; + } + /** + * required int64 originSequenceNr = 2; + * + * @return This builder for chaining. + */ + public Builder clearOriginSequenceNr() { + bitField0_ = (bitField0_ & ~0x00000002); + originSequenceNr_ = 0L; + onChanged(); + return this; + } + + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + versionVector_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + versionVectorBuilder_; + /** + * required .VersionVector versionVector = 3; + * + * @return Whether the versionVector field is set. + */ + public boolean hasVersionVector() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * required .VersionVector versionVector = 3; + * + * @return The versionVector. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVersionVector() { + if (versionVectorBuilder_ == null) { + return versionVector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : versionVector_; + } else { + return versionVectorBuilder_.getMessage(); + } + } + /** required .VersionVector versionVector = 3; */ + public Builder setVersionVector( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (versionVectorBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + versionVector_ = value; + onChanged(); + } else { + versionVectorBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** required .VersionVector versionVector = 3; */ + public Builder setVersionVector( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { + if (versionVectorBuilder_ == null) { + versionVector_ = builderForValue.build(); + onChanged(); + } else { + versionVectorBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + /** required .VersionVector versionVector = 3; */ + public Builder mergeVersionVector( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (versionVectorBuilder_ == null) { + if (((bitField0_ & 0x00000004) != 0) + && versionVector_ != null + && versionVector_ + != akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance()) { + versionVector_ = + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .newBuilder(versionVector_) + .mergeFrom(value) + .buildPartial(); + } else { + versionVector_ = value; + } + onChanged(); + } else { + versionVectorBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** required .VersionVector versionVector = 3; */ + public Builder clearVersionVector() { + if (versionVectorBuilder_ == null) { + versionVector_ = null; + onChanged(); + } else { + versionVectorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + /** required .VersionVector versionVector = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + getVersionVectorBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getVersionVectorFieldBuilder().getBuilder(); + } + /** required .VersionVector versionVector = 3; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionVectorOrBuilder() { + if (versionVectorBuilder_ != null) { + return versionVectorBuilder_.getMessageOrBuilder(); + } else { + return versionVector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : versionVector_; + } + } + /** required .VersionVector versionVector = 3; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + getVersionVectorFieldBuilder() { + if (versionVectorBuilder_ == null) { + versionVectorBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVectorOrBuilder>( + getVersionVector(), getParentForChildren(), isClean()); + versionVector_ = null; + } + return versionVectorBuilder_; + } + + private boolean concurrent_; + /** + * required bool concurrent = 4; + * + * @return Whether the concurrent field is set. + */ + public boolean hasConcurrent() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * required bool concurrent = 4; + * + * @return The concurrent. + */ + public boolean getConcurrent() { + return concurrent_; + } + /** + * required bool concurrent = 4; + * + * @param value The concurrent to set. + * @return This builder for chaining. + */ + public Builder setConcurrent(boolean value) { + bitField0_ |= 0x00000008; + concurrent_ = value; + onChanged(); + return this; + } + /** + * required bool concurrent = 4; + * + * @return This builder for chaining. + */ + public Builder clearConcurrent() { + bitField0_ = (bitField0_ & ~0x00000008); + concurrent_ = false; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:ReplicatedEventMetadata) + } + + // @@protoc_insertion_point(class_scope:ReplicatedEventMetadata) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public ReplicatedEventMetadata parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new ReplicatedEventMetadata(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface ReplicatedSnapshotMetadataOrBuilder + extends + // @@protoc_insertion_point(interface_extends:ReplicatedSnapshotMetadata) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * required .VersionVector version = 1; + * + * @return Whether the version field is set. + */ + boolean hasVersion(); + /** + * required .VersionVector version = 1; + * + * @return The version. + */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVersion(); + /** required .VersionVector version = 1; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionOrBuilder(); + + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen> + getSeenPerReplicaList(); + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata.Seen + getSeenPerReplica(int index); + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + int getSeenPerReplicaCount(); + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.SeenOrBuilder> + getSeenPerReplicaOrBuilderList(); + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .SeenOrBuilder + getSeenPerReplicaOrBuilder(int index); + } + /** Protobuf type {@code ReplicatedSnapshotMetadata} */ + public static final class ReplicatedSnapshotMetadata + extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:ReplicatedSnapshotMetadata) + ReplicatedSnapshotMetadataOrBuilder { + private static final long serialVersionUID = 0L; + // Use ReplicatedSnapshotMetadata.newBuilder() to construct. + private ReplicatedSnapshotMetadata( + akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ReplicatedSnapshotMetadata() { + seenPerReplica_ = java.util.Collections.emptyList(); + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new ReplicatedSnapshotMetadata(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ReplicatedSnapshotMetadata( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + subBuilder = null; + if (((bitField0_ & 0x00000001) != 0)) { + subBuilder = version_.toBuilder(); + } + version_ = + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(version_); + version_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + case 18: + { + if (!((mutable_bitField0_ & 0x00000002) != 0)) { + seenPerReplica_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen>(); + mutable_bitField0_ |= 0x00000002; + } + seenPerReplica_.add( + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.PARSER, + extensionRegistry)); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + if (((mutable_bitField0_ & 0x00000002) != 0)) { + seenPerReplica_ = java.util.Collections.unmodifiableList(seenPerReplica_); + } + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Builder.class); + } + + public interface SeenOrBuilder + extends + // @@protoc_insertion_point(interface_extends:ReplicatedSnapshotMetadata.Seen) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * required string replicaId = 1; + * + * @return Whether the replicaId field is set. + */ + boolean hasReplicaId(); + /** + * required string replicaId = 1; + * + * @return The replicaId. + */ + java.lang.String getReplicaId(); + /** + * required string replicaId = 1; + * + * @return The bytes for replicaId. + */ + akka.protobufv3.internal.ByteString getReplicaIdBytes(); + + /** + * required int64 sequenceNr = 2; + * + * @return Whether the sequenceNr field is set. + */ + boolean hasSequenceNr(); + /** + * required int64 sequenceNr = 2; + * + * @return The sequenceNr. + */ + long getSequenceNr(); + } + /** Protobuf type {@code ReplicatedSnapshotMetadata.Seen} */ + public static final class Seen extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:ReplicatedSnapshotMetadata.Seen) + SeenOrBuilder { + private static final long serialVersionUID = 0L; + // Use Seen.newBuilder() to construct. + private Seen(akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private Seen() { + replicaId_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new Seen(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private Seen( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + akka.protobufv3.internal.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + replicaId_ = bs; + break; + } + case 16: + { + bitField0_ |= 0x00000002; + sequenceNr_ = input.readInt64(); + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_Seen_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_Seen_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.Builder.class); + } + + private int bitField0_; + public static final int REPLICAID_FIELD_NUMBER = 1; + private volatile java.lang.Object replicaId_; + /** + * required string replicaId = 1; + * + * @return Whether the replicaId field is set. + */ + public boolean hasReplicaId() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required string replicaId = 1; + * + * @return The replicaId. + */ + public java.lang.String getReplicaId() { + java.lang.Object ref = replicaId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + replicaId_ = s; + } + return s; + } + } + /** + * required string replicaId = 1; + * + * @return The bytes for replicaId. + */ + public akka.protobufv3.internal.ByteString getReplicaIdBytes() { + java.lang.Object ref = replicaId_; + if (ref instanceof java.lang.String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + replicaId_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + + public static final int SEQUENCENR_FIELD_NUMBER = 2; + private long sequenceNr_; + /** + * required int64 sequenceNr = 2; + * + * @return Whether the sequenceNr field is set. + */ + public boolean hasSequenceNr() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required int64 sequenceNr = 2; + * + * @return The sequenceNr. + */ + public long getSequenceNr() { + return sequenceNr_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasReplicaId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSequenceNr()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + akka.protobufv3.internal.GeneratedMessageV3.writeString(output, 1, replicaId_); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeInt64(2, sequenceNr_); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.GeneratedMessageV3.computeStringSize(1, replicaId_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeInt64Size(2, sequenceNr_); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata.Seen + other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen) + obj; + + if (hasReplicaId() != other.hasReplicaId()) return false; + if (hasReplicaId()) { + if (!getReplicaId().equals(other.getReplicaId())) return false; + } + if (hasSequenceNr() != other.hasSequenceNr()) return false; + if (hasSequenceNr()) { + if (getSequenceNr() != other.getSequenceNr()) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasReplicaId()) { + hash = (37 * hash) + REPLICAID_FIELD_NUMBER; + hash = (53 * hash) + getReplicaId().hashCode(); + } + if (hasSequenceNr()) { + hash = (37 * hash) + SEQUENCENR_FIELD_NUMBER; + hash = (53 * hash) + akka.protobufv3.internal.Internal.hashLong(getSequenceNr()); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code ReplicatedSnapshotMetadata.Seen} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:ReplicatedSnapshotMetadata.Seen) + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .SeenOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_Seen_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_Seen_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata.Seen.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) {} + } + + @java.lang.Override + public Builder clear() { + super.clear(); + replicaId_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + sequenceNr_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_Seen_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + to_bitField0_ |= 0x00000001; + } + result.replicaId_ = replicaId_; + if (((from_bitField0_ & 0x00000002) != 0)) { + result.sequenceNr_ = sequenceNr_; + to_bitField0_ |= 0x00000002; + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.getDefaultInstance()) return this; + if (other.hasReplicaId()) { + bitField0_ |= 0x00000001; + replicaId_ = other.replicaId_; + onChanged(); + } + if (other.hasSequenceNr()) { + setSequenceNr(other.getSequenceNr()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (!hasReplicaId()) { + return false; + } + if (!hasSequenceNr()) { + return false; + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.lang.Object replicaId_ = ""; + /** + * required string replicaId = 1; + * + * @return Whether the replicaId field is set. + */ + public boolean hasReplicaId() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required string replicaId = 1; + * + * @return The replicaId. + */ + public java.lang.String getReplicaId() { + java.lang.Object ref = replicaId_; + if (!(ref instanceof java.lang.String)) { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + replicaId_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string replicaId = 1; + * + * @return The bytes for replicaId. + */ + public akka.protobufv3.internal.ByteString getReplicaIdBytes() { + java.lang.Object ref = replicaId_; + if (ref instanceof String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + replicaId_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + /** + * required string replicaId = 1; + * + * @param value The replicaId to set. + * @return This builder for chaining. + */ + public Builder setReplicaId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + replicaId_ = value; + onChanged(); + return this; + } + /** + * required string replicaId = 1; + * + * @return This builder for chaining. + */ + public Builder clearReplicaId() { + bitField0_ = (bitField0_ & ~0x00000001); + replicaId_ = getDefaultInstance().getReplicaId(); + onChanged(); + return this; + } + /** + * required string replicaId = 1; + * + * @param value The bytes for replicaId to set. + * @return This builder for chaining. + */ + public Builder setReplicaIdBytes(akka.protobufv3.internal.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + replicaId_ = value; + onChanged(); + return this; + } + + private long sequenceNr_; + /** + * required int64 sequenceNr = 2; + * + * @return Whether the sequenceNr field is set. + */ + public boolean hasSequenceNr() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * required int64 sequenceNr = 2; + * + * @return The sequenceNr. + */ + public long getSequenceNr() { + return sequenceNr_; + } + /** + * required int64 sequenceNr = 2; + * + * @param value The sequenceNr to set. + * @return This builder for chaining. + */ + public Builder setSequenceNr(long value) { + bitField0_ |= 0x00000002; + sequenceNr_ = value; + onChanged(); + return this; + } + /** + * required int64 sequenceNr = 2; + * + * @return This builder for chaining. + */ + public Builder clearSequenceNr() { + bitField0_ = (bitField0_ & ~0x00000002); + sequenceNr_ = 0L; + onChanged(); + return this; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:ReplicatedSnapshotMetadata.Seen) + } + + // @@protoc_insertion_point(class_scope:ReplicatedSnapshotMetadata.Seen) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public Seen parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new Seen(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + private int bitField0_; + public static final int VERSION_FIELD_NUMBER = 1; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector version_; + /** + * required .VersionVector version = 1; + * + * @return Whether the version field is set. + */ + public boolean hasVersion() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required .VersionVector version = 1; + * + * @return The version. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVersion() { + return version_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : version_; + } + /** required .VersionVector version = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionOrBuilder() { + return version_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : version_; + } + + public static final int SEENPERREPLICA_FIELD_NUMBER = 2; + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen> + seenPerReplica_; + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen> + getSeenPerReplicaList() { + return seenPerReplica_; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.SeenOrBuilder> + getSeenPerReplicaOrBuilderList() { + return seenPerReplica_; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public int getSeenPerReplicaCount() { + return seenPerReplica_.size(); + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + getSeenPerReplica(int index) { + return seenPerReplica_.get(index); + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .SeenOrBuilder + getSeenPerReplicaOrBuilder(int index) { + return seenPerReplica_.get(index); + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (!hasVersion()) { + memoizedIsInitialized = 0; + return false; + } + if (!getVersion().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getSeenPerReplicaCount(); i++) { + if (!getSeenPerReplica(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + output.writeMessage(1, getVersion()); + } + for (int i = 0; i < seenPerReplica_.size(); i++) { + output.writeMessage(2, seenPerReplica_.get(i)); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeMessageSize(1, getVersion()); + } + for (int i = 0; i < seenPerReplica_.size(); i++) { + size += + akka.protobufv3.internal.CodedOutputStream.computeMessageSize( + 2, seenPerReplica_.get(i)); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata) + obj; + + if (hasVersion() != other.hasVersion()) return false; + if (hasVersion()) { + if (!getVersion().equals(other.getVersion())) return false; + } + if (!getSeenPerReplicaList().equals(other.getSeenPerReplicaList())) return false; + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasVersion()) { + hash = (37 * hash) + VERSION_FIELD_NUMBER; + hash = (53 * hash) + getVersion().hashCode(); + } + if (getSeenPerReplicaCount() > 0) { + hash = (37 * hash) + SEENPERREPLICA_FIELD_NUMBER; + hash = (53 * hash) + getSeenPerReplicaList().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code ReplicatedSnapshotMetadata} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:ReplicatedSnapshotMetadata) + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadataOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) { + getVersionFieldBuilder(); + getSeenPerReplicaFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + if (versionBuilder_ == null) { + version_ = null; + } else { + versionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (seenPerReplicaBuilder_ == null) { + seenPerReplica_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + } else { + seenPerReplicaBuilder_.clear(); + } + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedSnapshotMetadata_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + if (versionBuilder_ == null) { + result.version_ = version_; + } else { + result.version_ = versionBuilder_.build(); + } + to_bitField0_ |= 0x00000001; + } + if (seenPerReplicaBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0)) { + seenPerReplica_ = java.util.Collections.unmodifiableList(seenPerReplica_); + bitField0_ = (bitField0_ & ~0x00000002); + } + result.seenPerReplica_ = seenPerReplica_; + } else { + result.seenPerReplica_ = seenPerReplicaBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.getDefaultInstance()) return this; + if (other.hasVersion()) { + mergeVersion(other.getVersion()); + } + if (seenPerReplicaBuilder_ == null) { + if (!other.seenPerReplica_.isEmpty()) { + if (seenPerReplica_.isEmpty()) { + seenPerReplica_ = other.seenPerReplica_; + bitField0_ = (bitField0_ & ~0x00000002); + } else { + ensureSeenPerReplicaIsMutable(); + seenPerReplica_.addAll(other.seenPerReplica_); + } + onChanged(); + } + } else { + if (!other.seenPerReplica_.isEmpty()) { + if (seenPerReplicaBuilder_.isEmpty()) { + seenPerReplicaBuilder_.dispose(); + seenPerReplicaBuilder_ = null; + seenPerReplica_ = other.seenPerReplica_; + bitField0_ = (bitField0_ & ~0x00000002); + seenPerReplicaBuilder_ = + akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders + ? getSeenPerReplicaFieldBuilder() + : null; + } else { + seenPerReplicaBuilder_.addAllMessages(other.seenPerReplica_); + } + } + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (!hasVersion()) { + return false; + } + if (!getVersion().isInitialized()) { + return false; + } + for (int i = 0; i < getSeenPerReplicaCount(); i++) { + if (!getSeenPerReplica(i).isInitialized()) { + return false; + } + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector version_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + versionBuilder_; + /** + * required .VersionVector version = 1; + * + * @return Whether the version field is set. + */ + public boolean hasVersion() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * required .VersionVector version = 1; + * + * @return The version. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVersion() { + if (versionBuilder_ == null) { + return version_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : version_; + } else { + return versionBuilder_.getMessage(); + } + } + /** required .VersionVector version = 1; */ + public Builder setVersion( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (versionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + version_ = value; + onChanged(); + } else { + versionBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** required .VersionVector version = 1; */ + public Builder setVersion( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { + if (versionBuilder_ == null) { + version_ = builderForValue.build(); + onChanged(); + } else { + versionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** required .VersionVector version = 1; */ + public Builder mergeVersion( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (versionBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) + && version_ != null + && version_ + != akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance()) { + version_ = + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .newBuilder(version_) + .mergeFrom(value) + .buildPartial(); + } else { + version_ = value; + } + onChanged(); + } else { + versionBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** required .VersionVector version = 1; */ + public Builder clearVersion() { + if (versionBuilder_ == null) { + version_ = null; + onChanged(); + } else { + versionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** required .VersionVector version = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + getVersionBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getVersionFieldBuilder().getBuilder(); + } + /** required .VersionVector version = 1; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionOrBuilder() { + if (versionBuilder_ != null) { + return versionBuilder_.getMessageOrBuilder(); + } else { + return version_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : version_; + } + } + /** required .VersionVector version = 1; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + getVersionFieldBuilder() { + if (versionBuilder_ == null) { + versionBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVectorOrBuilder>(getVersion(), getParentForChildren(), isClean()); + version_ = null; + } + return versionBuilder_; + } + + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen> + seenPerReplica_ = java.util.Collections.emptyList(); + + private void ensureSeenPerReplicaIsMutable() { + if (!((bitField0_ & 0x00000002) != 0)) { + seenPerReplica_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen>(seenPerReplica_); + bitField0_ |= 0x00000002; + } + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.SeenOrBuilder> + seenPerReplicaBuilder_; + + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen> + getSeenPerReplicaList() { + if (seenPerReplicaBuilder_ == null) { + return java.util.Collections.unmodifiableList(seenPerReplica_); + } else { + return seenPerReplicaBuilder_.getMessageList(); + } + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public int getSeenPerReplicaCount() { + if (seenPerReplicaBuilder_ == null) { + return seenPerReplica_.size(); + } else { + return seenPerReplicaBuilder_.getCount(); + } + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + getSeenPerReplica(int index) { + if (seenPerReplicaBuilder_ == null) { + return seenPerReplica_.get(index); + } else { + return seenPerReplicaBuilder_.getMessage(index); + } + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public Builder setSeenPerReplica( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + value) { + if (seenPerReplicaBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSeenPerReplicaIsMutable(); + seenPerReplica_.set(index, value); + onChanged(); + } else { + seenPerReplicaBuilder_.setMessage(index, value); + } + return this; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public Builder setSeenPerReplica( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder + builderForValue) { + if (seenPerReplicaBuilder_ == null) { + ensureSeenPerReplicaIsMutable(); + seenPerReplica_.set(index, builderForValue.build()); + onChanged(); + } else { + seenPerReplicaBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public Builder addSeenPerReplica( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + value) { + if (seenPerReplicaBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSeenPerReplicaIsMutable(); + seenPerReplica_.add(value); + onChanged(); + } else { + seenPerReplicaBuilder_.addMessage(value); + } + return this; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public Builder addSeenPerReplica( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + value) { + if (seenPerReplicaBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureSeenPerReplicaIsMutable(); + seenPerReplica_.add(index, value); + onChanged(); + } else { + seenPerReplicaBuilder_.addMessage(index, value); + } + return this; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public Builder addSeenPerReplica( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder + builderForValue) { + if (seenPerReplicaBuilder_ == null) { + ensureSeenPerReplicaIsMutable(); + seenPerReplica_.add(builderForValue.build()); + onChanged(); + } else { + seenPerReplicaBuilder_.addMessage(builderForValue.build()); + } + return this; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public Builder addSeenPerReplica( + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder + builderForValue) { + if (seenPerReplicaBuilder_ == null) { + ensureSeenPerReplicaIsMutable(); + seenPerReplica_.add(index, builderForValue.build()); + onChanged(); + } else { + seenPerReplicaBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public Builder addAllSeenPerReplica( + java.lang.Iterable< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen> + values) { + if (seenPerReplicaBuilder_ == null) { + ensureSeenPerReplicaIsMutable(); + akka.protobufv3.internal.AbstractMessageLite.Builder.addAll(values, seenPerReplica_); + onChanged(); + } else { + seenPerReplicaBuilder_.addAllMessages(values); + } + return this; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public Builder clearSeenPerReplica() { + if (seenPerReplicaBuilder_ == null) { + seenPerReplica_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + onChanged(); + } else { + seenPerReplicaBuilder_.clear(); + } + return this; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public Builder removeSeenPerReplica(int index) { + if (seenPerReplicaBuilder_ == null) { + ensureSeenPerReplicaIsMutable(); + seenPerReplica_.remove(index); + onChanged(); + } else { + seenPerReplicaBuilder_.remove(index); + } + return this; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder + getSeenPerReplicaBuilder(int index) { + return getSeenPerReplicaFieldBuilder().getBuilder(index); + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .SeenOrBuilder + getSeenPerReplicaOrBuilder(int index) { + if (seenPerReplicaBuilder_ == null) { + return seenPerReplica_.get(index); + } else { + return seenPerReplicaBuilder_.getMessageOrBuilder(index); + } + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public java.util.List< + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.SeenOrBuilder> + getSeenPerReplicaOrBuilderList() { + if (seenPerReplicaBuilder_ != null) { + return seenPerReplicaBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(seenPerReplica_); + } + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder + addSeenPerReplicaBuilder() { + return getSeenPerReplicaFieldBuilder() + .addBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.getDefaultInstance()); + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder + addSeenPerReplicaBuilder(int index) { + return getSeenPerReplicaFieldBuilder() + .addBuilder( + index, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.getDefaultInstance()); + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.Builder> + getSeenPerReplicaBuilderList() { + return getSeenPerReplicaFieldBuilder().getBuilderList(); + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.SeenOrBuilder> + getSeenPerReplicaFieldBuilder() { + if (seenPerReplicaBuilder_ == null) { + seenPerReplicaBuilder_ = + new akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.SeenOrBuilder>( + seenPerReplica_, + ((bitField0_ & 0x00000002) != 0), + getParentForChildren(), + isClean()); + seenPerReplica_ = null; + } + return seenPerReplicaBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:ReplicatedSnapshotMetadata) + } + + // @@protoc_insertion_point(class_scope:ReplicatedSnapshotMetadata) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public ReplicatedSnapshotMetadata parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new ReplicatedSnapshotMetadata(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface ReplicatedPublishedEventMetaDataOrBuilder + extends + // @@protoc_insertion_point(interface_extends:ReplicatedPublishedEventMetaData) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * optional string replicaId = 1; + * + * @return Whether the replicaId field is set. + */ + boolean hasReplicaId(); + /** + * optional string replicaId = 1; + * + * @return The replicaId. + */ + java.lang.String getReplicaId(); + /** + * optional string replicaId = 1; + * + * @return The bytes for replicaId. + */ + akka.protobufv3.internal.ByteString getReplicaIdBytes(); + + /** + * optional .VersionVector versionVector = 2; + * + * @return Whether the versionVector field is set. + */ + boolean hasVersionVector(); + /** + * optional .VersionVector versionVector = 2; + * + * @return The versionVector. + */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVersionVector(); + /** optional .VersionVector versionVector = 2; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionVectorOrBuilder(); + } + /** Protobuf type {@code ReplicatedPublishedEventMetaData} */ + public static final class ReplicatedPublishedEventMetaData + extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:ReplicatedPublishedEventMetaData) + ReplicatedPublishedEventMetaDataOrBuilder { + private static final long serialVersionUID = 0L; + // Use ReplicatedPublishedEventMetaData.newBuilder() to construct. + private ReplicatedPublishedEventMetaData( + akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private ReplicatedPublishedEventMetaData() { + replicaId_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new ReplicatedPublishedEventMetaData(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private ReplicatedPublishedEventMetaData( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + akka.protobufv3.internal.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + replicaId_ = bs; + break; + } + case 18: + { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + subBuilder = null; + if (((bitField0_ & 0x00000002) != 0)) { + subBuilder = versionVector_.toBuilder(); + } + versionVector_ = + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(versionVector_); + versionVector_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000002; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedPublishedEventMetaData_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedPublishedEventMetaData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.Builder.class); + } + + private int bitField0_; + public static final int REPLICAID_FIELD_NUMBER = 1; + private volatile java.lang.Object replicaId_; + /** + * optional string replicaId = 1; + * + * @return Whether the replicaId field is set. + */ + public boolean hasReplicaId() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * optional string replicaId = 1; + * + * @return The replicaId. + */ + public java.lang.String getReplicaId() { + java.lang.Object ref = replicaId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + replicaId_ = s; + } + return s; + } + } + /** + * optional string replicaId = 1; + * + * @return The bytes for replicaId. + */ + public akka.protobufv3.internal.ByteString getReplicaIdBytes() { + java.lang.Object ref = replicaId_; + if (ref instanceof java.lang.String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + replicaId_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + + public static final int VERSIONVECTOR_FIELD_NUMBER = 2; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + versionVector_; + /** + * optional .VersionVector versionVector = 2; + * + * @return Whether the versionVector field is set. + */ + public boolean hasVersionVector() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional .VersionVector versionVector = 2; + * + * @return The versionVector. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVersionVector() { + return versionVector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : versionVector_; + } + /** optional .VersionVector versionVector = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionVectorOrBuilder() { + return versionVector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : versionVector_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (hasVersionVector()) { + if (!getVersionVector().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + akka.protobufv3.internal.GeneratedMessageV3.writeString(output, 1, replicaId_); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeMessage(2, getVersionVector()); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.GeneratedMessageV3.computeStringSize(1, replicaId_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += + akka.protobufv3.internal.CodedOutputStream.computeMessageSize(2, getVersionVector()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedPublishedEventMetaData + other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData) + obj; + + if (hasReplicaId() != other.hasReplicaId()) return false; + if (hasReplicaId()) { + if (!getReplicaId().equals(other.getReplicaId())) return false; + } + if (hasVersionVector() != other.hasVersionVector()) return false; + if (hasVersionVector()) { + if (!getVersionVector().equals(other.getVersionVector())) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasReplicaId()) { + hash = (37 * hash) + REPLICAID_FIELD_NUMBER; + hash = (53 * hash) + getReplicaId().hashCode(); + } + if (hasVersionVector()) { + hash = (37 * hash) + VERSIONVECTOR_FIELD_NUMBER; + hash = (53 * hash) + getVersionVector().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code ReplicatedPublishedEventMetaData} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:ReplicatedPublishedEventMetaData) + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaDataOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedPublishedEventMetaData_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedPublishedEventMetaData_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedPublishedEventMetaData.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) { + getVersionVectorFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + replicaId_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + if (versionVectorBuilder_ == null) { + versionVector_ = null; + } else { + versionVectorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ReplicatedPublishedEventMetaData_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + to_bitField0_ |= 0x00000001; + } + result.replicaId_ = replicaId_; + if (((from_bitField0_ & 0x00000002) != 0)) { + if (versionVectorBuilder_ == null) { + result.versionVector_ = versionVector_; + } else { + result.versionVector_ = versionVectorBuilder_.build(); + } + to_bitField0_ |= 0x00000002; + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.getDefaultInstance()) return this; + if (other.hasReplicaId()) { + bitField0_ |= 0x00000001; + replicaId_ = other.replicaId_; + onChanged(); + } + if (other.hasVersionVector()) { + mergeVersionVector(other.getVersionVector()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (hasVersionVector()) { + if (!getVersionVector().isInitialized()) { + return false; + } + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.lang.Object replicaId_ = ""; + /** + * optional string replicaId = 1; + * + * @return Whether the replicaId field is set. + */ + public boolean hasReplicaId() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * optional string replicaId = 1; + * + * @return The replicaId. + */ + public java.lang.String getReplicaId() { + java.lang.Object ref = replicaId_; + if (!(ref instanceof java.lang.String)) { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + replicaId_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string replicaId = 1; + * + * @return The bytes for replicaId. + */ + public akka.protobufv3.internal.ByteString getReplicaIdBytes() { + java.lang.Object ref = replicaId_; + if (ref instanceof String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + replicaId_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + /** + * optional string replicaId = 1; + * + * @param value The replicaId to set. + * @return This builder for chaining. + */ + public Builder setReplicaId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + replicaId_ = value; + onChanged(); + return this; + } + /** + * optional string replicaId = 1; + * + * @return This builder for chaining. + */ + public Builder clearReplicaId() { + bitField0_ = (bitField0_ & ~0x00000001); + replicaId_ = getDefaultInstance().getReplicaId(); + onChanged(); + return this; + } + /** + * optional string replicaId = 1; + * + * @param value The bytes for replicaId to set. + * @return This builder for chaining. + */ + public Builder setReplicaIdBytes(akka.protobufv3.internal.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + replicaId_ = value; + onChanged(); + return this; + } + + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + versionVector_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + versionVectorBuilder_; + /** + * optional .VersionVector versionVector = 2; + * + * @return Whether the versionVector field is set. + */ + public boolean hasVersionVector() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional .VersionVector versionVector = 2; + * + * @return The versionVector. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVersionVector() { + if (versionVectorBuilder_ == null) { + return versionVector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : versionVector_; + } else { + return versionVectorBuilder_.getMessage(); + } + } + /** optional .VersionVector versionVector = 2; */ + public Builder setVersionVector( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (versionVectorBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + versionVector_ = value; + onChanged(); + } else { + versionVectorBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** optional .VersionVector versionVector = 2; */ + public Builder setVersionVector( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { + if (versionVectorBuilder_ == null) { + versionVector_ = builderForValue.build(); + onChanged(); + } else { + versionVectorBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + /** optional .VersionVector versionVector = 2; */ + public Builder mergeVersionVector( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { + if (versionVectorBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) + && versionVector_ != null + && versionVector_ + != akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance()) { + versionVector_ = + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .newBuilder(versionVector_) + .mergeFrom(value) + .buildPartial(); + } else { + versionVector_ = value; + } + onChanged(); + } else { + versionVectorBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + /** optional .VersionVector versionVector = 2; */ + public Builder clearVersionVector() { + if (versionVectorBuilder_ == null) { + versionVector_ = null; + onChanged(); + } else { + versionVectorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + /** optional .VersionVector versionVector = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + getVersionVectorBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getVersionVectorFieldBuilder().getBuilder(); + } + /** optional .VersionVector versionVector = 2; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionVectorOrBuilder() { + if (versionVectorBuilder_ != null) { + return versionVectorBuilder_.getMessageOrBuilder(); + } else { + return versionVector_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() + : versionVector_; + } + } + /** optional .VersionVector versionVector = 2; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> + getVersionVectorFieldBuilder() { + if (versionVectorBuilder_ == null) { + versionVectorBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVectorOrBuilder>( + getVersionVector(), getParentForChildren(), isClean()); + versionVector_ = null; + } + return versionVectorBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:ReplicatedPublishedEventMetaData) + } + + // @@protoc_insertion_point(class_scope:ReplicatedPublishedEventMetaData) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public ReplicatedPublishedEventMetaData parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new ReplicatedPublishedEventMetaData(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public interface PublishedEventOrBuilder + extends + // @@protoc_insertion_point(interface_extends:PublishedEvent) + akka.protobufv3.internal.MessageOrBuilder { + + /** + * optional string persistenceId = 1; + * + * @return Whether the persistenceId field is set. + */ + boolean hasPersistenceId(); + /** + * optional string persistenceId = 1; + * + * @return The persistenceId. + */ + java.lang.String getPersistenceId(); + /** + * optional string persistenceId = 1; + * + * @return The bytes for persistenceId. + */ + akka.protobufv3.internal.ByteString getPersistenceIdBytes(); + + /** + * optional int64 sequenceNr = 2; + * + * @return Whether the sequenceNr field is set. + */ + boolean hasSequenceNr(); + /** + * optional int64 sequenceNr = 2; + * + * @return The sequenceNr. + */ + long getSequenceNr(); + + /** + * optional .Payload payload = 3; + * + * @return Whether the payload field is set. + */ + boolean hasPayload(); + /** + * optional .Payload payload = 3; + * + * @return The payload. + */ + akka.remote.ContainerFormats.Payload getPayload(); + /** optional .Payload payload = 3; */ + akka.remote.ContainerFormats.PayloadOrBuilder getPayloadOrBuilder(); + + /** + * optional int64 timestamp = 4; + * + * @return Whether the timestamp field is set. + */ + boolean hasTimestamp(); + /** + * optional int64 timestamp = 4; + * + * @return The timestamp. + */ + long getTimestamp(); + + /** + * optional .ReplicatedPublishedEventMetaData metadata = 5; + * + * @return Whether the metadata field is set. + */ + boolean hasMetadata(); + /** + * optional .ReplicatedPublishedEventMetaData metadata = 5; + * + * @return The metadata. + */ + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedPublishedEventMetaData + getMetadata(); + /** optional .ReplicatedPublishedEventMetaData metadata = 5; */ + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaDataOrBuilder + getMetadataOrBuilder(); + } + /** Protobuf type {@code PublishedEvent} */ + public static final class PublishedEvent extends akka.protobufv3.internal.GeneratedMessageV3 + implements + // @@protoc_insertion_point(message_implements:PublishedEvent) + PublishedEventOrBuilder { + private static final long serialVersionUID = 0L; + // Use PublishedEvent.newBuilder() to construct. + private PublishedEvent(akka.protobufv3.internal.GeneratedMessageV3.Builder builder) { + super(builder); + } + + private PublishedEvent() { + persistenceId_ = ""; + } + + @java.lang.Override + @SuppressWarnings({"unused"}) + protected java.lang.Object newInstance( + akka.protobufv3.internal.GeneratedMessageV3.UnusedPrivateParameter unused) { + return new PublishedEvent(); + } + + @java.lang.Override + public final akka.protobufv3.internal.UnknownFieldSet getUnknownFields() { + return this.unknownFields; + } + + private PublishedEvent( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + this(); + if (extensionRegistry == null) { + throw new java.lang.NullPointerException(); + } + int mutable_bitField0_ = 0; + akka.protobufv3.internal.UnknownFieldSet.Builder unknownFields = + akka.protobufv3.internal.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + case 10: + { + akka.protobufv3.internal.ByteString bs = input.readBytes(); + bitField0_ |= 0x00000001; + persistenceId_ = bs; + break; + } + case 16: + { + bitField0_ |= 0x00000002; + sequenceNr_ = input.readInt64(); + break; + } + case 26: + { + akka.remote.ContainerFormats.Payload.Builder subBuilder = null; + if (((bitField0_ & 0x00000004) != 0)) { + subBuilder = payload_.toBuilder(); + } + payload_ = + input.readMessage( + akka.remote.ContainerFormats.Payload.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(payload_); + payload_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000004; + break; + } + case 32: + { + bitField0_ |= 0x00000008; + timestamp_ = input.readInt64(); + break; + } + case 42: + { + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.Builder + subBuilder = null; + if (((bitField0_ & 0x00000010) != 0)) { + subBuilder = metadata_.toBuilder(); + } + metadata_ = + input.readMessage( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.PARSER, + extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(metadata_); + metadata_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000010; + break; + } + default: + { + if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new akka.protobufv3.internal.InvalidProtocolBufferException(e) + .setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_PublishedEvent_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_PublishedEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent.Builder + .class); + } + + private int bitField0_; + public static final int PERSISTENCEID_FIELD_NUMBER = 1; + private volatile java.lang.Object persistenceId_; + /** + * optional string persistenceId = 1; + * + * @return Whether the persistenceId field is set. + */ + public boolean hasPersistenceId() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * optional string persistenceId = 1; + * + * @return The persistenceId. + */ + public java.lang.String getPersistenceId() { + java.lang.Object ref = persistenceId_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + persistenceId_ = s; + } + return s; + } + } + /** + * optional string persistenceId = 1; + * + * @return The bytes for persistenceId. + */ + public akka.protobufv3.internal.ByteString getPersistenceIdBytes() { + java.lang.Object ref = persistenceId_; + if (ref instanceof java.lang.String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + persistenceId_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + + public static final int SEQUENCENR_FIELD_NUMBER = 2; + private long sequenceNr_; + /** + * optional int64 sequenceNr = 2; + * + * @return Whether the sequenceNr field is set. + */ + public boolean hasSequenceNr() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional int64 sequenceNr = 2; + * + * @return The sequenceNr. + */ + public long getSequenceNr() { + return sequenceNr_; + } + + public static final int PAYLOAD_FIELD_NUMBER = 3; + private akka.remote.ContainerFormats.Payload payload_; + /** + * optional .Payload payload = 3; + * + * @return Whether the payload field is set. + */ + public boolean hasPayload() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * optional .Payload payload = 3; + * + * @return The payload. + */ + public akka.remote.ContainerFormats.Payload getPayload() { + return payload_ == null + ? akka.remote.ContainerFormats.Payload.getDefaultInstance() + : payload_; + } + /** optional .Payload payload = 3; */ + public akka.remote.ContainerFormats.PayloadOrBuilder getPayloadOrBuilder() { + return payload_ == null + ? akka.remote.ContainerFormats.Payload.getDefaultInstance() + : payload_; + } + + public static final int TIMESTAMP_FIELD_NUMBER = 4; + private long timestamp_; + /** + * optional int64 timestamp = 4; + * + * @return Whether the timestamp field is set. + */ + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * optional int64 timestamp = 4; + * + * @return The timestamp. + */ + public long getTimestamp() { + return timestamp_; + } + + public static final int METADATA_FIELD_NUMBER = 5; + private akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + metadata_; + /** + * optional .ReplicatedPublishedEventMetaData metadata = 5; + * + * @return Whether the metadata field is set. + */ + public boolean hasMetadata() { + return ((bitField0_ & 0x00000010) != 0); + } + /** + * optional .ReplicatedPublishedEventMetaData metadata = 5; + * + * @return The metadata. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + getMetadata() { + return metadata_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.getDefaultInstance() + : metadata_; + } + /** optional .ReplicatedPublishedEventMetaData metadata = 5; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaDataOrBuilder + getMetadataOrBuilder() { + return metadata_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.getDefaultInstance() + : metadata_; + } + + private byte memoizedIsInitialized = -1; + + @java.lang.Override + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized == 1) return true; + if (isInitialized == 0) return false; + + if (hasPayload()) { + if (!getPayload().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasMetadata()) { + if (!getMetadata().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + @java.lang.Override + public void writeTo(akka.protobufv3.internal.CodedOutputStream output) + throws java.io.IOException { + if (((bitField0_ & 0x00000001) != 0)) { + akka.protobufv3.internal.GeneratedMessageV3.writeString(output, 1, persistenceId_); + } + if (((bitField0_ & 0x00000002) != 0)) { + output.writeInt64(2, sequenceNr_); + } + if (((bitField0_ & 0x00000004) != 0)) { + output.writeMessage(3, getPayload()); + } + if (((bitField0_ & 0x00000008) != 0)) { + output.writeInt64(4, timestamp_); + } + if (((bitField0_ & 0x00000010) != 0)) { + output.writeMessage(5, getMetadata()); + } + unknownFields.writeTo(output); + } + + @java.lang.Override + public int getSerializedSize() { + int size = memoizedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) != 0)) { + size += akka.protobufv3.internal.GeneratedMessageV3.computeStringSize(1, persistenceId_); + } + if (((bitField0_ & 0x00000002) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeInt64Size(2, sequenceNr_); + } + if (((bitField0_ & 0x00000004) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeMessageSize(3, getPayload()); + } + if (((bitField0_ & 0x00000008) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeInt64Size(4, timestamp_); + } + if (((bitField0_ & 0x00000010) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream.computeMessageSize(5, getMetadata()); + } + size += unknownFields.getSerializedSize(); + memoizedSize = size; + return size; + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj + instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent) obj; + + if (hasPersistenceId() != other.hasPersistenceId()) return false; + if (hasPersistenceId()) { + if (!getPersistenceId().equals(other.getPersistenceId())) return false; + } + if (hasSequenceNr() != other.hasSequenceNr()) return false; + if (hasSequenceNr()) { + if (getSequenceNr() != other.getSequenceNr()) return false; + } + if (hasPayload() != other.hasPayload()) return false; + if (hasPayload()) { + if (!getPayload().equals(other.getPayload())) return false; + } + if (hasTimestamp() != other.hasTimestamp()) return false; + if (hasTimestamp()) { + if (getTimestamp() != other.getTimestamp()) return false; + } + if (hasMetadata() != other.hasMetadata()) return false; + if (hasMetadata()) { + if (!getMetadata().equals(other.getMetadata())) return false; + } + if (!unknownFields.equals(other.unknownFields)) return false; + return true; + } + + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptor().hashCode(); + if (hasPersistenceId()) { + hash = (37 * hash) + PERSISTENCEID_FIELD_NUMBER; + hash = (53 * hash) + getPersistenceId().hashCode(); + } + if (hasSequenceNr()) { + hash = (37 * hash) + SEQUENCENR_FIELD_NUMBER; + hash = (53 * hash) + akka.protobufv3.internal.Internal.hashLong(getSequenceNr()); + } + if (hasPayload()) { + hash = (37 * hash) + PAYLOAD_FIELD_NUMBER; + hash = (53 * hash) + getPayload().hashCode(); + } + if (hasTimestamp()) { + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + akka.protobufv3.internal.Internal.hashLong(getTimestamp()); + } + if (hasMetadata()) { + hash = (37 * hash) + METADATA_FIELD_NUMBER; + hash = (53 * hash) + getMetadata().hashCode(); + } + hash = (29 * hash) + unknownFields.hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom( + java.nio.ByteBuffer data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom( + akka.protobufv3.internal.ByteString data, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom(byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseDelimitedFrom( + java.io.InputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input, extensionRegistry); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom(akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + parseFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException( + PARSER, input, extensionRegistry); + } + + @java.lang.Override + public Builder newBuilderForType() { + return newBuilder(); + } + + public static Builder newBuilder() { + return DEFAULT_INSTANCE.toBuilder(); + } + + public static Builder newBuilder( + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent prototype) { + return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); + } + + @java.lang.Override + public Builder toBuilder() { + return this == DEFAULT_INSTANCE ? new Builder() : new Builder().mergeFrom(this); + } + + @java.lang.Override + protected Builder newBuilderForType( + akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** Protobuf type {@code PublishedEvent} */ + public static final class Builder + extends akka.protobufv3.internal.GeneratedMessageV3.Builder + implements + // @@protoc_insertion_point(builder_implements:PublishedEvent) + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEventOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_PublishedEvent_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_PublishedEvent_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent.Builder + .class); + } + + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(akka.protobufv3.internal.GeneratedMessageV3.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + + private void maybeForceBuilderInitialization() { + if (akka.protobufv3.internal.GeneratedMessageV3.alwaysUseFieldBuilders) { + getPayloadFieldBuilder(); + getMetadataFieldBuilder(); + } + } + + @java.lang.Override + public Builder clear() { + super.clear(); + persistenceId_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + sequenceNr_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + if (payloadBuilder_ == null) { + payload_ = null; + } else { + payloadBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + timestamp_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); + if (metadataBuilder_ == null) { + metadata_ = null; + } else { + metadataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + @java.lang.Override + public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_PublishedEvent_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + buildPartial() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent result = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) != 0)) { + to_bitField0_ |= 0x00000001; + } + result.persistenceId_ = persistenceId_; + if (((from_bitField0_ & 0x00000002) != 0)) { + result.sequenceNr_ = sequenceNr_; + to_bitField0_ |= 0x00000002; + } + if (((from_bitField0_ & 0x00000004) != 0)) { + if (payloadBuilder_ == null) { + result.payload_ = payload_; + } else { + result.payload_ = payloadBuilder_.build(); + } + to_bitField0_ |= 0x00000004; + } + if (((from_bitField0_ & 0x00000008) != 0)) { + result.timestamp_ = timestamp_; + to_bitField0_ |= 0x00000008; + } + if (((from_bitField0_ & 0x00000010) != 0)) { + if (metadataBuilder_ == null) { + result.metadata_ = metadata_; + } else { + result.metadata_ = metadataBuilder_.build(); + } + to_bitField0_ |= 0x00000010; + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + @java.lang.Override + public Builder clone() { + return super.clone(); + } + + @java.lang.Override + public Builder setField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.setField(field, value); + } + + @java.lang.Override + public Builder clearField(akka.protobufv3.internal.Descriptors.FieldDescriptor field) { + return super.clearField(field); + } + + @java.lang.Override + public Builder clearOneof(akka.protobufv3.internal.Descriptors.OneofDescriptor oneof) { + return super.clearOneof(oneof); + } + + @java.lang.Override + public Builder setRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, + int index, + java.lang.Object value) { + return super.setRepeatedField(field, index, value); + } + + @java.lang.Override + public Builder addRepeatedField( + akka.protobufv3.internal.Descriptors.FieldDescriptor field, java.lang.Object value) { + return super.addRepeatedField(field, value); + } + + @java.lang.Override + public Builder mergeFrom(akka.protobufv3.internal.Message other) { + if (other + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent) { + return mergeFrom( + (akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent other) { + if (other + == akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + .getDefaultInstance()) return this; + if (other.hasPersistenceId()) { + bitField0_ |= 0x00000001; + persistenceId_ = other.persistenceId_; + onChanged(); + } + if (other.hasSequenceNr()) { + setSequenceNr(other.getSequenceNr()); + } + if (other.hasPayload()) { + mergePayload(other.getPayload()); + } + if (other.hasTimestamp()) { + setTimestamp(other.getTimestamp()); + } + if (other.hasMetadata()) { + mergeMetadata(other.getMetadata()); + } + this.mergeUnknownFields(other.unknownFields); + onChanged(); + return this; + } + + @java.lang.Override + public final boolean isInitialized() { + if (hasPayload()) { + if (!getPayload().isInitialized()) { + return false; + } + } + if (hasMetadata()) { + if (!getMetadata().isInitialized()) { + return false; + } + } + return true; + } + + @java.lang.Override + public Builder mergeFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.lang.Object persistenceId_ = ""; + /** + * optional string persistenceId = 1; + * + * @return Whether the persistenceId field is set. + */ + public boolean hasPersistenceId() { + return ((bitField0_ & 0x00000001) != 0); + } + /** + * optional string persistenceId = 1; + * + * @return The persistenceId. + */ + public java.lang.String getPersistenceId() { + java.lang.Object ref = persistenceId_; + if (!(ref instanceof java.lang.String)) { + akka.protobufv3.internal.ByteString bs = (akka.protobufv3.internal.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + persistenceId_ = s; + } + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * optional string persistenceId = 1; + * + * @return The bytes for persistenceId. + */ + public akka.protobufv3.internal.ByteString getPersistenceIdBytes() { + java.lang.Object ref = persistenceId_; + if (ref instanceof String) { + akka.protobufv3.internal.ByteString b = + akka.protobufv3.internal.ByteString.copyFromUtf8((java.lang.String) ref); + persistenceId_ = b; + return b; + } else { + return (akka.protobufv3.internal.ByteString) ref; + } + } + /** + * optional string persistenceId = 1; + * + * @param value The persistenceId to set. + * @return This builder for chaining. + */ + public Builder setPersistenceId(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + persistenceId_ = value; + onChanged(); + return this; + } + /** + * optional string persistenceId = 1; + * + * @return This builder for chaining. + */ + public Builder clearPersistenceId() { + bitField0_ = (bitField0_ & ~0x00000001); + persistenceId_ = getDefaultInstance().getPersistenceId(); + onChanged(); + return this; + } + /** + * optional string persistenceId = 1; + * + * @param value The bytes for persistenceId to set. + * @return This builder for chaining. + */ + public Builder setPersistenceIdBytes(akka.protobufv3.internal.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + persistenceId_ = value; + onChanged(); + return this; + } + + private long sequenceNr_; + /** + * optional int64 sequenceNr = 2; + * + * @return Whether the sequenceNr field is set. + */ + public boolean hasSequenceNr() { + return ((bitField0_ & 0x00000002) != 0); + } + /** + * optional int64 sequenceNr = 2; + * + * @return The sequenceNr. + */ + public long getSequenceNr() { + return sequenceNr_; + } + /** + * optional int64 sequenceNr = 2; + * + * @param value The sequenceNr to set. + * @return This builder for chaining. + */ + public Builder setSequenceNr(long value) { + bitField0_ |= 0x00000002; + sequenceNr_ = value; + onChanged(); + return this; + } + /** + * optional int64 sequenceNr = 2; + * + * @return This builder for chaining. + */ + public Builder clearSequenceNr() { + bitField0_ = (bitField0_ & ~0x00000002); + sequenceNr_ = 0L; + onChanged(); + return this; + } + + private akka.remote.ContainerFormats.Payload payload_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.remote.ContainerFormats.Payload, + akka.remote.ContainerFormats.Payload.Builder, + akka.remote.ContainerFormats.PayloadOrBuilder> + payloadBuilder_; + /** + * optional .Payload payload = 3; + * + * @return Whether the payload field is set. + */ + public boolean hasPayload() { + return ((bitField0_ & 0x00000004) != 0); + } + /** + * optional .Payload payload = 3; + * + * @return The payload. + */ + public akka.remote.ContainerFormats.Payload getPayload() { + if (payloadBuilder_ == null) { + return payload_ == null + ? akka.remote.ContainerFormats.Payload.getDefaultInstance() + : payload_; + } else { + return payloadBuilder_.getMessage(); + } + } + /** optional .Payload payload = 3; */ + public Builder setPayload(akka.remote.ContainerFormats.Payload value) { + if (payloadBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + payload_ = value; + onChanged(); + } else { + payloadBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** optional .Payload payload = 3; */ + public Builder setPayload(akka.remote.ContainerFormats.Payload.Builder builderForValue) { + if (payloadBuilder_ == null) { + payload_ = builderForValue.build(); + onChanged(); + } else { + payloadBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + /** optional .Payload payload = 3; */ + public Builder mergePayload(akka.remote.ContainerFormats.Payload value) { + if (payloadBuilder_ == null) { + if (((bitField0_ & 0x00000004) != 0) + && payload_ != null + && payload_ != akka.remote.ContainerFormats.Payload.getDefaultInstance()) { + payload_ = + akka.remote.ContainerFormats.Payload.newBuilder(payload_) + .mergeFrom(value) + .buildPartial(); + } else { + payload_ = value; + } + onChanged(); + } else { + payloadBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + /** optional .Payload payload = 3; */ + public Builder clearPayload() { + if (payloadBuilder_ == null) { + payload_ = null; + onChanged(); + } else { + payloadBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + /** optional .Payload payload = 3; */ + public akka.remote.ContainerFormats.Payload.Builder getPayloadBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getPayloadFieldBuilder().getBuilder(); + } + /** optional .Payload payload = 3; */ + public akka.remote.ContainerFormats.PayloadOrBuilder getPayloadOrBuilder() { + if (payloadBuilder_ != null) { + return payloadBuilder_.getMessageOrBuilder(); + } else { + return payload_ == null + ? akka.remote.ContainerFormats.Payload.getDefaultInstance() + : payload_; + } + } + /** optional .Payload payload = 3; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.remote.ContainerFormats.Payload, + akka.remote.ContainerFormats.Payload.Builder, + akka.remote.ContainerFormats.PayloadOrBuilder> + getPayloadFieldBuilder() { + if (payloadBuilder_ == null) { + payloadBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.remote.ContainerFormats.Payload, + akka.remote.ContainerFormats.Payload.Builder, + akka.remote.ContainerFormats.PayloadOrBuilder>( + getPayload(), getParentForChildren(), isClean()); + payload_ = null; + } + return payloadBuilder_; + } + + private long timestamp_; + /** + * optional int64 timestamp = 4; + * + * @return Whether the timestamp field is set. + */ + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000008) != 0); + } + /** + * optional int64 timestamp = 4; + * + * @return The timestamp. + */ + public long getTimestamp() { + return timestamp_; + } + /** + * optional int64 timestamp = 4; + * + * @param value The timestamp to set. + * @return This builder for chaining. + */ + public Builder setTimestamp(long value) { + bitField0_ |= 0x00000008; + timestamp_ = value; + onChanged(); + return this; + } + /** + * optional int64 timestamp = 4; + * + * @return This builder for chaining. + */ + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000008); + timestamp_ = 0L; + onChanged(); + return this; + } + + private akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + metadata_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaDataOrBuilder> + metadataBuilder_; + /** + * optional .ReplicatedPublishedEventMetaData metadata = 5; + * + * @return Whether the metadata field is set. + */ + public boolean hasMetadata() { + return ((bitField0_ & 0x00000010) != 0); + } + /** + * optional .ReplicatedPublishedEventMetaData metadata = 5; + * + * @return The metadata. + */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + getMetadata() { + if (metadataBuilder_ == null) { + return metadata_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.getDefaultInstance() + : metadata_; + } else { + return metadataBuilder_.getMessage(); + } + } + /** optional .ReplicatedPublishedEventMetaData metadata = 5; */ + public Builder setMetadata( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + value) { + if (metadataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + metadata_ = value; + onChanged(); + } else { + metadataBuilder_.setMessage(value); + } + bitField0_ |= 0x00000010; + return this; + } + /** optional .ReplicatedPublishedEventMetaData metadata = 5; */ + public Builder setMetadata( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.Builder + builderForValue) { + if (metadataBuilder_ == null) { + metadata_ = builderForValue.build(); + onChanged(); + } else { + metadataBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000010; + return this; + } + /** optional .ReplicatedPublishedEventMetaData metadata = 5; */ + public Builder mergeMetadata( + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData + value) { + if (metadataBuilder_ == null) { + if (((bitField0_ & 0x00000010) != 0) + && metadata_ != null + && metadata_ + != akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.getDefaultInstance()) { + metadata_ = + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.newBuilder(metadata_) + .mergeFrom(value) + .buildPartial(); + } else { + metadata_ = value; + } + onChanged(); + } else { + metadataBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000010; + return this; + } + /** optional .ReplicatedPublishedEventMetaData metadata = 5; */ + public Builder clearMetadata() { + if (metadataBuilder_ == null) { + metadata_ = null; + onChanged(); + } else { + metadataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + /** optional .ReplicatedPublishedEventMetaData metadata = 5; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.Builder + getMetadataBuilder() { + bitField0_ |= 0x00000010; + onChanged(); + return getMetadataFieldBuilder().getBuilder(); + } + /** optional .ReplicatedPublishedEventMetaData metadata = 5; */ + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaDataOrBuilder + getMetadataOrBuilder() { + if (metadataBuilder_ != null) { + return metadataBuilder_.getMessageOrBuilder(); + } else { + return metadata_ == null + ? akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.getDefaultInstance() + : metadata_; + } + } + /** optional .ReplicatedPublishedEventMetaData metadata = 5; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaDataOrBuilder> + getMetadataFieldBuilder() { + if (metadataBuilder_ == null) { + metadataBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaData.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedPublishedEventMetaDataOrBuilder>( + getMetadata(), getParentForChildren(), isClean()); + metadata_ = null; + } + return metadataBuilder_; + } + + @java.lang.Override + public final Builder setUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.setUnknownFields(unknownFields); + } + + @java.lang.Override + public final Builder mergeUnknownFields( + final akka.protobufv3.internal.UnknownFieldSet unknownFields) { + return super.mergeUnknownFields(unknownFields); + } + + // @@protoc_insertion_point(builder_scope:PublishedEvent) + } + + // @@protoc_insertion_point(class_scope:PublishedEvent) + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent(); + } + + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + getDefaultInstance() { + return DEFAULT_INSTANCE; + } + + @java.lang.Deprecated + public static final akka.protobufv3.internal.Parser PARSER = + new akka.protobufv3.internal.AbstractParser() { + @java.lang.Override + public PublishedEvent parsePartialFrom( + akka.protobufv3.internal.CodedInputStream input, + akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return new PublishedEvent(input, extensionRegistry); + } + }; + + public static akka.protobufv3.internal.Parser parser() { + return PARSER; + } + + @java.lang.Override + public akka.protobufv3.internal.Parser getParserForType() { + return PARSER; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ReplicatedEventSourcing.PublishedEvent + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_Counter_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_Counter_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_CounterUpdate_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_CounterUpdate_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_ORSet_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_ORSet_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_ORSetDeltaGroup_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_ORSetDeltaGroup_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_ORSetDeltaGroup_Entry_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_VersionVector_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_VersionVector_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_VersionVector_Entry_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_VersionVector_Entry_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_ReplicatedEventMetadata_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_ReplicatedEventMetadata_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_ReplicatedSnapshotMetadata_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_ReplicatedSnapshotMetadata_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_ReplicatedSnapshotMetadata_Seen_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_ReplicatedSnapshotMetadata_Seen_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_ReplicatedPublishedEventMetaData_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_ReplicatedPublishedEventMetaData_fieldAccessorTable; + private static final akka.protobufv3.internal.Descriptors.Descriptor + internal_static_PublishedEvent_descriptor; + private static final akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internal_static_PublishedEvent_fieldAccessorTable; + + public static akka.protobufv3.internal.Descriptors.FileDescriptor getDescriptor() { + return descriptor; + } + + private static akka.protobufv3.internal.Descriptors.FileDescriptor descriptor; + + static { + java.lang.String[] descriptorData = { + "\n\035ReplicatedEventSourcing.proto\032\026Contain" + + "erFormats.proto\"\030\n\007Counter\022\r\n\005value\030\001 \002(" + + "\014\"\036\n\rCounterUpdate\022\r\n\005delta\030\001 \002(\014\"\304\001\n\005OR" + + "Set\022\020\n\010originDc\030\001 \002(\t\022\037\n\007vvector\030\002 \002(\0132\016" + + ".VersionVector\022\034\n\004dots\030\003 \003(\0132\016.VersionVe" + + "ctor\022\026\n\016stringElements\030\004 \003(\t\022\027\n\013intEleme" + + "nts\030\005 \003(\021B\002\020\001\022\030\n\014longElements\030\006 \003(\022B\002\020\001\022" + + "\037\n\rotherElements\030\007 \003(\0132\010.Payload\"\201\001\n\017ORS" + + "etDeltaGroup\022\'\n\007entries\030\001 \003(\0132\026.ORSetDel" + + "taGroup.Entry\032E\n\005Entry\022 \n\toperation\030\001 \002(" + + "\0162\r.ORSetDeltaOp\022\032\n\nunderlying\030\002 \002(\0132\006.O" + + "RSet\"]\n\rVersionVector\022%\n\007entries\030\001 \003(\0132\024" + + ".VersionVector.Entry\032%\n\005Entry\022\013\n\003key\030\001 \002" + + "(\t\022\017\n\007version\030\002 \002(\003\"\205\001\n\027ReplicatedEventM" + + "etadata\022\025\n\roriginReplica\030\001 \002(\t\022\030\n\020origin" + + "SequenceNr\030\002 \002(\003\022%\n\rversionVector\030\003 \002(\0132" + + "\016.VersionVector\022\022\n\nconcurrent\030\004 \002(\010\"\246\001\n\032" + + "ReplicatedSnapshotMetadata\022\037\n\007version\030\001 " + + "\002(\0132\016.VersionVector\0228\n\016seenPerReplica\030\002 " + + "\003(\0132 .ReplicatedSnapshotMetadata.Seen\032-\n" + + "\004Seen\022\021\n\treplicaId\030\001 \002(\t\022\022\n\nsequenceNr\030\002" + + " \002(\003\"\\\n ReplicatedPublishedEventMetaData" + + "\022\021\n\treplicaId\030\001 \001(\t\022%\n\rversionVector\030\002 \001" + + "(\0132\016.VersionVector\"\236\001\n\016PublishedEvent\022\025\n" + + "\rpersistenceId\030\001 \001(\t\022\022\n\nsequenceNr\030\002 \001(\003" + + "\022\031\n\007payload\030\003 \001(\0132\010.Payload\022\021\n\ttimestamp" + + "\030\004 \001(\003\0223\n\010metadata\030\005 \001(\0132!.ReplicatedPub" + + "lishedEventMetaData*-\n\014ORSetDeltaOp\022\007\n\003A" + + "dd\020\000\022\n\n\006Remove\020\001\022\010\n\004Full\020\002B(\n$akka.persi" + + "stence.typed.serializationH\001" + }; + descriptor = + akka.protobufv3.internal.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom( + descriptorData, + new akka.protobufv3.internal.Descriptors.FileDescriptor[] { + akka.remote.ContainerFormats.getDescriptor(), + }); + internal_static_Counter_descriptor = getDescriptor().getMessageTypes().get(0); + internal_static_Counter_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_Counter_descriptor, + new java.lang.String[] { + "Value", + }); + internal_static_CounterUpdate_descriptor = getDescriptor().getMessageTypes().get(1); + internal_static_CounterUpdate_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_CounterUpdate_descriptor, + new java.lang.String[] { + "Delta", + }); + internal_static_ORSet_descriptor = getDescriptor().getMessageTypes().get(2); + internal_static_ORSet_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_ORSet_descriptor, + new java.lang.String[] { + "OriginDc", + "Vvector", + "Dots", + "StringElements", + "IntElements", + "LongElements", + "OtherElements", + }); + internal_static_ORSetDeltaGroup_descriptor = getDescriptor().getMessageTypes().get(3); + internal_static_ORSetDeltaGroup_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_ORSetDeltaGroup_descriptor, + new java.lang.String[] { + "Entries", + }); + internal_static_ORSetDeltaGroup_Entry_descriptor = + internal_static_ORSetDeltaGroup_descriptor.getNestedTypes().get(0); + internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_ORSetDeltaGroup_Entry_descriptor, + new java.lang.String[] { + "Operation", "Underlying", + }); + internal_static_VersionVector_descriptor = getDescriptor().getMessageTypes().get(4); + internal_static_VersionVector_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_VersionVector_descriptor, + new java.lang.String[] { + "Entries", + }); + internal_static_VersionVector_Entry_descriptor = + internal_static_VersionVector_descriptor.getNestedTypes().get(0); + internal_static_VersionVector_Entry_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_VersionVector_Entry_descriptor, + new java.lang.String[] { + "Key", "Version", + }); + internal_static_ReplicatedEventMetadata_descriptor = getDescriptor().getMessageTypes().get(5); + internal_static_ReplicatedEventMetadata_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_ReplicatedEventMetadata_descriptor, + new java.lang.String[] { + "OriginReplica", "OriginSequenceNr", "VersionVector", "Concurrent", + }); + internal_static_ReplicatedSnapshotMetadata_descriptor = + getDescriptor().getMessageTypes().get(6); + internal_static_ReplicatedSnapshotMetadata_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_ReplicatedSnapshotMetadata_descriptor, + new java.lang.String[] { + "Version", "SeenPerReplica", + }); + internal_static_ReplicatedSnapshotMetadata_Seen_descriptor = + internal_static_ReplicatedSnapshotMetadata_descriptor.getNestedTypes().get(0); + internal_static_ReplicatedSnapshotMetadata_Seen_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_ReplicatedSnapshotMetadata_Seen_descriptor, + new java.lang.String[] { + "ReplicaId", "SequenceNr", + }); + internal_static_ReplicatedPublishedEventMetaData_descriptor = + getDescriptor().getMessageTypes().get(7); + internal_static_ReplicatedPublishedEventMetaData_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_ReplicatedPublishedEventMetaData_descriptor, + new java.lang.String[] { + "ReplicaId", "VersionVector", + }); + internal_static_PublishedEvent_descriptor = getDescriptor().getMessageTypes().get(8); + internal_static_PublishedEvent_fieldAccessorTable = + new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( + internal_static_PublishedEvent_descriptor, + new java.lang.String[] { + "PersistenceId", "SequenceNr", "Payload", "Timestamp", "Metadata", + }); + akka.remote.ContainerFormats.getDescriptor(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/akka-persistence-typed/src/main/mima-filters/2.6.8.backwards.excludes/29217-replicated-event-sourcing.excludes b/akka-persistence-typed/src/main/mima-filters/2.6.8.backwards.excludes/29217-replicated-event-sourcing.excludes new file mode 100644 index 0000000000..c0cfd45bdd --- /dev/null +++ b/akka-persistence-typed/src/main/mima-filters/2.6.8.backwards.excludes/29217-replicated-event-sourcing.excludes @@ -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") diff --git a/akka-persistence-typed/src/main/protobuf/ReplicatedEventSourcing.proto b/akka-persistence-typed/src/main/protobuf/ReplicatedEventSourcing.proto new file mode 100644 index 0000000000..a5f8e734ef --- /dev/null +++ b/akka-persistence-typed/src/main/protobuf/ReplicatedEventSourcing.proto @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2017-2020 Lightbend Inc. + */ + +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; +} \ No newline at end of file diff --git a/akka-persistence-typed/src/main/resources/reference.conf b/akka-persistence-typed/src/main/resources/reference.conf index 7361c8113f..afc0976127 100644 --- a/akka-persistence-typed/src/main/resources/reference.conf +++ b/akka-persistence-typed/src/main/resources/reference.conf @@ -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 { diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/PersistenceId.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/PersistenceId.scala index 28b9321339..0f694fe4f8 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/PersistenceId.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/PersistenceId.scala @@ -124,7 +124,6 @@ object PersistenceId { */ def ofUniqueId(id: String): PersistenceId = new PersistenceId(id) - } /** diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/PublishedEvent.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/PublishedEvent.scala new file mode 100644 index 0000000000..1bdd6889b1 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/PublishedEvent.scala @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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] +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicaId.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicaId.scala new file mode 100644 index 0000000000..a842bf1f35 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicaId.scala @@ -0,0 +1,10 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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) diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicationId.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicationId.scala new file mode 100644 index 0000000000..ed10be3dee --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicationId.scala @@ -0,0 +1,51 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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) + } +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/Counter.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/Counter.scala new file mode 100644 index 0000000000..06ea24ccd5 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/Counter.scala @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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) +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/LwwTime.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/LwwTime.scala new file mode 100644 index 0000000000..5e34438e4b --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/LwwTime.scala @@ -0,0 +1,36 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 + } +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/ORSet.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/ORSet.scala new file mode 100644 index 0000000000..b8e5f1ffc0 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/ORSet.scala @@ -0,0 +1,501 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 comprehensive study of Convergent and Commutative Replicated Data Types. + * This is more space efficient and doesn't accumulate garbage for removed elements. + * It is described in the paper + * An optimized conflict-free replicated set + * The implementation is inspired by the Riak DT + * riak_dt_orswot. + * + * 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 + } +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/OpCrdt.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/OpCrdt.scala new file mode 100644 index 0000000000..bcd3b92157 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/OpCrdt.scala @@ -0,0 +1,15 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +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 +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala index 1ff5521cfd..fb745a89a8 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala @@ -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 = { diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala index e989af5bca..ca050239c4 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala @@ -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 } diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ExternalInteractions.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ExternalInteractions.scala index 4b8298f366..5f807e9694 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ExternalInteractions.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ExternalInteractions.scala @@ -5,7 +5,6 @@ package akka.persistence.typed.internal import scala.collection.immutable - import akka.actor.ActorRef import akka.actor.typed.Behavior import akka.actor.typed.PostStop @@ -19,40 +18,60 @@ import akka.annotation.InternalStableApi import akka.persistence._ import akka.persistence.JournalProtocol.ReplayMessages import akka.persistence.SnapshotProtocol.LoadSnapshot -import akka.util.unused +import akka.util.{ unused, OptionVal } + +/** INTERNAL API */ +@InternalApi +private[akka] object JournalInteractions { + + type EventOrTaggedOrReplicated = Any // `Any` since can be `E` or `Tagged` or a `ReplicatedEvent` + + final case class EventToPersist( + adaptedEvent: EventOrTaggedOrReplicated, + manifest: String, + metadata: Option[ReplicatedEventMetadata]) + +} /** INTERNAL API */ @InternalApi private[akka] trait JournalInteractions[C, E, S] { - def setup: BehaviorSetup[C, E, S] + import JournalInteractions._ - type EventOrTagged = Any // `Any` since can be `E` or `Tagged` + def setup: BehaviorSetup[C, E, S] protected def internalPersist( ctx: ActorContext[_], cmd: Any, state: Running.RunningState[S], - event: EventOrTagged, - eventAdapterManifest: String): Running.RunningState[S] = { + event: EventOrTaggedOrReplicated, + eventAdapterManifest: String, + metadata: OptionVal[Any]): Running.RunningState[S] = { - val newState = state.nextSequenceNr() + val newRunningState = state.nextSequenceNr() val repr = PersistentRepr( event, persistenceId = setup.persistenceId.id, - sequenceNr = newState.seqNr, + sequenceNr = newRunningState.seqNr, manifest = eventAdapterManifest, writerUuid = setup.writerIdentity.writerUuid, sender = ActorRef.noSender) + // FIXME check cinnamon is okay with this being null + // https://github.com/akka/akka/issues/29262 onWriteInitiated(ctx, cmd, repr) - val write = AtomicWrite(repr) :: Nil + val write = AtomicWrite(metadata match { + case OptionVal.Some(meta) => repr.withMetadata(meta) + case OptionVal.None => repr + }) :: Nil + setup.journal .tell(JournalProtocol.WriteMessages(write, setup.selfClassic, setup.writerIdentity.instanceId), setup.selfClassic) - newState + newRunningState } @InternalStableApi @@ -65,20 +84,24 @@ private[akka] trait JournalInteractions[C, E, S] { ctx: ActorContext[_], cmd: Any, state: Running.RunningState[S], - events: immutable.Seq[(EventOrTagged, String)]): Running.RunningState[S] = { + events: immutable.Seq[EventToPersist]): Running.RunningState[S] = { if (events.nonEmpty) { var newState = state val writes = events.map { - case (event, eventAdapterManifest) => + case EventToPersist(event, eventAdapterManifest, metadata) => newState = newState.nextSequenceNr() - PersistentRepr( + val repr = PersistentRepr( event, persistenceId = setup.persistenceId.id, sequenceNr = newState.seqNr, manifest = eventAdapterManifest, writerUuid = setup.writerIdentity.writerUuid, sender = ActorRef.noSender) + metadata match { + case Some(metadata) => repr.withMetadata(metadata) + case None => repr + } } onWritesInitiated(ctx, cmd, writes) @@ -167,12 +190,19 @@ private[akka] trait SnapshotInteractions[C, E, S] { setup.log.debug("Saving snapshot sequenceNr [{}]", state.seqNr) if (state.state == null) throw new IllegalStateException("A snapshot must not be a null state.") - else + else { + val meta = setup.replication match { + case Some(_) => + val m = ReplicatedSnapshotMetadata(state.version, state.seenPerReplica) + Some(m) + case None => None + } setup.snapshotStore.tell( SnapshotProtocol.SaveSnapshot( - SnapshotMetadata(setup.persistenceId.id, state.seqNr), + new SnapshotMetadata(setup.persistenceId.id, state.seqNr, meta), setup.snapshotAdapter.toJournal(state.state)), setup.selfClassic) + } } /** Deletes the snapshots up to and including the `sequenceNr`. */ diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/FastForwardingFilter.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/FastForwardingFilter.scala new file mode 100644 index 0000000000..a607f0b907 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/FastForwardingFilter.scala @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed.internal + +import akka.annotation.InternalApi +import akka.persistence.query.EventEnvelope +import akka.stream.Attributes +import akka.stream.FlowShape +import akka.stream.Inlet +import akka.stream.Outlet +import akka.stream.stage.GraphStageLogic +import akka.stream.stage.GraphStageWithMaterializedValue +import akka.stream.stage.InHandler +import akka.stream.stage.OutHandler + +/** + * INTERNAL API + */ +@InternalApi +private[akka] trait ReplicationStreamControl { + def fastForward(sequenceNumber: Long): Unit +} + +/** + * INTERNAL API + */ +@InternalApi +private[akka] class FastForwardingFilter + extends GraphStageWithMaterializedValue[FlowShape[EventEnvelope, EventEnvelope], ReplicationStreamControl] { + + val in = Inlet[EventEnvelope]("FastForwardingFilter.in") + val out = Outlet[EventEnvelope]("FastForwardingFilter.out") + + override val shape = FlowShape[EventEnvelope, EventEnvelope](in, out) + + override def createLogicAndMaterializedValue( + inheritedAttributes: Attributes): (GraphStageLogic, ReplicationStreamControl) = { + var replicationStreamControl: ReplicationStreamControl = null + val logic = new GraphStageLogic(shape) with InHandler with OutHandler { + // -1 means not currently fast forwarding + @volatile private var fastForwardTo = -1L + + override def onPush(): Unit = { + val eventEnvelope = grab(in) + if (fastForwardTo == -1L) + push(out, eventEnvelope) + else { + if (eventEnvelope.sequenceNr <= fastForwardTo) pull(in) + else { + fastForwardTo = -1L + push(out, eventEnvelope) + } + } + } + override def onPull(): Unit = pull(in) + + replicationStreamControl = new ReplicationStreamControl { + override def fastForward(sequenceNumber: Long): Unit = { + require(sequenceNumber > 0) // only the stage may complete a fast forward + fastForwardTo = sequenceNumber + } + } + + setHandlers(in, out, this) + } + + (logic, replicationStreamControl) + } + +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala index c61da7ce27..a9d62c7ba6 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala @@ -6,7 +6,6 @@ package akka.persistence.typed.internal import scala.concurrent.duration._ import scala.util.control.NonFatal - import akka.actor.typed.{ Behavior, Signal } import akka.actor.typed.internal.PoisonPill import akka.actor.typed.internal.UnstashException @@ -19,8 +18,9 @@ import akka.persistence.typed.EmptyEventSeq import akka.persistence.typed.EventsSeq import akka.persistence.typed.RecoveryCompleted import akka.persistence.typed.RecoveryFailed +import akka.persistence.typed.ReplicaId import akka.persistence.typed.SingleEventSeq -import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState +import akka.persistence.typed.internal.EventSourcedBehaviorImpl.{ GetSeenSequenceNr, GetState } import akka.persistence.typed.internal.ReplayingEvents.ReplayingState import akka.persistence.typed.internal.Running.WithSeqNrAccessible import akka.util.OptionVal @@ -51,7 +51,9 @@ private[akka] object ReplayingEvents { eventSeenInInterval: Boolean, toSeqNr: Long, receivedPoisonPill: Boolean, - recoveryStartTime: Long) + recoveryStartTime: Long, + version: VersionVector, + seenSeqNrPerReplica: Map[ReplicaId, Long]) def apply[C, E, S](setup: BehaviorSetup[C, E, S], state: ReplayingState[S]): Behavior[InternalProtocol] = Behaviors.setup { _ => @@ -88,12 +90,15 @@ private[akka] final class ReplayingEvents[C, E, S]( override def onMessage(msg: InternalProtocol): Behavior[InternalProtocol] = { msg match { - case JournalResponse(r) => onJournalResponse(r) - case SnapshotterResponse(r) => onSnapshotterResponse(r) - case RecoveryTickEvent(snap) => onRecoveryTick(snap) - case cmd: IncomingCommand[C] => onCommand(cmd) - case get: GetState[S @unchecked] => stashInternal(get) - case RecoveryPermitGranted => Behaviors.unhandled // should not happen, we already have the permit + case JournalResponse(r) => onJournalResponse(r) + case SnapshotterResponse(r) => onSnapshotterResponse(r) + case RecoveryTickEvent(snap) => onRecoveryTick(snap) + case evt: ReplicatedEventEnvelope[E] => onInternalCommand(evt) + case pe: PublishedEventImpl => onInternalCommand(pe) + case cmd: IncomingCommand[C] => onInternalCommand(cmd) + case get: GetState[S @unchecked] => stashInternal(get) + case get: GetSeenSequenceNr => stashInternal(get) + case RecoveryPermitGranted => Behaviors.unhandled // should not happen, we already have the permit } } @@ -117,7 +122,45 @@ private[akka] final class ReplayingEvents[C, E, S]( def handleEvent(event: E): Unit = { eventForErrorReporting = OptionVal.Some(event) state = state.copy(seqNr = repr.sequenceNr) - state = state.copy(state = setup.eventHandler(state.state, event), eventSeenInInterval = true) + + val replicatedMetaAndSelfReplica: Option[(ReplicatedEventMetadata, ReplicaId, ReplicationSetup)] = + setup.replication match { + case Some(replication) => + val meta = repr.metadata match { + case Some(m) => m.asInstanceOf[ReplicatedEventMetadata] + case None => + throw new IllegalStateException( + s"Replicated Event Sourcing enabled but existing event has no metadata. Migration isn't supported yet.") + + } + replication.setContext(recoveryRunning = true, meta.originReplica, meta.concurrent) + Some((meta, replication.replicaId, replication)) + case None => None + } + + val newState = setup.eventHandler(state.state, event) + + setup.replication match { + case Some(replication) => + replication.clearContext() + case None => + } + + replicatedMetaAndSelfReplica match { + case Some((meta, selfReplica, replication)) if meta.originReplica != selfReplica => + // keep track of highest origin seqnr per other replica + state = state.copy( + state = newState, + eventSeenInInterval = true, + version = meta.version, + seenSeqNrPerReplica = state.seenSeqNrPerReplica + (meta.originReplica -> meta.originSequenceNr)) + replication.clearContext() + case Some((_, _, replication)) => + replication.clearContext() + state = state.copy(state = newState, eventSeenInInterval = true) + case _ => + state = state.copy(state = newState, eventSeenInInterval = true) + } } eventSeq match { @@ -154,7 +197,7 @@ private[akka] final class ReplayingEvents[C, E, S]( } } - private def onCommand(cmd: InternalProtocol): Behavior[InternalProtocol] = { + private def onInternalCommand(cmd: InternalProtocol): Behavior[InternalProtocol] = { // during recovery, stash all incoming commands if (state.receivedPoisonPill) { if (setup.settings.logOnStashing) @@ -237,7 +280,15 @@ private[akka] final class ReplayingEvents[C, E, S]( Behaviors.stopped else { val running = - Running[C, E, S](setup, Running.RunningState[S](state.seqNr, state.state, state.receivedPoisonPill)) + Running[C, E, S]( + setup, + Running.RunningState[S]( + seqNr = state.seqNr, + state = state.state, + receivedPoisonPill = state.receivedPoisonPill, + state.version, + seenPerReplica = state.seenSeqNrPerReplica, + replicationControl = Map.empty)) tryUnstashOne(running) } diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingSnapshot.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingSnapshot.scala index e71ffec70b..4257995eb1 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingSnapshot.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingSnapshot.scala @@ -11,9 +11,10 @@ import akka.annotation.{ InternalApi, InternalStableApi } import akka.persistence._ import akka.persistence.SnapshotProtocol.LoadSnapshotFailed import akka.persistence.SnapshotProtocol.LoadSnapshotResult -import akka.persistence.typed.RecoveryFailed -import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState +import akka.persistence.typed.{ RecoveryFailed, ReplicaId } +import akka.persistence.typed.internal.EventSourcedBehaviorImpl.{ GetSeenSequenceNr, GetState } import akka.util.unused +import akka.actor.typed.scaladsl.LoggerOps /** * INTERNAL API @@ -57,9 +58,11 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup def stay(receivedPoisonPill: Boolean): Behavior[InternalProtocol] = { Behaviors .receiveMessage[InternalProtocol] { - case SnapshotterResponse(r) => onSnapshotterResponse(r, receivedPoisonPill) - case JournalResponse(r) => onJournalResponse(r) - case RecoveryTickEvent(snapshot) => onRecoveryTick(snapshot) + case SnapshotterResponse(r) => onSnapshotterResponse(r, receivedPoisonPill) + case JournalResponse(r) => onJournalResponse(r) + case RecoveryTickEvent(snapshot) => onRecoveryTick(snapshot) + case evt: ReplicatedEventEnvelope[E] => onReplicatedEvent(evt) + case pe: PublishedEventImpl => onPublishedEvent(pe) case cmd: IncomingCommand[C] => if (receivedPoisonPill) { if (setup.settings.logOnStashing) @@ -68,6 +71,7 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup } else onCommand(cmd) case get: GetState[S @unchecked] => stashInternal(get) + case get: GetSeenSequenceNr => stashInternal(get) case RecoveryPermitGranted => Behaviors.unhandled // should not happen, we already have the permit } .receiveSignal(returnPermitOnStop.orElse { @@ -122,6 +126,14 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup stashInternal(cmd) } + def onReplicatedEvent(evt: InternalProtocol.ReplicatedEventEnvelope[E]): Behavior[InternalProtocol] = { + stashInternal(evt) + } + + def onPublishedEvent(event: PublishedEventImpl): Behavior[InternalProtocol] = { + stashInternal(event) + } + def onJournalResponse(response: JournalProtocol.Response): Behavior[InternalProtocol] = { setup.log.debug( "Unexpected response from journal: [{}], may be due to an actor restart, ignoring...", @@ -136,14 +148,32 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup case LoadSnapshotResult(sso, toSnr) => var state: S = setup.emptyState - val seqNr: Long = sso match { + val (seqNr: Long, seenPerReplica, version) = sso match { case Some(SelectedSnapshot(metadata, snapshot)) => state = setup.snapshotAdapter.fromJournal(snapshot) - metadata.sequenceNr - case None => 0 // from the beginning please + setup.context.log.debug("Loaded snapshot with metadata [{}]", metadata) + metadata.metadata match { + case Some(rm: ReplicatedSnapshotMetadata) => (metadata.sequenceNr, rm.seenPerReplica, rm.version) + case _ => (metadata.sequenceNr, Map.empty[ReplicaId, Long].withDefaultValue(0L), VersionVector.empty) + } + case None => (0L, Map.empty[ReplicaId, Long].withDefaultValue(0L), VersionVector.empty) } - becomeReplayingEvents(state, seqNr, toSnr, receivedPoisonPill) + setup.context.log.debugN("Snapshot recovered from {} {} {}", seqNr, seenPerReplica, version) + + setup.cancelRecoveryTimer() + + ReplayingEvents[C, E, S]( + setup, + ReplayingEvents.ReplayingState( + seqNr, + state, + eventSeenInInterval = false, + toSnr, + receivedPoisonPill, + System.nanoTime(), + version, + seenPerReplica)) case LoadSnapshotFailed(cause) => onRecoveryFailure(cause) @@ -153,22 +183,4 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup } } - private def becomeReplayingEvents( - state: S, - lastSequenceNr: Long, - toSnr: Long, - receivedPoisonPill: Boolean): Behavior[InternalProtocol] = { - setup.cancelRecoveryTimer() - - ReplayingEvents[C, E, S]( - setup, - ReplayingEvents.ReplayingState( - lastSequenceNr, - state, - eventSeenInInterval = false, - toSnr, - receivedPoisonPill, - System.nanoTime())) - } - } diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplicationSetup.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplicationSetup.scala new file mode 100644 index 0000000000..a118c27dea --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplicationSetup.scala @@ -0,0 +1,107 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed.internal + +import akka.annotation.InternalApi +import akka.persistence.typed.PersistenceId +import akka.persistence.typed.ReplicaId +import akka.persistence.typed.ReplicationId +import akka.util.OptionVal +import akka.util.WallClock +import akka.util.ccompat.JavaConverters._ + +/** + * INTERNAL API + */ +@InternalApi +private[akka] final class ReplicationContextImpl( + val replicationId: ReplicationId, + val replicasAndQueryPlugins: Map[ReplicaId, String]) + extends akka.persistence.typed.scaladsl.ReplicationContext + with akka.persistence.typed.javadsl.ReplicationContext { + val allReplicas: Set[ReplicaId] = replicasAndQueryPlugins.keySet + // these are not volatile as they are set on the same thread as they should be accessed + var _currentThread: OptionVal[Thread] = OptionVal.None + var _origin: OptionVal[ReplicaId] = OptionVal.None + var _recoveryRunning: Boolean = false + var _concurrent: Boolean = false + + private def checkAccess(functionName: String): Unit = { + val callerThread = Thread.currentThread() + def error() = + throw new UnsupportedOperationException( + s"Unsupported access to ReplicationContext operation from the outside of event handler. " + + s"$functionName can only be called from the event handler") + _currentThread match { + case OptionVal.Some(t) => + if (callerThread ne t) error() + case OptionVal.None => + error() + } + } + + /** + * The origin of the current event. + * Undefined result if called from anywhere other than an event handler. + */ + override def origin: ReplicaId = { + checkAccess("origin") + _origin match { + case OptionVal.Some(origin) => origin + case OptionVal.None => throw new IllegalStateException("origin can only be accessed from the event handler") + } + } + + /** + * Whether the happened concurrently with an event from another replica. + * Undefined result if called from any where other than an event handler. + */ + override def concurrent: Boolean = { + checkAccess("concurrent") + _concurrent + } + + override def persistenceId: PersistenceId = replicationId.persistenceId + + override def currentTimeMillis(): Long = { + WallClock.AlwaysIncreasingClock.currentTimeMillis() + } + override def recoveryRunning: Boolean = { + checkAccess("recoveryRunning") + _recoveryRunning + } + + override def getAllReplicas: java.util.Set[ReplicaId] = allReplicas.asJava +} + +/** + * INTERNAL API + */ +@InternalApi +private[akka] final case class ReplicationSetup( + replicaId: ReplicaId, + allReplicasAndQueryPlugins: Map[ReplicaId, String], + replicationContext: ReplicationContextImpl) { + + val allReplicas: Set[ReplicaId] = allReplicasAndQueryPlugins.keySet + + /** + * Must only be called on the same thread that will execute the user code + */ + def setContext(recoveryRunning: Boolean, originReplica: ReplicaId, concurrent: Boolean): Unit = { + replicationContext._currentThread = OptionVal.Some(Thread.currentThread()) + replicationContext._recoveryRunning = recoveryRunning + replicationContext._concurrent = concurrent + replicationContext._origin = OptionVal.Some(originReplica) + } + + def clearContext(): Unit = { + replicationContext._currentThread = OptionVal.None + replicationContext._recoveryRunning = false + replicationContext._concurrent = false + replicationContext._origin = OptionVal.None + } + +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/Running.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/Running.scala index e9456f9795..9dc8567f50 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/Running.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/Running.scala @@ -4,15 +4,21 @@ package akka.persistence.typed.internal +import java.time.Instant +import java.time.LocalDateTime +import java.time.ZoneId +import java.time.format.DateTimeFormatter +import java.util.concurrent.atomic.AtomicReference + import scala.annotation.tailrec import scala.collection.immutable - import akka.actor.UnhandledMessage -import akka.actor.typed.Behavior -import akka.actor.typed.Signal +import akka.actor.typed.eventstream.EventStream +import akka.actor.typed.{ Behavior, Signal } import akka.actor.typed.internal.PoisonPill import akka.actor.typed.scaladsl.{ AbstractBehavior, ActorContext, Behaviors, LoggerOps } import akka.annotation.{ InternalApi, InternalStableApi } +import akka.event.Logging import akka.persistence.DeleteMessagesFailure import akka.persistence.DeleteMessagesSuccess import akka.persistence.DeleteSnapshotFailure @@ -26,20 +32,37 @@ import akka.persistence.SaveSnapshotFailure import akka.persistence.SaveSnapshotSuccess import akka.persistence.SnapshotProtocol import akka.persistence.journal.Tagged -import akka.persistence.typed.DeleteEventsCompleted -import akka.persistence.typed.DeleteEventsFailed -import akka.persistence.typed.DeleteSnapshotsCompleted -import akka.persistence.typed.DeleteSnapshotsFailed -import akka.persistence.typed.DeletionTarget -import akka.persistence.typed.EventRejectedException -import akka.persistence.typed.SnapshotCompleted -import akka.persistence.typed.SnapshotFailed -import akka.persistence.typed.SnapshotMetadata -import akka.persistence.typed.SnapshotSelectionCriteria -import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState +import akka.persistence.query.{ EventEnvelope, PersistenceQuery } +import akka.persistence.query.scaladsl.EventsByPersistenceIdQuery +import akka.persistence.typed.ReplicaId +import akka.persistence.typed.ReplicationId +import akka.persistence.typed.{ + DeleteEventsCompleted, + DeleteEventsFailed, + DeleteSnapshotsCompleted, + DeleteSnapshotsFailed, + DeletionTarget, + EventRejectedException, + PersistenceId, + SnapshotCompleted, + SnapshotFailed, + SnapshotMetadata, + SnapshotSelectionCriteria +} +import akka.persistence.typed.internal.EventSourcedBehaviorImpl.{ GetSeenSequenceNr, GetState } +import akka.persistence.typed.internal.InternalProtocol.ReplicatedEventEnvelope +import akka.persistence.typed.internal.JournalInteractions.EventToPersist import akka.persistence.typed.internal.Running.WithSeqNrAccessible import akka.persistence.typed.scaladsl.Effect +import akka.stream.scaladsl.Keep +import akka.stream.SystemMaterializer +import akka.stream.WatchedActorTerminatedException +import akka.stream.scaladsl.Source +import akka.stream.scaladsl.{ RestartSource, Sink } +import akka.stream.typed.scaladsl.ActorFlow +import akka.util.OptionVal import akka.util.unused +import akka.util.Timeout /** * INTERNAL API @@ -66,7 +89,13 @@ private[akka] object Running { def currentSequenceNumber: Long } - final case class RunningState[State](seqNr: Long, state: State, receivedPoisonPill: Boolean) { + final case class RunningState[State]( + seqNr: Long, + state: State, + receivedPoisonPill: Boolean, + version: VersionVector, + seenPerReplica: Map[ReplicaId, Long], + replicationControl: Map[ReplicaId, ReplicationStreamControl]) { def nextSequenceNr(): RunningState[State] = copy(seqNr = seqNr + 1) @@ -82,7 +111,114 @@ private[akka] object Running { def apply[C, E, S](setup: BehaviorSetup[C, E, S], state: RunningState[S]): Behavior[InternalProtocol] = { val running = new Running(setup.setMdcPhase(PersistenceMdc.RunningCmds)) - new running.HandlingCommands(state) + val initialState = setup.replication match { + case Some(replication) => startReplicationStream(setup, state, replication) + case None => state + } + new running.HandlingCommands(initialState) + } + + def startReplicationStream[C, E, S]( + setup: BehaviorSetup[C, E, S], + state: RunningState[S], + replicationSetup: ReplicationSetup): RunningState[S] = { + import scala.concurrent.duration._ + val system = setup.context.system + val ref = setup.context.self + + val query = PersistenceQuery(system) + replicationSetup.allReplicas.foldLeft(state) { (state, replicaId) => + if (replicaId != replicationSetup.replicaId) { + val pid = ReplicationId( + replicationSetup.replicationContext.replicationId.typeName, + replicationSetup.replicationContext.entityId, + replicaId) + val queryPluginId = replicationSetup.allReplicasAndQueryPlugins(replicaId) + val replication = query.readJournalFor[EventsByPersistenceIdQuery](queryPluginId) + + implicit val timeout = Timeout(30.seconds) + implicit val scheduler = setup.context.system.scheduler + implicit val ec = setup.context.system.executionContext + + val controlRef = new AtomicReference[ReplicationStreamControl]() + + import akka.actor.typed.scaladsl.AskPattern._ + val source = RestartSource + .withBackoff(2.seconds, 10.seconds, randomFactor = 0.2) { () => + Source.futureSource { + setup.context.self.ask[Long](replyTo => GetSeenSequenceNr(replicaId, replyTo)).map { seqNr => + replication + .eventsByPersistenceId(pid.persistenceId.id, seqNr + 1, Long.MaxValue) + // from each replica, only get the events that originated there, this prevents most of the event filtering + // the downside is that events can't be received via other replicas in the event of an uneven network partition + .filter(event => + event.eventMetadata match { + case Some(replicatedMeta: ReplicatedEventMetadata) => replicatedMeta.originReplica == replicaId + case _ => + throw new IllegalArgumentException( + s"Replication stream from replica ${replicaId} for ${setup.persistenceId} contains event " + + s"(sequence nr ${event.sequenceNr}) without replication metadata. " + + s"Is the persistence id used by a regular event sourced actor there or the journal for that replica (${queryPluginId}) " + + "used that does not support Replicated Event Sourcing?") + }) + .viaMat(new FastForwardingFilter)(Keep.right) + .mapMaterializedValue(streamControl => controlRef.set(streamControl)) + } + } + } + // needs to be outside of the restart source so that it actually cancels when terminating the replica + .via(ActorFlow + .ask[EventEnvelope, ReplicatedEventEnvelope[E], ReplicatedEventAck.type](ref) { (eventEnvelope, replyTo) => + // Need to handle this not being available migration from non-replicated is supported + val meta = eventEnvelope.eventMetadata.get.asInstanceOf[ReplicatedEventMetadata] + val re = + ReplicatedEvent[E]( + eventEnvelope.event.asInstanceOf[E], + meta.originReplica, + meta.originSequenceNr, + meta.version) + ReplicatedEventEnvelope(re, replyTo) + } + .recoverWithRetries(1, { + // not a failure, the replica is stopping, complete the stream + case _: WatchedActorTerminatedException => + Source.empty + })) + + source.runWith(Sink.ignore)(SystemMaterializer(system).materializer) + + // TODO support from journal to fast forward https://github.com/akka/akka/issues/29311 + state.copy( + replicationControl = + state.replicationControl.updated(replicaId, new ReplicationStreamControl { + override def fastForward(sequenceNumber: Long): Unit = { + // (logging is safe here since invoked on message receive + OptionVal(controlRef.get) match { + case OptionVal.Some(control) => + if (setup.log.isDebugEnabled) + setup.log.debug("Fast forward replica [{}] to [{}]", replicaId, sequenceNumber) + control.fastForward(sequenceNumber) + case OptionVal.None => + // stream not started yet, ok, fast forward is an optimization + if (setup.log.isDebugEnabled) + setup.log.debug( + "Ignoring fast forward replica [{}] to [{}], stream not started yet", + replicaId, + sequenceNumber) + } + } + })) + } else { + state + } + } + } + + private val timestampFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS") + private val UTC = ZoneId.of("UTC") + + def formatTimestamp(time: Long): String = { + timestampFormatter.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(time), UTC)) } } @@ -96,6 +232,7 @@ private[akka] object Running { import BehaviorSetup._ import InternalProtocol._ import Running.RunningState + import Running.formatTimestamp // Needed for WithSeqNrAccessible, when unstashing private var _currentSequenceNumber = 0L @@ -106,12 +243,19 @@ private[akka] object Running { _currentSequenceNumber = state.seqNr + private def alreadySeen(e: ReplicatedEvent[_]): Boolean = { + e.originSequenceNr <= state.seenPerReplica.getOrElse(e.originReplica, 0L) + } + def onMessage(msg: InternalProtocol): Behavior[InternalProtocol] = msg match { - case IncomingCommand(c: C @unchecked) => onCommand(state, c) - case JournalResponse(r) => onDeleteEventsJournalResponse(r, state.state) - case SnapshotterResponse(r) => onDeleteSnapshotResponse(r, state.state) - case get: GetState[S @unchecked] => onGetState(get) - case _ => Behaviors.unhandled + case IncomingCommand(c: C @unchecked) => onCommand(state, c) + case re: ReplicatedEventEnvelope[E @unchecked] => onReplicatedEvent(state, re, setup.replication.get) + case pe: PublishedEventImpl => onPublishedEvent(state, pe) + case JournalResponse(r) => onDeleteEventsJournalResponse(r, state.state) + case SnapshotterResponse(r) => onDeleteSnapshotResponse(r, state.state) + case get: GetState[S @unchecked] => onGetState(get) + case get: GetSeenSequenceNr => onGetSeenSequenceNr(get) + case _ => Behaviors.unhandled } override def onSignal: PartialFunction[Signal, Behavior[InternalProtocol]] = { @@ -130,12 +274,303 @@ private[akka] object Running { else next } + def onReplicatedEvent( + state: Running.RunningState[S], + envelope: ReplicatedEventEnvelope[E], + replication: ReplicationSetup): Behavior[InternalProtocol] = { + setup.log.infoN( + "Replica {} received replicated event. Replica seqs nrs: {}. Envelope {}", + setup.replication, + state.seenPerReplica, + envelope) + envelope.ack ! ReplicatedEventAck + if (envelope.event.originReplica != replication.replicaId && !alreadySeen(envelope.event)) { + setup.log.debug( + "Saving event [{}] from [{}] as first time", + envelope.event.originSequenceNr, + envelope.event.originReplica) + handleExternalReplicatedEventPersist(replication, envelope.event) + } else { + setup.log.debug( + "Filtering event [{}] from [{}] as it was already seen", + envelope.event.originSequenceNr, + envelope.event.originReplica) + tryUnstashOne(this) + } + } + + def onPublishedEvent(state: Running.RunningState[S], event: PublishedEventImpl): Behavior[InternalProtocol] = { + val newBehavior: Behavior[InternalProtocol] = setup.replication match { + case None => + setup.log.warn( + "Received published event for [{}] but not an Replicated Event Sourcing actor, dropping", + event.persistenceId) + this + + case Some(replication) => + event.replicatedMetaData match { + case None => + setup.log.warn("Received published event for [{}] but with no replicated metadata, dropping") + this + case Some(replicatedEventMetaData) => + onPublishedEvent(state, replication, replicatedEventMetaData, event) + } + } + tryUnstashOne(newBehavior) + } + + private def onPublishedEvent( + state: Running.RunningState[S], + replication: ReplicationSetup, + replicatedMetadata: ReplicatedPublishedEventMetaData, + event: PublishedEventImpl): Behavior[InternalProtocol] = { + val log = setup.log + val separatorIndex = event.persistenceId.id.indexOf(PersistenceId.DefaultSeparator) + val idPrefix = event.persistenceId.id.substring(0, separatorIndex) + val originReplicaId = replicatedMetadata.replicaId + if (!setup.persistenceId.id.startsWith(idPrefix)) { + log.warn("Ignoring published replicated event for the wrong actor [{}]", event.persistenceId) + this + } else if (originReplicaId == replication.replicaId) { + if (log.isDebugEnabled) + log.debug( + "Ignoring published replicated event with seqNr [{}] from our own replica id [{}]", + event.sequenceNumber, + originReplicaId) + this + } else if (!replication.allReplicas.contains(originReplicaId)) { + log.warnN( + "Received published replicated event from replica [{}], which is unknown. Replicated Event Sourcing must be set up with a list of all replicas (known are [{}]).", + originReplicaId, + replication.allReplicas.mkString(", ")) + this + } else { + val expectedSequenceNumber = state.seenPerReplica(originReplicaId) + 1 + if (expectedSequenceNumber > event.sequenceNumber) { + // already seen + if (log.isDebugEnabled) + log.debugN( + "Ignoring published replicated event with seqNr [{}] from replica [{}] because it was already seen ([{}])", + event.sequenceNumber, + originReplicaId, + expectedSequenceNumber) + this + } else if (expectedSequenceNumber != event.sequenceNumber) { + // gap in sequence numbers (message lost or query and direct replication out of sync, should heal up by itself + // once the query catches up) + if (log.isDebugEnabled) { + log.debugN( + "Ignoring published replicated event with replication seqNr [{}] from replica [{}] " + + "because expected replication seqNr was [{}] ", + event.sequenceNumber, + originReplicaId, + expectedSequenceNumber) + } + this + } else { + if (log.isTraceEnabled) { + log.traceN( + "Received published replicated event [{}] with timestamp [{} (UTC)] from replica [{}] seqNr [{}]", + Logging.simpleName(event.event.getClass), + formatTimestamp(event.timestamp), + originReplicaId, + event.sequenceNumber) + } + + // fast forward stream for source replica + state.replicationControl.get(originReplicaId).foreach(_.fastForward(event.sequenceNumber)) + + handleExternalReplicatedEventPersist( + replication, + ReplicatedEvent( + event.event.asInstanceOf[E], + originReplicaId, + event.sequenceNumber, + replicatedMetadata.version)) + } + + } + } + // Used by EventSourcedBehaviorTestKit to retrieve the state. def onGetState(get: GetState[S]): Behavior[InternalProtocol] = { get.replyTo ! state.state this } + def onGetSeenSequenceNr(get: GetSeenSequenceNr): Behavior[InternalProtocol] = { + get.replyTo ! state.seenPerReplica(get.replica) + this + } + + private def handleExternalReplicatedEventPersist( + replication: ReplicationSetup, + event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { + _currentSequenceNumber = state.seqNr + 1 + val isConcurrent: Boolean = event.originVersion <> state.version + val updatedVersion = event.originVersion.merge(state.version) + + if (setup.log.isDebugEnabled()) + setup.log.debugN( + "Processing event [{}] with version [{}]. Local version: {}. Updated version {}. Concurrent? {}", + Logging.simpleName(event.event.getClass), + event.originVersion, + state.version, + updatedVersion, + isConcurrent) + + replication.setContext(recoveryRunning = false, event.originReplica, concurrent = isConcurrent) + + val stateAfterApply = state.applyEvent(setup, event.event) + val eventToPersist = adaptEvent(event.event) + val eventAdapterManifest = setup.eventAdapter.manifest(event.event) + + replication.clearContext() + + val newState2: RunningState[S] = internalPersist( + setup.context, + null, + stateAfterApply, + eventToPersist, + eventAdapterManifest, + OptionVal.Some( + ReplicatedEventMetadata(event.originReplica, event.originSequenceNr, updatedVersion, isConcurrent))) + val shouldSnapshotAfterPersist = setup.shouldSnapshot(newState2.state, event.event, newState2.seqNr) + // FIXME validate this is the correct sequence nr from that replica https://github.com/akka/akka/issues/29259 + val updatedSeen = newState2.seenPerReplica.updated(event.originReplica, event.originSequenceNr) + persistingEvents( + newState2.copy(seenPerReplica = updatedSeen, version = updatedVersion), + state, + numberOfEvents = 1, + shouldSnapshotAfterPersist, + shouldPublish = false, + Nil) + } + + private def handleEventPersist( + event: E, + cmd: Any, + sideEffects: immutable.Seq[SideEffect[S]]): (Behavior[InternalProtocol], Boolean) = { + try { + // apply the event before persist so that validation exception is handled before persisting + // the invalid event, in case such validation is implemented in the event handler. + // also, ensure that there is an event handler for each single event + _currentSequenceNumber = state.seqNr + 1 + + setup.replication.foreach(r => r.setContext(recoveryRunning = false, r.replicaId, concurrent = false)) + + val stateAfterApply = state.applyEvent(setup, event) + val eventToPersist = adaptEvent(event) + val eventAdapterManifest = setup.eventAdapter.manifest(event) + + val newState2 = setup.replication match { + case Some(replication) => + val updatedVersion = stateAfterApply.version.updated(replication.replicaId.id, _currentSequenceNumber) + val r = internalPersist( + setup.context, + cmd, + stateAfterApply, + eventToPersist, + eventAdapterManifest, + OptionVal.Some( + ReplicatedEventMetadata( + replication.replicaId, + _currentSequenceNumber, + updatedVersion, + concurrent = false))).copy(version = updatedVersion) + + if (setup.log.isTraceEnabled()) + setup.log.traceN( + "Event persisted [{}]. Version vector after: [{}]", + Logging.simpleName(event.getClass), + r.version) + + r + case None => + internalPersist(setup.context, cmd, stateAfterApply, eventToPersist, eventAdapterManifest, OptionVal.None) + } + + val shouldSnapshotAfterPersist = setup.shouldSnapshot(newState2.state, event, newState2.seqNr) + ( + persistingEvents( + newState2, + state, + numberOfEvents = 1, + shouldSnapshotAfterPersist, + shouldPublish = true, + sideEffects), + false) + } finally { + setup.replication.foreach(_.clearContext()) + } + } + + private def handleEventPersistAll( + events: immutable.Seq[E], + cmd: Any, + sideEffects: immutable.Seq[SideEffect[S]]): (Behavior[InternalProtocol], Boolean) = { + if (events.nonEmpty) { + try { + // apply the event before persist so that validation exception is handled before persisting + // the invalid event, in case such validation is implemented in the event handler. + // also, ensure that there is an event handler for each single event + _currentSequenceNumber = state.seqNr + + val metadataTemplate: Option[ReplicatedEventMetadata] = setup.replication match { + case Some(replication) => + replication.setContext(recoveryRunning = false, replication.replicaId, concurrent = false) // local events are never concurrent + Some(ReplicatedEventMetadata(replication.replicaId, 0L, state.version, concurrent = false)) // we replace it with actual seqnr later + case None => None + } + + var currentState = state + var shouldSnapshotAfterPersist: SnapshotAfterPersist = NoSnapshot + var eventsToPersist: List[EventToPersist] = Nil + + events.foreach { event => + _currentSequenceNumber += 1 + if (shouldSnapshotAfterPersist == NoSnapshot) + shouldSnapshotAfterPersist = setup.shouldSnapshot(currentState.state, event, _currentSequenceNumber) + val evtManifest = setup.eventAdapter.manifest(event) + val adaptedEvent = adaptEvent(event) + val eventMetadata = metadataTemplate match { + case Some(template) => + val updatedVersion = currentState.version.updated(template.originReplica.id, _currentSequenceNumber) + if (setup.log.isDebugEnabled) + setup.log.traceN( + "Processing event [{}] with version vector [{}]", + Logging.simpleName(event.getClass), + updatedVersion) + currentState = currentState.copy(version = updatedVersion) + Some(template.copy(originSequenceNr = _currentSequenceNumber, version = updatedVersion)) + case None => None + } + + currentState = currentState.applyEvent(setup, event) + + eventsToPersist = EventToPersist(adaptedEvent, evtManifest, eventMetadata) :: eventsToPersist + } + + val newState2 = + internalPersistAll(setup.context, cmd, currentState, eventsToPersist.reverse) + + ( + persistingEvents( + newState2, + state, + events.size, + shouldSnapshotAfterPersist, + shouldPublish = true, + sideEffects = sideEffects), + false) + } finally { + setup.replication.foreach(_.clearContext()) + } + } else { + // run side-effects even when no events are emitted + (applySideEffects(sideEffects, state), true) + } + } @tailrec def applyEffects( msg: Any, state: RunningState[S], @@ -154,46 +589,10 @@ private[akka] object Running { applyEffects(msg, state, eff, currentSideEffects ++ sideEffects) case Persist(event) => - // apply the event before persist so that validation exception is handled before persisting - // the invalid event, in case such validation is implemented in the event handler. - // also, ensure that there is an event handler for each single event - _currentSequenceNumber = state.seqNr + 1 - val newState = state.applyEvent(setup, event) - - val eventToPersist = adaptEvent(event) - val eventAdapterManifest = setup.eventAdapter.manifest(event) - - val newState2 = internalPersist(setup.context, msg, newState, eventToPersist, eventAdapterManifest) - - val shouldSnapshotAfterPersist = setup.shouldSnapshot(newState2.state, event, newState2.seqNr) - - (persistingEvents(newState2, state, numberOfEvents = 1, shouldSnapshotAfterPersist, sideEffects), false) + handleEventPersist(event, msg, sideEffects) case PersistAll(events) => - if (events.nonEmpty) { - // apply the event before persist so that validation exception is handled before persisting - // the invalid event, in case such validation is implemented in the event handler. - // also, ensure that there is an event handler for each single event - _currentSequenceNumber = state.seqNr - val (newState, shouldSnapshotAfterPersist) = events.foldLeft((state, NoSnapshot: SnapshotAfterPersist)) { - case ((currentState, snapshot), event) => - _currentSequenceNumber += 1 - val shouldSnapshot = - if (snapshot == NoSnapshot) setup.shouldSnapshot(currentState.state, event, _currentSequenceNumber) - else snapshot - (currentState.applyEvent(setup, event), shouldSnapshot) - } - - val eventsToPersist = events.map(evt => (adaptEvent(evt), setup.eventAdapter.manifest(evt))) - - val newState2 = internalPersistAll(setup.context, msg, newState, eventsToPersist) - - (persistingEvents(newState2, state, events.size, shouldSnapshotAfterPersist, sideEffects), false) - - } else { - // run side-effects even when no events are emitted - (applySideEffects(sideEffects, state), true) - } + handleEventPersistAll(events, msg, sideEffects) case _: PersistNothing.type => (applySideEffects(sideEffects, state), true) @@ -232,9 +631,10 @@ private[akka] object Running { visibleState: RunningState[S], // previous state until write success numberOfEvents: Int, shouldSnapshotAfterPersist: SnapshotAfterPersist, + shouldPublish: Boolean, sideEffects: immutable.Seq[SideEffect[S]]): Behavior[InternalProtocol] = { setup.setMdcPhase(PersistenceMdc.PersistingEvents) - new PersistingEvents(state, visibleState, numberOfEvents, shouldSnapshotAfterPersist, sideEffects) + new PersistingEvents(state, visibleState, numberOfEvents, shouldSnapshotAfterPersist, shouldPublish, sideEffects) } /** INTERNAL API */ @@ -243,6 +643,7 @@ private[akka] object Running { var visibleState: RunningState[S], // previous state until write success numberOfEvents: Int, shouldSnapshotAfterPersist: SnapshotAfterPersist, + shouldPublish: Boolean, var sideEffects: immutable.Seq[SideEffect[S]], persistStartTime: Long = System.nanoTime()) extends AbstractBehavior[InternalProtocol](setup.context) @@ -252,12 +653,15 @@ private[akka] object Running { override def onMessage(msg: InternalProtocol): Behavior[InternalProtocol] = { msg match { - case JournalResponse(r) => onJournalResponse(r) - case in: IncomingCommand[C @unchecked] => onCommand(in) - case get: GetState[S @unchecked] => stashInternal(get) - case SnapshotterResponse(r) => onDeleteSnapshotResponse(r, visibleState.state) - case RecoveryTickEvent(_) => Behaviors.unhandled - case RecoveryPermitGranted => Behaviors.unhandled + case JournalResponse(r) => onJournalResponse(r) + case in: IncomingCommand[C @unchecked] => onCommand(in) + case re: ReplicatedEventEnvelope[E @unchecked] => onReplicatedEvent(re) + case pe: PublishedEventImpl => onPublishedEvent(pe) + case get: GetState[S @unchecked] => stashInternal(get) + case getSeqNr: GetSeenSequenceNr => onGetSeenSequenceNr(getSeqNr) + case SnapshotterResponse(r) => onDeleteSnapshotResponse(r, visibleState.state) + case RecoveryTickEvent(_) => Behaviors.unhandled + case RecoveryPermitGranted => Behaviors.unhandled } } @@ -271,6 +675,27 @@ private[akka] object Running { } } + def onGetSeenSequenceNr(get: GetSeenSequenceNr): PersistingEvents = { + get.replyTo ! state.seenPerReplica(get.replica) + this + } + + def onReplicatedEvent(event: InternalProtocol.ReplicatedEventEnvelope[E]): Behavior[InternalProtocol] = { + if (state.receivedPoisonPill) { + Behaviors.unhandled + } else { + stashInternal(event) + } + } + + def onPublishedEvent(event: PublishedEventImpl): Behavior[InternalProtocol] = { + if (state.receivedPoisonPill) { + Behaviors.unhandled + } else { + stashInternal(event) + } + } + final def onJournalResponse(response: Response): Behavior[InternalProtocol] = { if (setup.log.isDebugEnabled) { setup.log.debug2( @@ -285,6 +710,13 @@ private[akka] object Running { onWriteSuccess(setup.context, p) + if (setup.publishEvents && shouldPublish) { + val meta = setup.replication.map(replication => + new ReplicatedPublishedEventMetaData(replication.replicaId, state.version)) + context.system.eventStream ! EventStream.Publish( + PublishedEventImpl(setup.persistenceId, p.sequenceNr, p.payload, p.timestamp, meta)) + } + // only once all things are applied we can revert back if (eventCounter < numberOfEvents) { onWriteDone(setup.context, p) @@ -427,6 +859,8 @@ private[akka] object Running { } case get: GetState[S @unchecked] => stashInternal(get) + case get: GetSeenSequenceNr => + stashInternal(get) case _ => Behaviors.unhandled } diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/StashManagement.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/StashManagement.scala index 49df6ae06b..5fca6336d5 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/StashManagement.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/StashManagement.scala @@ -62,7 +62,7 @@ private[akka] trait StashManagement[C, E, S] { } /** - * `tryUnstashOne` is called at the end of processing each command or when persist is completed + * `tryUnstashOne` is called at the end of processing each command, published event, or when persist is completed */ protected def tryUnstashOne(behavior: Behavior[InternalProtocol]): Behavior[InternalProtocol] = { val buffer = diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/VersionVector.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/VersionVector.scala new file mode 100644 index 0000000000..3043a7ad30 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/VersionVector.scala @@ -0,0 +1,322 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed.internal +import scala.annotation.tailrec +import scala.collection.immutable.TreeMap +import akka.annotation.InternalApi + +/** + * INTERNAL API + * + * VersionVector module with helper classes and methods. + */ +@InternalApi +private[akka] object VersionVector { + + private val emptyVersions: TreeMap[String, Long] = TreeMap.empty + val empty: VersionVector = ManyVersionVector(emptyVersions) + + def apply(): VersionVector = empty + + def apply(versions: TreeMap[String, Long]): VersionVector = + if (versions.isEmpty) empty + else if (versions.size == 1) apply(versions.head._1, versions.head._2) + else ManyVersionVector(versions) + + def apply(key: String, version: Long): VersionVector = OneVersionVector(key, version) + + /** INTERNAL API */ + @InternalApi private[akka] def apply(versions: List[(String, Long)]): VersionVector = + if (versions.isEmpty) empty + else if (versions.tail.isEmpty) apply(versions.head._1, versions.head._2) + else apply(emptyVersions ++ versions) + + sealed trait Ordering + case object After extends Ordering + case object Before extends Ordering + case object Same extends Ordering + case object Concurrent extends Ordering + + /** + * Marker to ensure that we do a full order comparison instead of bailing out early. + */ + private case object FullOrder extends Ordering + + /** INTERNAL API */ + @InternalApi private[akka] object Timestamp { + final val Zero = 0L + final val EndMarker = Long.MinValue + } + + /** + * Marker to signal that we have reached the end of a version vector. + */ + private val cmpEndMarker = (null, Timestamp.EndMarker) + +} + +/** + * INTERNAL API + * + * Representation of a Vector-based clock (counting clock), inspired by Lamport logical clocks. + * {{{ + * Reference: + * 1) Leslie Lamport (1978). "Time, clocks, and the ordering of events in a distributed system". Communications of the ACM 21 (7): 558-565. + * 2) Friedemann Mattern (1988). "Virtual Time and Global States of Distributed Systems". Workshop on Parallel and Distributed Algorithms: pp. 215-226 + * }}} + * + * Based on `akka.cluster.ddata.VersionVector`. + * + * This class is immutable, i.e. "modifying" methods return a new instance. + */ +@InternalApi +private[akka] sealed abstract class VersionVector { + + type T = VersionVector + + import VersionVector._ + + /** + * Increment the version for the key passed as argument. Returns a new VersionVector. + */ + def +(key: String): VersionVector = increment(key) + + /** + * Increment the version for the key passed as argument. Returns a new VersionVector. + */ + def increment(key: String): VersionVector + + def updated(key: String, version: Long): VersionVector + + def isEmpty: Boolean + + /** + * INTERNAL API + */ + @InternalApi private[akka] def size: Int + + def versionAt(key: String): Long + + /** + * INTERNAL API + */ + @InternalApi private[akka] def contains(key: String): Boolean + + /** + * Returns true if this and that are concurrent else false. + */ + def <>(that: VersionVector): Boolean = compareOnlyTo(that, Concurrent) eq Concurrent + + /** + * Returns true if this is before that else false. + */ + def <(that: VersionVector): Boolean = compareOnlyTo(that, Before) eq Before + + /** + * Returns true if this is after that else false. + */ + def >(that: VersionVector): Boolean = compareOnlyTo(that, After) eq After + + /** + * Returns true if this VersionVector has the same history as the 'that' VersionVector else false. + */ + def ==(that: VersionVector): Boolean = compareOnlyTo(that, Same) eq Same + + /** + * Version vector comparison according to the semantics described by compareTo, with the ability to bail + * out early if the we can't reach the Ordering that we are looking for. + * + * The ordering always starts with Same and can then go to Same, Before or After + * If we're on After we can only go to After or Concurrent + * If we're on Before we can only go to Before or Concurrent + * If we go to Concurrent we exit the loop immediately + * + * If you send in the ordering FullOrder, you will get a full comparison. + */ + private final def compareOnlyTo(that: VersionVector, order: Ordering): Ordering = { + def nextOrElse[A](iter: Iterator[A], default: A): A = if (iter.hasNext) iter.next() else default + + def compare(i1: Iterator[(String, Long)], i2: Iterator[(String, Long)], requestedOrder: Ordering): Ordering = { + @tailrec + def compareNext(nt1: (String, Long), nt2: (String, Long), currentOrder: Ordering): Ordering = + if ((requestedOrder ne FullOrder) && (currentOrder ne Same) && (currentOrder ne requestedOrder)) currentOrder + else if ((nt1 eq cmpEndMarker) && (nt2 eq cmpEndMarker)) currentOrder + // i1 is empty but i2 is not, so i1 can only be Before + else if (nt1 eq cmpEndMarker) { + if (currentOrder eq After) Concurrent else Before + } + // i2 is empty but i1 is not, so i1 can only be After + else if (nt2 eq cmpEndMarker) { + if (currentOrder eq Before) Concurrent else After + } else { + // compare the entries + val nc = nt1._1.compareTo(nt2._1) + if (nc == 0) { + // both entries exist compare the timestamps + // same timestamp so just continue with the next entry + if (nt1._2 == nt2._2) compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), currentOrder) + else if (nt1._2 < nt2._2) { + // t1 is less than t2, so i1 can only be Before + if (currentOrder eq After) Concurrent + else compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), Before) + } else { + // t2 is less than t1, so i1 can only be After + if (currentOrder eq Before) Concurrent + else compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), After) + } + } else if (nc < 0) { + // this entry only exists in i1 so i1 can only be After + if (currentOrder eq Before) Concurrent + else compareNext(nextOrElse(i1, cmpEndMarker), nt2, After) + } else { + // this entry only exists in i2 so i1 can only be Before + if (currentOrder eq After) Concurrent + else compareNext(nt1, nextOrElse(i2, cmpEndMarker), Before) + } + } + + compareNext(nextOrElse(i1, cmpEndMarker), nextOrElse(i2, cmpEndMarker), Same) + } + + if (this eq that) Same + else compare(this.versionsIterator, that.versionsIterator, if (order eq Concurrent) FullOrder else order) + } + + /** + * INTERNAL API + */ + @InternalApi private[akka] def versionsIterator: Iterator[(String, Long)] + + /** + * Compare two version vectors. The outcome will be one of the following: + *

+ * {{{ + * 1. Version 1 is SAME (==) as Version 2 iff for all i c1(i) == c2(i) + * 2. Version 1 is BEFORE (<) Version 2 iff for all i c1(i) <= c2(i) and there exist a j such that c1(j) < c2(j) + * 3. Version 1 is AFTER (>) Version 2 iff for all i c1(i) >= c2(i) and there exist a j such that c1(j) > c2(j). + * 4. Version 1 is CONCURRENT (<>) to Version 2 otherwise. + * }}} + */ + def compareTo(that: VersionVector): Ordering = { + compareOnlyTo(that, FullOrder) + } + + def merge(that: VersionVector): VersionVector + +} + +/** + * INTERNAL API + */ +@InternalApi private[akka] final case class OneVersionVector private[akka] (key: String, version: Long) + extends VersionVector { + import VersionVector.Timestamp + + override def isEmpty: Boolean = false + + /** INTERNAL API */ + @InternalApi private[akka] override def size: Int = 1 + + override def increment(k: String): VersionVector = { + val v = version + 1 + if (k == key) copy(version = v) + else ManyVersionVector(TreeMap(key -> version, k -> v)) + } + + override def updated(k: String, v: Long): VersionVector = { + if (k == key) copy(version = v) + else ManyVersionVector(TreeMap(key -> version, k -> v)) + } + + override def versionAt(k: String): Long = + if (k == key) version + else Timestamp.Zero + + /** INTERNAL API */ + @InternalApi private[akka] override def contains(k: String): Boolean = + k == key + + /** INTERNAL API */ + @InternalApi private[akka] override def versionsIterator: Iterator[(String, Long)] = + Iterator.single((key, version)) + + override def merge(that: VersionVector): VersionVector = { + that match { + case OneVersionVector(n2, v2) => + if (key == n2) if (version >= v2) this else OneVersionVector(n2, v2) + else ManyVersionVector(TreeMap(key -> version, n2 -> v2)) + case ManyVersionVector(vs2) => + val v2 = vs2.getOrElse(key, Timestamp.Zero) + val mergedVersions = + if (v2 >= version) vs2 + else vs2.updated(key, version) + VersionVector(mergedVersions) + } + } + override def toString: String = + s"VersionVector($key -> $version)" + +} + +// TODO we could add more specialized/optimized implementations for 2 and 3 entries, because +// that will be the typical number of data centers + +/** + * INTERNAL API + */ +@InternalApi private[akka] final case class ManyVersionVector(versions: TreeMap[String, Long]) extends VersionVector { + import VersionVector.Timestamp + + override def isEmpty: Boolean = versions.isEmpty + + /** INTERNAL API */ + @InternalApi private[akka] override def size: Int = versions.size + + override def increment(key: String): VersionVector = { + val v = versionAt(key) + 1 + VersionVector(versions.updated(key, v)) + } + + override def updated(key: String, v: Long): VersionVector = + VersionVector(versions.updated(key, v)) + + override def versionAt(key: String): Long = versions.get(key) match { + case Some(v) => v + case None => Timestamp.Zero + } + + /** INTERNAL API */ + @InternalApi private[akka] override def contains(key: String): Boolean = + versions.contains(key) + + /** INTERNAL API */ + @InternalApi private[akka] override def versionsIterator: Iterator[(String, Long)] = + versions.iterator + + override def merge(that: VersionVector): VersionVector = { + if (that.isEmpty) this + else if (this.isEmpty) that + else + that match { + case ManyVersionVector(vs2) => + var mergedVersions = vs2 + for ((key, time) <- versions) { + val mergedVersionsCurrentTime = mergedVersions.getOrElse(key, Timestamp.Zero) + if (time > mergedVersionsCurrentTime) + mergedVersions = mergedVersions.updated(key, time) + } + VersionVector(mergedVersions) + case OneVersionVector(n2, v2) => + val v1 = versions.getOrElse(n2, Timestamp.Zero) + val mergedVersions = + if (v1 >= v2) versions + else versions.updated(n2, v2) + VersionVector(mergedVersions) + } + } + + override def toString: String = + versions.map { case (k, v) => k + " -> " + v }.mkString("VersionVector(", ", ", ")") +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventSourcedBehavior.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventSourcedBehavior.scala index 0b896c4b0e..ec5ea886c2 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventSourcedBehavior.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/EventSourcedBehavior.scala @@ -176,9 +176,16 @@ abstract class EventSourcedBehavior[Command, Event, State] private[akka] ( def snapshotAdapter(): SnapshotAdapter[State] = NoOpSnapshotAdapter.instance[State] /** - * INTERNAL API: DeferredBehavior init + * INTERNAL API: DeferredBehavior init, not for user extension */ - @InternalApi override def apply(context: typed.TypedActorContext[Command]): Behavior[Command] = { + @InternalApi override def apply(context: typed.TypedActorContext[Command]): Behavior[Command] = + createEventSourcedBehavior() + + /** + * INTERNAL API + */ + @InternalApi private[akka] final def createEventSourcedBehavior() + : scaladsl.EventSourcedBehavior[Command, Event, State] = { val snapshotWhen: (State, Event, Long) => Boolean = (state, event, seqNr) => shouldSnapshot(state, event, seqNr) val tagger: Event => Set[String] = { event => diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcedBehavior.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcedBehavior.scala new file mode 100644 index 0000000000..06f6288330 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcedBehavior.scala @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed.javadsl + +import java.util.Optional + +import akka.actor.typed.BackoffSupervisorStrategy +import akka.actor.typed.Behavior +import akka.actor.typed.TypedActorContext +import akka.annotation.ApiMayChange +import akka.annotation.InternalApi +import akka.persistence.typed.internal.ReplicationContextImpl + +/** + * Base class for replicated event sourced behaviors. + */ +@ApiMayChange +abstract class ReplicatedEventSourcedBehavior[Command, Event, State]( + replicationContext: ReplicationContext, + onPersistFailure: Optional[BackoffSupervisorStrategy]) + extends EventSourcedBehavior[Command, Event, State](replicationContext.persistenceId, onPersistFailure) { + + def this(replicationContext: ReplicationContext) = this(replicationContext, Optional.empty()) + + /** + * Override and return false to disable events being published to the system event stream as + * [[akka.persistence.typed.PublishedEvent]] after they have been persisted. + */ + def withEventPublishing: Boolean = true + + protected def getReplicationContext(): ReplicationContext = replicationContext + + /** + * INTERNAL API: DeferredBehavior init, not for user extension + */ + @InternalApi override def apply(context: TypedActorContext[Command]): Behavior[Command] = { + createEventSourcedBehavior() + // context not user extendable so there should never be any other impls + .withReplication(replicationContext.asInstanceOf[ReplicationContextImpl]) + .withEventPublishing(withEventPublishing) + } +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcing.scala new file mode 100644 index 0000000000..1ba2d88151 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcing.scala @@ -0,0 +1,125 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed.javadsl + +import java.util.function.{ Function => JFunction } +import java.util.{ Set => JSet } +import java.util.{ Map => JMap } + +import akka.annotation.DoNotInherit +import akka.persistence.typed.PersistenceId +import akka.persistence.typed.ReplicaId +import akka.persistence.typed.ReplicationId +import akka.persistence.typed.internal.ReplicationContextImpl +import akka.util.ccompat.JavaConverters._ + +/** + * Provides access to replication specific state + * + * Not for user extension + */ +@DoNotInherit +trait ReplicationContext { + + def replicationId: ReplicationId + + /** + * @return The replica id of this replicated event sourced actor + */ + def replicaId: ReplicaId + + /** + * @return The ids of all replicas of this replicated event sourced actor + */ + def getAllReplicas: JSet[ReplicaId] + + /** + * @return The unique id of this replica, including the replica id + */ + def persistenceId: PersistenceId + + /** + * @return The unique id of this replica, not including the replica id + */ + def entityId: String + + /** + * Must only be called from the event handler + * @return true when the event handler is invoked during recovery. + */ + def recoveryRunning: Boolean + + /** + * Must only be called from the event handler + * @return the replica id where the current event was persisted + */ + def origin: ReplicaId + + /** + * Must only be called from the event handler + * @return true if this event happened concurrent with an event from another replica + */ + def concurrent: Boolean + + /** + * @return a timestamp that will always be increasing (is monotonic) + */ + def currentTimeMillis(): Long +} + +object ReplicatedEventSourcing { + + /** + * Initialize a replicated event sourced behavior where all entity replicas are share the same journal configuration. + * This is typical if there is a shared database and no replica specific configuratin is required. + * + * Events from each replica for the same entityId will be replicated to every copy. + * Care must be taken to handle events in any order as events can happen concurrently at different replicas. + * + * Using an replicated event sourced behavior means there is no longer the single writer guarantee. + * + * A different journal plugin id can be configured using withJournalPluginId after creation. Different databases + * can be used for each replica. + * The events from other replicas are read using PersistentQuery. + * + * @param queryPluginId A single query plugin used to read the events from other replicas. Must be the query side of your configured journal plugin. + */ + def commonJournalConfig[Command, Event, State]( + replicationId: ReplicationId, + allReplicaIds: JSet[ReplicaId], + queryPluginId: String, + behaviorFactory: JFunction[ReplicationContext, EventSourcedBehavior[Command, Event, State]]) + : EventSourcedBehavior[Command, Event, State] = + perReplicaJournalConfig( + replicationId, + allReplicaIds.asScala.map(id => id -> queryPluginId).toMap.asJava, + behaviorFactory) + + /** + * Initialize a replicated event sourced behavior where each journal has different journal configuration e.g. + * each replica uses a different database or requires different database configuration for a shared database. + * + * Events from each replica for the same entityId will be replicated to every copy. + * Care must be taken to handle events in any order as events can happen concurrently at different replicas. + * + * Using an replicated event sourced behavior means there is no longer the single writer guarantee. + * + * The journal plugin id for the entity itself can be configured using withJournalPluginId after creation. + * A query side identifier is passed per replica allowing for separate database/journal configuration per + * replica. The events from other replicas are read using PersistentQuery. + * + * @param allReplicasAndQueryPlugins All replica ids and a query plugin per replica id. These need to be known to receive events from all replicas + * and configured with the query plugin for the journal that each replica uses. + */ + def perReplicaJournalConfig[Command, Event, State]( + replicationId: ReplicationId, + allReplicasAndQueryPlugins: JMap[ReplicaId, String], + eventSourcedBehaviorFactory: JFunction[ReplicationContext, EventSourcedBehavior[Command, Event, State]]) + : EventSourcedBehavior[Command, Event, State] = { + val context = new ReplicationContextImpl(replicationId, allReplicasAndQueryPlugins.asScala.toMap) + eventSourcedBehaviorFactory(context) + } + +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/EventSourcedBehavior.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/EventSourcedBehavior.scala index 7583715b9e..60c13ab218 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/EventSourcedBehavior.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/EventSourcedBehavior.scala @@ -4,8 +4,6 @@ package akka.persistence.typed.scaladsl -import scala.annotation.tailrec - import akka.actor.typed.BackoffSupervisorStrategy import akka.actor.typed.Behavior import akka.actor.typed.Signal @@ -13,13 +11,17 @@ import akka.actor.typed.internal.BehaviorImpl.DeferredBehavior import akka.actor.typed.internal.InterceptorImpl import akka.actor.typed.internal.LoggerClass import akka.actor.typed.scaladsl.ActorContext +import akka.annotation.ApiMayChange import akka.annotation.DoNotInherit +import akka.annotation.InternalApi import akka.persistence.typed.EventAdapter import akka.persistence.typed.PersistenceId import akka.persistence.typed.SnapshotAdapter import akka.persistence.typed.SnapshotSelectionCriteria import akka.persistence.typed.internal._ +import scala.annotation.tailrec + object EventSourcedBehavior { /** @@ -215,4 +217,16 @@ object EventSourcedBehavior { * By default, snapshots and events are recovered. */ def withRecovery(recovery: Recovery): EventSourcedBehavior[Command, Event, State] + + /** + * Publish events to the system event stream as [[akka.persistence.typed.PublishedEvent]] after they have been persisted + */ + @ApiMayChange + def withEventPublishing(enabled: Boolean): EventSourcedBehavior[Command, Event, State] + + /** + * INTERNAL API + */ + @InternalApi + private[akka] def withReplication(context: ReplicationContextImpl): EventSourcedBehavior[Command, Event, State] } diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ReplicatedEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ReplicatedEventSourcing.scala new file mode 100644 index 0000000000..c9d12d6655 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ReplicatedEventSourcing.scala @@ -0,0 +1,117 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed.scaladsl + +import akka.annotation.DoNotInherit +import akka.persistence.typed.PersistenceId +import akka.persistence.typed.ReplicaId +import akka.persistence.typed.ReplicationId +import akka.persistence.typed.internal.ReplicationContextImpl + +/** + * Provides access to replication specific state + * + * Not for user extension + */ +@DoNotInherit +trait ReplicationContext { + + def replicationId: ReplicationId + + /** + * @return The unique id of this replica, including the replica id + */ + def persistenceId: PersistenceId + + /** + * @return The replica id of this replicated event sourced actor + */ + def replicaId: ReplicaId = replicationId.replicaId + + /** + * @return The ids of all replicas of this replicated event sourced actor + */ + def allReplicas: Set[ReplicaId] + + /** + * @return The entity id of this replicated event sourced actor (not including the replica id) + */ + def entityId: String = replicationId.entityId + + /** + * Must only be called from the event handler + * @return the replica id where the current event was persisted + */ + def origin: ReplicaId + + /** + * Must only be called from the event handler + * @return true if this event happened concurrent with an event from another replica + */ + def concurrent: Boolean + + /** + * Must only be called from the event handler + * @return true when the event handler is invoked during recovery. + */ + def recoveryRunning: Boolean + + /** + * @return a timestamp that will always be increasing (is monotonic) + */ + def currentTimeMillis(): Long + +} + +object ReplicatedEventSourcing { + + /** + * Initialize a replicated event sourced behavior where all entity replicas are stored in the same journal. + * + * Events from each replica for the same entityId will be replicated to every copy. + * Care must be taken to handle events in any order as events can happen concurrently at different replicas. + * + * Using an replicated event sourced behavior means there is no longer the single writer guarantee. + * + * A different journal plugin id can be configured using withJournalPluginId after creation. Different databases + * can be used for each replica. + * The events from other replicas are read using PersistentQuery. + * + * @param allReplicaIds All replica ids. These need to be known to receive events from all replicas. + * @param queryPluginId A single query plugin used to read the events from other replicas. Must be the query side of your configured journal plugin. + */ + def commonJournalConfig[Command, Event, State]( + replicationId: ReplicationId, + allReplicaIds: Set[ReplicaId], + queryPluginId: String)( + eventSourcedBehaviorFactory: ReplicationContext => EventSourcedBehavior[Command, Event, State]) + : EventSourcedBehavior[Command, Event, State] = + perReplicaJournalConfig(replicationId, allReplicaIds.map(id => id -> queryPluginId).toMap)( + eventSourcedBehaviorFactory) + + /** + * Initialize a replicated event sourced behavior. + * + * Events from each replica for the same entityId will be replicated to every copy. + * Care must be taken to handle events in any order as events can happen concurrently at different replicas. + * + * Using an replicated event sourced behavior means there is no longer the single writer guarantee. + * + * The journal plugin id for the entity itself can be configured using withJournalPluginId after creation. + * A query side identifier is passed per replica allowing for separate database/journal configuration per + * replica. The events from other replicas are read using PersistentQuery. + * @param allReplicasAndQueryPlugins All replica ids and a query plugin per replica id. These need to be known to receive events from all replicas + * and configured with the query plugin for the journal that each replica uses. + */ + def perReplicaJournalConfig[Command, Event, State]( + replicationId: ReplicationId, + allReplicasAndQueryPlugins: Map[ReplicaId, String])( + eventSourcedBehaviorFactory: ReplicationContext => EventSourcedBehavior[Command, Event, State]) + : EventSourcedBehavior[Command, Event, State] = { + val context = new ReplicationContextImpl(replicationId, allReplicasAndQueryPlugins) + eventSourcedBehaviorFactory(context).withReplication(context) + } + +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializer.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializer.scala new file mode 100644 index 0000000000..556244179d --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializer.scala @@ -0,0 +1,401 @@ +/* + * Copyright (C) 2009-2020 Lightbend Inc. + */ + +package akka.persistence.typed.serialization + +import java.io.NotSerializableException +import java.util.{ ArrayList, Collections, Comparator } +import java.{ lang => jl } + +import akka.actor.ExtendedActorSystem +import akka.annotation.InternalApi +import akka.persistence.typed.PersistenceId +import akka.persistence.typed.ReplicaId +import akka.persistence.typed.crdt.{ Counter, ORSet } +import akka.persistence.typed.internal.PublishedEventImpl +import akka.persistence.typed.internal.ReplicatedEventMetadata +import akka.persistence.typed.internal.ReplicatedSnapshotMetadata +import akka.persistence.typed.internal.ReplicatedPublishedEventMetaData +import akka.persistence.typed.internal.VersionVector +import akka.protobufv3.internal.ByteString +import akka.remote.ContainerFormats.Payload +import akka.remote.serialization.WrappedPayloadSupport +import akka.serialization.{ BaseSerializer, SerializerWithStringManifest } + +import scala.annotation.tailrec +import akka.util.ccompat.JavaConverters._ + +import scala.collection.immutable.TreeMap + +/** + * INTERNAL API + */ +@InternalApi private[akka] object ReplicatedEventSourcingSerializer { + object Comparator extends Comparator[Payload] { + override def compare(a: Payload, b: Payload): Int = { + val aByteString = a.getEnclosedMessage + val bByteString = b.getEnclosedMessage + val aSize = aByteString.size + val bSize = bByteString.size + if (aSize == bSize) { + val aIter = aByteString.iterator + val bIter = bByteString.iterator + @tailrec def findDiff(): Int = { + if (aIter.hasNext) { + val aByte = aIter.nextByte() + val bByte = bIter.nextByte() + if (aByte < bByte) -1 + else if (aByte > bByte) 1 + else findDiff() + } else 0 + } + findDiff() + } else if (aSize < bSize) -1 + else 1 + } + } +} + +/** + * INTERNAL API + */ +@InternalApi private[akka] final class ReplicatedEventSourcingSerializer(val system: ExtendedActorSystem) + extends SerializerWithStringManifest + with BaseSerializer { + + private val wrappedSupport = new WrappedPayloadSupport(system) + + private val CrdtCounterManifest = "AA" + private val CrdtCounterUpdatedManifest = "AB" + + private val ORSetManifest = "CA" + private val ORSetAddManifest = "CB" + private val ORSetRemoveManifest = "CC" + private val ORSetFullManifest = "CD" + private val ORSetDeltaGroupManifest = "CE" + + private val VersionVectorManifest = "DA" + + private val ReplicatedEventMetadataManifest = "RE" + private val ReplicatedSnapshotMetadataManifest = "RS" + + private val PublishedEventManifest = "PA" + + def manifest(o: AnyRef) = o match { + case _: ORSet[_] => ORSetManifest + case _: ORSet.AddDeltaOp[_] => ORSetAddManifest + case _: ORSet.RemoveDeltaOp[_] => ORSetRemoveManifest + case _: ORSet.DeltaGroup[_] => ORSetDeltaGroupManifest + case _: ORSet.FullStateDeltaOp[_] => ORSetFullManifest + + case _: Counter => CrdtCounterManifest + case _: Counter.Updated => CrdtCounterUpdatedManifest + + case _: VersionVector => VersionVectorManifest + + case _: ReplicatedEventMetadata => ReplicatedEventMetadataManifest + case _: ReplicatedSnapshotMetadata => ReplicatedSnapshotMetadataManifest + + case _: PublishedEventImpl => PublishedEventManifest + case _ => + throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]") + } + + def toBinary(o: AnyRef) = o match { + case m: ReplicatedEventMetadata => replicatedEventMetadataToProto(m).toByteArray + case m: ReplicatedSnapshotMetadata => replicatedSnapshotMetadataToByteArray(m) + + case m: VersionVector => versionVectorToProto(m).toByteArray + + case m: ORSet[_] => orsetToProto(m).toByteArray + case m: ORSet.AddDeltaOp[_] => orsetToProto(m.underlying).toByteArray + case m: ORSet.RemoveDeltaOp[_] => orsetToProto(m.underlying).toByteArray + case m: ORSet.DeltaGroup[_] => orsetDeltaGroupToProto(m).toByteArray + case m: ORSet.FullStateDeltaOp[_] => orsetToProto(m.underlying).toByteArray + + case m: Counter => counterToProtoByteArray(m) + case m: Counter.Updated => counterUpdatedToProtoBufByteArray(m) + + case m: PublishedEventImpl => publishedEventToProtoByteArray(m) + + case _ => + throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass}") + } + + def fromBinary(bytes: Array[Byte], manifest: String) = manifest match { + + case ReplicatedEventMetadataManifest => replicatedEventMetadataFromBinary(bytes) + case ReplicatedSnapshotMetadataManifest => replicatedSnapshotMetadataFromBinary(bytes) + + case VersionVectorManifest => versionVectorFromBinary(bytes) + + case ORSetManifest => orsetFromBinary(bytes) + case ORSetAddManifest => orsetAddFromBinary(bytes) + case ORSetRemoveManifest => orsetRemoveFromBinary(bytes) + case ORSetFullManifest => orsetFullFromBinary(bytes) + case ORSetDeltaGroupManifest => orsetDeltaGroupFromBinary(bytes) + + case CrdtCounterManifest => counterFromBinary(bytes) + case CrdtCounterUpdatedManifest => counterUpdatedFromBinary(bytes) + + case PublishedEventManifest => publishedEventFromBinary(bytes) + + case _ => + throw new NotSerializableException( + s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]") + } + + def publishedEventToProtoByteArray(impl: PublishedEventImpl): Array[Byte] = { + val builder = ReplicatedEventSourcing.PublishedEvent + .newBuilder() + .setPersistenceId(impl.persistenceId.id) + .setSequenceNr(impl.sequenceNumber) + .setPayload(wrappedSupport.payloadBuilder(impl.payload)) + .setTimestamp(impl.timestamp) + + (impl.replicatedMetaData match { + case None => + builder + case Some(m) => + builder.setMetadata( + ReplicatedEventSourcing.ReplicatedPublishedEventMetaData + .newBuilder() + .setReplicaId(m.replicaId.id) + .setVersionVector(versionVectorToProto(m.version)) + .build()) + }).build().toByteArray + } + + def publishedEventFromBinary(bytes: Array[Byte]): PublishedEventImpl = { + val p = ReplicatedEventSourcing.PublishedEvent.parseFrom(bytes) + PublishedEventImpl( + PersistenceId.ofUniqueId(p.getPersistenceId), + p.getSequenceNr, + wrappedSupport.deserializePayload(p.getPayload), + p.getTimestamp, + if (p.hasMetadata) { + val protoMeta = p.getMetadata + Some( + new ReplicatedPublishedEventMetaData( + ReplicaId(protoMeta.getReplicaId), + versionVectorFromProto(protoMeta.getVersionVector))) + } else None) + } + + def counterFromBinary(bytes: Array[Byte]): Counter = + Counter(BigInt(ReplicatedEventSourcing.Counter.parseFrom(bytes).getValue.toByteArray)) + + def counterUpdatedFromBinary(bytes: Array[Byte]): Counter.Updated = + Counter.Updated(BigInt(ReplicatedEventSourcing.CounterUpdate.parseFrom(bytes).getDelta.toByteArray)) + + def counterToProtoByteArray(counter: Counter): Array[Byte] = + ReplicatedEventSourcing.Counter + .newBuilder() + .setValue(ByteString.copyFrom(counter.value.toByteArray)) + .build() + .toByteArray + + def counterUpdatedToProtoBufByteArray(updated: Counter.Updated): Array[Byte] = + ReplicatedEventSourcing.CounterUpdate + .newBuilder() + .setDelta(ByteString.copyFrom(updated.delta.toByteArray)) + .build() + .toByteArray + + def orsetToProto(orset: ORSet[_]): ReplicatedEventSourcing.ORSet = + orsetToProtoImpl(orset.asInstanceOf[ORSet[Any]]) + + private def orsetToProtoImpl(orset: ORSet[Any]): ReplicatedEventSourcing.ORSet = { + val b = + ReplicatedEventSourcing.ORSet + .newBuilder() + .setOriginDc(orset.originReplica) + .setVvector(versionVectorToProto(orset.vvector)) + // using java collections and sorting for performance (avoid conversions) + val stringElements = new ArrayList[String] + val intElements = new ArrayList[Integer] + val longElements = new ArrayList[jl.Long] + val otherElements = new ArrayList[Payload] + var otherElementsMap = Map.empty[Payload, Any] + orset.elementsMap.keysIterator.foreach { + case s: String => stringElements.add(s) + case i: Int => intElements.add(i) + case l: Long => longElements.add(l) + case other => + val enclosedMsg = wrappedSupport.payloadBuilder(other).build() + otherElements.add(enclosedMsg) + // need the mapping back to the `other` when adding dots + otherElementsMap = otherElementsMap.updated(enclosedMsg, other) + } + + def addDots(elements: ArrayList[_]): Unit = { + // add corresponding dots in same order + val iter = elements.iterator + while (iter.hasNext) { + val element = iter.next() match { + case enclosedMsg: Payload => otherElementsMap(enclosedMsg) + case e => e + } + b.addDots(versionVectorToProto(orset.elementsMap(element))) + } + } + + if (!stringElements.isEmpty) { + Collections.sort(stringElements) + b.addAllStringElements(stringElements) + addDots(stringElements) + } + if (!intElements.isEmpty) { + Collections.sort(intElements) + b.addAllIntElements(intElements) + addDots(intElements) + } + if (!longElements.isEmpty) { + Collections.sort(longElements) + b.addAllLongElements(longElements) + addDots(longElements) + } + if (!otherElements.isEmpty) { + Collections.sort(otherElements, ReplicatedEventSourcingSerializer.Comparator) + b.addAllOtherElements(otherElements) + addDots(otherElements) + } + + b.build() + } + + def replicatedEventMetadataToProto(rem: ReplicatedEventMetadata): ReplicatedEventSourcing.ReplicatedEventMetadata = { + ReplicatedEventSourcing.ReplicatedEventMetadata + .newBuilder() + .setOriginSequenceNr(rem.originSequenceNr) + .setConcurrent(rem.concurrent) + .setOriginReplica(rem.originReplica.id) + .setVersionVector(versionVectorToProto(rem.version)) + .build() + } + + def replicatedSnapshotMetadataToByteArray(rsm: ReplicatedSnapshotMetadata): Array[Byte] = { + ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .newBuilder() + .setVersion(versionVectorToProto(rsm.version)) + .addAllSeenPerReplica(rsm.seenPerReplica.map(seenToProto).asJava) + .build() + .toByteArray + } + + def seenToProto(t: (ReplicaId, Long)): ReplicatedEventSourcing.ReplicatedSnapshotMetadata.Seen = { + ReplicatedEventSourcing.ReplicatedSnapshotMetadata.Seen + .newBuilder() + .setReplicaId(t._1.id) + .setSequenceNr(t._2) + .build() + } + + def orsetFromBinary(bytes: Array[Byte]): ORSet[Any] = + orsetFromProto(ReplicatedEventSourcing.ORSet.parseFrom(bytes)) + + private def orsetAddFromBinary(bytes: Array[Byte]): ORSet.AddDeltaOp[Any] = + new ORSet.AddDeltaOp(orsetFromProto(ReplicatedEventSourcing.ORSet.parseFrom(bytes))) + + private def orsetRemoveFromBinary(bytes: Array[Byte]): ORSet.RemoveDeltaOp[Any] = + new ORSet.RemoveDeltaOp(orsetFromProto(ReplicatedEventSourcing.ORSet.parseFrom(bytes))) + + private def orsetFullFromBinary(bytes: Array[Byte]): ORSet.FullStateDeltaOp[Any] = + new ORSet.FullStateDeltaOp(orsetFromProto(ReplicatedEventSourcing.ORSet.parseFrom(bytes))) + + private def orsetDeltaGroupToProto(deltaGroup: ORSet.DeltaGroup[_]): ReplicatedEventSourcing.ORSetDeltaGroup = { + def createEntry(opType: ReplicatedEventSourcing.ORSetDeltaOp, u: ORSet[_]) = { + ReplicatedEventSourcing.ORSetDeltaGroup.Entry.newBuilder().setOperation(opType).setUnderlying(orsetToProto(u)) + } + + val b = ReplicatedEventSourcing.ORSetDeltaGroup.newBuilder() + deltaGroup.ops.foreach { + case ORSet.AddDeltaOp(u) => + b.addEntries(createEntry(ReplicatedEventSourcing.ORSetDeltaOp.Add, u)) + case ORSet.RemoveDeltaOp(u) => + b.addEntries(createEntry(ReplicatedEventSourcing.ORSetDeltaOp.Remove, u)) + case ORSet.FullStateDeltaOp(u) => + b.addEntries(createEntry(ReplicatedEventSourcing.ORSetDeltaOp.Full, u)) + case ORSet.DeltaGroup(_) => + throw new IllegalArgumentException("ORSet.DeltaGroup should not be nested") + } + b.build() + } + + private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = { + val deltaGroup = ReplicatedEventSourcing.ORSetDeltaGroup.parseFrom(bytes) + val ops: Vector[ORSet.DeltaOp] = + deltaGroup.getEntriesList.asScala.map { entry => + if (entry.getOperation == ReplicatedEventSourcing.ORSetDeltaOp.Add) + ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)) + else if (entry.getOperation == ReplicatedEventSourcing.ORSetDeltaOp.Remove) + ORSet.RemoveDeltaOp(orsetFromProto(entry.getUnderlying)) + else if (entry.getOperation == ReplicatedEventSourcing.ORSetDeltaOp.Full) + ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying)) + else + throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}") + }.toVector + ORSet.DeltaGroup(ops) + } + + def orsetFromProto(orset: ReplicatedEventSourcing.ORSet): ORSet[Any] = { + val elements: Iterator[Any] = + (orset.getStringElementsList.iterator.asScala ++ + orset.getIntElementsList.iterator.asScala ++ + orset.getLongElementsList.iterator.asScala ++ + orset.getOtherElementsList.iterator.asScala.map(wrappedSupport.deserializePayload)) + + val dots = orset.getDotsList.asScala.map(versionVectorFromProto).iterator + val elementsMap = elements.zip(dots).toMap + + new ORSet(orset.getOriginDc, elementsMap, vvector = versionVectorFromProto(orset.getVvector)) + } + + def versionVectorToProto(versionVector: VersionVector): ReplicatedEventSourcing.VersionVector = { + val b = ReplicatedEventSourcing.VersionVector.newBuilder() + versionVector.versionsIterator.foreach { + case (key, value) => + b.addEntries(ReplicatedEventSourcing.VersionVector.Entry.newBuilder().setKey(key).setVersion(value)) + } + b.build() + } + + def versionVectorFromBinary(bytes: Array[Byte]): VersionVector = + versionVectorFromProto(ReplicatedEventSourcing.VersionVector.parseFrom(bytes)) + + def versionVectorFromProto(versionVector: ReplicatedEventSourcing.VersionVector): VersionVector = { + val entries = versionVector.getEntriesList + if (entries.isEmpty) + VersionVector.empty + else if (entries.size == 1) + VersionVector(entries.get(0).getKey, entries.get(0).getVersion) + else { + val versions = TreeMap.empty[String, Long] ++ versionVector.getEntriesList.asScala.map(entry => + entry.getKey -> entry.getVersion) + VersionVector(versions) + } + } + + def replicatedEventMetadataFromBinary(bytes: Array[Byte]): ReplicatedEventMetadata = { + val parsed = ReplicatedEventSourcing.ReplicatedEventMetadata.parseFrom(bytes) + metadataFromProto(parsed) + } + + private def metadataFromProto(parsed: ReplicatedEventSourcing.ReplicatedEventMetadata): ReplicatedEventMetadata = { + ReplicatedEventMetadata( + ReplicaId(parsed.getOriginReplica), + parsed.getOriginSequenceNr, + versionVectorFromProto(parsed.getVersionVector), + parsed.getConcurrent) + } + + def replicatedSnapshotMetadataFromBinary(bytes: Array[Byte]): ReplicatedSnapshotMetadata = { + val parsed: ReplicatedEventSourcing.ReplicatedSnapshotMetadata = + ReplicatedEventSourcing.ReplicatedSnapshotMetadata.parseFrom(bytes) + ReplicatedSnapshotMetadata( + versionVectorFromProto(parsed.getVersion), + parsed.getSeenPerReplicaList.asScala.map(seen => ReplicaId(seen.getReplicaId) -> seen.getSequenceNr).toMap) + } + +} diff --git a/akka-persistence-typed/src/test/resources/logback-test.xml b/akka-persistence-typed/src/test/resources/logback-test.xml index 22c45c93b6..901bdbb4aa 100644 --- a/akka-persistence-typed/src/test/resources/logback-test.xml +++ b/akka-persistence-typed/src/test/resources/logback-test.xml @@ -5,7 +5,7 @@ - %date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%n + %date{ISO8601} [%-5level] [%logger] [%marker] [%X{akkaSource}] [%X{persistencePhase}] [%X{persistenceId}] - %msg %n diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSerializationSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSerializationSpec.scala new file mode 100644 index 0000000000..05c6e3fa43 --- /dev/null +++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSerializationSpec.scala @@ -0,0 +1,40 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed + +import akka.actor.testkit.typed.scaladsl.LogCapturing +import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit +import akka.persistence.typed.crdt.Counter +import akka.persistence.typed.crdt.ORSet +import akka.persistence.typed.internal.ReplicatedEventMetadata +import akka.persistence.typed.internal.ReplicatedSnapshotMetadata +import akka.persistence.typed.internal.VersionVector +import org.scalatest.wordspec.AnyWordSpecLike + +class ReplicatedEventSourcingSerializationSpec + extends ScalaTestWithActorTestKit(ClusterSingletonPersistenceSpec.config) + with AnyWordSpecLike + with LogCapturing { + + "The Replicated Event Sourcing components that needs to be serializable" must { + + "be serializable" in { + serializationTestKit.verifySerialization( + ReplicatedEventMetadata(ReplicaId("DC-A"), 2L, VersionVector.empty.increment("DC-B"), true)) + + serializationTestKit.verifySerialization( + ReplicatedSnapshotMetadata( + VersionVector.empty.increment("DC-B"), + Map(ReplicaId("DC-A") -> 1L, ReplicaId("DC-B") -> 2L))) + + serializationTestKit.verifySerialization(Counter(BigInt(24))) + serializationTestKit.verifySerialization(Counter.Updated(BigInt(1))) + serializationTestKit.verifySerialization(ORSet(ReplicaId("DC-A"))) + serializationTestKit.verifySerialization(ORSet.AddDeltaOp(ORSet(ReplicaId("DC-A")))) + // FIXME DeltaGroup? + } + } + +} diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorWatchSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorWatchSpec.scala index b6d7bfe066..ef52d0ab7e 100644 --- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorWatchSpec.scala +++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorWatchSpec.scala @@ -66,7 +66,9 @@ class EventSourcedBehaviorWatchSpec RetentionCriteria.disabled, holdingRecoveryPermit = false, settings = settings, - stashState = new StashState(context.asInstanceOf[ActorContext[InternalProtocol]], settings)) + stashState = new StashState(context.asInstanceOf[ActorContext[InternalProtocol]], settings), + replication = None, + publishEvents = false) "A typed persistent parent actor watching a child" must { diff --git a/akka-persistence-typed/src/test/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializerSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializerSpec.scala new file mode 100644 index 0000000000..fea36bf9b1 --- /dev/null +++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializerSpec.scala @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed.serialization + +import akka.actor.testkit.typed.scaladsl.LogCapturing +import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit +import akka.persistence.typed.PersistenceId +import akka.persistence.typed.ReplicaId +import akka.persistence.typed.crdt.Counter +import akka.persistence.typed.crdt.ORSet +import akka.persistence.typed.internal.PublishedEventImpl +import akka.persistence.typed.internal.ReplicatedPublishedEventMetaData +import akka.persistence.typed.internal.VersionVector +import org.scalatest.wordspec.AnyWordSpecLike + +class ReplicatedEventSourcingSerializerSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing { + + "ReplicatedEventSourcingSerializer" should { + "serializer" in { + serializationTestKit.verifySerialization(ORSet.empty(ReplicaId("R1"))) + serializationTestKit.verifySerialization(ORSet.empty(ReplicaId("R1")).add("cat")) + serializationTestKit.verifySerialization(ORSet.empty(ReplicaId("R1")).remove("cat")) + serializationTestKit.verifySerialization(ORSet.empty(ReplicaId("R1")).addAll(Set("one", "two"))) + serializationTestKit.verifySerialization(ORSet.empty(ReplicaId("R1")).removeAll(Set("one", "two"))) + + serializationTestKit.verifySerialization(Counter.empty) + serializationTestKit.verifySerialization(Counter.Updated(BigInt(10))) + serializationTestKit.verifySerialization(Counter.empty.applyOperation(Counter.Updated(BigInt(12)))) + + serializationTestKit.verifySerialization(VersionVector.empty) + serializationTestKit.verifySerialization(VersionVector.empty.updated("a", 10)) + + serializationTestKit.verifySerialization( + PublishedEventImpl( + PersistenceId.ofUniqueId("cat"), + 10, + "payload", + 1, + Some(new ReplicatedPublishedEventMetaData(ReplicaId("R1"), VersionVector.empty))), + assertEquality = false) + + serializationTestKit.verifySerialization( + PublishedEventImpl(PersistenceId.ofUniqueId("cat"), 10, "payload", 1, None), + assertEquality = false) + } + } + +} diff --git a/akka-persistence/src/main/java/akka/persistence/serialization/MessageFormats.java b/akka-persistence/src/main/java/akka/persistence/serialization/MessageFormats.java index 9cb54077c0..52a61acb07 100644 --- a/akka-persistence/src/main/java/akka/persistence/serialization/MessageFormats.java +++ b/akka-persistence/src/main/java/akka/persistence/serialization/MessageFormats.java @@ -169,6 +169,21 @@ public final class MessageFormats { * @return The timestamp. */ long getTimestamp(); + + /** + * optional .PersistentPayload metadata = 15; + * @return Whether the metadata field is set. + */ + boolean hasMetadata(); + /** + * optional .PersistentPayload metadata = 15; + * @return The metadata. + */ + akka.persistence.serialization.MessageFormats.PersistentPayload getMetadata(); + /** + * optional .PersistentPayload metadata = 15; + */ + akka.persistence.serialization.MessageFormats.PersistentPayloadOrBuilder getMetadataOrBuilder(); } /** * Protobuf type {@code PersistentMessage} @@ -272,6 +287,19 @@ public final class MessageFormats { timestamp_ = input.readSInt64(); break; } + case 122: { + akka.persistence.serialization.MessageFormats.PersistentPayload.Builder subBuilder = null; + if (((bitField0_ & 0x00000100) != 0)) { + subBuilder = metadata_.toBuilder(); + } + metadata_ = input.readMessage(akka.persistence.serialization.MessageFormats.PersistentPayload.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(metadata_); + metadata_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000100; + break; + } default: { if (!parseUnknownField( input, unknownFields, extensionRegistry, tag)) { @@ -591,6 +619,29 @@ public final class MessageFormats { return timestamp_; } + public static final int METADATA_FIELD_NUMBER = 15; + private akka.persistence.serialization.MessageFormats.PersistentPayload metadata_; + /** + * optional .PersistentPayload metadata = 15; + * @return Whether the metadata field is set. + */ + public boolean hasMetadata() { + return ((bitField0_ & 0x00000100) != 0); + } + /** + * optional .PersistentPayload metadata = 15; + * @return The metadata. + */ + public akka.persistence.serialization.MessageFormats.PersistentPayload getMetadata() { + return metadata_ == null ? akka.persistence.serialization.MessageFormats.PersistentPayload.getDefaultInstance() : metadata_; + } + /** + * optional .PersistentPayload metadata = 15; + */ + public akka.persistence.serialization.MessageFormats.PersistentPayloadOrBuilder getMetadataOrBuilder() { + return metadata_ == null ? akka.persistence.serialization.MessageFormats.PersistentPayload.getDefaultInstance() : metadata_; + } + private byte memoizedIsInitialized = -1; @java.lang.Override public final boolean isInitialized() { @@ -604,6 +655,12 @@ public final class MessageFormats { return false; } } + if (hasMetadata()) { + if (!getMetadata().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -635,6 +692,9 @@ public final class MessageFormats { if (((bitField0_ & 0x00000080) != 0)) { output.writeSInt64(14, timestamp_); } + if (((bitField0_ & 0x00000100) != 0)) { + output.writeMessage(15, getMetadata()); + } unknownFields.writeTo(output); } @@ -672,6 +732,10 @@ public final class MessageFormats { size += akka.protobufv3.internal.CodedOutputStream .computeSInt64Size(14, timestamp_); } + if (((bitField0_ & 0x00000100) != 0)) { + size += akka.protobufv3.internal.CodedOutputStream + .computeMessageSize(15, getMetadata()); + } size += unknownFields.getSerializedSize(); memoizedSize = size; return size; @@ -727,6 +791,11 @@ public final class MessageFormats { if (getTimestamp() != other.getTimestamp()) return false; } + if (hasMetadata() != other.hasMetadata()) return false; + if (hasMetadata()) { + if (!getMetadata() + .equals(other.getMetadata())) return false; + } if (!unknownFields.equals(other.unknownFields)) return false; return true; } @@ -773,6 +842,10 @@ public final class MessageFormats { hash = (53 * hash) + akka.protobufv3.internal.Internal.hashLong( getTimestamp()); } + if (hasMetadata()) { + hash = (37 * hash) + METADATA_FIELD_NUMBER; + hash = (53 * hash) + getMetadata().hashCode(); + } hash = (29 * hash) + unknownFields.hashCode(); memoizedHashCode = hash; return hash; @@ -902,6 +975,7 @@ public final class MessageFormats { if (akka.protobufv3.internal.GeneratedMessageV3 .alwaysUseFieldBuilders) { getPayloadFieldBuilder(); + getMetadataFieldBuilder(); } } @java.lang.Override @@ -927,6 +1001,12 @@ public final class MessageFormats { bitField0_ = (bitField0_ & ~0x00000040); timestamp_ = 0L; bitField0_ = (bitField0_ & ~0x00000080); + if (metadataBuilder_ == null) { + metadata_ = null; + } else { + metadataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000100); return this; } @@ -991,6 +1071,14 @@ public final class MessageFormats { result.timestamp_ = timestamp_; to_bitField0_ |= 0x00000080; } + if (((from_bitField0_ & 0x00000100) != 0)) { + if (metadataBuilder_ == null) { + result.metadata_ = metadata_; + } else { + result.metadata_ = metadataBuilder_.build(); + } + to_bitField0_ |= 0x00000100; + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -1072,6 +1160,9 @@ public final class MessageFormats { if (other.hasTimestamp()) { setTimestamp(other.getTimestamp()); } + if (other.hasMetadata()) { + mergeMetadata(other.getMetadata()); + } this.mergeUnknownFields(other.unknownFields); onChanged(); return this; @@ -1084,6 +1175,11 @@ public final class MessageFormats { return false; } } + if (hasMetadata()) { + if (!getMetadata().isInitialized()) { + return false; + } + } return true; } @@ -1737,6 +1833,126 @@ public final class MessageFormats { onChanged(); return this; } + + private akka.persistence.serialization.MessageFormats.PersistentPayload metadata_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.serialization.MessageFormats.PersistentPayload, akka.persistence.serialization.MessageFormats.PersistentPayload.Builder, akka.persistence.serialization.MessageFormats.PersistentPayloadOrBuilder> metadataBuilder_; + /** + * optional .PersistentPayload metadata = 15; + * @return Whether the metadata field is set. + */ + public boolean hasMetadata() { + return ((bitField0_ & 0x00000100) != 0); + } + /** + * optional .PersistentPayload metadata = 15; + * @return The metadata. + */ + public akka.persistence.serialization.MessageFormats.PersistentPayload getMetadata() { + if (metadataBuilder_ == null) { + return metadata_ == null ? akka.persistence.serialization.MessageFormats.PersistentPayload.getDefaultInstance() : metadata_; + } else { + return metadataBuilder_.getMessage(); + } + } + /** + * optional .PersistentPayload metadata = 15; + */ + public Builder setMetadata(akka.persistence.serialization.MessageFormats.PersistentPayload value) { + if (metadataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + metadata_ = value; + onChanged(); + } else { + metadataBuilder_.setMessage(value); + } + bitField0_ |= 0x00000100; + return this; + } + /** + * optional .PersistentPayload metadata = 15; + */ + public Builder setMetadata( + akka.persistence.serialization.MessageFormats.PersistentPayload.Builder builderForValue) { + if (metadataBuilder_ == null) { + metadata_ = builderForValue.build(); + onChanged(); + } else { + metadataBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000100; + return this; + } + /** + * optional .PersistentPayload metadata = 15; + */ + public Builder mergeMetadata(akka.persistence.serialization.MessageFormats.PersistentPayload value) { + if (metadataBuilder_ == null) { + if (((bitField0_ & 0x00000100) != 0) && + metadata_ != null && + metadata_ != akka.persistence.serialization.MessageFormats.PersistentPayload.getDefaultInstance()) { + metadata_ = + akka.persistence.serialization.MessageFormats.PersistentPayload.newBuilder(metadata_).mergeFrom(value).buildPartial(); + } else { + metadata_ = value; + } + onChanged(); + } else { + metadataBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000100; + return this; + } + /** + * optional .PersistentPayload metadata = 15; + */ + public Builder clearMetadata() { + if (metadataBuilder_ == null) { + metadata_ = null; + onChanged(); + } else { + metadataBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000100); + return this; + } + /** + * optional .PersistentPayload metadata = 15; + */ + public akka.persistence.serialization.MessageFormats.PersistentPayload.Builder getMetadataBuilder() { + bitField0_ |= 0x00000100; + onChanged(); + return getMetadataFieldBuilder().getBuilder(); + } + /** + * optional .PersistentPayload metadata = 15; + */ + public akka.persistence.serialization.MessageFormats.PersistentPayloadOrBuilder getMetadataOrBuilder() { + if (metadataBuilder_ != null) { + return metadataBuilder_.getMessageOrBuilder(); + } else { + return metadata_ == null ? + akka.persistence.serialization.MessageFormats.PersistentPayload.getDefaultInstance() : metadata_; + } + } + /** + * optional .PersistentPayload metadata = 15; + */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.serialization.MessageFormats.PersistentPayload, akka.persistence.serialization.MessageFormats.PersistentPayload.Builder, akka.persistence.serialization.MessageFormats.PersistentPayloadOrBuilder> + getMetadataFieldBuilder() { + if (metadataBuilder_ == null) { + metadataBuilder_ = new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.serialization.MessageFormats.PersistentPayload, akka.persistence.serialization.MessageFormats.PersistentPayload.Builder, akka.persistence.serialization.MessageFormats.PersistentPayloadOrBuilder>( + getMetadata(), + getParentForChildren(), + isClean()); + metadata_ = null; + } + return metadataBuilder_; + } @java.lang.Override public final Builder setUnknownFields( final akka.protobufv3.internal.UnknownFieldSet unknownFields) { @@ -7059,27 +7275,28 @@ public final class MessageFormats { descriptor; static { java.lang.String[] descriptorData = { - "\n\024MessageFormats.proto\"\275\001\n\021PersistentMes" + + "\n\024MessageFormats.proto\"\343\001\n\021PersistentMes" + "sage\022#\n\007payload\030\001 \001(\0132\022.PersistentPayloa" + "d\022\022\n\nsequenceNr\030\002 \001(\003\022\025\n\rpersistenceId\030\003" + " \001(\t\022\017\n\007deleted\030\004 \001(\010\022\016\n\006sender\030\013 \001(\t\022\020\n" + "\010manifest\030\014 \001(\t\022\022\n\nwriterUuid\030\r \001(\t\022\021\n\tt" + - "imestamp\030\016 \001(\022\"S\n\021PersistentPayload\022\024\n\014s" + - "erializerId\030\001 \002(\005\022\017\n\007payload\030\002 \002(\014\022\027\n\017pa" + - "yloadManifest\030\003 \001(\014\"2\n\013AtomicWrite\022#\n\007pa" + - "yload\030\001 \003(\0132\022.PersistentMessage\"\356\001\n\033AtLe" + - "astOnceDeliverySnapshot\022\031\n\021currentDelive" + - "ryId\030\001 \002(\003\022O\n\025unconfirmedDeliveries\030\002 \003(" + - "\01320.AtLeastOnceDeliverySnapshot.Unconfir" + - "medDelivery\032c\n\023UnconfirmedDelivery\022\022\n\nde" + - "liveryId\030\001 \002(\003\022\023\n\013destination\030\002 \002(\t\022#\n\007p" + - "ayload\030\003 \002(\0132\022.PersistentPayload\"\\\n\032Pers" + - "istentStateChangeEvent\022\027\n\017stateIdentifie" + - "r\030\001 \002(\t\022\017\n\007timeout\030\002 \001(\t\022\024\n\014timeoutNanos" + - "\030\003 \001(\003\"h\n\025PersistentFSMSnapshot\022\027\n\017state" + - "Identifier\030\001 \002(\t\022 \n\004data\030\002 \002(\0132\022.Persist" + - "entPayload\022\024\n\014timeoutNanos\030\003 \001(\003B\"\n\036akka" + - ".persistence.serializationH\001" + "imestamp\030\016 \001(\022\022$\n\010metadata\030\017 \001(\0132\022.Persi" + + "stentPayload\"S\n\021PersistentPayload\022\024\n\014ser" + + "ializerId\030\001 \002(\005\022\017\n\007payload\030\002 \002(\014\022\027\n\017payl" + + "oadManifest\030\003 \001(\014\"2\n\013AtomicWrite\022#\n\007payl" + + "oad\030\001 \003(\0132\022.PersistentMessage\"\356\001\n\033AtLeas" + + "tOnceDeliverySnapshot\022\031\n\021currentDelivery" + + "Id\030\001 \002(\003\022O\n\025unconfirmedDeliveries\030\002 \003(\0132" + + "0.AtLeastOnceDeliverySnapshot.Unconfirme" + + "dDelivery\032c\n\023UnconfirmedDelivery\022\022\n\ndeli" + + "veryId\030\001 \002(\003\022\023\n\013destination\030\002 \002(\t\022#\n\007pay" + + "load\030\003 \002(\0132\022.PersistentPayload\"\\\n\032Persis" + + "tentStateChangeEvent\022\027\n\017stateIdentifier\030" + + "\001 \002(\t\022\017\n\007timeout\030\002 \001(\t\022\024\n\014timeoutNanos\030\003" + + " \001(\003\"h\n\025PersistentFSMSnapshot\022\027\n\017stateId" + + "entifier\030\001 \002(\t\022 \n\004data\030\002 \002(\0132\022.Persisten" + + "tPayload\022\024\n\014timeoutNanos\030\003 \001(\003B\"\n\036akka.p" + + "ersistence.serializationH\001" }; descriptor = akka.protobufv3.internal.Descriptors.FileDescriptor .internalBuildGeneratedFileFrom(descriptorData, @@ -7090,7 +7307,7 @@ public final class MessageFormats { internal_static_PersistentMessage_fieldAccessorTable = new akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable( internal_static_PersistentMessage_descriptor, - new java.lang.String[] { "Payload", "SequenceNr", "PersistenceId", "Deleted", "Sender", "Manifest", "WriterUuid", "Timestamp", }); + new java.lang.String[] { "Payload", "SequenceNr", "PersistenceId", "Deleted", "Sender", "Manifest", "WriterUuid", "Timestamp", "Metadata", }); internal_static_PersistentPayload_descriptor = getDescriptor().getMessageTypes().get(1); internal_static_PersistentPayload_fieldAccessorTable = new diff --git a/akka-persistence/src/main/mima-filters/2.6.8.backwards.excludes/29217-active-active-event-sourcing.excludes b/akka-persistence/src/main/mima-filters/2.6.8.backwards.excludes/29217-active-active-event-sourcing.excludes new file mode 100644 index 0000000000..4ac25cf0b7 --- /dev/null +++ b/akka-persistence/src/main/mima-filters/2.6.8.backwards.excludes/29217-active-active-event-sourcing.excludes @@ -0,0 +1,16 @@ +# Changes to internal/private + +ProblemFilters.exclude[IncompatibleSignatureProblem]("akka.persistence.journal.inmem.InmemMessages.*") +ProblemFilters.exclude[IncompatibleSignatureProblem]("akka.persistence.journal.inmem.InmemJournal.*") + +# marked as do not inherit +ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.PersistentRepr.metadata") +ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.PersistentRepr.withMetadata") + +# changes to internal class +ProblemFilters.exclude[Problem]("akka.persistence.PersistentImpl*") + +# protobuf +ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.serialization.MessageFormats#PersistentMessageOrBuilder.hasMetadata") +ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.serialization.MessageFormats#PersistentMessageOrBuilder.getMetadata") +ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.serialization.MessageFormats#PersistentMessageOrBuilder.getMetadataOrBuilder") diff --git a/akka-persistence/src/main/protobuf/MessageFormats.proto b/akka-persistence/src/main/protobuf/MessageFormats.proto index e5954743f1..3bdc13b5f1 100644 --- a/akka-persistence/src/main/protobuf/MessageFormats.proto +++ b/akka-persistence/src/main/protobuf/MessageFormats.proto @@ -21,6 +21,7 @@ message PersistentMessage { optional string manifest = 12; optional string writerUuid = 13; optional sint64 timestamp = 14; + optional PersistentPayload metadata = 15; } message PersistentPayload { diff --git a/akka-persistence/src/main/scala/akka/persistence/Persistent.scala b/akka-persistence/src/main/scala/akka/persistence/Persistent.scala index f260f79124..159b3ae5d3 100644 --- a/akka-persistence/src/main/scala/akka/persistence/Persistent.scala +++ b/akka-persistence/src/main/scala/akka/persistence/Persistent.scala @@ -102,6 +102,10 @@ final case class AtomicWrite(payload: immutable.Seq[PersistentRepr]) extends Per def withTimestamp(newTimestamp: Long): PersistentRepr + def metadata: Option[Any] + + def withMetadata(metadata: Any): PersistentRepr + /** * Unique identifier of the writing persistent actor. * Used to detect anomalies with overlapping writes from multiple @@ -163,7 +167,7 @@ object PersistentRepr { deleted: Boolean = false, sender: ActorRef = null, writerUuid: String = PersistentRepr.Undefined): PersistentRepr = - PersistentImpl(payload, sequenceNr, persistenceId, manifest, deleted, sender, writerUuid, 0L) + PersistentImpl(payload, sequenceNr, persistenceId, manifest, deleted, sender, writerUuid, 0L, None) /** * Java API, Plugin API. @@ -188,7 +192,8 @@ private[persistence] final case class PersistentImpl( override val deleted: Boolean, override val sender: ActorRef, override val writerUuid: String, - override val timestamp: Long) + override val timestamp: Long, + override val metadata: Option[Any]) extends PersistentRepr with NoSerializationVerificationNeeded { @@ -203,6 +208,10 @@ private[persistence] final case class PersistentImpl( if (this.timestamp == newTimestamp) this else copy(timestamp = newTimestamp) + override def withMetadata(metadata: Any): PersistentRepr = { + copy(metadata = Some(metadata)) + } + def update(sequenceNr: Long, persistenceId: String, deleted: Boolean, sender: ActorRef, writerUuid: String) = copy( sequenceNr = sequenceNr, @@ -221,6 +230,7 @@ private[persistence] final case class PersistentImpl( result = HashCode.hash(result, sender) result = HashCode.hash(result, writerUuid) // timestamp not included in equals for backwards compatibility + // meta not included in equals for backwards compatibility result } @@ -233,7 +243,6 @@ private[persistence] final case class PersistentImpl( } override def toString: String = { - s"PersistentRepr($persistenceId,$sequenceNr,$writerUuid,$timestamp)" + s"PersistentRepr($persistenceId,$sequenceNr,$writerUuid,$timestamp,$metadata)" } - } diff --git a/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala b/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala index ec4559ce1b..d1d8f151ef 100644 --- a/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala +++ b/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala @@ -3,6 +3,7 @@ */ package akka.persistence +import scala.runtime.AbstractFunction3 /** * Snapshot metadata. @@ -10,12 +11,58 @@ package akka.persistence * @param persistenceId id of persistent actor from which the snapshot was taken. * @param sequenceNr sequence number at which the snapshot was taken. * @param timestamp time at which the snapshot was saved, defaults to 0 when unknown. + * @param metadata a journal can optionally support persisting metadata separate to the domain state, used for Replicated Event Sourcing support */ -@SerialVersionUID(1L) //#snapshot-metadata -final case class SnapshotMetadata(persistenceId: String, sequenceNr: Long, timestamp: Long = 0L) -//#snapshot-metadata +@SerialVersionUID(1L) +final class SnapshotMetadata( + val persistenceId: String, + val sequenceNr: Long, + val timestamp: Long, + val metadata: Option[Any]) + extends Product3[String, Long, Long] + with Serializable { -object SnapshotMetadata { + def this(persistenceId: String, sequenceNr: Long, timestamp: Long) = { + this(persistenceId, sequenceNr, timestamp, None) + } + + private[akka] def this(persistenceId: String, sequenceNr: Long, meta: Option[Any]) = { + this(persistenceId, sequenceNr, 0L, meta) + } + + def withMetadata(metadata: Any): SnapshotMetadata = + new SnapshotMetadata(persistenceId, sequenceNr, timestamp, Some(metadata)) + + // for bincompat, used to be a case class + def copy( + persistenceId: String = this.persistenceId, + sequenceNr: Long = this.sequenceNr, + timestamp: Long = this.timestamp): SnapshotMetadata = + SnapshotMetadata(persistenceId, sequenceNr, timestamp, metadata) + + override def toString = s"SnapshotMetadata($persistenceId, $sequenceNr, $timestamp, $metadata)" + + // Product 3 + override def productPrefix = "SnapshotMetadata" + override def _1: String = persistenceId + override def _2: Long = sequenceNr + override def _3: Long = timestamp + override def canEqual(that: Any): Boolean = that.isInstanceOf[SnapshotMetadata] + + override def equals(other: Any): Boolean = other match { + case that: SnapshotMetadata => + persistenceId == that.persistenceId && + sequenceNr == that.sequenceNr && + timestamp == that.timestamp + case _ => false + } + override def hashCode(): Int = { + val state = Seq[Any](persistenceId, sequenceNr, timestamp) + state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) + } +} + +object SnapshotMetadata extends AbstractFunction3[String, Long, Long, SnapshotMetadata] { implicit val ordering: Ordering[SnapshotMetadata] = Ordering.fromLessThan[SnapshotMetadata] { (a, b) => if (a eq b) false else if (a.persistenceId != b.persistenceId) a.persistenceId.compareTo(b.persistenceId) < 0 @@ -23,6 +70,22 @@ object SnapshotMetadata { else if (a.timestamp != b.timestamp) a.timestamp < b.timestamp else false } + + def apply(persistenceId: String, sequenceNr: Long, timestamp: Long, meta: Option[Any]): SnapshotMetadata = + new SnapshotMetadata(persistenceId, sequenceNr, timestamp, meta) + + def apply(persistenceId: String, sequenceNr: Long, timestamp: Long): SnapshotMetadata = + new SnapshotMetadata(persistenceId, sequenceNr, timestamp, None) + + def apply(persistenceId: String, sequenceNr: Long): SnapshotMetadata = + new SnapshotMetadata(persistenceId, sequenceNr, 0, None) + + def unapply(sm: SnapshotMetadata): Option[(String, Long, Long)] = + Some((sm.persistenceId, sm.sequenceNr, sm.timestamp)) + + def apply$default$3(): Long = 0L + + def `$default$3`: Long = 0L } /** diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/inmem/InmemJournal.scala b/akka-persistence/src/main/scala/akka/persistence/journal/inmem/InmemJournal.scala index be2f0122ef..d07b37c3d3 100644 --- a/akka-persistence/src/main/scala/akka/persistence/journal/inmem/InmemJournal.scala +++ b/akka-persistence/src/main/scala/akka/persistence/journal/inmem/InmemJournal.scala @@ -4,21 +4,25 @@ package akka.persistence.journal.inmem +import akka.actor.ActorRef + import scala.collection.immutable import scala.concurrent.Future import scala.util.Try import scala.util.control.NonFatal - import com.typesafe.config.Config import com.typesafe.config.ConfigFactory - import akka.annotation.ApiMayChange import akka.annotation.InternalApi +import akka.event.Logging import akka.persistence.AtomicWrite +import akka.persistence.JournalProtocol.RecoverySuccess import akka.persistence.PersistentRepr +import akka.persistence.journal.inmem.InmemJournal.{ MessageWithMeta, ReplayWithMeta } import akka.persistence.journal.{ AsyncWriteJournal, Tagged } import akka.serialization.SerializationExtension import akka.serialization.Serializers +import akka.util.OptionVal /** * The InmemJournal publishes writes and deletes to the `eventStream`, which tests may use to @@ -32,8 +36,17 @@ object InmemJournal { sealed trait Operation final case class Write(event: Any, persistenceId: String, sequenceNr: Long) extends Operation - final case class Delete(persistenceId: String, toSequenceNr: Long) extends Operation + + @InternalApi + private[persistence] case class ReplayWithMeta( + from: Long, + to: Long, + limit: Long, + persistenceId: String, + replyTo: ActorRef) + @InternalApi + private[persistence] case class MessageWithMeta(pr: PersistentRepr, meta: OptionVal[Any]) } /** @@ -45,6 +58,8 @@ object InmemJournal { def this() = this(ConfigFactory.empty()) + private val log = Logging(context.system, classOf[InmemJournal]) + private val testSerialization = { val key = "test-serialization" if (cfg.hasPath(key)) cfg.getBoolean("test-serialization") @@ -78,7 +93,9 @@ object InmemJournal { recoveryCallback: PersistentRepr => Unit): Future[Unit] = { val highest = highestSequenceNr(persistenceId) if (highest != 0L && max != 0L) - read(persistenceId, fromSequenceNr, math.min(toSequenceNr, highest), max).foreach(recoveryCallback) + read(persistenceId, fromSequenceNr, math.min(toSequenceNr, highest), max).foreach { + case (pr, _) => recoveryCallback(pr) + } Future.successful(()) } @@ -93,6 +110,19 @@ object InmemJournal { Future.successful(()) } + override def receivePluginInternal: Receive = { + case ReplayWithMeta(fromSequenceNr, toSequenceNr, max, persistenceId, replyTo) => + log.debug("ReplayWithMeta {} {} {} {}", fromSequenceNr, toSequenceNr, max, persistenceId) + val highest = highestSequenceNr(persistenceId) + if (highest != 0L && max != 0L) { + read(persistenceId, fromSequenceNr, math.min(toSequenceNr, highest), max).foreach { + case (pr, meta) => replyTo ! MessageWithMeta(pr, meta) + } + } + replyTo ! RecoverySuccess(highest) + + } + private def verifySerialization(event: Any): Unit = { if (testSerialization) { val eventAnyRef = event.asInstanceOf[AnyRef] @@ -109,31 +139,32 @@ object InmemJournal { */ @InternalApi private[persistence] trait InmemMessages { // persistenceId -> persistent message - var messages = Map.empty[String, Vector[PersistentRepr]] + var messages = Map.empty[String, Vector[(PersistentRepr, OptionVal[Any])]] // persistenceId -> highest used sequence number private var highestSequenceNumbers = Map.empty[String, Long] def add(p: PersistentRepr): Unit = { val pr = p.payload match { - case Tagged(payload, _) => p.withPayload(payload) - case _ => p + case Tagged(payload, _) => (p.withPayload(payload).withTimestamp(System.currentTimeMillis()), OptionVal.None) + case _ => (p.withTimestamp(System.currentTimeMillis()), OptionVal.None) } - messages = messages + (messages.get(pr.persistenceId) match { - case Some(ms) => pr.persistenceId -> (ms :+ pr) - case None => pr.persistenceId -> Vector(pr) + + messages = messages + (messages.get(p.persistenceId) match { + case Some(ms) => p.persistenceId -> (ms :+ pr) + case None => p.persistenceId -> Vector(pr) }) highestSequenceNumbers = - highestSequenceNumbers.updated(pr.persistenceId, math.max(highestSequenceNr(pr.persistenceId), pr.sequenceNr)) + highestSequenceNumbers.updated(p.persistenceId, math.max(highestSequenceNr(p.persistenceId), p.sequenceNr)) } def delete(pid: String, snr: Long): Unit = messages = messages.get(pid) match { - case Some(ms) => messages + (pid -> ms.filterNot(_.sequenceNr == snr)) + case Some(ms) => messages + (pid -> ms.filterNot(_._1.sequenceNr == snr)) case None => messages } - def read(pid: String, fromSnr: Long, toSnr: Long, max: Long): immutable.Seq[PersistentRepr] = + def read(pid: String, fromSnr: Long, toSnr: Long, max: Long): immutable.Seq[(PersistentRepr, OptionVal[Any])] = messages.get(pid) match { - case Some(ms) => ms.filter(m => m.sequenceNr >= fromSnr && m.sequenceNr <= toSnr).take(safeLongToInt(max)) + case Some(ms) => ms.filter(m => m._1.sequenceNr >= fromSnr && m._1.sequenceNr <= toSnr).take(safeLongToInt(max)) case None => Nil } diff --git a/akka-persistence/src/main/scala/akka/persistence/serialization/MessageSerializer.scala b/akka-persistence/src/main/scala/akka/persistence/serialization/MessageSerializer.scala index 8916b0bb7a..11a5540fc4 100644 --- a/akka-persistence/src/main/scala/akka/persistence/serialization/MessageSerializer.scala +++ b/akka-persistence/src/main/scala/akka/persistence/serialization/MessageSerializer.scala @@ -166,6 +166,12 @@ class MessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer if (persistent.manifest != PersistentRepr.Undefined) builder.setManifest(persistent.manifest) builder.setPayload(persistentPayloadBuilder(persistent.payload.asInstanceOf[AnyRef])) + persistent.metadata match { + case Some(meta) => + builder.setMetadata(persistentPayloadBuilder(meta.asInstanceOf[AnyRef])) + case _ => + } + builder.setSequenceNr(persistent.sequenceNr) // deleted is not used in new records from 2.4 if (persistent.writerUuid != Undefined) builder.setWriterUuid(persistent.writerUuid) @@ -199,7 +205,7 @@ class MessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer // private def persistent(persistentMessage: mf.PersistentMessage): PersistentRepr = { - val repr = PersistentRepr( + var repr = PersistentRepr( payload(persistentMessage.getPayload), persistentMessage.getSequenceNr, if (persistentMessage.hasPersistenceId) persistentMessage.getPersistenceId else Undefined, @@ -209,7 +215,8 @@ class MessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer else Actor.noSender, if (persistentMessage.hasWriterUuid) persistentMessage.getWriterUuid else Undefined) - if (persistentMessage.hasTimestamp) repr.withTimestamp(persistentMessage.getTimestamp) else repr + repr = if (persistentMessage.hasTimestamp) repr.withTimestamp(persistentMessage.getTimestamp) else repr + if (persistentMessage.hasMetadata) repr.withMetadata(payload(persistentMessage.getMetadata)) else repr } private def atomicWrite(atomicWrite: mf.AtomicWrite): AtomicWrite = { diff --git a/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorFailureSpec.scala b/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorFailureSpec.scala index 0712f9af4c..1b008cebb5 100644 --- a/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorFailureSpec.scala +++ b/akka-persistence/src/test/scala/akka/persistence/EventSourcedActorFailureSpec.scala @@ -39,10 +39,10 @@ object EventSourcedActorFailureSpec { val readFromStore = read(persistenceId, fromSequenceNr, toSequenceNr, max) if (readFromStore.isEmpty) Future.successful(()) - else if (isCorrupt(readFromStore)) + else if (isCorrupt(readFromStore.map(_._1))) Future.failed(new SimulatedException(s"blahonga $fromSequenceNr $toSequenceNr")) else { - readFromStore.foreach(recoveryCallback) + readFromStore.map(_._1).foreach(recoveryCallback) Future.successful(()) } } diff --git a/akka-persistence/src/test/scala/akka/persistence/journal/chaos/ChaosJournal.scala b/akka-persistence/src/test/scala/akka/persistence/journal/chaos/ChaosJournal.scala index 1381222971..393c5c4ba9 100644 --- a/akka-persistence/src/test/scala/akka/persistence/journal/chaos/ChaosJournal.scala +++ b/akka-persistence/src/test/scala/akka/persistence/journal/chaos/ChaosJournal.scala @@ -66,11 +66,11 @@ class ChaosJournal extends AsyncWriteJournal { replayCallback: (PersistentRepr) => Unit): Future[Unit] = if (shouldFail(replayFailureRate)) { val rm = read(persistenceId, fromSequenceNr, toSequenceNr, max) - val sm = rm.take(random.nextInt(rm.length + 1)) + val sm = rm.take(random.nextInt(rm.length + 1)).map(_._1) sm.foreach(replayCallback) Future.failed(new ReplayFailedException(sm)) } else { - read(persistenceId, fromSequenceNr, toSequenceNr, max).foreach(replayCallback) + read(persistenceId, fromSequenceNr, toSequenceNr, max).map(_._1).foreach(replayCallback) Future.successful(()) } diff --git a/akka-persistence/src/test/scala/akka/persistence/serialization/MessageSerializerSpec.scala b/akka-persistence/src/test/scala/akka/persistence/serialization/MessageSerializerSpec.scala new file mode 100644 index 0000000000..added09810 --- /dev/null +++ b/akka-persistence/src/test/scala/akka/persistence/serialization/MessageSerializerSpec.scala @@ -0,0 +1,22 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.serialization + +import akka.persistence.PersistentRepr +import akka.serialization.SerializationExtension +import akka.testkit.AkkaSpec + +class MessageSerializerSpec extends AkkaSpec { + + "Message serializer" should { + "serialize metadata for persistent repr" in { + val pr = PersistentRepr("payload", 1L, "pid1").withMetadata("meta") + val serialization = SerializationExtension(system) + val deserialzied = serialization.deserialize(serialization.serialize(pr).get, classOf[PersistentRepr]).get + deserialzied.metadata shouldEqual Some("meta") + } + } + +} diff --git a/akka-serialization-jackson/src/main/resources/reference.conf b/akka-serialization-jackson/src/main/resources/reference.conf index d06d1fce3d..fd041f9012 100644 --- a/akka-serialization-jackson/src/main/resources/reference.conf +++ b/akka-serialization-jackson/src/main/resources/reference.conf @@ -12,7 +12,6 @@ akka.serialization.jackson { jackson-modules += "akka.serialization.jackson.AkkaJacksonModule" # AkkaTypedJacksonModule optionally included if akka-actor-typed is in classpath jackson-modules += "akka.serialization.jackson.AkkaTypedJacksonModule" - // FIXME how does that optional loading work?? # AkkaStreamsModule optionally included if akka-streams is in classpath jackson-modules += "akka.serialization.jackson.AkkaStreamJacksonModule" jackson-modules += "com.fasterxml.jackson.module.paramnames.ParameterNamesModule" diff --git a/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/AkkaSerializationModule.scala b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/AkkaSerializationModule.scala new file mode 100644 index 0000000000..406a07f62e --- /dev/null +++ b/akka-serialization-jackson/src/main/scala/akka/serialization/jackson/AkkaSerializationModule.scala @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2019-2020 Lightbend Inc. + */ + +package akka.serialization.jackson + +import com.fasterxml.jackson.core.{ JsonGenerator, JsonParser, ObjectCodec } +import com.fasterxml.jackson.databind.{ DeserializationContext, JsonNode, SerializerProvider } +import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer +import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer +import akka.serialization.{ SerializationExtension, Serializer, Serializers } + +final class AkkaSerializationSerializer extends StdScalarSerializer[AnyRef](classOf[AnyRef]) with ActorSystemAccess { + def serialization = SerializationExtension(currentSystem()) + override def serialize(value: AnyRef, jgen: JsonGenerator, provider: SerializerProvider): Unit = { + val serializer: Serializer = serialization.findSerializerFor(value) + val serId = serializer.identifier + val manifest = Serializers.manifestFor(serializer, value) + val serialized = serializer.toBinary(value) + jgen.writeStartObject() + jgen.writeStringField("serId", serId.toString) + jgen.writeStringField("serManifest", manifest) + jgen.writeBinaryField("payload", serialized) + jgen.writeEndObject() + } +} + +final class AkkaSerializationDeserializer + extends StdScalarDeserializer[AnyRef](classOf[AnyRef]) + with ActorSystemAccess { + + def serialization = SerializationExtension(currentSystem()) + + def deserialize(jp: JsonParser, ctxt: DeserializationContext): AnyRef = { + val codec: ObjectCodec = jp.getCodec() + val jsonNode = codec.readTree[JsonNode](jp) + val id = jsonNode.get("serId").textValue().toInt + val manifest = jsonNode.get("serManifest").textValue() + val payload = jsonNode.get("payload").binaryValue() + serialization.deserialize(payload, id, manifest).get + } +} diff --git a/akka-serialization-jackson/src/test/resources/reference.conf b/akka-serialization-jackson/src/test/resources/reference.conf index d75292e3c0..668c907f19 100644 --- a/akka-serialization-jackson/src/test/resources/reference.conf +++ b/akka-serialization-jackson/src/test/resources/reference.conf @@ -2,6 +2,7 @@ akka { actor { serialization-bindings { "akka.serialization.jackson.CborSerializable" = jackson-cbor + "akka.serialization.jackson.JsonSerializable" = jackson-json } } } diff --git a/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JacksonSerializerSpec.scala b/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JacksonSerializerSpec.scala index 29229a0f8b..38780bac8a 100644 --- a/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JacksonSerializerSpec.scala +++ b/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JacksonSerializerSpec.scala @@ -14,16 +14,13 @@ import java.util.Arrays import java.util.Locale import java.util.Optional import java.util.UUID - import java.util.logging.FileHandler import scala.collection.immutable +import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration._ -import akka.actor.ActorRef -import akka.actor.ActorSystem -import akka.actor.Address -import akka.actor.BootstrapSetup -import akka.actor.ExtendedActorSystem + +import com.fasterxml.jackson.annotation.JsonIgnore import com.fasterxml.jackson.annotation.JsonSubTypes import com.fasterxml.jackson.annotation.JsonTypeInfo import com.fasterxml.jackson.core.JsonFactory @@ -37,6 +34,8 @@ import com.fasterxml.jackson.databind.MapperFeature import com.fasterxml.jackson.databind.Module import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.databind.SerializationFeature +import com.fasterxml.jackson.databind.annotation.JsonDeserialize +import com.fasterxml.jackson.databind.annotation.JsonSerialize import com.fasterxml.jackson.databind.exc.InvalidTypeIdException import com.fasterxml.jackson.databind.json.JsonMapper import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule @@ -47,12 +46,17 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike -import scala.concurrent.duration.FiniteDuration +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.actor.Address +import akka.actor.BootstrapSetup +import akka.actor.ExtendedActorSystem import akka.actor.Status import akka.actor.setup.ActorSystemSetup import akka.actor.typed.scaladsl.Behaviors import akka.serialization.Serialization import akka.serialization.SerializationExtension +import akka.serialization.SerializerWithStringManifest import akka.testkit.TestActors import akka.testkit.TestKit @@ -114,6 +118,42 @@ object ScalaTestMessages { extends TestMessage // #jackson-scala-enumeration + //delegate to AkkaSerialization + object HasAkkaSerializer { + def apply(description: String): HasAkkaSerializer = new HasAkkaSerializer(description) + } + // make sure jackson would fail + class HasAkkaSerializer private (@JsonIgnore val description: String) { + + override def toString: String = s"InnerSerialization($description)" + + def canEqual(other: Any): Boolean = other.isInstanceOf[HasAkkaSerializer] + + override def equals(other: Any): Boolean = other match { + case that: HasAkkaSerializer => + (that.canEqual(this)) && + description == that.description + case _ => false + } + + override def hashCode(): Int = { + val state = Seq(description) + state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) + } + } + + class InnerSerializationSerializer extends SerializerWithStringManifest { + override def identifier: Int = 123451 + override def manifest(o: AnyRef): String = "M" + override def toBinary(o: AnyRef): Array[Byte] = o.asInstanceOf[HasAkkaSerializer].description.getBytes() + override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = HasAkkaSerializer(new String(bytes)) + } + + final case class WithAkkaSerializer( + @JsonDeserialize(using = classOf[AkkaSerializationDeserializer]) + @JsonSerialize(using = classOf[AkkaSerializationSerializer]) + akkaSerializer: HasAkkaSerializer) + extends TestMessage } class JacksonCborSerializerSpec extends JacksonSerializerSpec("jackson-cbor") { @@ -614,6 +654,15 @@ object JacksonSerializerSpec { } } akka.serialization.jackson.allowed-class-prefix = ["akka.serialization.jackson.ScalaTestMessages$$OldCommand"] + + akka.actor { + serializers { + inner-serializer = "akka.serialization.jackson.ScalaTestMessages$$InnerSerializationSerializer" + } + serialization-bindings { + "akka.serialization.jackson.ScalaTestMessages$$HasAkkaSerializer" = "inner-serializer" + } + } """ } @@ -1163,6 +1212,10 @@ abstract class JacksonSerializerSpec(serializerName: String) } } + "delegate to akka serialization" in { + checkSerialization(WithAkkaSerializer(HasAkkaSerializer("cat"))) + } + } } diff --git a/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JsonSerializable.scala b/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JsonSerializable.scala new file mode 100644 index 0000000000..d0ec9e82b1 --- /dev/null +++ b/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JsonSerializable.scala @@ -0,0 +1,10 @@ +/* + * Copyright (C) 2019-2020 Lightbend Inc. + */ + +package akka.serialization.jackson + +/** + * Marker trait for serialization with Jackson JSON in tests + */ +trait JsonSerializable diff --git a/build.sbt b/build.sbt index 1abf2356e0..04c18a061f 100644 --- a/build.sbt +++ b/build.sbt @@ -82,6 +82,7 @@ lazy val aggregatedProjects: Seq[ProjectReference] = List[ProjectReference]( persistenceShared, persistenceTck, persistenceTyped, + persistenceTypedTests, persistenceTestkit, protobuf, protobufV3, @@ -314,6 +315,14 @@ lazy val persistenceTestkit = akkaModule("akka-persistence-testkit") .settings(AutomaticModuleName.settings("akka.persistence.testkit")) .disablePlugins(MimaPlugin) +lazy val persistenceTypedTests = akkaModule("akka-persistence-typed-tests") + .dependsOn(persistenceTyped, persistenceTestkit % "test", actorTestkitTyped % "test", jackson % "test->test") + .settings(AkkaBuild.mayChangeSettings) + .settings(Dependencies.persistenceTypedTests) + .settings(javacOptions += "-parameters") // for Jackson + .disablePlugins(MimaPlugin) + .enablePlugins(NoPublish) + lazy val protobuf = akkaModule("akka-protobuf") .settings(OSGi.protobuf) .settings(AutomaticModuleName.settings("akka.protobuf")) @@ -455,8 +464,10 @@ lazy val actorTyped = akkaModule("akka-actor-typed") lazy val persistenceTyped = akkaModule("akka-persistence-typed") .dependsOn( actorTyped, + streamTyped, + remote, persistence % "compile->compile;test->test", - persistenceQuery % "test", + persistenceQuery, actorTestkitTyped % "test->test", clusterTyped % "test->test", actorTestkitTyped % "test->test", @@ -464,6 +475,9 @@ lazy val persistenceTyped = akkaModule("akka-persistence-typed") .settings(javacOptions += "-parameters") // for Jackson .settings(Dependencies.persistenceShared) .settings(AutomaticModuleName.settings("akka.persistence.typed")) + .settings(Protobuf.settings) + // To be able to import ContainerFormats.proto + .settings(Protobuf.importPath := Some(baseDirectory.value / ".." / "akka-remote" / "src" / "main" / "protobuf")) .settings(OSGi.persistenceTyped) lazy val clusterTyped = akkaModule("akka-cluster-typed") @@ -493,7 +507,7 @@ lazy val clusterShardingTyped = akkaModule("akka-cluster-sharding-typed") clusterSharding % "compile->compile;compile->CompileJdk9;multi-jvm->multi-jvm", actorTestkitTyped % "test->test", actorTypedTests % "test->test", - persistenceTyped % "test->test", + persistenceTyped % "optional->compile;test->test", persistenceTestkit % "test->test", remote % "compile->CompileJdk9;test->test", remoteTests % "test->test", diff --git a/project/AkkaDisciplinePlugin.scala b/project/AkkaDisciplinePlugin.scala index 7222af7938..60bd70e5a0 100644 --- a/project/AkkaDisciplinePlugin.scala +++ b/project/AkkaDisciplinePlugin.scala @@ -35,6 +35,8 @@ object AkkaDisciplinePlugin extends AutoPlugin { "akka-distributed-data", // references to deprecated PARSER fields in generated message formats? "akka-cluster-sharding-typed", + // references to deprecated PARSER fields in generated message formats? + "akka-persistence-typed", "akka-docs") val looseProjects = Set( diff --git a/project/Dependencies.scala b/project/Dependencies.scala index d5aa1d87be..d82f901596 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -265,7 +265,9 @@ object Dependencies { val persistenceTestKit = l ++= Seq(Test.scalatest, Test.logback) - val persistenceShared = l ++= Seq(Provided.levelDB, Provided.levelDBNative) + val persistenceTypedTests = l ++= Seq(Test.scalatest, Test.logback) + + val persistenceShared = l ++= Seq(Provided.levelDB, Provided.levelDBNative, Test.logback) val jackson = l ++= Seq( jacksonCore, diff --git a/project/OSGi.scala b/project/OSGi.scala index 37c44093f9..ca3c9c9966 100644 --- a/project/OSGi.scala +++ b/project/OSGi.scala @@ -65,15 +65,14 @@ object OSGi { val protobuf = exports(Seq("akka.protobuf.*")) val protobufV3 = osgiSettings ++ Seq( - OsgiKeys.importPackage := Seq( - "!sun.misc", - scalaJava8CompatImport(), - scalaVersion(scalaImport).value, - configImport(), - "*"), - OsgiKeys.exportPackage := Seq("akka.protobufv3.internal.*"), - OsgiKeys.privatePackage := Seq("google.protobuf.*") - ) + OsgiKeys.importPackage := Seq( + "!sun.misc", + scalaJava8CompatImport(), + scalaVersion(scalaImport).value, + configImport(), + "*"), + OsgiKeys.exportPackage := Seq("akka.protobufv3.internal.*"), + OsgiKeys.privatePackage := Seq("google.protobuf.*")) val jackson = exports(Seq("akka.serialization.jackson.*"))