From ad2d7e2d0023217ef389a6722480ceeb1e3a4870 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Mon, 22 Jun 2020 12:46:57 +0100 Subject: [PATCH 01/50] Initial prototype of active active event sourcing (#29265) --- .../src/main/resources/reference.conf | 1 - .../persistence/typed/PersistenceId.scala | 16 ++ .../typed/internal/BehaviorSetup.scala | 6 +- .../internal/EventSourcedBehaviorImpl.scala | 21 +- .../typed/internal/ExternalInteractions.scala | 14 +- .../typed/internal/ReplayingEvents.scala | 23 +- .../typed/internal/ReplayingSnapshot.scala | 11 +- .../persistence/typed/internal/Running.scala | 153 ++++++++-- .../scaladsl/ActiveActiveEventSourcing.scala | 115 ++++++++ .../typed/scaladsl/EventSourcedBehavior.scala | 16 +- .../src/test/resources/logback-test.xml | 2 +- .../EventSourcedBehaviorWatchSpec.scala | 3 +- .../typed/aa/AAAuctionExampleSpec.scala | 270 ++++++++++++++++++ .../typed/aa/AABlogExampleSpec.scala | 162 +++++++++++ build.sbt | 3 +- project/Dependencies.scala | 2 +- 16 files changed, 761 insertions(+), 57 deletions(-) create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala create mode 100644 akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AAAuctionExampleSpec.scala create mode 100644 akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AABlogExampleSpec.scala diff --git a/akka-persistence-typed/src/main/resources/reference.conf b/akka-persistence-typed/src/main/resources/reference.conf index 7361c8113f..99c897d34e 100644 --- a/akka-persistence-typed/src/main/resources/reference.conf +++ b/akka-persistence-typed/src/main/resources/reference.conf @@ -21,7 +21,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..7614edec35 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 @@ -3,6 +3,7 @@ */ package akka.persistence.typed +import akka.annotation.InternalApi object PersistenceId { @@ -125,6 +126,21 @@ object PersistenceId { def ofUniqueId(id: String): PersistenceId = new PersistenceId(id) + /** + * Constructs a persistence id from a unique entity id that includes the replica id. + */ + @InternalApi + private[akka] def replicatedUniqueId(entityId: String, replicaId: String): PersistenceId = { + if (entityId.contains(DefaultSeparator)) + throw new IllegalArgumentException( + s"entityId [$entityId] contains [$DefaultSeparator] which is a reserved character") + + if (replicaId.contains(DefaultSeparator)) + throw new IllegalArgumentException( + s"replicaId [$replicaId] contains [$DefaultSeparator] which is a reserved character") + + new PersistenceId(entityId + DefaultSeparator + replicaId) + } } /** 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..6f87ad5134 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.typed.Signal import akka.actor.typed.scaladsl.ActorContext import akka.annotation.InternalApi import akka.persistence._ +import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive import akka.persistence.typed.{ EventAdapter, PersistenceId, SnapshotAdapter } import akka.persistence.typed.scaladsl.{ EventSourcedBehavior, RetentionCriteria } import akka.util.OptionVal @@ -48,7 +47,8 @@ private[akka] final class BehaviorSetup[C, E, S]( val retention: RetentionCriteria, var holdingRecoveryPermit: Boolean, val settings: EventSourcedSettings, - val stashState: StashState) { + val stashState: StashState, + val activeActive: Option[ActiveActive]) { import BehaviorSetup._ import InternalProtocol.RecoveryTickEvent 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..bc32587879 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 @@ -36,6 +36,7 @@ import akka.persistence.typed.SnapshotAdapter import akka.persistence.typed.SnapshotCompleted import akka.persistence.typed.SnapshotFailed import akka.persistence.typed.SnapshotSelectionCriteria +import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive import akka.persistence.typed.scaladsl._ import akka.persistence.typed.scaladsl.{ Recovery => TypedRecovery } import akka.persistence.typed.scaladsl.RetentionCriteria @@ -87,7 +88,8 @@ 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, + activeActive: Option[ActiveActive] = None) extends EventSourcedBehavior[Command, Event, State] { import EventSourcedBehaviorImpl.WriterIdentity @@ -150,7 +152,8 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( retention, holdingRecoveryPermit = false, settings = settings, - stashState = stashState) + stashState = stashState, + activeActive = activeActive) // needs to accept Any since we also can get messages from the journal // not part of the user facing Command protocol @@ -237,6 +240,14 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( override def withRecovery(recovery: TypedRecovery): EventSourcedBehavior[Command, Event, State] = { copy(recovery = recovery.toClassic) } + + override private[akka] def withActiveActive( + context: ActiveActiveContext, + id: String, + allIds: Set[String], + queryPluginId: String): EventSourcedBehavior[Command, Event, State] = { + copy(activeActive = Some(ActiveActive(id, allIds, context, queryPluginId))) + } } /** Protocol used internally by the eventsourced behaviors. */ @@ -247,4 +258,10 @@ 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 } + +final case class ReplicatedEvent[E](event: E, originReplica: String, originSequenceNr: Long) +case object ReplicatedEventAck 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..8345a26d30 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 @@ -27,32 +27,34 @@ private[akka] trait JournalInteractions[C, E, S] { def setup: BehaviorSetup[C, E, S] - type EventOrTagged = Any // `Any` since can be `E` or `Tagged` + type EventOrTaggedOrReplicated = Any // `Any` since can be `E` or `Tagged` or a `ReplicatedEvent` protected def internalPersist( ctx: ActorContext[_], cmd: Any, state: Running.RunningState[S], - event: EventOrTagged, + event: EventOrTaggedOrReplicated, eventAdapterManifest: String): 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 setup.journal .tell(JournalProtocol.WriteMessages(write, setup.selfClassic, setup.writerIdentity.instanceId), setup.selfClassic) - newState + newRunningState } @InternalStableApi @@ -65,7 +67,7 @@ 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[(EventOrTaggedOrReplicated, String)]): Running.RunningState[S] = { if (events.nonEmpty) { var newState = state 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..0b2735723d 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 @@ -87,13 +87,16 @@ private[akka] final class ReplayingEvents[C, E, S]( () override def onMessage(msg: InternalProtocol): Behavior[InternalProtocol] = { + // FIXME deal with a replicated event and ack + // https://github.com/akka/akka/issues/29256 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 cmd: IncomingCommand[C] => onInternalCommand(cmd) + case get: GetState[S @unchecked] => stashInternal(get) + case RecoveryPermitGranted => Behaviors.unhandled // should not happen, we already have the permit } } @@ -154,7 +157,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) @@ -236,8 +239,12 @@ private[akka] final class ReplayingEvents[C, E, S]( if (state.receivedPoisonPill && isInternalStashEmpty && !isUnstashAllInProgress) Behaviors.stopped else { + val seenPerReplica: Map[String, Long] = + setup.activeActive.map(aa => aa.allReplicas.map(replica => replica -> 0L).toMap).getOrElse(Map.empty) val running = - Running[C, E, S](setup, Running.RunningState[S](state.seqNr, state.state, state.receivedPoisonPill)) + Running[C, E, S]( + setup, + Running.RunningState[S](state.seqNr, state.state, state.receivedPoisonPill, seenPerReplica)) 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..6c471b9201 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 @@ -57,9 +57,10 @@ 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 cmd: IncomingCommand[C] => if (receivedPoisonPill) { if (setup.settings.logOnStashing) @@ -122,6 +123,10 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup stashInternal(cmd) } + def onReplicatedEvent(evt: InternalProtocol.ReplicatedEventEnvelope[E]): Behavior[InternalProtocol] = { + stashInternal(evt) + } + def onJournalResponse(response: JournalProtocol.Response): Behavior[InternalProtocol] = { setup.log.debug( "Unexpected response from journal: [{}], may be due to an actor restart, ignoring...", 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..66e7f8732c 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 @@ -6,10 +6,8 @@ package akka.persistence.typed.internal 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.{ ActorRef, ActorSystem, Behavior, Signal } import akka.actor.typed.internal.PoisonPill import akka.actor.typed.scaladsl.{ AbstractBehavior, ActorContext, Behaviors, LoggerOps } import akka.annotation.{ InternalApi, InternalStableApi } @@ -26,20 +24,30 @@ 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.query.{ EventEnvelope, PersistenceQuery } +import akka.persistence.query.scaladsl.EventsByPersistenceIdQuery +import akka.persistence.typed.{ + DeleteEventsCompleted, + DeleteEventsFailed, + DeleteSnapshotsCompleted, + DeleteSnapshotsFailed, + DeletionTarget, + EventRejectedException, + PersistenceId, + SnapshotCompleted, + SnapshotFailed, + SnapshotMetadata, + SnapshotSelectionCriteria +} import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState +import akka.persistence.typed.internal.InternalProtocol.ReplicatedEventEnvelope import akka.persistence.typed.internal.Running.WithSeqNrAccessible import akka.persistence.typed.scaladsl.Effect -import akka.util.unused +import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive +import akka.stream.{ SharedKillSwitch, SystemMaterializer } +import akka.stream.scaladsl.{ RestartSource, Sink } +import akka.stream.typed.scaladsl.ActorFlow +import akka.util.{ unused, Timeout } /** * INTERNAL API @@ -66,7 +74,12 @@ 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, + seenPerReplica: Map[String, Long], + replicationKillSwitch: Option[SharedKillSwitch] = None) { def nextSequenceNr(): RunningState[State] = copy(seqNr = seqNr + 1) @@ -82,8 +95,40 @@ 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)) + setup.activeActive.foreach(aa => startReplicationStream(setup.context.system, setup.context.self, state, aa)) new running.HandlingCommands(state) } + + def startReplicationStream[E, S]( + system: ActorSystem[_], + ref: ActorRef[InternalProtocol], + state: RunningState[S], + aa: ActiveActive): Unit = { + import scala.concurrent.duration._ + + val query = PersistenceQuery(system) + aa.allReplicas.foreach { replica => + if (replica != aa.replicaId) { + val seqNr = state.seenPerReplica(replica) + val pid = PersistenceId.replicatedUniqueId(aa.aaContext.entityId, replica) + // FIXME support different configuration per replica https://github.com/akka/akka/issues/29257 + val replication = query.readJournalFor[EventsByPersistenceIdQuery](aa.queryPluginId) + + implicit val timeout = Timeout(30.seconds) + + val source = RestartSource.withBackoff(2.seconds, 10.seconds, randomFactor = 0.2) { () => + replication + .eventsByPersistenceId(pid.id, seqNr + 1, Long.MaxValue) + .via(ActorFlow.ask[EventEnvelope, ReplicatedEventEnvelope[E], ReplicatedEventAck.type](ref) { + (eventEnvelope, replyTo) => + ReplicatedEventEnvelope(eventEnvelope.event.asInstanceOf[ReplicatedEvent[E]], replyTo) + }) + } + + source.runWith(Sink.ignore)(SystemMaterializer(system).materializer) + } + } + } } // =============================================== @@ -106,12 +151,17 @@ 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) + case JournalResponse(r) => onDeleteEventsJournalResponse(r, state.state) + case SnapshotterResponse(r) => onDeleteSnapshotResponse(r, state.state) + case get: GetState[S @unchecked] => onGetState(get) + case _ => Behaviors.unhandled } override def onSignal: PartialFunction[Signal, Behavior[InternalProtocol]] = { @@ -130,12 +180,45 @@ private[akka] object Running { else next } + def onReplicatedEvent( + state: Running.RunningState[S], + envelope: ReplicatedEventEnvelope[E]): Behavior[InternalProtocol] = { + // FIXME set the details on the context https://github.com/akka/akka/issues/29258 + setup.log.infoN( + "Replica {} received replicated event. Replica seqs nrs: {}", + setup.activeActive, + state.seenPerReplica) + envelope.ack ! ReplicatedEventAck + if (envelope.event.originReplica != setup.activeActive.get.replicaId && !alreadySeen(envelope.event)) { + setup.log.info("Saving event as first time") + handleReplicatedEventPersist(envelope.event) + } else { + setup.log.info("Filtering event as already seen") + this + } + } + // Used by EventSourcedBehaviorTestKit to retrieve the state. def onGetState(get: GetState[S]): Behavior[InternalProtocol] = { get.replyTo ! state.state this } + private def handleReplicatedEventPersist(event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { + _currentSequenceNumber = state.seqNr + 1 + val newState: RunningState[S] = state.applyEvent(setup, event.event) + val newState2: RunningState[S] = internalPersist(setup.context, null, newState, event, "") + 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), + state, + numberOfEvents = 1, + shouldSnapshotAfterPersist, + Nil) + } + @tailrec def applyEffects( msg: Any, state: RunningState[S], @@ -162,8 +245,13 @@ private[akka] object Running { val eventToPersist = adaptEvent(event) val eventAdapterManifest = setup.eventAdapter.manifest(event) - - val newState2 = internalPersist(setup.context, msg, newState, eventToPersist, eventAdapterManifest) + val newState2 = setup.activeActive match { + case Some(aa) => + val replicatedEvent = ReplicatedEvent(eventToPersist, aa.replicaId, _currentSequenceNumber) + internalPersist(setup.context, msg, newState, replicatedEvent, eventAdapterManifest) + case None => + internalPersist(setup.context, msg, newState, eventToPersist, eventAdapterManifest) + } val shouldSnapshotAfterPersist = setup.shouldSnapshot(newState2.state, event, newState2.seqNr) @@ -252,12 +340,13 @@ 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 get: GetState[S @unchecked] => stashInternal(get) + case SnapshotterResponse(r) => onDeleteSnapshotResponse(r, visibleState.state) + case RecoveryTickEvent(_) => Behaviors.unhandled + case RecoveryPermitGranted => Behaviors.unhandled } } @@ -271,6 +360,14 @@ private[akka] object Running { } } + def onReplicatedEvent(event: InternalProtocol.ReplicatedEventEnvelope[E]): Behavior[InternalProtocol] = { + if (state.receivedPoisonPill) { + Behaviors.unhandled + } else { + stashInternal(event) + } + } + final def onJournalResponse(response: Response): Behavior[InternalProtocol] = { if (setup.log.isDebugEnabled) { setup.log.debug2( diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala new file mode 100644 index 0000000000..847b2cbff6 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala @@ -0,0 +1,115 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed.scaladsl + +import akka.persistence.typed.PersistenceId + +/** + * Utility class for comparing timestamp and data center + * identifier when implementing last-writer wins. + */ +final case class LwwTime(timestamp: Long, originDc: String) { + + /** + * 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: String): 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.originDc.compareTo(originDc) > 0) true + else false + } +} + +trait ActiveActiveContext { + def timestamp: Long + def origin: String + def concurrent: Boolean + def replicaId: String + def allReplicas: Set[String] + def persistenceId: PersistenceId + def recoveryRunning: Boolean + def entityId: String + def currentTimeMillis(): Long +} + +// FIXME, parts of this can be set during initialisation +// Other fields will be set before executing the event handler as they change per event +// https://github.com/akka/akka/issues/29258 +private[akka] class ActiveActiveContextImpl(val entityId: String, val replicaId: String, val allReplicas: Set[String]) + extends ActiveActiveContext { + var _timestamp: Long = -1 + var _origin: String = null + var _concurrent: Boolean = false + + // FIXME check illegal access https://github.com/akka/akka/issues/29264 + + /** + * The timestamp of the event. Always increases per data center + * Undefined result if called from any where other than an event handler. + */ + override def timestamp: Long = _timestamp + + /** + * The origin of the current event. + * Undefined result if called from anywhere other than an event handler. + */ + override def origin: String = _origin + + /** + * 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 = _concurrent + override def persistenceId: PersistenceId = PersistenceId.replicatedUniqueId(entityId, replicaId) + override def currentTimeMillis(): Long = { + // FIXME always increasing + System.currentTimeMillis() + } + override def recoveryRunning: Boolean = false +} + +object ActiveActiveEventSourcing { + + /** + * 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. + * + * 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. + * TODO support a different query plugin id per replicas: https://github.com/akka/akka/issues/29257 + * + * @param replicaId The unique identity for this entity. The underlying persistence id will include the replica. + * @param allReplicaIds All replica ids. These need to be known to receive events from all replicas. + * @param queryPluginId Used to read the events from other replicas. Must be the query side of your configured journal plugin. + * @return + */ + def apply[Command, Event, State]( + entityId: String, + replicaId: String, + allReplicaIds: Set[String], + queryPluginId: String)(activeActiveContext: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) + : EventSourcedBehavior[Command, Event, State] = { + val context = new ActiveActiveContextImpl(entityId, replicaId, allReplicaIds) + activeActiveContext(context).withActiveActive(context, replicaId, allReplicaIds, queryPluginId) + } + +} 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..e0a4984587 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 @@ -5,7 +5,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,7 +12,7 @@ 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.DoNotInherit +import akka.annotation.{ DoNotInherit, InternalApi } import akka.persistence.typed.EventAdapter import akka.persistence.typed.PersistenceId import akka.persistence.typed.SnapshotAdapter @@ -22,6 +21,13 @@ import akka.persistence.typed.internal._ object EventSourcedBehavior { + @InternalApi + private[akka] case class ActiveActive( + replicaId: String, + allReplicas: Set[String], + aaContext: ActiveActiveContext, + queryPluginId: String) + /** * Type alias for the command handler function that defines how to act on commands. * @@ -145,6 +151,12 @@ object EventSourcedBehavior { */ def withJournalPluginId(id: String): EventSourcedBehavior[Command, Event, State] + private[akka] def withActiveActive( + context: ActiveActiveContext, + replicaId: String, + allReplicaIds: Set[String], + queryPluginId: String): EventSourcedBehavior[Command, Event, State] + /** * Change the snapshot store plugin id that this actor should use. */ diff --git a/akka-persistence-typed/src/test/resources/logback-test.xml b/akka-persistence-typed/src/test/resources/logback-test.xml index 22c45c93b6..c980894390 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{persistenceId}] - %msg %n 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..c033e614ed 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,8 @@ class EventSourcedBehaviorWatchSpec RetentionCriteria.disabled, holdingRecoveryPermit = false, settings = settings, - stashState = new StashState(context.asInstanceOf[ActorContext[InternalProtocol]], settings)) + stashState = new StashState(context.asInstanceOf[ActorContext[InternalProtocol]], settings), + None) "A typed persistent parent actor watching a child" must { diff --git a/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AAAuctionExampleSpec.scala b/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AAAuctionExampleSpec.scala new file mode 100644 index 0000000000..9ca783e315 --- /dev/null +++ b/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AAAuctionExampleSpec.scala @@ -0,0 +1,270 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package docs.akka.persistence.typed.aa + +import java.time.Instant + +import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } +import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, _ } +import akka.actor.typed.{ ActorRef, Behavior } +import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal +import akka.persistence.typed.scaladsl.{ ActiveActiveContext, ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import com.typesafe.config.ConfigFactory +import org.scalatest.concurrent.{ Eventually, ScalaFutures } +import org.scalatest.matchers.should.Matchers +import org.scalatest.wordspec.AnyWordSpecLike + +object AAAuctionExampleSpec { + val config = ConfigFactory.parseString(""" + akka.actor.provider = cluster + akka.loglevel = info + akka.persistence { + journal { + plugin = "akka.persistence.journal.inmem" + } + } + """) + type MoneyAmount = Int + + case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originDc: String) + + // commands + sealed trait AuctionCommand + case object Finish extends AuctionCommand // A timer needs to schedule this event at each replica + final case class OfferBid(bidder: String, offer: MoneyAmount) extends AuctionCommand + final case class GetHighestBid(replyTo: ActorRef[Bid]) extends AuctionCommand + final case class IsClosed(replyTo: ActorRef[Boolean]) extends AuctionCommand + private final case object Close extends AuctionCommand // Internal, should not be sent from the outside + + sealed trait AuctionEvent + final case class BidRegistered(bid: Bid) extends AuctionEvent + final case class AuctionFinished(atDc: String) extends AuctionEvent + final case class WinnerDecided(atDc: String, winningBid: Bid, highestCounterOffer: MoneyAmount) extends AuctionEvent + + sealed trait AuctionPhase + case object Running extends AuctionPhase + final case class Closing(finishedAtDc: Set[String]) extends AuctionPhase + case object Closed extends AuctionPhase + + case class AuctionState( + phase: AuctionPhase, + highestBid: Bid, + highestCounterOffer: MoneyAmount // in ebay style auctions, we need to keep track of current highest counter offer + ) { + + def applyEvent(event: AuctionEvent): 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.originDc.compareTo( + second.originDc) < 0) + } + + case class AuctionSetup( + name: String, + initialBid: Bid, // the initial bid is basically the minimum price bidden at start time by the owner + closingAt: Instant, + responsibleForClosing: Boolean, + allDcs: Set[String]) + + def commandHandler(setup: AuctionSetup, ctx: ActorContext[AuctionCommand], aaContext: ActiveActiveContext)( + state: AuctionState, + command: AuctionCommand): Effect[AuctionEvent, AuctionState] = { + state.phase match { + case Closing(_) | Closed => + command match { + case GetHighestBid(replyTo) => + replyTo ! state.highestBid + Effect.none + case IsClosed(replyTo) => + replyTo ! (state.phase == Closed) + Effect.none + case Finish => + ctx.log.info("Finish") + Effect.persist(AuctionFinished(aaContext.replicaId)) + case Close => + ctx.log.info("Close") + require(shouldClose(setup, state)) + // TODO send email (before or after persisting) + Effect.persist(WinnerDecided(aaContext.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(aaContext.currentTimeMillis()), aaContext.replicaId))) + case GetHighestBid(replyTo) => + replyTo ! state.highestBid + Effect.none + case Finish => + Effect.persist(AuctionFinished(aaContext.replicaId)) + case Close => + ctx.log.warn("Premature close") + // Close should only be triggered when we have already finished + Effect.unhandled + case IsClosed(replyTo) => + replyTo ! false + Effect.none + } + } + } + + private def shouldClose(auctionSetup: AuctionSetup, state: AuctionState): Boolean = { + auctionSetup.responsibleForClosing && (state.phase match { + case Closing(alreadyFinishedAtDc) => + val allDone = auctionSetup.allDcs.diff(alreadyFinishedAtDc).isEmpty + if (!allDone) { + println( + s"Not closing auction as not all DCs have reported finished. All DCs: ${auctionSetup.allDcs}. Reported finished ${alreadyFinishedAtDc}") + } + allDone + case _ => + false + }) + } + + def eventHandler(ctx: ActorContext[AuctionCommand], aaCtx: ActiveActiveContext, setup: AuctionSetup)( + state: AuctionState, + event: AuctionEvent): AuctionState = { + + val newState = state.applyEvent(event) + ctx.log.infoN("Applying event {}. New start {}", event, newState) + if (!aaCtx.recoveryRunning) { + eventTriggers(setup, ctx, aaCtx, event, newState) + } + newState + + } + + private def eventTriggers( + setup: AuctionSetup, + ctx: ActorContext[AuctionCommand], + aaCtx: ActiveActiveContext, + event: AuctionEvent, + newState: AuctionState) = { + event match { + case finished: AuctionFinished => + newState.phase match { + case Closing(alreadyFinishedAtDc) => + ctx.log.infoN( + "AuctionFinished at {}, already finished at [{}]", + finished.atDc, + alreadyFinishedAtDc.mkString(", ")) + if (alreadyFinishedAtDc(aaCtx.replicaId)) { + if (shouldClose(setup, newState)) ctx.self ! Close + } else { + ctx.log.info("Sending finish to self") + ctx.self ! Finish + } + + case _ => // no trigger for this state + } + case _ => // no trigger for this event + } + } + + def initialState(setup: AuctionSetup) = + AuctionState(phase = Running, highestBid = setup.initialBid, highestCounterOffer = setup.initialBid.offer) + + def behavior(replica: String, setup: AuctionSetup): Behavior[AuctionCommand] = Behaviors.setup[AuctionCommand] { + ctx => + ActiveActiveEventSourcing(setup.name, replica, setup.allDcs, LeveldbReadJournal.Identifier) { aaCtx => + EventSourcedBehavior( + aaCtx.persistenceId, + initialState(setup), + commandHandler(setup, ctx, aaCtx), + eventHandler(ctx, aaCtx, setup)) + } + } +} + +class AAAuctionExampleSpec + extends ScalaTestWithActorTestKit(AAAuctionExampleSpec.config) + with AnyWordSpecLike + with Matchers + with LogCapturing + with ScalaFutures + with Eventually { + import AAAuctionExampleSpec._ + + "Auction example" should { + "work" in { + val Replicas = Set("DC-A", "DC-B") + val setupA = + AuctionSetup( + "old-skis", + Bid("chbatey", 12, Instant.now(), "DC-A"), + Instant.now().plusSeconds(10), + responsibleForClosing = true, + Replicas) + + val setupB = setupA.copy(responsibleForClosing = false) + + val dcAReplica: ActorRef[AuctionCommand] = spawn(behavior("DC-A", setupA)) + val dcBReplica: ActorRef[AuctionCommand] = spawn(behavior("DC-B", setupB)) + + 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/src/test/scala/docs/akka/persistence/typed/aa/AABlogExampleSpec.scala b/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AABlogExampleSpec.scala new file mode 100644 index 0000000000..9f5b288ee9 --- /dev/null +++ b/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AABlogExampleSpec.scala @@ -0,0 +1,162 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package docs.akka.persistence.typed.aa + +import java.util.UUID + +import akka.Done +import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } +import akka.actor.typed.ActorRef +import akka.actor.typed.scaladsl.{ ActorContext, Behaviors } +import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal +import akka.persistence.typed.scaladsl._ +import akka.serialization.jackson.CborSerializable +import com.typesafe.config.ConfigFactory +import org.scalatest.concurrent.{ Eventually, ScalaFutures } +import org.scalatest.matchers.should.Matchers +import org.scalatest.time.{ Millis, Span } +import org.scalatest.wordspec.AnyWordSpecLike + +object AABlogExampleSpec { + val config = + ConfigFactory.parseString(s""" + + akka.actor.allow-java-serialization = true + // FIXME serializers for replicated event or akka persistence support for metadata: https://github.com/akka/akka/issues/29260 + akka.actor.provider = cluster + akka.loglevel = debug + akka.persistence { + journal { + plugin = "akka.persistence.journal.leveldb" + leveldb { + native = off + dir = "target/journal-AABlogExampleSpec-${UUID.randomUUID()}" + } + } + } + """) + + final case class BlogState(content: Option[PostContent], contentTimestamp: LwwTime, published: Boolean) { + def withContent(newContent: PostContent, timestamp: LwwTime): BlogState = + copy(content = Some(newContent), contentTimestamp = timestamp) + def isEmpty: Boolean = content.isEmpty + } + val emptyState: BlogState = BlogState(None, LwwTime(Long.MinValue, ""), published = false) + + final case class PostContent(title: String, body: String) + final case class PostSummary(postId: String, title: String) + final case class Published(postId: String) extends BlogEvent + + sealed trait BlogCommand + final case class AddPost(postId: String, content: PostContent, replyTo: ActorRef[AddPostDone]) extends BlogCommand + final case class AddPostDone(postId: String) + final case class GetPost(postId: String, replyTo: ActorRef[PostContent]) extends BlogCommand + final case class ChangeBody(postId: String, newContent: PostContent, replyTo: ActorRef[Done]) extends BlogCommand + final case class Publish(postId: String, replyTo: ActorRef[Done]) extends BlogCommand + + sealed trait BlogEvent extends CborSerializable + final case class PostAdded(postId: String, content: PostContent, timestamp: LwwTime) extends BlogEvent + final case class BodyChanged(postId: String, newContent: PostContent, timestamp: LwwTime) extends BlogEvent +} + +class AABlogExampleSpec + extends ScalaTestWithActorTestKit(AABlogExampleSpec.config) + with AnyWordSpecLike + with Matchers + with LogCapturing + with ScalaFutures + with Eventually { + import AABlogExampleSpec._ + + implicit val config: PatienceConfig = PatienceConfig(timeout = Span(timeout.duration.toMillis, Millis)) + + def behavior(aa: ActiveActiveContext, ctx: ActorContext[BlogCommand]) = + EventSourcedBehavior[BlogCommand, BlogEvent, BlogState]( + aa.persistenceId, + emptyState, + (state, cmd) => + cmd match { + case AddPost(_, content, replyTo) => + val evt = + PostAdded(aa.persistenceId.id, content, state.contentTimestamp.increase(aa.timestamp, aa.replicaId)) + Effect.persist(evt).thenRun { _ => + replyTo ! AddPostDone(aa.entityId) + } + case ChangeBody(_, newContent, replyTo) => + val evt = + BodyChanged(aa.persistenceId.id, newContent, state.contentTimestamp.increase(aa.timestamp, aa.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 + }, + (state, event) => { + ctx.log.info(s"${aa.entityId}:${aa.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) + } + }) + + "Blog Example" should { + "work" in { + val refDcA: ActorRef[BlogCommand] = + spawn(Behaviors.setup[BlogCommand] { ctx => + ActiveActiveEventSourcing("cat", "DC-A", Set("DC-A", "DC-B"), LeveldbReadJournal.Identifier) { + (aa: ActiveActiveContext) => + behavior(aa, ctx) + } + }, "dc-a") + + val refDcB: ActorRef[BlogCommand] = + spawn(Behaviors.setup[BlogCommand] { ctx => + ActiveActiveEventSourcing("cat", "DC-B", Set("DC-A", "DC-B"), LeveldbReadJournal.Identifier) { + (aa: ActiveActiveContext) => + behavior(aa, ctx) + } + }, "dc-b") + + import akka.actor.typed.scaladsl.AskPattern._ + import akka.util.Timeout + + import scala.concurrent.duration._ + 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/build.sbt b/build.sbt index a53103bdac..0e151f8fc4 100644 --- a/build.sbt +++ b/build.sbt @@ -453,8 +453,9 @@ lazy val actorTyped = akkaModule("akka-actor-typed") lazy val persistenceTyped = akkaModule("akka-persistence-typed") .dependsOn( actorTyped, + streamTyped, persistence % "compile->compile;test->test", - persistenceQuery % "test", + persistenceQuery, actorTestkitTyped % "test->test", clusterTyped % "test->test", actorTestkitTyped % "test->test", diff --git a/project/Dependencies.scala b/project/Dependencies.scala index b3cc7cb359..0072a183c4 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -255,7 +255,7 @@ object Dependencies { val persistenceTestKit = l ++= Seq(Test.scalatest, Test.logback) - val persistenceShared = l ++= Seq(Provided.levelDB, Provided.levelDBNative) + val persistenceShared = l ++= Seq(Provided.levelDB, Provided.levelDBNative, Test.logback) val jackson = l ++= Seq( jacksonCore, From 08182bbdebe9635c6f9a1b096d16ed000332d648 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Wed, 24 Jun 2020 15:37:20 +0100 Subject: [PATCH 02/50] ActiveActive: Events with metadata and events by persistence id for (#29287) --- .../persistence/query/EventEnvelope.scala | 41 ++++- .../leveldb/EventsByPersistenceIdStage.scala | 1 - .../journal/leveldb/EventsByTagStage.scala | 1 - .../query/journal/leveldb/TestActor.scala | 10 +- .../src/main/resources/reference.conf | 4 + .../persistence/testkit/EventStorage.scala | 44 +++-- .../testkit/PersistenceTestKitPlugin.scala | 26 ++- .../internal/SerializedEventStorageImpl.scala | 42 +++-- .../internal/SimpleEventStorageImpl.scala | 7 +- ...ersistenceTestKitReadJournalProvider.scala | 16 ++ .../internal/EventsByPersistenceIdStage.scala | 87 +++++++++ .../PersistenceTestKitReadJournal.scala | 32 ++++ .../PersistenceTestKitReadJournal.scala | 46 +++++ .../testkit/scaladsl/PersistenceTestKit.scala | 11 +- .../query/EventsByPersistenceIdSpec.scala | 169 ++++++++++++++++++ .../EventSourcedBehaviorTestKitSpec.scala | 2 +- .../src/test/resources/logback-test.xml | 32 ++++ .../AAAuctionExampleSpec.scala | 24 +-- .../AABlogExampleSpec.scala | 31 +--- ...9217-active-active-event-sourcing.excludes | 6 + .../internal/EventSourcedBehaviorImpl.scala | 9 +- .../persistence/typed/internal/Running.scala | 14 +- ...9217-active-active-event-sourcing.excludes | 4 + .../journal/EventWithMetaData.scala | 33 ++++ .../journal/inmem/InmemJournal.scala | 64 +++++-- .../EventSourcedActorFailureSpec.scala | 4 +- .../journal/chaos/ChaosJournal.scala | 4 +- build.sbt | 7 + project/Dependencies.scala | 2 + 29 files changed, 657 insertions(+), 116 deletions(-) create mode 100644 akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/PersistenceTestKitReadJournalProvider.scala create mode 100644 akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/internal/EventsByPersistenceIdStage.scala create mode 100644 akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/javadsl/PersistenceTestKitReadJournal.scala create mode 100644 akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/scaladsl/PersistenceTestKitReadJournal.scala create mode 100644 akka-persistence-testkit/src/test/scala/akka/persistence/testkit/query/EventsByPersistenceIdSpec.scala create mode 100644 akka-persistence-typed-tests/src/test/resources/logback-test.xml rename {akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa => akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed}/AAAuctionExampleSpec.scala (94%) rename {akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa => akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed}/AABlogExampleSpec.scala (85%) create mode 100644 akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes create mode 100644 akka-persistence/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes create mode 100644 akka-persistence/src/main/scala/akka/persistence/journal/EventWithMetaData.scala 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..7285a43b8f 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 @@ -59,7 +84,7 @@ final class EventEnvelope( } 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..693ee109a3 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 @@ -6,12 +6,16 @@ package akka.persistence.query.journal.leveldb import akka.actor.Props import akka.persistence.PersistentActor +import akka.persistence.journal.EventWithMetaData +import akka.persistence.query.journal.leveldb.TestActor.WithMeta object TestActor { def props(persistenceId: String): Props = Props(new TestActor(persistenceId)) case class DeleteCmd(toSeqNr: Long = Long.MaxValue) + + case class WithMeta(payload: String, meta: Any) } class TestActor(override val persistenceId: String) extends PersistentActor { @@ -26,10 +30,14 @@ class TestActor(override val persistenceId: String) extends PersistentActor { case DeleteCmd(toSeqNr) => deleteMessages(toSeqNr) sender() ! s"$toSeqNr-deleted" + case WithMeta(payload, meta) => + persist(EventWithMetaData(payload, meta)) { _ => + sender() ! s"$payload-done" + } case cmd: String => persist(cmd) { evt => - sender() ! evt + "-done" + sender() ! s"$evt-done" } } 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..ff458085c7 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.testkit.EventStorage.{ JournalPolicies, Metadata } import akka.persistence.testkit.ProcessingPolicy.DefaultPolicies import akka.persistence.testkit.internal.TestKitStorage import akka.util.ccompat.JavaConverters._ @@ -19,7 +19,7 @@ import akka.util.ccompat.JavaConverters._ * INTERNAL API */ @InternalApi -private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, PersistentRepr] { +private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, (PersistentRepr, Metadata)] { import EventStorage._ @@ -31,21 +31,23 @@ 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, Metadata)): Long = repr._1.sequenceNr - def add(elems: immutable.Seq[PersistentRepr]): Unit = - elems.groupBy(_.persistenceId).foreach(gr => add(gr._1, gr._2)) + def add(elems: immutable.Seq[(PersistentRepr, Metadata)]): Unit = + elems.groupBy(_._1.persistenceId).foreach { gr => + add(gr._1, gr._2) + } override protected val DefaultPolicy = JournalPolicies.PassAll /** * @throws Exception from StorageFailure in the current writing policy */ - def tryAdd(elems: immutable.Seq[PersistentRepr]): Try[Unit] = { - val grouped = elems.groupBy(_.persistenceId) + def tryAdd(elems: immutable.Seq[(PersistentRepr, Metadata)]): Try[Unit] = { + val grouped = elems.groupBy(_._1.persistenceId) val processed = grouped.map { - case (pid, els) => currentPolicy.tryProcess(pid, WriteEvents(els.map(_.payload))) + case (pid, els) => currentPolicy.tryProcess(pid, WriteEvents(els.map(_._1.payload))) } val reduced: ProcessingResult = @@ -71,8 +73,8 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, Per persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, - max: Long): immutable.Seq[PersistentRepr] = { - val batch = read(persistenceId, fromSequenceNr, toSequenceNr, max) + max: Long): immutable.Seq[(PersistentRepr, Metadata)] = { + val batch: immutable.Seq[(PersistentRepr, Metadata)] = read(persistenceId, fromSequenceNr, toSequenceNr, max) currentPolicy.tryProcess(persistenceId, ReadEvents(batch)) match { case ProcessingSuccess => batch case Reject(ex) => throw ex @@ -96,9 +98,9 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, Per } } - private def mapAny(key: String, elems: immutable.Seq[Any]): immutable.Seq[PersistentRepr] = { + private def mapAny(key: String, elems: immutable.Seq[Any]): immutable.Seq[(PersistentRepr, Metadata)] = { val sn = getHighestSeqNumber(key) + 1 - elems.zipWithIndex.map(p => PersistentRepr(p._1, p._2 + sn, key)) + elems.zipWithIndex.map(p => (PersistentRepr(p._1, p._2 + sn, key), NoMetadata)) } } @@ -107,6 +109,24 @@ object EventStorage { object JournalPolicies extends DefaultPolicies[JournalOperation] + /** + * INTERNAL API + */ + @InternalApi + private[testkit] sealed trait Metadata + + /** + * INTERNAL API + */ + @InternalApi + private[testkit] case object NoMetadata extends Metadata + + /** + * INTERNAL API + */ + @InternalApi + private[testkit] final case class WithMetadata(payload: Any) extends Metadata + } /** 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..7d8e899b89 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 @@ -7,13 +7,12 @@ package akka.persistence.testkit 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, EventWithMetaData, Tagged } import akka.persistence.snapshot.SnapshotStore +import akka.persistence.testkit.EventStorage.{ NoMetadata, WithMetadata } import akka.persistence.testkit.internal.{ InMemStorageExtension, SnapshotStorageEmulatorExtension } /** @@ -25,15 +24,25 @@ import akka.persistence.testkit.internal.{ InMemStorageExtension, SnapshotStorag class PersistenceTestKitPlugin extends AsyncWriteJournal { private final val storage = InMemStorageExtension(context.system) + private val eventStream = context.system.eventStream 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 + // TODO define how to handle tagged and metadata + case Tagged(p, _) => (pl.withPayload(p).withTimestamp(System.currentTimeMillis()), NoMetadata) + case evt: EventWithMetaData => + (pl.withPayload(evt.event).withTimestamp(System.currentTimeMillis()), WithMetadata(evt.metaData)) + case _ => (pl.withTimestamp(System.currentTimeMillis()), NoMetadata) }) - 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] = @@ -41,7 +50,8 @@ class PersistenceTestKitPlugin extends AsyncWriteJournal { 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(_._1).foreach(recoveryCallback))) override def asyncReadHighestSequenceNr(persistenceId: String, fromSequenceNr: Long): Future[Long] = Future.fromTry(Try { @@ -64,6 +74,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/internal/SerializedEventStorageImpl.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SerializedEventStorageImpl.scala index 9d25479dba..dd6242ee95 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,56 @@ 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.testkit.EventStorage -import akka.serialization.{ Serialization, SerializationExtension } +import akka.persistence.testkit.EventStorage.Metadata +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], + metadata: Metadata) +} /** * 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(repr: (PersistentRepr, Metadata)): Serialized = Serialization.withTransportInformation(system.asInstanceOf[ExtendedActorSystem]) { () => - val s = serialization.findSerializerFor(repr) - (s.identifier, s.toBinary(repr)) + val (pr, meta) = repr + val payload = pr.payload.asInstanceOf[AnyRef] + val s = serialization.findSerializerFor(payload) + val manifest = Serializers.manifestFor(s, payload) + Serialized(pr.persistenceId, pr.sequenceNr, s.identifier, manifest, pr.writerUuid, s.toBinary(payload), meta) } /** * @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, Metadata) = { + val event = serialization.deserialize(internal.payload, internal.payloadSerId, internal.payloadSerManifest).get + ( + PersistentRepr(event, internal.sequenceNr, internal.persistenceId, writerUuid = internal.writerUuid), + internal.metadata) + } } diff --git a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SimpleEventStorageImpl.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SimpleEventStorageImpl.scala index 712c311498..25b63c8042 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SimpleEventStorageImpl.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SimpleEventStorageImpl.scala @@ -7,6 +7,7 @@ package akka.persistence.testkit.internal import akka.annotation.InternalApi import akka.persistence._ import akka.persistence.testkit.EventStorage +import akka.persistence.testkit.EventStorage.Metadata /** * INTERNAL API @@ -14,10 +15,10 @@ import akka.persistence.testkit.EventStorage @InternalApi private[testkit] class SimpleEventStorageImpl extends EventStorage { - override type InternalRepr = PersistentRepr + override type InternalRepr = (PersistentRepr, Metadata) - override def toInternal(repr: PersistentRepr): PersistentRepr = repr + override def toInternal(repr: (PersistentRepr, Metadata)): (PersistentRepr, Metadata) = repr - override def toRepr(internal: PersistentRepr): PersistentRepr = internal + override def toRepr(internal: (PersistentRepr, Metadata)): (PersistentRepr, Metadata) = internal } 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..dcd6246d40 --- /dev/null +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/PersistenceTestKitReadJournalProvider.scala @@ -0,0 +1,16 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.testkit.query +import akka.actor.ExtendedActorSystem +import akka.persistence.query.ReadJournalProvider + +class PersistenceTestKitReadJournalProvider(system: ExtendedActorSystem) extends ReadJournalProvider { + + override def scaladslReadJournal(): scaladsl.PersistenceTestKitReadJournal = + new scaladsl.PersistenceTestKitReadJournal(system) + + 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..af115ae4dc --- /dev/null +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/internal/EventsByPersistenceIdStage.scala @@ -0,0 +1,87 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.testkit.query.internal +import akka.actor.ActorRef +import akka.annotation.InternalApi +import akka.persistence.query.{ EventEnvelope, Sequence } +import akka.persistence.testkit.{ EventStorage, PersistenceTestKitPlugin } +import akka.persistence.testkit.EventStorage.{ NoMetadata, WithMetadata } +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 => + log.debug("Write notification {} {}", pid, toSequenceNr) + 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, meta)) => + push( + out, + EventEnvelope( + Sequence(pr.sequenceNr), + pr.persistenceId, + pr.sequenceNr, + pr.payload, + pr.timestamp, + meta match { + case NoMetadata => None + case WithMetadata(m) => Some(m) + })) + 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..9114994f10 --- /dev/null +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/javadsl/PersistenceTestKitReadJournal.scala @@ -0,0 +1,32 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.testkit.query.javadsl +import akka.NotUsed +import akka.persistence.query.EventEnvelope +import akka.persistence.query.javadsl.{ CurrentEventsByPersistenceIdQuery, 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 { + + 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 +} 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..fc52243c67 --- /dev/null +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/scaladsl/PersistenceTestKitReadJournal.scala @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.testkit.query.scaladsl +import akka.NotUsed +import akka.actor.ExtendedActorSystem +import akka.persistence.query.{ EventEnvelope, Sequence } +import akka.persistence.query.scaladsl.{ CurrentEventsByPersistenceIdQuery, EventsByPersistenceIdQuery, ReadJournal } +import akka.persistence.testkit.EventStorage +import akka.persistence.testkit.EventStorage.{ NoMetadata, WithMetadata } +import akka.persistence.testkit.internal.InMemStorageExtension +import akka.persistence.testkit.query.internal.EventsByPersistenceIdStage +import akka.stream.scaladsl.Source + +object PersistenceTestKitReadJournal { + val Identifier = "akka.persistence.testkit.query" +} + +final class PersistenceTestKitReadJournal(system: ExtendedActorSystem) + extends ReadJournal + with EventsByPersistenceIdQuery + with CurrentEventsByPersistenceIdQuery { + + private final val storage: EventStorage = InMemStorageExtension(system) + + 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 { + case (pr, meta) => + EventEnvelope(Sequence(pr.sequenceNr), persistenceId, pr.sequenceNr, pr.payload, pr.timestamp, meta match { + case NoMetadata => None + case WithMetadata(payload) => Some(payload) + }) + } + } +} 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..4a959adfc1 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.testkit.EventStorage.Metadata import akka.persistence.testkit._ import akka.persistence.testkit.internal.InMemStorageExtension import akka.persistence.testkit.internal.SnapshotStorageEmulatorExtension @@ -424,9 +423,9 @@ object SnapshotTestKit { */ @ApiMayChange class PersistenceTestKit(system: ActorSystem) - extends PersistenceTestKitOps[PersistentRepr, JournalOperation] - with ExpectOps[PersistentRepr] - with HasStorage[JournalOperation, PersistentRepr] { + extends PersistenceTestKitOps[(PersistentRepr, Metadata), JournalOperation] + with ExpectOps[(PersistentRepr, Metadata)] + with HasStorage[JournalOperation, (PersistentRepr, Metadata)] { require( Try(Persistence(system).journalFor(PersistenceTestKitPlugin.PluginId)).isSuccess, "The test persistence plugin is not configured.") @@ -495,7 +494,7 @@ 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, Metadata)): Any = repr._1.payload } @ApiMayChange 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..144a98108c --- /dev/null +++ b/akka-persistence-testkit/src/test/scala/akka/persistence/testkit/query/EventsByPersistenceIdSpec.scala @@ -0,0 +1,169 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.testkit.query + +import akka.{ Done, NotUsed } +import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } +import akka.actor.typed.ActorRef +import akka.persistence.journal.EventWithMetaData +import akka.persistence.query.{ EventEnvelope, PersistenceQuery } +import akka.persistence.testkit.PersistenceTestKitPlugin +import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal +import akka.persistence.typed.{ EventAdapter, EventSeq, PersistenceId } +import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior } +import akka.stream.scaladsl.Source +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) + }.eventAdapter(new EventAdapter[String, Any] { + override def toJournal(e: String): Any = { + if (e.startsWith("m")) { + EventWithMetaData(e, s"$e-meta") + } else { + e + } + } + override def manifest(event: String): String = "" + override def fromJournal(p: Any, manifest: String): EventSeq[String] = p match { + case e: EventWithMetaData => EventSeq.single(e.event.toString) + case _ => EventSeq.single(p.toString) + } + }) + +} + +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() + } + + "return metadata in queries" in { + val ackProbe = createTestProbe[Done]() + val ref = setupEmpty("with-meta") + ref ! Command("m-1", ackProbe.ref) + ref ! Command("m-2", ackProbe.ref) + val src: Source[EventEnvelope, NotUsed] = queries.eventsByPersistenceId("with-meta", 0L, Long.MaxValue) + val probe = + src.runWith(TestSink.probe[Any]).request(3) + probe.expectNextPF { + case e @ EventEnvelope(_, "with-meta", 1L, "m-1") if e.eventMetadata.contains("m-1-meta") => + } + + probe.expectNextPF { + case e @ EventEnvelope(_, "with-meta", 2L, "m-2") if e.eventMetadata.contains("m-2-meta") => + } + } + } +} 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-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..c980894390 --- /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{persistenceId}] - %msg %n + + + + + + + + + + + + + + + + diff --git a/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AAAuctionExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AAAuctionExampleSpec.scala similarity index 94% rename from akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AAAuctionExampleSpec.scala rename to akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AAAuctionExampleSpec.scala index 9ca783e315..4577431cb6 100644 --- a/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AAAuctionExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AAAuctionExampleSpec.scala @@ -2,30 +2,23 @@ * Copyright (C) 2020 Lightbend Inc. */ -package docs.akka.persistence.typed.aa +package docs.akka.persistence.typed import java.time.Instant import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, _ } import akka.actor.typed.{ ActorRef, Behavior } -import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal +import akka.persistence.testkit.PersistenceTestKitPlugin +import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal import akka.persistence.typed.scaladsl.{ ActiveActiveContext, ActiveActiveEventSourcing, Effect, EventSourcedBehavior } -import com.typesafe.config.ConfigFactory +import akka.serialization.jackson.CborSerializable import org.scalatest.concurrent.{ Eventually, ScalaFutures } import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike object AAAuctionExampleSpec { - val config = ConfigFactory.parseString(""" - akka.actor.provider = cluster - akka.loglevel = info - akka.persistence { - journal { - plugin = "akka.persistence.journal.inmem" - } - } - """) + type MoneyAmount = Int case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originDc: String) @@ -38,7 +31,7 @@ object AAAuctionExampleSpec { final case class IsClosed(replyTo: ActorRef[Boolean]) extends AuctionCommand private final case object Close extends AuctionCommand // Internal, should not be sent from the outside - sealed trait AuctionEvent + sealed trait AuctionEvent extends CborSerializable final case class BidRegistered(bid: Bid) extends AuctionEvent final case class AuctionFinished(atDc: String) extends AuctionEvent final case class WinnerDecided(atDc: String, winningBid: Bid, highestCounterOffer: MoneyAmount) extends AuctionEvent @@ -208,7 +201,7 @@ object AAAuctionExampleSpec { def behavior(replica: String, setup: AuctionSetup): Behavior[AuctionCommand] = Behaviors.setup[AuctionCommand] { ctx => - ActiveActiveEventSourcing(setup.name, replica, setup.allDcs, LeveldbReadJournal.Identifier) { aaCtx => + ActiveActiveEventSourcing(setup.name, replica, setup.allDcs, PersistenceTestKitReadJournal.Identifier) { aaCtx => EventSourcedBehavior( aaCtx.persistenceId, initialState(setup), @@ -219,7 +212,7 @@ object AAAuctionExampleSpec { } class AAAuctionExampleSpec - extends ScalaTestWithActorTestKit(AAAuctionExampleSpec.config) + extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) with AnyWordSpecLike with Matchers with LogCapturing @@ -228,6 +221,7 @@ class AAAuctionExampleSpec import AAAuctionExampleSpec._ "Auction example" should { + "work" in { val Replicas = Set("DC-A", "DC-B") val setupA = diff --git a/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AABlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala similarity index 85% rename from akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AABlogExampleSpec.scala rename to akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala index 9f5b288ee9..4fc552a0f1 100644 --- a/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/aa/AABlogExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala @@ -2,41 +2,22 @@ * Copyright (C) 2020 Lightbend Inc. */ -package docs.akka.persistence.typed.aa - -import java.util.UUID +package docs.akka.persistence.typed import akka.Done import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } import akka.actor.typed.ActorRef import akka.actor.typed.scaladsl.{ ActorContext, Behaviors } -import akka.persistence.query.journal.leveldb.scaladsl.LeveldbReadJournal +import akka.persistence.testkit.PersistenceTestKitPlugin +import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal import akka.persistence.typed.scaladsl._ import akka.serialization.jackson.CborSerializable -import com.typesafe.config.ConfigFactory import org.scalatest.concurrent.{ Eventually, ScalaFutures } import org.scalatest.matchers.should.Matchers import org.scalatest.time.{ Millis, Span } import org.scalatest.wordspec.AnyWordSpecLike object AABlogExampleSpec { - val config = - ConfigFactory.parseString(s""" - - akka.actor.allow-java-serialization = true - // FIXME serializers for replicated event or akka persistence support for metadata: https://github.com/akka/akka/issues/29260 - akka.actor.provider = cluster - akka.loglevel = debug - akka.persistence { - journal { - plugin = "akka.persistence.journal.leveldb" - leveldb { - native = off - dir = "target/journal-AABlogExampleSpec-${UUID.randomUUID()}" - } - } - } - """) final case class BlogState(content: Option[PostContent], contentTimestamp: LwwTime, published: Boolean) { def withContent(newContent: PostContent, timestamp: LwwTime): BlogState = @@ -62,7 +43,7 @@ object AABlogExampleSpec { } class AABlogExampleSpec - extends ScalaTestWithActorTestKit(AABlogExampleSpec.config) + extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) with AnyWordSpecLike with Matchers with LogCapturing @@ -124,7 +105,7 @@ class AABlogExampleSpec "work" in { val refDcA: ActorRef[BlogCommand] = spawn(Behaviors.setup[BlogCommand] { ctx => - ActiveActiveEventSourcing("cat", "DC-A", Set("DC-A", "DC-B"), LeveldbReadJournal.Identifier) { + ActiveActiveEventSourcing("cat", "DC-A", Set("DC-A", "DC-B"), PersistenceTestKitReadJournal.Identifier) { (aa: ActiveActiveContext) => behavior(aa, ctx) } @@ -132,7 +113,7 @@ class AABlogExampleSpec val refDcB: ActorRef[BlogCommand] = spawn(Behaviors.setup[BlogCommand] { ctx => - ActiveActiveEventSourcing("cat", "DC-B", Set("DC-A", "DC-B"), LeveldbReadJournal.Identifier) { + ActiveActiveEventSourcing("cat", "DC-B", Set("DC-A", "DC-B"), PersistenceTestKitReadJournal.Identifier) { (aa: ActiveActiveContext) => behavior(aa, ctx) } diff --git a/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes b/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes new file mode 100644 index 0000000000..b0597f04c8 --- /dev/null +++ b/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes @@ -0,0 +1,6 @@ +# Changes to internal/private +ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.typed.scaladsl.EventSourcedBehavior.withActiveActive") +ProblemFilters.exclude[Problem]("akka.persistence.typed.internal.Running*") +ProblemFilters.exclude[Problem]("akka.persistence.typed.internal.EventSourcedBehaviorImpl.*") +ProblemFilters.exclude[Problem]("akka.persistence.typed.internal.BehaviorSetup*") + 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 bc32587879..4b4f8205bc 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 @@ -263,5 +263,10 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( extends InternalProtocol } -final case class ReplicatedEvent[E](event: E, originReplica: String, originSequenceNr: Long) -case object ReplicatedEventAck +// FIXME serializer +@InternalApi +private[akka] final case class ReplicatedEventMetaData(originDc: String) +@InternalApi +private[akka] final case class ReplicatedEvent[E](event: E, originReplica: String, originSequenceNr: Long) +@InternalApi +private[akka] case object ReplicatedEventAck 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 66e7f8732c..846051b758 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 @@ -23,7 +23,7 @@ import akka.persistence.PersistentRepr import akka.persistence.SaveSnapshotFailure import akka.persistence.SaveSnapshotSuccess import akka.persistence.SnapshotProtocol -import akka.persistence.journal.Tagged +import akka.persistence.journal.{ EventWithMetaData, Tagged } import akka.persistence.query.{ EventEnvelope, PersistenceQuery } import akka.persistence.query.scaladsl.EventsByPersistenceIdQuery import akka.persistence.typed.{ @@ -121,7 +121,11 @@ private[akka] object Running { .eventsByPersistenceId(pid.id, seqNr + 1, Long.MaxValue) .via(ActorFlow.ask[EventEnvelope, ReplicatedEventEnvelope[E], ReplicatedEventAck.type](ref) { (eventEnvelope, replyTo) => - ReplicatedEventEnvelope(eventEnvelope.event.asInstanceOf[ReplicatedEvent[E]], replyTo) + val re = ReplicatedEvent[E]( + eventEnvelope.event.asInstanceOf[E], + eventEnvelope.eventMetadata.get.asInstanceOf[ReplicatedEventMetaData].originDc, + eventEnvelope.sequenceNr) // FIXME, this is the wrong sequence nr, we need origin sequence nr, follow up with tests that show this + ReplicatedEventEnvelope(re, replyTo) }) } @@ -206,8 +210,9 @@ private[akka] object Running { private def handleReplicatedEventPersist(event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { _currentSequenceNumber = state.seqNr + 1 + val replicatedEvent = new EventWithMetaData(event.event, ReplicatedEventMetaData(event.originReplica)) val newState: RunningState[S] = state.applyEvent(setup, event.event) - val newState2: RunningState[S] = internalPersist(setup.context, null, newState, event, "") + val newState2: RunningState[S] = internalPersist(setup.context, null, newState, replicatedEvent, "") 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) @@ -241,10 +246,11 @@ private[akka] object Running { // 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 newState: RunningState[S] = state.applyEvent(setup, event) val eventToPersist = adaptEvent(event) val eventAdapterManifest = setup.eventAdapter.manifest(event) + val newState2 = setup.activeActive match { case Some(aa) => val replicatedEvent = ReplicatedEvent(eventToPersist, aa.replicaId, _currentSequenceNumber) diff --git a/akka-persistence/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes b/akka-persistence/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes new file mode 100644 index 0000000000..91d5ea01ed --- /dev/null +++ b/akka-persistence/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes @@ -0,0 +1,4 @@ +# Changes to internal/private + +ProblemFilters.exclude[IncompatibleSignatureProblem]("akka.persistence.journal.inmem.InmemMessages.*") +ProblemFilters.exclude[IncompatibleSignatureProblem]("akka.persistence.journal.inmem.InmemJournal.*") diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/EventWithMetaData.scala b/akka-persistence/src/main/scala/akka/persistence/journal/EventWithMetaData.scala new file mode 100644 index 0000000000..c1bfc6aa11 --- /dev/null +++ b/akka-persistence/src/main/scala/akka/persistence/journal/EventWithMetaData.scala @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.journal + +object EventWithMetaData { + + def apply(event: Any, metaData: Any): EventWithMetaData = new EventWithMetaData(event, metaData) + + /** + * If meta data could not be deserialized it will not fail the replay/query. + * The "invalid" meta data is represented with this `UnknownMetaData` and + * it and the event will be wrapped in `EventWithMetaData`. + * + * The reason for not failing the replay/query is that meta data should be + * optional, e.g. the tool that wrote the meta data has been removed. This + * is typically because the serializer for the meta data has been removed + * from the class path (or configuration). + */ + final class UnknownMetaData(val serializerId: Int, val manifest: String) +} + +/** + * If the event is wrapped in this class the `metaData` will + * be serialized and stored separately from the event payload. This can be used by event + * adapters or other tools to store additional meta data without altering + * the actual domain event. + * + * Check the documentation of the persistence plugin in use to use if it supports + * EventWithMetaData. + */ +final class EventWithMetaData(val event: Any, val metaData: Any) 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..53cf784f70 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.{ AsyncWriteJournal, Tagged } +import akka.persistence.journal.inmem.InmemJournal.{ MessageWithMeta, ReplayWithMeta } +import akka.persistence.journal.{ AsyncWriteJournal, EventWithMetaData, 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,35 @@ 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] + // FIXME, which way around should Tagged/EventWithMeta go? https://github.com/akka/akka/issues/29284 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 meta: EventWithMetaData => + (p.withPayload(meta.event).withTimestamp(System.currentTimeMillis()), OptionVal.Some(meta.metaData)) + 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/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/build.sbt b/build.sbt index 0e151f8fc4..72bc671395 100644 --- a/build.sbt +++ b/build.sbt @@ -314,6 +314,13 @@ 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) + .disablePlugins(MimaPlugin) + .enablePlugins(NoPublish) + lazy val protobuf = akkaModule("akka-protobuf") .settings(OSGi.protobuf) .settings(AutomaticModuleName.settings("akka.protobuf")) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 0072a183c4..884fbae40a 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -255,6 +255,8 @@ object Dependencies { val persistenceTestKit = l ++= Seq(Test.scalatest, Test.logback) + val persistenceTypedTests = l ++= Seq(Test.scalatest, Test.logback) + val persistenceShared = l ++= Seq(Provided.levelDB, Provided.levelDBNative, Test.logback) val jackson = l ++= Seq( From e98f1311f3432d47c3c02c13ee8b6a0865713675 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Thu, 25 Jun 2020 14:52:55 +0100 Subject: [PATCH 03/50] Alternative approach for storing metadata (#29309) --- .../akka/util/AlwaysIncreasingClock.scala | 29 ++++ .../src/main/scala/akka/util/WallClock.scala | 23 +++ .../persistence/query/EventEnvelope.scala | 2 +- .../query/journal/leveldb/TestActor.scala | 8 - .../akka/persistence/CapabilityFlags.scala | 6 + .../persistence/journal/JournalSpec.scala | 45 +++++- .../persistence/testkit/EventStorage.scala | 43 ++--- .../testkit/PersistenceTestKitPlugin.scala | 21 ++- .../internal/SerializedEventStorageImpl.scala | 22 +-- .../internal/SimpleEventStorageImpl.scala | 7 +- .../internal/EventsByPersistenceIdStage.scala | 9 +- .../PersistenceTestKitReadJournal.scala | 11 +- .../testkit/scaladsl/PersistenceTestKit.scala | 9 +- .../query/EventsByPersistenceIdSpec.scala | 38 +---- .../PersistenceTestKitJournalCompatSpec.scala | 1 + .../persistence/typed/ActiveActiveSpec.scala | 148 ++++++++++++++++++ .../AABlogExampleSpec.scala | 10 +- .../internal/EventSourcedBehaviorImpl.scala | 4 +- .../typed/internal/ExternalInteractions.scala | 12 +- .../typed/internal/ReplayingEvents.scala | 21 ++- .../persistence/typed/internal/Running.scala | 97 +++++++----- .../scaladsl/ActiveActiveEventSourcing.scala | 23 ++- .../typed/scaladsl/EventSourcedBehavior.scala | 16 +- .../scala/akka/persistence/Persistent.scala | 15 +- .../journal/EventWithMetaData.scala | 33 ---- .../journal/inmem/InmemJournal.scala | 7 +- 26 files changed, 425 insertions(+), 235 deletions(-) create mode 100644 akka-actor/src/main/scala/akka/util/AlwaysIncreasingClock.scala create mode 100644 akka-actor/src/main/scala/akka/util/WallClock.scala create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala delete mode 100644 akka-persistence/src/main/scala/akka/persistence/journal/EventWithMetaData.scala diff --git a/akka-actor/src/main/scala/akka/util/AlwaysIncreasingClock.scala b/akka-actor/src/main/scala/akka/util/AlwaysIncreasingClock.scala new file mode 100644 index 0000000000..8bc84f8261 --- /dev/null +++ b/akka-actor/src/main/scala/akka/util/AlwaysIncreasingClock.scala @@ -0,0 +1,29 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.util + +import java.util.concurrent.atomic.AtomicLong +import java.util.function.LongUnaryOperator + +import akka.annotation.InternalApi + +/** + * 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-actor/src/main/scala/akka/util/WallClock.scala b/akka-actor/src/main/scala/akka/util/WallClock.scala new file mode 100644 index 0000000000..1bda0e8485 --- /dev/null +++ b/akka-actor/src/main/scala/akka/util/WallClock.scala @@ -0,0 +1,23 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.util + +import akka.annotation.ApiMayChange + +/** + * A time source. + */ +@ApiMayChange +trait WallClock { + def currentTimeMillis(): Long +} + +object WallClock { + + /** + * Always increasing time source. + */ + val AlwaysIncreasingClock: WallClock = new AlwaysIncreasingClock() +} 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 7285a43b8f..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 @@ -79,7 +79,7 @@ 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 } 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 693ee109a3..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 @@ -6,16 +6,12 @@ package akka.persistence.query.journal.leveldb import akka.actor.Props import akka.persistence.PersistentActor -import akka.persistence.journal.EventWithMetaData -import akka.persistence.query.journal.leveldb.TestActor.WithMeta object TestActor { def props(persistenceId: String): Props = Props(new TestActor(persistenceId)) case class DeleteCmd(toSeqNr: Long = Long.MaxValue) - - case class WithMeta(payload: String, meta: Any) } class TestActor(override val persistenceId: String) extends PersistentActor { @@ -30,10 +26,6 @@ class TestActor(override val persistenceId: String) extends PersistentActor { case DeleteCmd(toSeqNr) => deleteMessages(toSeqNr) sender() ! s"$toSeqNr-deleted" - case WithMeta(payload, meta) => - persist(EventWithMetaData(payload, meta)) { _ => - sender() ! s"$payload-done" - } case cmd: String => persist(cmd) { evt => 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..7434737402 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 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..1d6eaed53b 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,34 @@ 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 = 1L, + 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, 1L, Pid, _, _, Actor.noSender, WriterUuid, _, Some(`meta`)), + _) => + payload should be(event) + } + } + } } } 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 ff458085c7..14f8c90a46 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 @@ -10,7 +10,6 @@ import scala.collection.immutable import scala.util.{ Failure, Success, Try } import akka.annotation.InternalApi import akka.persistence.PersistentRepr -import akka.persistence.testkit.EventStorage.{ JournalPolicies, Metadata } import akka.persistence.testkit.ProcessingPolicy.DefaultPolicies import akka.persistence.testkit.internal.TestKitStorage import akka.util.ccompat.JavaConverters._ @@ -19,7 +18,7 @@ import akka.util.ccompat.JavaConverters._ * INTERNAL API */ @InternalApi -private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, (PersistentRepr, Metadata)] { +private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, PersistentRepr] { import EventStorage._ @@ -31,10 +30,10 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, (Pe // 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, Metadata)): Long = repr._1.sequenceNr + override def reprToSeqNum(repr: (PersistentRepr)): Long = repr.sequenceNr - def add(elems: immutable.Seq[(PersistentRepr, Metadata)]): Unit = - elems.groupBy(_._1.persistenceId).foreach { gr => + def add(elems: immutable.Seq[PersistentRepr]): Unit = + elems.groupBy(_.persistenceId).foreach { gr => add(gr._1, gr._2) } @@ -43,11 +42,11 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, (Pe /** * @throws Exception from StorageFailure in the current writing policy */ - def tryAdd(elems: immutable.Seq[(PersistentRepr, Metadata)]): Try[Unit] = { - val grouped = elems.groupBy(_._1.persistenceId) + def tryAdd(elems: immutable.Seq[PersistentRepr]): Try[Unit] = { + val grouped = elems.groupBy(_.persistenceId) val processed = grouped.map { - case (pid, els) => currentPolicy.tryProcess(pid, WriteEvents(els.map(_._1.payload))) + case (pid, els) => currentPolicy.tryProcess(pid, WriteEvents(els.map(_.payload))) } val reduced: ProcessingResult = @@ -73,8 +72,8 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, (Pe persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long, - max: Long): immutable.Seq[(PersistentRepr, Metadata)] = { - val batch: immutable.Seq[(PersistentRepr, Metadata)] = read(persistenceId, fromSequenceNr, toSequenceNr, max) + max: Long): immutable.Seq[PersistentRepr] = { + val batch = read(persistenceId, fromSequenceNr, toSequenceNr, max) currentPolicy.tryProcess(persistenceId, ReadEvents(batch)) match { case ProcessingSuccess => batch case Reject(ex) => throw ex @@ -98,35 +97,15 @@ private[testkit] trait EventStorage extends TestKitStorage[JournalOperation, (Pe } } - private def mapAny(key: String, elems: immutable.Seq[Any]): immutable.Seq[(PersistentRepr, Metadata)] = { + private def mapAny(key: String, elems: immutable.Seq[Any]): immutable.Seq[PersistentRepr] = { val sn = getHighestSeqNumber(key) + 1 - elems.zipWithIndex.map(p => (PersistentRepr(p._1, p._2 + sn, key), NoMetadata)) + elems.zipWithIndex.map(p => PersistentRepr(p._1, p._2 + sn, key)) } } object EventStorage { - object JournalPolicies extends DefaultPolicies[JournalOperation] - - /** - * INTERNAL API - */ - @InternalApi - private[testkit] sealed trait Metadata - - /** - * INTERNAL API - */ - @InternalApi - private[testkit] case object NoMetadata extends Metadata - - /** - * INTERNAL API - */ - @InternalApi - private[testkit] final case class WithMetadata(payload: Any) extends Metadata - } /** 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 7d8e899b89..77dcd86a25 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 @@ -10,9 +10,8 @@ import scala.util.Try import com.typesafe.config.{ Config, ConfigFactory } import akka.annotation.InternalApi import akka.persistence._ -import akka.persistence.journal.{ AsyncWriteJournal, EventWithMetaData, Tagged } +import akka.persistence.journal.{ AsyncWriteJournal, Tagged } import akka.persistence.snapshot.SnapshotStore -import akka.persistence.testkit.EventStorage.{ NoMetadata, WithMetadata } import akka.persistence.testkit.internal.{ InMemStorageExtension, SnapshotStorageEmulatorExtension } /** @@ -26,32 +25,30 @@ class PersistenceTestKitPlugin extends AsyncWriteJournal { private final val storage = InMemStorageExtension(context.system) 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 { - // TODO define how to handle tagged and metadata - case Tagged(p, _) => (pl.withPayload(p).withTimestamp(System.currentTimeMillis()), NoMetadata) - case evt: EventWithMetaData => - (pl.withPayload(evt.event).withTimestamp(System.currentTimeMillis()), WithMetadata(evt.metaData)) - case _ => (pl.withTimestamp(System.currentTimeMillis()), NoMetadata) + case Tagged(p, _) => pl.withPayload(p).withTimestamp(System.currentTimeMillis()) + case _ => pl.withTimestamp(System.currentTimeMillis()) }) val result: Try[Unit] = storage.tryAdd(data) result.foreach { _ => - messages.foreach(aw => - eventStream.publish(PersistenceTestKitPlugin.Write(aw.persistenceId, aw.highestSequenceNr))) + 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).map(_._1).foreach(recoveryCallback))) + Future.fromTry(Try(storage.tryRead(persistenceId, fromSequenceNr, toSequenceNr, max).foreach(recoveryCallback))) override def asyncReadHighestSequenceNr(persistenceId: String, fromSequenceNr: Long): Future[Long] = Future.fromTry(Try { 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 dd6242ee95..895874a423 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 @@ -8,7 +8,6 @@ import akka.actor.{ ActorSystem, ExtendedActorSystem } import akka.annotation.InternalApi import akka.persistence.PersistentRepr import akka.persistence.testkit.EventStorage -import akka.persistence.testkit.EventStorage.Metadata import akka.persistence.testkit.internal.SerializedEventStorageImpl.Serialized import akka.serialization.{ Serialization, SerializationExtension, Serializers } @@ -21,7 +20,7 @@ private[testkit] object SerializedEventStorageImpl { payloadSerManifest: String, writerUuid: String, payload: Array[Byte], - metadata: Metadata) + metadata: Option[Any]) } /** @@ -37,23 +36,28 @@ private[testkit] class SerializedEventStorageImpl(system: ActorSystem) extends E /** * @return (serializer id, serialized bytes) */ - override def toInternal(repr: (PersistentRepr, Metadata)): Serialized = + override def toInternal(pr: PersistentRepr): Serialized = Serialization.withTransportInformation(system.asInstanceOf[ExtendedActorSystem]) { () => - val (pr, meta) = repr val payload = pr.payload.asInstanceOf[AnyRef] val s = serialization.findSerializerFor(payload) val manifest = Serializers.manifestFor(s, payload) - Serialized(pr.persistenceId, pr.sequenceNr, s.identifier, manifest, pr.writerUuid, s.toBinary(payload), meta) + Serialized( + pr.persistenceId, + pr.sequenceNr, + s.identifier, + manifest, + pr.writerUuid, + s.toBinary(payload), + pr.metadata) } /** * @param internal (serializer id, serialized bytes) */ - override def toRepr(internal: Serialized): (PersistentRepr, Metadata) = { + override def toRepr(internal: Serialized): PersistentRepr = { val event = serialization.deserialize(internal.payload, internal.payloadSerId, internal.payloadSerManifest).get - ( - PersistentRepr(event, internal.sequenceNr, internal.persistenceId, writerUuid = internal.writerUuid), - internal.metadata) + val pr = PersistentRepr(event, 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/SimpleEventStorageImpl.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SimpleEventStorageImpl.scala index 25b63c8042..712c311498 100644 --- a/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SimpleEventStorageImpl.scala +++ b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/SimpleEventStorageImpl.scala @@ -7,7 +7,6 @@ package akka.persistence.testkit.internal import akka.annotation.InternalApi import akka.persistence._ import akka.persistence.testkit.EventStorage -import akka.persistence.testkit.EventStorage.Metadata /** * INTERNAL API @@ -15,10 +14,10 @@ import akka.persistence.testkit.EventStorage.Metadata @InternalApi private[testkit] class SimpleEventStorageImpl extends EventStorage { - override type InternalRepr = (PersistentRepr, Metadata) + override type InternalRepr = PersistentRepr - override def toInternal(repr: (PersistentRepr, Metadata)): (PersistentRepr, Metadata) = repr + override def toInternal(repr: PersistentRepr): PersistentRepr = repr - override def toRepr(internal: (PersistentRepr, Metadata)): (PersistentRepr, Metadata) = internal + override def toRepr(internal: PersistentRepr): PersistentRepr = internal } 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 index af115ae4dc..1f2bc623d0 100644 --- 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 @@ -7,7 +7,6 @@ import akka.actor.ActorRef import akka.annotation.InternalApi import akka.persistence.query.{ EventEnvelope, Sequence } import akka.persistence.testkit.{ EventStorage, PersistenceTestKitPlugin } -import akka.persistence.testkit.EventStorage.{ NoMetadata, WithMetadata } import akka.stream.{ Attributes, Outlet, SourceShape } import akka.stream.stage.{ GraphStage, GraphStageLogic, GraphStageLogicWithLogging, OutHandler } @@ -37,7 +36,6 @@ final private[akka] class EventsByPersistenceIdStage( val (_, msg) = in msg match { case PersistenceTestKitPlugin.Write(pid, toSequenceNr) if pid == persistenceId => - log.debug("Write notification {} {}", pid, toSequenceNr) if (toSequenceNr >= currentSequenceNr) { tryPush() } @@ -50,7 +48,7 @@ final private[akka] class EventsByPersistenceIdStage( val event = storage.tryRead(persistenceId, currentSequenceNr, currentSequenceNr, 1) log.debug("tryPush available. Query for {} {} result {}", currentSequenceNr, currentSequenceNr, event) event.headOption match { - case Some((pr, meta)) => + case Some(pr) => push( out, EventEnvelope( @@ -59,10 +57,7 @@ final private[akka] class EventsByPersistenceIdStage( pr.sequenceNr, pr.payload, pr.timestamp, - meta match { - case NoMetadata => None - case WithMetadata(m) => Some(m) - })) + pr.metadata)) if (currentSequenceNr == toSequenceNr) { completeStage() } else { 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 index fc52243c67..ebc50c233a 100644 --- 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 @@ -8,7 +8,6 @@ import akka.actor.ExtendedActorSystem import akka.persistence.query.{ EventEnvelope, Sequence } import akka.persistence.query.scaladsl.{ CurrentEventsByPersistenceIdQuery, EventsByPersistenceIdQuery, ReadJournal } import akka.persistence.testkit.EventStorage -import akka.persistence.testkit.EventStorage.{ NoMetadata, WithMetadata } import akka.persistence.testkit.internal.InMemStorageExtension import akka.persistence.testkit.query.internal.EventsByPersistenceIdStage import akka.stream.scaladsl.Source @@ -22,7 +21,7 @@ final class PersistenceTestKitReadJournal(system: ExtendedActorSystem) with EventsByPersistenceIdQuery with CurrentEventsByPersistenceIdQuery { - private final val storage: EventStorage = InMemStorageExtension(system) + private val storage: EventStorage = InMemStorageExtension(system) override def eventsByPersistenceId( persistenceId: String, @@ -35,12 +34,8 @@ final class PersistenceTestKitReadJournal(system: ExtendedActorSystem) persistenceId: String, fromSequenceNr: Long, toSequenceNr: Long): Source[EventEnvelope, NotUsed] = { - Source(storage.tryRead(persistenceId, fromSequenceNr, toSequenceNr, Long.MaxValue)).map { - case (pr, meta) => - EventEnvelope(Sequence(pr.sequenceNr), persistenceId, pr.sequenceNr, pr.payload, pr.timestamp, meta match { - case NoMetadata => None - case WithMetadata(payload) => Some(payload) - }) + Source(storage.tryRead(persistenceId, fromSequenceNr, toSequenceNr, Long.MaxValue)).map { pr => + EventEnvelope(Sequence(pr.sequenceNr), persistenceId, pr.sequenceNr, 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 4a959adfc1..1aa3910860 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 @@ -18,7 +18,6 @@ import akka.annotation.ApiMayChange import akka.persistence.Persistence import akka.persistence.PersistentRepr import akka.persistence.SnapshotMetadata -import akka.persistence.testkit.EventStorage.Metadata import akka.persistence.testkit._ import akka.persistence.testkit.internal.InMemStorageExtension import akka.persistence.testkit.internal.SnapshotStorageEmulatorExtension @@ -423,9 +422,9 @@ object SnapshotTestKit { */ @ApiMayChange class PersistenceTestKit(system: ActorSystem) - extends PersistenceTestKitOps[(PersistentRepr, Metadata), JournalOperation] - with ExpectOps[(PersistentRepr, Metadata)] - with HasStorage[JournalOperation, (PersistentRepr, Metadata)] { + extends PersistenceTestKitOps[PersistentRepr, JournalOperation] + with ExpectOps[PersistentRepr] + with HasStorage[JournalOperation, PersistentRepr] { require( Try(Persistence(system).journalFor(PersistenceTestKitPlugin.PluginId)).isSuccess, "The test persistence plugin is not configured.") @@ -494,7 +493,7 @@ 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, Metadata)): Any = repr._1.payload + override private[testkit] def reprToAny(repr: PersistentRepr): Any = repr.payload } @ApiMayChange 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 index 144a98108c..a642a79788 100644 --- 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 @@ -4,16 +4,14 @@ package akka.persistence.testkit.query -import akka.{ Done, NotUsed } +import akka.Done import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } import akka.actor.typed.ActorRef -import akka.persistence.journal.EventWithMetaData import akka.persistence.query.{ EventEnvelope, PersistenceQuery } import akka.persistence.testkit.PersistenceTestKitPlugin import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal -import akka.persistence.typed.{ EventAdapter, EventSeq, PersistenceId } +import akka.persistence.typed.PersistenceId import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior } -import akka.stream.scaladsl.Source import akka.stream.testkit.scaladsl.TestSink import com.typesafe.config.ConfigFactory import org.scalatest.wordspec.AnyWordSpecLike @@ -40,20 +38,7 @@ object EventsByPersistenceIdSpec { command.ack ! Done }, (state, _) => state) - }.eventAdapter(new EventAdapter[String, Any] { - override def toJournal(e: String): Any = { - if (e.startsWith("m")) { - EventWithMetaData(e, s"$e-meta") - } else { - e - } - } - override def manifest(event: String): String = "" - override def fromJournal(p: Any, manifest: String): EventSeq[String] = p match { - case e: EventWithMetaData => EventSeq.single(e.event.toString) - case _ => EventSeq.single(p.toString) - } - }) + } } @@ -148,22 +133,5 @@ class EventsByPersistenceIdSpec probe.cancel() } - - "return metadata in queries" in { - val ackProbe = createTestProbe[Done]() - val ref = setupEmpty("with-meta") - ref ! Command("m-1", ackProbe.ref) - ref ! Command("m-2", ackProbe.ref) - val src: Source[EventEnvelope, NotUsed] = queries.eventsByPersistenceId("with-meta", 0L, Long.MaxValue) - val probe = - src.runWith(TestSink.probe[Any]).request(3) - probe.expectNextPF { - case e @ EventEnvelope(_, "with-meta", 1L, "m-1") if e.eventMetadata.contains("m-1-meta") => - } - - probe.expectNextPF { - case e @ EventEnvelope(_, "with-meta", 2L, "m-2") if e.eventMetadata.contains("m-2-meta") => - } - } } } 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/scala/akka/persistence/typed/ActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala new file mode 100644 index 0000000000..bb2ee63290 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala @@ -0,0 +1,148 @@ +/* + * 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 +import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal +import akka.persistence.typed.scaladsl.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import org.scalatest.concurrent.Eventually +import org.scalatest.wordspec.AnyWordSpecLike + +object ActiveActiveSpec { + + val AllReplicas = Set("R1", "R2", "R3") + + sealed trait Command + case class GetState(replyTo: ActorRef[State]) extends Command + case class StoreMe(description: String, replyTo: ActorRef[Done]) extends Command + case class GetReplica(replyTo: ActorRef[(String, Set[String])]) extends Command + + case class State(all: List[String]) + def testBehavior(entityId: String, replicaId: String, probe: ActorRef[EventAndContext]): Behavior[Command] = + testBehavior(entityId, replicaId, Some(probe)) + + def testBehavior( + entityId: String, + replicaId: String, + probe: Option[ActorRef[EventAndContext]] = None): Behavior[Command] = + ActiveActiveEventSourcing(entityId, replicaId, AllReplicas, PersistenceTestKitReadJournal.Identifier)( + aaContext => + EventSourcedBehavior[Command, String, State]( + aaContext.persistenceId, + State(Nil), + (state, command) => + command match { + case GetState(replyTo) => + replyTo ! state + Effect.none + case GetReplica(replyTo) => + replyTo.tell((aaContext.replicaId, aaContext.allReplicas)) + Effect.none + case StoreMe(evt, ack) => + Effect.persist(evt).thenRun(_ => ack ! Done) + }, + (state, event) => { + probe.foreach(_ ! EventAndContext(event, aaContext.origin, aaContext.recoveryRunning)) + state.copy(all = event :: state.all) + })) + +} + +case class EventAndContext(event: Any, origin: String, recoveryRunning: Boolean = false) + +class ActiveActiveSpec + extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) + with AnyWordSpecLike + with LogCapturing + with Eventually { + import ActiveActiveSpec._ + val ids = new AtomicInteger(0) + def nextEntityId = s"e-${ids.getAndIncrement()}" + "ActiveActiveEventSourcing" should { + "replicate events between 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") + } + } + + "have access to replica information" in { + val entityId = nextEntityId + val probe = createTestProbe[(String, Set[String])]() + val r1 = spawn(testBehavior(entityId, "R1")) + r1 ! GetReplica(probe.ref) + probe.expectMessage(("R1", Set("R1", "R2", "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", "R1")) + eventProbeR1.expectMessage(EventAndContext("from r1", "R1")) + + r2 ! StoreMe("from r2", replyProbe.ref) + eventProbeR1.expectMessage(EventAndContext("from r2", "R2")) + eventProbeR2.expectMessage(EventAndContext("from r2", "R2")) + } + + "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", "R1", recoveryRunning = false)) + replyProbe.expectMessage(Done) + + val recoveryProbe = createTestProbe[EventAndContext]() + spawn(testBehavior(entityId, "R1", recoveryProbe.ref)) + recoveryProbe.expectMessage(EventAndContext("Event", "R1", recoveryRunning = true)) + } + + "persist all" in { + pending + } + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala index 4fc552a0f1..99106b5c71 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala @@ -61,13 +61,19 @@ class AABlogExampleSpec cmd match { case AddPost(_, content, replyTo) => val evt = - PostAdded(aa.persistenceId.id, content, state.contentTimestamp.increase(aa.timestamp, aa.replicaId)) + PostAdded( + aa.persistenceId.id, + content, + state.contentTimestamp.increase(aa.currentTimeMillis(), aa.replicaId)) Effect.persist(evt).thenRun { _ => replyTo ! AddPostDone(aa.entityId) } case ChangeBody(_, newContent, replyTo) => val evt = - BodyChanged(aa.persistenceId.id, newContent, state.contentTimestamp.increase(aa.timestamp, aa.replicaId)) + BodyChanged( + aa.persistenceId.id, + newContent, + state.contentTimestamp.increase(aa.currentTimeMillis(), aa.replicaId)) Effect.persist(evt).thenRun { _ => replyTo ! Done } 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 4b4f8205bc..0087e5d20b 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 @@ -242,7 +242,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( } override private[akka] def withActiveActive( - context: ActiveActiveContext, + context: ActiveActiveContextImpl, id: String, allIds: Set[String], queryPluginId: String): EventSourcedBehavior[Command, Event, State] = { @@ -265,7 +265,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( // FIXME serializer @InternalApi -private[akka] final case class ReplicatedEventMetaData(originDc: String) +private[akka] final case class ReplicatedEventMetaData(originReplica: String, originSequenceNr: Long) @InternalApi private[akka] final case class ReplicatedEvent[E](event: E, originReplica: String, originSequenceNr: Long) @InternalApi 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 8345a26d30..ec6291e5bf 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,7 +18,7 @@ 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 @@ -34,7 +33,8 @@ private[akka] trait JournalInteractions[C, E, S] { cmd: Any, state: Running.RunningState[S], event: EventOrTaggedOrReplicated, - eventAdapterManifest: String): Running.RunningState[S] = { + eventAdapterManifest: String, + metadata: OptionVal[Any]): Running.RunningState[S] = { val newRunningState = state.nextSequenceNr() @@ -50,7 +50,11 @@ private[akka] trait JournalInteractions[C, E, S] { // 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) 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 0b2735723d..eafe1e9eb6 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 @@ -120,7 +119,21 @@ 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) + + setup.activeActive match { + case Some(aa) => + val meta = repr.metadata match { + case Some(m) => m.asInstanceOf[ReplicatedEventMetaData] + case None => + throw new IllegalStateException( + s"Active active enabled but existing event has no metadata. Migration isn't supported yet.") + + } + aa.setContext(recoveryRunning = true, meta.originReplica) + case None => + } + val newState = setup.eventHandler(state.state, event) + state = state.copy(state = newState, eventSeenInInterval = true) } eventSeq match { @@ -240,7 +253,9 @@ private[akka] final class ReplayingEvents[C, E, S]( Behaviors.stopped else { val seenPerReplica: Map[String, Long] = - setup.activeActive.map(aa => aa.allReplicas.map(replica => replica -> 0L).toMap).getOrElse(Map.empty) + setup.activeActive + .map(aa => aa.allReplicas.filterNot(_ == aa.replicaId).map(replica => replica -> 0L).toMap) + .getOrElse(Map.empty) val running = Running[C, E, S]( setup, 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 846051b758..e932ccf873 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 @@ -23,7 +23,7 @@ import akka.persistence.PersistentRepr import akka.persistence.SaveSnapshotFailure import akka.persistence.SaveSnapshotSuccess import akka.persistence.SnapshotProtocol -import akka.persistence.journal.{ EventWithMetaData, Tagged } +import akka.persistence.journal.Tagged import akka.persistence.query.{ EventEnvelope, PersistenceQuery } import akka.persistence.query.scaladsl.EventsByPersistenceIdQuery import akka.persistence.typed.{ @@ -47,7 +47,7 @@ import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive import akka.stream.{ SharedKillSwitch, SystemMaterializer } import akka.stream.scaladsl.{ RestartSource, Sink } import akka.stream.typed.scaladsl.ActorFlow -import akka.util.{ unused, Timeout } +import akka.util.{ unused, OptionVal, Timeout } /** * INTERNAL API @@ -116,15 +116,16 @@ private[akka] object Running { implicit val timeout = Timeout(30.seconds) + // FIXME config val source = RestartSource.withBackoff(2.seconds, 10.seconds, randomFactor = 0.2) { () => replication .eventsByPersistenceId(pid.id, seqNr + 1, Long.MaxValue) .via(ActorFlow.ask[EventEnvelope, ReplicatedEventEnvelope[E], ReplicatedEventAck.type](ref) { (eventEnvelope, replyTo) => - val re = ReplicatedEvent[E]( - eventEnvelope.event.asInstanceOf[E], - eventEnvelope.eventMetadata.get.asInstanceOf[ReplicatedEventMetaData].originDc, - eventEnvelope.sequenceNr) // FIXME, this is the wrong sequence nr, we need origin sequence nr, follow up with tests that show this + // Need to handle this not being available migration from non-active-active is supported + val meta = eventEnvelope.eventMetadata.get.asInstanceOf[ReplicatedEventMetaData] + val re = + ReplicatedEvent[E](eventEnvelope.event.asInstanceOf[E], meta.originReplica, meta.originSequenceNr) ReplicatedEventEnvelope(re, replyTo) }) } @@ -161,7 +162,7 @@ private[akka] object Running { def onMessage(msg: InternalProtocol): Behavior[InternalProtocol] = msg match { case IncomingCommand(c: C @unchecked) => onCommand(state, c) - case re: ReplicatedEventEnvelope[E @unchecked] => onReplicatedEvent(state, re) + case re: ReplicatedEventEnvelope[E @unchecked] => onReplicatedEvent(state, re, setup.activeActive.get) case JournalResponse(r) => onDeleteEventsJournalResponse(r, state.state) case SnapshotterResponse(r) => onDeleteSnapshotResponse(r, state.state) case get: GetState[S @unchecked] => onGetState(get) @@ -186,18 +187,20 @@ private[akka] object Running { def onReplicatedEvent( state: Running.RunningState[S], - envelope: ReplicatedEventEnvelope[E]): Behavior[InternalProtocol] = { - // FIXME set the details on the context https://github.com/akka/akka/issues/29258 + envelope: ReplicatedEventEnvelope[E], + activeActive: ActiveActive): Behavior[InternalProtocol] = { setup.log.infoN( - "Replica {} received replicated event. Replica seqs nrs: {}", + "Replica {} received replicated event. Replica seqs nrs: {}. Envelope {}", setup.activeActive, - state.seenPerReplica) + state.seenPerReplica, + envelope) envelope.ack ! ReplicatedEventAck - if (envelope.event.originReplica != setup.activeActive.get.replicaId && !alreadySeen(envelope.event)) { - setup.log.info("Saving event as first time") - handleReplicatedEventPersist(envelope.event) + if (envelope.event.originReplica != activeActive.replicaId && !alreadySeen(envelope.event)) { + activeActive.setContext(false, envelope.event.originReplica) + setup.log.debug("Saving event as first time") + handleExternalReplicatedEventPersist(envelope.event) } else { - setup.log.info("Filtering event as already seen") + setup.log.debug("Filtering event as already seen") this } } @@ -208,11 +211,16 @@ private[akka] object Running { this } - private def handleReplicatedEventPersist(event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { + private def handleExternalReplicatedEventPersist(event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { _currentSequenceNumber = state.seqNr + 1 - val replicatedEvent = new EventWithMetaData(event.event, ReplicatedEventMetaData(event.originReplica)) val newState: RunningState[S] = state.applyEvent(setup, event.event) - val newState2: RunningState[S] = internalPersist(setup.context, null, newState, replicatedEvent, "") + val newState2: RunningState[S] = internalPersist( + setup.context, + null, + newState, + event.event, + "", + OptionVal.Some(ReplicatedEventMetaData(event.originReplica, event.originSequenceNr))) 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) @@ -224,6 +232,37 @@ private[akka] object Running { Nil) } + private def handleEventPersist(event: E, cmd: Any, sideEffects: immutable.Seq[SideEffect[S]]) = { + // 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.activeActive.foreach { aa => + aa.setContext(recoveryRunning = false, aa.replicaId) + } + val newState: RunningState[S] = state.applyEvent(setup, event) + + val eventToPersist = adaptEvent(event) + val eventAdapterManifest = setup.eventAdapter.manifest(event) + + val newState2 = setup.activeActive match { + case Some(aa) => + internalPersist( + setup.context, + cmd, + newState, + eventToPersist, + eventAdapterManifest, + OptionVal.Some(ReplicatedEventMetaData(aa.replicaId, _currentSequenceNumber))) + case None => + internalPersist(setup.context, cmd, newState, eventToPersist, eventAdapterManifest, OptionVal.None) + } + + val shouldSnapshotAfterPersist = setup.shouldSnapshot(newState2.state, event, newState2.seqNr) + (persistingEvents(newState2, state, numberOfEvents = 1, shouldSnapshotAfterPersist, sideEffects), false) + } + @tailrec def applyEffects( msg: Any, state: RunningState[S], @@ -242,27 +281,7 @@ 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: RunningState[S] = state.applyEvent(setup, event) - - val eventToPersist = adaptEvent(event) - val eventAdapterManifest = setup.eventAdapter.manifest(event) - - val newState2 = setup.activeActive match { - case Some(aa) => - val replicatedEvent = ReplicatedEvent(eventToPersist, aa.replicaId, _currentSequenceNumber) - internalPersist(setup.context, msg, newState, replicatedEvent, eventAdapterManifest) - case None => - 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 diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala index 847b2cbff6..f0695b7b8d 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala @@ -5,6 +5,7 @@ package akka.persistence.typed.scaladsl import akka.persistence.typed.PersistenceId +import akka.util.WallClock /** * Utility class for comparing timestamp and data center @@ -34,8 +35,9 @@ final case class LwwTime(timestamp: Long, originDc: String) { } } +// FIXME docs trait ActiveActiveContext { - def timestamp: Long + def origin: String def concurrent: Boolean def replicaId: String @@ -44,6 +46,7 @@ trait ActiveActiveContext { def recoveryRunning: Boolean def entityId: String def currentTimeMillis(): Long + } // FIXME, parts of this can be set during initialisation @@ -51,18 +54,11 @@ trait ActiveActiveContext { // https://github.com/akka/akka/issues/29258 private[akka] class ActiveActiveContextImpl(val entityId: String, val replicaId: String, val allReplicas: Set[String]) extends ActiveActiveContext { - var _timestamp: Long = -1 var _origin: String = null - var _concurrent: Boolean = false + var _recoveryRunning: Boolean = false // FIXME check illegal access https://github.com/akka/akka/issues/29264 - /** - * The timestamp of the event. Always increases per data center - * Undefined result if called from any where other than an event handler. - */ - override def timestamp: Long = _timestamp - /** * The origin of the current event. * Undefined result if called from anywhere other than an event handler. @@ -73,13 +69,14 @@ private[akka] class ActiveActiveContextImpl(val entityId: String, val replicaId: * 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 = _concurrent + override def concurrent: Boolean = throw new UnsupportedOperationException("TODO") + override def persistenceId: PersistenceId = PersistenceId.replicatedUniqueId(entityId, replicaId) + override def currentTimeMillis(): Long = { - // FIXME always increasing - System.currentTimeMillis() + WallClock.AlwaysIncreasingClock.currentTimeMillis() } - override def recoveryRunning: Boolean = false + override def recoveryRunning: Boolean = _recoveryRunning } object ActiveActiveEventSourcing { 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 e0a4984587..e53a1918a0 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 @@ -25,8 +25,18 @@ object EventSourcedBehavior { private[akka] case class ActiveActive( replicaId: String, allReplicas: Set[String], - aaContext: ActiveActiveContext, - queryPluginId: String) + aaContext: ActiveActiveContextImpl, + queryPluginId: String) { + + /** + * Must only be called on the same thread that will execute the user code + */ + def setContext(recoveryRunning: Boolean, originReplica: String): Unit = { + aaContext._recoveryRunning = recoveryRunning + aaContext._origin = originReplica + } + + } /** * Type alias for the command handler function that defines how to act on commands. @@ -152,7 +162,7 @@ object EventSourcedBehavior { def withJournalPluginId(id: String): EventSourcedBehavior[Command, Event, State] private[akka] def withActiveActive( - context: ActiveActiveContext, + context: ActiveActiveContextImpl, replicaId: String, allReplicaIds: Set[String], queryPluginId: String): EventSourcedBehavior[Command, Event, State] diff --git a/akka-persistence/src/main/scala/akka/persistence/Persistent.scala b/akka-persistence/src/main/scala/akka/persistence/Persistent.scala index f260f79124..b7285058f9 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 } @@ -235,5 +245,4 @@ private[persistence] final case class PersistentImpl( override def toString: String = { s"PersistentRepr($persistenceId,$sequenceNr,$writerUuid,$timestamp)" } - } diff --git a/akka-persistence/src/main/scala/akka/persistence/journal/EventWithMetaData.scala b/akka-persistence/src/main/scala/akka/persistence/journal/EventWithMetaData.scala deleted file mode 100644 index c1bfc6aa11..0000000000 --- a/akka-persistence/src/main/scala/akka/persistence/journal/EventWithMetaData.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Copyright (C) 2020 Lightbend Inc. - */ - -package akka.persistence.journal - -object EventWithMetaData { - - def apply(event: Any, metaData: Any): EventWithMetaData = new EventWithMetaData(event, metaData) - - /** - * If meta data could not be deserialized it will not fail the replay/query. - * The "invalid" meta data is represented with this `UnknownMetaData` and - * it and the event will be wrapped in `EventWithMetaData`. - * - * The reason for not failing the replay/query is that meta data should be - * optional, e.g. the tool that wrote the meta data has been removed. This - * is typically because the serializer for the meta data has been removed - * from the class path (or configuration). - */ - final class UnknownMetaData(val serializerId: Int, val manifest: String) -} - -/** - * If the event is wrapped in this class the `metaData` will - * be serialized and stored separately from the event payload. This can be used by event - * adapters or other tools to store additional meta data without altering - * the actual domain event. - * - * Check the documentation of the persistence plugin in use to use if it supports - * EventWithMetaData. - */ -final class EventWithMetaData(val event: Any, val metaData: Any) 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 53cf784f70..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 @@ -19,7 +19,7 @@ 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, EventWithMetaData, Tagged } +import akka.persistence.journal.{ AsyncWriteJournal, Tagged } import akka.serialization.SerializationExtension import akka.serialization.Serializers import akka.util.OptionVal @@ -143,13 +143,10 @@ object InmemJournal { // persistenceId -> highest used sequence number private var highestSequenceNumbers = Map.empty[String, Long] - // FIXME, which way around should Tagged/EventWithMeta go? https://github.com/akka/akka/issues/29284 def add(p: PersistentRepr): Unit = { val pr = p.payload match { case Tagged(payload, _) => (p.withPayload(payload).withTimestamp(System.currentTimeMillis()), OptionVal.None) - case meta: EventWithMetaData => - (p.withPayload(meta.event).withTimestamp(System.currentTimeMillis()), OptionVal.Some(meta.metaData)) - case _ => (p.withTimestamp(System.currentTimeMillis()), OptionVal.None) + case _ => (p.withTimestamp(System.currentTimeMillis()), OptionVal.None) } messages = messages + (messages.get(p.persistenceId) match { From c44302bd1e8fca873cd5eaf89a5b1d3fe2150156 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Mon, 29 Jun 2020 08:06:59 +0200 Subject: [PATCH 04/50] Speculative replication - step 1 (#29289) * Step 1: general event-publishing-to-topic feature * Step 2: an actor subscribes to the topic and forwards events to the sharded replicas * Another half piece of the puzzle, receive the PublishedEvent in the ESB internals * Stash published events while replaying rather than drop * Publish on the event stream instead of a topic * Active active actor receiving a published event * Some smaller changes * Public API for published event * Better name for the sharding component * Naive test for the Active Active Sharding Replication * Java API for ActiveActiveShardingReplication * Spelling * Use ShardingEnvelope for publishing the event across sharding * Fast forwarding filter stage * Move test to testkit, enable the see-event-twice test (fails) * Use persistence testkit journal * Various smaller review feedback things * Trying to figure out why duplicate event write test fails * Missing unstash after processing published event Co-authored-by: Christopher Batey --- ...ctiveActiveShardingDirectReplication.scala | 90 ++++++++++ .../typed/Murmur2MessageExtractor.scala | 2 +- ...eActiveShardingDirectReplicationSpec.scala | 52 ++++++ .../src/test/resources/logback-test.xml | 4 +- .../ActiveActiveEventPublishingSpec.scala | 150 ++++++++++++++++ .../typed/EventPublishingSpec.scala | 84 +++++++++ .../persistence/typed/PublishedEvent.scala | 31 ++++ .../typed/internal/BehaviorSetup.scala | 13 +- .../internal/EventSourcedBehaviorImpl.scala | 42 ++++- .../typed/internal/FastForwardingFilter.scala | 72 ++++++++ .../typed/internal/ReplayingEvents.scala | 8 +- .../typed/internal/ReplayingSnapshot.scala | 5 + .../persistence/typed/internal/Running.scala | 166 ++++++++++++++++-- .../typed/internal/StashManagement.scala | 2 +- .../typed/scaladsl/EventSourcedBehavior.scala | 10 +- .../src/test/resources/logback-test.xml | 2 +- .../EventSourcedBehaviorWatchSpec.scala | 3 +- build.sbt | 2 +- 18 files changed, 709 insertions(+), 29 deletions(-) create mode 100644 akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala create mode 100644 akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/EventPublishingSpec.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/PublishedEvent.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/FastForwardingFilter.scala diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala new file mode 100644 index 0000000000..fdb843962b --- /dev/null +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala @@ -0,0 +1,90 @@ +/* + * 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.ApiMayChange +import akka.annotation.DoNotInherit +import akka.annotation.InternalApi +import akka.persistence.typed.PublishedEvent + +import scala.collection.JavaConverters._ + +/** + * Used when sharding Active Active 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 Active Active entities will run (the same nodes that you start + * sharding on). + * + * 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. + */ +@ApiMayChange +object ActiveActiveShardingDirectReplication { + + /** + * Not for user extension + */ + @DoNotInherit + sealed trait Command + + /** + * INTERNAL API + */ + @InternalApi + private[akka] case class VerifyStarted(replyTo: ActorRef[Done]) extends Command + + /** + * Java API: + * @param selfReplica The replica id of the replica that runs on this node + * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system + */ + def create[T](selfReplica: String, replicaShardingProxies: java.util.Map[String, ActorRef[T]]): Behavior[Command] = + apply(selfReplica, replicaShardingProxies.asScala.toMap) + + /** + * Scala API: + * @param selfReplica The replica id of the replica that runs on this node + * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system + */ + def apply[T](selfReplica: String, replicaShardingProxies: Map[String, ActorRef[T]]): Behavior[Command] = + Behaviors + .setup[Any] { context => + context.log.debug( + "Subscribing to event stream to forward events to [{}] sharded replicas", + replicaShardingProxies.size - 1) + context.system.eventStream ! EventStream.Subscribe[PublishedEvent](context.self) + + Behaviors.receiveMessagePartial { + case event: PublishedEvent => + context.log.trace( + "Forwarding event for persistence id [{}] sequence nr [{}] to replicas", + event.persistenceId, + event.sequenceNumber) + replicaShardingProxies.foreach { + case (replica, proxy) => + val envelopedEvent = ShardingEnvelope(event.persistenceId.id, event) + if (replica != selfReplica) + proxy.asInstanceOf[ActorRef[ShardingEnvelope[PublishedEvent]]] ! envelopedEvent + } + Behaviors.same + case VerifyStarted(replyTo) => + replyTo ! Done + Behaviors.same + } + } + .narrow[Command] + +} 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/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala new file mode 100644 index 0000000000..b9a939f387 --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala @@ -0,0 +1,52 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.cluster.sharding.typed + +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.PersistenceId +import akka.persistence.typed.internal.PublishedEventImpl +import org.scalatest.wordspec.AnyWordSpecLike + +class ActiveActiveShardingDirectReplicationSpec + extends ScalaTestWithActorTestKit + with AnyWordSpecLike + with LogCapturing { + + "Active active sharding replication" must { + + "replicate published events to all sharding proxies" in { + val replicaAProbe = createTestProbe[Any]() + val replicaBProbe = createTestProbe[Any]() + val replicaCProbe = createTestProbe[Any]() + + val replicationActor = spawn( + ActiveActiveShardingDirectReplication( + "ReplicaA", + replicaShardingProxies = + Map("ReplicaA" -> replicaAProbe.ref, "ReplicaB" -> replicaBProbe.ref, "ReplicaC" -> replicaCProbe.ref))) + + val upProbe = createTestProbe[Done]() + replicationActor ! ActiveActiveShardingDirectReplication.VerifyStarted(upProbe.ref) + upProbe.receiveMessage() // not bullet proof wrt to subscription being complete but good enough + + val event = PublishedEventImpl( + Some("ReplicaA"), + PersistenceId.replicatedUniqueId("pid", "ReplicaA"), + 1L, + "event", + System.currentTimeMillis()) + system.eventStream ! EventStream.Publish(event) + + replicaBProbe.expectMessageType[ShardingEnvelope[_]].message should equal(event) + replicaCProbe.expectMessageType[ShardingEnvelope[_]].message should equal(event) + replicaAProbe.expectNoMessage() // no publishing to the replica emitting it + } + + } + +} diff --git a/akka-persistence-typed-tests/src/test/resources/logback-test.xml b/akka-persistence-typed-tests/src/test/resources/logback-test.xml index c980894390..df3cd05c69 100644 --- a/akka-persistence-typed-tests/src/test/resources/logback-test.xml +++ b/akka-persistence-typed-tests/src/test/resources/logback-test.xml @@ -5,7 +5,7 @@ - %date{ISO8601} [%-5level] [%logger] [%marker] [%X{akkaSource}] [%X{persistenceId}] - %msg %n + %date{ISO8601} [%-5level] [%logger] [%marker] [%X{akkaSource}] [%X{persistencePhase}] [%X{persistenceId}] - %msg %n @@ -25,7 +25,7 @@ - + diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala new file mode 100644 index 0000000000..c1238b2531 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala @@ -0,0 +1,150 @@ +/* + * 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.scaladsl.ActiveActiveEventSourcing +import akka.persistence.typed.scaladsl.Effect +import akka.persistence.typed.scaladsl.EventSourcedBehavior +import org.scalatest.wordspec.AnyWordSpecLike + +object ActiveActiveEventPublishingSpec { + + object MyActiveActive { + trait Command + case class Add(text: String, replyTo: ActorRef[Done]) extends Command + case class Get(replyTo: ActorRef[Set[String]]) extends Command + + def apply(entityId: String, replicaId: String, allReplicas: Set[String]): Behavior[Command] = + Behaviors.setup { ctx => + ActiveActiveEventSourcing(entityId, replicaId, allReplicas, PersistenceTestKitReadJournal.Identifier)( + aactx => + EventSourcedBehavior[Command, String, Set[String]]( + aactx.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 + }, + (state, string) => state + string)) + } + } +} + +class ActiveActiveEventPublishingSpec + extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) + with AnyWordSpecLike + with LogCapturing { + + import ActiveActiveEventPublishingSpec._ + + "An active active actor" must { + "move forward when a published event from a replica is received" in { + + val actor = spawn(MyActiveActive("myId1", "DC-A", Set("DC-A", "DC-B"))) + val probe = createTestProbe[Any]() + actor ! MyActiveActive.Add("one", probe.ref) + probe.expectMessage(Done) + + // simulate a published event from another replica + actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( + Some("DC-B"), + PersistenceId.replicatedUniqueId("myId1", "DC-B"), + 1L, + "two", + System.currentTimeMillis()) + actor ! MyActiveActive.Add("three", probe.ref) + probe.expectMessage(Done) + + actor ! MyActiveActive.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 actor = spawn(MyActiveActive("myId2", "DC-A", Set("DC-A", "DC-B"))) + val probe = createTestProbe[Any]() + actor ! MyActiveActive.Add("one", probe.ref) + probe.expectMessage(Done) + + // simulate a published event from another replica + actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( + Some("DC-B"), + PersistenceId.replicatedUniqueId("myId2", "DC-B"), + 2L, // missing 1L + "two", + System.currentTimeMillis()) + actor ! MyActiveActive.Add("three", probe.ref) + probe.expectMessage(Done) + + actor ! MyActiveActive.Get(probe.ref) + probe.expectMessage(Set("one", "three")) + } + + "ignore a published event from an unknown replica" in { + val actor = spawn(MyActiveActive("myId3", "DC-A", Set("DC-A", "DC-B"))) + val probe = createTestProbe[Any]() + actor ! MyActiveActive.Add("one", probe.ref) + probe.expectMessage(Done) + + // simulate a published event from another replica + actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( + Some("DC-C"), + PersistenceId.replicatedUniqueId("myId3", "DC-C"), + 1L, + "two", + System.currentTimeMillis()) + actor ! MyActiveActive.Add("three", probe.ref) + probe.expectMessage(Done) + + actor ! MyActiveActive.Get(probe.ref) + probe.expectMessage(Set("one", "three")) + } + + "ignore an already seen event from a replica" in { + val actor = spawn(MyActiveActive("myId4", "DC-A", Set("DC-A", "DC-B"))) + val probe = createTestProbe[Any]() + actor ! MyActiveActive.Add("one", probe.ref) + probe.expectMessage(Done) + + // simulate a published event from another replica + actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( + Some("DC-B"), + PersistenceId.replicatedUniqueId("myId4", "DC-B"), + 1L, + "two", + System.currentTimeMillis()) + // simulate another published event from that replica + actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( + Some("DC-B"), + PersistenceId.replicatedUniqueId("myId4", "DC-B"), + 1L, + "two-again", // ofc this would be the same in the real world, different just so we can detect + System.currentTimeMillis()) + + actor ! MyActiveActive.Add("three", probe.ref) + probe.expectMessage(Done) + + actor ! MyActiveActive.Get(probe.ref) + probe.expectMessage(Set("one", "two", "three")) + } + + } + +} 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..a899beebc4 --- /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() + } +} + +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[Any]() + system.eventStream ! EventStream.Subscribe[PublishedEvent](topicProbe.ref.narrow) + // 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.expectMessageType[PublishedEvent] + 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.expectMessageType[PublishedEvent] + 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/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..67a9560b83 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/PublishedEvent.scala @@ -0,0 +1,31 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed + +import java.util.Optional + +import akka.annotation.DoNotInherit + +/** + * 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 Active Active actor this will contain the replica id */ + def replicaId: Option[String] + + /** Java API: When emitted from an Active Active actor this will contain the replica id */ + def getReplicaId: Optional[String] + 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/internal/BehaviorSetup.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala index 6f87ad5134..20281bace5 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 @@ -7,7 +7,7 @@ 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 @@ -48,7 +48,8 @@ private[akka] final class BehaviorSetup[C, E, S]( var holdingRecoveryPermit: Boolean, val settings: EventSourcedSettings, val stashState: StashState, - val activeActive: Option[ActiveActive]) { + val activeActive: Option[ActiveActive], + 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[String] = activeActive.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 0087e5d20b..a14d0735f4 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,7 @@ 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.SnapshotAdapter import akka.persistence.typed.SnapshotCompleted import akka.persistence.typed.SnapshotFailed @@ -89,7 +92,8 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( retention: RetentionCriteria = RetentionCriteria.disabled, supervisionStrategy: SupervisorStrategy = SupervisorStrategy.stop, override val signalHandler: PartialFunction[(State, Signal), Unit] = PartialFunction.empty, - activeActive: Option[ActiveActive] = None) + activeActive: Option[ActiveActive] = None, + publishEvents: Boolean = false) extends EventSourcedBehavior[Command, Event, State] { import EventSourcedBehaviorImpl.WriterIdentity @@ -153,7 +157,8 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( holdingRecoveryPermit = false, settings = settings, stashState = stashState, - activeActive = activeActive) + activeActive = activeActive, + publishEvents = publishEvents) // needs to accept Any since we also can get messages from the journal // not part of the user facing Command protocol @@ -241,6 +246,10 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( copy(recovery = recovery.toClassic) } + override def withEventPublishing(): EventSourcedBehavior[Command, Event, State] = { + copy(publishEvents = true) + } + override private[akka] def withActiveActive( context: ActiveActiveContextImpl, id: String, @@ -261,6 +270,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( final case class ReplicatedEventEnvelope[E](event: ReplicatedEvent[E], ack: ActorRef[ReplicatedEventAck.type]) extends InternalProtocol + } // FIXME serializer @@ -270,3 +280,31 @@ private[akka] final case class ReplicatedEventMetaData(originReplica: String, or private[akka] final case class ReplicatedEvent[E](event: E, originReplica: String, originSequenceNr: Long) @InternalApi private[akka] case object ReplicatedEventAck + +/** + * INTERNAL API + */ +@InternalApi +private[akka] final case class PublishedEventImpl( + replicaId: Option[String], + persistenceId: PersistenceId, + sequenceNumber: Long, + payload: Any, + timestamp: Long) + extends PublishedEvent + with InternalProtocol { + import scala.compat.java8.OptionConverters._ + + override def getReplicaId: Optional[String] = replicaId.asJava + + 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 + } + +} 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 eafe1e9eb6..c229c842a1 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 @@ -93,6 +93,7 @@ private[akka] final class ReplayingEvents[C, E, S]( 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 RecoveryPermitGranted => Behaviors.unhandled // should not happen, we already have the permit @@ -259,7 +260,12 @@ private[akka] final class ReplayingEvents[C, E, S]( val running = Running[C, E, S]( setup, - Running.RunningState[S](state.seqNr, state.state, state.receivedPoisonPill, seenPerReplica)) + Running.RunningState[S]( + seqNr = state.seqNr, + state = state.state, + receivedPoisonPill = state.receivedPoisonPill, + seenPerReplica = seenPerReplica, + 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 6c471b9201..cfabb7a43c 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 @@ -61,6 +61,7 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup 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) @@ -127,6 +128,10 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup 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...", 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 e932ccf873..f24914e4f8 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,13 +4,17 @@ package akka.persistence.typed.internal +import java.util.concurrent.atomic.AtomicReference + import scala.annotation.tailrec import scala.collection.immutable import akka.actor.UnhandledMessage -import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, 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 @@ -44,10 +48,14 @@ import akka.persistence.typed.internal.InternalProtocol.ReplicatedEventEnvelope import akka.persistence.typed.internal.Running.WithSeqNrAccessible import akka.persistence.typed.scaladsl.Effect import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive +import akka.stream.scaladsl.Keep import akka.stream.{ SharedKillSwitch, SystemMaterializer } import akka.stream.scaladsl.{ RestartSource, Sink } import akka.stream.typed.scaladsl.ActorFlow -import akka.util.{ unused, OptionVal, Timeout } +import akka.util.Helpers +import akka.util.OptionVal +import akka.util.unused +import akka.util.Timeout /** * INTERNAL API @@ -79,6 +87,7 @@ private[akka] object Running { state: State, receivedPoisonPill: Boolean, seenPerReplica: Map[String, Long], + replicationControl: Map[String, ReplicationStreamControl], replicationKillSwitch: Option[SharedKillSwitch] = None) { def nextSequenceNr(): RunningState[State] = @@ -95,31 +104,38 @@ 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)) - setup.activeActive.foreach(aa => startReplicationStream(setup.context.system, setup.context.self, state, aa)) - new running.HandlingCommands(state) + val initialState = setup.activeActive match { + case Some(aa) => startReplicationStream(setup, state, aa) + case None => state + } + new running.HandlingCommands(initialState) } - def startReplicationStream[E, S]( - system: ActorSystem[_], - ref: ActorRef[InternalProtocol], + def startReplicationStream[C, E, S]( + setup: BehaviorSetup[C, E, S], state: RunningState[S], - aa: ActiveActive): Unit = { + aa: ActiveActive): RunningState[S] = { import scala.concurrent.duration._ + val system = setup.context.system + val ref = setup.context.self val query = PersistenceQuery(system) - aa.allReplicas.foreach { replica => - if (replica != aa.replicaId) { - val seqNr = state.seenPerReplica(replica) - val pid = PersistenceId.replicatedUniqueId(aa.aaContext.entityId, replica) + aa.allReplicas.foldLeft(state) { (state, replicaId) => + if (replicaId != aa.replicaId) { + val seqNr = state.seenPerReplica(replicaId) + val pid = PersistenceId.replicatedUniqueId(aa.aaContext.entityId, replicaId) // FIXME support different configuration per replica https://github.com/akka/akka/issues/29257 val replication = query.readJournalFor[EventsByPersistenceIdQuery](aa.queryPluginId) implicit val timeout = Timeout(30.seconds) - // FIXME config + val controlRef = new AtomicReference[ReplicationStreamControl]() + val source = RestartSource.withBackoff(2.seconds, 10.seconds, randomFactor = 0.2) { () => replication .eventsByPersistenceId(pid.id, seqNr + 1, Long.MaxValue) + .viaMat(new FastForwardingFilter)(Keep.right) + .mapMaterializedValue(streamControl => controlRef.set(streamControl)) .via(ActorFlow.ask[EventEnvelope, ReplicatedEventEnvelope[E], ReplicatedEventAck.type](ref) { (eventEnvelope, replyTo) => // Need to handle this not being available migration from non-active-active is supported @@ -131,6 +147,30 @@ private[akka] object Running { } source.runWith(Sink.ignore)(SystemMaterializer(system).materializer) + + // FIXME 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 } } } @@ -163,6 +203,7 @@ private[akka] object Running { def onMessage(msg: InternalProtocol): Behavior[InternalProtocol] = msg match { case IncomingCommand(c: C @unchecked) => onCommand(state, c) case re: ReplicatedEventEnvelope[E @unchecked] => onReplicatedEvent(state, re, setup.activeActive.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) @@ -205,6 +246,91 @@ private[akka] object Running { } } + def onPublishedEvent(state: Running.RunningState[S], event: PublishedEventImpl): Behavior[InternalProtocol] = { + val newBehavior: Behavior[InternalProtocol] = setup.activeActive match { + case None => + setup.log + .warn("Received published event for [{}] but not an active active actor, dropping", event.persistenceId) + this + + case Some(activeActive) => + event.replicaId match { + case None => + setup.log.warn("Received published event for [{}] but with no replica id, dropping") + this + case Some(replicaId) => + onPublishedEvent(state, activeActive, replicaId, event) + } + } + tryUnstashOne(newBehavior) + } + + private def onPublishedEvent( + state: Running.RunningState[S], + activeActive: ActiveActive, + originReplicaId: String, + event: PublishedEventImpl): Behavior[InternalProtocol] = { + val log = setup.log + val separatorIndex = event.persistenceId.id.indexOf(PersistenceId.DefaultSeparator) + val idPrefix = event.persistenceId.id.substring(0, separatorIndex) + if (!setup.persistenceId.id.startsWith(idPrefix)) { + log.warn("Ignoring published replicated event for the wrong actor [{}]", event.persistenceId) + this + } else if (originReplicaId == activeActive.replicaId) { + if (log.isDebugEnabled) + log.debug( + "Ignoring published replicated event with seqNr [{}] from our own replica id [{}]", + event.sequenceNumber, + originReplicaId) + this + } else if (!activeActive.allReplicas.contains(originReplicaId)) { + log.warnN( + "Received published replicated event from replica [{}], which is unknown. Active active must be set up with a list of all replicas (known are [{}]).", + originReplicaId, + activeActive.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, + event.replicaId, + expectedSequenceNumber) + } + this + } else { + if (log.isTraceEnabled) + log.traceN( + "Received published replicated event [{}] with timestamp [{}] from replica [{}] seqNr [{}]", + Logging.simpleName(event.event.getClass), + Helpers.timestamp(event.timestamp), + originReplicaId, + event.sequenceNumber) + + // fast forward stream for source replica + state.replicationControl.get(originReplicaId).foreach(_.fastForward(event.sequenceNumber)) + + handleExternalReplicatedEventPersist( + ReplicatedEvent(event.event.asInstanceOf[E], originReplicaId, event.sequenceNumber)) + } + + } + } + // Used by EventSourcedBehaviorTestKit to retrieve the state. def onGetState(get: GetState[S]): Behavior[InternalProtocol] = { get.replyTo ! state.state @@ -368,6 +494,7 @@ private[akka] object Running { 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 SnapshotterResponse(r) => onDeleteSnapshotResponse(r, visibleState.state) case RecoveryTickEvent(_) => Behaviors.unhandled @@ -393,6 +520,14 @@ private[akka] object Running { } } + 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( @@ -407,6 +542,11 @@ private[akka] object Running { onWriteSuccess(setup.context, p) + if (setup.publishEvents) { + context.system.eventStream ! EventStream.Publish( + PublishedEventImpl(setup.replicaId, setup.persistenceId, p.sequenceNr, p.payload, p.timestamp)) + } + // only once all things are applied we can revert back if (eventCounter < numberOfEvents) { onWriteDone(setup.context, p) 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/scaladsl/EventSourcedBehavior.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/EventSourcedBehavior.scala index e53a1918a0..921f9a014f 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 @@ -12,6 +12,7 @@ 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, InternalApi } import akka.persistence.typed.EventAdapter import akka.persistence.typed.PersistenceId @@ -21,8 +22,9 @@ import akka.persistence.typed.internal._ object EventSourcedBehavior { + // FIXME move to internal @InternalApi - private[akka] case class ActiveActive( + private[akka] final case class ActiveActive( replicaId: String, allReplicas: Set[String], aaContext: ActiveActiveContextImpl, @@ -237,4 +239,10 @@ 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(): EventSourcedBehavior[Command, Event, State] } diff --git a/akka-persistence-typed/src/test/resources/logback-test.xml b/akka-persistence-typed/src/test/resources/logback-test.xml index c980894390..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] [%X{akkaSource}] [%X{persistenceId}] - %msg %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/scaladsl/EventSourcedBehaviorWatchSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/EventSourcedBehaviorWatchSpec.scala index c033e614ed..b3175e5777 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 @@ -67,7 +67,8 @@ class EventSourcedBehaviorWatchSpec holdingRecoveryPermit = false, settings = settings, stashState = new StashState(context.asInstanceOf[ActorContext[InternalProtocol]], settings), - None) + activeActive = None, + publishEvents = false) "A typed persistent parent actor watching a child" must { diff --git a/build.sbt b/build.sbt index 72bc671395..85977aabed 100644 --- a/build.sbt +++ b/build.sbt @@ -499,7 +499,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", From e79f5ac3c47aa630dd10dc7bd2b5642e141cc67c Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 29 Jun 2020 15:55:44 +0200 Subject: [PATCH 05/50] better timestamp format for active-active (#29329) --- .../persistence/typed/internal/Running.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) 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 f24914e4f8..0efb312c69 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,10 +4,15 @@ 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.eventstream.EventStream import akka.actor.typed.{ Behavior, Signal } @@ -52,7 +57,6 @@ import akka.stream.scaladsl.Keep import akka.stream.{ SharedKillSwitch, SystemMaterializer } import akka.stream.scaladsl.{ RestartSource, Sink } import akka.stream.typed.scaladsl.ActorFlow -import akka.util.Helpers import akka.util.OptionVal import akka.util.unused import akka.util.Timeout @@ -174,6 +178,13 @@ private[akka] object Running { } } } + + 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)) + } } // =============================================== @@ -186,6 +197,7 @@ private[akka] object Running { import BehaviorSetup._ import InternalProtocol._ import Running.RunningState + import Running.formatTimestamp // Needed for WithSeqNrAccessible, when unstashing private var _currentSequenceNumber = 0L @@ -315,9 +327,9 @@ private[akka] object Running { } else { if (log.isTraceEnabled) log.traceN( - "Received published replicated event [{}] with timestamp [{}] from replica [{}] seqNr [{}]", + "Received published replicated event [{}] with timestamp [{} (UTC)] from replica [{}] seqNr [{}]", Logging.simpleName(event.event.getClass), - Helpers.timestamp(event.timestamp), + formatTimestamp(event.timestamp), originReplicaId, event.sequenceNumber) From 238d55a413e7da34d00cf66f230e609a617d61ea Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 30 Jun 2020 09:28:48 +0200 Subject: [PATCH 06/50] message adapter in ActiveActiveShardingDirectReplication (#29328) * I don't think it's worth cheating with Any and narrow just to save one or two allocations given how many other things that are needed for sending the messages --- ...ctiveActiveShardingDirectReplication.scala | 49 ++++++++++--------- ...eActiveShardingDirectReplicationSpec.scala | 14 +++--- .../typed/EventPublishingSpec.scala | 8 +-- 3 files changed, 37 insertions(+), 34 deletions(-) diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala index fdb843962b..1c1315b459 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala @@ -46,6 +46,8 @@ object ActiveActiveShardingDirectReplication { @InternalApi private[akka] case class VerifyStarted(replyTo: ActorRef[Done]) extends Command + private final case class WrappedPublishedEvent(publishedEvent: PublishedEvent) extends Command + /** * Java API: * @param selfReplica The replica id of the replica that runs on this node @@ -60,31 +62,30 @@ object ActiveActiveShardingDirectReplication { * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system */ def apply[T](selfReplica: String, replicaShardingProxies: Map[String, ActorRef[T]]): Behavior[Command] = - Behaviors - .setup[Any] { context => - context.log.debug( - "Subscribing to event stream to forward events to [{}] sharded replicas", - replicaShardingProxies.size - 1) - context.system.eventStream ! EventStream.Subscribe[PublishedEvent](context.self) + 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.receiveMessagePartial { - case event: PublishedEvent => - context.log.trace( - "Forwarding event for persistence id [{}] sequence nr [{}] to replicas", - event.persistenceId, - event.sequenceNumber) - replicaShardingProxies.foreach { - case (replica, proxy) => - val envelopedEvent = ShardingEnvelope(event.persistenceId.id, event) - if (replica != selfReplica) - proxy.asInstanceOf[ActorRef[ShardingEnvelope[PublishedEvent]]] ! envelopedEvent - } - Behaviors.same - case VerifyStarted(replyTo) => - replyTo ! Done - Behaviors.same - } + 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 envelopedEvent = ShardingEnvelope(event.persistenceId.id, event) + if (replica != selfReplica) + proxy.asInstanceOf[ActorRef[ShardingEnvelope[PublishedEvent]]] ! envelopedEvent + } + Behaviors.same + case VerifyStarted(replyTo) => + replyTo ! Done + Behaviors.same } - .narrow[Command] + } } diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala index b9a939f387..79952ccf1a 100644 --- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala +++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala @@ -4,13 +4,15 @@ 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.PersistenceId +import akka.persistence.typed.PublishedEvent import akka.persistence.typed.internal.PublishedEventImpl -import org.scalatest.wordspec.AnyWordSpecLike class ActiveActiveShardingDirectReplicationSpec extends ScalaTestWithActorTestKit @@ -20,9 +22,9 @@ class ActiveActiveShardingDirectReplicationSpec "Active active sharding replication" must { "replicate published events to all sharding proxies" in { - val replicaAProbe = createTestProbe[Any]() - val replicaBProbe = createTestProbe[Any]() - val replicaCProbe = createTestProbe[Any]() + val replicaAProbe = createTestProbe[ShardingEnvelope[PublishedEvent]]() + val replicaBProbe = createTestProbe[ShardingEnvelope[PublishedEvent]]() + val replicaCProbe = createTestProbe[ShardingEnvelope[PublishedEvent]]() val replicationActor = spawn( ActiveActiveShardingDirectReplication( @@ -42,8 +44,8 @@ class ActiveActiveShardingDirectReplicationSpec System.currentTimeMillis()) system.eventStream ! EventStream.Publish(event) - replicaBProbe.expectMessageType[ShardingEnvelope[_]].message should equal(event) - replicaCProbe.expectMessageType[ShardingEnvelope[_]].message should equal(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-persistence-typed-tests/src/test/scala/akka/persistence/typed/EventPublishingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/EventPublishingSpec.scala index a899beebc4..3f33a41381 100644 --- 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 @@ -49,8 +49,8 @@ class EventPublishingSpec "EventPublishing support" must { "publish events after written for any actor" in { - val topicProbe = createTestProbe[Any]() - system.eventStream ! EventStream.Subscribe[PublishedEvent](topicProbe.ref.narrow) + 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 @@ -61,7 +61,7 @@ class EventPublishingSpec wowSuchActor ! WowSuchEventSourcingBehavior.StoreThis("great stuff", tagIt = false, replyTo = persistProbe.ref) persistProbe.expectMessage(Done) - val published1 = topicProbe.expectMessageType[PublishedEvent] + val published1 = topicProbe.receiveMessage() published1.persistenceId should ===(myId) published1.event should ===(WowSuchEventSourcingBehavior.Event("great stuff", false)) published1.sequenceNumber should ===(1L) @@ -72,7 +72,7 @@ class EventPublishingSpec anotherActor ! WowSuchEventSourcingBehavior.StoreThis("another event", tagIt = true, replyTo = persistProbe.ref) persistProbe.expectMessage(Done) - val published2 = topicProbe.expectMessageType[PublishedEvent] + val published2 = topicProbe.receiveMessage() published2.persistenceId should ===(anotherId) published2.event should ===(WowSuchEventSourcingBehavior.Event("another event", true)) published2.sequenceNumber should ===(1L) From b86b6df7c7c696202bd6388746fb43ac85283a0a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 1 Jul 2020 17:55:10 +0200 Subject: [PATCH 07/50] Support persistAll for active active (#29337) --- .../persistence/typed/ActiveActiveSpec.scala | 24 +++++- .../typed/internal/ExternalInteractions.scala | 28 ++++-- .../persistence/typed/internal/Running.scala | 86 +++++++++++++------ 3 files changed, 104 insertions(+), 34 deletions(-) diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala index bb2ee63290..15606caa5d 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala @@ -22,6 +22,7 @@ object ActiveActiveSpec { 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[(String, Set[String])]) extends Command case class State(all: List[String]) @@ -47,6 +48,8 @@ object ActiveActiveSpec { Effect.none case StoreMe(evt, ack) => Effect.persist(evt).thenRun(_ => ack ! Done) + case StoreUs(evts, replyTo) => + Effect.persist(evts).thenRun(_ => replyTo ! Done) }, (state, event) => { probe.foreach(_ ! EventAndContext(event, aaContext.origin, aaContext.recoveryRunning)) @@ -142,7 +145,26 @@ class ActiveActiveSpec } "persist all" in { - pending + val entityId = nextEntityId + val probe = createTestProbe[Done]() + val r1 = spawn(testBehavior(entityId, "R1")) + 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() + + 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") + } + } } } 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 ec6291e5bf..3fd33ab510 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 @@ -18,15 +18,29 @@ import akka.annotation.InternalStableApi import akka.persistence._ import akka.persistence.JournalProtocol.ReplayMessages import akka.persistence.SnapshotProtocol.LoadSnapshot +import akka.persistence.typed.internal.JournalInteractions.EventOrTaggedOrReplicated 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 EventOrTaggedOrReplicated = Any // `Any` since can be `E` or `Tagged` or a `ReplicatedEvent` + def setup: BehaviorSetup[C, E, S] protected def internalPersist( ctx: ActorContext[_], @@ -71,20 +85,24 @@ private[akka] trait JournalInteractions[C, E, S] { ctx: ActorContext[_], cmd: Any, state: Running.RunningState[S], - events: immutable.Seq[(EventOrTaggedOrReplicated, 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) 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 0efb312c69..e4aa609dcf 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 @@ -12,7 +12,6 @@ import java.util.concurrent.atomic.AtomicReference import scala.annotation.tailrec import scala.collection.immutable - import akka.actor.UnhandledMessage import akka.actor.typed.eventstream.EventStream import akka.actor.typed.{ Behavior, Signal } @@ -50,6 +49,7 @@ import akka.persistence.typed.{ } import akka.persistence.typed.internal.EventSourcedBehaviorImpl.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.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive @@ -250,10 +250,16 @@ private[akka] object Running { envelope.ack ! ReplicatedEventAck if (envelope.event.originReplica != activeActive.replicaId && !alreadySeen(envelope.event)) { activeActive.setContext(false, envelope.event.originReplica) - setup.log.debug("Saving event as first time") + setup.log.debug( + "Saving event [{}] from [{}] as first time", + envelope.event.originSequenceNr, + envelope.event.originReplica) handleExternalReplicatedEventPersist(envelope.event) } else { - setup.log.debug("Filtering event as already seen") + setup.log.debug( + "Filtering event [{}] from [{}] as it was already seen", + envelope.event.originSequenceNr, + envelope.event.originReplica) this } } @@ -370,7 +376,10 @@ private[akka] object Running { Nil) } - private def handleEventPersist(event: E, cmd: Any, sideEffects: immutable.Seq[SideEffect[S]]) = { + private def handleEventPersist( + event: E, + cmd: Any, + sideEffects: immutable.Seq[SideEffect[S]]): (Behavior[InternalProtocol], Boolean) = { // 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 @@ -401,6 +410,49 @@ private[akka] object Running { (persistingEvents(newState2, state, numberOfEvents = 1, shouldSnapshotAfterPersist, sideEffects), false) } + private def handleEventPersistAll( + events: immutable.Seq[E], + cmd: Any, + sideEffects: immutable.Seq[SideEffect[S]]): (Behavior[InternalProtocol], Boolean) = { + 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 metadataTemplate: Option[ReplicatedEventMetaData] = setup.activeActive match { + case Some(aa) => + aa.setContext(recoveryRunning = false, aa.replicaId) + Some(ReplicatedEventMetaData(aa.replicaId, 0L)) // 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) => Some(template.copy(originSequenceNr = _currentSequenceNumber)) + 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, sideEffects), false) + } else { + // run side-effects even when no events are emitted + (applySideEffects(sideEffects, state), true) + } + } + @tailrec def applyEffects( msg: Any, state: RunningState[S], @@ -420,31 +472,9 @@ private[akka] object Running { case Persist(event) => 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) From 82b8d699ca7d1ccbc70e76c87d6665317f038fb8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Mon, 6 Jul 2020 15:40:49 +0200 Subject: [PATCH 08/50] Test coverage for active active published events after replay (#29335) * Test coverage for published events after recovery * Keep track of origin seqnrs when replaying --- .../ActiveActiveEventPublishingSpec.scala | 98 +++++++++++++++++-- .../persistence/typed/ActiveActiveSpec.scala | 39 ++++++++ .../typed/internal/ReplayingEvents.scala | 45 +++++---- .../typed/internal/ReplayingSnapshot.scala | 5 +- 4 files changed, 159 insertions(+), 28 deletions(-) diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala index c1238b2531..1acb5e67e7 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala @@ -23,6 +23,7 @@ object ActiveActiveEventPublishingSpec { 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: String, allReplicas: Set[String]): Behavior[Command] = Behaviors.setup { ctx => @@ -42,6 +43,8 @@ object ActiveActiveEventPublishingSpec { case Get(replyTo) => replyTo ! state Effect.none + case Stop => + Effect.stop() }, (state, string) => state + string)) } @@ -53,12 +56,18 @@ class ActiveActiveEventPublishingSpec with AnyWordSpecLike with LogCapturing { + private var idCounter = 0 + def nextEntityId(): String = { + idCounter += 1 + s"myId$idCounter" + } + import ActiveActiveEventPublishingSpec._ "An active active actor" must { "move forward when a published event from a replica is received" in { - - val actor = spawn(MyActiveActive("myId1", "DC-A", Set("DC-A", "DC-B"))) + val id = nextEntityId() + val actor = spawn(MyActiveActive(id, "DC-A", Set("DC-A", "DC-B"))) val probe = createTestProbe[Any]() actor ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) @@ -66,7 +75,7 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( Some("DC-B"), - PersistenceId.replicatedUniqueId("myId1", "DC-B"), + PersistenceId.replicatedUniqueId(id, "DC-B"), 1L, "two", System.currentTimeMillis()) @@ -78,7 +87,8 @@ class ActiveActiveEventPublishingSpec } "ignore a published event from a replica is received but the sequence number is unexpected" in { - val actor = spawn(MyActiveActive("myId2", "DC-A", Set("DC-A", "DC-B"))) + val id = nextEntityId() + val actor = spawn(MyActiveActive(id, "DC-A", Set("DC-A", "DC-B"))) val probe = createTestProbe[Any]() actor ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) @@ -86,7 +96,7 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( Some("DC-B"), - PersistenceId.replicatedUniqueId("myId2", "DC-B"), + PersistenceId.replicatedUniqueId(id, "DC-B"), 2L, // missing 1L "two", System.currentTimeMillis()) @@ -98,7 +108,8 @@ class ActiveActiveEventPublishingSpec } "ignore a published event from an unknown replica" in { - val actor = spawn(MyActiveActive("myId3", "DC-A", Set("DC-A", "DC-B"))) + val id = nextEntityId() + val actor = spawn(MyActiveActive(id, "DC-A", Set("DC-A", "DC-B"))) val probe = createTestProbe[Any]() actor ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) @@ -106,7 +117,7 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( Some("DC-C"), - PersistenceId.replicatedUniqueId("myId3", "DC-C"), + PersistenceId.replicatedUniqueId(id, "DC-C"), 1L, "two", System.currentTimeMillis()) @@ -118,7 +129,8 @@ class ActiveActiveEventPublishingSpec } "ignore an already seen event from a replica" in { - val actor = spawn(MyActiveActive("myId4", "DC-A", Set("DC-A", "DC-B"))) + val id = nextEntityId() + val actor = spawn(MyActiveActive(id, "DC-A", Set("DC-A", "DC-B"))) val probe = createTestProbe[Any]() actor ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) @@ -133,7 +145,7 @@ class ActiveActiveEventPublishingSpec // simulate another published event from that replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( Some("DC-B"), - PersistenceId.replicatedUniqueId("myId4", "DC-B"), + PersistenceId.replicatedUniqueId(id, "DC-B"), 1L, "two-again", // ofc this would be the same in the real world, different just so we can detect System.currentTimeMillis()) @@ -145,6 +157,74 @@ class ActiveActiveEventPublishingSpec probe.expectMessage(Set("one", "two", "three")) } + "handle published events after replay" in { + val id = nextEntityId() + val probe = createTestProbe[Any]() + val activeActiveBehavior = MyActiveActive(id, "DC-A", Set("DC-A", "DC-B")) + val incarnation1 = spawn(activeActiveBehavior) + incarnation1 ! MyActiveActive.Add("one", probe.ref) + probe.expectMessage(Done) + + incarnation1 ! MyActiveActive.Stop + probe.expectTerminated(incarnation1) + + val incarnation2 = spawn(activeActiveBehavior) + + incarnation2 ! MyActiveActive.Get(probe.ref) + probe.expectMessage(Set("one")) + // replay completed + + // simulate a published event from another replica + incarnation2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( + Some("DC-B"), + PersistenceId.replicatedUniqueId(id, "DC-B"), + 1L, + "two", + System.currentTimeMillis()) + + incarnation2 ! MyActiveActive.Add("three", probe.ref) + probe.expectMessage(Done) + + incarnation2 ! MyActiveActive.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 activeActiveBehaviorA = MyActiveActive(id, "DC-A", Set("DC-A", "DC-B")) + val incarnationA1 = spawn(activeActiveBehaviorA) + incarnationA1 ! MyActiveActive.Add("one", probe.ref) + probe.expectMessage(Done) + + // simulate a published event from another replica + incarnationA1.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( + Some("DC-B"), + PersistenceId.replicatedUniqueId(id, "DC-B"), + 1L, + "two", + System.currentTimeMillis()) + + incarnationA1 ! MyActiveActive.Stop + probe.expectTerminated(incarnationA1) + + val incarnationA2 = spawn(activeActiveBehaviorA) + + // simulate a published event from another replica + incarnationA2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( + Some("DC-B"), + PersistenceId.replicatedUniqueId(id, "DC-B"), + 2L, + "three", + System.currentTimeMillis()) + + incarnationA2 ! MyActiveActive.Add("four", probe.ref) + probe.expectMessage(Done) + + incarnationA2 ! MyActiveActive.Get(probe.ref) + probe.expectMessage(Set("one", "two", "three", "four")) + } + } } diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala index 15606caa5d..5b7dba7ba9 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala @@ -24,6 +24,7 @@ object ActiveActiveSpec { 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[(String, Set[String])]) extends Command + case object Stop extends Command case class State(all: List[String]) def testBehavior(entityId: String, replicaId: String, probe: ActorRef[EventAndContext]): Behavior[Command] = @@ -50,6 +51,8 @@ object ActiveActiveSpec { 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, aaContext.origin, aaContext.recoveryRunning)) @@ -104,6 +107,42 @@ class ActiveActiveSpec } } + "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[(String, Set[String])]() 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 c229c842a1..a4502bb699 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 @@ -50,7 +50,8 @@ private[akka] object ReplayingEvents { eventSeenInInterval: Boolean, toSeqNr: Long, receivedPoisonPill: Boolean, - recoveryStartTime: Long) + recoveryStartTime: Long, + seenSeqNrPerReplica: Map[String, Long]) def apply[C, E, S](setup: BehaviorSetup[C, E, S], state: ReplayingState[S]): Behavior[InternalProtocol] = Behaviors.setup { _ => @@ -121,20 +122,32 @@ private[akka] final class ReplayingEvents[C, E, S]( eventForErrorReporting = OptionVal.Some(event) state = state.copy(seqNr = repr.sequenceNr) - setup.activeActive match { - case Some(aa) => - val meta = repr.metadata match { - case Some(m) => m.asInstanceOf[ReplicatedEventMetaData] - case None => - throw new IllegalStateException( - s"Active active enabled but existing event has no metadata. Migration isn't supported yet.") + val aaMetaAndSelfReplica: Option[(ReplicatedEventMetaData, String)] = + setup.activeActive match { + case Some(aa) => + val meta = repr.metadata match { + case Some(m) => m.asInstanceOf[ReplicatedEventMetaData] + case None => + throw new IllegalStateException( + s"Active active enabled but existing event has no metadata. Migration isn't supported yet.") - } - aa.setContext(recoveryRunning = true, meta.originReplica) - case None => - } + } + aa.setContext(recoveryRunning = true, meta.originReplica) + Some(meta -> aa.replicaId) + case None => None + } val newState = setup.eventHandler(state.state, event) - state = state.copy(state = newState, eventSeenInInterval = true) + + aaMetaAndSelfReplica match { + case Some((meta, selfReplica)) if meta.originReplica != selfReplica => + // keep track of highest origin seqnr per other replica + state = state.copy( + state = newState, + eventSeenInInterval = true, + seenSeqNrPerReplica = state.seenSeqNrPerReplica + (meta.originReplica -> meta.originSequenceNr)) + case _ => + state = state.copy(state = newState, eventSeenInInterval = true) + } } eventSeq match { @@ -253,10 +266,6 @@ private[akka] final class ReplayingEvents[C, E, S]( if (state.receivedPoisonPill && isInternalStashEmpty && !isUnstashAllInProgress) Behaviors.stopped else { - val seenPerReplica: Map[String, Long] = - setup.activeActive - .map(aa => aa.allReplicas.filterNot(_ == aa.replicaId).map(replica => replica -> 0L).toMap) - .getOrElse(Map.empty) val running = Running[C, E, S]( setup, @@ -264,7 +273,7 @@ private[akka] final class ReplayingEvents[C, E, S]( seqNr = state.seqNr, state = state.state, receivedPoisonPill = state.receivedPoisonPill, - seenPerReplica = seenPerReplica, + 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 cfabb7a43c..4977e36d78 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 @@ -178,7 +178,10 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup eventSeenInInterval = false, toSnr, receivedPoisonPill, - System.nanoTime())) + System.nanoTime(), + // FIXME seqNrs for other replicas needs to come from snapshot + seenSeqNrPerReplica = + setup.activeActive.map(_.allReplicas.map(replica => replica -> 0L).toMap).getOrElse(Map.empty))) } } From 398ab2efe0c4ba4b6cbbb3b2206d9b195d246f4b Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Wed, 8 Jul 2020 09:04:59 +0100 Subject: [PATCH 09/50] Introduce version vectors for replicated events (#29332) * ActiveActive: Events with metadata and events by persistence id for (#29287) * Introduce version vectors for replicated events * Set concurrent on recovery * Group together AA related fields in a published event --- ...eActiveShardingDirectReplicationSpec.scala | 7 +- .../ActiveActiveEventPublishingSpec.scala | 33 +- .../persistence/typed/ActiveActiveSpec.scala | 138 +++++++- .../persistence/typed/PublishedEvent.scala | 6 +- .../internal/EventSourcedBehaviorImpl.scala | 34 +- .../typed/internal/ReplayingEvents.scala | 7 +- .../typed/internal/ReplayingSnapshot.scala | 3 +- .../persistence/typed/internal/Running.scala | 95 ++++-- .../typed/internal/VersionVector.scala | 323 ++++++++++++++++++ .../scaladsl/ActiveActiveEventSourcing.scala | 3 +- .../typed/scaladsl/EventSourcedBehavior.scala | 3 +- 11 files changed, 582 insertions(+), 70 deletions(-) create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/VersionVector.scala diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala index 79952ccf1a..9b2653399e 100644 --- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala +++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala @@ -5,14 +5,13 @@ 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.PersistenceId import akka.persistence.typed.PublishedEvent -import akka.persistence.typed.internal.PublishedEventImpl +import akka.persistence.typed.internal.{ PublishedEventImpl, ReplicatedPublishedEventMetaData, VersionVector } class ActiveActiveShardingDirectReplicationSpec extends ScalaTestWithActorTestKit @@ -37,11 +36,11 @@ class ActiveActiveShardingDirectReplicationSpec upProbe.receiveMessage() // not bullet proof wrt to subscription being complete but good enough val event = PublishedEventImpl( - Some("ReplicaA"), PersistenceId.replicatedUniqueId("pid", "ReplicaA"), 1L, "event", - System.currentTimeMillis()) + System.currentTimeMillis(), + Some(new ReplicatedPublishedEventMetaData("ReplicaA", VersionVector.empty))) system.eventStream ! EventStream.Publish(event) replicaBProbe.receiveMessage().message should equal(event) diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala index 1acb5e67e7..f844bb8538 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala @@ -12,6 +12,7 @@ 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.ActiveActiveEventSourcing import akka.persistence.typed.scaladsl.Effect import akka.persistence.typed.scaladsl.EventSourcedBehavior @@ -74,11 +75,11 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - Some("DC-B"), PersistenceId.replicatedUniqueId(id, "DC-B"), 1L, "two", - System.currentTimeMillis()) + System.currentTimeMillis(), + Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) actor ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -95,11 +96,11 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - Some("DC-B"), PersistenceId.replicatedUniqueId(id, "DC-B"), 2L, // missing 1L "two", - System.currentTimeMillis()) + System.currentTimeMillis(), + Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) actor ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -116,11 +117,11 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - Some("DC-C"), PersistenceId.replicatedUniqueId(id, "DC-C"), 1L, "two", - System.currentTimeMillis()) + System.currentTimeMillis(), + Some(new ReplicatedPublishedEventMetaData("DC-C", VersionVector.empty))) actor ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -137,18 +138,18 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - Some("DC-B"), PersistenceId.replicatedUniqueId("myId4", "DC-B"), 1L, "two", - System.currentTimeMillis()) + System.currentTimeMillis(), + Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) // simulate another published event from that replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - Some("DC-B"), PersistenceId.replicatedUniqueId(id, "DC-B"), 1L, "two-again", // ofc this would be the same in the real world, different just so we can detect - System.currentTimeMillis()) + System.currentTimeMillis(), + Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) actor ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -176,11 +177,11 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica incarnation2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - Some("DC-B"), PersistenceId.replicatedUniqueId(id, "DC-B"), 1L, "two", - System.currentTimeMillis()) + System.currentTimeMillis(), + Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) incarnation2 ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -199,11 +200,11 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica incarnationA1.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - Some("DC-B"), PersistenceId.replicatedUniqueId(id, "DC-B"), 1L, "two", - System.currentTimeMillis()) + System.currentTimeMillis(), + Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) incarnationA1 ! MyActiveActive.Stop probe.expectTerminated(incarnationA1) @@ -212,11 +213,11 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica incarnationA2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - Some("DC-B"), PersistenceId.replicatedUniqueId(id, "DC-B"), 2L, "three", - System.currentTimeMillis()) + System.currentTimeMillis(), + Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) incarnationA2 ! MyActiveActive.Add("four", probe.ref) probe.expectMessage(Done) diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala index 5b7dba7ba9..58effb19cf 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala @@ -55,13 +55,13 @@ object ActiveActiveSpec { Effect.stop() }, (state, event) => { - probe.foreach(_ ! EventAndContext(event, aaContext.origin, aaContext.recoveryRunning)) + probe.foreach(_ ! EventAndContext(event, aaContext.origin, aaContext.recoveryRunning, aaContext.concurrent)) state.copy(all = event :: state.all) })) } -case class EventAndContext(event: Any, origin: String, recoveryRunning: Boolean = false) +case class EventAndContext(event: Any, origin: String, recoveryRunning: Boolean, concurrent: Boolean) class ActiveActiveSpec extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) @@ -161,12 +161,12 @@ class ActiveActiveSpec val r2 = spawn(testBehavior(entityId, "R2", eventProbeR2.ref)) r1 ! StoreMe("from r1", replyProbe.ref) - eventProbeR2.expectMessage(EventAndContext("from r1", "R1")) - eventProbeR1.expectMessage(EventAndContext("from r1", "R1")) + eventProbeR2.expectMessage(EventAndContext("from r1", "R1", false, false)) + eventProbeR1.expectMessage(EventAndContext("from r1", "R1", false, false)) r2 ! StoreMe("from r2", replyProbe.ref) - eventProbeR1.expectMessage(EventAndContext("from r2", "R2")) - eventProbeR2.expectMessage(EventAndContext("from r2", "R2")) + eventProbeR1.expectMessage(EventAndContext("from r2", "R2", false, false)) + eventProbeR2.expectMessage(EventAndContext("from r2", "R2", false, false)) } "set recovery running" in { @@ -175,24 +175,33 @@ class ActiveActiveSpec val replyProbe = createTestProbe[Done]() val r1 = spawn(testBehavior(entityId, "R1", eventProbeR1.ref)) r1 ! StoreMe("Event", replyProbe.ref) - eventProbeR1.expectMessage(EventAndContext("Event", "R1", recoveryRunning = false)) + eventProbeR1.expectMessage(EventAndContext("Event", "R1", recoveryRunning = false, false)) replyProbe.expectMessage(Done) val recoveryProbe = createTestProbe[EventAndContext]() spawn(testBehavior(entityId, "R1", recoveryProbe.ref)) - recoveryProbe.expectMessage(EventAndContext("Event", "R1", recoveryRunning = true)) + recoveryProbe.expectMessage(EventAndContext("Event", "R1", recoveryRunning = true, false)) } "persist all" in { val entityId = nextEntityId val probe = createTestProbe[Done]() - val r1 = spawn(testBehavior(entityId, "R1")) + 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", "R1", false, concurrent = false)) + eventProbeR1.expectMessage(EventAndContext("2 from r1", "R1", false, concurrent = false)) + eventProbeR1.expectMessage(EventAndContext("1 from r2", "R2", false, concurrent = true)) + eventProbeR1.expectMessage(EventAndContext("2 from r2", "R2", false, concurrent = true)) + eventually { val probe = createTestProbe[State]() r1 ! GetState(probe.ref) @@ -203,7 +212,116 @@ class ActiveActiveSpec 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", "R1", recoveryRunning = false, concurrent = false)) + eventProbeR2.expectMessage(EventAndContext("from r2", "R2", recoveryRunning = false, concurrent = false)) + + // then the replicated remote events, which will be concurrent + eventProbeR1.expectMessage(EventAndContext("from r2", "R2", recoveryRunning = false, concurrent = true)) + eventProbeR2.expectMessage(EventAndContext("from r1", "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", "R1", false, concurrent = false)) + eventProbeR2.expectMessage(EventAndContext("from r1 2", "R1", false, concurrent = false)) + r1 ! StoreMe("from r1 3", probe.ref) // R2 2 R2 1 + eventProbeR1.expectMessage(EventAndContext("from r1 3", "R1", false, concurrent = false)) + eventProbeR2.expectMessage(EventAndContext("from r1 3", "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", "R2", false, concurrent = false)) + eventProbeR2.expectMessage(EventAndContext("from r2 2", "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 + eventProbeR2.expectMessage(EventAndContext("from r1 1", "R1", false, false)) + eventProbeR2.expectMessage(EventAndContext("from r1 2", "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 + val eventProbeR3 = createTestProbe[EventAndContext]() + spawn(testBehavior(entityId, "R3", eventProbeR3.ref)) + eventProbeR3.expectMessage(EventAndContext("from r1 1", "R1", false, false)) + eventProbeR3.expectMessage(EventAndContext("from r1 2", "R1", false, false)) + eventProbeR3.expectMessage(EventAndContext("from r2 1", "R2", false, false)) + eventProbeR3.expectMessage(EventAndContext("from r2 2", "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", "R1", recoveryRunning = false, concurrent = false)) + eventProbeR1.expectMessage(EventAndContext("from r2", "R2", recoveryRunning = false, concurrent = true)) + + // take 2 + val eventProbeR1Take2 = createTestProbe[EventAndContext]() + spawn(testBehavior(entityId, "R1", eventProbeR1Take2.ref)) + eventProbeR1Take2.expectMessage(EventAndContext("from r1", "R1", recoveryRunning = true, concurrent = false)) + eventProbeR1Take2.expectMessage(EventAndContext("from r2", "R2", recoveryRunning = true, concurrent = true)) + } + } } 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 index 67a9560b83..1311ea6574 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/PublishedEvent.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/PublishedEvent.scala @@ -7,6 +7,7 @@ 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. @@ -17,10 +18,11 @@ import akka.annotation.DoNotInherit trait PublishedEvent { /** Scala API: When emitted from an Active Active actor this will contain the replica id */ - def replicaId: Option[String] + def replicatedMetaData: Option[ReplicatedPublishedEventMetaData] /** Java API: When emitted from an Active Active actor this will contain the replica id */ - def getReplicaId: Optional[String] + def getReplicatedMetaData: Optional[ReplicatedPublishedEventMetaData] + def persistenceId: PersistenceId def sequenceNumber: Long 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 a14d0735f4..5c46366211 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 @@ -274,29 +274,50 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( } // FIXME serializer +/** + * @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: String, originSequenceNr: Long) +private[akka] final case class ReplicatedEventMetaData( + originReplica: String, + originSequenceNr: Long, + version: VersionVector, + concurrent: Boolean) // whether when the event handler was executed the event was concurrent + +/** + * 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: String, originSequenceNr: Long) +private[akka] final case class ReplicatedEvent[E]( + event: E, + originReplica: String, + originSequenceNr: Long, + originVersion: VersionVector) @InternalApi private[akka] case object ReplicatedEventAck +final class ReplicatedPublishedEventMetaData(val replicaId: String, private[akka] val version: VersionVector) + /** * INTERNAL API */ @InternalApi private[akka] final case class PublishedEventImpl( - replicaId: Option[String], persistenceId: PersistenceId, sequenceNumber: Long, payload: Any, - timestamp: Long) + timestamp: Long, + replicatedMetaData: Option[ReplicatedPublishedEventMetaData]) extends PublishedEvent with InternalProtocol { import scala.compat.java8.OptionConverters._ - override def getReplicaId: Optional[String] = replicaId.asJava - def tags: Set[String] = payload match { case t: Tagged => t.tags case _ => Set.empty @@ -307,4 +328,5 @@ private[akka] final case class PublishedEventImpl( case _ => payload } + override def getReplicatedMetaData: Optional[ReplicatedPublishedEventMetaData] = replicatedMetaData.asJava } 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 a4502bb699..d2612bce34 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 @@ -51,6 +51,7 @@ private[akka] object ReplayingEvents { toSeqNr: Long, receivedPoisonPill: Boolean, recoveryStartTime: Long, + version: VersionVector, seenSeqNrPerReplica: Map[String, Long]) def apply[C, E, S](setup: BehaviorSetup[C, E, S], state: ReplayingState[S]): Behavior[InternalProtocol] = @@ -87,8 +88,6 @@ private[akka] final class ReplayingEvents[C, E, S]( () override def onMessage(msg: InternalProtocol): Behavior[InternalProtocol] = { - // FIXME deal with a replicated event and ack - // https://github.com/akka/akka/issues/29256 msg match { case JournalResponse(r) => onJournalResponse(r) case SnapshotterResponse(r) => onSnapshotterResponse(r) @@ -132,7 +131,7 @@ private[akka] final class ReplayingEvents[C, E, S]( s"Active active enabled but existing event has no metadata. Migration isn't supported yet.") } - aa.setContext(recoveryRunning = true, meta.originReplica) + aa.setContext(recoveryRunning = true, meta.originReplica, meta.concurrent) Some(meta -> aa.replicaId) case None => None } @@ -144,6 +143,7 @@ private[akka] final class ReplayingEvents[C, E, S]( state = state.copy( state = newState, eventSeenInInterval = true, + version = meta.version, seenSeqNrPerReplica = state.seenSeqNrPerReplica + (meta.originReplica -> meta.originSequenceNr)) case _ => state = state.copy(state = newState, eventSeenInInterval = true) @@ -273,6 +273,7 @@ private[akka] final class ReplayingEvents[C, E, S]( seqNr = state.seqNr, state = state.state, receivedPoisonPill = state.receivedPoisonPill, + state.version, seenPerReplica = state.seenSeqNrPerReplica, replicationControl = Map.empty)) 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 4977e36d78..9f2d464ec3 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 @@ -179,7 +179,8 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup toSnr, receivedPoisonPill, System.nanoTime(), - // FIXME seqNrs for other replicas needs to come from snapshot + VersionVector.empty, + // FIXME seqNrs for other replicas needs to come from snapshot. seenSeqNrPerReplica = setup.activeActive.map(_.allReplicas.map(replica => replica -> 0L).toMap).getOrElse(Map.empty))) } 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 e4aa609dcf..7c6af4a82a 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 @@ -54,7 +54,7 @@ import akka.persistence.typed.internal.Running.WithSeqNrAccessible import akka.persistence.typed.scaladsl.Effect import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive import akka.stream.scaladsl.Keep -import akka.stream.{ SharedKillSwitch, SystemMaterializer } +import akka.stream.SystemMaterializer import akka.stream.scaladsl.{ RestartSource, Sink } import akka.stream.typed.scaladsl.ActorFlow import akka.util.OptionVal @@ -90,9 +90,9 @@ private[akka] object Running { seqNr: Long, state: State, receivedPoisonPill: Boolean, + version: VersionVector, seenPerReplica: Map[String, Long], - replicationControl: Map[String, ReplicationStreamControl], - replicationKillSwitch: Option[SharedKillSwitch] = None) { + replicationControl: Map[String, ReplicationStreamControl]) { def nextSequenceNr(): RunningState[State] = copy(seqNr = seqNr + 1) @@ -138,6 +138,9 @@ private[akka] object Running { val source = RestartSource.withBackoff(2.seconds, 10.seconds, randomFactor = 0.2) { () => replication .eventsByPersistenceId(pid.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(_.eventMetadata.get.asInstanceOf[ReplicatedEventMetaData].originReplica == replicaId) .viaMat(new FastForwardingFilter)(Keep.right) .mapMaterializedValue(streamControl => controlRef.set(streamControl)) .via(ActorFlow.ask[EventEnvelope, ReplicatedEventEnvelope[E], ReplicatedEventAck.type](ref) { @@ -145,7 +148,11 @@ private[akka] object Running { // Need to handle this not being available migration from non-active-active is supported val meta = eventEnvelope.eventMetadata.get.asInstanceOf[ReplicatedEventMetaData] val re = - ReplicatedEvent[E](eventEnvelope.event.asInstanceOf[E], meta.originReplica, meta.originSequenceNr) + ReplicatedEvent[E]( + eventEnvelope.event.asInstanceOf[E], + meta.originReplica, + meta.originSequenceNr, + meta.version) ReplicatedEventEnvelope(re, replyTo) }) } @@ -249,18 +256,17 @@ private[akka] object Running { envelope) envelope.ack ! ReplicatedEventAck if (envelope.event.originReplica != activeActive.replicaId && !alreadySeen(envelope.event)) { - activeActive.setContext(false, envelope.event.originReplica) setup.log.debug( "Saving event [{}] from [{}] as first time", envelope.event.originSequenceNr, envelope.event.originReplica) - handleExternalReplicatedEventPersist(envelope.event) + handleExternalReplicatedEventPersist(activeActive, envelope.event) } else { setup.log.debug( "Filtering event [{}] from [{}] as it was already seen", envelope.event.originSequenceNr, envelope.event.originReplica) - this + tryUnstashOne(this) } } @@ -272,12 +278,12 @@ private[akka] object Running { this case Some(activeActive) => - event.replicaId match { + event.replicatedMetaData match { case None => - setup.log.warn("Received published event for [{}] but with no replica id, dropping") + setup.log.warn("Received published event for [{}] but with no replicated metadata, dropping") this - case Some(replicaId) => - onPublishedEvent(state, activeActive, replicaId, event) + case Some(replicatedEventMetaData) => + onPublishedEvent(state, activeActive, replicatedEventMetaData, event) } } tryUnstashOne(newBehavior) @@ -286,11 +292,12 @@ private[akka] object Running { private def onPublishedEvent( state: Running.RunningState[S], activeActive: ActiveActive, - originReplicaId: String, + 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 @@ -326,7 +333,7 @@ private[akka] object Running { "Ignoring published replicated event with replication seqNr [{}] from replica [{}] " + "because expected replication seqNr was [{}] ", event.sequenceNumber, - event.replicaId, + originReplicaId, expectedSequenceNumber) } this @@ -343,7 +350,12 @@ private[akka] object Running { state.replicationControl.get(originReplicaId).foreach(_.fastForward(event.sequenceNumber)) handleExternalReplicatedEventPersist( - ReplicatedEvent(event.event.asInstanceOf[E], originReplicaId, event.sequenceNumber)) + activeActive, + ReplicatedEvent( + event.event.asInstanceOf[E], + originReplicaId, + event.sequenceNumber, + replicatedMetadata.version)) } } @@ -355,8 +367,22 @@ private[akka] object Running { this } - private def handleExternalReplicatedEventPersist(event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { + private def handleExternalReplicatedEventPersist( + activeActive: ActiveActive, + event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { _currentSequenceNumber = state.seqNr + 1 + val isConcurrent: Boolean = event.originVersion <> state.version + val updatedVersion = event.originVersion.merge(state.version) + activeActive.setContext(false, event.originReplica, isConcurrent) + + setup.log.debugN( + "Processing event [{}] with version [{}]. Local version: {}. Updated version {}. Concurrent? {}", + event.event, + event.originVersion, + state.version, + updatedVersion, + isConcurrent) + val newState: RunningState[S] = state.applyEvent(setup, event.event) val newState2: RunningState[S] = internalPersist( setup.context, @@ -364,12 +390,13 @@ private[akka] object Running { newState, event.event, "", - OptionVal.Some(ReplicatedEventMetaData(event.originReplica, event.originSequenceNr))) + 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), + newState2.copy(seenPerReplica = updatedSeen, version = updatedVersion), state, numberOfEvents = 1, shouldSnapshotAfterPersist, @@ -386,8 +413,10 @@ private[akka] object Running { _currentSequenceNumber = state.seqNr + 1 setup.activeActive.foreach { aa => - aa.setContext(recoveryRunning = false, aa.replicaId) + // set concurrent to false, local events are never concurrent + aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false) } + val newState: RunningState[S] = state.applyEvent(setup, event) val eventToPersist = adaptEvent(event) @@ -395,13 +424,20 @@ private[akka] object Running { val newState2 = setup.activeActive match { case Some(aa) => - internalPersist( + val updatedVersion = newState.version.updated(aa.replicaId, _currentSequenceNumber) + val r = internalPersist( setup.context, cmd, newState, eventToPersist, eventAdapterManifest, - OptionVal.Some(ReplicatedEventMetaData(aa.replicaId, _currentSequenceNumber))) + OptionVal.Some( + ReplicatedEventMetaData(aa.replicaId, _currentSequenceNumber, updatedVersion, concurrent = false))) + .copy(version = updatedVersion) + + setup.log.debug("Event persisted [{}]. Version vector after: [{}]", eventToPersist, r.version) + + r case None => internalPersist(setup.context, cmd, newState, eventToPersist, eventAdapterManifest, OptionVal.None) } @@ -422,14 +458,15 @@ private[akka] object Running { val metadataTemplate: Option[ReplicatedEventMetaData] = setup.activeActive match { case Some(aa) => - aa.setContext(recoveryRunning = false, aa.replicaId) - Some(ReplicatedEventMetaData(aa.replicaId, 0L)) // we replace it with actual seqnr later + aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false) // local events are never concurrent + Some(ReplicatedEventMetaData(aa.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) @@ -437,14 +474,19 @@ private[akka] object Running { val evtManifest = setup.eventAdapter.manifest(event) val adaptedEvent = adaptEvent(event) val eventMetadata = metadataTemplate match { - case Some(template) => Some(template.copy(originSequenceNr = _currentSequenceNumber)) - case None => None + case Some(template) => + val updatedVersion = currentState.version.updated(template.originReplica, _currentSequenceNumber) + setup.log.trace("Processing event [{}] with version vector [{}]", event, 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) + val newState2 = + internalPersistAll(setup.context, cmd, currentState, eventsToPersist.reverse) (persistingEvents(newState2, state, events.size, shouldSnapshotAfterPersist, sideEffects), false) } else { @@ -585,8 +627,9 @@ private[akka] object Running { onWriteSuccess(setup.context, p) if (setup.publishEvents) { + val meta = setup.activeActive.map(aa => new ReplicatedPublishedEventMetaData(aa.replicaId, state.version)) context.system.eventStream ! EventStream.Publish( - PublishedEventImpl(setup.replicaId, setup.persistenceId, p.sequenceNr, p.payload, p.timestamp)) + PublishedEventImpl(setup.persistenceId, p.sequenceNr, p.payload, p.timestamp, meta)) } // only once all things are applied we can revert back 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..42e5b7bbf5 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/VersionVector.scala @@ -0,0 +1,323 @@ +/* + * 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 +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. + */ +@SerialVersionUID(1L) +@InternalApi +sealed abstract class VersionVector extends Serializable { + + 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/scaladsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala index f0695b7b8d..888c7e3487 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala @@ -56,6 +56,7 @@ private[akka] class ActiveActiveContextImpl(val entityId: String, val replicaId: extends ActiveActiveContext { var _origin: String = null var _recoveryRunning: Boolean = false + var _concurrent: Boolean = false // FIXME check illegal access https://github.com/akka/akka/issues/29264 @@ -69,7 +70,7 @@ private[akka] class ActiveActiveContextImpl(val entityId: String, val replicaId: * 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 = throw new UnsupportedOperationException("TODO") + override def concurrent: Boolean = _concurrent override def persistenceId: PersistenceId = PersistenceId.replicatedUniqueId(entityId, replicaId) 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 921f9a014f..c7357ac198 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 @@ -33,8 +33,9 @@ object EventSourcedBehavior { /** * Must only be called on the same thread that will execute the user code */ - def setContext(recoveryRunning: Boolean, originReplica: String): Unit = { + def setContext(recoveryRunning: Boolean, originReplica: String, concurrent: Boolean): Unit = { aaContext._recoveryRunning = recoveryRunning + aaContext._concurrent = concurrent aaContext._origin = originReplica } From 36a8b6f24a437adb75f59e3c29c7d3bf559d300b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 8 Jul 2020 11:20:45 +0200 Subject: [PATCH 10/50] Active active per replica journal selection (#29350) * Support for having multiple isolated testkit journals and corresponding read journals in persistence testkit * Read journal selection per active-active replica * a dedicated ReplicaId type to avoid stringly typed mismatches --- ...ctiveActiveShardingDirectReplication.scala | 7 +- ...eActiveShardingDirectReplicationSpec.scala | 14 ++- .../testkit/PersistenceTestKitPlugin.scala | 12 +- .../testkit/ProcessingPolicy.scala | 4 +- .../persistence/testkit/SnapshotStorage.scala | 7 +- .../internal/InMemStorageExtension.scala | 45 +++++-- .../SnapshotStorageEmulatorExtension.scala | 3 +- .../testkit/internal/TestKitStorage.scala | 3 +- ...ersistenceTestKitReadJournalProvider.scala | 6 +- .../PersistenceTestKitReadJournal.scala | 14 ++- .../testkit/scaladsl/PersistenceTestKit.scala | 2 +- .../scaladsl/MultipleJournalsSpec.scala | 115 ++++++++++++++++++ .../ActiveActiveEventPublishingSpec.scala | 56 +++++---- .../persistence/typed/ActiveActiveSpec.scala | 94 ++++++++------ .../typed/MultiJournalActiveActiveSpec.scala | 111 +++++++++++++++++ .../AAAuctionExampleSpec.scala | 41 ++++--- .../AABlogExampleSpec.scala | 33 +++-- .../persistence/typed/PersistenceId.scala | 8 +- .../akka/persistence/typed/ReplicaId.scala | 10 ++ .../typed/internal/BehaviorSetup.scala | 3 +- .../internal/EventSourcedBehaviorImpl.scala | 14 +-- .../typed/internal/ReplayingEvents.scala | 5 +- .../persistence/typed/internal/Running.scala | 13 +- .../scaladsl/ActiveActiveEventSourcing.scala | 63 +++++++--- .../typed/scaladsl/EventSourcedBehavior.scala | 17 +-- 25 files changed, 532 insertions(+), 168 deletions(-) create mode 100644 akka-persistence-testkit/src/test/scala/akka/persistence/testkit/scaladsl/MultipleJournalsSpec.scala create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalActiveActiveSpec.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicaId.scala diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala index 1c1315b459..3700e26ac6 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala @@ -13,6 +13,7 @@ import akka.annotation.ApiMayChange import akka.annotation.DoNotInherit import akka.annotation.InternalApi import akka.persistence.typed.PublishedEvent +import akka.persistence.typed.ReplicaId import scala.collection.JavaConverters._ @@ -53,7 +54,9 @@ object ActiveActiveShardingDirectReplication { * @param selfReplica The replica id of the replica that runs on this node * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system */ - def create[T](selfReplica: String, replicaShardingProxies: java.util.Map[String, ActorRef[T]]): Behavior[Command] = + def create[T]( + selfReplica: ReplicaId, + replicaShardingProxies: java.util.Map[ReplicaId, ActorRef[T]]): Behavior[Command] = apply(selfReplica, replicaShardingProxies.asScala.toMap) /** @@ -61,7 +64,7 @@ object ActiveActiveShardingDirectReplication { * @param selfReplica The replica id of the replica that runs on this node * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system */ - def apply[T](selfReplica: String, replicaShardingProxies: Map[String, ActorRef[T]]): Behavior[Command] = + def apply[T](selfReplica: 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", diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala index 9b2653399e..bb85730a88 100644 --- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala +++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala @@ -9,9 +9,11 @@ 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.PersistenceId import akka.persistence.typed.PublishedEvent import akka.persistence.typed.internal.{ PublishedEventImpl, ReplicatedPublishedEventMetaData, VersionVector } +import akka.persistence.typed.ReplicaId class ActiveActiveShardingDirectReplicationSpec extends ScalaTestWithActorTestKit @@ -27,20 +29,22 @@ class ActiveActiveShardingDirectReplicationSpec val replicationActor = spawn( ActiveActiveShardingDirectReplication( - "ReplicaA", - replicaShardingProxies = - Map("ReplicaA" -> replicaAProbe.ref, "ReplicaB" -> replicaBProbe.ref, "ReplicaC" -> replicaCProbe.ref))) + typed.ReplicaId("ReplicaA"), + replicaShardingProxies = Map( + ReplicaId("ReplicaA") -> replicaAProbe.ref, + ReplicaId("ReplicaB") -> replicaBProbe.ref, + ReplicaId("ReplicaC") -> replicaCProbe.ref))) val upProbe = createTestProbe[Done]() replicationActor ! ActiveActiveShardingDirectReplication.VerifyStarted(upProbe.ref) upProbe.receiveMessage() // not bullet proof wrt to subscription being complete but good enough val event = PublishedEventImpl( - PersistenceId.replicatedUniqueId("pid", "ReplicaA"), + PersistenceId.replicatedUniqueId("pid", ReplicaId("ReplicaA")), 1L, "event", System.currentTimeMillis(), - Some(new ReplicatedPublishedEventMetaData("ReplicaA", VersionVector.empty))) + Some(new ReplicatedPublishedEventMetaData(ReplicaId("ReplicaA"), VersionVector.empty))) system.eventStream ! EventStream.Publish(event) replicaBProbe.receiveMessage().message should equal(event) 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 77dcd86a25..8738d8c206 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,6 +4,8 @@ package akka.persistence.testkit +import akka.actor.ActorLogging + import scala.collection.immutable import scala.concurrent.Future import scala.util.Try @@ -13,6 +15,7 @@ import akka.persistence._ import akka.persistence.journal.{ AsyncWriteJournal, Tagged } import akka.persistence.snapshot.SnapshotStore import akka.persistence.testkit.internal.{ InMemStorageExtension, SnapshotStorageEmulatorExtension } +import akka.util.unused /** * INTERNAL API @@ -20,9 +23,12 @@ 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]]] = { @@ -60,7 +66,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._ 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/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..6f85dd7d83 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 @@ -9,7 +9,6 @@ import java.util.concurrent.atomic.AtomicReference import scala.collection.immutable -import akka.actor.Extension import akka.annotation.InternalApi import akka.persistence.testkit.ProcessingPolicy @@ -151,4 +150,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 index dcd6246d40..cd0644590d 100644 --- 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 @@ -5,11 +5,13 @@ package akka.persistence.testkit.query import akka.actor.ExtendedActorSystem import akka.persistence.query.ReadJournalProvider +import com.typesafe.config.Config -class PersistenceTestKitReadJournalProvider(system: ExtendedActorSystem) extends ReadJournalProvider { +class PersistenceTestKitReadJournalProvider(system: ExtendedActorSystem, config: Config, configPath: String) + extends ReadJournalProvider { override def scaladslReadJournal(): scaladsl.PersistenceTestKitReadJournal = - new scaladsl.PersistenceTestKitReadJournal(system) + 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/scaladsl/PersistenceTestKitReadJournal.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/query/scaladsl/PersistenceTestKitReadJournal.scala index ebc50c233a..1df3f34bde 100644 --- 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 @@ -11,17 +11,27 @@ 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) +final class PersistenceTestKitReadJournal(system: ExtendedActorSystem, @unused config: Config, configPath: String) extends ReadJournal with EventsByPersistenceIdQuery with CurrentEventsByPersistenceIdQuery { - private val storage: EventStorage = InMemStorageExtension(system) + 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) + } override def eventsByPersistenceId( persistenceId: String, 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 1aa3910860..2fa2033421 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 @@ -431,7 +431,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) 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-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala index f844bb8538..56c8042ac8 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala @@ -26,9 +26,13 @@ object ActiveActiveEventPublishingSpec { case class Get(replyTo: ActorRef[Set[String]]) extends Command case object Stop extends Command - def apply(entityId: String, replicaId: String, allReplicas: Set[String]): Behavior[Command] = + def apply(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] = Behaviors.setup { ctx => - ActiveActiveEventSourcing(entityId, replicaId, allReplicas, PersistenceTestKitReadJournal.Identifier)( + ActiveActiveEventSourcing.withSharedJournal( + entityId, + replicaId, + allReplicas, + PersistenceTestKitReadJournal.Identifier)( aactx => EventSourcedBehavior[Command, String, Set[String]]( aactx.persistenceId, @@ -57,6 +61,10 @@ class ActiveActiveEventPublishingSpec 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 @@ -68,18 +76,18 @@ class ActiveActiveEventPublishingSpec "An active active actor" must { "move forward when a published event from a replica is received" in { val id = nextEntityId() - val actor = spawn(MyActiveActive(id, "DC-A", Set("DC-A", "DC-B"))) + val actor = spawn(MyActiveActive(id, DCA, Set(DCA, DCB))) val probe = createTestProbe[Any]() actor ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, "DC-B"), + PersistenceId.replicatedUniqueId(id, DCB), 1L, "two", System.currentTimeMillis(), - Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) + Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) actor ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -89,18 +97,18 @@ class ActiveActiveEventPublishingSpec "ignore a published event from a replica is received but the sequence number is unexpected" in { val id = nextEntityId() - val actor = spawn(MyActiveActive(id, "DC-A", Set("DC-A", "DC-B"))) + val actor = spawn(MyActiveActive(id, DCA, Set(DCA, DCB))) val probe = createTestProbe[Any]() actor ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, "DC-B"), + PersistenceId.replicatedUniqueId(id, DCB), 2L, // missing 1L "two", System.currentTimeMillis(), - Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) + Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) actor ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -110,18 +118,18 @@ class ActiveActiveEventPublishingSpec "ignore a published event from an unknown replica" in { val id = nextEntityId() - val actor = spawn(MyActiveActive(id, "DC-A", Set("DC-A", "DC-B"))) + val actor = spawn(MyActiveActive(id, DCA, Set(DCA, DCB))) val probe = createTestProbe[Any]() actor ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, "DC-C"), + PersistenceId.replicatedUniqueId(id, DCC), 1L, "two", System.currentTimeMillis(), - Some(new ReplicatedPublishedEventMetaData("DC-C", VersionVector.empty))) + Some(new ReplicatedPublishedEventMetaData(DCC, VersionVector.empty))) actor ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -131,25 +139,25 @@ class ActiveActiveEventPublishingSpec "ignore an already seen event from a replica" in { val id = nextEntityId() - val actor = spawn(MyActiveActive(id, "DC-A", Set("DC-A", "DC-B"))) + val actor = spawn(MyActiveActive(id, DCA, Set(DCA, DCB))) val probe = createTestProbe[Any]() actor ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId("myId4", "DC-B"), + PersistenceId.replicatedUniqueId("myId4", DCB), 1L, "two", System.currentTimeMillis(), - Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) + Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) // simulate another published event from that replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, "DC-B"), + PersistenceId.replicatedUniqueId(id, DCB), 1L, "two-again", // ofc this would be the same in the real world, different just so we can detect System.currentTimeMillis(), - Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) + Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) actor ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -161,7 +169,7 @@ class ActiveActiveEventPublishingSpec "handle published events after replay" in { val id = nextEntityId() val probe = createTestProbe[Any]() - val activeActiveBehavior = MyActiveActive(id, "DC-A", Set("DC-A", "DC-B")) + val activeActiveBehavior = MyActiveActive(id, DCA, Set(DCA, DCB)) val incarnation1 = spawn(activeActiveBehavior) incarnation1 ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) @@ -177,11 +185,11 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica incarnation2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, "DC-B"), + PersistenceId.replicatedUniqueId(id, DCB), 1L, "two", System.currentTimeMillis(), - Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) + Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) incarnation2 ! MyActiveActive.Add("three", probe.ref) probe.expectMessage(Done) @@ -193,18 +201,18 @@ class ActiveActiveEventPublishingSpec "handle published events before and after replay" in { val id = nextEntityId() val probe = createTestProbe[Any]() - val activeActiveBehaviorA = MyActiveActive(id, "DC-A", Set("DC-A", "DC-B")) + val activeActiveBehaviorA = MyActiveActive(id, DCA, Set(DCA, DCB)) val incarnationA1 = spawn(activeActiveBehaviorA) incarnationA1 ! MyActiveActive.Add("one", probe.ref) probe.expectMessage(Done) // simulate a published event from another replica incarnationA1.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, "DC-B"), + PersistenceId.replicatedUniqueId(id, DCB), 1L, "two", System.currentTimeMillis(), - Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) + Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) incarnationA1 ! MyActiveActive.Stop probe.expectTerminated(incarnationA1) @@ -213,11 +221,11 @@ class ActiveActiveEventPublishingSpec // simulate a published event from another replica incarnationA2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, "DC-B"), + PersistenceId.replicatedUniqueId(id, DCB), 2L, "three", System.currentTimeMillis(), - Some(new ReplicatedPublishedEventMetaData("DC-B", VersionVector.empty))) + Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) incarnationA2 ! MyActiveActive.Add("four", probe.ref) probe.expectMessage(Done) diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala index 58effb19cf..e9f33b0bc2 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala @@ -7,23 +7,27 @@ 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.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.scaladsl.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import akka.persistence.typed.scaladsl.ActiveActiveEventSourcing +import akka.persistence.typed.scaladsl.Effect +import akka.persistence.typed.scaladsl.EventSourcedBehavior import org.scalatest.concurrent.Eventually import org.scalatest.wordspec.AnyWordSpecLike object ActiveActiveSpec { - val AllReplicas = Set("R1", "R2", "R3") + 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[(String, Set[String])]) extends Command + case class GetReplica(replyTo: ActorRef[(ReplicaId, Set[ReplicaId])]) extends Command case object Stop extends Command case class State(all: List[String]) @@ -34,7 +38,11 @@ object ActiveActiveSpec { entityId: String, replicaId: String, probe: Option[ActorRef[EventAndContext]] = None): Behavior[Command] = - ActiveActiveEventSourcing(entityId, replicaId, AllReplicas, PersistenceTestKitReadJournal.Identifier)( + ActiveActiveEventSourcing.withSharedJournal( + entityId, + ReplicaId(replicaId), + AllReplicas, + PersistenceTestKitReadJournal.Identifier)( aaContext => EventSourcedBehavior[Command, String, State]( aaContext.persistenceId, @@ -61,7 +69,7 @@ object ActiveActiveSpec { } -case class EventAndContext(event: Any, origin: String, recoveryRunning: Boolean, concurrent: Boolean) +case class EventAndContext(event: Any, origin: ReplicaId, recoveryRunning: Boolean, concurrent: Boolean) class ActiveActiveSpec extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) @@ -145,10 +153,10 @@ class ActiveActiveSpec "have access to replica information" in { val entityId = nextEntityId - val probe = createTestProbe[(String, Set[String])]() + val probe = createTestProbe[(ReplicaId, Set[ReplicaId])]() val r1 = spawn(testBehavior(entityId, "R1")) r1 ! GetReplica(probe.ref) - probe.expectMessage(("R1", Set("R1", "R2", "R3"))) + probe.expectMessage((ReplicaId("R1"), Set(ReplicaId("R1"), ReplicaId("R2"), ReplicaId("R3")))) } "have access to event origin" in { @@ -161,12 +169,12 @@ class ActiveActiveSpec val r2 = spawn(testBehavior(entityId, "R2", eventProbeR2.ref)) r1 ! StoreMe("from r1", replyProbe.ref) - eventProbeR2.expectMessage(EventAndContext("from r1", "R1", false, false)) - eventProbeR1.expectMessage(EventAndContext("from r1", "R1", false, false)) + 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", "R2", false, false)) - eventProbeR2.expectMessage(EventAndContext("from r2", "R2", false, false)) + eventProbeR1.expectMessage(EventAndContext("from r2", ReplicaId("R2"), false, false)) + eventProbeR2.expectMessage(EventAndContext("from r2", ReplicaId("R2"), false, false)) } "set recovery running" in { @@ -175,12 +183,12 @@ class ActiveActiveSpec val replyProbe = createTestProbe[Done]() val r1 = spawn(testBehavior(entityId, "R1", eventProbeR1.ref)) r1 ! StoreMe("Event", replyProbe.ref) - eventProbeR1.expectMessage(EventAndContext("Event", "R1", recoveryRunning = false, false)) + 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", "R1", recoveryRunning = true, false)) + recoveryProbe.expectMessage(EventAndContext("Event", ReplicaId("R1"), recoveryRunning = true, false)) } "persist all" in { @@ -197,10 +205,10 @@ class ActiveActiveSpec // events at r2 happened concurrently with events at r1 - eventProbeR1.expectMessage(EventAndContext("1 from r1", "R1", false, concurrent = false)) - eventProbeR1.expectMessage(EventAndContext("2 from r1", "R1", false, concurrent = false)) - eventProbeR1.expectMessage(EventAndContext("1 from r2", "R2", false, concurrent = true)) - eventProbeR1.expectMessage(EventAndContext("2 from r2", "R2", false, concurrent = true)) + 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]() @@ -225,12 +233,16 @@ class ActiveActiveSpec r2 ! StoreMe("from r2", probe.ref) // R2 0 R1 0 -> R2 1 R1 0 // each gets its local event - eventProbeR1.expectMessage(EventAndContext("from r1", "R1", recoveryRunning = false, concurrent = false)) - eventProbeR2.expectMessage(EventAndContext("from r2", "R2", recoveryRunning = false, concurrent = false)) + 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", "R2", recoveryRunning = false, concurrent = true)) - eventProbeR2.expectMessage(EventAndContext("from r1", "R1", recoveryRunning = false, concurrent = true)) + 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 { @@ -246,11 +258,11 @@ class ActiveActiveSpec // 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", "R1", false, concurrent = false)) - eventProbeR2.expectMessage(EventAndContext("from r1 2", "R1", false, concurrent = false)) + 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", "R1", false, concurrent = false)) - eventProbeR2.expectMessage(EventAndContext("from r1 3", "R1", false, concurrent = false)) + 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) @@ -259,8 +271,8 @@ class ActiveActiveSpec // 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", "R2", false, concurrent = false)) - eventProbeR2.expectMessage(EventAndContext("from r2 2", "R2", false, concurrent = false)) + 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) @@ -286,8 +298,8 @@ class ActiveActiveSpec probe.expectMessage(Done) // r2 - eventProbeR2.expectMessage(EventAndContext("from r1 1", "R1", false, false)) - eventProbeR2.expectMessage(EventAndContext("from r1 2", "R1", false, false)) + 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) @@ -297,10 +309,10 @@ class ActiveActiveSpec // r3 should only get the events 1, not R2s stored version of them val eventProbeR3 = createTestProbe[EventAndContext]() spawn(testBehavior(entityId, "R3", eventProbeR3.ref)) - eventProbeR3.expectMessage(EventAndContext("from r1 1", "R1", false, false)) - eventProbeR3.expectMessage(EventAndContext("from r1 2", "R1", false, false)) - eventProbeR3.expectMessage(EventAndContext("from r2 1", "R2", false, false)) - eventProbeR3.expectMessage(EventAndContext("from r2 2", "R2", false, false)) + eventProbeR3.expectMessage(EventAndContext("from r1 1", ReplicaId("R1"), false, false)) + eventProbeR3.expectMessage(EventAndContext("from r1 2", ReplicaId("R1"), false, false)) + eventProbeR3.expectMessage(EventAndContext("from r2 1", ReplicaId("R2"), false, false)) + eventProbeR3.expectMessage(EventAndContext("from r2 2", ReplicaId("R2"), false, false)) eventProbeR3.expectNoMessage() } @@ -313,14 +325,18 @@ class ActiveActiveSpec 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", "R1", recoveryRunning = false, concurrent = false)) - eventProbeR1.expectMessage(EventAndContext("from r2", "R2", recoveryRunning = false, concurrent = true)) + 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", "R1", recoveryRunning = true, concurrent = false)) - eventProbeR1Take2.expectMessage(EventAndContext("from r2", "R2", recoveryRunning = true, concurrent = true)) + eventProbeR1Take2.expectMessage( + EventAndContext("from r1", ReplicaId("R1"), recoveryRunning = true, concurrent = false)) + eventProbeR1Take2.expectMessage( + EventAndContext("from r2", ReplicaId("R2"), recoveryRunning = true, concurrent = true)) } } diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalActiveActiveSpec.scala new file mode 100644 index 0000000000..d4e0803997 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalActiveActiveSpec.scala @@ -0,0 +1,111 @@ +/* + * 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.ActiveActiveEventSourcing +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 MultiJournalActiveActiveSpec { + + 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] = { + ActiveActiveEventSourcing( + entityId, + ReplicaId(replicaId), + Map(ReplicaId("R1") -> "journal1.query", ReplicaId("R2") -> "journal2.query"))( + aaContext => + EventSourcedBehavior[Command, String, Set[String]]( + aaContext.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 MultiJournalActiveActiveSpec + extends ScalaTestWithActorTestKit(MultiJournalActiveActiveSpec.separateJournalsConfig) + with AnyWordSpecLike + with LogCapturing + with Eventually { + import MultiJournalActiveActiveSpec._ + val ids = new AtomicInteger(0) + def nextEntityId = s"e-${ids.getAndIncrement()}" + "ActiveActiveEventSourcing" 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("id1|R1", 0L, Long.MaxValue).runWith(Sink.seq).futureValue + eventsForJournal1.map(_.event).toSet should ===(Set("r1 m1", "r2 m1")) + + val eventsForJournal2 = + readJournal2.currentEventsByPersistenceId("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/docs.akka.persistence.typed/AAAuctionExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AAAuctionExampleSpec.scala index 4577431cb6..65dcbf9a14 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AAAuctionExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AAAuctionExampleSpec.scala @@ -11,6 +11,7 @@ import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, _ } import akka.actor.typed.{ ActorRef, Behavior } import akka.persistence.testkit.PersistenceTestKitPlugin import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal +import akka.persistence.typed.ReplicaId import akka.persistence.typed.scaladsl.{ ActiveActiveContext, ActiveActiveEventSourcing, Effect, EventSourcedBehavior } import akka.serialization.jackson.CborSerializable import org.scalatest.concurrent.{ Eventually, ScalaFutures } @@ -21,7 +22,7 @@ object AAAuctionExampleSpec { type MoneyAmount = Int - case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originDc: String) + case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originDc: ReplicaId) // commands sealed trait AuctionCommand @@ -33,12 +34,13 @@ object AAAuctionExampleSpec { sealed trait AuctionEvent extends CborSerializable final case class BidRegistered(bid: Bid) extends AuctionEvent - final case class AuctionFinished(atDc: String) extends AuctionEvent - final case class WinnerDecided(atDc: String, winningBid: Bid, highestCounterOffer: MoneyAmount) extends AuctionEvent + final case class AuctionFinished(atDc: ReplicaId) extends AuctionEvent + final case class WinnerDecided(atDc: ReplicaId, winningBid: Bid, highestCounterOffer: MoneyAmount) + extends AuctionEvent sealed trait AuctionPhase case object Running extends AuctionPhase - final case class Closing(finishedAtDc: Set[String]) extends AuctionPhase + final case class Closing(finishedAtDc: Set[ReplicaId]) extends AuctionPhase case object Closed extends AuctionPhase case class AuctionState( @@ -85,8 +87,8 @@ object AAAuctionExampleSpec { // 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.originDc.compareTo( - second.originDc) < 0) + (first.offer == second.offer && first.timestamp.equals(second.timestamp) && first.originDc.id + .compareTo(second.originDc.id) < 0) } case class AuctionSetup( @@ -94,7 +96,7 @@ object AAAuctionExampleSpec { initialBid: Bid, // the initial bid is basically the minimum price bidden at start time by the owner closingAt: Instant, responsibleForClosing: Boolean, - allDcs: Set[String]) + allDcs: Set[ReplicaId]) def commandHandler(setup: AuctionSetup, ctx: ActorContext[AuctionCommand], aaContext: ActiveActiveContext)( state: AuctionState, @@ -199,15 +201,16 @@ object AAAuctionExampleSpec { def initialState(setup: AuctionSetup) = AuctionState(phase = Running, highestBid = setup.initialBid, highestCounterOffer = setup.initialBid.offer) - def behavior(replica: String, setup: AuctionSetup): Behavior[AuctionCommand] = Behaviors.setup[AuctionCommand] { + def behavior(replica: ReplicaId, setup: AuctionSetup): Behavior[AuctionCommand] = Behaviors.setup[AuctionCommand] { ctx => - ActiveActiveEventSourcing(setup.name, replica, setup.allDcs, PersistenceTestKitReadJournal.Identifier) { aaCtx => - EventSourcedBehavior( - aaCtx.persistenceId, - initialState(setup), - commandHandler(setup, ctx, aaCtx), - eventHandler(ctx, aaCtx, setup)) - } + ActiveActiveEventSourcing + .withSharedJournal(setup.name, replica, setup.allDcs, PersistenceTestKitReadJournal.Identifier) { aaCtx => + EventSourcedBehavior( + aaCtx.persistenceId, + initialState(setup), + commandHandler(setup, ctx, aaCtx), + eventHandler(ctx, aaCtx, setup)) + } } } @@ -223,19 +226,19 @@ class AAAuctionExampleSpec "Auction example" should { "work" in { - val Replicas = Set("DC-A", "DC-B") + val Replicas = Set(ReplicaId("DC-A"), ReplicaId("DC-B")) val setupA = AuctionSetup( "old-skis", - Bid("chbatey", 12, Instant.now(), "DC-A"), + Bid("chbatey", 12, Instant.now(), ReplicaId("DC-A")), Instant.now().plusSeconds(10), responsibleForClosing = true, Replicas) val setupB = setupA.copy(responsibleForClosing = false) - val dcAReplica: ActorRef[AuctionCommand] = spawn(behavior("DC-A", setupA)) - val dcBReplica: ActorRef[AuctionCommand] = spawn(behavior("DC-B", setupB)) + val dcAReplica: ActorRef[AuctionCommand] = spawn(behavior(ReplicaId("DC-A"), setupA)) + val dcBReplica: ActorRef[AuctionCommand] = spawn(behavior(ReplicaId("DC-B"), setupB)) dcAReplica ! OfferBid("me", 100) dcAReplica ! OfferBid("me", 99) diff --git a/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala index 99106b5c71..6c297f4ea1 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala @@ -10,6 +10,7 @@ import akka.actor.typed.ActorRef import akka.actor.typed.scaladsl.{ ActorContext, Behaviors } import akka.persistence.testkit.PersistenceTestKitPlugin import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal +import akka.persistence.typed.ReplicaId import akka.persistence.typed.scaladsl._ import akka.serialization.jackson.CborSerializable import org.scalatest.concurrent.{ Eventually, ScalaFutures } @@ -24,7 +25,7 @@ object AABlogExampleSpec { copy(content = Some(newContent), contentTimestamp = timestamp) def isEmpty: Boolean = content.isEmpty } - val emptyState: BlogState = BlogState(None, LwwTime(Long.MinValue, ""), published = false) + val emptyState: BlogState = BlogState(None, LwwTime(Long.MinValue, ReplicaId("")), published = false) final case class PostContent(title: String, body: String) final case class PostSummary(postId: String, title: String) @@ -110,20 +111,30 @@ class AABlogExampleSpec "Blog Example" should { "work" in { val refDcA: ActorRef[BlogCommand] = - spawn(Behaviors.setup[BlogCommand] { ctx => - ActiveActiveEventSourcing("cat", "DC-A", Set("DC-A", "DC-B"), PersistenceTestKitReadJournal.Identifier) { - (aa: ActiveActiveContext) => + spawn( + Behaviors.setup[BlogCommand] { ctx => + ActiveActiveEventSourcing.withSharedJournal( + "cat", + ReplicaId("DC-A"), + Set(ReplicaId("DC-A"), ReplicaId("DC-B")), + PersistenceTestKitReadJournal.Identifier) { (aa: ActiveActiveContext) => behavior(aa, ctx) - } - }, "dc-a") + } + }, + "dc-a") val refDcB: ActorRef[BlogCommand] = - spawn(Behaviors.setup[BlogCommand] { ctx => - ActiveActiveEventSourcing("cat", "DC-B", Set("DC-A", "DC-B"), PersistenceTestKitReadJournal.Identifier) { - (aa: ActiveActiveContext) => + spawn( + Behaviors.setup[BlogCommand] { ctx => + ActiveActiveEventSourcing.withSharedJournal( + "cat", + ReplicaId("DC-B"), + Set(ReplicaId("DC-A"), ReplicaId("DC-B")), + PersistenceTestKitReadJournal.Identifier) { (aa: ActiveActiveContext) => behavior(aa, ctx) - } - }, "dc-b") + } + }, + "dc-b") import akka.actor.typed.scaladsl.AskPattern._ import akka.util.Timeout 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 7614edec35..6018759038 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 @@ -130,16 +130,16 @@ object PersistenceId { * Constructs a persistence id from a unique entity id that includes the replica id. */ @InternalApi - private[akka] def replicatedUniqueId(entityId: String, replicaId: String): PersistenceId = { + private[akka] def replicatedUniqueId(entityId: String, replicaId: ReplicaId): PersistenceId = { if (entityId.contains(DefaultSeparator)) throw new IllegalArgumentException( s"entityId [$entityId] contains [$DefaultSeparator] which is a reserved character") - if (replicaId.contains(DefaultSeparator)) + if (replicaId.id.contains(DefaultSeparator)) throw new IllegalArgumentException( - s"replicaId [$replicaId] contains [$DefaultSeparator] which is a reserved character") + s"replicaId [${replicaId.id}] contains [$DefaultSeparator] which is a reserved character") - new PersistenceId(entityId + DefaultSeparator + replicaId) + new PersistenceId(entityId + DefaultSeparator + replicaId.id) } } 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..b01082a980 --- /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 Active Active eventsourcing, 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/internal/BehaviorSetup.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala index 20281bace5..4528b602c4 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 @@ -12,6 +12,7 @@ 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.scaladsl.EventSourcedBehavior.ActiveActive import akka.persistence.typed.{ EventAdapter, PersistenceId, SnapshotAdapter } import akka.persistence.typed.scaladsl.{ EventSourcedBehavior, RetentionCriteria } @@ -61,7 +62,7 @@ private[akka] final class BehaviorSetup[C, E, S]( val journal: ClassicActorRef = persistence.journalFor(settings.journalPluginId) val snapshotStore: ClassicActorRef = persistence.snapshotStoreFor(settings.snapshotPluginId) - val replicaId: Option[String] = activeActive.map(_.replicaId) + val replicaId: Option[ReplicaId] = activeActive.map(_.replicaId) def selfClassic: ClassicActorRef = context.self.toClassic 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 5c46366211..95e474860e 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 @@ -35,6 +35,7 @@ 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 @@ -252,10 +253,9 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( override private[akka] def withActiveActive( context: ActiveActiveContextImpl, - id: String, - allIds: Set[String], - queryPluginId: String): EventSourcedBehavior[Command, Event, State] = { - copy(activeActive = Some(ActiveActive(id, allIds, context, queryPluginId))) + replicaId: ReplicaId, + allReplicaIdsAndQueryPlugins: Map[ReplicaId, String]): EventSourcedBehavior[Command, Event, State] = { + copy(activeActive = Some(ActiveActive(replicaId, allReplicaIdsAndQueryPlugins, context))) } } @@ -282,7 +282,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( */ @InternalApi private[akka] final case class ReplicatedEventMetaData( - originReplica: String, + originReplica: ReplicaId, originSequenceNr: Long, version: VersionVector, concurrent: Boolean) // whether when the event handler was executed the event was concurrent @@ -296,13 +296,13 @@ private[akka] final case class ReplicatedEventMetaData( @InternalApi private[akka] final case class ReplicatedEvent[E]( event: E, - originReplica: String, + originReplica: ReplicaId, originSequenceNr: Long, originVersion: VersionVector) @InternalApi private[akka] case object ReplicatedEventAck -final class ReplicatedPublishedEventMetaData(val replicaId: String, private[akka] val version: VersionVector) +final class ReplicatedPublishedEventMetaData(val replicaId: ReplicaId, private[akka] val version: VersionVector) /** * INTERNAL API 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 d2612bce34..96b90d565f 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 @@ -18,6 +18,7 @@ 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.ReplayingEvents.ReplayingState @@ -52,7 +53,7 @@ private[akka] object ReplayingEvents { receivedPoisonPill: Boolean, recoveryStartTime: Long, version: VersionVector, - seenSeqNrPerReplica: Map[String, Long]) + seenSeqNrPerReplica: Map[ReplicaId, Long]) def apply[C, E, S](setup: BehaviorSetup[C, E, S], state: ReplayingState[S]): Behavior[InternalProtocol] = Behaviors.setup { _ => @@ -121,7 +122,7 @@ private[akka] final class ReplayingEvents[C, E, S]( eventForErrorReporting = OptionVal.Some(event) state = state.copy(seqNr = repr.sequenceNr) - val aaMetaAndSelfReplica: Option[(ReplicatedEventMetaData, String)] = + val aaMetaAndSelfReplica: Option[(ReplicatedEventMetaData, ReplicaId)] = setup.activeActive match { case Some(aa) => val meta = repr.metadata match { 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 7c6af4a82a..c0d83579c5 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 @@ -34,6 +34,7 @@ import akka.persistence.SnapshotProtocol import akka.persistence.journal.Tagged import akka.persistence.query.{ EventEnvelope, PersistenceQuery } import akka.persistence.query.scaladsl.EventsByPersistenceIdQuery +import akka.persistence.typed.ReplicaId import akka.persistence.typed.{ DeleteEventsCompleted, DeleteEventsFailed, @@ -91,8 +92,8 @@ private[akka] object Running { state: State, receivedPoisonPill: Boolean, version: VersionVector, - seenPerReplica: Map[String, Long], - replicationControl: Map[String, ReplicationStreamControl]) { + seenPerReplica: Map[ReplicaId, Long], + replicationControl: Map[ReplicaId, ReplicationStreamControl]) { def nextSequenceNr(): RunningState[State] = copy(seqNr = seqNr + 1) @@ -128,8 +129,8 @@ private[akka] object Running { if (replicaId != aa.replicaId) { val seqNr = state.seenPerReplica(replicaId) val pid = PersistenceId.replicatedUniqueId(aa.aaContext.entityId, replicaId) - // FIXME support different configuration per replica https://github.com/akka/akka/issues/29257 - val replication = query.readJournalFor[EventsByPersistenceIdQuery](aa.queryPluginId) + val queryPluginId = aa.allReplicasAndQueryPlugins(replicaId) + val replication = query.readJournalFor[EventsByPersistenceIdQuery](queryPluginId) implicit val timeout = Timeout(30.seconds) @@ -424,7 +425,7 @@ private[akka] object Running { val newState2 = setup.activeActive match { case Some(aa) => - val updatedVersion = newState.version.updated(aa.replicaId, _currentSequenceNumber) + val updatedVersion = newState.version.updated(aa.replicaId.id, _currentSequenceNumber) val r = internalPersist( setup.context, cmd, @@ -475,7 +476,7 @@ private[akka] object Running { val adaptedEvent = adaptEvent(event) val eventMetadata = metadataTemplate match { case Some(template) => - val updatedVersion = currentState.version.updated(template.originReplica, _currentSequenceNumber) + val updatedVersion = currentState.version.updated(template.originReplica.id, _currentSequenceNumber) setup.log.trace("Processing event [{}] with version vector [{}]", event, updatedVersion) currentState = currentState.copy(version = updatedVersion) Some(template.copy(originSequenceNr = _currentSequenceNumber, version = updatedVersion)) diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala index 888c7e3487..77340336f5 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala @@ -5,20 +5,21 @@ package akka.persistence.typed.scaladsl import akka.persistence.typed.PersistenceId +import akka.persistence.typed.ReplicaId import akka.util.WallClock /** * Utility class for comparing timestamp and data center * identifier when implementing last-writer wins. */ -final case class LwwTime(timestamp: Long, originDc: String) { +final case class LwwTime(timestamp: Long, originDc: 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: String): LwwTime = + def increase(t: Long, replicaId: ReplicaId): LwwTime = LwwTime(math.max(timestamp + 1, t), replicaId) /** @@ -30,7 +31,7 @@ final case class LwwTime(timestamp: Long, originDc: String) { def isAfter(other: LwwTime): Boolean = { if (timestamp > other.timestamp) true else if (timestamp < other.timestamp) false - else if (other.originDc.compareTo(originDc) > 0) true + else if (other.originDc.id.compareTo(originDc.id) > 0) true else false } } @@ -38,10 +39,10 @@ final case class LwwTime(timestamp: Long, originDc: String) { // FIXME docs trait ActiveActiveContext { - def origin: String + def origin: ReplicaId def concurrent: Boolean - def replicaId: String - def allReplicas: Set[String] + def replicaId: ReplicaId + def allReplicas: Set[ReplicaId] def persistenceId: PersistenceId def recoveryRunning: Boolean def entityId: String @@ -52,9 +53,13 @@ trait ActiveActiveContext { // FIXME, parts of this can be set during initialisation // Other fields will be set before executing the event handler as they change per event // https://github.com/akka/akka/issues/29258 -private[akka] class ActiveActiveContextImpl(val entityId: String, val replicaId: String, val allReplicas: Set[String]) +private[akka] class ActiveActiveContextImpl( + val entityId: String, + val replicaId: ReplicaId, + val replicasAndQueryPlugins: Map[ReplicaId, String]) extends ActiveActiveContext { - var _origin: String = null + val allReplicas: Set[ReplicaId] = replicasAndQueryPlugins.keySet + var _origin: ReplicaId = null var _recoveryRunning: Boolean = false var _concurrent: Boolean = false @@ -64,7 +69,7 @@ private[akka] class ActiveActiveContextImpl(val entityId: String, val replicaId: * The origin of the current event. * Undefined result if called from anywhere other than an event handler. */ - override def origin: String = _origin + override def origin: ReplicaId = _origin /** * Whether the happened concurrently with an event from another replica. @@ -83,7 +88,7 @@ private[akka] class ActiveActiveContextImpl(val entityId: String, val replicaId: object ActiveActiveEventSourcing { /** - * Initialize a replicated event sourced behavior. + * 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. @@ -93,21 +98,43 @@ object ActiveActiveEventSourcing { * 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. - * TODO support a different query plugin id per replicas: https://github.com/akka/akka/issues/29257 * * @param replicaId The unique identity for this entity. The underlying persistence id will include the replica. * @param allReplicaIds All replica ids. These need to be known to receive events from all replicas. - * @param queryPluginId Used to read the events from other replicas. Must be the query side of your configured journal plugin. - * @return + * @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 withSharedJournal[Command, Event, State]( + entityId: String, + replicaId: ReplicaId, + allReplicaIds: Set[ReplicaId], + queryPluginId: String)(activeActiveContext: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) + : EventSourcedBehavior[Command, Event, State] = + apply(entityId, replicaId, allReplicaIds.map(id => id -> queryPluginId).toMap)(activeActiveContext) + + /** + * 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 replicaId The unique identity for this entity. The underlying persistence id will include the replica. + * @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 apply[Command, Event, State]( entityId: String, - replicaId: String, - allReplicaIds: Set[String], - queryPluginId: String)(activeActiveContext: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) + replicaId: ReplicaId, + allReplicasAndQueryPlugins: Map[ReplicaId, String])( + activeActiveContext: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = { - val context = new ActiveActiveContextImpl(entityId, replicaId, allReplicaIds) - activeActiveContext(context).withActiveActive(context, replicaId, allReplicaIds, queryPluginId) + val context = new ActiveActiveContextImpl(entityId, replicaId, allReplicasAndQueryPlugins) + activeActiveContext(context).withActiveActive(context, replicaId, allReplicasAndQueryPlugins) } } 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 c7357ac198..56dcf975cf 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 @@ -16,6 +16,7 @@ import akka.annotation.ApiMayChange import akka.annotation.{ DoNotInherit, InternalApi } import akka.persistence.typed.EventAdapter import akka.persistence.typed.PersistenceId +import akka.persistence.typed.ReplicaId import akka.persistence.typed.SnapshotAdapter import akka.persistence.typed.SnapshotSelectionCriteria import akka.persistence.typed.internal._ @@ -25,15 +26,16 @@ object EventSourcedBehavior { // FIXME move to internal @InternalApi private[akka] final case class ActiveActive( - replicaId: String, - allReplicas: Set[String], - aaContext: ActiveActiveContextImpl, - queryPluginId: String) { + replicaId: ReplicaId, + allReplicasAndQueryPlugins: Map[ReplicaId, String], + aaContext: ActiveActiveContextImpl) { + + 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: String, concurrent: Boolean): Unit = { + def setContext(recoveryRunning: Boolean, originReplica: ReplicaId, concurrent: Boolean): Unit = { aaContext._recoveryRunning = recoveryRunning aaContext._concurrent = concurrent aaContext._origin = originReplica @@ -166,9 +168,8 @@ object EventSourcedBehavior { private[akka] def withActiveActive( context: ActiveActiveContextImpl, - replicaId: String, - allReplicaIds: Set[String], - queryPluginId: String): EventSourcedBehavior[Command, Event, State] + replicaId: ReplicaId, + allReplicasAndQueryPlugins: Map[ReplicaId, String]): EventSourcedBehavior[Command, Event, State] /** * Change the snapshot store plugin id that this actor should use. From 2c0e837183d888572f028986c980c53414513345 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Thu, 16 Jul 2020 13:14:56 +0100 Subject: [PATCH 11/50] Initial docs for active active (#29386) --- .../main/paradox/typed/images/causality.png | Bin 0 -> 289837 bytes .../src/main/paradox/typed/images/lww.png | Bin 0 -> 106895 bytes .../typed/images/replicated-events1.png | Bin 0 -> 58339 bytes .../typed/images/replicated-events2.png | Bin 0 -> 70701 bytes .../main/paradox/typed/index-persistence.md | 1 + .../typed/persistence-active-active.md | 192 ++++++++++++++++++ .../typed}/AAAuctionExampleSpec.scala | 0 .../typed}/AABlogExampleSpec.scala | 0 .../typed/ActiveActiveCompileOnlySpec.scala | 28 +++ .../scaladsl/ActiveActiveEventSourcing.scala | 9 +- 10 files changed, 226 insertions(+), 4 deletions(-) create mode 100644 akka-docs/src/main/paradox/typed/images/causality.png create mode 100644 akka-docs/src/main/paradox/typed/images/lww.png create mode 100644 akka-docs/src/main/paradox/typed/images/replicated-events1.png create mode 100644 akka-docs/src/main/paradox/typed/images/replicated-events2.png create mode 100644 akka-docs/src/main/paradox/typed/persistence-active-active.md rename akka-persistence-typed-tests/src/test/scala/{docs.akka.persistence.typed => docs/akka/persistence/typed}/AAAuctionExampleSpec.scala (100%) rename akka-persistence-typed-tests/src/test/scala/{docs.akka.persistence.typed => docs/akka/persistence/typed}/AABlogExampleSpec.scala (100%) create mode 100644 akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala 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 0000000000000000000000000000000000000000..cc19c839aa7d9e242e6f966b867fce55260d7386 GIT binary patch literal 289837 zcmeAS@N?(olHy`uVBq!ia0y~yU_HRVz}&*Y#=yYPwrbKM1_mCfOlRi+PiJR^fTH}g z%$!sP29M6E;p`#7p5pVugE%;K1sF9Knmblaz1Q7PvO#bX!@?E;4y|vSRy>Fb5Ioqm zu0dmmn1h(;#so)}H>HlQsj0C#Hv)dDb1r!O@9CfKb1FaYwf=tZ=et_t_h%Rs@=ptK z7zMC082#LNh@)OVEvANP@)d3a1I7~#oeb$4i&^G9e5lB9D&l{6Wx24zqkpq@T3oX~ z{(rSb(GJTTh6x27lPxUY_;mC$HLTvudMJj0V{X8DbM@nb znxk@bn$YYO9Qhr!3<|-KI*D8iMUQ@*u_@!~QC4rbG3ornkBawLH%ePiJ=(Z~{gLO) znUMmggB8vEe(Cw0x}T)hdpu=h(6Nms;++#}D=eFMuI>51S>}JVk+Dm1@ry%lzM^a{ z9mQ>>tG&*1i6x(q&3vZb+UB^10;x&(@^wg1T7bV1)K zt9fPGv!17|ksBf>?F>=;cEY!pYr?Oj!aH+zOfck=S=YaQ3iGjtpBB6COe`p^bnW>i z#$j~G{q08CJ3Rk6GUsk|wyd0(yzlPLf=18R4Ypp%tgYTZli#!Kd>9y%XTj&xVE%#8 zN1;iVGo{hwz-x&NiH7`3g7ycPoF*{sI8YN}{7g_=^5Nlhl@sjsPDfZRAJ@fA3e-E% z{7e6F)XU2Onaem^;KRROJ&A8IGv?0VuGeYL|H$Fb8QbFCBHbc)NLGM(p2wn|sI$*>h5sFx z!?~GlkJJI@RCt6>o07^MK9hNv~1b+5A{ z_{|z>k91ndrayXCP|NnJ(Ecdv(#O@ONCY{LHIlkYN)hGOXlKMpGQ{5-eBJ-b;l@V@^ z#~hADs`<1j8~dltJrg%mZRTpDTBE}vu3dshrwX}xF4kE(B`8btxyEe+)p18`zbZJ#^dVuSdtyjFVxWDp!LOAVHIgl`Gr2|g1Nvv^1R$3>5dRVPhaIBC6*duHn;xuwQl{VyeA```Jd zU!H#X`=#?2e^2Q=<#wv^ROTtyQ|wc%r?7`k3CRgf3VF6_%IY}*FB21ZoZS?Zn>Y3q z&M2JxF|+9z%X!W_oU7HZ_`R6-A+B`Ms&=jLtpQv8zP7zmH(fSsebla7Y+Kp3vcKA~ zYvnJKua>W@ubf}W?-Mfbh0#RYjRlV$c9xq4@CNsC^!}8pHo0ZmX0p}fuPN)SXw&l< zd>MvYY_?q5VwO4g%B$_5RrYKDtMLos7uR2RICkKo!qtRs$Jqy~os|zq z9G=?7+jhP;q?f7pQ*Tz!JL$_kcY8PYYWJ+4E#NETd&zg(?2=iFmX$4Ax$Nn(@UsEN z)yDH~mTg|=+2ZLp`Nw1}wYx^kXI0O7KJ)sF?@6RNSZ@yf-d2jK$&xz0X z$DE116#H$r&HN;jR_oOFQ;x6OwNB^y-F3EkKHFWk$L86-vwElVo$I^1T$P+mf34hk z8(*8tHvAR)zHIs;R3-bl>iyd{doS!h%KrJ|cg-&^KZJhj{igf5_Qmqo(;tODHh+=- z-2VQ4)dm3pnGK2$!WN_{SZ}Z@FgT%@k$B-|LRLXaLF9+U0+$66JuZioWL>z@8~naq zs^ycj^dX0cgd4{;us?Krc=O?YUu8e;`C)c%Gh0HBG<9tAsCx3|#NUbU7wuVe&Gl}# zS)Z8e_G2fGpXpfHS1JEmI9lemu(@Sk;+>}zUn}Bjl9@CW)e_?mT`0N#a`y7?XE)3a zG956IdXHqijYvU@OSmPwAPJU5zd~Cp0_8l-Msfk_l>v= z|2Z{3`|DGq&(uzPGb?7!ADg45smWgTtE#g+Uoo7Y9e%8Os@Q~W#&2t@BC1kE=bw`kVdMV(RKe1?SgLsH!@%XSDLTJA zugrcvEqvbhM87k0l#4~5e|o;{yz$)aRrfa6?Ee%0?D6NDpQFWJ=tS&VP`>c_p}XA^ zTs^y^#9!;iZQPOH`G3-?rBjx^^j@#CIVNn+o422CJoVK!3D=4I_(T2w{4A~~TrI5I z!*;E?vy&_SX!O;Mx6-~|yBg1J**d*Tr`xN`ZhhkVonJOzJ-^hxe~oY4?TzQ^?)H5> z^7YSGbMqbfE40FN&gs|fDSUP8m+u+h*4g*+=I#BN)R}woobD9eMY^_26_@r#f8D|t46nRv)|RmaSNKZ~eylSl%e*RWI!g9y>gDoZsBL z_1V_luO&)d+YcySTxehC`{(PkhYt>~b(iVCB4=u`>#fA!h(y~U`}K7j6t^y1Zn$sb z>xv5>uc=L6mb^dmPuR^hmt*Rjo*vmXdCFvMzrH!A>`kvIWk_1gE&5gX`pvcK^hf7n z?5*GC9N+n%Sp3}Yx%QWz-7I_6^t$y~_c`(Y_@X^m3U2-@s@)y?d!u$)cwgy_@+0dFAF6)1|LdwpV|DnQeFdTs;4tx}R<@de_aqXr6z6R^jUBX3xKUww5f9yL)!G zbxr=s+CzU2v2*cz$z|EB`EmEj_g!+o=Lzq%+xxa(N&as2nYurZJkL!&U#mUu(!6Ol zm+Qj*-uYEoyg$DFx%}n-FRB$Q3~d5FEPpircAV6Hn_tt9$G>eZY_473vfeFjedXeB z%Ra4rd;8kL4-4Dh8{OyKum5Mkw}r13KR&+Af8+dlyyt)V{EGQ?^ojT7>krm{cQOUV?`_{gRa~>*$1|LO6q&-Jy|B+lT_vSqpq`evF8!F1}}bwMbjAs zn9gfhCm1)(;Hv)k(fSng0yDG6xgkZ9a@Q=~xZ;DD^A7`~Pj12tLBb62Ey~J=Z@atg zd()cop5ekI-M!yCTRG1E*nOm2yTCPH^+(`8*6V#aa-GM;rZO-va29w(7Beu2se&-0 zXOPMV1_s7?nIRD+5xzcF$@#f@i7EL>sd^Q;1q>iyQ(;w+TacStlBiITo0C^;Rbi_H zR$-M_Yy}e5S5g2gDap1~itr6kaLzAERWQ{v(KAr8<5EyiuqjGOvkG!?gK95INwZbT zC@Cqh($_C9FV`zK*2^zS*Eh7ZwA42+(l;{FElNq#Ew0QfNvzP#D^`XW0yD=YwK%yb zv!En1KTiQej`FVDRCgw&qU=0{j2rF$6S|br!Ee#Bibt1_?b=v5Ie1PO5NN|Bg zgIwI~xNP*n0SgLbJFcWzCm9(S7-TYC{6iQR7??nD!@$6Jh=GAY141(%Vqg|xS7P;f z!NB0a;OXKRQo;D+H1S~jYSooOO8*y>>By-f< z{mJmJulEJ#b0XBsu!uE&X8hyRg3CY@>nhX*E#wc}KbdIjD2vw_FkL?x>Yfql@W~u= z_#eD~vSHo=eEx$O?D(LD!M>Q75RqZI=gD%8{X=__E311;9=I8J~NaurSCFB(A&I3Og z{`v8}=6pp|5;@2I!I+)6SefhcpoXDdz5OdoDZ#V~a_n>FA2XSWij@z3N*~;Kk(PNT>ilPXF5l|S6%D-f-NR1ehXH-<$v(K|BhYF^txy1bbRi))~EIVf32_2+f~o=|5Zih8aZs)&|jQs{a404{I`na_tk!xR`+Xp zeO2(+-|;Qn_B;K>MUKx#apZhImUr^ortW_zz3<_~eIKRwe+;}<_xx{(Tk#$%Y^BY7 z8K(7r8Ta(xw=4R8#sBY>x*z{e&i`{#H@yDW^#}7~Dvn=5@ntzPqx`nNPs`=^?|(Ua z-^;z#pA`4>Kdfb`H&?O`lwq-pWO-M=rdYpzQ=CjT)TfZVcQ2P^PvoNNo!iW3?w`p2 z;P2J%`v3In&i?xL_49T2`=500Z%VD_eYf%1rw)Cg8Fu{^e}DZtUpck@lX_j!|E8C> z?LN;Gz4&Zm`-zFi*WWoBvhM8T^X?V$PxpU~-e3Cr>%@i6mp^y@YtGaE>3ob`&1LiY z&xh@cD)#@(s{X_JdyD_^Uvt*Zm$HsGH?#l$^ZfqrOZ97CCU*1JeDrRZFKhk!{gG4K zw&nlbmVf)-^Pls0U-J1u`r*C@pMI&PWiP_VO>OM}@ zc~fTBH@|moS=Ez@n@dhx%rZ|2*k>C4<)Xggriv)$nk&=zYqI^;?|*Vk?cbw}b&r+T z=RclZ^jY-%WtQNTKeuM579_@BTHYJ`Rkz=7-^=91Ha+2KZ?^k&PtLUwl7IPR{l7m- ztL4~UhPZ!vD}OV?Q2y@2>|T}Q``=^-|LdG-{oODnSUxWK#PnY`o9F+l(Mg|gSDUzW z-5lM@&vWPBJQVCxabv;td7D2MmPAiJal>T!3HO@|(!chet2yqzye#+6q2)E-&qtkd zudPt|bM>pT)mBdq{-FN$yth9i=lZe4S8Q27@u;`m?OXeDzXj|xdw$d)$EGC5=W?E2 zSK&w5-6c=lD+^Dk{(2OgSC_F^{HgHXlp6I;Z;X(i5d=``_$d zR{Z<$J=2N@KN8?_&-;CW{x3BxE?eEZgroD`FddojAt+t+B)5IHn z*ZjMj(I5G*>lD4s_XmD{>3ZL^clpK;x0)a^^ZAKVa(9#8y_ZVn3by;O)NqlqZBbR( zyV#1&nQ}_o%6Qt+pDy#?-*Vzn<-Og@472&jJ#xfj>|R6BOh^rtu5`^ze2py+*i zNHyo1u5sDbsV_~AJ=wJTvAu=D+c)(x?`P#4R?Rw_)tvv?$NvAJ$oZctuh;y%`!%!X z`WcnG`OeQ)m0qsASas`bQeXX<_foHK+SJc2KWoW8|E=}4MIX2PdvNn+rq9~b-xjR2 z`4*9TZPmYf2j2$Adq02h{}=mz*AKdU(G`d7<9;*maLo#ox)T5K{-4wR%x4qcC(Ug% zpK&$!6x-F74L45SS9hBIEsx>)!sd1x%fGQ)Tg(@JdHwoAv#_l0gHttLdHgF^F1G!% z?2OyCE%(itgbVnOZ0!CJ^+Zj@+Rmv{RCIl}isYU*pB^8t`{Mq6f=au8w8Y6oNm1+D zr@q|zs@gwajx)3WM9b>KmbN*W=k@uYp9}vz)xi^8v%yfo>Qk)zs=DXP#h2X*5bx|g{-fXTQ%ZOJ&#UV{E_|$5|5DI< z-y~DLm*s|z6}t@9<{u7!^JVG>)x+!|Z=NJ9yqjA4cb?hX*r1=&>%Lt%dDysa=eLe^ zCC5+mf1VVdbEqr+dgd^Gj{&~?8iRrurD$_0Jxy8G_@y;QEA9jx={>*1lagZcUQ*|E(9@`QvJjUE33QElj3j>&eI0+i!_~&Xs=NzRT{BqI33-|7xW&0?% z?)rhdwb$pa{Ji6H@ZOJG{%3}l+bL~a{rYgU{hL?a?^Exa)-Ew&e{fFi@0%B|*PXYE zuBrI7PV%|j_da)>aOt|L-R_@0zucXcEO$3$yUn-h@wF*0r?*eEs+zz4@NLb^^#c=;bOH!*W{zpQw-o9+KE^Iad`{gIvhM@9Y>p8Z>8=RNh))tA3|?$7>lI$Wjgo@wuLL!;wAB>86^+xKJp z=WmY}-!nC6aQ!Gf{aNzkM7cNjEUI?5f1arG%VdA{gX>nJ$*)cRcSo;zl;bDCotm3+ zCUnp3vcs|Q5+^oq?ftB5^D;b5d|FZNL(lwOH*#__s&CBw_D5~EUGKbSynjEcH~U9k z+MaA_R8;##bMo4K$+?LOJ{ ztMC7~CVpSNmhYc_{e$=$KM&dczh|=d8v6$^b_K@Ium9c2zsv7Xo?t2SqbTz!TMYNU zgA68K>D}vFTlq~rrk?)#KHf2o`K^LO{{E=>^J!zRelB0`hewyi{=WFLb%&Y61y8qq@=x5#_kT@PuReKd_lJk! ze)STUud*z2mYn~MW4*+QWBuk_+utmY-Yao=b>D*-dhI#MYx;GqCM{)8-Q2yfuD#-M zYx}vH`WN!CAMT&N^Wlkl-P7tz)AFUaf83{0S@oH-f0K;l@h1;c&%`$TICoz!OPjrW zV(>LNvx=80>(|6<{j2V|mHXi1&(l&@j?RkP^(8y{=E}*J_u2W{=o#MnvHEUB=hDjM z`ltA7ZoS+yxh%QqQoi3s&58Qgo@}rE6~Jz0w`;;2p1m)x@Gt)T>&S6=o4=wyWnUkr z2P^5{{iu^B-x2U%^ssYUO^L*6>F|n)>gKIeKQ8{iNG0bCXYqz^miQMBqW`VxUcc{; zmfD^b-~Nbe*Sx;$$6vQ;>*fDfrrCY^xropFQ<9LjDHk;`g;>b%7c`^_AC@7Jh1uiIAK*>UxLWOB}K^@nTg*}exJTfDcj!M?Ki z>&xA2>n$7Q#XgmEh`&7UZWm{B>cpK_e=gpO38=eRepaG(&(arFGs}G+Z?L#z{HgrW z?w@^C;+EFd*NwyZ7k5YvpHU*lPz z{Hz>X@TRTL{Xg^Zuj-Sp)!myt>vaECbBT)TsOk@m#x)69JX0U4TyIaWTwmO%6ZiV{ zzqefLQoiY$e)<*n1(YSf-}}Abef8Vg=Q1@L-&g&I4X+1&eO_2k#h zQv1vj_9+*w=yjRN6KCgLaU%8h@pR!^j4Re2X7Ha~x~Hgp+aBNJjH_y%C$^vM@D-0P z{4mx3Xj^!+-O9(S_s@8-XtGY-+lmj~w|*{|yT|O$q_WbGm?Y)q={;pT&6dRHn7w?n ziM^gheUqO3m-f&f7oFu#ncKZ-Wv(;d|87r;iq+Q5E6<3X;3Ze0Fm zQ%Cufca0YnMeP4iEc^3BWlH|3UtO(ssdWlplMZa3(PJyE_W!K;Upt{oXVpJ%m|;Hc zVxrF(^UvP14|m%=-v5~MzINZ%@|*trac@5-*Iqs+p3D*c@5D!sl1GZ|GEJ}EDE|8p z$#d&)iS>Ho?V8$b8t!HtDpUoW;S-`;TSD^a$$Q zJkYNne^PY0S*hWNUi;V9_RC{$&agd}|3jd*;_c#z+dk-~S#7--*dwR=LG`qm{G-$I zmUVBBZ9E=p_2SC%zVNwvN3NOJU8^+vpyu!9#cuiWLi4P}daF;}D(^dUqjT-XhS^5a zCDUhrPb@nwr9bC}>yww)=IOtxc+t(DU)*|T?vch96HYu{IoJQz%hkPcshP>rGjmea z<#tVd|Et?La@W+Yi{EHwpI@JIZ2f#5x!Rn?HLB(34wk={`Q!h8qy3M2=YLn)=hqiL znOmqP&Hu9dnCv=p&ihjSScFqh(4n?|7XW{*L$Xq+@72) zVgI{jZ#G-Q@#snOCzU?e`>^buX~Dkw!v^vXm?yt`aaOYQ^ViG2);0RSmhpRK-LEe9 zS(j(+>bcg&>_;xo`K~9q>@UZE&;8jwrZqpW#-;Ubv)`hh;hg07D9iTI%Duajie>co zCMu+9M&ImI|F^r&>ZNy?p^WS^oTW=`>k$=2-u zm*4;Yx@^tEdvk6y);oh6xF_=kXV&chcK+YA_GsOGzgu#fZuJD#`hHT|lyLHO`GGIj zXNftqY(Lq5-pys*;mFx7Vf$N?KHW3ZmoMAm`sAj7aO57L_$7Q1WFRlA@a^lnxk9e)OvNFQ_01yl3R3nPehe)7EAw;e z?q4_3y=U|16eUlN*>!FI#*_SICrTYQ?`)Uk(cAmuaKpJ3X1{j-(R$9j_e;8R?U#4`#FHfWYs|lNyshrB)XK{b7w67+m>#|^-g=e2^wAUbsZ~FYudCelIjv-ad)e`j z|1bQe^=vZxI8*)S3O~QRn2#%duYU5#tnSVC|7r`gZC~5xw5#_Vak1VU&H9P;9RsGlQgL0A5KcC80Cn@_c+v#L}ujazK zGjabuOqP3FarN}W;^Q3sRa5?#TrEF$u$b@NuMO|d?#bsjU!FE;&n3$r$Nyin-ynaW zmgRf+1G}c#ho2>V-fdY49v1q^QKDwa{^s(({dWIP)qmY@JLCS7x_5Djd+(ojOJ|XP za*pTu%&*z=U-aym?>%?@Tt}I^*0c9MQTmi({r#g?(!@9RC7U*;KfZC~HJ@{R$vxBS zv6UN&)@Q8|yM9oU=l$=W^L`fJpYVR};Tv&}FV3jAZc=wFZudrmyppVH;rFw?NcmS? zec6|F`JQ~k2j}X~J~Bf0Czb9Gy0*>aukx0nR~xz}W$QhXZe2ZJ_S=@1%Z+1RywDY@ z`?KSAuHN^)W%{SqS?w&j_D?|gmdS8o1o*57S)dnWQfh!y+4^Z&c}=J)Y;w~Opd`RMcX;p(z&FU=EQe!adSC*lPE{&n8x1b^cvGs(523i%=eI-`{Jb$k?YN!LWTV}GLXS01`8iQ8_vB&q zH8)a&ZTI?@zI1;$G5*#R^Lrch&t0`PystU`#YuM~{_4ltvugyo|2-3)-Tm~#Yv&Iy z4$ICg`P#vG`O&1w52b_nxp!W%GTi^jTR-yOQujRF9bY!&`}#fK+YIlWA^rfz;D-kpWJvnKjzM_sb4kI|J&I~zxboUW;yfC%E^lt?+?g| z-G1|N|EK0XMZW|0Z@H}2Das$Tzu&O-O;&hY@N3J;O%*RP=N)qSeX8gDuMZiE5ASN? z_POaDH&6cMQswzyjyL=Je0@Ce#fZcJI3E z%i%snn>?!R<`sXK)BJ44&!l^M&shF&|KC{ueDD1KpX>jr{jUl5FJ=)NRGhDV_h+Gi z1&56|X;ufmi#U@!b2@ z$!hyP*4e*YIklSq@0*L&_e{@!dwBH8Az8n7VYd?YNci1m-L<*3~MTKj(aXSb^R zSC}uG%bapi{qMS))~AlBJr@rW`+Tqb-htg|?;DHlYH#d5R@1^SAJuHYJ^lIK>=QF? zcf>@L$!+tCdm4NA`c=zpzkjRVOWDnGeiX3pKYxn-o7l;Juf6NL?DlKwd#S4G`})V4 zPrlxh{VU?+G8X@0#+Bx3MK@+>-#Yx#cTe`4(r3*(6Xbd(e!p9IBb#^QZ2z@axmT_K zwfmUdf%1+Si^&uv+GJ?u=8&C6xRT$8uIj7`?-embZ5{IM;C zx$|mf?SFsvz}`!iKf>#;ufOs4lvHM&dHa2re{OaS%=eC8eE(biErduo>suYb5X z^5&kCFQ4<<@@duFl5H_lul@4mxmC%}BWe34WGZ{Pmsb9m@H~3sir3Xo7H{?ql#VRR zkvQRct=%B`h}!=X$Jvc{{qFE+vo8^Gu2XGym*4;5hkj{D75i-WbUS+kwrJl)SyOKeZgg89Dx-m;e7x>@vM#^eu2Zp)mm zul@1#$wa$fmlpHydw9g{Q~8{epR)PCX&&Zpm6iP5FCX;Z@`0?gO-HZc4#N{Km%car z7e2Sv=*Q6$*DG#Z=wp}6U*b3a$D*&9U!Ke^d-~<{)rm%Xzi67Xf4%>=V}4IrX~YMA zOWP+u?;YmPe=H*ZXKJsvNZzea{ND<)^e(W@ub4VpyW+utY2SW5SQI-~&$7~NLX7>b z+T=e)8AB4i5FscgZI4LwOw_!zf;B1uWt{j zReu!E-jVVDk@&tHMe~3Ed2V+7+2XmD3G$)l+hgq&YdHl^7Cp{j%@a2_kI;Wv(2i@`+I|GSLeUkpt;ohf8Wa= zE^WFl^(tXiCm%Y#477XxVDnDS?qgNY1#|8_EdJ2{XVLxN_qNyn-d=x{z2NiGoqUs> zL4)<|$_e`0_V3cJ|JWU0^Zxz6yZe9N+y62BfB)U<@g?@#@0Y~wX1~#X?@FEB?7#E4 zzg&EM=E3G&DQm6yYu?Oidt0vdsPz1cx+`}Z>(5tTw`e}bhAFZ zX`ELh@OYaq&!hh5i@$x|JDZP9|JS;gJ?+uoJB}6^G%?%9eT}_*+`aIEDx2ZE9cLme zw?8~kmVB}-qV-^H&&fxg&&8e{e>Kl;&t35wNBAdyS-m@XP47cfb%E&FKP+?aioV{= zcjUr(08Bn$}NAT;;#(x8VO~EDvvX%2Z(h0MbMrrr zbzTcr;-R51yVFAhi&sgs4yY>7* z{J(qg`|8gm*GXJ{cFgKGTiu?E{15JbOs~8AxZz>+Ikt6Gv$uG2Zusgcwz6NkDd$t( zZylM*cheTmJNM+*9=&b8{da+nylna2 zq|ej!zPx&zZ1!T(4{i69sV@)fua8~l$IjDh`d81TM9n+?-xW8r>RTTsEM0wiahayj z{cFqD?l>gozo+DmzQdb0OaEoL^VrnI;YR*PQ}Z6QYyLccE}r`r zukrq`*_*Fjite+I+;>4yb8_U)i+Af)9rA7;FJo7<-||vBuk`QDxc!L+yJj2zkQUB8 z6_nceEWPaWS(!uGdtP4I_a|xZpTKzgWxY1`mp5;^Zj+XO{C$`G-HJtDR$uoEKf-65 z^J03mXSHK71eyjhzrTrq0x0_VmHox-n zbiU=?t1pGxEuRXmx~H1?{N)ETv$e*bN)GKQ_3iU}d+UWl{<^Dm`+xjCU$iyfM!c<8 z+Va>PRJ{R--`%+#1M*dg1y!-)sd;X38m}LC?uOIc>^ZBttfxr(g zmNz%QrM+*E|MyV7?*02ebN7F~m;d|o|LfXspP!4|HE;X;-G?pB?ascRb#ARCkGT48 zkMtRArPcap=IzU#ll<*jf7SH^2j@O`S?6cI+rZ^bnf=Y*pSOHGZ|inX?z-i}m^$fu zrhi1AGr#W4e*ZmOI_Jww?T=Fq{|&uont1A+2HX2b66{g5a zsPNdyy?jaTY-7#6-z9Eb$@6*tani~8Nzb|O%3t5T%$&RajBmOBzE9rLzq|j2UR$Qm zbF`rTdQ117a*?_nr)~yJ)9Q|n-_(8f#QUn*KYO?T{@SPdFY~c7$ zL;9WN2+;@9derKxW&fe^l56WN7f39n+1x4E7P5U+k zKJT`Odm8I3Hq9X9-VZ$;d)@c5j;7QIJ!apVeTK96yYIg2bCwn79AB@lU1*xo7pebj zj%6nQ?)S#uF6}p}XM8um3Vyhm@A26TraaFHtLv=vUXSr0fTYE+P9LzZX z1~uMv5vGP zcdZsr|N1Lh`sC@aQ>A7<>-eWJ?^S`gxc$2$`}M3Ic3zM9biQ)~qyIkBbuX{E#qBrz zwm`G4;_Ch?eV^;!jvVNJy7$({1tsNzZ?-?^es4Oh;>4ztZB?I+x&D+>kG~RYX4ZGH zxqVKV;i~7x^LFReEe+OtsqVjg*QO)-DLyYRFZY$-`{rozfAbGtJOwLjF2#!59<9i~ z?pD4}#QWn#-)TH@n$;B(3(F5W{+P7#oOo`M`>UfK=5tGW7QWs8d%3^uBYoSF7n|FD zuBz34_r`yB-LBb-pCsD#*?fESD35=McKpZZ?6xaP7yE?y{0Q8alUny;@}tXg^*dHx z*1cH!T-3kjCO`YOOIM@ppFglp?U-9})>9VZl$zS>9Z%(hgz9GAR*Z%Bpeg|n+=T-2# zb5T4+e}C+apZ9%T;l9_eV;B3)pYhY#-Ei;Q_p>;2>I)UN$)4_fn|jamiT=UkcbEKR z-MsYV?)3fnIa}h3J^8|mC&fJ8`ni0{4D*+lm&eRK_qG1kv}N^H;o-%*D&4m|;cM;B z`FmcZ=4Wld-HONF@o#sxrTIm;|N7bT-Sxidqs`aO9_O#RKI3pnd*__}DcRE(Cvm?I zue`n`tt&9+(X-`04$anmTYX;kcaE9f+VeHfL;)N5w{sxm;KFUKkhlV zd)eQYiHG+U{yaYI+ndt<>v3;t3@-^y!VsYs?;5I9pdL|=cFBv zkImRVG28vgO%e7aYx~~k*N)3oW$m8v<)QZ4bGMIrSU)OfUs$y7;c?~V!9nkfu0Gw> zTW%#Ead~$9qXWjgx5d(49$h`(u69F&RNdD<@%GDkGf(H)l-K4g{x$DdWB7a(IlFJu zcRtgUx6zmt|Ksv`{^Z|Zm=4}MW@%}2{r$x3-V3w){=8RuaPf*&p0D)pe_c&^W!5n# zyEjU7O^!e1aZXNX{)}n$Ul#B^)Jv28{$oa**@xB3-RH`GIqH7>nDMrUp*_D|x}2&| zvH7!x_y4*7o!4a+*dIC{w#1Kb;^NgGFD#q$x8}(DUY-9BSDk6ksdR|?eN1M5_l}PX z;(6Y3EAL3&ZPB;x)1!%(btf)ez3uFAx8J+|{aMx1USE`)@3-h^VToR6d}Yz^z>nS6 z|2!`L&1Ng~*LuF%xepu9ef>CFaU%z1sN+xgb{y%B|>Gyf|53!2R zk2^lND!76F_T|6fh5wlDuz$G!PyFAnclYDZ{=HN8o~Lm6gz~hV-*-%YyVLG%wVj3c zvz&^#<-C7OdN$g+zpTAj&Ko~{j{oF2y*0}YTdF;LdG76=!_RYc&Ay)7^LF>Sl&ZV& zTT1Iq^h%_yKk0v7*BDqe&(ik$g&+X@1;!|$7+W+z4u&R${ zXY48>yTEqo*+sSMY8%=gR9q%^TqZZV%zUL>wG8O|844AS692eu^nGFDyvo9?6R73@<(^Z-$!S+^#?p!qP;ie zw#~~^E-{+hXqa-Ho@pGXIzlf35U+({;50!tsB8Y+C3$m&f{3Z|bd8j}7e;^v$O5KXlx0 zwCJ#-?>$zB+S=W}j;``w^J%K2^qWVnU-io_+wuMu47H7|==Qv`Hh`2Vhm zF#FcYyg%Z_j?<48H_tKp$C1CpcF)Pz%HsQX75(O!_vVB6ME56KM1wzGIM`k><>QwB z2O?~1zloWxt>5-y`iZ;3k}dALcebA}JO7#8|82$5=64dQ%<}&@?Dze-;BIgC>wRs- z*2Q01*mqj*|93aK_j2pPpB7y6XUfGMn^yBrYd(0$>CWZi4fV`_{;>Yo{Is}EjwOzj zv3kqjYwRE7e=U^1#sA>`XYv2nwBJ9!rd4FNW&ZBt%JP}F&fi`A=I^(Pla-Ea>oZJj~rsRy6K;nd#3X7n}znEx%?xK@5Jcn+x&>FIdxZiOYN22Y;5nA zz5Jtc&(u2Z`kV*Jw%Oa|Z#D%-2|R1B3(=_&Vz1kHp>X<9cXrFv zKZ5r>cF*`QDZlsV`gJ+^XWYtube~YR+On~8d&QKW0{uqw>I_r#RxW@4Z(ZZ$p9k1> zRx36CSbTSe@;|NR!F|ii_Usjg#1}j7f0pfb4=%df+u2=S_*ZYfe8o-v-PZP%#r3rk zl~;C!K3l%=uEjaa9Tr=^#eexNSMzXB1mCf@(Y@Al?oTcLz`pF z!mo%d2OU1r=-D6<;QYp zri{az-zsN6ocvu-a-#3N_zs`?NQZUK7as2TW7hgGRan)3`zFqn=i3hT-A~dwym-cw z3bFo={_~9{Ew_$&v2^i|T$8CEH=W&YcK_F=$!la@sym1EBp#1{JYE0COAWWmkol)O zTYab3{Suh_>S#=~*;mhA&iMg$w^Sa?(SH(Pc~fKVt0z}a9u!S&=Gp(@hidoEN%cQ| zCS1POZ*Kc4R%dIg_}yn~X57B=;-ItJ?v1aKO1Gb%wsY&RG@Tz_ZkN+`zrchXJFr?(B;#4UvtgN`MCSE$i0#~Z}jv&?5&qO^=qW{-~LRK zJH$Bgvfr}0+{bs$G3Ng15TAeJZ)(k^Yf1Z3<7Lir=Jqz)takmrp zljisQ)L54!lK)<i=Gs$$zR}wtM#eob=5ctFO;!e|Biy-jmVadmHE2{kph%U-qZj^^YFbe@+sOx1W}O>g@C0 z_p|;-obXP{=KIudx98W7nsW!!>rJ+07ktRfy&1mc+3-c|owd*0?vPX^x% z`HbhMr}P^B`uF~qYW{ilg8wXB6Le)3Si3N;<$1$)f~{wjw7``En$jBo`Qj5UI!^Bu zs&Bq^h|8c>H_7H!?uM-&nf+XEL*s5P$vm##a?eRqy zFEZ|G|90Zb!udM?KPZ(R(fez6CnH#X+0hfn(~SNF@BNVwyywT^4V$VgCY-e2_F;uI zyN#~I#+9eLdH21lxas};+2PaXpA7z;`P2NZV&XqNrIFPmN)Ej@c|{@XuS zlO-%eF#S_Sw?j&YWFex&znw z_NBjGyt`7#?#i)kWy|N~?elsv!6@kXG3JEr+;~v*nYpp`IS&yvFrSeKJ^lx zC-WWAchM8tzwuE23+G!)iuvj)brN~3ceMPsWS@U_iyqq817#!uajYklV!|4V_Ep0;oW_P@?~G#wa;ym%bLr0?M3&#>^t|r z$IRJ!cjuR7_f3-%&hOj)H8wfE`66hR(f#|q-xjympKYIBdws{_uOIU5m)DzpTz}*I z?a#Z*XIwm0?Jx6h_Z+)&)w-Gc6Vvl-Q`(OIytH>t&%Sp7ds%;6t#moxzBl{Cg!J_5 zXSw%ef2fLobJIT6?9Ra(Px);tek@pT{I2<)>9N*1OYN7vQaKaQ|XuKnK|_y3!%toweRrE#vRJI}slr z&S{>1XZNz5|F>50{rh?UTlf8W(7|0jA?}c(iCYqGy{JAD>A`mG(DdWGxqnA}kZlrE zkUQx#W%BXckJjcSKHk{ydx4|fqTqPxt}?3^D&FgzC)s)K<`tWGG}>A=_nF?l+8>?n z?CK}NGlb-Q*cY&b2J6n#^*BY*a?UYXhr$1WfAJ(eo_wAo0`>_6;Ow8v|35rza3EwabGMG%pEoNP7p!@|&Ut^?u6ehvxrf`maV@*G&F}k- zFh6xYJZr=Z*?u%C3A35;o;NbMYpO2GL6l9ca|L~mp^*N)V zsy$SmF83$+rq%wh(e@g9H(c&}a@u&;OZDAv#kZB;{&ovC&8Te&c z;+PqK@BRDhALF0$KM(%@-NC%?P2Iaa1^-I=PR->#5vf>utt_o%-iE7^ZO3PR_S~EO z_muZ;?C&`}yW+aexg_b?j~r{-mYUVA68DejU^eADb*zi$&!kM5%bRVc z`{f!eE56QQua|Qpow@8+b^QzXFIulB*I(HEEKmOaFWa)XFR{XGIbGfR%WvPV`7 z|2Ns2eS|CD;_mmzIYN-llR#!e_)in?{gZ%MY-1PAFt?3+84ie zy=VGd=Fh43)yH1UX_l|!`^W$Pn*G;rH4NXyADno!AwkDyC2R2Sx;1-meT?oA3;WL* z#O)u={d?VV->EU};dKJFw z)|%=4|F0`L?c4FIz`2dr&UP#7;$JT|-Bd{3_SxI=|D=P@N5?F>lut4N_CjR-6f!$jD-q3{@or}A*}p#d&Xe7C%~v6|8`ILi@9c0s ze&^AWX8kwX(=Sc$O(&soB*$Kh^XvHml3Lbo}+wZ+=z8hsc@hE4pf* zUw*O5UT)XKs=|rE`kDETtD-_#rNX$!xHhozbz<#8~fPrsoc|bFY+|qZY=7*b$)kxP1LvI=g01~ zuK%2LzFE>fojJ&TZer16&9vG_<@YR;Rcg6Y$rMG;uS!wOo{qJ@ki?}>>{Uh<^ z+m5}dd@p4uu3WcA|I@mI&z-f6++P@|y-5v@e;QtS{Xo0^2cGI}`)0kDDvZuA+V{bC zCrkMfK4F`+mT!|Tg-i1Hm|a`#|G)D3nYpza96#jc9!z_F=*?{F=V@mTzq9+E(D&hC zrS!h{rtY^M|J2j_vbBQm-+}V~m$pA={;~dd@%~cwRY&t5Ou5Y?edaDt&L!g=-|f2Z zvJ^B-H0GT0af{}AwZfB@I#zFdYK&y9)NFpgoW4E7qVj&hjj!)ho@>hr$sIZWcl*NQ z^VJr|*zf-GE`Q7N`v(`Fle4tZb=eqsq-68T{=2WLnW ztds3ZKh?Zm9Cjn)u=_lp$BXy>=nk$gd>pJ+|J`+NeUgve{x27^ZH_wfJ>{+qOqN?# z@+5k?WUj`~ivEZ3+>uz&B<~#Q5 z%Vg)?#6M=K5gQfnReaRG&w2hY)4}^S2e*GaHh=3;x!u=yymBt_imiBXKK`%J+$UeZ zE1sM2ynBY=-VYN$*XQ=%JGQiN>1w;8oBjULF8+UBpMUXQh^_vwY{9Rs3$I$17Cvx1 zwtMG-`S!p5Of{JE@lxxS%c>JM&g+?6SJT0ld+b4G&XXU1OLE?QJM0i`cWHKP#RgY@ ziI@E+{EwfDwt0DBot~}k_G4Yfcm01(UK||xha=wbL-76-yCuqUmX@DB2r^4r%6=8E z|MX_U!PWZrs%P7U>^G{kx%H_%|INZ*Qo?_Jz2CUi{-gMQ7dzXZHodDa&uf@&Kk4Uz zy8oV9wJA4TZXR0fQ~qn~#lXM!wpU;Map+!>Rn2by-kNHqe{ylw;qB9NlYN&NaSnLBI`;NG@paa|zdmLhZ@+Q>(}@YUcmJJQuK!{# z)8}`~zxEba%P;lb|Lw1G|HY3+@1+VqoQ}A3Fy3;0ze6^>R`f-x<4v{GTR!vwM0i9vIup6`K8fsJ8x&)bA~Ovd^siw)l@p zSQ1C|kLUHdMO%tuiQE*Z4{nz`lf6VdAK5z4=|A?M+{Mque7Uky-N(w)I z`^QY~m`C0nspZe^F1CC9B-V=k{wn?l`=9Oq8(v<(6L4~oX@*&P%EMR2E5s6K*K^0q zN&fNOtg*HHg{HaY3nRk-?ieF0J>;-zEF3Wc6-=$2&hS^WVSe zU&>3ZX*mPfv<=wulT>R*r(~7gK%KvX1o-OzHmWs`-z)HJ~ ztv{y3R~&gME*ksn+xiRFw!MvKV1S$ju-sRE|NH4~pAy~L>nwMFNl$M1KH-+cddqvR z6Z>ke`m@h|V;R$R&}xDI-XAAtD!*DQoV~o%W&1ZX2dk4MmIu7&o0SR9+i=}2&PQ)& z(XMhGH@^8x_L@4(oBwj5e1Fxi&!X3EJ+3|F&VN@!(nk9GQvbcpb#MK2RumrWR=@kD zQ+xIww_`E!pXaRGy|(0pV4RJuNZZec+1qZ2{kQeo^kmED#@d>rJFV95GP6naYm@zT z{a>!$L+}1NLn-y08y`Ertx&!4SgrOMm-WLo_qp?*EPXb;tmxK*_7_*fcK-M=@$Z+X z?fY^jU7h#$(^OOWWBgVhKQzy=bKw_SZ6p`sXI|SoGveg4_51!@ImRb_>*~i%-;+0= zzs)}78~Udoi00>XB0k51Cvbn7=q z*ZlcE!|Kz`D?h#ruM>Vh>yNkjn;VbqCrO7(xFxRCuRo!gz1TbERZ{Qr7<-@13;Vi$ zmwqd*y5aa!#qG%ZKZlghO|QH@!MA+#uiyDxP|AD=sFH?#k9 z?B?WOg72mFbvXOz|6Vtf;Tzw*Jv{Z6F$rb+Z+FI4$;tOjitIUK{p;e1FY69o{*&o5 ztNNvw_h)};nF-rd>?R6+|D6;2FVp$Un;O}k31y$A9(C;h@xJht`f^$R-#=cg21Vbs zS(U=mlHz`E-#NwhZH=I^4cOG=$S|HqPdcW-U}z78~vX=B*E^<%^w zwP({Sug_Ti%=K|@->u*8ix1uK|&#+z&U6SgpR$SgXYxZ&Hfy}W*X z?Xpi#r1foio4;oH9`?hjvwfHkh1I2Pw)^zUTINRhHqKIy7uWUN|17!B=CAJ+ZUob+7YiE*6!_bumK zA8{yLcD<{kx@?IkcbwIapqVbP%KO^F2kYZMY+78KJgY4CW%j)%&O3f@ z-*CBezU-%rdSmNKqg%U9WE_sSxovKC_O;mmf7Z7PX0(*o&XaiA-#(|TVrh4Nue#5N z*VEs9RCQkW$>7b?hQ4Dnd2MPIXzTo86P(}P`R%vG=Il2ob_k?b8_LBN72l4Z^lkdi zHyd{D^EszxSGBugWBVufIGiudv)u@iSwRKJgxqS`BMLj-Jhk?H(oeA ze>d;v)s_?g)rfgl&C;$AymRDi*}nr`xBvOqPk${L*LCai)cQXP@qd0zRh?4%@6K^4 z`QMYuJS}%7r5NA*^&#_du|>Q7?8GTL_dMh+tpxsm**)bVfAYr{*W1JQ7ge3L{1WpgW^oIJSm^XZ?H{c=y8Om*(lmrP#5tAF?F-SYvtuWA(z z>DS)+_W9`RuKxNBdy?1s-Z}bd(|f63yI1c!StgW5@p=SNm^4@Co*?)@NoBsZdWoF*j z!fIEZP|W|4Ti5UMwtw-4nflAV=#+)e`SpHd_;s~+N$=&pTfLw4=Jd}wyp?v+%iq-9 z*qwH~U{0&3PTbd6<6~mcGZwu0{Jb{Vw&p|L-q*o?eETh`X1`d^Kfg}mTHKo)vHDFF z%C%2kIQ+R>Uw?gu{j-I3dwxy5e0z7B{jU#p=gap^oP5UcYsm>g*`RH9zVD@yE&29; zeJ@pf`&Yxpk6FB@E?Tn3|NVXM^SOijzwF)r=j~7RfI85ouKwl&i{30KONh9VJ5M)6 z_{GE=ufvP)C1oZXZYxK_lByI zbN?-!Jx|a5OqTlYw};z5&6TNsc~Cp_`0~6pf^qxa%<$P;ae3q4W7#KcPX0IW*p6GQ+~2Vao>g0 z{dyN3xjbWx)>!lLxv}LE-lMC}ZCi3?+MO!p$@Uv|oL>3*^TfG&Umx*UzZJ6EHRXNP z%VckBNjsal=b!P&{r#e{m~XL${nIZml~=!6bjHs1r*B^Jth(O?MXwKUx%;*)RzlTV zK8cyX_ST>CEmM9v=v$WBOtLz(I>%1Jz4-mP#X9HY*1g%ncVb@ghnshHCd;^8WB7CC z*aqXcxX30?edre*T=0YXs>(?(?{&26m_}*6jz0XUC(x{;cI+rQbV0f2vz{`}EBOW1Sx#GjE^oz5d8zhv@sh z!w<~Q+26BS|7%?%v#tK8b%y3;XV{7@Cf@yZ!oK$Uj_n6#|Nk_XU;N%PmDmFH|3>d; zJ>!_G|7qP#e0IvG23pujWs@}vwzpOq?W6HV=vFRf8R8x@4_u1_3ER(FBiYvBrR6) zsk3%l;O~>4@0r%LE6ZN$-k1Ht#UM?5;(Pw~Pc!Xb*tuOjX%?6A@y>nuTT%Hp1Fruo z+>?DH{kX*4`{Hk&Zr-uy?IP>4l=ttY>O(G98854tSXA~RtnIMKz4rJUt%8vs?{Cc6 zvvu#O-N!7x-EG#}_T*8`-2eGs z-T>))bNv_B|2*Dic4XSVFBd%9>vDyBeuU26czC+sPP1gc`-}5+ZGKO*+4pV6vxdD5 zyuS`N@frPe=8l&$ntuM)C0}-*!`{0iw$=U+sI)q{G5k%%(pNvOyubhb#l0E&w%;x7 zPTf0U`J?&%=hOGr@jsBwW5|BQz@n+)N2SLyDZ4+IlTivd?+iIRG=1~(XbRz9`JNv#5 zvt{I8T+r7{Pd=XWs%FXL`?@_FmT3R`b8jM}drW@B*gtJ$YVPq+X3Jx438Zl?K$jM_Brx?e{;!+L&+^^UO%P zBu)I?lk4dldt~IVJYMU&ZmpHvrXLe`?JLQAY%UWcb^W;avxC5h%Fp!v@}fHa|CY|^ z*>#2&50{r#eB@5k+p&9|TG_6vXH6eFtJz<27O(sCJh0wO_6+n=5Da)$>%f`P@4)7nbGO zoS3zMcZ z-ar1x%-M2LaM6qhr^W7Wxb*J8(aVf3Vf&Bx?R%b_`*Zik7>iq5a{W5~y*M{J|5umi z{wcn1f5eDvk&Z6=`QSjRyUq`({9hj)xIaS%b>~FZIym-8Lj*iZY z@N@r;8UC?z`owz8Uw_({ncPoa2swY-`f_)?%7@wBHHPV5xBH1z8}{kvbm`AE2)gEO zZ~5<2;M{#l3+rWW{rJ&)RWaXswMtCWe~zGR)2=ik{|`b%)$7Y*ec=~1HUq{>5 zURs?0ozW6=%4o z`0o3txckf1^Ut4Ml(qg_lIlIP{?!jD<&PUS=PnM9la}0eW9p9!J1#o&@z<$@`%nJ` z?jNmN@cT|s{=JOpA7D-Y?#$PRv!9#Kt@xn)Y0|c$8Tt1=Jh(iM|Kx+`>_KiX56!ke`S9uW z(|fYdGzae5wd?b45};WL5Xe=6k01P8#@! zzwbKg_~x;4?U$U&lAljmW*%!>`-f+CQjEd#<9}nByZ^+^IbQy2>SllQUn|Oa_Z;z5 zzxn1eYrpx|bqn{KJ6n97xOe$`scqt$vw7z4%YJc5-gN#;F83b`Kh#ex_ciK%o3Q4b zSpC`QH4#0>oSIKT`cb_xNV-hQ;Ms*X9y8Hj<+_rD&%k$x1b1u5})d$Vio%iO$ zW@lylCr@rI$<5lD%kwyWU(Vk{kKZ}(y5UsgRFfbqy11z7r|7(w6TCHIR!^QRU|II% ztNqoJZFX<%*fs|KdGf%|FM8iY-}!TR{~82}WpDd(d^5ZD(!00}09K@3iUxb^nn~5L3Vr}yE^4$Bce}6b-|9$8A-23bLKW^-nx}7v>_VSl` zF(+SLUdIz%Xv%+hLhf{*ZCk&F^QH*$Om}0qtq`cK_z{~^QWAMl%`W%Ns>z|x^kZwz z=GUI$n}6K-(V-OH_hxm50?8&{}vKjZdBJI{&8`{X3q4t zu_cE%(i|B+YPd7j`X}(rfv6k;!fvvc5nY2>_749@MOE< zp4!}(qVD2%eq5Tl@7M$P{a=<|tppuIBhj#&McK0H{<(>g`mfK_wCptG{2eQ89QR!2 z&nK(4&dQs1T-r0{?Ps{3Sqhp6;cnd9e`a3Ee};dz|NZ{|Il2D&4d&>}*EavX7?;uS zQoSy=dfC5ob8Y$4`6U{kZkFHDe@lDI{)Ov3)6>>}4BWnad|vrpYR^%2?qoLZO{WTuB+IsR+P!waD{-N>zb1Qnti*de zqaEMYedOlJ%`X39(*GidPt*IwK~4ByXWndE!z!m zRhIP%?~J|v`rzjO-)C%G=q~@r_`cxHt+qBVtD`~Fl{MSgB~Smpw~5)>zGwHx-dp-T zn@hW9ojC6Hf?*h{^vY9 zOX*+sFJx*~|6}?S{&(g54UIQccsv@Ilp5ySr7?2|G_ZIvuhhxRzrbv9_ts1E7ia8=LjtJ-$6)FW)5 z)sgml`FQ0e1@C{oSo!~X!iTo2wQoKxZF)oYHMpvwsW!Tznq1>|FU2+0xm0 zKUO|xuQS`#lGfk*B+=h|%^B6g&%*q+HxKb!TOC@r?@v^RcpnHPoiYCc5Q zZ?a~)e7x`5v%XDJ`mELd99iFgw`R%Y|0N1H9wwTfd2ytn_HFnP*Z2o#eDOXputWw->+JKfM3-Z~w=}_p@qm>|Ol3^7mtzi1@;^o62(?J?ixL zpDgEv#75o?UhQi`cVdyN#RwmFhj`SPH#-Mur< zz3Vx;`Yq^mzt+;@)4yKdBWZKx{O)HrW$qNtvgDADkX--!#@?0Xymw}w&Z)igUh4A| zs|4}fn()7s?`IYB*&XKadsuqz;B|ZU`Ukt~U*7&K{s4TMNbC}`#@}fdS~f(MPu%Dk zalzK=#v9|efX~~u8`L}MkpSq>Ise}LIs>jdk zcdh>Xkvsm!qaWXRe@ov_`u{Eee_2ws!{%H^n-n+kyCtO?Ulgb=?(b>l`MEk#oVQTx z{^OqQ6^BB6gznv4;Kna~G^(zD<7uA#DjN?x>@&75d8O>lKl|pRe4A5W9+asI+h&bFFFW_|czOL=g~a3Kzph^VWoY#z z+}*5`y#M_8x;y9a>3G(+@t^(o|M=orqnEkdeXr46|Ky%_8@nqup?s(77GBm5JJiwR zV)y@K|HJg+c7OB#@5){*5BGWVMoRGWhtI;5ACzovsY;{=%f%(}YTMj;`lsvF^ZhM9 zyv6LTe|%gzJ^t6$_b~^`KZ@+Hb5Xm$Wn=%8t?oy~`?fyTulsZLXT`aL>(5zM{8OKC z_S|H7&`P@MxqmTGE zF0nl~A1lY+{hPJ?b*+_YPuySqU+X;cHaY_3fmO(fifx ziY8T+y$m*A_bH`EHfNT0QX=!k#%sq;oSgrConhJ0S@&Q6R`oakUnYCE5ZMu5|ZKghKXTR5b^7Xvt z{Hy<2K3ng9Z*D%_j6JB~L~Y3Cg{B|3C|U~mZgXHw5RTm6-+g$;U1p z+hnb9oxxOrH{@&1$=f$>9yva*fAekzH9pJzA8+YAH51tNLp`jBw_@Mt(3{7-KILjX zT6X-{8ejXLwi<2qpJ(TleNq1QqG#1<)hmzGou}=;^vQ>B^P`M8S6+%v_jQw<_}Kc@ z2gTr?+lKZp;_gXKH`7dR(cv@t;Q4;r=j>a#S87zx-}jK6SQ%gQm|yy+_VL61?dGzo z657YR-RJIoDtr6akHYQj%B97J<9(ig-^Qzd>iLI0^*z%v>oUyepRn=~+<#;059a71 z&<?6D$837do0h|mO#a@IUR{&2 zJ7UVmEqt*v1n$-Jn9qJ?yXA~!%};l}wY?Xk|Nk<6yW!?P?S0q$cb)Wn8JSkJn0qO^ z{UrVOt+UN`yph^v@XD!u`kt0{*_|nTOJ}eBJ?nmiUeC+?+D~(R_RWl5;Vrhez;ynL z|A+ejoakHo?V9Tje+B7otJ;4_t0Z2gwDZjA&A0l|mj8d%Q5o6D&kDC|5~baeqIQ0q zJ?~H4eEpm+mzUemekeBIGS0qQd4th8}i+Io{uY|1$sh{^#iauXefmyXK!?w*Ptjb4}HAo#!~x8$bKMyLaK|sei}6 zu1h?B?{ND$ZNu_!Cqa9!W3@DNDCfPKWbp5vX$@cWCiySx9_}_5vWb5Z>-?VKYs7Kued2k?n$|9VKdYwx@_VU$ zGt?i>uPrs}y$c>Lkb7gb&Hs$ecD;T7ch{W+RobA_6ejQdd*e>7i!8%!L;D;f->Sc2 zjZXx&3jY1T|M$YCZJJv$bOh=HJ?1wjh`%m4rMk&%N`o@HPi@XEh9_nr>}gdmA0My1 ze5EaX-W)4Qm*u@0cHDE$z54azvYN-b!>u{uKC_wRZz??2S3B{uVane>m*j5l=<<)Y z@s+Dn*p{Wu_jQt)&Bs%JOLv<6QmtF!?pMC;FmHdaESs7Cgm-qUuVtK?ig#|<<=Wi;|E~JY z&TF%Z&m5kWynear`HHIiy|Q&{zkW78F6A999{!sB5}R`LshW!E*X?Ak$3Lyfjoa~} zeYuQT+tJ1kFE8kqm2Ny7GVerIe(kr!2)m2VvfgA~-=6qj_MEz|^K$clwmQ%JzrtBz z_gU`TH*LqQ_dI!V{r;a<=U;s}apnD|Uh(<5j~(hoh3lVOF*pB}k+fPqG0i{rr^=_q ztLL^qzsLT*_JT#Wt(lpW{4xLJ+Yu{sWbVozIDfNr`g@)|I?UIyf1mxy_~-fl-}P%Z zZjU|x{mqZ2uXpxY&pz}k>;1<0H67KyzZ2en)A>DNE6c~5kqvXcE-jX@%zgEIZ`${@ zH*dd~_kC92J<}e)|6gBSuUubvZ|d!%)*t8n)_?PBo9R~5aK0%=|0upU?d7-3=RN=Z zCjYH(c1HQB$;NX|&5eDbWLPTJB%3?mZ|CRsiVrtFKX@;jc8bHSU{! zle=U6cTRd7Bop@4EhB z@%yJ`zdyY4mk>D9oo{fMR}edgpUiJ>AmTJ@@(dS!XYw$kcYPiQ-@GESJ1@ z&g7HR|8enS&z32b6OIq?e_wXru z=?dMZk}Ee`7H<>IZfB6VnZ8(J%E?XnL5H&+Z}xn!+0(u{=flGf*B2Js`}F$A@4vC+ zdE(?bJFafs$~c`p|KG=>Q)MEnJ}o(W(0Gr5M)l=X=2p>qsq1n)Iq%$zjcRPp$A7%2 ze7cXfKmXfD+dnsWo=oKZ*|hdsiujp73E7jvzizbDtKTvE@y7OB2~)+-|5&yE!;ge` zX`%YU_TXqs5$Cr#m#_Qz%cn1oIO1w|vu4eG?|aYU*5ux)bxG#m_e&=&${B_|0~<~?-id?u#xlmtN48~YOzTlek`~j@uoR1;)iD5#vi)8rGFnjOqW<4 zFLPR&`PEaixW_KiahCBnCB^UccK$JoJ8@ViFZHL&eBJl#+d8lQP`~{!HYerE&5t^5 z2jw%L>&GV^5Z9MpyyNklo;$1DpM02j<7db6b8>NyXU?g3Dp30I0-yS=8DH+cXjlFz zIzztqNXjo?qra$C{IO|BS@Oyl2;|yw2@5y0bgI zKju5Dk=!5Q)tmY5Y?XFDB^dq2Eq?1f_3af;(sNETCv7iowX(C(nDgbxw)9z$_0I9XdAerCjWDzOfyH)~Ha}n0toyU_o$04Lwl8YmoBj&Ee_;RL@}B!|Pixl| zPMSLX*vV&E6-7cD8~ZO^|L6LB*7M?)xNOI=J)aZztjd4K&JBv1-r&c_A{6bLZpTlZ z9skAluJDaN5#ObDoMM%E_3(RGd!+Q8`LFjryT^Y2-S2yQ`OjJX?EC#-^)KlMb4flPxe!dLg9qZ8`@7?O1#{3<&n!aSMM{6x7fX$bX||FDrr;3mK~c<_qE&b z;2J)@-;cBUD|t-N60}|L>ZO=iBNv z?;9F@dYZAZS2y2qVQ>1MPm22YzRY4jIicPB->LPj%ViFnTNiO7a&Nc$^grwL!*}m$ zw{?l1+grTw*z1NV_1{h{{C>>f-F&)0sM8lQW!@nui(V=3o*Kb}U-yf?ez zPuA&uR<-`Mk@6zWA5E{n$ni;D{?UW|yu@j~*(E*GuYI^yR~z~7{%zY^ zspyohK*>yT|JQo;KdE5gFb`|fNj~(9quDSm2_rI6^st5S&oBfVwk2~9%=hq&7 zX5Mk1q5fo5o`3PR|1s}OclZ0})jf%^vdKR8E_E~i&DZB_SKD5!%bUMBeS7(>^LEFp zd28+}9#OK(=Wn_#eE#3=>K}V{Pe0i{XR7}52KL^+k|O<|ooaTyskylK*uDQ7%zw>4 zy_fBo?BnoxN77^-DCX~+pkFy#!Q9U>v*ydR@5gSPiaR+m`RlowS4A;#ue;Ah89bPH zx&F;*`TK8H*zdn?vE8~pYkT{olgH-mR#SUF{Z96r??=8(s_y-BNSE*5L#OWCq6D8K z@i(>8pB%C0S{*Ds&+O-6WA>|`AD@*|TD|V?>pj6LO>(}nM zwETLlL!5x*uVc)=f9?If+VGU$n+crGZeq4QHqoPK3_2tLShDx*J%_g^Q|CDSz|19@I^94Kx`!0p@-`?5r z=-^`6YqB$fr*W5TiwybI;nKgRY{?mW={MEK#o3j(KP#(mPIxLV|Bb!Zwv<2qD|7Ul zIh7~$qdj70&O3W{|Dw;ChaVhIalUm^J=9)q+%2V#|Kb_#a z%O_QmV3m8>{bRy`X69?Ja<6II*hkr%c=A2x@$~gq=UeSAhu`Sc zlZP*oH{Br@QrRZ(+>=;XF&n=l6MAPBZgIpGxPcnIT=X;24+9F|JC9 zmy@p6y;}WP`No9%+uTYnrT*_xUUO6Bv|hc2&Bwihd$(F!DlGOta;*Qjh5Luo6T2B@ zzv`LZ`YO23@VLHp$>h|@wH1?$%}*^hemb*H5{^y)n zS^no+)~4q>uiDMet@+vW!J7TRjeV!YtZSyU|GDJ&>eQDbxAz{4ymZ6jymHMXQT3o@ zhtKc6lYQ;`i)sD$o)(@DNDhv_s^)%lLjL=TyUF|ds?Ro5B($=he17T5(PW+nC!bsD z9aXO{a%$W6t?2BrXW93P3i_rN*Gysm^re@vYX0rLZZRJ^cN%Z~bAo%%bIZ@C-Gc{o6QY^Iwc7du3{G?mZSW z-TUX|_W08EipST*%zw>M|0}yTd-t(s!{2t_rGESno?e$|9)Itwak79}N#FMGd#``! zepmb99((W4<5DKYXDW=JMeFxWzP9Grqt(BJHR_*T|Nki1;QF$ZAkG`hwn*Rg*~cf> zB)a2Uh1jg)jhRAM+y#nqDwRDKU4L?y_e)gBme~iy`0P$@sDHKTYFZ3?O1b!gl@mID zC{4HTw)*?yPv-IO-`Bssk>VG9lV4u?=(T6tKl<6MD0_O<_R+Q#PrCgKPqE+sbh3F6aZ z%r)bwdKzeRs+wWmj+39FeeT(QNw_#~t=)tyZNE>l5+9y4$?v*#;g;ln!Jh?>J~sZi zY15N4o8NBtLyqn{$F>)~ZL`XMs=7nXj)nc^J?pYIalfKBlQhFqj^2J|)hn%TCz(H8 zF2ugO_OtcI$@*+%wtqSw>qtkZ|9Nh2_BO5VXVdzMNzd2Y99a1Gi0rjLtJ?+NI|lQ$ zZ;Yt_$4$q@BTLSi8=aP}v^w@(BTsVvzehVZ zXg|*uea^nW;#Pu{;ii>)3)j@2%8uAPGtMj_>|9x0S8tsEu46Y}^~fpB_Gz~lcQgC{ z^!xt(`JZI($ab80X?Nz3$F`kE<>Y2R-L}!=-IwF`w>6}%G1|Q5ls>vdW@GpH5zdQMP>HMQ1F>{~4%)9;L%Q><8MUAgE-+QC|nQ`_WQ}t6*%RWxZ)H&F@ zf8&|o-S14#ExygZxOUXAJ z?pp78`QhrFRr)tIx7+7^mnwF%zj}Q3{+G@*Hs7Uo&ODNHZfdV)(bMR6PkU>M!}rME*cvcPHaJ<;KRG0A=pkyBhE3OJv-3t^HBUq1v54 z(ZN^1?4zxk%f4ADAGgeR@t3aN`fw{_)@ijmX~sB<#1`iFF*klJe9tK#BPSi$y?Yy< zgc#S`g3Z%&w_WkA*mUdUL0)ZXU85h-@{!#go3``&f4TZ}vaS58#D%N<^khqq%rmkr zbbF-K?e1oI=eJ`%%h&X(qfYugw?1r+U-#h>xA2@>;@4Y!ZE~*0KU(14{^n0bll7;g zk8{_&^!_Ec>C>ydRf}EExkdMUwQa7rseFFP-C^>#O!@K`L{mb zpAnSqet+ubW3eBz!o`G@+JAiL=Q=j$ljoD~jk&jOHn3kj|Bd0^w_nk}pJjjPX3rPD zS-JTrUuDtFn>PLDju{$$Uw<-Db^aNZ#OE_BdP2=MMgEhwl$Sqs)X&W3S5xf9kIc%o z65nl8m)PDvmY2-@?--M%RYCRV!}^k+DsmJa2hFX2>tP%9*L%9X(WB)4b>{o*qHlh5 zJ;to9@negltTW9&yZv6kE z?5Ue&Ok((6^5~ojJ|0TJz=FhyJ$E~fNDc<>+ z*qt-o?IiS|6lC%mW!Y7fO>ppVrMo!Tk)~2{+#vWlb`i|Klm&CAnlHC zIomyb=5HLuAAhnJgsuI0_Tl+l+xz_?o0ls}SgS|JuQ6R~{H?I`{PA7aEtE^nPw@V2 z@aS@#_VKy4_e$0;e80!-nw5s!d-J>5a}2jUU0=HSv)ukSy`Q%Vi`7^v_FuQ-6Q48p ze0r$)x9zoFQ(}zZoAUU`qn$Re~YB@o6BGG@-CaEB}}+K zb#IYJ-p2lgZ|7I8p1kn#-ZtHRC-wU#u9QD9-|WT9+kd(jeoU2_5bB=t@b->naXh!| zzV1C%W2|@m`xDPKNjl;`z8*F(Z!2;LpOf_a!gr}{Ke@$iJl>u9yO+Q23E4_E)P?O=6d@SS32@UvY?Hvo~D5{ooeEiO)N`&)m>$d%Gsny7r?a+l{hDxw*v~bZTlS58kp%w7HJP356BclG&S zuKZkGKiRY0cB6{+^Dlq3wd!&!ZZ=Ka*B<-Fz314;E72dq-yPcJ_pF`u`JENzLB2jU zCu1!OW;_!Ad*Sz?)BP1Mp5DI^A^q7c_gjshl9+hh4EM^Puijj^e$SClzcM#x&E3x0 z*TId=wUv3zDf}1PW&H|Ia!bB_^>eCv>ZX@(+D~45EdNnG|Cm&D(zg%0Ev4gcy_T-s zT`6sM=i^7Q(;uYyD`fXPco2O1m+YLI>FFE$a&Nwg&NaDu z%im16aYH3~&znWhW))RjyME(VVf}}flTZ3foZJ2{ZSwzLE|#(pCpU}gTiT@l`x)4_ zeWP_(Lf-9SnH)2U5VK;DaO11 zX#8GZ5p_NM|EGm-9QPh(Zg01%7BHWHvn)Nq%-K_{YOwTO+vw5}r*O{MzIxEbrp9s`Hwf}!F zr(*F}+fVT&b*toW1}$@c-u=$>7-x~)+2hLZGK24Zdwo{zjPzpTGgD4Vboa;f_SERS z@ABXCzG!`7_xk7GXMNiGI6D5!;ca#jtw*QcKeyr4hf}&rvun#9==;BVQ+si5n_P5` z(x+$Qb>F4xEV@{l$emDC_#Q%=1H3ych;XT>^ z>F2*0-=*rVy-%pJIC0*N7-TN_5$L`lxDSvDEWsBd} zeqgUxJ6n+-Q@09Ki?aQg|8M^P&F)P)A#%*CeV4edJ(?)q75QfGtp@*$Xr;+ds~9To4~^cVwZD{+>6FTz|Z-uX#B+>F{^Ot~dKXPPX4^RIp;c(W@^{ znMCd5+MoQ{>^U!|sQSd7U)jH1rDuoheYAPxVj#x+`;FpLK0QAElnLL8PK0l>N|Wv`|Drs<6ECt z|Acqm#q4(%Zrk+v-}v!-ay~oPepSb1b5xjj?-QQyJI%L!!`kE0GxPu7yTAL!{_lI= z+vd#qDde5ZpjMZdE3>6=L%hzt46C{3tCYPT z-_?FN{r%wnkH71m{+plhj$>E-9m$GHwl$Nl?fcC9L;v6U|KHa+UQ>EKce&}gU1@)> zU#q%TRk5zMWYPB6^EYpP{;*Z7PWH7WkMNsg|1H-%|7L4h`9}Lw)mzP{;rEt1%hg_r zYyH3RooTH@OSa*un{QtJsy*-8yz}D)2_wDm1zH;TWU$cU~^-n@6}+dn54?@=t;u;KPk&iJ`|&;6`8r&zGyT=Mh1%M6$N zePdpAR<^3Uw&u{W{q3L@z@&x2^S(cRcq3)C%#`!}rRyJV-W;@3Yx}+KYoH*A`a*KuD z?Dxg%AF{Vy^Z0hvf6dQZ5l8InrXH5d+v(#rJ7P;paXWi$s<2Hz=-u`i-y+-Tx z^qQ)}di}S0!mVzaeEV@US-4Ms*Yy{(e<{nXeVJdq{^5R$S<`!~i(}7hI3d{7&R_Vz z{pgywb75)EHvahez`LfSCgSAO_&?7|ig()Ot(le$jo(Q3CmMflw;u+TzkiG}{(kYj z!CwFP`M(#hf6h<%cXR*m>19Ted;VSK5I&S(#_20^p6|s>hmJctD;`X|c<`21^=uhm zmA7KaPn=Kqt$O&;^%&EYC6BD5pDSHWx_NY#uJ9bY*wnJa3+^N}Etiwq)gi~FzyD!C zp5i9PRnNP5=H7a|d1-I_6}6-X;^KCa{7LO^d$W@^&T!?c|Ep~zFE+dVedLj+PbTu$ z9rrVQ!4;vh9CG)F3 zm@oU1|MOWp@83&5?pR1ozwFo6>p9^`?p*zvO{a>E`#t@vus6bO8jp2>a?R@lm#(_) zXJ)Fj+jDq}gWA(-n96*-lKNe8+!xoZk%JcEG>HT-Tc?nP5V5;ZKj?U zwl-KPoqdce`i;80y-+z#O9F!8yWdEEz`P1bexja|Ow;sRFeoIRF+4)m)k2NG~ zme~LNwd_rE{cHX|Q-3r6`1S`hICa|i!ms)|hW}6N|D3N+vE9CHPP%pWZH~2%&k5&5 z9uBC!`O5Zo{>dwK_p+m&-`M>Aw)finx6U_D^?%)1eCDx?{l44FUua!EFE9V-!12ps z^H}Zd&zXiw@2@CH%-&nKZ(hFLv9z}(prrDpqxe4txs}`FJ1qTd15>NM_qFc|GSs}GQW4en{9Rd!_@ho z{@Z<*s+Ip%wDIGM?45HhALZ7pf1mABb91lTpXYayzt_GqEy}r5e{pZy{5D;&8ttf zXA3u)tP88Cd!bzS@}+P^z~AQnJ+F^{jyr6B|9#PVL;GV7ZY}+9c|SX0s&eIKww=u- z^KUaeYoAm3B)0FBvDMnrTTcRJ*PNETdG_wsefv-EWz*XhWqy;7ea@@rd)?+8o+NYs z)UR24A2m&VtH1ktj?LoJ{q4GD6KZbu-^e{PA@TDW=6PlQZy^Kq=gaH=_3xd>|DgWW z`#-I7uc%FCUUD#p>4!z?qsSf74}SbxHkn3Epaby6W-N$r}zo*=TUQxw5dwrrvH_)nc)yU)85+m**&$`2~I7!0BDPm@hkb z`MV^!YTXqnU6%i!Bt8hAue_~$`rn$znv>IwBaMnW+^6$@+q6(GUi$Wjqr1wM$?zF* z*xz~bLZbYQeMX90$xUs&PdQbazn=J>;8O9y?W_4*K89N-HYA>rjL*(4c;Wox$6@Yd z%fpM8->sg%(KRo#V(|}+&&hW#8i&1-zo*^Yk)mUMqh}ynar6e#iP>g7Nzo%-tTGZodEKg!qpuH~XHS9aDPa$V>C) zzdyOZ7T0~6nDbGL{bpuOLXZEw#eBv>%Ku;Ge)yVec**f-(4U9Pes4&Qi7Aw{_mC<1 z`86e9`Z@cboQ;BSf5=7uJn+1IjoH&@8ucHK+?(aM&90;&c$)L3H&Vf8Z+~ifFKGRD zL#wsTkCHVXe_fJd5B_^?hKl?3+eaPs#4d|U+sXaa?bj{)xl;a0;WPc(yeUzcqpC=0Df$ z|7N`2X_>$8w*DLA6}CKmcR#1eZ+t)L=ftncUruH_-&$Ope>i!6aoZ<5<@a;8RsZ?$ zVvF5J|9!hk*YHpNcG16d{l+pYZ;^h9{cqxe|CvoY*;}>qsC8y)+x4h7HH}wZX6i40 zS+;LFbMB9yH7Bd~rElMO@n^^Xb1wPC>nF;8EENk$EB@%ulkr$0uBfDE_vg6ankSRB z^*;To|JnT2_q)`O#h=`tFa0iM_tpJpptv&6sh<@OWMigoeH$Skm%h>bZg$s9`Lgel zcXP6@pT8Ghx;}D_YQnuQKg_njIQ=wfzS;X{O7Um+XI9*4K3iK<^rG;bPTkJ7%HI_^ zlOH$F`TnXmULvk)edL+zH4oHpA7wHYQm%g`|2a;1|0DfprFDNNI7>vaZ=8E^ZQ@h5 z{LeOfVdl~vUWu>SkDFy}iitNeTEiRlZEG+Fs%jcx5|g@@eT906O%r><$O51?c<6w<;lWs?H*P&^M9or^_VU*zb5^*b!Bdf?NJB+ zxpTi(KWwu6V=zg3{=HiQvPazZJxZ-Ue*eXW&P)8SU(VRR_Im{*dt*jqCx!nA&_QT8l*KVeq;omt){eAu^ zyC2u<@9W>)&v=hv$Nh4)xEYMEH+=p1hvDDc`+weUP4lgr|MkwyMf?`?t8Aa&{&(@; z&xP+y|1JDkxKHNyqg#8kuf^y8yY~F+vEQqzdyUuGxa!BApX*xAKi9Z%@7=>-D}RSeE;UOzP}%T?p2$6>_lo= zu9SUx{Vwej@^)u#3B)I!I``OCA}lW@@uI!(kz*exRrmgLIeN*WG4W@z{QK8CuIJ=1 zAG*VFzG;^HZa$woU)%qQbl0e9x6j_5eD8v~cAVKcJC8DpJuDJ{^ssKQ*Qs0 z#FF~!>NStA9#47Nx6X9p+Aj|jX4je*3BNdM)q4PO0ZL zbFKK}7hm41@^{0$f0vJUpICUz{oBVW-Trg$B}vzR`e5T9|9o#-zTuu})o1(upIkoY zntPG<{bSqNYkR7B?|;>m|8DoA@;z*>mA9atc@OKE1KT9tJ*w7iyA|~zDrb-GhUW)2 zG2Uu12spVw%xJR9&Eh1D zF!}Z6cX!iH-@kNJeYZ(|s^vs`t5^NsZ(P2gG@Y9z+(_2z;fD>1$4;#_|5Wj7`<|2i z+rC=b9TCl&ar6HDkg4l6=O^>Cy{%YnTw3_}BlqNrr|W*ye)v4U=-2USIo*aNT8}cGek3FFp~L-5twOlCdDGP$>iX&)H=7?sT`@DOn2dg{%gVxKGX!*g z{9;O5!fPM@b+y%`?HevB=HJceJ2|t+Wtwfo9{<`>ex7?dUvEq1eB&=lJTc#@GXGoo z9iDZ^Zk+4Se6C&pQ_C*D=$GnlgRaZY^Kw(`PHd`wB)mA{=BGKI5}(c2eVw}Frs8{f zoj*SXbDtbqFLSZAJGb&u*#1c?<8ORZQLcYEAvjIDvZ#V%{a2=V+gP(i*LyXm!+#zO z{?=k$Q*rC#r}kNdyg{B!;v*E#-=Ey0%3_rBZx zc=_#M{9ot#xBnRKE!YFisH;oA{9$;P->^T$cKg1vvL*Meey+S;{9Nzf+VgLv@2$_Z zoqsmK`(DK@o_{ITJ+7k_9yOe z+TXhYEi33V^WT6e!lIZ;l7VI7P-su z`9I%q>dO(vcJsTRl)u-k>%aKa_{^K~^?`QB@3H>zesB6{>h4GH;|{Csxwql)_l56G zKfF@647j(^tYXgeqkk5@H?8ULw0WvEzhJ>pLy0Tq|9>UFn)k`i?f1Of%l2-!c;&pS zsrIFx?!DRnGH&tR8K>g+b)Av&MNb`8)15S5q>pEH81xbi}}ni z_Iz*JXM=wUeGkmSrI~MCS^SIDjbA#&?%H0rzIU}B4%gnF?f3iGo96vryXzl)tzr1j z|Nr>^lIJrx(>7)VW=wj!sKED#nyjVpdQKnrSoLTd9WgZ%%O`th|DAH)_wb>VkBP^V zR^4&B?6Q3IokJgL6cg?%U0!rLf5yj`|5w+uzP|iubuYig){WsY2A95@Z{8ZY<;3Bq zi+SGEy$+O~xBXjh=_|3MZBaXp9JRCjxp2Pij;)igySdG@kvS#zsiWU^kI&qmJ6jzx z)sMS@9=#Pl&wB9lY)5vE}`T7R`IV;luL1l}_{YF3JB{{AEk!m6wZ* zj~BH(C}~K(Ei9J%lxx1Oy#Gz*%k!6J?puCb?@Ush&f@OphU-2iwtDVM__W~p<38E4 zmsQmXAKEhi#{Bi4@0+_}(@8lVJ+BIny?aFRza3t9PHgA1s$H)hr#wHjm*4K|?3k(H zKUd0s<-Zj7|MW_u%E#U}7vG;?mw8v=^Q`X&Z*SHA9mXF%@kC5t%lVxB z!~Q?~|NBjE?0Wpy?Dfxizte1;cC($WN_|_nt@3hJ-e1SLt!pzY?p@n@uBK#dZq8Qr zx7AN>JmbE)@9D($n{^W3&T-{@ZCr0=_l?Ib|4pQM;sj|U`Nwga<(KYz68HE`V9lpR zf0li%VTwPKA9eob42H6J+w;OVCuUa)FMj?+`?RQCP+8oX`X^!Uu72ElefpEf>f!&N zxaYlm`$NiF{&n2p_8Wrt3p(EVexH@3x_-}(bv88@_qzR)R)5)TS)WsK?Pi2N&!^O$ z?CIJp;#MEC~sL=BUU0TU+hi^w%Wi-Py>!+kEG} zTTj={2+w=-aALCf`R0VCwK4Y3Cr=cw3?J!)Es% zE)y2}_UUX~#n$e2zFS`t?!D^0_~qKW4@Vibt%`M?zo>n0`sX8~agumi<;`se??k1` z9AxnS8T5U@^T&C`n&QdDpY5Oe*RH==`smdBigP;)-Z$G*@dv~Gc}-sKI2zK?&Y zTxR&1$bPKk=T^=Sf_k~St9ahJ6`Xu^Lz^*DVIIG{7lpS;lk*9Hp5Xj5r5#l{ zNteU^-MlH|XY*z%x5bkcg(tTe&d&c@k@I0^t@gak#r(p5fA!f~)l@v^ulwA^YxT)v z{v7uHoF(VNmb}f;S1J(|2gcacPL%jEZ0Cn|BpoY)^^o5IS(?I zx9_htX*)cnZ~O8eC)9WESk}95U&G;}0slTsFnq+|IRDQlx962-9;#oS!;$`9B22!7 zBf)sjqi5Z+RmZk%`y@Q?+2q2VkC~5g`-TR~eD7F(U*x8Q+Vo=!zfFD;Jg<9g%~SdPH-3b*dG2=a`>^NcGw%I$q5U6jOTXCQ zzFy+w!^zrwpH}<~wvOHQdF`DYUeo1jjclL%_{^98{@R9W`90qqlaJr%k&3T){==;7 z-|zb8`*+$mOkeT4HvK`X#=k<$o@>?f0b* zX_s%>`pcY4?vUBEujB6RxYm7{-)H@uf64yz#Ld3xlZtfA=Sr5v?)&u3?)xm;*L`>2 zFWcvO8!;c<2~0Ux$2&~|Lbivzni@*@%D=Yfqx!_>Wf#~%`Q~+KPGVdnYZD} zc3=ac#DXFhYWxSQ?v^QD~azI#vOKJTdSm(#M3KfZU_?tu4?tc(9_d~f<`NBI^> z^P;%;6V-pY!dvp^z5HKvbB1Ty@$LV19^~g!rrVBBH^21#7w(a7dYKN zYPC1!^s}nZe%U^m&vIYZVdqbu9-KXt(rlSJ^{4Yq<@skze)gU6v@tv6&lVqRf1zAq z!^E-=ith!5Ys}VEeA!g^D{S4l`j1cB`z!uz{}EO2H(NWFy|_kt>7CCf!@8@=x?BBf zdzRbs=sx)sD)Upgc-zN$`8r?D|9SH;H)pdT-^bHaD^49(ORf9CE^l!|L3w>jk8i%= z(&N+F+ft4s-}5!)|V`j(Xq)f;>5kLdgG zHQh_GecG2>_^Hz@Y|q4Swdne%>A%x+uAYlGdUNvQr({(HZo4|clZ4nQuY6OPP}aQvD^Fd zqhnjXe{D)`Qn$>^PjlZNn@|~}cXN5TO(O5&@14h^4s82($*upt!GnV=&V`bbE7JGw ze?Kqx?VGB@ysLLM$^Cga@%iNcvbUA_?d^7U)Nk9g-dA$H%_-YG-|c=(5#Kps`LPKx zj}}(T|G8NEA^&gV{-Sz@`xgr#744gS2HF39g3?R<^sjf~9`t-z_Vv!J{#(|!`)~1= z+0QI}AN$Vqt?fB2{hik@Y~Fvf`QFpG%j_0AKi<$jdVQaopV8qPF?-Hd>4El)ocdIH zYj*GF_H&uHPOUZixnafTi~m-y^NoFHwRlB&pUQi|%hzkyH(s=GoSWOl`t2CcyfT|F zmUprfZrkvi{k=8A(EOalkrjSzea6!*zD%}z8Fx8i#eBQ()xA6B@Jr9WZGYY3aP<*m zqer64ckU?u)HT;<=ZgOy*M#O=xnGpEhwriazS29{S5EGEyZ2bd^Y*vqce4|mbN0R8 z+s2=rdSym{Ip3e6n+%n6{%+}xUGtR1U+=?&U7I8>RIYz`e(vKh&dX$~ET5rQ~lvzwdqj?~i=7t^9$%oFbEtDmBJDn_Y5x zhWslvp{#|5gx?)%cYwWm@JopCR!=-`vCRd^*Y zCHHGXU(Y`!_w_TLddc2ram<~Sa%;{H)x&&tL2>G_Uryfnn-uFma-FtES{%>vW^ZmEk z+?rLNl%D-~cFm*iiSOHcwf9#%xp6t{-p5t;{VVrf4?j_(z%M=b*nBHV!zuCGK0J7x zzxQqX$BWAE1+C})^KP{JH7(rk!QC6Z>ou3p{$JF1|BjCOhy33O3$@v6epP>X%5m>0 zf7r&U;vcoXcT7E>aw__#!M>l()A?raoqAN@Tfs|ho45B?#8@Qc-&y?q&hyUXsn6VP zww0}mt(;a}w&k$$y`)2*JAC6N*nDhv7n>+JoxLn#^U3*THf2+{-(SDuitSqOly|is z*sJ%}O!3}1t@!7p|Gzx@rUrg`8a;nPy7TN^MONwmUxa_nzhw90{@>{RHT4YlCl*8U zWW@Kip3m6}>Y0wt`dqr^d9~U`wR2Tz`}5y#uh-Q+rQUNeo~%0=kLnDJp7s4 z*G+Pl{}d-2_cJT^nX|0h^99%5Pha;u+q=j4=tO&)x`!6;O@Ds7)^}-Z{DG$ZJCl=N zJ=Hw(Q`KEu^Yr`A)xN(TPRcI&a_ex9ap@oV@T+tIFGpiwf-bUwqXF-t6g2%Fdnqe)_%aCtD07?(O+mK7E3IofP|CY3`?8GBNU>%Yb+&wDxJxJ|~I7y7BWzXEq|RLcA8_W$MXN5{4GOHM4VmjCnT z+rj!D)Bm3Uw?_Z)|39g>NsNzq_32#Z^9@JdZ%8mwvI*JExVyRR&4#BJ(-OBOY`osK z=&ZZh*EgZ(_c!{UXFH(xDXrN2+Qx9{xPRwZC-X&0%m3y(@my(^?R&AScAqw!eK-5W z5i`c83v50#t1l~hYP9~Z`u?wLdoR123IE&}*`sr(L;pti(pH0*H_y%=THnKNEiKJG z^JL`CBd5)Nel^^4H)VOFj?#MHHyvvt4lV23=HKQYCL^|@XnFm~*Zw;lZhg0{c`GPe z^wrhOuRf{!$L5pi^L6*6yhuO%Z1H^ERlhbAPCx&NVSnMm{)9j^?d`XIO^j~3!Zv;Jdp~;fw>>$&+i=mIZ+ddlNl)VY%J#*i ze*19p$!+O}pLg0#`#dH8pP+p8j(NL^>Tdg-FO2bfIq`LYQ+(dlUVe)$`q#9>B(D5S zn7!6ulDXeGi4$`_9+2Kwll}bstqV4n-Ji&9>s@^@RK0zR@-L40zJERQ-ulbh`+SWy z+^CqgQ9FHg%p~`R3d(J-Zd=&@=sNHAG$-r%=ASeFJo}w-*nZ>l*>g`lov=6QYp1J@0>PvAWoe8?N_qx;XdU`tYgzee?YPb`|vu_aD}PGH0CRhO#LA^ZXC~ zKfV9^e*D#{yuI7__h;H(-}d|T8|G`}DR1A$J+`lX=ab%_SwG{@r?-LM1b2Po+IG*S z+HdyXmYUyD&+UF^?K=P7^i|EF3-@IF_8scKE;&O+{_V?CkNS_x{p>v&nf&Co^v4U& zQodX}{^47UZOyice==V0Ia-z;_V!iA_Zywk=e~B=A6;z!;D({ukASMn8~$!R^V3nh z?U$1D(?vHXwV%DI`n%@gNzPjq`ZDol>mOc@RQr>^>-r1#Z};BEeJ)XJ$}O(^T3)-$ zLQQ?cobK*_FW>*#)%4yAcmJH%+&L^jc?=POai~XVW?T0bXZlp1PRBZpyF>{W2ea@W6 z7WY?|KTLLydooS;vq)br>$Qys&3L|S){Rem;@y+ZbF%lsVYeU0e%yN4ZvXtzvzz_5 z_qweqy836zby@4l?>FWXB*ti+O&PamWJ-uJ^ho#Dn-&FZXNh| z(}3~kmJQo)uTz%{Sy|D2qEeE*8Ye75&;DxYkN zxp?@rUu~#;#o;CTCzUKtOc$=%c;KocSAMB`*vy(rZ8M8&{IlC_zXkq0y}!~q&;0Ab zB{CDA9-kx9zt7=~dDvM;tCSCu1*0pEua~#lGI6o<_b;AdJ{70t1-H$!vf0qOeB*|? zBUVNRUuGMvo&0HL?5>)vc9V4Hr+mr(w|DEulixi*99TLxCRuR5rMumpmy?c0@4Tpc z{AkDgo^4Mag!zq;EFcAKSaExZjvxX7_UbZRfh;x6JEq-t)QLM*DpK(N!@$j|%-n+2g9_ zZah)UZ}PF=qkzkJ2G;v zUteja7nWDj%UEVpF)w9{YWX|K6o0RN{as(CBr2@-Iac8>?*D(!*}ZOZQn$Z|eKvK! zVrzYTeaykO^q3R(E&t0~n%dbduI62n=wxx9!{#lbGuuMm~S$;s?GQ1zF)7k zKF9IeJ7;EV>xDmOx$iuu!t5=7^V)+y2a=bcUE67F`)kIBlAG4zvO5kRRI8kA{-~J! z)E?ajiI?2MZ=Nah|K5rF zwrTnH@BFOgpZZmQEjq)e^f-}yT1Cwz#q>#E4*Jf`$t~&1How*>zb|oJY|*WkmLD!Y z^_M@{Xmuy$%Fm_yx$mU9@9+Bi#=7cN$GUdex(`1C+G{`Z@!P$X%RTD*{Ya$G&*1z! zWm`iWaIxSk6JY6^U z)mO{g9Xh{nZPlDC?p(XvCp=bq?jMcA-faAqG0)4N|NgLL@>>2AxpBwWoAujlWbL+! zPyFEaBV|>)d-T&!7bN7H`~S4s?oP2S@!xIDcYONB@aT zYiLza@mRUC?#3*mHy!%@W=3ZVKfB0%yb+MMW&^K#Rdz<*glL&&-TFr^ug;q9zql-p z=U0gBD*f{^2i@o0|FeXdU8(s*#%jIi*OY&iY|XXbU|HC@ao=gS$Za>dgUy~Uz60Lt zEv0?d^6|+VJN~QBu$Fx&ynH>T&N>8rmjySqERzw<%! z?rPuPSKet~zGiu}?n<0@`Sa_Uzq*)rrsuZQ z+62~JukM{w@+@rH@pCrS^Jf0rQU|Jt-`~x?^!Civ$d-9?rq>tUp8uGAK7Z+rlb7t@ zo6hCmdeOA))76>FHZt)?-Nl(q+S8o@3FW)#z{+IBdXZde?&aU5eeM9?? zCQJF<+cuPbd|)8ArCRyv+0w5QBYED$KK*py-nXAOs-L{?G6%)w&9nUxy`0JG&FW56 zyuW4fHp$^8XE!)9Ypsf0+wJpfSl$~<6ECZYIN}!;Z*pAx{*6P@%Gtkz?`0<#g`XCz zIrY>lraL!DVfV%!9;L;-d|5TA-nYLo&lX-B5m>u?j{LT%=IP<{wwx~B`F~Zl@2+R7 zYQ#@}`+BuM=g{`#(kDm4W1{D6`z2QQ^ilile$X(&`|4{4u8BSM-^ssk)ql=s^M8Kb z|HEWK`iGlOcrMw=Y0hThjbXm*aF@}7`6|QJYY+U+Ex2}Vvyt-kC7<#xrA}Vc>Q<}% z$}}=s;B~;ZC?8*$PaU?C7oR*AC;vDn=apKVeBO!0Uv93Ay}9i1*?Oa#6YaaUeX(9} zf|uvyXGJk?-pW$5E3=PeKl#F z<-|ipyOSRyw?XQ*u?mYQ7#*UsG_YRdMHr&Byz9Z4*x3R8#Wel>G0-8@}CKc6^`3m0tBFJz}@F zTr_T*EbRUBB)iXZi}*W>pFd#C-nhBkN3a&CGj*^+9t3zK<^Nv_|2q; z-27?#_HMnNZ(XF+u0Ag+ucl+vp-bwY|{R}?3=as&#~QpQ&swHI}b``;EHbIafQ=7G!44stJ+Y3tTc-E_NMx%*Uk#rxV1ne*@L z+wuM2@dzWn+3d%z^DK$|-K&_}Yn#lsa{i}R$A0vkz4-sC{g>n$UEaLR~ zf3M?|{eF4mcsD1%e5JnY>G6uH2a>iw1SHL*r9W}7N9|XB9vizRdVj^Q2{oIvEB+|- zfAg`)pJD%H?=gv_@OwuB@BMi9PRM`HCx?AzH=^TDYbE@Sxzh!f- z@Becxj&Jtvr@6(p+pg|^6Q{gC#rEkUiHMBV(?J&@=QCkJ z__^n1ikrQBr1)uT`1gbJ_St>;Heq4+i~M`Xrd}jM5ZdKSY zoxgj>oWrimEN*fd+x;x6KJ}#A=61&#tG#!h^v$@jWc}Me3fk;H9xdd)d`n8sDD2}= zAxZux))iO9Ww}m%-d=cd_9hv{9b2ce=kHu_SX{cJ|CGT*W4Xli?Rz#}cg*};^3-_t z(@8S2zbgK0?EG+A{C2^DKj!~VA542bJjW+(a9X!=PZ3*qa?_ko=a|3gN6tuS zKK$q7w8I}(U3NYt_wD9o_Se;a#PbbLHWnRe+ZFLjy)-r1YuzhrL!Eseuigr+dARKQ zoU#v-s;mBNTFh&=ZS6|~!aBKK% ziJMyNHyguuTK&(NoSXA>cKnii*0p8X&!^iZiC_FF!;M`mN_yZ4~2*Y-#4ewf$& zUw=pbKzZ0b&|V%J=5HdefBt0rBVPALQ-4kg|6Kn$E;#<+{#rxJ zTQz5QG}fQr8T@VzYx&;`yFYLFv#IgMiaqI91{mZ=d1r^2M3w({?_Y z&R4rfev|#5bLlzz&+c`*llf{fTkGG53ZTY$T)X^lL z7v&!py)%6}x%b)TvdY8R`_$FtXWz~|teswvyND0X3GBxsIQdfe{PX|CnNv;g}C^6kK1j(Uf;t1@7Vdj7q5S2{Ik6N zZF<=zlb0oYS({#p@bBx}AgFQs!d_do`K$|mNzIT~TWcoY6eycx9I!27#s)FgZ)Jir z9S`zHv$4j=eKL5sbwb7ZyP4|$olU+P{5-$;g_O&a8^sn&&foiaad+9ShpNqoj!rXD zT6}l0PFm{QoGlZkElx@B==c9K&tOgS&x7*wSj}!1wQsrTT%#dwem}{&GWo?v!AX2` zZr#jGTz1)gbHTLiZjGB?3x9^)x7+p0@uu+q9)7ni`=46o*PV!b@p8+-2cqFSkKX3} z-6j6*zH$A&W%U^e(|OkXJaW4xu23ug@1f&z`#+tWZ&d$A^f5Kaygnv#j#}>G$>Zw_cT z``gCMH~QDKr``Rq{7#JqzwEu78ztuVPWsngwa`|7y4*0xX6s?b{rmp^fA+pi`E&f6 zlrGm=et}!=VQX%Qef}UdxqJ3R#ax45yJsBJ()qXf44-t8Q_79+9mjVatNZmMW%mA8 zj|4ZfADlEd^$dU63lsaBi%)-hR-7}9zx3^`2Vb5w`)>Kv>}y;5$wT(-582-}b8Z|B zxx1U)zWUTr)#(MDz47^Pix#ER;h0fecq;C zwoZ}xvgd{^>uS5Id1IJA^}gQu_4=+~rR$#;sp!vMuN}EzYV_;G>^V{VQ*PcJg8~K=V@Aul=XWsMwKI>=inS;;e1jH(=#V@?qJ8=JV z_1Q-f`wcGDCrJH193gk7CR<$o_S1*H-=%goH`Y97OD#LHe9!0nS4V@bWlD}d{@3_x zg3q#u2R|=-m)bX7&Hqhx!nDn@!jI$YJ{;9I`?1~b>8EqYZr{mP6YT$CK0kSL&5G>j z;pe_oX0!Z$DnIX-YNZQ%r+Q71__t$;wQJY@NndYg|9fTK_iOJQO9W-+O%AKSb@YSY zcPUHLYi9h(0k-xJbLTcLpYc%tUBLxYKl6VRL)~-g*HriJGsruv^{*&U#^AuZx8QZ+ zKfXxt@BbJ7XZQN&%s=M;-CdvAczcH~L#~@{klLJ+>wO#4nC9&|`5@QuVEDCZeV$Dx zHoC{(y#3_iML zmbq86%>TNZn*GzD=&v6>cOKcegMZHtGsR7}1a_ZFy6S&R@ZKY7`M-}lWEL4{PChNR z`{@+jBXLIe{``0wmj8Hyu7uz0Tbni;C2ybcGh@$`)qLE?jw<(c{;yo~WQ(+ME|WoN zg(hF&A?LQ<&rFvSOFo{w*xLK%+rg>KW9#!=+;4qI(0HDh+%F^P zvh3rwP2qcOI=)VcUFyvrFOxE{&QbQwl%K2Wr`dlCH2f$(QF*@Ywx12$_oKG#Sk{-7 zzWB!$?bGS6MehFc*u70KN=`Z{#rgj!)=HJ_iIsh8=NwpeXGTbTbh2{Ht;YwCrC0dN z|G4I0_i~DPYHmU0metCsHil1?*Zg?e#QW1j%zn+O9~wLVq&rJj9@(xgeUqoW?B<6F zr5agpgWIo4Kee{oee=@uANPOX`yS?=bF{}_-2dOnEA7WBv#l#P-3*mazJ2e%#??FR zq0*K=zwG;dP`=0b=f&-x7Rj8{T^0Z3;9h>ar!MPyO3#PC1#JoL@blXj^Sjwc&d)QK&9AQeRT5vd{-nK4_f5%+Uu+v!-<|huS#|FXD|5SFX%#Oo ze^scst!#f~Z`)kk-!accQXf0kz1jO|^YgVI|1YcV{Tvu^vNPD`obA`h^kzW@6?*F|^J^GA1? zuB+UOw6Onha`qPAe_u{+Ss0~qB7(u@l-k=hv-oXoT&`U{G|}e%;TYK)6Am}{&hW7^ z6Xs@=sn9(k>u_re@3E2}Nz6B|BqS>D+g6wT=VSeHIr}ex5qoXTv~%AaXiTtso4I~% z@2p&_S_2#N&|NtPoc#Aq{e1HHGmpfMzqh_R*GjP2d^@q!-OYNQk^P4akJp~B3F!Oe z8+Yc_hSkdXKfj21*D8Eh{$t?~d&vK<<)*FukACT0n>t&*=7VTm>c=k&6Zs3Kzw7+C zv#&KWr^CPQhU25q?TMQ2j~Q*MQ7X!f`g1CjU2?~1#>(6z{qouU8$;uyZyP+Qd@*5l zukW@WP3`shn{KMzUdZzQ$gDMXA?y4nU9azDu3Yq9HuO8+|6@6Ed?$ljw_xRZz#U{#+J(~?_a&CBz)|vwNbCX(NU*k%6a+QesCw( zvh&LpB|cC7829l>f33yU^K({RUiOhtz@gJd;i{J3wIUz_vcUHtvZ(e zccN|4)9d*jf6u-DbMBmN*^i6&q!^F?lD7V#?pG%uJ0-Tp@{zyv++XwJQ*#AvKRuZe z`0Lcu{=9!LwND>r`F!E9bBgU#rOJKnJFOnx`)K#=#FFCo&9jf4ZQpZZah=6)33-3H zr@6(}St+M}-`xJS(Jou}d27z6wZ`u^@!zXw{3KPs$8aNR((w3#++&MzZI74_Fh}N^JCB2=T>uf-n?pGq@~ZHXIXu6FWa)O z=4D5ccwQJ+-@CNsHM4)Z^p3<1|9NwtKi*fYKc{57zWX)1?D%`d4_-(#7(r(`-^j~>$In!?X)=m6HPqj1fA{kJIj>vN@u%{4U4PJB z`gq^3Y487J#P?hlYb~th`@HpDw%Uf}`*uANU2S8M_tbA)Z|c*tSD$=0y^}4)f4g3I z@<-KaZ=Ywsd22EE#z%phQ}-WxshQ7{vHSZ$`F`*wo`U_1c@y5L*)VVIp3{0x!PH={ zE!(S|fh%Y8@RoIFJXN#!EA~Y8iSfW(+3zd=c-i`K0{_RwXXGTG z#9G?F*`OPid+KDyr>_bT($oZ{@%VZ$>DxU`U6{| z1l?`h=gJ@W%l4x^=KR|ICwBeYc)j@B#IJXgZC@Wg`A+iN67KWf?@hmH|4h7ntMkg+ z<#N^j>3xhF({+p9G1Y9hKh%6{@{I$R)l-fvI=5_l-K8*_<#|RqH8-pFrElEm-R5`Y z-z@*qCF}m*+Whd+w{730{@oFLU@yNf@hZEu-IKzUIa8lpk9<_Ts{eeRqn}?Q7<21NVj}m!j%~G1p6cst-TQWLTkXyp{qK`k z?@MYwE1Udywf);4`@T#4J7=lPS9c_6-qWMX^Y1;4{oTIjde7s%<_8X(Uk;i{{-~7P z`2H;8Kf|~$2bb@-J|p^Ael|DH2N#1^%}--%ew;7Nxg)js7f6+Gr$axSwn``oGJHT7?5KkVNB z;duP{`3^7h8)mDpEpYLgazrsdSFUFAsrIQU+20NOdydY&^E2Yg=Jdp1KY!`SCui63 zZ4!|9z|k+aZ@=udwo7cy{N=M%=GI(Dovw4oth`A5>3rU8FFKYsI#26qk1e{neZvON zIKEplw_iMMw%vB(La6`z+!~vmzT3)5XIwddv}v!ci(fVOjuonq+sp2} zo;K&klZ#0*ITddHRE zj~H`Vm-TqhxOvO9?#-m+^VOy9aXB9pYKj;AWai#~bV={MPi^bvT7#`^`RDvN8WO*B zz3wFL%0-K7&EIBDdHiI{M#0qU){?s(DX^D+p0wANMefF?>Gow4YFe9hRn|S5Z)U07 zf4)pwbI-A0!z$&nY2}HR^nV*le#^WZx9*f{@tQqb<2E|)+ouq|SK`pI@|_#wG=J-A znO(Lt%sTFV@8)Cf)biqjvj=l$$iK<>-ZFLGRvyj62e$8<&|H&w-){G(qipiGy6tEF zc(vlk9_{pQo#>mMaq-)ZJ;>bE<_Jschd{*`i-^~xiU#{Ah zz9rIs->W#|*_Y>6SU3_!)$F*zc z72jXBesBMQx%WSu+*tYUqkiOh>+A#j%GggA&;QUWTvqqt-nY2J?Zv#d&u_|qGBtd# z@%x`RW54U~r{B$fp=iY6_k88AM*mcPrTOQc#n>#qU7?cyM#B8VhOeRDXT4K@qX8Ol znqMa0q&>$VcfOy@F@ZY6leTY;F4mot{4d-7``0I1Hhx_2F)-)k23_$li9hGOH~r<2 z_iBGpsNSsPr)zf~Qa;w{Kl9G9n0M-qZ@T2C&gL&&Z}{(PZu+WHBm0Dh*FPVy+H_30 z>3Q+)>pVA(F5`a^8*^;toLb@crn|*_-h8+FQ6gTO`}@KFPw{`N!JWiEx9i_+^qf&3 zwlP*<`5mVug(wRhrMlbK8&5lZzM-*<_w6%-O;$PLizY5Syi7f-I5^)p^U&nOi}OEe zKVJEMnf;ClMm}F|HR#Bls<`#S`ovTJy{#LXow|Lka@OBF5-9iNXz<@N-1TCK7Ux;H z<0J&?KCTp>FL7wz-+Rgcu0JakJM$?r@5IN0+crt;xp+&;=G7C4HcN2^1_uUD z7srspsS6YNGbh{EU3>Opn?gJH`z`LrcWl@>IWbu7$*qG+cDy*a>96^p58b+VE8f(+ zj=q0OqO1A5fu7zrb-x;Z*(qmkNlE@OE3KM$)XeHsu8Dy(ch13?F}0V~wDw({ZJsor z&zdjs`HYqKt>(1$@<|8UJv8z=`T6*Cvn6-j0@h@w7adq`B$;3N(EHu71N(j^pWa=w zexFQC-Ic)PDZK29=N?x!K6d^3vDI}a9#tOExo@yB`i!-jy4{vKc4Zxds_G-_*6H`m zel+p+hW3pTH`K4!n?BfH`6~aX*S!06$ximCw{E-jbN!mwd$$VCb9M~6&o#SsSn`2LvwGyi|< z|2ukXqIB+?OO~}^*X7*4{Zu^9Kl|$5mk%mm$KUv1Tkt@A`=-n6Ur&O%lI!2Dnwzpn)C))fHUjOsK-H5C|8To5IY~Qx-qg!13qV?JG zd;YWiSoZ&4Vfur;MhUVT*UyKpwdmh8&-!*$y4{z($8NQ(@s!J+VOl7*FIIhR^4#Me z-hNEl)|dHuPQ(1V>fF~dmR4(*u3dk1ukBn@yC+q8eA(YWe~pQ?S$j8r+q}&e?G(ea zUq_t(U;phL^SkS-R|)@RbGqf}#HFj?I;Ci0qKm!;2alGl;FSXxW;e$Ka3nZ8mb3(> ze|W?o_D(}9(1A%}WhUE1rP3uXt2|tUITtP!R%nd8|J824z4d!rOH0#}2mk-7nfLu> z_1n<8?6?1(oUMK%e8RHPRz)toyYH%8pwT(ESGRtAcB}CCW0`(xu1(3kQ)z$H-~V}% z*#7s0np#=;*SN`r(=C3SpKKQ@zPhg6@_FQptrMI5Yd`N*+qqh0&9ON)X=hICjp-1S zKX@g?>P`Czm-7`W`+vqAmbcNU{C;jn_%nT*og&h=pIO(Rx_8n%^6P^Tcgx}phWE2? zl|5~mdt|<`Lfntfd({jl8~X+PUR(9s>>5||rW=V@#f$FQ9oc%sQB6YFxqoTb?YXW0 z?-s6aY}TDIQRV=Px{txD^_KEaK0XQh_iW=DUVmxHU!9!%Z(r?I`}Rcj-%-ZRRy-$V z>tA)q@f7Xecm2)IYTs*%zuN6E{NH=B$NjV7XN%vh+vFA>%ba@k?B1T_uW`y*y@!nJ zCYC>wlh2ucAWL59@;0@-Yf{6n+3g7b%ln~v@27>UkNPjtl@pTkVvmT>vCI7I7rA`G z=Nlq7E_8~0ERl+QDkt$&omp*@SkRKbO+Plt+W)9IaHx+_*tW>_0Eg}o_Ia(7<$gYR z=KlMQ`i~bkHUz%8l>O$*DZk^pj`-?HJiaX5Zx{b0JzjE2QP-C8f8O^NIhj}8nvxrn za-);SvLgJKYI_sA{6*#GXZwESbX26ITmLlX<RA+s!mLe_M3SeCMCjhu*y2|3h7P{@m!=0Gpa3)(wWI@{g^q**dq2eSfX4&5s|O z7o^uLnaAv$^z|Kcz^72Fin+pn1HS$+;r~&1fkRE=PuN&~%p z53a0}>q&Xi*LNrLwakGio7Acw^L|~-|MknA=Z{kDOx|_1Q|gKq-~H3P`tjQBW>a>~ zkGOFq|K6h(-<{8se}27v?8h&|j~i6YU*7)ly#MYEFZ(Tze+k}BYG}TH!}EW6-1Iq@ zPs_9`9aI$M`=$zIm{?$KvCvu(U&N~Q|he*Jl}g72N2bE>sj zqEFokEqAlB8_&#tToI1`vUv5p8IKk}j?6i+c$d3%WPYva>_3Z-IJ1jwdbri$Z%6M} zyB+t~dpRGoNO~V@bGQ7w{&;$??dzEzTYC2P=qCTMy#H^@*ZhX^v}^X)ByarWi8y|C z_fO^@lkfe|U;WprSg&~6^0$%aH(#54_U5rY_hi4%n!_w)Dqpc7%J_}zitj5vSx(;U zvEH=$Mcm{aKYh+^Qm=k|aOz&uYTbF8xrF$OwiSe3o@VX+#p>Cl`?BAq?EWN8Gd26! zcv(5T=4sqw{f9#7??gL|H$AaFey+|tGUs;G`OE4#rzY=v^KZ?kb@#HR;*T;h+lWT= z3#KN^+b1u-t^M7&zwXti%ORC7|E(-P_vwvRcdOs)P+PmFee%yYBzUbUDdM|wJT>L2 z_@9ll_vPPxwDNfP^RktjC5`U?*uSTmw?3!xNQMrJeRQpwF3*SUJ~y8fw=O;A{ABOe zg=cDKG?kX$&HmANb$4;#jVaE*+-1`&)}8u3>xe?%+r8IQVp7j+n)qY#>o1KlFPJyg zl-P&Gc`dv=^`GIn=_c`t6k0$FqV} zkIP9OS;jx%lls3GYUMfSpB{^;ORuz>wEYIpv9rHwj_~jQ)tU2HC$-Gxi_Opd+%H;s zm=RAviAL#^!Mt8MBmMlO?Cz6_XHslWE;f>v zD-N|y-j%Gg=To14d(P3m@|@#*_wM+=D%j*NCimn~X3eD3@VUM7m{t9^{a%prQmyYs zVsc#ci%-t}Ilm_4JY$$=wxs9E%YzM@{7dpqg>im)YFPK+%Cp>@!>jyn9C&`;y7X_G zk=#*+$?Y*8Hmuh*+p_q?f#v=^+rb&wc)rneqp%+7 zGHK)T1OE4ScKCnW{yffj{`;!<&C7KoeoU3VapB#r>tAMVe{=hruicS^^Mx9nO! zz3x^{cmI!*Rrh}CS00((KJWNmwc4$l;**oy-02LOe^9FK2>s4Jl~*1@1ef+mu=7ECKrFslGXnI zXbSgEzr0C@e>}CW^qDhv`mEotp6#+~YPYS~cGKyV#EN5%L2QOMtRW65-8#>ib2ryhNd3HcwO zrL_I#GKt6Q-%iUdt;&&Tej_=(v)%8@4d0nEO4Il8sQr1+Z~sZI#wxYr#!^1FKZ}pd znDH}3`snpLV)3_{wRbLDEgL`Q*Four>T;h>x;|cAmMgcbvzLcQE?Y$DzW+_@g45Hs zo!;i|!f76eg7w-R>Y2GdF3(hFS3Y&qEWP8wGw(MyllQ)~UZ3flVtzk6;jQh)noXTy zmNDO4e>(Lp|91YWT$LzjBIgtDYMX_*ceAx7T>Tj|&+6Z)=%Tkh7s@YentZ1tR&MXa z&FqJqi`swv*)iq$$Af;~y-!Z=U4Lz*#FkVWw|<%WS~u;#U(%09|M7jC8~uIOkB#a|$+jsyEZcu->PzYub^lnp`DfhZGNTP8b=>0hpJhDr^j^m$ z$6ZvPZnJsjpMxyYP3E=Fc%8pI3D-$^Sr)f?j^#e(@3VTE&-vbb(7pG;k?G?8pJo?* zk4|jW&JHu^RQw?<|MOUs-4|19jptQgj|J~JGGoTpvpX|ys`j7yc=g*Uey_Ys_X7VWqDEl4{LQn>Hg5_%!Dc)#(R~ zC2bYUwp4!ZR=Iu1dA<4NrOI25e$Sa4$0zgT;mf1x2WK4Ct*x^W6PfFK?#TD;tD3s) ze!minDpfI?vdcocey?=(&8(W!hdQ5c*ZmRsF!oP+qUvG4j=e_~$@o0-vgFR$@L;pE zU0s^>LGO7wwfz-AedqfYKY8(FqP9wZan0E^@+A*Xw;eQp)Xi&kP<-~U8*PbyQua(( zc)3!y=B50%7n$b|tG_wPZCm?vt*-paD?gX}oBgVIWhvm|wtveS=iry03?GNMZxoKX zm$UGvx$|LdR?oL(?EIe&&N{qh*Nw^g`MEhKT#rYWeq>c|&RH0@W!UdNu}!us%uLp|yV{+F?cgyLw{%w|`-t}WQh|88aNhyC|oNxtTPu-9r${Tky9pCvbh z-L`+m|G@t5viiMuzTWwHeCCO`$$Jz1?`OZ+{Y|u5zw7;*@>_50Z+0IE4~&9{s{`@MBe zdXZgXw)=b2TDA3`d2LQ@*mT@}{>O#!A9~g%bord?J;olJuKe64-SIQqI5$pSzb`6({oerNhg)q^Y zzaMPd|Gu<7Y=5eKK>Yh=>CXXrJ*kgxB>URETxq}OdksVVfo%EemHlmM>OvAmmQ#Hb z@@Fq^_~`r0=ZOj1m+it4I&-)`e`AV~4gi+zM2%T+vHS{VQ z%g>44=i@f}{pYVmyF$BQkyzNb%>wMwdpOvR^s%cRFWK^a;i4a6e`lWP4z3IO_OAKd zquKLidXz3(omM%p+3?`ghnr8QUVo^*b78Z;+{vv)PZGse$2@89E3ej`10oYdzQ^z|77{c=`wR~*8lRVHZ7T5l(gmHmmk`DgW?jsbZTB|7oJ|e zF{!64=c&BewjU~ox0&VddE(sjMtb*~pZ4K5lpc55K7FAUUUXpF###H4%KMsgo~ORF z|MtP=%hj7{$E7cRKcL2zDx!1o=wkKj2YtlYHf^3;S#wf2{#aJdq^I4sZ%^{e|6!eN z6?A@~!l#m(g44}^Yy3UnSMytA{rw`(lzzXY&&&G#+5PyVKko=>7tL&Uw6#k-S>?+% z-*(@#{}spNz2%T7G5{@nh|Xy%s7yb}dyo9{E<`PJ^n+Kv4G zuIbhn{$;djuKgRAo}g}=(7WZ_dHDnL>rO@9jyeDK;?@PU~R`=R0=`ULEDSzSYP1T)=)`g$v-bhs8H?A)_s6Lwmo-QW;{&4bG`Z2D` ziaD;&t9kFey0bqaadUB0u14XLJM(1J!=G{HeBSHUm#|1Xyof0{IpR&1tx-+Z`#T}K zt=7Ei@c;DZOoi3DNsZx_H+IW=$<>R--wc0#*}CGznhNik{vVZcbJYKzWR_q3P3!gg zBiDcG&%J4H-@#M&_eD|CnZ0iBesHVRRsET=F5<_;^Kn&fDCY((C z-dYo$_cXkxqG*L**ty=&joF8%|GshPb=^vv*ZZe@l?=LHv%azQe2@3H!2ZXR($)8^ ze^K3A*m6c$;>hl`)n}{Z@9cYD{mH6-vy{Z#=*P$RoT*COGh_R)@c#Z(-OK7X`HlSc zg?>LM-2ZRk-sj9e`u{v#Tvd3X&`S8F0yo>dUZ0yTpRToi_;_N*2amn%E;l=4a>m@s}SKg)~(9F8-hNX-rP{A9{0&3_W!YYtxdKy{}wC9zk0K2 zI&;UL3uo1?Se@9d@lW~q7U(R{q|`V4P%v%Mdc=Ucp}JZJxYkDN4bOWesx>9-#jOq?-^ zeWQY<&zx5u&%X1iIWMAfusY%IapA&w)#of%-&r-=oiBbS_mfHNH+n3$ooGu6wl%C! zmp>_N`SZ&Ey4@;eFE_64oo7CYd8g+;VeR`*uJqpSS^eYMyc(Bw+qWAYA54=wsUKK4C}%dc+!x21oTbb8Vk#|Gaj<1mRW?Z=lFJ=wA_firGT!UOH1Qwtvi zhtIqH;m5}{-w!^0e>&{{sr;h(b_XZ#KC^#suKb_1(SHv8V&1{He_1J8nIiMGgx{C{ zF#dTl*=}>x?#=t&t^9iDSF*ih__3Q0@-1E*KdIdE-mSfDJ)1@TueM#6qqSLD=FbLENsAYm?cx8n9z3vYpYy8? ztdos&?(Sde+*hYjc!ce*?B44Km}c8lD&G*O&bZ2c)vwLo#m~~Rl~@09&%PNSk2IV( zb@u7q>>H)q1(Tm|F4(kQkN?)q`R`4C`Ru!KzDA+HX>D;t)w!iPPvV08!ahFUeAF&l z^nhDTXq)T*<@*==ht0V)<>%ttGj7J^Je%$QM3~*Szna&&L;q(Q`}xK7H(gF1UUx71 zOTJ|M!@KjUdGGvF?zTB$nR9H?j!V}K)YwmT9-M7=_erOW^|902-kIJKwB32KR;lvY z?Aotq{(kr{^{J&W^vd&g^UPZM z#rISkJmB{vDR{T#hI{PUep1g}Bj^9y`?gu#<|+I4dHfH)*M3zFlNTzPb#i9oJkLo- z-Hu1`Y}z=f->2lW;W3s(bt5swCqeT&r=4t{@5*~#Wxjali?3-L9wg81kCQmP?clLI zw)rRXKONJLd|Ue@vuFbv9GGDwUp)V#`&svPJ?1+#gx0X-G+7}w^ z(H|l#a_4+J$u=|L&7R1XuKq2X-Q%Ti2IQ=mJoEmklBi_`=?9NI*S_)MXq`dUWA8ch zPF(1fPc3{gxyIotTYkaDz^UJspRI3b*q{7l@vm+PA<6cGOi3f)TbzZigc(mE}v?hCbQd@7|@f+v! zD^g@8Oc&4pb>ja+N3lAaZId=1xl5F;>Ef%t~8&Wb7;CaC~wPs{URIv z>PJD<71$2+|t5n1f z`pL6y-WT=0`Tnin4}QrXe51Jk-^OzX^imH*Wya36Z?ON_VqB%$o0$D+Uc~ugyP{>Y zzb`ubdgtF8l3aanY&SfO5vVb^C46ME?bNk%gy)`oWBD;#{?ffoq37yOvn7eEmhF)bex9K&eYu+)We2ChA@7Qj` zU0bVp_x7k(l@!hEyUTP#;`VeI(Yzn#?@YheS6q$pv`c(#?R}GruWXWhYFSj<2c_vU zTh1p%t3PN^|D3+>`VltyQ*SOeHuv^)KMQ|j$*gwmi}#-m5a}-@1FVl)XwWCoR2lmN!MM}4;mBO z>ROZ2J|)q6Uz4Y7-j=o1yf%`-o%4Nn6;xlj{(V-j>C)p(TbF*C^ikpXkC*d5d*_F%w zm5(oG&V03djrot7hnrpRO|EQOem2Iq{mkSO7v{dr&y~Nk`k&yL-+Nv?oEy@;{=g;9 zQ!l5esBv!WRPnquW#YM+uYbIHc%ECJ($lZ#WJ=-b#D&)Dz66#uufILb@5;L3#AE!f z>}?+sKW%ud*nHxGzpc?j*ZDdBufIu_m+jINE)*#9{{>eA;AzSr42w!fW{Upw96 zgVn!D(~bWGd^&QjW`k$l$&2MHxW3K`+#}l7Wo#I=<9VHW^Noh{mv=l}=3%)*`8MxY z<=P8xRp);cw$6Fhtmad6y6vB9r+j~o!6xR#HJ>I{uJmcIoi1a!^mN{>TLphFnAbd- z{QcXHgq{~?vXv#bCG)%4e0_0h=bzNP-@4y#q!oYOu}qz>c5RPP@$D!1QvJ7@nd_$P zImv(b$po1!`J(V|#&RzeZcO~|z*hdKGq<$xQmbv{k4_XhMP|dcxFF$-u=@u2IsbIdmxBU0z>bxnHCp$M) z&a6FV^RC$GZ07HjZ*6ZJOK!}6opR-4vD#+62aDGT)=W&4&#zs-^VQw#Rms23ERdgj z{6gQl=Q&S!_EoOAsl7=(X5x3L89h7Y-iwC!Uw_FH!n=Csg0;tAUDD&XEI)i-{?_ZW za<^*!z1|s;Zfsfb%t-v@;wMM%JZR?qmvq{?I3xX?Y0f6`_>Y%){&85iRYsH%;f;jWbq@ZNJaz+k9(_^1q|cI{Ee) z#eI0#dWrpRoZ-Uy*L}Du2qpyEChM3uhbV z*hsIJWY5u-cCLB7eV(%F@#j_YcYb^qd*ZP_;{BedS1kPQJu%EZwKIQ`Ytd@`r&IVY zwau?8TCZCy+8^`haeB^|Tf4Swc6PIteLDB=zHq+s+jcv`YZ><6&z9TiHBV)cN@&vW z`j7EeSKM5WF-`Xi^ml(@BKvgh<5DBbuYcI$UwwSsdz^3QjZd>TZFE+vdCKD;EG@rr zqUXGRsUI6x$FTjL+PW|~l1FR&y)Tpv| z8h`uQklHuZ|E-*zaQ*49ySHt=`R$+dlv_3Lt)zCRw>8g4@wrCVHmdL6c)b4K;s&!_ zmT%JhHgvxD;MsQf!5NuJ##TQgbo6#!;^UV!`>}Af?7AnD`cG(2_N+VNnRjk!ol^e0 z&)#xubtfi1*WStT@5sH4)B4J$#Hi|(Z9U09r|1lS?UjFGJg-t-9;v(c<79ZzoS%Hj z&t_E|@h*GEZZ+>lgJP=PlZpM2;`8}>PTlj6T)BMi{T}n&qzlK=z6+*JwA%YZO)uic zSB=GA+;8nM&fAiD=VGU9*~!Y|;cu#z*M3{fvpO9#^>}d2^kZEc8_QDV|Mb59v-9hG zhnG6*pkr43H`1T+Kk&Ewc)x1z{cBZm-F4Pm^()>;pA){x{8sbJ+3#;GZ}M;6pZk9A zvhPneZMA-HdXQhfNmg&;jmHxt=0sokb|<^Y?(|HJ!cP-xZPJY_9V5A?{+#>0H1}lk z=hJ@Ya<2K66}xr6$r0vn+wwH$@Uc}kKVR@qxLnz%d&J!OSHyft{oUEN{^yFWJ>tGM zLE>)qkMe&P?K7`Wn{)74qwdE`j&s$2own@kkKK9wK$gvh*E5gzd|Y(CXnmttS8!*3GVf)*!wM>N8G&lZsOu2{&H>CR`01=9)GfnyRv%A z_Wehv?buZ-X}qy{`5im)oQ>COLpB~-Q~yNEFx79%#b1u?vPQkF{|`;9EB&{8^WN`L zl_~mv51y}Czw+2QcDeaQ9%WNczf#}%dSh7K&0E6XrOYPVCp*ar%q!$(epR>bpLgEp zWB&Cgr`ieom-NJLIyN`r#JRr?%fC*uoBsI5;^X%6oAvDGKKSPL!^8INpETPmpzem2 zfnrJB6A!OP$9v%44k`@u*1&-1^{vu}K4en9nb%QeXpDkm7s zAH@Ef_|0cR$uho!n;$I7N%>)RGcVd|^=w7deGHSJEsiTCXa_Ci1||6YN1hxwbv<-q*x9Uw4XcwB_vO zMqT`=%$D2}8M|*JTTAb?@>#)NcT_peE+I;G-;v!%PNqNd=G${pO>fsn*6r@I_hdWY zdU??1|F+rNUSuerU(V@i|2FeOF zlC>A>G^MzD%*P3CMb5x$W5?{)~w!q zczwl|jqKvlNgpM;D-L?<8+9J6Sa{m)(@dL5=NZ+0Chq?@<*$}WUmxqUPoi^*)aD)W zK7QEUu5EK?!b9W7i)MaRzVRoFMe^gO=^LA;f0UE8k(ku{FClV9V(zLkqi5Mkjh*$S zxpGhTFFGZ=|4m0QPY2(nW8q1QSL=M&Fx(_7UvfDOJ8aT7kF*~zyYB55-#OK9#aHgt=9b@trQ@cbW;^vG(9cgOzyIy0 z<$mpXzYOgrZ%;^R&-w6(y?xt@jh3$`OuWD6GJi?ZX=|&L$=m<+3#hhFmw2)9Xy3jk z<$L+-K7Uf{KXyyY-qh{M=Hs7@+`cTXn`FQ9P2t8K*4oCN`BT<1M{;jH+P>-IoPGbp zZn9sOKk)U+2uetr1y-=?o=u6ciIuiA5! z*GbocW^S&Fil4YOtmbC^Nu?Q6PG%`Do?~jB{dW7Is(C*qF~2|iebyXjEB2}Em8X9? z?t8NRZOFM_HhSsadW#q9y?-hG%cJ_1k=Tb-3vZwPu=LlW-+XPU`YN|?a5G=pr~O@O zTa~=t?}FHSd)d+?j<)usrv8u$pSt(q8B6<}J2$;EoqPMxl)`z%nmtMUxv%3U+jOda z+}QlA+V^Bo4ge>M8$ z9Gz!A^NscGUDwxqKls(oqW^Z!(z_e^--XH_c>ncO`#QB4G4t-lEl-$N7wPMnzVy)( z-XgwZ){oCe-nM4Xe4?f_SH*JMr-@U}-V4!5+Zv#mt<26Y-yxq`7rk7JqnoAY-|Ch3 zAAC(zlIMNSd-_p#)k4iRkuitb=lb$X)*Th#V*B=SbH$-7LB5Hb88!+3uXwd_;!*wX zW#yblT=pk!n0T?#*G}ip&*M_(tiRvMo2+m2VUsie%JYUF=d%Xa3CYikNZ{?uX)D>( zpeFt$Nqghv<+;Es*S$ghuRlTpT<%{kXK6c)pd*{Ui<>S5UN)H!4 zyuQ!ooW*>nYow+_#w)U-j!ygn<0M8JCawy?Sy&{r)=|L#q7*Zk~j*Ymo)t(@$sbFV{Y zpZTkw;kmi1D~k1QD0QE3KW_a|g8!I>Tx6w-X8ZYlb_?3YVj*Yl-pWva~I*nj`~!}!ND|IhT*SE}yGH%|K-v&P_deo@>D+b!R=$?m*v zk$!Xcru}L6H$A`o+UC~BxA_a-?5+tt{OZr8^sS!f?B*GVoLhBMIa#?{ljqdLe*Juj zZO`K#?>VBrXYcvxa+{`Sd;8q`)UNkBPFUs*>*5%8_1L;3?Ty{S=WjBceeP-4?Yi51 zcY*1(M;jX-_4_3C>+jmgoczx8*Cvr@`;X;Ip9ort>_2DmZhy|f8x|+0_0HKp*|_RY z-rKlf-r9R}Yd6W>`MC2?_$Nv6M&hG=AD0X z{Ex?fR{r}MM%Gp9J>!I&uW6c1{y%r;^%K&2PHwNTs(++ucd0D$`>Zo-wI9VTzHfA_ z>W!G*jf4+oYM=b-pXl}_e0AIXIBs!$_1qbsj;h!FIan3C7nI#k|KFEhH&cA(<-K87 zf831zc5v@2{{?q`-)R4O#IVw*Ot{K;X2Q)IRZPBDl?Uj+6QUGL7H@F2A?E0R0(W8(b3na&$S+pO6u4!(VR zGNI(ci8O)zj~r8u-1_q=(m}6PTJK1Hk?ii0N7^^8eiU|ob5gvPyTr_8)2WYw_OEUw z^u*>?SiaG8zc<_O&5}E2)0mx7_S`&^w(?#C_vRfz-^2QO4Lcp>j$Yj8V^uS0`sRf5PA5;U=AE$ihUoT>7rV26dRs^S zSK48<|JeI!wR|$iZ6EnZeZOS?V2x$+t&oq8)?bg=_aysqxA;6;yW)H4$L;5Q-S}~v z|NiVb-+ub97b}oBGBrlE{LdlXzP$AjPk&9jzWZmzmA7&>@2V!hmyiBt`{B~N<+sB1 z{=KTW+8%HJ?+x?uzT*8=|J?3Rv-7f>q;B)E+jrltxJ`dvEq*e0p3MWrU;maCy|pb9 z``y3iZu7Cl`+hslZ4b8ly6t0her@3COZj^znQwnGdENep%k;#59dG^=cI}w_BH`$_ zyNedf*43o$7w!9^xo6|jyOVeQ3p1N$RMzCZ=GGV2qNny6k0gcvtn8NUldS*n=zOtB zd)YploTrW3|LS~9)ltbS()byEELdDP-eacUucB-7b6xblrR{vebMML2J??uhX11T^ z-*)}?gJZ`#K|?QFk9{yOD=(3c`Oo&lX7{zHrEFz-%$C|`V^8J4+`{&fH^U`%U z=vn-J^JZ_`+UyQ=GBELZYzFYYxm%RV*@}yvN$%zLimF@o= z-&X$4^wx{x>h`ycrxk_u^Z2|o{j)`<=jrsmijNN+QobMAyX?TW*c(4wYSL8d+zuCc^qHkId{7L&tE?t{S@y#Kd1Ph#n&(Tz1wU57(d=}zV4;b zdsACJt@X3to1W|bRB>85uG;3;x$P3=d!J5BK3ymMKm6lT!<;MoK5g|rw))<0|Hm`R zYm(+vUo>j-k}s9leXN}9^5+=K|Bm%%r1O&-=WYJ?mH)0$+`l8cx69mXyP1D1RxUz9 zT(r&j{LX?omET$)8-LyNJb3=v=WE^-?>zrEG5nqBvHnU1`9B{t-mk0nJ=SegS$2Hy zu^Z=PPTu+8_I7#FE4x+uF37L8i%NOX{$l0i^LGN5P2M->>tl7RTS@uT4l7rO?!2Dj zEAqHX{?3hJWBadD_g#*U^v`zqVsm5m_k(+1nIEWSc+a<@?*zkHl}mwYOWmb6F=U-j zIqL2HXyen9D%bkf=Lwn_=1%yykoV<>7ZM_7q1=mIFCLrgeBsS(d1=26UJ;&>pDtz| zN6hS#xc$r~vP$ktP&7a8eSCB3>1o>zPiw!|GdFhM#2;IfZMGdfwvPXA%HOouPYdVu zhn@QJ^UAsKdzK|9R4a2kSI+(Exc|$G4UhL8y!V5jqoirJzW{d-YBCB7M6bLd}0ldrE&;{qpOV@85AZF!WZ0@Hw0R zYYJbb+nb%U`d|DwJAQwKOy!5t2n{WTk=o6r66rS0sFS}mwdjk@ zNpsnnuU|n+%ilKdulzhe{y%SlaI)`fo;OZxZ`!~0os~aO&AaaFjjG>kx6MnJKXvom zzU2IuwRtzoH|c|?uHJOA=k2=w#Aj7@Rey51A&yxT%U6A z+f(-2;Yaqm{hO0JVb8mh)xO8WEq&*Iy|LEf>*ko}J{SM)eWH{uk!!jAq`lksSYCILeg4^Ho1CY`Z*zKeQu$uv^&iJ7PH(PLI&1ys-|YT*hH({_+FzM()xYzu z`s~%$Rz+WI^R4ARGH+jHcc*B8zS&6mGbJ7eEHMfRTPnEw}kS?nw1{}BCf zfBh-@p71@N<#+Qhp7T-hzQ_L4t^YoX)ff4=#h$w`v0dKg-j~haUw`WR_v65Fd;7Zg zPxyPk|9>sM-%{*!`fYuATe(yHyEYlFsQ<#$`JQ$>dikpW5$lRmksT*)hTL@f=-8hlZ*_{_Zob@<9UpIc#RZXH1nvJ+K5Lb?w>$CC zGWn46mo18)I#oYa6r8sVo+qAOCVV-DuiwTx>5Ina)%91VPTX{7S&UobE8i8lQ)FH4 z3qMk?b(|+x^D96)c3ZkliE-bJQ|)&X7EIo`c+LE6Zf07)a=Py>&O4LJ>}P#D>0QKw zHR;n5m4S*Etz}m=TqVTw=I4K=f^(n+Ea1j`FRydbH2|X-{-EG)a<-_js&-F z>ft$UvDQD|?>(4%yj%Xmp2A~0)B9G(zkT&L@ZZk*&wmQL(qUo!dnJmJ*s zk0XwW$-jL4PWj)d`eQ$PiqB0iE8WCl_F17_e@~BW(O>!KQ_t_bpXpp7x39)QW3lv% zQ)xfuPR`0-^{r>=4Uc(0zdWdZvgmVwRf^NWdpErFUgs7~|LeBfIlADq;=Ml~rq{i( zTXW~b-H21^e|GJe_Q>s?kT+vdB)?nPrhRF8n{Sq#PAfRs&Mq&OYG>WIN?JeZ;N`YG zH{TcSo!ZM=w4n=>#KP0#XZqZVE}!VBXZGCV&ZDgQZdn_%A4&c1s^s^4`gZUu`v#p0KUtwSTY0l$&#`k`9)~~KU`|qsx{FmYK```b+Yc9#Z|MA)5U)OE@Sik?@ zCGGo`e*aEL-hE$`x^4#hlkKJJ4Ua!D`RwGkuKGgV;~()qE`IxcJO9t`kMl0u?LQUU zzx3DJ_sra%5B`oXmf7>(^w`AqTuJ+i*ZI=_AKvbJu2= z(|Y#H&3!u!795Mty=BaAeE&zr{?ysEuO-%$91-U#%P)$|KRH>n`>WUPZ4ah7@A>)A zzi!Jxl{7PH@1H&Q|2@5MX)1rQLP_eq?1Y&2&CC6yuSPKjG{}0)A(W^YbA5d=e3)%_4MmK2i@!^e)DT$tK>gOtv_jB_bL43tD23vW<{q; zPIPd8TzU8R&hI~#7QUbT|G|&{_lkY$8S4MMmA_d3>uF*A&nweX7Z=Bv?GapaLaOV- z!%P`(n?K8RYhU)t&z{&B^C7dJKiOd8rTkUfH%|KZ`I$6d{iCJN-H%G`k@k|de>}Oi z)=ivs_eZ~D?N_`mWch4b%FEVYwSW4)<&(`Sgt>3mq~v{QKC$S*NtzpnhEmv4K1-l=ta89y#Px4vx2ob9#D`#vNw->bZz(fUhpcS%|k z@7I~>ooj!4+v2|iof^mdu{CaHh=E+Lub<737q4yY@fV%`kk)H=Yw)$pB_0k zZBxv>-}W?2`tOsO`7QG!rfmJZ{{NMu=hZ6A6)mc zG52?PfA#x{%k2M`zFq!b?{i%6{+7=V-tYcd@oR6M^|zJ(FMt2Dz9!e^-BI;^{@P!g z-@f1U&eXpC{q4TR?>@-BeJ`9o&v@QH@%L};|1xtEE>03r7rHF?_2FOtx>xTWyS3*_ z#_#EjRky3;>CNXBQa(G9UbQoVAYcHF1qX_guCNyd5~)&YS0S1Y>d-VE`Iza8nFj_a(}Rc)SH7jSNI?4FD1+#g$~m)Shs^WXMObM@8x z4Bz_`UTt`{pz5{56vqXJbw60$e&~??%y8z`%h%VfIs4@7BI(<=7Edo{>tYnMtWp!7 z$LSq5bD!0t>3epW@#mxj7*tko>PT};kFU9xtl9qORr{5Pm*!kno?m8n^7y`^jc+aW-fH|1(zcs@QuKd`;LOmSH6^zv$-jM* zy}e`7)pXG;Iic&yN2iH2j*8k%I`6+{j>gdnS7b{w4SNF%G_y|ac}PZJ9giAUitj= z{=G$s5|6|3+t@$;jTR$G^JW~8z`hKF#|9$)Kef>W7@?F;Hm~i_kwzxw}lgpc&^pS{msFgea`vpJ(5VZ8BRv*2~^t z?n##TG5@WGat#Z6t)0!pt7`l?3f~fyH-i>pBwP!5e0+yNURfDvD5=7Kt1f zP5%#^?*I12IVV=|`>b=GM-S(pJ5qCaTEyJXYcgL=FJ1p~v(r}P!fyAN>80y0`sMAr z{^5Q|#S_(?9(8w5{;CPct%%-z{lmYmTjue1%M}#`mlwb1VQDYR|5UWY`km>?{SmWH$=}N7 zzn5(kcXV<76NfdcW5o5ZYzlq3ZKeK|mp>Th-DjwOwz1y)-j7EA?K#JTH*Uz!X?b*B z`tC#Pewp*tFZg>7zjFQ{{_1`6{}07|YwgeO_{ccJ$Kd66>+AV_|IZz+U-0!_`K{;2 zF2{ZQR(SE~eVgV?J>``*uB?F>{ji6!t^ry@ANw+>u+s)9_Or9yEy;mhm^cy?(2<3 z4~IUCW8VA+H1zbW)n?82gS~rgUUwBv_!ju9(@sy_!#bUpnLl&8#^%=i$IG%4 zjWRa>oU}QkQrYO#Je{g9x=JNKyKQdol@Q^}xB9Y~V_WZ`!+iCxz9yFzByF$kj^9>0 z=jGAji;DX%v2J)JHh){J(W6KGc}eQW$}}o&y1puY_hjZeiCt@N9e&-)_pVEu-Kyx( zlE-Jm9)0MO-S;5iyV`379r=Ul#*MO1vrqfc5nEIuEPV6INv-PY7bWr2{HMn=3i>9ktA`u~!NW!rYgywACn)0S{Ry?c^;;pf|yy8AZGQAm9}HGba| z<=gL5do~Kz-k4y>AS|0&CvAQ^N#Ca84V(8drrWuT|G4SRxDoPg%O53f{oD;_K zKP)5UcKug;{N(t)Rd4NIFS#$1|Ko*c?_T?YFq^rHRoR?A>DV1ntG2J`-rXbq??ip} zz5M-ezaKws|6_V=TIUKS_KWPnzBNQN*TS z-M{Z`^Ks#KA6AI{y=Y(k#NzLq^CgzQl7i3Gt9a+#@UA;@rA(V|--#F9KNP-Sxc%?O zkDJ!^g@690`^nn>ZA<_1RlZ)|W1RZw!x#6jm%Q~p9C7OBkxllv zIpfT%358QPuDQEZLt@WE_mjTQRBs-6+WX;&&$~npe{1Vozmv-!I$gW>L1}(Ocx}Mn z>HntO{^NW)^01Zs)9!y&@^^MzII;bo(e~cS@8&;xng4U@>YeAG6z%2DtKpa=o*eV< znPJ|ZQ?s}IubRC0dOdTD)3q(<4(O#E*p*yA&%VL_;nw-DD%0-nT6DcQebt+fH{Y*e zKc^FEzV*9Z_@19}!Q5Hz_kZ5C>di(z_y3#TndaOyKe;i~uWIedDBJbVZ{6Ix=Ib2$ z^qifceY}>rx38vunJeDpA9HQn9M4N@OPp_={C;!wj91?>tqN{;KMZ-dEsI}&R}s6p z((-v*r|Ev1SpH_k;f3#f)zfD@NDf|eLbA;+=3dQl_cxOkOUI-HtFM0ZR#RDf&94qV zo!wRICrTe_-}j(6`@2*!^GDwO7JFZsgx;+1jQl^t`d0b%y>91z>jXyJ@QqK^IU#t7 z|B=W2N1f}xo_70iW>Zdv_~eMo=N}!l|8k)Ac>MoI$L2FsUR!<7GVcHP<)&@2yRQFf zpWP>J`$_Lk-~GDBzmK23|Fct1`um@Qt^eM1ewWz3(q8t{)nC&0zyE$~-gAEb?mvsa zwldEXnA^AWhb_BT{O-+vliz)r`tBuYSO{wnvsa9U`rP^)^kL=@Ko0)4kC7V;82LBc$guc|6S$pG%b|={Gr?Xf9~5j<+z^_@0+9dU!PHbZ12Bm@jrvyKkdKAUY;Xgb65D? zmh!*6AKv|$^}{$-L4v_!uXwMNgqr+5^=?o9O{-?FQHv_kSvK+P64mTSpJw-4oO-gy z_Rq@d?`-j@E$N3pom;l}u9LsMT~UJZ$B)hZy><^S#1}lfu<28{-Nao-HM-pkDzhi| z8{OM<^hQOvZIMJ|#Lx4Joc1%GaBp3IbXw7;OS3Od>i&`HZONVd=F;KG%tjxMPP192 zZ?$~mPEEV24j!+)=8a4BpSr{td=SlA^Hg~M?N$Ci+}~`Fto-(j_2!|qXP*iGJr-GX z^TW}Z5??K)EtgDs+T8o&Nbonc+tREvkMdRS+j3rB^P#AGwB+(6weF_n5^GoAt2@PB zS$mv)ck!N_3~~222PS+xAs2mCIR2@1WnSyAgZ*b``210ezhyP=($e-TH!p_SSM52) zTGM^tn8iA-*)KjY^Gh%PGriZo@sqat|7#X6gX?b>N<>>mbSv6@*?lA6_sZ?t#P59H z`MA&g-~RUcGoKRce=aFJSNVKP|AW1;wHE&t%m1Hyt@^IJzr_BydT;5roAca0+f=@( z;HsDR@Vdsn{k2TNkBYa-eJ|^6KDU1D&zJlBuXlg^y|(G{Umj2Auib3wHGk>*%i#ul zKPmptt~vhDT=snB>CeYs+f^mq{?U2+&xs#T|CfCE@o4&ax$m~m5+6@mpMS>E@)^I~ z(>0M5FOw}M>DOJ|`nmP}LGEvIhr930>F){L^vmk8QoC7g@B6!jk;P)SpRwHklQsLe z^t`es2FL7fJ_!DPVq*OajejS;?~2~^=LdfZN1fru)pK_2D$G2dE&uGKQQOw^wH>^5 zN8k3|2$}bE*TT9pMiVcK*;yZ)`mZnVm#cM9d!uZ`N86l-)vIS~-;94YC)2wp<#KuC z`hYRh!Pt@qf?|Gn3*o&M)7@9zi2`~GIW=6_&2>pJ96l^gwQ7K7HO z)?MEJEvg`O&GV-_&g(=T|9tOO)w;Erm#f~{T&+%7oB8JE!7>RC3Px8zbg!Z&+jx_-Jq z)$XeG7t?=Pr1RSSe0)DT{v)S*f#%_rvG)Q#boJ|gUfL4XrsKtP@vW78>ij?F?qvT+ z&3|Y8({JAG!oLdJH-De?V?z6xa=WX6zi#C>?Eg8veof{1s;|3`dCLFW-uH9UJJW(w zinBg{KlJPDJL_xt6~B+`$p6~^uWU~7yUMrC{^#H3)f}lf`M7?^*SK)`oBvxx!^~&E?>~F2(zF+x0?S0_?AJ0}kUYh?u;`W^S7mI%^72mVD$LJsLtL>-m zf7tT7Z1Q~u`=$T3{je#2(l7(1HY5GL>7OUb z7Jpv7(NbSm7GQs(z`*{5z)$8kF|UF_dV<^zgKP7aQ}HI|Nbir-k+_DwBV@{ zKHhR7sVO~arE;n8wWI}WCYKi6x*g}cVC~=QI-6z|loV&2O1Yl)@O5W|#I7^$KewIu zy1QnoL;kaS7tbmQz1ckXSon+^AGR<0^zm4eR-CZhMCVT1+{iD9*QS*xZ4XecVR1fD z-#f>ApH7hMW2N)<@h3gs9BbP)Gr?ou%L(Tf+dV$_Xtsp+2H%sck!3>L&85Vx&Z*>0(&VEzT!&-KONAtC`)#v<4o39pi@4a!L`7%?|9u+>DDP_M3UIkCN z@gX?P*zRUly()Kqk--tO$|rMqLnG@xCGJQ)F|{svrP&^vgR@V2e7N}BeXb7A%@4xd z)t~g9znOgZmWBFr$7ixN$8tWX%sb|kchLLZjW5y<)a`yvUG@0p@<08BkJSG!%bn1! zYM2sJy?HI~O6~ivI%fAL%Y9k&%=YV#c*%VGw|8~b?LN8R|M>B7;Qh}J^1iFP{QFq^ z|4Q*DsoB%zPjcEmR!{buD{oWZ{o9IN&d~1X&w9D_f77DKK=T)iwgUv%b4!_)Z4rNV%Y;zeu?AupG1@Q*M5_= zFaNW?cYEEhSlQ@L^J5Mx@qOFr3|Z~wNypB~aTEB77W z7IWaz<3$Ar+nzM(+2tI}^i7rKS2e#^Q1#qCW~r>c)hS8ioP&w7AKMIHvqX2c+UyN5 znCPi@+3fVi?+1bzcD?7f`|;r8(_h=wlI$zeY*Q;9LzeFL^jGH0P|hfS)BKwM!OVlN zzgsa|&u7fux;N_|!$0HW^Y>l-dS_n3e#8E?U%#=u-e=p)Te>7~{z*&o>d0Hw^($<% zs^6LZvpNXGyT;1gn#L}n$Vrs zpVTbbf1zgEg!#{|T37$wcJ1?%OOdvJ@8|zq-)`H#@qk~=r-gI&8Ma+1Ube#Py7c)Z zXY1KJudDPuv3mO}X7*?4Qx!AcA7Rbvzmxr-b@9e45gvc;S>AtpYVWZdY0){+($7EY zzRSIv{iHqT!=(4szJJ+%c-sG0UiJ9q`pWYb59R&d8`eaoXFr}TYv~@T|8M>4_-vWa z|L^Rrbl?Bx{juBjb;p+evHd!4yKMii(`$GAJ*aP$So6p@{#9L4wpyWD`?3D7>d)gU zul)P5`|hs#;@^|!+gATMy8Hjit54Ocj!iM#u(p~v`jP#o3%aZKGyXeseb4s)EMC9X z9oz~*9v^d;G3on#f4^2>5dosV?iPc)T zx^&s^y-6O@Yo?d4pSbu={F76u`OJ`k#E-Kx#z3W>#CIhFkMWwZ}Q|zn08uV zjp8r4`VJ;e&HZut_>&J4HvRc=pz( zd;7zpdq>pv_bxS<{C;_g?j((EQ;ol!xX}4$>9O38FEhR`_%eTA@HIX6y@Jzq*?Dau zKHhZwoOXM5(Z?evkNsi%6Ylr#{a5t>znG^DbAQ^0oc$|ke~k5C-P7Mo+~uwLZk3<@ zarpb+%KwKa7R}hR_sa_L-uJuS=^kgd|NZdelgIOK%4@!y`0-rDss!^ly0xlhe-HZC zOuTsg))9fAcQ?GlW^X*ZZN^ty>tAyI*7r9~G?G8R=k>+MjNAA5|F@K=7dvLb-}d~@ zU&+UI+kY(lxUIhKdPUS<-p^kSpDMrk?j!5$jT?JdCB8cU*0!7E{MI0QcJd+h*YbRA zy6qk|$@h2XhWMO}|0OLpqqn*!>H4*u8nNF_Y+0LiJuUXsj-=^%rY3G1-R*ZC4t`gs z`R17L?pHa=-@cle~bOY-M#Z>@3%B>UelLw;!fE4x;bUl>-Q?xlx8>BhF0FYed5OVt&fjyD?R#a zlkIn@D$B{wa!$_M^f%z*folasqjtt zw(bglRgT5v%J;AKw%Pyu{H;USue$!60qd5}lRt-jE1%I6EUEVONb;PMu}>zO+uYcj z{zyNWA%6ag<{LBLn;LY^-Sh3nyFJtO=GUm$z5R2v?wrM|Uo&K^JtBYCtT!}&W_;_~ zJJT-$)~96W8wNyt(T)Er(sz2-^)ptgcRqd65}$i~)6|WV_gs3o-Y_)s-_P4itIvL6 z{1YDc_uVgHjrp&BYa9HhatUGjUcXjgP zJ7?x^bV;jzXL{W><^R?EhUd&X3?1bwU$Yy2&cCxXp6|wYsmJ-ReCOomJT1pc}^No7No?ynRZ;?x=UX?8d$7?@a&sKR%;gc$h=ji^n!jOD*ET zf&Lz|&%X9AtBsQOt*Pey`(zq_sf+oxtBDVl-xi$-|M9BkrG2WziS$2fzdJ_WnPLCS z`JK_UOXasVhrO6QFaD~0QDy3z=KPvVvESz1hioW{+GAYd8}X#Mkyl+f?ojCgn}sJI z&eXV7>L41{ZF*D3BIV?g$0Y)Hb*B9}Gg*BylrVhy*jOwD%}t@%G4 zk_09u_RrhIxO5KZ`DwhZcJn!&y;ynqAosTu6`SY&vRV1@iBp_$!q&;{mZ@K#iCzmR zns$(Xbvk`RIVD)VbjzuYe=@!2d@;65IvsuQ#$)U5rT^dZ_nlCe zf6^?Tf6Z4qMzrR6<~#nK7oYJ?-lJ^sROo55!u^!5ia8(Wd1>~rpB!&!+d;&b~q zopd{lTjI~l<{ut+rOh^NzMt@%bd`qQ}Y#iNt9 zZ(beL=Kr%?zqwgY_Mf2qtB-|89NYVY>xv%xFZkhZ5dVLo@b3i2`F>AgkGy!9UKeOE zS@<~jtM?7}pDw#^ey{k~@8cJb+pkW25wmvJPg(zp!#j>$U2hood7^pG^Syt+SD%?7 zU-I|ZI=e~o6+h2dEs+%~`{BJOX>RdV39soo(w)+AHv0}Gv-5A9{%7@X-S_5pk5V5u zWWT##|NA%d*Msx+{8?;$_sO@%%k1_nc#&*|)VuIkmD&f8SKW zH4{}I2exy{&8W`Pmxx~ef)9(g}PzirUW=g`Zw zGrD}${?_8<_ER3uo_|E-?}<%E-Cm_!we94Ndm9;^c4y+{z7q#H@3)G_T=>iBF2CiJ z_@ex}&8OLUJ{;X=*Z^L-a^{DToRQrWXG!N{=fAyceqGOEd^39gbc5HfQxo+Ca@>gwNXp2bc6&GIwhonc7fo(B`lWy^Oyo%q;x zzE!r((mHOLi+g{W>q)+^_!d~Ye&?-z@C+aLjD? zOMSDCQsp;YCU*8;x%tSr&}`o-^|Desqqc{awk`fX%jlBDgMR*Pm*QXYm-;_ynqG5l zp=9}!Ey_j(hqmv2XZkGs*~C5Tz8~!Uz2o1^Uf6fdec|u-3JpdD3m3gK635| z|Ng)F!t?sRo%rxj@i2eTw#WzF%#ssceG%X}buoMHD)#m=n-+I7tBRWknlCz~9A418 z+?JaowxTFwnY_|$=Du$?X4if`ZD!5Kz~R5=$&smNwdAvs<9Bb%Q7d^OCx1-m+Xw5E z3ibXCAD$PimD#Ivq2|Hn)R-625r4`U=b720T6GFf`*C4fjP~UAM%gJ6m)}dyf4ly} z=4igCu(}^-o~?M}c+2Tzx?R=W`g_Oj=UqORU$n=nMnqrQA}43l#eLrHd|$uzSpKv6 zcIhQ^e&XLx)AhA}o~nMT{XGmPw}*A1^tc^GEh=eD&wg$E5AbJ{_~roBct#?&aoYBmEqSN3ZJEeN(^x za(~b9y7SJ*#P$9>+Hrlp-IC)?Hu9Axz8-rmSLgHk*Y3OZpYBaBkCa#Xc`ohDH1_^n z!;%{B{mb8(ANV1@de7vE=3ifEKhA8Htn>e4`fbyC{}YvctM-5X+g)Dy$>Q^#&BvJI z|IIk{xGmN);eErQKd;#Lr^J+>-g=z--#h-}8|7(`s5fBTWn(`V{pRd2q{Qr>rpQLZaDb=$|6n%{$~?#*0$Yl6hn@aPU*emkEOqc-F6 zCt?SMG<;IZO`K_ zKYth1ef0Usx?}#|JoBe~o^96uSMPbc8&z-eM+`cP^=XCTjZM9mFo?A_qEBbWgerJ5%+m?4(h}*X8`?BiZt>J5` zKgA`>e(6q{{oZtMT-dv9-{ThRgG>H9vA-Y0eU*N|U-34%dU0uof#8JgtjArvEZci> zN`CH`%->+QC~C(a{n)?p=e({v?TI>bDvvKxEaKZ{hO|VL*MZN?yO*|xNw&o$tj=+E z{vlTJRYvPWLVEXHnMp5JoiBP=8NKDmb-Pn$wW-{^(myXG1uJTovAb5Byd;{uqI9+G z|Jz}Isumw{pRUV1!S7GnN1qzL@{%Y0JU*v3uYI@v&|$w%0yQ7p`(^4%K3+YVI(g}N zyEQc(k!!xlJQCtQ^G2=w&BUwQ4^G>0M7a8(8(Y8rznjOK+3k0HYHPIrcGKeho#OCX znP~Pa_e2VbamRo=y5S{p9!i`;WaU znvwN)(o*MV8<^LXuH=89e&s&HbM>+l$~UGKOTLuoT0ZYna{m7dgZp)(0hx3f$K0XlNJ~w^Nk2@c)|NnAro9T1g$NIG=!oNFawjOu*y~-l+8JC=- z*%|4==Z}`06^>8(+_Cq#%gw@s*5H3}pBnB^GLT{6Go zw|oEmJ%0}$Uw+=M_Ro#^_RF5zRM*XRvA+NDv{}w8j$<>0q-`yq9lkZG>XW?eV?O<2 zfxPxpKOU%C{f+yTe@)vUmrsm8_DPf5{RqdC`-^_-SnfF&`_R|t)x?9{wtF5ta<995 zOm6QFgZeMaKR(HyHEF4{pPo(0$r;^$PHZ-sWSx8~>LK$E!wY`{%U*PK?*DP*^qi&r z=k4ZeX6rNDmH)FAv^(!c^2Hl#DkoTG{E4||_v5vB?WFD!s#lKB{yZ8DH?oHQOHvPROHDPZ~`a9FlJJ-GyQ>*T;U4DLQ>ivC_ zH-~=a?*5^ae!oTechR>sYdoI>d`sTg-Cn!toA<9r-ZS4^duMv9R^po6{+ru>_RDXZ zv+_sZ{d@D5|9Ep4lH`0hrt0hEyqTcNoPApS--}LW`H6?am!IbU`|j|iht4v`{65W$ zIlQORR$W^hl;)Jn{)ldKwYRX|f3&$hr{L*XvwG0!wduBULfZ0w4j;`6-aGZ<&#xMX zz5L!BR5q9Ucx2k1Qa`V-e<#xXCeD3t`o|)_vYWkhX5P)1z^%vk74o0{b}#$IOMj~z zWt+kWzhAH~H!D1}f7kV#o9~0<5BOVr{c^2S_7g{0YOVS4$vc<-EiC`NuCSz*W&e(M zLEG!UZG7v0FRyHF^YM6_mvXt!CD+$HxA=K@`TC;s%jRrW?*G*OJkb52rh56QPa-;} za&pe^Rr~n4`fs+Zh2M{(>;~q3YxdXwJ869W*^=v`6YC#Kr*41ce#Ks_YVNPxpReVM z4(urXt0}IOUh}f)BYS=As~@M!>-T)VyI(bar>WS3FFEh`vMuY`eR}tGkrnS2uW@{5 z`iQ;De`j{orxiJS9+kgQZGPfie@g$XvYuM7twZ9;sS^#~ zat>r2o_=(*dSh1in%093?y-82+|ovYkIuI#RBgTJU9UFBN9o7OR-Z?aKYSv7oLrP} zX75Jd8!tci%(vm0ce6xsvTE$jGZ&Hz%`ZOwo|{YrI8NJt-jLI zy995S9NU*(rE}+{rGVB zFyFs#TW#x8HoS9)Fl)mL$Ajg1RtzqxaMtHp`7N7JA8MV_4WygVuP)GXWcw{Fcp zv~mt_>i@=ltDH9+^Lr!}e`FR<%!$2Ur^cqvIKno6--k7_A7%Y&US@y&F>QzX-raF2 z&!74)_`&;Ov3>oap!zpk=O5M2%Xzi`MbPrtzq#uxK7V||EPwXOj(_%dH~p_z^69F6 z>C}(<+y7P{Eu3%u_o&6o+wvzfcpZ`wAim_V>)c$^7__FZD-MDM#|5$!} z(qAf9BJX4IboGt&Ihzm5mn9TD*O$NZ@_d{Bxl5uwaeE$Zet2GO)9+msPXz4#9Gq_F zCmVe>o~L5!QE$nO?94GanwuBLO4dKtw=Gz8{Pw2FcH5dFZSBr~-f_`&n!d;GN=E+= zn%VD}t8ZS`n|AB@uS4?pn@m4$eYv!~X8Y82A3n9p{&h~DH&yUo!lkzU{h#FgpNTw; z`|TzEs_5jpYX^@ymT%?L`xJ9;?LUq*{)tzQYcWNOFn-=pS^JOa&%yTkmpa9tmYsY4 ztSbBSv~$mI)?b?bwd34o>+3Vb*WCZGXpQAgafR-{>zCcbubF4BOn;ql%_+O;ns(L8 z$oo6n^lJ9XiTl4l{@QQ8+3VR`&lGRy_`TC|{^4`{WjE&hUbuGSl=B>UHlGrrZLfE% zH@^Stoax7tkxo97!e!@A^4@=Q`hp`$zoj=8E!F+$>VHoC)`u7QQ+{k-*vGr&im`3> z;pTYD=Th-j9dm!~bjuS>e%*a%*}QBa{=aA0>VNvlzuLGurfkc?yW94E_@o~2hi!*_ z)5>1^-sDfgb9ZcgcgSmcS?QnP`rE7Dn$IixoxSzmYum3A3XdInv*XwMxSTyd&gb3v z`n~cc=i|rf_3KZUbltyZb^So1&$&+djf?N?$lRyNR{efY-kz_4^(NvsSA6>JZ=L+- z_HTRhdvW@o?f>`wYM%Gh{rH+kecQgo1fQDrcgw!w-hFBLwa;grYyPypR7A|?B;&sF z{eG34+2>4)F4#YKIr;PBoo)6n??0)kc`SZs=6CawbuSGgD>vX0hV(v?sIm+5V(mcgZi_=>7L-g5e==wThQ1%wknv!@v95wl^{RKk{C0 zvGCpPO;gMMY-pXX>r#LJw0mJFKYv`RxMcWiowSITPo?cY`GkwHaZ5zjblA*K@|UfB z$-U#_hzk1ku`o*>z!DUCM)it!s?z^OIH_utDcGHIqQzagx zA1~&}->^FP8NW~Gu%KE>6J5k zS9BovnESrMOFQn*mcLr4q%8jN*;1Xq`uqQWKF%n<|53$h^*)*NHP^YV{0^>(jnn|oTH+bp(_0-zkzmosm*MGcwn}GF`A3pq* zhHjGDr{cC{A8!9}Kg*cZOIWfxsT`mIK`Ko$vyaV z!g{^Oo|b#&9#{2_KR-E1``c6Qn>%xhlG@6_l~E7>PX&FWd^K_HmsZ!es?R^9JzcEVX&!>y0tzul29- zl(xDa@mNs0B<0R*>laPeWfEd9uY2Qld)l1rN7Az+53ens_mfTfx8?O4{%P?OYejrE zOi*W+o;fG`fuGEp^^x;WYCcvyegEs+><<$T?>`dpy6({ezvyqAZ-Z?QUy68ruFdw` zu{qf%zTflsKQ-pWpFInI+EklvJHni27}6#xx8%>B06a z>EE&{60Vx5R|?#@yFXaIBz{`#r(((4hJ;mokx#N-J~6MX*tLK8K4Jg!3wh^ePs%Qi zt4ee~_2C5T$9BiLdkXKbDfZpVVE=66_N>4Dm6zF%1Tc_tIAcltgE`4FU$=PwzBxBrZ`F z|E7I^dtcm6+|AuT_gZYmn_Y)@{c`)gF8O$A_vVIg*ROVecrO2W^iSRLJ{vaI z=grGLv%B-`pP0!qt@cwQy7o(?#a)`WanYP&-)qddzqJ2-(EWHn>iMjodpnP?ok^PU zSIOs1=hANvCdf-yuk(z%Ah`F`%jMU!<4=|SdAHH@>uL#wDy8Lbylh@XtZnYz6Th}O zeO>eIEBOugpM;jbIBD1UMwRzfqp`FM+r(X09vdo5>{Mlr*xk7BY=Ys>igSln%yAT{ zWt(D^EBRyI<_wL;Z}Xk~K7EPOP?^`t#3r0nq06@LhEHtH+1t$%=P_-H`S3C0F@I(1 z12~D{n2#R_S=i=_Lb-6`lY^|*vJ?)P`? z4*2kyJ-+U>pS0XtF|k_ty%oa0b(C%`GOtyNd(}Ku=2OxB8M|ffrcCzkbiS=uS#)t@ zPvOsNMpv)>X%#$rKj+M@O?EGwYTne$NisVwuT!x};eh1+6PwS=$-jJaW7_$7Cu)8L%SYtzIZJUV**Noq~|WwsUnH!l7)nNeC>#F0Qo_oCT z`cD7+4eP!xik|XU>b2zq={KKvGv>rUC|Q_L z?mz#cZ2eEExIZ7`V=JaCX0QJw{=Zc}K>FFC9jABAJ#KvO4-* zd0F_LqZOBY6%S=WUj3G2Qrpoh$9D-nPTn zcX_k<|I&DR#Jx^U{$qB#zqRzE<@<^&53xr-S~F*c)boni5``)wx#oirk!zc~x32mf z7_)pw=gb+eP9O1Jb9HCHBX2Q&ul-t&z5h)8>KK|8wJ#!Id2h_Sjqd;RFIoLqqis|7 zRaoQh@74XGEN1phw&$<^GOA~++izaG?K;zI%}su0S7nm&X1*8SWVXWnnZ=Z^hpjq~ z*DT?UzxMWL&9ZxcL~BpIIuV(DX6d|>J9d9KciyyMW47+CS-{2roGJes_c6L-+?~Gv zVN5f>>-OhQf*v`v-FtW6v%ThpY4Iu9+_Uy;7q0bRcV%i#^77KFv2&jC+L<-S|NeE( zw0HB5rs;m)CMkcpxG;96+}j^X&atOnem`MSe{>>`te)M(@MqaMOLObib=Lmc%>U%Q z<=pHSD$OrkH(XFJtG?!6{lxsh{6EM3%edE-d}NZjuQoq>UH#9G8)nM}e5*5E7kQuI zf9hY6u9{ob%e#9PFP|^@^R)B7Vwuky*Kgr}`Tj`5*`KF%u$5gx3$KU(e%jR^HIWso-tJhfx*ycRsX4|y7w4m_Yh3`qqX-U^Z z&KG6L>D%Z{`O#%9H>EE0*z_OT>i=%NIJ!E1i{^8=>M!%dpU*PlyLPBN;`IC;@noNM zC4SS_Z#aC*s8#S>kL=Ft`}`{B{n^4N_qT(2``q~smLIN`=NxBoRG(F06uI@Y=<2Pq z*BICEJ9?-;>kBtv@t#|)#`E;W5@pXN_gbGFT;9%CdQp>ea)8iTmua5*zwVxV*xh?9 zyfk%1p}Jz}IwLzygXx~?o9|bB2<=I7m$fNbaAZQip4m~8iYuDRxu@10Yuf&(^TdVj z`Megl{I*seJOAA$OD%8PXQwQ=!zU&cU-Vr+J+1QGTweW8N2L0FJI!3wWB*Nb>^tzF zb+%Qm#=eO!E2j7TSU=a#Y)gCi-dzuZ{KZyAOE2HK#%$l@c5SPq-BYLdPT3cDto=vB zjU~x_nl{C;pTC{>sQGugZ{NO`-#!_4XQkZTd?Vap7QdOx^c&mvt@OQHZZ&4|I)4P*#6k^E4%fV_mTB1pLhJ0xwUB@^l+=*mJUnHx<}3*a_SG3K4#rs^=pNk zoPYK2;^PnJ+bz3&<52r@!zpu$K5Wul_xZ3F_cixlHzW>=?|kuk--=&Gl|jmtiQx?eVW-#*VcZCU7bzw`5+ zcwuKFy^a2U8tU(U9Vq_gZ7;<D6+d-nNTx-9AP==CCbY2d23m zn3KRae?P*YPGeU@vdOMlLN z_BpP2b#Cov%iGT~x9t3Ex!3l)xZaeXdqm9VeJ-*qpC%Y=H#hwGtS|A`tT%edTw7Ot z*ZuN6_u2a<3TK};O=`dP&G*Z$bbA%`7X|z;nUA=YoVmTV^4heo5qCbkKA*RKPUHI1 zlk<#f94>0-f0|PpbL-B_)5brq1sZ)~Iw*IDefRE-J?CD=T(0oaw=MxEmLKXmb;JE(T+?rC{dJ;l<+nAZ zeNK1%k2c$VyZXL|zvkiWyZtG980v2?&cB)KZ}VmUvBIyD)soJ{)m+n%d&yk!hxsn+ ztAF+HD&7{ZUi~&+;`ol^_VVU-U+wS5e&dwF!O4o_?HgPukK8F2XPC&}JfwvX{&`}*D4|0=oPuk4Zh z{fX)St91VTIcx6l*xRh)ZScp-v;V2^OW)S}YM@}ZNta)z>PyAD3Q@6-50#r7{~vN- zxB27o&m_3^f-n0&%XqO`qg%rIHU%G_>gazxRd(x1Q`x2`x7==?T(sz%tG*0ns2l%FTEHZ{^aBG{a01* zB{}z-?ez7pc<4D_b8_S_^R23E@4kHMm8%iI{dUpm{095Cvgf7UYhKOBYcK9x`(x_! zva_7Oruwnm?#}-|r|_$HTx!jWC;dI_vfp;(dF#oPoe%G=E!+8{;%jGnpZLE2OLLw7 zv+&!Q{Ydz8M89s)8P!ucul_IfIk~d>Q1AbmI}g8$&9Ip{RHBw1{%2Yn^KjUYonN zbY0?PyKQS`UR(WhUfbEY)md{-%N>f^8S~rw+njTzC6Bl7%RRO8SR8j`^5%Jmy{22e zeYaO{I&ayOs}G}hUk~VSuTPmPT(x%Hjl5So+vKL~^_rKMy!-pL%fI^boj+UZJo(Xa zjy$*cik)QHM?>7zCq{0zb9|b&A#-#@E{F`rVQ8#{Y+OH{LT|8%ke2LO|{eABn zA4~myZ)+D~ulngg_`c1zuRQ)}to-}UzRML~tL=^dz2Eq?SaIp*zq3C)&{lrS{O`xn zxYzT)-9B~etNqL)0e6M-YQGj9U;V!SY0gX2_y7M?B;L%gK3e+mm-39CydTu}eq?@p zLU{iUneHhjeTOd8i9b72EE# zU4CBd^W;I`_1Xx_1IwR;-uk)L_W7&?<i`}*L^p!Zda9iNEqs=|(OMKK^#En)ie84yD zhv@DKBSts>FSGv~$?<#jpuJm0a#!p}d;NKdCN~c`@b7)3dTe5F`k~P4-3}j3`z~qs z-P(Hbyjtu>-EL{Q)XCa8^FHoge&^e&yTwT>e7?V2KKI(!0v+|rU8lrsSu=KWbhK9O zyU6SHZRW|KgL5|51I*g?#II+pXu6DDIk2)xE;nYn{g7>k`ZV zG@AZn5x2DIu)R_n@$2O!H#Lb5yLZ*k+HLSa^X82F_*P zq4#lLc|Rooe=}j*GxY#I>z6B*<~%wpzvD&8!3F34uqkbtkzan0yTAO-`?JY+Td((q z*Z-LCk^lYA=f`AsPQUZ7@^RL083Vh3`}^unJnYY}KXH+F@}Ec3|0yc_{Ymq;{}go)>@DONeefR=E9t?eD?|sreE2Psa6pUiT%?Y?u4%xDz zQeLI6b9O{qczve(t=~Iu)nC4T@T&a5H+u24Ym_#)3K~qmt$ilG!T!yO^vKcVfL>KE#Nj{9ahyLg@9 z^-KA($>tR|gYK7cIh&fPeSH(PdGobuw~veEUmgC{mGG`QahZN$cl^d~_IW>^?!0|N zJ2vI{mC~d7VIQ66X76L0yzufbU-#FV^=8jvf*oI2q|_^h|4@zllCt*Jlw#hwA3o?B zhAgYDeLm~W{Tse#go-7mylhl`F7@kD`YF9V6Vtut2<01g+3#Oh%xf=we2V@rfxL~+ zlxx1Vyi6|t*K*>t>Cc8s;jbQ<%bqCy5xjk2F>l;`2K#p(w`W$`Hx}Jsx4%ClMCIPQ zUnhSoGk^SD=lLw|*n&NuJGV=}e&zaM_5IH)KkiZX&$%zF*5il?>K?l)q8 zZO|~e{&D`FUlxBB$4Af0*4y~I?{SB_-an3K!uLM>`&TKp@!al)FYNnozL%N(Ys=%{ zUGYB;%NYM`$lU-`g&r^QnEl!67Ae ztF%zN?dMICSd?YHUQFJ4npI$0?$?;Z(|cV1mBdf6uPIu4?Vu}LTFu&P2an#EEd9yt zpTzWe&tE?<`|57+|HXxBJ+`v1TaR8(Nj*^9FUn=YM)PzkP*8 zhwBFS_~bc19&*ST9+0p9cwE2qN6cCCqYHCqT~W6!$O!k3+nc@b#P;4#$`vP<$eL|i z4Irl?8{$tCDi=FIyb0Wfn=j!9& z`+JV^*XMkYJve!vvOj0SH`rMKlmFexer|jG7u5XPp?Cgxcu}LwskwR5 z1>gSsInkZ>`%PZ`5y5)Xdw-99*Zs3zD*1h&c}2i~8?Ue5cAV9IZ|P_ERXWyRJo@GK zW1$-b^u=#i%v%4Z!tTpEed*~pi}yc0U6R|my(meM?c@H&tM11WFD>ug*B<-Z?r~iB z-s1fS_Pti*kNG98^QilX`OaBCpM0p^Ulo&ez4qAu-ydEdx8L{wX)eF~-Qt*!hhzOF zT$gL=-v7=cI_1(bS+#$b0;e)R)%q31faW1sJI%a8N0jR}pI=t>?Ze5$jr@9V-zb${d?>d!C)BzBbkFuL^X;O~ znI3I6`Zo1tgLTmccKO|JRMg(}w!S?iRQavr*c2)%~!!d{(c{>TT?Y^7?6O-%NZt|2D5* zsqE=2b?JFunjrYxoU0dpPk-|?G00zB$ee|p=i=O&o7LB4kAJMM-2JTb zl=T1XJ*B@mr}TLLJQ?=qFbDS{>*upx$Nro2*tNdu@w)mW`}bAvc|BQud(02H(hs4> zZP(Yg9x^<(c7Aol-;F0PTeZJT(*Ap`((Gn(@jQ0f>si)c9)(i~JrrlUH}H{-u8Bnbc#w>$~3k zc*gH1y}k0_#k0$6`?PPID3AAf`y}w!aoLHnr+jMFrer^PYRo zy^C?)Hz)i5iwnzt{A}CyB}RF@bLY~!#rHSYq~+fX`}4V&x8{%F{}V0Z_Y(Z>zv8bc zvMBW2pUShcpYQw9dz#`mxbq)*vDVke6QuL3_cP8De<10@ z_A+z%+_?K$)}QxRM|k)+u{p5KSW==|d37<581s7di(VOTK0Q;^D_HTmP4_@b9?wgRV@^*VpJ+<1h!UhLvdI}1VE zFImmZsZN3YFFazWtIOXsE=&_zU~IQk@6VU)_jaqQ4R$>~dVLp{b4rl@x)a9rkNNi< z@tdjlZ$t7$#p!mP>$W`RmD_jGxL>~Z<>C{c6hFQCa5&j`f~$>ivS$3T)}At}$M%(G zTUVB=zh1Y`oax8U=ska93_0>X20W9m-1j4%>9crog_Qr+rTTkCxbg~@G{gMly)w=r`|6SSszxKc2nI}wtbEgMwD#b~F8+OGHqT}M$G@H^{N{!Ct{jmst#vB1 zf4%Ge^uD3CE`QIfKK&e>`eLWQDW}u>a(6yyt>ayGEPl>G&fPN}q;4<$|BC7H+V6E| zZxpxRJd2RZgU-9o_ow%z{ zoc$r&&@aZ{S*QHZeY_v%H<5Ac7 z5<4zC$G^J%Nmc!Wa#d`RV(Q0)X1;ID+EzDb-p+B|+<)R=V}INdxut!1r3Tuozw<1< zb8S1{@`LVSZ>-n9nm6aQ<+58ZBG)#rE)Dpna(;4ik#glrz4RxO3Mc&W<4b$jyXH^# zpI7DwdJl=NRoaj$Xz;A(;@7QQ3W1P*UIj)_Vo?gDrbN?;jSG#{O zhbM{m%*=o5`o!~>irKB@o6hUW+)C`UKjwBy|z*Aq`v z{cq2{b)&_6Ta?`UGs~>DEz;i6Jh}f@$b5_ICxU0Zbea45>M=g6<;A>zQ+~4+{M>E$ zTxy@$$JI|Jr|0N>^#3T9{ZHY>4JH2KoN}8}+p5HEUvk?db*OK<6Mg@Q^31*Ew=;k8 ze%Qax{?Uv61T}H59Zyd_aDBAd#U=FXpQ|4$=KFtu?YwD?|Gtg7b~V-y%Jr<`zBVvRpT2zWQ)A%~_kFv* z#Ll@^zV2m=wrt5K-#Jef{kge2uHm_QfZXp}-}@dfw{JShu=|64a{Q-XJYsg~an;9U z*PHTLCEWk`aeZz1o%~;H6~7vNjgI9!^1pBM-6sFdPB!^}KX)$stG(yo%^S=2^~IGw zZ!h+3wCR3d&?=N~_}II@Kg;LcQa_oiFOOILmw5bWUH;~OCqh-GAK~0zw#(}L>R!3m zDg6&O`ELIQNk4X>P9^tFa9d8UJZaM~srj?v{FvT9W?!WrnE$sDTH^0WH4#7#55`0xu!*B!b4Vfo?DhmP+L z>QuD(youFOcGhpHKRE}_9p4tb=T(tvTZz`=%@&87HyvP2_pb}ybK}8g>&P;lOC0vn z9<^^SoG&kZ_e<%+x!TC`^)xxu+@*eF)LB! z`7I|eirF-Ac{r9)9JN#bhpC>&><+YVO{@a}QJ{b9Fx#5(v`)yx%K3?|KSl#~b z(euX~^><&kcq?w7x%&f4_A##iS6V-`ytwRk{l5I4>wk~y=l|Pe-Jk#GL-gtSWe2hk$(BlC7aOp zCUvtX1;-?BKTE9oF)O|DS>%7|?Kv5Xr~7Yo6^}GaZukAuX?#!ibUnMpY*+SuXKr2- zwtC_A1~mG9BYJJ~^LJK1dQRI_edYZiR|=g)e3tY2{U6pJ%=bS13|n#bS7QI0m}K4; z&og^Bl`VKX`BiP@J+DqI+dnz^qHq1T=ej1pXMA65dR|DIcXN{)Y9X0>a+n$F9| zMRPTZFC#=@2T|B1Gs z`{y4;=GHO)IdS~nuGMm%XO;Iaw)?cy*XsI#pKa}OCg1K?PyfaHLBDUm^`^Q>OEUx@oOjmt`UOq-qm%dj;1##-G~VZHd40{bgGa6Q z?<+nV`D@p&;GC;F*Izq$R8Bp+{f?M)vP;bbnNmsa{OT9I+vess$bZTIdiIQxaKC3K zv&_fnH2c3V&%bg#ST>h|&0pQsGJoMlf#3lYE`A6xGgzSH`-1yw2uAd#`u5SC}i>tl(vLlycqaW+V z{i;$ndgvo`{QeKQ%0Dx6zM9YBu$%HDEz$pfW^%jSjfYLia?LfHn|X6P1#f;>`P|(0 z=A8`wPW^cTF)!JzCRn9f{W#ROtMBL){y&%GdQSbgbJP8clFg|n$z(DT z&i(lLXz5a$B<8a3%zA(nvZ)QZ*;dQ{j(-6gM63;R1N$8&~lD_x1oELSrJeziWLrsymda=U17s>077yg`QRd$ne+ZR^L zkB&(Kc6&O`Ub(n>p0EDmL>Ghaf3EmLyRkMq(%ATUntt=ani%=F-rp|YRLp}3_nc_Sm#UoD`Pag#-ef+^pZk@Ef99IgZz88Ze?BWuyY|Zj z>*dA1xAkn_#-CI)JTWh>x~5%_&1}>3+cS!N-y1L5{m6s=P08UG_bgV~{T0|Rti0op zd)xglIoi)>-P=B$e}i;(%|m9P{I5w``#SPpJ$pXu+5DufX2+))>PZ%r?0Ru$POX`eAeG@cUIOdOS7#`hT3>KAsYu8C|`u(NTK#lP7bn?@C);7YV*C5MK7IcH!AMnsym=>pbf= zavDee+r4?G<#VaW&3omzQ}t_k+2dm#&8&(zxnY0h^I2D|G}=J1-*(>gwB@d%V9T8v zbw$R7cb3&LJza49-^Zj$tIj6)H?b92rYkJUiSSWkGk;X!Vaa^iCDFL~{E?S_%d2eI zR(bDK@5;KK_NZWGxxdn*vrE!0xQoqCm|W`qW)kI@(`_HlYkqrh>~Q>S1>poUjuO}SxRY&1461JA@4VFh;;n?w z#tY~4zg~2f>#uX`G_{>5F8}t)V*k0{Vm`*+{~&z!_KU~GNoU;Op5!&E^?j`HvFHAm ztvM$}I*vKU?XIpZsr@AsXV@EOQ!_7lLGkswWv`v*Y_jV;r&DCR?T2&y#BWb(UTgjS zu;KLd6W>!pC9WKtbhf!-b#wHY583y>Dp~g@|NFw5d)R-0%*U0<({q1(JS16cC5KnaHuF9o zQmkE}o9sAqk9CstF{4{$KQq@{Iw&rZb8u#z&+OeBXT`jIep21&<>ez`HB+qFO4ZwM zl|A&5U)`N^>Y}>Y>e7O(d}{rZ4u5jlcWUx4iSxS-Jr2LmAU*}u=VKK#IJd^~v-ksk zyFcExSHZS`{c9Fi!h8-`SP5<_Wd$alJeQF!u{SE)PwO*9H;`V;e@Ndx{)%U5Vy*YpT z#p_~+LlYZs=N|N1_Br#=l6n8`On*Ab^^ah%q5qxD(-*6o*k8YqJ?&DT04$h5x_%@k4Xn{vTOuu5`b#|M@)mxcdL+)n~2kuOG?ZF~7e5b-#2>-Tbnbwex=; zs|fiU_d{~O#p`N;Ur#kQsp~aAW((hQrQ+S)b>`bv+i!cxpK15U^nc;BV&6BN@ddk= z&%d*H`*8vJto;4w<*fV7?VcRi_H}yupXV087RTpR-+cVad&(c5f2W>zzuok=;%4pt z1^Zq9&6WR~a_99?^(+1d=3M*h=lAA`!}If3+&iCNv$+06j%&v&{n}@bKL7k1b9raz z&Iub&ver)Ccz(X!{!qRj?M#1LA0_|V^4dEu>rHf$E}zDpcPowEzO1)fwD|YRcbV6( z8?3u_@T!#t`y}@wjmP`7UmvcS{53>Qw^+jYX`yPu^9L9EEedY`Wc#u9Udekok(v`q z$1IsAY~EM8Jjc*g&A6GJ|I##1Gv5eL;4_s(&A zHS=A}tD9w&Gv7$hxMj1Lr>^e7*FBfdS$}VOJyZYP%nvW$d+vF4f~8n{HUIgyub=F0 zN$l_OTqALdZOtRs?L{3mNz!*$dcQfjBfwz8!akkf6W@A<+AO=#!RYfO;L(QH>vn&W zU;TMa@7fnWvU5+p5`LYKYOVh|S$m^yLNBj>%{euN8HqFXujL#(C?|j8e=75dcS@w)3w?iVD(Zp-7tFE>qtwH+a(W|makl0lI3sj!nQ3b@87%aTJc-z znE$-6udLl{^L+1rlfIv4ICH0;+ZJA>S8|ZMfA03%g>UxcE#H{`_sQ|z@B82F_*Zvb z>UMm^N8M-duldiidA7FR*6+_T&$aIVmfV+${wX=Xe8-7zk58Ice%W#KY2Kwh@;`rh zKEC<>-|>$(-`6Axo?g|P|L5l7y4k-cUavl2@#B+u?!WhQzU?i%e%*FMoYDMEbIhOk zaBlto>pI&7?%Pi;U7CJv!wdVP>~XK}J!ZVV=eot$9{JwpxVq`T-nV~cx7D7Pr0G}V zP;f%I>TBUAUcc%2`{k?WJ0Ek^+j6(!;O}^&xWD(?x7@k)v3>eE8Kz%5zpM&&dvn_O zdu6?p@w}f*?6M!Z^<^c~Uj>@|cozHC-sYmWA@Mb^?$RswMmITxOCTW-5*davr3 z_O_GxMW6qC&RO&Pw!-JS4*oBfl&7VWEq zYyR>;d(&~_yCtBdIL{>wFV@d|{?zGL)mFZ|KUP21oVL&V%CyJ*4s=9*=ZRT{^^E^I zlK0*FY`6CL*P1-NE$1Tbr)|rA8+^{{T6XW(mDg-OYtD?hrfat~e(S%r?_-+Hf9Ga@ zIyNUe%Q0u>n(3WCEYCjA+-`S#S&>KewNo$rV*flB{@ua%wkF_yp_crrqMhllBYZaA zoKx)kK3L8A(T$I@zOZWFer#lWqfAXy?SN*r>?DMAAPR(IF zaglNHu5HF!)$?Xot!w1>lUQ6``&_EVcXPz%&RzLG16BM5Yd;tB#=YMAaoS#m|3`gp z@7aGVa|I^R@cJb%6wHmf}=Lhp_kKcZ`;?&!9mU$n4PtUv8U-NXz$2ZD$Hg=!S%Ij?0 z#A$B-?MCL^iYsUTf4BId_1kp6_}TKm#~1&0Eq=yZH}Rmo{dJY;*N*9Ze13oFw_Bw@ z-3{i4{bwjRq`vP~e27xSi=6q!nP{a{l6C4nSF~%mMN7Bc>XZo_g zFY)W@ADvU}{Oro(Lu{V7K2JP4bB?$D3-Rd{Yp)&jt>3R+zWH9);vH%GgsKh2pRe7? zC-?Qk>>O+T1Non)*}pyNqY&dUfo1vR^=xbFzg~Z~Q@wN678OG|;cnLjlW(zvZz`63 z#cClOC1(GNr|HDQnV*lT#lFelc9Yn%WFyDyJ$JUIUS2szT;@Y5yLf+f{L=Gt|9o9pS}k>-a4Md)~PEKiQ&r zScR?ToVm_RQ;vXqBatp~NeS8F3a|AM74!6$bv8>Jk9$4mm-4@ltowfOGT(W!_24th zeVfD8e@^11XRaEPoImQ9mh4RE z;ywRHx%A@Vd3$Cke*f^%xyE*Ny7`WTQ%hR(?Jjtye>v(OF(KljVZL~<)h5NQ4b{gU zznk^ArM>UQ_3L-zwmDb7*>L+)f5Ni7vTd;f=dANk+^o&K>mc>h*`*)>V+ z{T?!bMZ$9RHkY_xTuAr(`R7N*ULO0WU*7j#uKzZ9d+<#9|LLxA{HDM^5BcjhYzQ^Gx$t=6thdKZ zo(k@KqF6mIwzxXU_a5u^vebx|TzsV`6LU58tDQ4JCH>PvLuaJoMl5!rkD4 z{;`A0=Ji$H%Y5Emap|Gc$Hg1Nqpzu)|M4=Txcgt= z{5=<@{rGRTrg^pagOii4)`~Ws?Qi&b_>OHoiy;<}0K6B{%n+OOiIH4Dw#Y2-j3h z+{-BW@8ZnTs(t~a-P>%M(_xxPNXW=mY#{7;YP_s#$F<$3b{y=8LK z_aAxpxTtjP?6@uRzdx?;doKHR;>WM==iS<YWS}ELW~s`#EI3s9Ht!oY33H=4B^5uYPk; zpKWIPxwplSn8-4VUyE^2*yk+quft^33_quXn25_|b94`aYMj z{FdeMTeYPR+uDceJ$QM;=lfMF4KMCrzR~R4SBgJimoMsn*UA&pY}olmE@;|;ca!cP zQ#@aBVs?kqyouFpUbbI0AI-6Rv7FPGS^Lk%x0aJL)xtNu@>O~F*_B;Sf7K?2jB-V@ zPw@|Y0uDO5+1l+AW;VK*++gl&xbfl12kRD;2+v6gdCC&wu83JZqjjYOOEr&nF-ANwz^=;ysEPEEqR4aQ*-oYY;F+${He93X5oB3o09J? z;ZOIZ2!(&_-9Pis=9@fuHJy*6Zs)Z4|No?Hn9^Tcb@RoiDcq+Dly01Oc-h?ds-M1B z{l=$G?>g^Z`X`dUDN_E30`VtQ^B!@}em>>tYKay7z7m((OS+|PZ~eXyyzA=Y zVDa1UK5H8un?2*gX|>v)?Tt57SI@OAEhw`Wd!HwMGj-43fB9Ma&At@6)ms1BGJkGd z{ee&Hk1sE;$*XyFd%pDXx__~cr(E8*xyLnm&LiDDrSzwnG(>0Wp3>Cf#kFLwWb*e|JG`#HFK z#}wUNm;8$sX+^pJ3YlGX{>veT;$IW@6n1ZUZU666^BQl-vxV;Qe=^#BeVDWH#^a(x z%kL`cSo-KMn(bS6d49zqh48Jb)AcQ7cOL1~dnE#Nwz}@so*^9ny0>cE z4W5gR>3{m~t?|FrtZ(%={0)2bt;N>g?mWEWUQ+4yBw79Z&dXP4bg|3dzY!=tUr73G z@$GrW>@CyRR23Fo7x&nIU~3fn`8lV1>kTK(|D{!YZhf(A&As>bm0x*3oVx+qc0602 zv3T##ynjr865Z?8ihkQ;@z%QPj{j?)H7ig5*=zZ^@|kGawr@4_Of~&;?`+fvV^s##}TmQOnI?%15{4@$La&-T8{KUO1myX2$){q((C z>$c@pf7E&&lN{S#xbt0D^tD@0W?Jt%GBHkEy5eee-oL2zKM#vr6|Xy(uG?;yz1v8( zL&vgmVr_q!q3f#`$){U)#Taqim(h>iIQ>VKdy$mgjDX+4yKFBX=Ic3j|Bf0@iOIW1 zCvQ64XBVFR!{+g6i=6l$!Qt;8pAx%!&_jRrkInMSn%@b%+G|q%>Os%jyRXy({3?9}BsxbyqD z*T*LIN7gXcH7l&T$o)Tme$}rZ*ZBSA@BcaY^6?Mb%IE*G`6aaY$8$;knnxlvw?6lKC%Nip z*>g?a^&)@fso;NE-_P9p`7>tnjwAXrzRpa)as8O5?0q5k+9~OUiu)H9_x?NhIBIQk z`aE~x^g`FU{i4cC`p?L|cO(} z3JUt~3Rb>%UH>k{h$F#};ga0`6FW|%B$`_=ZI(KeeC<^fX^ZDSi z_$2YN)Y6IVzHu{LHVe#TIKp%Bs$o)l{^a!rFCI^*DlMM3W#{FYJ08Dv-Ie70U1t4^ zSIh5R2$%jOGk>}jm-wULk_tY)a=Q(aZ7$7ob*TP4CGV5oo}!8T91%e1Q7lna|C_NktGw6x!M#?ghxU0)_U=Ph`lFxyg8{_oBg3%L1t4jOM) zSI;?gRrm9*ij!^p+fHozc#}Q)$Bl!RH?FD{%ik#?|M6w~#>{<_TuYv;J2UUcjuXe+ z-u+q1{V=qTFI8M_QxencKgI4FZKt=&o_(8ryn9M)X8HAF*Ro_|);1R{{+x2ke&@n- z67TM;{N)?T{{Nqsf6QKn_Z{o(!US*p;fP>f!6t!0O{Yd)9yy(#X#*E%lmdRKe1T{a&zZ_nTG*y&NP zRhY4Cr~I4~#;G{cf`)k)%{rK=~oqoOS zbE&$64M&QC&zUA2-2X#Ou4H9-C-eL_qCefY@AK~2^gq6O-Np3>TjtliURCMz=gQp2 zJD&;d?p^(6xBK6o$7RXue=VM}{O_CTkG;R1&8iRi%zgjg!yoVZ#nLq_h1QfDerfr0 zOU=Ar-qKC(=a$9FTseE*G<2iQ>CXr6%q;HRJblN<$uZ{&W7nlu2`_&)xhuEe(baQm zb3i+V-y|KjwOjta>3H#*jr;c%^WL1}Yn3)xUv|OI<09;a3-%|6em~K*xGv%H()#nO zi+ziPwtapSb2$8OpHO_N;l*SBuUhShV(;~>RC0gP`n<>Rb4TT~9qzJQ-tJ-8e^mON z-uddF$xr5fFnz)6&?nXK`J@HkY?eSZwemd=``77k#8o{mEdAiMzw?w@!`VFxa+iw6 z{7RhR|Kx{YrBN}@7nXG6tL_`x;%#eQeN@Wjxc6g`PkGG&es9C_xn`w$SBjpBulaJ^ zJ@?__Zr|LQF3*Lx{z`gRR`tO{qnk@s?di__1?7KRI`?=pWd(}`bF)}>@0yX>t>g3A zsP?nOx!lLDy_>yRx93DWIg#lfV75z-%Hzq3FaFGYGdWfLAG`hTnY8g9XBq^?hKuDE&9En0G7a`8O>UzXZm|?-EzBoa=C5QoyY69d1iiI)vg_(+^oLq#>IIX8`X19m7MDT!e96HqyO%T zEc5RZKF+Dw@~*43%H~Jx=bajL$K?J#T<#s2H~Di?h~3X*or|sZpQ8#kX}3RAdA~Dx z)&5m`g8w=%NYAY}a!Pi-)lTE{lV1qz{$Y52|4#ld6Xt$C)b*n9;4-~BRlkSD$Hn)~ zJAAq8RkQ38-9IUp>fgqH{19GSYf^t~1^d@8o0@Y@OpX1r(5LRS1*p;*dM=c{k4Ox^&Nrdy@T(ouj`!qZ+i3Rn5*&}{oXl8Mc2={JYDj2 zg>`$^`MGj7w%0Gmew934^2l@E&dY1hZPi`Vyn4RF#cu5=*M!qb33i6=pe5m7a^rKC zK3kzX^ZfJAlGYbf^BuRB+74n;@Q?oV_su!eN8#tneU5^*@qGTcV9w4CojDt)TP)pwo@6JeESoSTVeJzM%aFm{g<<~0s6-${5scvsPD(4Cf~o7bF(k$ z-(8-{el6hCMftmrmg{|vfBq_Edd>#T$((iPqd&(eU;8}$^7CUm)i#=~`zakC_WD8O zdDBNQ7o^z!7| z3ZufqC(C~wIj;KhuJ5;wp0Xox6^GLU!^G6ftji3h^3?shTioe+-G6fB4T6~qj{Eu1wdY!J1 ztomhdcif$DQuZXTRa0nX-+sesx>j2h{~h)StLEUB{u4O=ht~6~HAg4=a^KvNy3yb7 zVB;0{`C|6}Uw#ih&S>R->z*=)T0!_LEp@^TJqs2w|h zX?@kTceka?z7$-%zw>b2M5CIb_Bfk$ITkv1-d_f`IRZO z=L>UAO2UHfo;?E5zigB@t#};2Yt|~>b2q;Bx79oo|9LI4=$TUOgPW}{dh<#v!~E`i zP<6k4>ZzpMlpm_st@yuuzw%B}?K5M%(f6O*oStn93z7eG@bRT<-w$bry?M4Mc=qo4 z60iS0eQDjc^qSY!*E94d?Bzf6Z}VfThoRf_ zJ}dv7D3Q^%NB> z_(eRtGpCsM{)8VJ6_+2;nR26h?yn=^JAc07>;Cm~pMCDh{ZB%Yw9Z`=m%lUpPwM&2 z-0Q0(%)ZAY@0qgc+Ql2E9A_jvf4s*(F~$4$&eC<7&u`ZoHmg@2k1y>E|8>x|=H1Qf zhcw%dy2t08+1a+glkxx8zw2Me9QNPnU-!{CIVk$u{SSva`Jc_ne)I8Ucl7avzbdom zWord=?_6H6)L+QHEU-XryREeMe$MB!&hS4ve}C(9sf`b||4?5Zq|WxzJGbOy@Sb0S z6Lar=-4n%rK1RR(_Vt6x@@0>6pG!aZ$FZz_?zx1G7nbB)kl>hPGkFW=D)oD-&Nt2O zJnNLobW&-8xDT6e1+R$BzL_}<7vEY=UT{?9)0eLd%?m3Mo7pF-CI@I=&UVOJ-Y@e? zT=LXW>*v?ciQW7mY57m{^UaooPkp@qsymy0w6N!!O}`r;|7wxDtlhT>6&pR(WeSe! z8|9=OdiDHK@pJ9HC%UDdRO(6B-g=QL-Z{TfGU>LfoLu?&hsUo^PJDSmzC?A|51Zxp z??&f!6&srUJMrLn^NShr6`Ow?xbZ{XM>jzJNnnjQ_pd9H1+|Md8uziKj=8ca&$KPxU<^eT4l?j+0J#jKx`oaf(Z z?9JO#vYDOv&cxv5>!r(I9v7cqCFA#_saEd$gY(_%UzoBbEYaR39bda)HGj|dgiB&| zRuc`osv2jM9K33=TK+_t%&D+rTg#TnUuK`T__FQ4mHaCo-@jG%>*~%I-*nigELOKT zp=`6xGP8S4#RYM@j|-aTUtFyl@gmUv_;jm?L%V;!`+8Y_tEb7ekGo$8bPL@3;{IW^ z^UV6M3+<+rmnc*{SmShkVy9lwryMGy@oSsAzsB7;8ecQ@=mndtQS94c z>*$zln2Sr}T@#Fz6Y4g_osDn!tvuuN&pUha=Q>=Qw8nPty|BOQKgTdj`*66OzxjX8 z-JQoO56kp!x109IHs3$H;Ln{oKkr!O?Jw|qE>)A%vF3G7*&J@^m`6R&UR&Ismi>Lp zx&s}X+*7Lcc7NGg@#9YQL9NYOzr{@6%V+cD5vQ9OXmLGXTG5BEduzIL%l~a$@1rmG z#4=oT*O~S_ZM)nJH+HvM`F$0*|M1c}Ik9a~=~Fj9SLcrYu<^Y4jF^&%fy;Gwz3rZI z*x&!ri@aQSP)uK+@#ElgPWFEZDu?B=qOXRD_O9O;9%+AQ{%6iOtAJmvIWHnK^rk&=u7lKxoEq(v+Y+Z!T5sSyW45u!9cY5XjTkk}~H(eCAdH3pw zr}dhP&ik#VTb)l@En`>y_0g?etKL-l-MVtn_Wajt+Z&5{V><4au*|oTNLc9nx3X~E z%Lt!Bjl*wGum2n~xlWk*;WXakT|2Z}N$=ae%A%2y0x1IArP3!N{4sUhOy}adeO2=& zTOHA>ex1$FFT*kI=aHGNyS_etD809|e}2j|>C-FkDc#z|C2aTUgLlr?B)8*5O*;Qi zC=|XfI=N%J&g{~Uj{heGm$6#^%}M>RB>&?B|8pDtmzV8Wd|9?r-}cAW*ZH6RT@{ln z%E?-+z2oGA%X0IdvhRJhMfLoZ)K8biRT;HsBji&iXTLgf-1tw) z$oj>vuG{L0pS z`bLp8D?h(c)r|kNmZjEd`$yrdHJ6=>X7W8vmCo5P#qV)(WPH)7Py04@I|n`z@j2Ku zeNTke_s>(9ADvg1__};gmdxb!7hU&PZTjPua(UkoJ-gK7#x|b|4YvFH`P^7GIrizf zo3G;QUeCYsFx>Cn<@5XY@w2@>x%^D}t1XwMCyDo#e`^1`#oa!qfg^K;n^uzPCD%a@#X503I)4S$KzxdOb%jcJB&$!IqDH>N)lInj%{QQT*Zeg(% ziN&e)&Bu*>u0=`&FZZ!CDmidWye#n1Gs{{=dCAfzx38}{e0uq{@UIKkt$zN4U;pNV zL(gySJa*5%`q5e2>n{3pqaVjyuJbSS$Nv0R3AZlnT=qC3Sz!G? zLFH{WuTS^A6e<4b$Ct|Amp-rmVz|RB^GbjDHzh2}_nutdS-I{Z^XyOQ@1j3NteLRK zrNUS~@7>O}|1;m;aH$bx|0!YfYIn>=51-4&3$$k+KlFUw-_>QmoF1K2H`1t`Zn#Ev zBm3bRo0Z;un&#(r9dFz9b7xz95#!0rJ=K3Ma&z0xoc|)D=y>?2$mS21KEAHlFV0i2 z{QhJ6n9k!&H8DTl#<2I-9cjFMj{C#x{r~djrc4SditI4bDA{)It;!s>eP+!HvT~Lu zCv3Zts&si;YrtHiy_4>B8qX_H!pdk)zN44FKasJz`3O#js5&sHUy zzI?GKW%ZL6&MC(37rgoJU$B|~YiB`b`}w(R6Bg)O^!8RoJQ2Lzx|-*k?ED|Q&7WOb z+%L4hFZb%l)ZChc-uDOH`)4dVBm1jA=gCPH`xP0@q*F*Q4lT+7r$?Q9^l&{|^cXrjfllvIu_T|3Y zdCZFQcx1u(KU>%Z55M_;qu2M}=h**iY_DIKz1Qu}xqo}U_p7#V5`S&sQ~0j!P{R$O z^*^ib@rrJ~mmG2X^7dBtqkMOhyyeb*U-8UZ-T!%$kO)_dIK#yWU)2T>AE~PFTTY&iqG{Z6|(vw>onEtNF_``8wOJYFlnJ zYgRj&*{J*7WIi}^qYB?^>+2T!E$fp*%cR3Ub}!ua!~XcC+*b9S4sNn>;eR~r4&E)) z`1{avu65jMzfJtv zkER#<9-B~YXl@@@VLsy8$I28Zg0QTC?)?-lijK>_)k-A?Q>-b+WYx}(9)f+|Z?|e3rJb!WKj|sD%=N27^jV`KuCw)J#C_s|$>#I*XV)L4B z8qZCdx#JjLW$v2;H}_1If3?};5k&6TopySA*ocCgF&sPCFt%XU|+w${IPF!z;uKn-(D zg37if@9JN!5oh-N;wo}JhUb@Ssp_l-CC}-N8i|`H8d^&8yiz;n7j*v3j^&dl9r!WD zc;<`^n#`OCl~JYd-}rLI@0KKr+&qU4KMR18BJ7pTN3$Uim%C!XTqBG zCyv&MedNh2boyZw@YC$StH&l=j+SrEBbwYHP!A)&fa%n)nz}v z)WgY_UE8Z>d~OfC^7PPoyPuXz`*h-Ny!hH9t{1V`{>BlOeAANumu7N?Sz0{_(BR&d zp88k#cTVZL4|{Vq%im9Y!Mgd$v+jDd1ZeQ}~(-AMdaZ2x%gnuDNoz(C_sotMghn(5F_-`F?*mJh_$0z+Ww~~x&Zyx$;cz$z^i?FlN#`W)izF*Wo z_l)dD@9joDxy$D@&tA7Px9q6PnwQQeZEIHi&iQ0rb?0V>kHInb9~08WWjX53XZp{( z^!(DhNBZ+VxZ9QQ*(fRh^15Gr|Id?Zu@!d&BW>QjN>Dg1-!eU~^kJ)WZBou2$=Ndx zHw*9QKI*?K?Dc~z*(upYO4DT~CV$HETj~Ao>6=%aG|5v;y{X74!+{RyQKWrwf z{-nh;TadAMr)lCpraR^b&UgR3W3la@^z~!4&lvI~em`rQve_SZQjfz5!sLP&))EydGE&cYde+BiJEUa_cDgL{j1xV=i>FY>us->{GZBm z_mi$Av-Zp@0x(dnY<%<;q>e`gdCy zsvsqRJ^PAz=gic9k=Q)HWN!3xsUJUQR-abEh`Qxuk( z?rrls^tun7!ZNY(M(=7kmx-<7-(|J!sPykgiABlhpE2j9N?-WV7e3SOYh2vw?AoL2 z&nnD+EHwMe^lyIxcjUx8pZvAY`rVCPIUyg9#fk39f3VQ{M&goa*?UJ)cb@Pyyx#Ue z?XzTK?{UUd_mx|_w%+j`XPvF_a79SeA9l2%>TKlkH6x{%IS~RZhy7tysh|} ziz{{9x2i6SjJQ#J@pk30eK$nrz3}?i<$CLh6Q^!+k9>ShMs2F_{yUGIf1T}q`a=>r~Ub6GoSwmetD+ zb=3Z+mssk$KHkz>Csfo=bmhPK{<%f9XrptS<-SW#XZqwkx;XDfM*sbX&GKwZ_@*6E z_+^vb&GYxmmr$#THy%A+UiEl6%Pzgb$hFP;C0{?dANOwKYVikize4*T*91Sm{lof0 zx$kF{_MJOF@5tonxjlP*&pmYRui{zf*G5)c`1R~_9J81Iv*O~KYf{B^=l^eY+x0oO zZrw)tP1_>-w>)(GY&xZ`YE#&}%RAMo)vX>r(C*61o^3R@B%Ob`-fNxsrj@O)l{cQ> z*SDtPOWo#E-xy0G?{E7OGkLGV?Si~lJKcC5vI z^={7jZ>N6Tb}ma$eo1WXn~TlR3 z-g?XC9|_+1d-1Ks`afsa+C}c!pmOZhhokJ}#XQy0)$24LzUw|>_?K_LRb^?}^I1n6 z*F3I&bakI!<J%2{iol#||?KA*ApXRqP!slVTB&c2*b-P3vS zVibpQQ?;SLUwrBM%AWlfbLBdG`ON-xihn#9mdzOVcj11i_a!%6o@g(BzBf7j*3_d- zH-h-z8$E9?Gs{|EKlw77`Jrf;!-vn$+qA9Z#;X343T7wfexG$!Qf|+w?{+`#Zsf1~ zoc`~?@z0EZ;(z7;TPwSO_X^v#a^b5-%mRIm6mO7zBGu9_{|KVt(C=Q)eZDYHr*m0tj zf7ZR%=D+ncrh2F1e)&Vk+2qdLyck~bi>vPM;vSu>UmW+n^l<^1*t?rOp>i``7nLwLfeK{F9WX{U_(t zT_J>K3=zCbZ&RuspFsW^FjD;%d}0*(S{pO&fjG*!79?`N$j47&+;Gd-ypg8%L6a} z`yuV^y?Wcb{r~CJR?0AEH~ViBw6nFm`(ydJx@5~WZ%**mSG@dwY%%-xqQ{r*gBJXn zYg4~s`=|0BiSIW}<(GI6x?N5p_*cJd#mA#}W5U}tYxC<)OrB_8xIB7y%;(+@UySdU z9QD7y_}%`5OX=3%*8SM}R9LPhT>j(fxMH0>+4VVjb1u)DXH~oL%KDU#r^|0#J}XxF zlzm=y+@0SKg8h2bEwkoqnYVKK;iBd9)h6E7|JXBmZuH^YybD+LkF_5Ear56yyFWJH z5B`5L|L6Dj{10whfA6=6(i`xt zZr{Ae^Q-0lK7ajlo$29x*(TB2z}J@huX$9bRbQMb91+Ez&Um-a|@7_4}yxra+;e%V!Sv_lmN4EQ~Z#cRB+w^rOQEnd@u$AbR$`Y@Juj z*{>zn%=^x3CAhbI=k*ge&;9R=Ki50CCfoXNj(nfb-=(p3pZ8~S>8xE|Fm0#DpvYUD3WYzW)n{HYhe(XP&cT4h%uEjAgqzhM` zW|p2`y8fo2nc1{sW?t*7dut9~jabIq{5YAcRjoFS(DQ~z;XaNHCBygd`Q zS3K$LF?F1CG;Vjy>+2<7rvA7*wb<;+`D)(yFR#3Red_*j%Jlx5()FDB)pEJLOOL(1 zdH$hFao_%u!@ElV9Z&mvWzUZ92g~Jk9^`}Qx~%a{5ulCbBs0mL)Xowcf8Vff1TaoczXY-C*LIZZ2heJ z&TMPRp4%=~3%Xc0I{rLTt^Z9L5zH`VKJuWD|pqO|3 z^NlIOWfyPu_x}?8%yovp&bEj}f6a#t2bWcTzWvPn*-4Z17l$U)KFzWD5WB5=V(k&O zdAn>Txj*%fJ#(zlXy&n5yXH*L^}i=)b~s?~#+bXadH2V*o6v262JOE%jA73g157_-tnP)xP%Q=BqQ>+v7Gaoh-Xaqq0bD zi{0|tzr5!o?pcPud2GF=*ZA9yGi>)?{hK44bBufUt;Fm5DxU9EG28ddw@vTot7rN_ z?$5a8th^1#%;6j zeeH*p@1Go!xc&Xh+2i~8@5vuHA6Y*AJyg^!5xl_wAfH zslQdDYqI2z9$j;;&DbNIZI|HQb4)2$1I^0r&ok~U4o*K&li8(_ zF7kZar{!zbS_bXA`7mDZ#2G^|Efd}6J#~?NDEj#3`41=gtDpXwxc}nx@BD@9Z|;83n*Tec z;N$nyJK0Yn;-}{S%t3>6+|S=L-kE+{AAdwyZ%O}^?vMA_=kI%88zO(;|B3j&$+s6;C!}9`f2dGb zMV9Z+>q-3)R(D*sVZSs<-Y@_V}dMcXy1I4pLq8MTeQ)whe?~6 zo$BrPaw^9^F@H8uww-@ZO-rPuWyFi4hxN}Fo!6K9b1M7yqukh<>HFp!3ePF>jPo%& zdDO40V5)NdpFamC&M~hwDmb;@St;hn*2C)TujDE(^@$(MSd-y@t*P?Ya^tT4k1WMz zQp#U`e%33`IWhCc^tj=X$VxmpQ@?L{?7bIlg>QTInBV@Q zGwG3O*v+>a{L>})|MY6t*`$acKe2MNYkR#}uWo7WOuhZ{zC4}w{@C1zGUk0bo8#BL zSouC;M$E+9&TZd+EbKm^artq&OrFe>s~^A4-Z$}cop(!ZnN8vuwYf=W+a<(bSM9Cq z$***f^_v=ApR-}L4qL>H{I#6h&AHasElggMG1C;c;ryy^3+HzRi*Jj54i!+*!X+&t^_J~OM{_c>Sfp9bvz z`8K!gOy*mIAO6A*+s;a;?Eksz(}}WDyQzoIOfUMJ#Js!i!RDuev+JUz?C!tGm@V)B zMmucN8@D$HZ2IIZpY_}Ae3LG`-|*4ddwcos{a)}fk>_nSTlnLrGWo~+Y94$2-2Cm} z{vX!=ul$vM;CtnFZTf>&$s2!jzb*RA{KLHNZT?%S|GU24*_UWv{c>;HzG9Kzr|x|d z-ZS_6H{0rqS8k`5@y|AW;<)(wuf)9y-!1B6ej7#F6n)eEJ##bPy?437=C*d9y~AdM zlFg3~mo}T--2KwMUcy z<4*-`&o{p_mHt1q`Iur; zWrW?C!^zp_FDR?qc_aW^4DM?$Kx{vXt>n93Z+~8V#XWiM0n%8Rfyo%fWLyiAM z==WKl^xDoozLCFn;qHIWdZ(Ox5j~@q-{_O3nAx7Q6R*ntd2?7#e}#>P`sLqGE;|0n zcpkq0!_uDXi}_}M{Jib^tgY6c?iIY>WuKgq8~5i;_Ty-tC$=@$^fx@Jy|&kFw%tFa zccul>=0&mp%(pjhtZOq@y1z>y%(Cv}-nRAC@_W8JCi8D)o2A`ro^14Q9k=rD$lnjv z|GD!1cSRlZo%s%eQ#e~$Zl^8Eo3xP6;H0NVH&^XsfygsAKFru**EB2TOW?k!ju&&K z7k+zE+H~~z?PX`$%6>!{Y&u-l{pi*KiA^0B-3>B7U)lL%_lz^g7~FT2?)YZ+W6Ak> zMw?zJRQ<>~*mx_U{pXe&=d1apl}}%nbl&!m%eidJ!Nqxp@37|ox#WKD<>~3)_`W^) ze!ymz8FR(qU8g=f&+WbURa8p9G=Df4` zpeD~}v|-9?vmZ4ZuCqGsyOQ|z->>WY_tmS%&fWVX=g@5VUvDmQe(u|Ou<+Be!rzOT zb*ys^g!Md`dei;hvn$7pYd(J3n)`)Kyt3-2R+!P3gj+H`GTPRke#q4Bsa=sae-36|;tw z{+Vl?y!hwAX=WR@#I1ew$H2@ye!BeAoU6y*^fvkz8EwijU3cH#M+N zIPm%4ZK<&No4xhd{A}cWt62W%!*}+NKa!T0{chb<+NGy;uZWdj@@>i2@W-P2=SWLm z`#n|i=9e1>pDQ2A?)y=%-oh#WG4I z%=8`SOZBZwPUhI9=TF=GEk)vns`LM!dymz<$@9Fv%d21J|3oY4y@?$1`)^IU{;YAK z-TOc4_bMJP{9j{hpKP=7@aFXRN~;egeTSSkl!#4@Fa0=C`SYaev)?YiIQ)B8HSe9r z3BURIeinrlwY~h~%r-Bn#XZt)!gjlxd-&q_d^O(t+sy4s9YdKM7_ot7pce17QlbrTG4gRCiGyC+z)!nRVOV;?lnQOm2u})%k z&b+!K-t+dES6|&aF?Dre?{Ri6dHZK|Uk#q@H>;`L`1$Fl!{OU*xcyqZXS3)0IYqU6 z!g8~h=Y0B=!*Wf2_WbL6+v@)&zd2NYFZ+py+3vyneg|3I#|pMGtOnbN#sG z!nEzTe)Mgb_+3N!-}QGVEKi(}wdu(F^!5HVck#L{?WQ{=BaJTpni1oBIJx-f;_0!Q zroL>`v|jGJ%cJgGSB%h9 z^v_q^Y&rjMVy;<&>A#;Jzx5kGKW_Tz$&1acAEf&K9SxPAn*WrK?a4Gfou7|`Bdd3= z>gBh|l~p)$JbtI*Ylqc$e?Te6n3)zQdXGi+gzu6@NXCS#;d4`j$-Jo}bO# zaoDd(G4?+Ezu@(`1>^pD$m=P#%IYvr6DKSQpfja_|a&Xo91-PyNJ%ht9p zJLa= zt+DveC7TU9Uw=BEU-84Oe+QfP++Rz&6>o3YXcV;P^*|Ht1_vZiA1)9l}MNtL&&&pvxE zrTs~}zQ(Ql@u}L>+&?eb?QFl?$u`Ts{B7HJseL{2+wOmjE3V7AdVhs+*3M1$4u;Qr z75#Qz_}%Ptb2fOcGu~nq?Xm8me*Rt0Y3pliK9|>3FFY>ibIXRquWs`}cDY00W@bO4 z-b{XPemDEXk!QxktZq-T~H)gC{)baC9d!+R`_n{n~$yb(Qobw_FLuhxD4)V53i{_yd| z0grb_YV-c32+R?@|HS%4Pou1NPTuV+6OHCied$uONX)UWEA@85EOxtZQ-4l2-c%H) zcfRt_vr~OOW~)!P*6*lzAY*gt8RPDWUr*jh-_O74lR2Yq|NdWRe~37L=H4H(dAh#c z5y8BT8`kIbY!LK&+#n}6^G(RTjpjaPKYGhzWo<1dSATk=P{eoV7jwV9$iB5P_UFvr zCdJGN_S<*oQ}c@q;m4PbhOh7csm9M8X?2$IX-;MK^=G+f9bY{ZpZ%l$_E+f#H`AW$ zT-X0_{rE4BnezT-dyiiI@n~~Zw&KtFoPWnxzB&2b^IGNG-~55ob>AkPShmk>{nIb^ z*x%O{JpW?K*BhJtr0=w3yY;m6Ig-=gZ{ojK&-8T8)bgL_4xBbh=-nP}^Pk~f)ji3t zr{m85U1>Ap&E~h^*93k{H2Zq%RNIX==f4H!rp%kXm3^z2>f^Wj*-~fAex16t?d)aA zCB;e4MU0+T_TD>n{@WOHN3CYB|5mi;+zt6pafi=a+`cpI_qmO~p2VC#S$1wR z`;8ALUJLKJcgkS%&kf#Xa;0zO(tF;XzZjk;(^0z7*KgCxbv-X$&3<$7^`9x{&pu8K z-+wt(JU*sm?pi6~SNeOeAJF5!*P&Z#VeMRY?OAd1d>fJ9268)_{O2uBx>sC#X5;;e zuW9FZ?u}W!`0V-COs&0I_L1w`r{rud zp7t}z|K3gKUw)GJo#)Bt7n<%n5tgs3^8ZrxSx{ZMz%pr#Wcab!hB-cG`}w!=*M-zG z9lg(BZmH`hbeZXH(}gq@&s!6hWPX3Jvf3xh^Xry}naA@#=Qbv|&$h~PY*@;B_0hR$ z+dAZp!~}P}(vUe-vMqS|n%KR3>D>KxvNfA>PB!){=~~_Tx$2ymWog-#=k0U0-MX2z z@ai*VFNv2I)VZZUg^5$e7+0rd$$KpzjW=rpE|M}^y@3I+_g+%Z#joGhK#C}h;POa;ae7pNy53k+CvZrs`xz**YpQ+qF8nfnk#r3`?UN@cN z`EIxz&Et&pO2T{d3-t5|J{iPozm;xZtmpP zuRryiS@`c(d#hVl61P2jxch~=x?YV@&6MU(5@|Kts>@XUdUxroCuY7^UVf);j{J_(o!`FiyxtN2rn~Ue^Sy3o z{*<(dGFP_rcYP22KFeKCJ*{$qd}rufpN%P-?}UAiyJ@+-MDF~w%|4Y=#b>CW^sn6a zXQFQTlbfs8`O3=Gy-rRwdQ_a|`|sntJ@)&rf6xu@F8^|hSN6H%bGLstxa}oF<5|e4nKHU{WE#}=6}yC*H5&!%+1mGobR*uU0m?HG8x~# z9?5>ISpHS#%66{HKeu_~jP9xbiyrh!aliV~yZYFd@UL;o<&!(y?au8n*;aN}QnK`~ zg8H#LmFp*l|Cu`T#_5CWJ}y=lf4)Pr{LjPc_j3%_rTNqbexFq>|L4nIxBuU2Q;vW6 z(r&u=U-mhZe_t1WtCs)s0s9@{b#Cqs}LkW*g_adXceEd&7ZqL1Ewms+KM2~Ln=;irkCyyp?F0zQ~ zxuqj^`p1Il@?}>~en>xPSGfMohX>BS0yBPSsujuoNosQ6ee}Mfuk6%dyOT<}a{F}a za&8A)Jvy;AzbfVAg6A6_FJHWux#?eW{@3*}>CG`4Be(pVsQgye#=2sH?zEB<)6L($ z)Sb3b`P?Jr=zA|TW@{|B?+#d^8~d2;U*U$c&y$Xx`}T9o( zx}R2Co1MsPwyinpDSkff`%0U>yMed<>6?Ga-SFgL+xpf^e{Rk3y+7%q`LT1)9$nI| z+x7H}b}m_NkLq*d+W}9WjAO3c+yiR|9+~Vtey7foqIOg zw;rFPbUEV6^$p6~5*!3 z!j@MbbGLi*b@R@Y*o)I=pVk&nmwzDb{&k`mOMA_y;QNmiZ%n?d_W$X^9SM#lTRsZ2 z8MY|eZYqkd`Elx5z{eAfR2$vvKf}!C$+A4m<~;;zihax9Ieqf4 z-CsZ4V_(11@U?&Py&YX;G3P73*gY4Pw48eW*x$eE0jF;L-Dmva6HmnRx7^R!3+kDc ze!XM$tR}hNK;Qbc#(SH&WlQeNOU@5S+x|26UKIQ6!*6a$Z=HPn{p5uK}>3#*La@~)v&4q8zw6Onv7I@#6uP4;*RO8OW z$3D4Fn`ih5)SW$Sdd}>h_Vz!|ChsqpVELPG*)vX`dFQIQE9x!?UQb@|D@Ze z{ydnzf5V<9amvTncAvZSzG!`8?ZwIhtJz{TiOpTTGOj5apURz8R`5NFCOb}ul*s-e(TfEN!7eP)!+X(e3$xo z<$i(Pvtv`g7cI<9p0|CE_V&1rXrHx`r@lN1;yZEL+j83KoQr{e)AZuymgvgOp6Z@Z z+_!z(@pByO`(!Sj=I42JjI;i`-H)U1o8_<0cldd@{%QZaC1y8xWSTE7N^Gz^An}83 zu49H_M#ICmkBpi(u{z#9b?$zx+=7WZhi5I}bj-mYr%};}T}`h3(8_{TsXKmr{&zONzWQ`h^6B|?9~gt~ zSv@(i+N^Tw{ZGE~D>rMutC{=jmU7uF?^8J$*N=~dH$VO z=cUy?KAbAQQN(y&%}3)y-R*Oatu$o_T`qktcZPh@!{dwp*!V@f=zOfQak6VoTJ^@b zit;3+VfwZ}I+dcOJB%;Fh~w(7bso-7xu5|(QCbAdo-Bk%ctWoNHmusAi{%3%lDE<@+q@h@LIlRfTQOvx|XqF8ElFQ>-* zee?f|_CM3#-DkKTx4-_5WJM+08s%@DpnmYbPxmXHR^{bxdOj)tf4=+epNneBbMaedVJzxA$aA9|zzOss45wi7t>DSUsf--CJXI_q~| z-*8iMeO>+SYTi2!D{fc&{yo0la7(RqS%A%q*THW;?mfEl?rw?j%DO4sGyJ^rH^%MT zwzsCw_VFB&gsd6N9TY0exJqoN8)hKobOV9x0>nyPfD5}^S$R$?7^QK zfA0J;C9z>^wXfOJ?KV~Aa#!`@@{Nv8+<(k&_m{9b14;FYr|iehEw9LVH$yHmwMfP{2`><(Ock0}j zx?5WgIh$4IMu=6|lyVx}U4w`1;>D zTvH>F+&{;B{}X{4KCfc}_fI_yj63Yne?BK9?74USZ(S8}PVDo#!R5tv)h=KSZw zrMCga$5>APlB)f7==46Bm)(Ef9OTtAdH?5KW5FByjq{JS&iHxp{JmENy5COnddY8d z&s(-9V!gy+H@ki1sWnD5bIg9MJX>*b-{Z2K{@aeYx39T#@OkW=n@bkm*=H4%n=QfGKe#`cq2;27Z>&2aR2d%FM zZ#rS!bL;ec!v_W|!LJvT-8l64t>c#;4#&0|&5O~P@BdgR&uGIWWAk%&J|4ASG+)0o z_gLdusf~x{6>93A`nYwbc99~R;pO|^rP@BT6DGyV>3c(1z=Jingd{U4L=ACLF^zVvlY z@69P?=dHeTuef^s-I~(3ysy43yteC_W6kflYHqu%_nBw!Z2om)?OY!F_tonK+xLFF zv-;;0d%L$AkFu}cI-75QV)gB>bLJJ#Hl5$Mx#7=^>Td1(kAmH9-I_0c`0Du@uAN%_ zy<_`toKUQNux7dam*=jx|y#E##Zv+2f@`DcyQW?aATQC`=w zv+(1eIO9Y9WzX+Q{`k4qE$--vg%%nU8-)`4|y6wT{2Op3B zyLQ~}lKahV5vSYtDO^uFyUps3^ye+74fnAhY(8}~>{)x6)vbsA>Ty<&8VXzFC0;&= zPT4Kfk$;?P_vLebc1I4e%dOfg^RRV#5AWq40{gB07aiA^i+>{B{N_REa&LdVO|{27 z;;MIkWEA2*IywK_552x$oNZ6`|Np(eKKb{zKVnvPwcC#RT>U-vdDm42k-rA zsQi$-?uB3F6#2)oG9_Y<_lW;a++TBW#`&XVU;Q5+OKi87m|zrpejUH?2|K(CjWT)dv0fTrEN#pQ<^pYQrTmk;j#esKBylqvjwd!GdUJ+r(@Wxd=_=b}B8^FA(}zv_DW zt5f&S{(dn3|6BX&dWQUlJMXQT_0Kev9@_Wi5A%=J*FHxs__ikbZ2j%8cPtO>dbTGa z_SUC0sk`^Z*8INoxtjOpxlpIY*QKm2o zU&gra+xlb5)pKq2f6CY%Jhr(!q4si>+_j?Q^?xqBu6+LT(YJ3cyneN3-~YasedB8S zugvpBMMhuWnckZF_Qw|mn=ALWP5pWCXK`-a zCj1M~bFX-K^nIN2cI6%a;!D>j3agd9V*goja&KBs!^RtX%4_USbI12Ce6s&vTQzTG zNM3yDdQJZn^YxS8f4L+&-)u{H^TeM4-=+TE{PE!MV)+keE%R$v#K?Sd&iia1aju&8 z?y=n#FE;FdoRQP}=$rQLc~3syyrpsO*xB^E*(Wyt-*`X!jC$pi&Ch1;{Ac%FYUeut z^VOX(=EuG?{`>Ucwf>&(d#``E`y+8P`~HtX@t@txjz8QY9sXx9+HskM+gl_vC)9lbP@}M<>~T zr*BVw#D_ipwRXC79(NL-bkAvzd;9Xy#MJW%$K&gpMg1-J7CFnUoSZrN$!WFjO*=Oo zwol!%dhwgz9_svc!ZK`zTUO3kAr}6vd+s``GX@>;vbyiGd2VzF?7h6s>QBj@n~uM_ z;q-PpFSmO~xRZfk4V4?n&haQQcJ`$pgGcP>r0|2JXV zr^PZG1J~Wq;@i~yu&dj@;^~p4-jO*^PL}OBoytCp)BeUE4N2$&}_s-7z)$jZ1ud~O` z!>iBO1*JE?{qmaI=*OGKieHnCoaUREbatB6Bj$)QX;T+}`Hwfg4H?+5+sq}}?SvYnhh=bC@Tub_M9cPgL%^lIVjh3#LwzkN8dDV^`p z`u*888q2R&DED7^85uq6c>FQDAKd@1%-?I@klzoUU$SAo#`^l#XZ{CkRpqm6w{MR~ zKeu^Ksmi{sYpQ!qSMu}x*}iP6m}qV_hxOF0Z!6R8<-CbX-8;+OXY1Cot?E&qx0J8n z`0ryo`{rM>-j8%Sugi`r>Vr&|6LzdU;k7||LwMB@`>sZ${`yZxVe;ORmj`_0!g4?L zgq5FKt6B85@^1DM(Y(Kx_h|3maDbJ+=j-0>8(sPS&*6TR^l5@}^DORLn?u+AaCkDg z`g(Qm)+W2zMN{wBY>e(ZS$U-O#GY4i!s~ub<%_Ic<@oeEPFP1;|W5fO5 z4|dU zH}(HNLCZY*|32!s7nAN@_~ebt?DCf7&Wq+wxbC#-tny4bY57QgYhSKT->~Spre|Us zZXDa3p)fap(`4JPKMr*NSYiBf)58hd=UO|JT)#f$quB2wl6H0gi+@(UQ83Kf-l({5OYQ6E`%$%av&7#_{NDEU z&EZhFJ3o>ZtM~CG-uieWtzYI-dc}>$>-TIveN5s;&GYGbMg)*3=;%31Ur(gN)Ecc2CS>&F;mU z?C#ag`F3sc<$vD}ZjsDSc{$(gM3vHy)4G)|a{cFzg-Fl*c5LCcFE@FT4ZFRqt#03R z|EW6P-sCRq^7U_Ty3@QT%F&af@ZFt%dV{Kf9q`NwsUqt?FWV1D&9=j zH+yG-ytvQ*s*xpUNy+SbVX|;Tf z)cvB;Z|9%o6ouBm@Za+*{@=b=zaPZ^imm_kk71wkKWO_tE;|1_|AT+(GuDbRr)hCH(PfyXT)3an#)_)Z57W* zuivQE-%}eecVmI~`purzaTPBn3m-max#`>WYv*z@g5z!``!{j-^QRV_`f~G(ztO$d z#VK1;&15R>QX%^Y)d5{a#beTRqqN#scos z5_k4YTmN+JSO0&G)89NVTp#%-eBI6|<|6Ss3&Y~>eT&~d~>;S^c3qwD9F$M~Mx zZnraH|K&Wp3D^0b1bm;Be|~Qo%k=HZk1tJ+w!9`@vUzFWhaZZUcVzX)H~vi2Ik=^` z_|K$d_W#cJvNv(|yW7TWXb~3MV%RfB;%R7IRQLQ_-URKaIa%O#Zv$nA+Z&y)WZ7*P3ig{Ls?4pMBlO&KtLWTIarHeKO_p^YZK6 z4-ZS!{W>K*|CB)F&)IFcr`4mMy6OE$El!?wAmgmZaggBsC8NPCTK)n-s=hb$(P^u+r-+`{;Yh{y#Lqb{r?~TWcbJae{+56 zls)vtzc9-W&cdwhWdi^^YzpLkOeE8hD{#|b20r}S04SP3V zpDm+h{^QRo?cWMq+l`WXY1r5G+tn`n<7b48va*=n zhZmbV#mlO!)!OneAN{_(^w3Fj*>)bg$CoWXKR?iNe7{9P>g+w{N$UT9XjuOF)G%HC zhOE=ap2=T+{91ee*Z*e=EdF2aI=DtwE}4aG_VS7wM;9-bwH4cR(3M|z+e!H~C5etN zCsg0tf9rC1+?mJx^Ezy|oeb4GmlSzNn19ZbZS(d1ziH`|efqWdX=x$<{W;UrU!T78 zbp4zM2jlB*1Y8e)sBqYh$H45}DaM_>x%X~OJ*jLl!Ahr>-=-?N{FYyI`t5VOHy$^S zellg^j4z%wx&D9upWJn7dBhJ*vrj+uwjVQCDT`qw>Uqc^UxKGT$zWDTmSC4e!@LD#!q1P({|pkXFE4m1zK)&+u*ye zaAx24-;ekGsv!b^oQB*BZWh_SpNq>DSuFzhv}(HmNJM`>(5gdyjqov*+s= zji1d<{w3gl&gRn!{a$9HrvLs-&+oIj{8(Ny zI(W~?*S}BQ+T4D>#z(w=ebRI0%EO1wJ-WI+{C?JQdACv*yWMXjlatKDv^HzA+j^s3~>hYrVR7G}Fk$I9!!&z{S;C0QGs(3C%2{>9#~g z`hLfZZRcdPjV)bH%@vn_{@`M3@?*KZ-}vpM^UuFG?Uy*FH@jDQ*SYZcsnxu5dhI`k z|LgnHe`nGRb@4B|`aAsPz8UJb-Fq6hIsJwb2E+f)C%qbg<_uPnDtQ~UQh&e^S}qhZbd(!YI^xET*UKAd9u^JeV>X?wM;RyrvssxI*= z%SFrGDF3+s&1BuOw`z<*ZntbY^#26Udz&;_KPBnl;{u#8*-4NH0IhHE%Gh)BEKF=p1zD+rE|8AUp>ch=s z=bT)<+1(#+FWgoaHNScHkB=Wud~fN0&pyr4W`g$gU%`B{!f#FaJe~L3O@}AtalVoJ zK5W0w*8e|gp(+1IcGS#_?Z`Is@L9}>t;*I#u7^Af3 zs%(7C<95b0%gg&d9<(-k^~{9ll=Sr%n_bzLx#tA)CGR^i=j|!$W$gahio2gE{k!>) z`=Voo#_dyIx)xg;bqSM}e)&58|5TQJrAzMHG&A$_9qDs6-kW23E=xVS`tkEC=j)4l zO=~K1V>TWv-}WGt|LEe&^>-KknRCOfAp5zo^izjw!%4~3(MMfAJyOUKM*RuJh$=tiH6eqR&{j@-_--o7(d^7F8%!V;@r23-|wDhU9a$b za`@fsH@cHs5A{)YdEg_M~zBS97}~spp^k-k8t3 zzE-;L`Nq}On{D!w2j&ly1J!60LJJU}pVjK6pin}bOs^1%JIbl72>H3T9IcDpgNY49oSK@;D#~=UK zRQvwrH;*hdnSV1)STkGsxLIUK&oQ|q|NW~sJM+E&`Q7%j?A_lF)U%uC)abu$uK(-* zr&#_x`v?0k^Z&1&uTy2bVzvR>j*|}^)p|Ie=LXvFGo{Q*xu>g9|2+Ty+`q#7YGuNw zlx{D1dA@f;boK0P`4cBLRX$SWZs&Y$cwo=$BWlY$cTKKN37mYJ<=)|jAIoB*-@Y<4 zF0H%u;77Qg`n{BE*P<$;d_J6FH~Y-?*_|egA9Hy0R&Cn_ittx2`d`@N{>7`j?+;?w@$!{a47Y z?8%II>=UyuE?fWZ?xOrZt|bXv{FdKN&HS}Z{zq9&{Jw6FE?4(#?(UaM&iyk% zGW|7w{nHQnJNZ6+>O8XA|Bso!p^g0XKPOhEZ|tef-`3qO%aXrSczN05pZ|ZJy?^K1 z-0k7_C;Ix=&C34bf2r))>a{V~%Zoambn@H1Jb02lZln0&Q|+lqlZC5po;qG>kzy?W z<>>Q;muH+61>~k@6o`ZdUzB_qI`Zm&N4CJpQe?Jb(S1j@G!xA+j5%@7uM>;&Ws8y+7fM zE4EtuR3t8({q|UQjeU-v&YOu3SJy0I-upzzNTlt&$$GKI%kI!wKo_=6f4zIm^!~f3x z-}c+BeYkmE^x^wuw)u10kJUL}-+Acw-zsPSI7P>{Vw&f z(^RJ9iSeQ1pVQ<^x6NqFz05b+{MP49hn6ny*mQYj;%DQkB$o{*R@;S&z5li6&!m?% z0enaPRo=<|q@P#uvr^8UC;ne0#OH4@MFww*7Un*VynPlxz-Q(vCm zSon3@cc~jsXZ@S`UUv4e+fQ#>WbjCo*Bo@=U$*)G;RUiAkCxeeII;QXq-tKDV-F5L zyK!p2S#jI?Ki!3&&1-H|~z%fs%*Eos(82 zpSb+qT<4a=%T4R=&epTe-Ivt=A<*jR-A7B$GBx_o*PH#sx}j~WtLNNUZeFhD?X6;Z zm5$Goro4Sqo&7znU+xv}<;ZZp{pB;h25)=)_}A{3$*tD6fBaeXc=4}Q%{-ECuz3p_t@N~SJCKJZ*+~_kF?{zv*Z8&T7FDSZc=*cP8UAS zoY~Fy=Xu{WvFn|`v0cjh<5RVGgK6awt2f`@yy;g)ieTMeWwyuiYDGUD<@>EY^d}+Q zy0UhqZ2zBi?3b6VvkiL2T`3)&QEVF#_}EF#{Qn;x>G&tFyK6pA^!@ciZqKv0cJ_ZK zmACy+4WD;He_iMG>f361c88jTiu(Hbos`0z3um&UWi!AuU})3J^lYz#pBI;Z4*A5eZ8i$ zZR*Pn3vG33I*KK3|C8DL>zCBrTQ?fm<);-?nJ+(S!O|Wre;1r2^pCCGnAB4v_xnNp zAOHW=-`PLRUHct!^uf+7eJlHh|F{3Y{lDk)x6s!+KL?aI-Q`|qd-~s^dokwk-`ZE4 zkW0M(pq=-3*WVLXfiwM5_AfiUZtt4E)Ayd6__Lw-`{9Z5OSk^Ei0;{AJ2~!o`}UpJ zx6HTuw(a2MIF95(P5pnBhXs}uWVx^3b^Qb5ZkvlsmECLd&Rmc4yMK?*{`aqLp1-$_ z%;hzj_d#db%EO-gU!VP6qkTR9UU%-V6~+wFPX+J&YF$6I_2s99y)_>u=o5v#C2#-TQy}^o<|?#66Db+@G9UGuz%^&y%>v^0Qw^ zMIVz55{s`%R*Nrs;rRT9((TQglC$3m%rX1iFZ=kgMOEGUmp$GxwjX+Go@dnEEHG=E z%2Peji08y^L$37COA0UUZ9AJQU;SiucT$3Xwb{+1-#a?xZhh^$?=9D_`?lm~+tunq z-}{x-N6w~4JZ(FFe%JM$N553u>k9krcf=K~U%20p=bgKKOr`Gr4O*w`|2~Xc+;`;H zw(nAQJ?%ADuiwpn;;HjUJO5tI!;hk3Mc>-ruI4`7DQQ;q_3Ei!-j5HqFMGH-HD+tt z-j{L3@0z`88bwSY{{M+!+hbbNKl|B_O&c$5y~wxY?}b17 z{})>Pzj(a;)={HPCALqdx%uzhxMd&v{eA0qOnK^aCn@62;^pU-m*1E=xqZ6+kE83? z#NMqgT=V5}nwoj~{~rl{+kD^kFv_{iTl*@#Ui+|u{np86`?gzMlDoFyqPy6oM<(i9 zq67B+%>Nv7zu7$cmRrWtGyFAHITFkqY;`v;h~-O_@vO+!pT?K?TTb$^L!Z>@nw)#b zN{)T8GT5wP)u2`gzR#t9bm=Icq9<^p~8Aw_AO9-Tb3E@ozRiHhAW=*X(KL zk2e>m3;((`VcWeQH`UyiTg7g=_alD3=?(ja6_an)_^&%*BDVG8Z*FWRM)@7#4CjDP1|7&6Wlk4?=uYa$%I>SHv%ZY=F{(pSp#S?92c5MC}QTaNH zE?&8s$BA6o;b-3HRr1g8XCkL&EBA5Yekd6Af&?QehM;s3<^8wHReL2<44 zu?ZVH-rZ-oZvvhV3ujxi8NAMH|L5!fx5qADd-B@%=dXX7*qvj_-#za(|F`-j78&tH z>kYr(cEA04PPtY6&GfSW8|=>1ZOT6Sqa^L%_Ph(X|GqjkP8 z*5-px+5Pk0KCYYCq+c;{TT+1ice_-}+LL?R_~z&IPyV`h+s&5x3iWrtwwuTPWm#?P zV%Ec7b&hX?<@%?~;~g&Q)$OV=)%!5jTDs$V=xfU}N6-3I&0T!u@y5V^KbrsBd|tJA zsqU?Pg7+`luit(BfV|tM?YE_a3{Ebo#Qa^Uov9 z%==G%3fHqfpWVK6U%t(R>~o#Mwx8_weZKiaW8OQ{W89NY+w%9>Zkzh@kjnmF6@UFJ zkH35IAiMl|&B?vXKAyY&Ed5Ux>)RbOY(n+;UwxRH|Mb_}n)Mg{{8&@%EC0`mS=xQY zt{-vb>u>&lRq;tx|7*X+6IQp{X0vIP-P2=ppWF}&|N69-hv!Fr@%qR;kK6UXy$>`l-8W70Z*K7QW!UCOM?$FI-K?WgqbADd$kR`E=%g z-{Xn*CARwN3GRG;L@xIIug-sBn-r z&F08S5&X+%^d@@4&+Lth=g%qM#Mdh;m(}jg^W|s8qw4&c$J`(9rbW%=>9POOY`^2< zpZoRS->;Lc+tRu6k66Cd>|nd+j=8&@+;AxpV5_=y-d=vgX~MZ2^`EVaa?+&z+S%TIc_<`q^T@LJSo^m(A1CiIYv(Mx z>6Bys?ojJ_HJ+P!%Ie8Bn{O+|%=F!2xualZ_Tt;mTG;t5zu6vZjp=ZoYh(ZF-k*?b ze8#dHGtS>THlpE9wjX644aPbFr(S#{ca|F2oTRUZ3l zW+)$a54rbL`Tjk*eJ4L#x5}Cym6j{|W3v89OTCrc4Xwitar5?EJ#TmHb))}IyBA5! z=eNn$vPM6CFy*nh*~X8~edmh==6&eW{C#iUr$gzriFF^{_tib-zSmp%<=mdlo8?`f z+d5?j+v~*|uD`!n*32j;>$KV2V{7+4bGH+moAK6HHTIX>r)%2nyYlN_NAv&L`uzCp zys{S?_3dhHc%`{5m&EE>MJ!$3JI^Y%+GXde{8inre?Qy)c>Dfi^Y*38fBP$=);Io> zGylYke)8ufdbeymY;WDs^Q-UhMbP>e@1~vCqJsMNJiXTZW$C|8x#Xmx`uENG_t^h8 z*Z-0Jp5I_z@$UQFM)9-*QYnA!>>K_c(f=zi|0M3R?V(%k=c;+*(iL*wYM$EdcCId& zcdOp5`S-uZZBFO;{o+Q&<-KY*&+odP!*l)KZMi>&YoC8pN%yY)k$yM(nZO4N?JN1; z{(X+S92pd^SoEuF-RGXq9?zN5W!}#JzBlds(hV``OTLAtOa4f#xV)Dw?|DT^T#20s zcUDZ7;Ch=2hpo^1{;}PEJ;VIm{gU;9-0?XZPV>p1e{cHdux41vxntNoOo|pbMIG>p4H;4*EjiQTG`J# zx9pKiZFS@6&FNPr+CMdmZ}ye>lcV$av*77g^*w(wobSo)Dr$ViUGe($uU`Ed=Jzfp zb9c_1YyADqfzB@T`d2@H8oe`3x*A{5!s(a1uI|$DTZh)g?kRF)%j`eZc<0C2*Ka0^ zXYZVR`TE`LUwj9iS*BePKYm6%xnb?+IAyuYmgROU`^(Ps-1~hm`(5pa)t_t6Tm02} z-(3H;|NmzB^Z5dU3GdQZ$4c- zl;pS};>T}iyDP4{XUu+a-Kod__>pvf%cS?epE;Y`%-d>jQ&54){r#4h&rs9D6}r@q@-1LxY>+58OHRv2+$i+K62nr$09UGK6iSibK5i?5E= zhW>tm?Y+A4H(pF?<}Lfie&N%Hr_1_-Q$SlsjxN(N=jdNs)Uy0;qm0{IgIPPaZoFF& zArl_6^)olOEcb>ALHXo@(@%SKq^k>NMI5=zc<5m0Y~NnJ`5U(0JoxP3b`W z@4OUcp8sp6e2Q~Q@P4I@OE&lXe0bovTAkgC{FvzizkhLjTsU{_+W6wxJk}eOEVK^i z_Do1Lzx^uZ$?@I$qGq1vmt11|bE@p@ds}n!+S!?#`tJr-9jcz{{_>&ro}-87wZ}#L zxYGA!`Zj}3ZJkHKX8(T1*)0Bj!0^}V@|)FXrk||eDYE_J`~3!q?zZ=z*8eOrF5JZ4 z|8rsdq(HY@iI2MH6)u>6^>y>V>}l7S9*55<`MGR=o!YW{DK`|ZS0t>jx%BbFaklyQ z<~`Z^{%P7H1HY#Yf0O*?^>66Ac$PnQU)Sa`t6Kqu8s_WvKIQ%PcKS-?O*a?uvj?RG z@BNlLL%zcHyWNi|#CyVh@u zHN4ITT9;EKkg8BO`&&EHF+kNN6=Jzho ze*M~2GyOlPEq*(y@d@9@qwMi_v%l>Bc|`nG+~xMHw~uu4ejU9yn`icuSIdjS_@27% zzGJFBWwQGBZQrG~T|8!FSoQwT&PQwZR;xuy^!i@8bMyNb>o>RcuS>+;JS}$nk;^HA z)u-PtU$Jk_ck#;gJHH?7&zAplb^gD-$&)|w=Qhm$oU}m-~oH_4l#3Xeo20% z=2k2@RZ z=48ary(w|(a^APY?oV=rQeGTXeZS#SX#cGR35xUI^36z053csF_>nr@-YA`K%FFK1 znSB+BAJ@;#N!{-I?o4fN?uuX1b|tSSar@`q`|W(xDBv*JIbE z@;N(rZrgDgmC1UyQ?8iDeLWa-dFcf`J3ZZjvv{9dNSa)`# zv{?SOy-$zs2y~m>etMbArBva6W`EyKo|x)fU3UGzp9a?R&h0#LQ~IhZFHQIOr``W( z@%kw47n2s6^Icmfqqyg?{+y(UH<#Vc1^4hJovIet{pMP@y4%xVJDPj!66V{?=UBgK z*QRgBMD9PE7O`dJ?EW{imG9kbUd_D1=+?)AA7|<;@673$Gox4Y+k?^wr>9JExo)&$tnfa@();~9of4T8FMShu{eec3IhlS=DWCZusA3G|4|7>=V zM$c`%wvUyMqHZ7lo>tHB&Z5bE@18k39^e1{p!~XBMLolP&Eo$%j6cLlZZMCyeV+e; z{omE~pEu^cpH;nf`>AVhzdif5a{9XGn+qyTjl8oaS!#;~kIS`G4%qF^W6mWuKqc_J5f1kddo$CMCb>7_kALr95SIfVcy!d_QnuBfr{W|qIj}^aa)XaVV&^hH2 z@BClP?@x4cZ_NJFI{Dr2JJ~-TtgiX6(d>9$#Gb@&|FrIBpNqL-U-9dN%)JV=>0f(y ze6)G;^nk^I`*VEOos?4kQhO$>?sm2BzMGfg-Hp)~b8ON^4Y%Tz*32`*?`CI-+wXn5f8)&B$A476&nnn-o^AK-<(JFPzWyZF_A6$- ziT?KZp5D&hKa$18b8?dpS>HK%wH&k=_wjvp`&Il8>YuFtw{dO@TfXpVrF##KIg6Pw zZ}Ck2U}kw@iT}3F2NPPXRA!o}&FbCV+m+Wn*Q_l2G@tzwAE(J2_l+6kX5Z>;+-DW} zZNm2Z2Exo)i?q5kMXuTH?dVarzIBQFwxqIcoz0UI8SU-7{V^HB-Rbrm{W7&W4QbQg=2@{)Bx`XdMSg72eToQUCOg%o&zqY30WT(DRH~ab+iTT<2B^Ub6dM`YyZ}aNd zb-8aRd-{5{la}v0^^Dov=ke+KN=9wzuWM^IESA&MKX&o9MVic|et*7rn`blkU0}cU zRM@ZXhQf2tbneg1Juw$er@g#-{6~^tb=}S6^y0N=uKx&eUv0c?&u{&}S?oXk`SiAZ zz5nd{CROn-wn-DGKT=rU6Tg3Qbzb$sGyESjh2`sCJ>}oOPb$5?nJ+fCSNL0C-IjwJ zitCH^|9G18BWHfm)SvhM_^s(xw0ZkS;TH#grhCq{XS&mV_452RRx{#D-)g$<#@my5 z`(DgQKc-^)?MJ{p!-AvkeEB?YW47FLl|H_uJ-x7JZ}FcAsp231-YI*%vh3H1KZgJO ztkv^sC$8DFt;XbAk7niW?+4$%t^IKR|F`*j>zQlPul{{}?!al|gxIa~=E)z}|1rD% zb*BAX(+aUCv6F9qzIK`WdU17cF=y4jdDh-Hr6<@PxR?C5{jA)!>PuU;>SriFKa(UM z{cmgfjp>^&&+N7P`OW3+n|a@*eq6d+%p>#dXQlg}s%^DjwyxYPx~=&9ss3+QrF(8Y zY&@zgUh`R1uKIiDY#qsO%5UFSufJ(4|8L_}NuF~@P5*sixB1o3^Xr&>YW~iqUcMUX z`$snaH8K=4uB%LlzH8TKcV(}e{UPS}p1txl{P#iqXYJ+ZV!j*^D_t+id|Ig8;_2Hx z*KLlSni~G^RPN(sULN_lPsf5(j`#A`SXDeZ^?3P{`7*VYH3>5ND(sF`_evaD{(s8RrGIu+`+mBeK0T+{p>ENy zz}WcA$??CANzOeMp3OY>7`Lre@8maY&wSW=a6|fsleTBps;kQtz5H-p>b}IP{MqbJ z0{?AMHp~8eZ=+G+1ZC&{J)LJjJHL-Vy7fo*)bjXik9QmHidbJQ)bq$~?fslLyUcdZ zVK6^dw592=^V(i(w{;Px1?7JolGfzkQ7u2m=Faa2@#UJmsi<@|hwcBx5-!)8k}fVi?I$f|`Sr<^)$Ojc zJ98&(nyho?=EG`3-ZXZhRBn+?5eGEp{Sg){gnc@(;P2 zzde#we%)Mm<8pXA@56%&=FN#Yv_kIu>^0>dFKhc>leO4ddE(msmj^WUj~>{+?Za&C zUyZTT6=oco&RhBUNq7G*gZoEa`Q{dL_W%8A`0vJd1&aiQ}K(g%3=FX>sR{QSZw{6eD2Qg2kO(C z&z^kZ7Juday{8)rZ}0xP=Fg?oYYUgh?+gEaF#gBG`TOk~q#4ZbN>=pqL`+}v{W<%G z`G217-&no+?+K~Y{Ef%#|{2yxWbN%||;OdGS#bUQNKRe=? zUKpESm!++*ZCsO|`{U>go|v1{ihq}-tM630fA8(u^P6_p8b2`&}X|?|+A5WFHnId-w6vr%&ci+oCm0o*)uiLvHZS2RAo-kLo3+F|c zsb9XfW5Mq~afjs z^P__d`m$dX-?^3*V1*8$NeD|K7ByKWg@w^T)hi$CW*LUm;s<9loyc=yI$5!4*G_8>dIRZ+p)^ z-SAc5f2D5+^XneF+pXh&Fe&fhvcouAF6yZ>A4lw)DdDIa_Oosuv}j%R;g&DL=C9MclpoBp~r;%eJ?H_4hj4(2(P zvN!QoZ}F!9X<4(qKZD;1e4MlXoFDtFCr8Au-Me{~U$XG>(a#6j!#=4o#eMq`d8YaH z0<)7oI^DU=HR4Y$np(b4tXdKA;p3j8y8k|T~VoVLAa@m$-l z6|ZtO9PQhcF1~8w>Fs8FlJ5Naux0sSc_DeZPkf85Z1*WY%{ijpe`3*58gk%g64sdi}9&WmN_*m{V&0 zHrGF}=bZgtB&$ZEyd&Q1#5ThNjqK4zZytQ!xPPhszK^MIBX?#B%UjoEE3c2+cO|e+ z?x$3I%8M^8o;pcQ;`zC~ZzSq}7;oR?Yd^o{WUq~KuKGdW{RKes@FWp znbw`FQ`r1xVzu3~C$Tn3-%RXZd}#lnfA7uo&4IUmZfN?<+?S)F{4LO`JN?V4$DeLW zKGK%nAhrC~Ywgl+b01GKcR$kTKdshm-u>)NuKCAbol)MEqd!ga^Z7Q=K%B*KrQ}mF zYM)nCpOvV1!p*lY!Fl%ejm`f5L%&PqK5Ax`yYZo`_iy4F+rJIRGEWiPBRCe04s~fhUpCqoZp&mn{AXW_{Plm+(Ae{!SPZ4!(>l~29{_R_U*DK z1E+kHe*ee5_PTfI+N#y}uI^n`&6jy?kNvS7$4rm?wVb~$)c42%_V`{w>+{Hq0G|C;@S(DU4)ZH!*?T6I58()?%^_1CQF?iID0zkH%P%NTAiiJl=A(BA8rR}gjJ zm2Sm*w*J0$o0>lRvfM8vHy?D%&pZA2P~vJizo)|1{|#;xJnZV7_v4X>eSYl@0mI9U z_m>=IRkMBXGIxWh_(XHdBz?1AVt*v;&0=@ue0lrqz`I9@$p`N?3b^m8seJTo?R=f@ z-1is0w_h$?{5N?^;m7TjvNm<6U1!KnIKS~cZ(Xunyurhj?<0KneA;bzc6R=r1l5wG zYU^)J?W}pYcNtX*pVNc1=kpNM=V(@;`Tt z{QqvZ&+WfxS-dMEeBI4Uozgb`8(*$pZ&vEWZ@=%s>DeDWzuZ>8aZ&L4JMG2G^N(kr z3XY6^ylC#c8yvrXJo0?QZmoVy{eQ{Ji1;NnxvBpe+^^lAHZN(>@p)_A_nY$-#mqg~ zck;oD

X2<@8pX5F!?H!IZ2%Kp5)(fjUM?z89bHYO|NpZc_8pZj);O;epWNqlbW zZC`Htei@D8IKMts=)S?Fjq-P%Ke$|zRrAzso@eoa zAJZ=XmZ|xC$Amw<$@ba%zxfUCuj~Z%^F1XutiSzy{(lC(FR{OOS~vb~_~meya(%QjwL_b%{$appCfJ#u^G?Aonu|9#!Q{ZQ@8a^AQj zmM8RkY(MMW*_Zw0X8ViFXVw4jnBaf+{Lg9J{WqqaelnB)x&HjSo?lAU9=jc{W&i%Y z_IkWE+4ygxbJ<- zx{U?jZ`t_F+|d5?kIJ(xGwi`EPmc z^*3`v<=p1q`(*j)qtt!V4S%0n6)$@~tMHqNT)1B^@7|Q-xwoFM>HoUD%tq+_qtyLx zFX#7^T{XfJMAKl`CHmuy?Nk( z_y6{q>u+wv{hX3r_UZbJ$GLGwIqUwtF!}%J*f+zpvolZHSC_q2Onh*rO+I(twTb(R z&R##+-S=aw=4|f#H4`81{KLj?{#0dM5@&P$+dtXe-7#9iZ=X&0@gw(LX1l)GuB5v& zo;-TAuz%j(t(7P2?!0?Y;!*SPzy-hOcI@wq3$8yn!(F^&=l0@VJFVWHz4?v#a|Tb@ z+?30I{;v7Q`@#Nq|Nm2Z0^J@HnldMDxv_<_Mtq{m*+bhbBOG*>vCg!=qWeP6_OGw= z|JT=(7fV!E82r#`{BZOAp*gI1g||Fp?(Eiln)2hqeTFpu;C!T-wL9NF0_<3j7_`)|&QwrQF zf0kJ0=N%Kj9$Y4K?7Zdu)6BNNp7OEztvx8-*@lr z3T`~TeBZ0_dQ8NDpUH(k_Sc+P+5Tku+h6;xm3*u;Z#(t0ZSL`Qulm)D`G1eK>&Fx* zpZc};=fdgz#kChr zGm0(iox3lmJM;Ik>pRts9t^9wb#q1i$D`s3bx*AKf4088ZO`(*PeSa@&)726+%nMr zQv1We?`nIWPCsDKb~fNms z%dG4EFRIQQ&y1=4w?(z)n5O+v9kX4pN)p=F=l=a{+HbgN*<%4V`)u=w@0LHlY>zv$ zzj*Gc>SX>?6wjgX@@d~|uOH!^ z|NOpb(MChR`5wyqtaK!8LuiIoIC5 zn8Ez3Co$&g{QB!1uDjLmoW1_Qw#IM%odd_B{a&cNY+t<2M=t-TZ0273Yu`U)+t=ML z_f4J?TfFrBtZQmHg+eyde~Sv+t&OdH^(*nP>?CPFis>`k$K5 zzOj|Q-EjE&wtvUPb1aVO@Fjk5=D$|UXIFb2T*PPEo0QjmUjP5&`SciIP8#k*T7jMh0S-if#ce{5-+3butTRt!KUw6xAt9s5i zcd3fS-2SuK{l4l+s^?3*wEXfTD1Y)}_nVh1E(L#2zSpU`f5+wI>xmzB-~V~8&vVY# z>L>MI_4fZf#%U(U7h!ekIb-NQ>F19=+$+3w=(6L{Y4&DEcJnhAzDQRt|MjEU`^L-H z4I1sguI1gksq^kgeUWXgczH|@_ewr%*|!x}p6kZn+cEj5+`FWPt9$H^vn;PDd(q`9 zE&X>b_rISnjE=Scll%Wn`2WMl;d=FI3Xcq#7wSy6h2#hk~h^yMr*m3~^tf4(YX zamaG+e*4eGzn9Ocw>b4>&+O&TX4X_}IhiZYQL#Iow{GJx*EP2uXqitp55GQ3`1HDo z58ou8cMX`U)Bf?OoH~ExZ|&zvs@M24-yXJPjo&F|esFnB3Fqd%Tk#L3asPViKl6Xm zTi@?j0*}p}a=s+Wyyk${m)Dn%sqrMo?n``?Z+D^cN$IbI1FgYlK3siSb>s493G4TI z5rvPBM|@TeSE>G`o?rXshwk!+C3;78-;n#;%qpJdbi8_XKP7VJ5OYHU1)pu*Y9;z1!=~zMlK|@-eslH>Ui2zIeB_^k3WG`8OX`Jv@C`bFF%A?ez!t zoOdnG^zOLOIQzb^ci&N$>wCVOFkI+gzCZfN^88mvwd|jl`~FLO5&i#j-n_au<-HL{ zS;HmcZOprxxMbQUrx>;gXd-EW~Dc;++XVFp11ew!3mS+?r%QIe)DftNm%}y$DbANn_Bld z=v8mCn8P$*f6v#C{!euBZrPT|{cT<>xwwb8m2hd%v!NSXMN@S#tL1QeY-AfVc5f2+qAr0 zKmV=8$Mtubxb5w3{fw9N`IDHjeDRqy`yb1F?e!8b1)tNuza-+&LS;1}wOC8dX$CJg zsK5Cf7*n@GIq{Kvn_2&jzY^a&a=$#DVpsoALjUo(Ys-!-zhimf@DKhPp88!mH@aAl z&$SL;Q?qGv!IQq0!_)Rv?V0|2qIfdDd1W}yvx1+Qe20#<);@M{k9~8RyIniHMnd^* z$&Gn;Zsh&n^YH0;HbwtmHc2PKV&|Ai+}V8EedE-}If zlB1b#T~S8O)~%A-+3Cr9elFzpmVd~u&VTd!l%D~|K3>rbFG=j2IHUQfW4zpt2Iu9L ztNg8F?r@yZ6FTb2Z^xJP#jNkmIr#@iMfWFtYFl}C@tpHdZFcOKSGLwVw{X?{*?S8g z%D3%upI>Enem#7ty}W4MzqRszFP^>s z{>=R;>Sw24n_-#XxAA+1`X}eE(7GGD`wRXv?9+b#@A0_<$I}l~ zKGdjC0DH5 z^YO&d2QQDiO8fL1+1A;*@Vq<__|E2S&GPQ%I-xeB>Y`s7>$cZk=Q;7JfAQDW*URg! zKe(`0s>0^zql3@)JnnRjF3&YHxX~H=&ch<-7qYC8H&*x5fAUcYJg$v)y4ug|~Es-1o6 zezP|pR4iEA=>K=!!El+`^^a{n)}JZowSLTb{GU?)-S0=Q-kK0@_4H5febY~i@9(La za@>v2v)Cgq>`C6dPZ{s#SUv61i;3af_n~{PTwOsIyI*DdZP__`I(6NX=b2evU3T7L z)9*X`vyW)&f2;o~bi424mbuBh9~)FoaR1zOW!uh~zh?T^J`Fi{_5SzRWWAgJ<9@|n z-T{uIS*M?^tNe4LeSKxtzFFFGpTFJ_j=KN$Y)RsuSKI498UJGZ^ZtJ;ub%+x6~+bc z>NB?{Ms^5o;7-3a`Bv8QDwJ;o_^q1Ze zIXx>Y#q4VyAM}fPHg)6K?RAIR4t$nK&);>gqIBh=cHz7?AC|8F|K+l3N}+rC6#2Rv zCxsiEUw@iilx~yTt+4u>7a7<0e=gtfaLMZVFOIJ+U9Nj?!;_zL>TjN0YUusqP*?DuEyA^@zG_RW ze_ZFY_iLxJ=Kq|kY42~Ss@K)Ls3=HfUvHX|`{MQDi!+XC|NqmrkY7)x;?bhzb@Q9Q zTV~AL^UU&x{r~j;hyOBHbpNiqFZtsH&yD4C^`G-Uxc_fz{l4_~yT28uHx=i8uY3Nu zG|u&*jriODU*_=NJJ+*!vc>LYXLq{&KfN#e+n?R%Bz`?Pvev%-wtdyz+gHAR?C0iV znRYzV_4;3qb$co{drsS5{MPFEw|jB+I>+VHN@!s`xZa$N?%#Wkj|906K z`t4J`<6fX=o}~O};`|>rr;O)4Gp#NTt^O{y>;3O_FXe8`33h@A~1g=QgZG5Brftf{g3H#?F~n7D{bAN>OMDO!!O-WX7AkZZ=HDa!0kV}hbEoA z$CS34{n{Su4u9KUFE=W_GWa_2c{6*lWvsNX%*)HFW~^3bp0f>nYRy=yy6tP+GxbGt zZnUmW;PuhLw{Csh-|9W0pu-^Y$ezH{txx04E-f=HU`G;-4 z`I(x-+Ut||m>GPC%GoIqc+D-oe&d@&tJ+s`OxBHa=bAk=K7MP{>iXmv$q9Jaeo;#gTj*`Kq~hxBDF=GA1p{vUG1 z@v7?8MAzqgd=9Gp`Lxwiu1cb6%e6uS}7B**tyScKI)b+`)fZ@*W4p&c6|( z@g-Nz?Y;Ev3 z9rN$4+x_t->ykU`?%mj8_TX?*b9?C?qf(g@?|8%aOKe%rzu#MY&Qrx1CYe|xhZhtKTYWHUF}cmL0W>OLPM zt53;&GdNs6^9tYcieD00e3$Rq#B8g$|NKEM-z@QGexG+&rq*TnM=f8=|GDJ(roZW@ zzVkmweFe(>KX@X7zr6%?)qj4i{~1|({f76Ptw$nL*ME3()AV<%v1hD&Y|!oLwbwTk zi>rQH^limE$6E;#ziqS4R$pGU`i)He%W~fzkD45<{NMKeS+=;Jf8*Q8|8I7;{ry~V zSfpRVyzgMCeS_ovm@aSoK)?8U!}?QoeEl7YCy)2N-7@{L#mnjUr#=)+ui3KySozt8 zeFr_y{g}vI+4cDLzU*n=p9ssB?X&z)^|IVI`BeDlj{no^uIt=?81(PS**}K+6NGo1 zD%mz^OLaaud3Z2s?Z-{K&>r*1y8`#Y~kl(W6Kz0;y<>)q!Mu0NLfv!;&Wzw!SU z`tKiYWXR<+yDqlUb$w2<-G8+;CvT>uI=*w7WA^^8Mvjbe;q3sQgw$HKm7KR1wXfZK z*5U7zqq9$*;`w-)ZvFfDTl`~cUS!LBEst!9pOafvJ)`g6VmqrZllWL| z9#lPderJB~Vcn;s`RL)zj+F5-5w`L(yMyv)A+tDq?Iz}gvK+w=cE zY47_V)BU#m+5H1){MRKeHabVKe^S}@)!gvHM0T^HIK3;<{AM4%d~}|F#PYYr(W$Y8 zTK3;2C+l5!n!WR!^}f@tetLbcKm1bO|K?9Wf6nK{{rdU*^DCmmUFx=Myjr*J<>Y#c zg2M;bR=zmAq}FG@Ooi5I!{`}L6D#A21;uyGduo*b_vQ4^;aeDOr?yjLSHJmkb-i4B!OQmBRlCn0T>g2n<;5AN z=DY8nQ@Pr_?%t)h54dyhVR+c|mD-)S%I z?=8RgqeMR7Sg6gZn-{|G|Eu&ru3VOT|McHGk#GMUeqPCbsQJPBiPQHs`|hsZQCIo@ z!h5Oj!9jUF*_97|_eB3Z6MCQF%e`;0%4~NyVoK*s-8(z{*U?w&SoZJSTKwTv@@|P& ziywQQ5|%62`?;`A_=WPl#0vBEN5g&I?4N(%^z{3#=RJ&uRcotYX8~NFUU&V?_Dfss zc3-|JeX7~a%KF+DHU8(f7Co=LUw*c3Gf#Z7+jeV>cP@6}>7VYJGut2YJJ)-Y=T!2( z%irJZnr~O}LvZK8%AbDn^;@+!Yx||XKKT2La!>KTBi~i;n?A0nJlJ^rc;D?QKSgdI zueqLcMp$0QVn^+X^4=P$@Qw2;1I+S2X!;+0zt;Zmu_bq|uy3AodUwpL?DaEFDU07K zw>d6Zb7^jP_4)R?>p!Mt`y`%p-}56h?|t|3ZN>XGgkSe#XTNjyQ>qK!nY!c zapsd(YzNtv<}HlQP}~2_>~HLr>r5sKbeFwJe8)7)Zr^-)2ZQsyi+MlVRNOjp_>?>M ze%FsXwpW|L^#yGU8pLnZ;iXIkP&Kz08ba^7sC{ zxNT`@IZ^rTyt@&{#g^4ve0)FJR&esSW9(&h{>7s?{xmPaoeoOvO-GAry zY%Tlmr{(QGytuO7c3zR5f6ar5{kOhE@BXv-y>ESS?NgR{X8n7g-pS5>U*vIpkGg^9 zj#Cu}j!j-`S6b{bQ`g#}WJ~4oe%`yyx#7%PA%7q4oFc-owfiNg7kZKJc;(j#8-vr&i?wYs zaFWzp?AXE3`ne+bap0^F}XFPQOQE=_>$C@*Be9ZRm=L(Bw8fKPHSK07%-khq( z6_qos-{pL{aiIBP#iQf*WYouBmo)0J<&9K5*9hV3+a`L>@Lak)=d?2f2?^EBhY^vmBL z?M{1s&R|;eSMxnjdmmhyx7l*?eeGqvEPb=F%yW}Ulq)3TuVbKmVJJo$4@WY=7``DA0Loxdks?o;6G zfAY)TOC?Xe^ZY@1{o)PZ?>v9-dT#UnpV$BYc>dXc!JnP=pTg(!O^lQ@(|IIt%eR;> zR(j>8vs;|Y`PW_C{PJb?GJIBs%Ed+zI^}8NNekjAC9`kzUgmHGuUEnazeWO^Nowmslxi|{CQ?GyR+}S(0P7v z>hX%3AAS}#Y*NoQu6lkfX5MGl^QsCp+oW$NXkQkN{(G-CH}_Zf;l`tNh3)%$gY0E4 zxz6|dcI#{YIrs20KM#CAUjOI#_V@pPZU0}!fAG_8d!7sOk>|v}+qxWI|7QyKpDBfz zXQb0}Pu;VqeDO}rKL5^*5`|Aw?W?|QPCcxD(f$1#?pro4ds^}(UM1?EKlmzW?f!kI zauOdtn>+u};prF7KNpd^^ELCj{;@Y#6IXohJpJs~q3GS2>9hL_KE<9pv$t@A-L>mb;rNX z5}f z_xox2_!?VvW&c{cAJ2}5Y||09)?a7IH2L1Kf}g?PUl{8hW9qA&ySB*g%F9)gKe?&L zeAHb3?#1QExMtr1HH#0L`rwSaR`&C_LKX+r&d!ClW`LF&t zn%_V5s8!dk^w-z-vXc4dw+L6huDks6{J!_I{yO&9NlI6o%bRiP>CQjFzjxJMf6!%o z@cW(fPebQ_pI&?Y#OY^w1uyJBRoxAidu)^Z{P@Xta=nf}^`Gx}^Mh9}r{bw;wApL% zpD)Aho;#WCD&Kczx4hd`>EMcP)5m*_lEdZeem1mEFRv@>Iex-M@#x|`|McHo{o1Wv zT`KYF&x22YcBlRAKVNg;ET5i@#I>exa-VBU*@NqTC_J;j$bS29oZpn2*YD2?+`cVg z!{Nz!$qV-zrkwA4kmK`tAwS#hM9yD2?BG3r{nxsO(f6KAdGGn=S>p8` zOXGdN)=kt;Zdv}R@A!=e>)u^de{S%!c)$O$cuV=0=EoCXEf#*ayykibXMW5S&OezI zAFdypUYEQkX7-Lrzh!D}8P`ReKN$airv0yOa6(KNvr{+RXh|$qf8}z&@1nOCv018&?{(yPBu zmhD-^Co7x%eRRz~;SFozEvrAD~h%O=I<%S`>)vT^53b9=dB(d~RYK8U?Jq|IOO!*{;lk9$Y{ z#P>~>ue`YK+%*X;^Z#E=^S6jE-fmTAu!|uh+)V6OCwG2L!bx%2ol6eu+uyoz`Q`mJ z$=+LzHvhA9`#WKktlZwGiu-0tZ}|FgxlUd6*=gGP|0MIf_*~cFJ>RbJ zUyo6{b<6$A%Y#$(Ym36(e7O1Oap3ZP?#7jC`ZtK%|1PrTy6xVto5_E9+hdy@iI?vG zw9Eh8_2O~#?z+k++~)63l|1ISZZE2$_wC{3tIPY$b!yJ$?z;6NBmDhhkG>l(KBq0- zBmelZxY~V{H+Q*z%Ed*Ux|d z{M*cLU!MQ2EbsfaWcr)u=fa<-r?bELYMS^c$-etC|K;*t>t4f~Kjuykp4JXh#*zOx zo5$v5d4JXIa^Ah~>YuglDWA9D_MZ6XdwwS5aM-^tKUMuM=DV6%x8aV&o!Ne==hbo_ zJ=|W{x}5Lw+#9kTbst|RKVNs0jlG(nyFJl+Gu8R!lNZllo8ENm-zR0ah(G1u zFQ&&G%76D#-QdWU`y9c~j4vL4|8jlZ&sFmKzRmliv!Ahfsn)_~#e1KY{ZYJc`r(r{ z`1z3{RAwCOe1-~8E8r1xxk&Gir3*K5AT zDywUM2F+ma^j&ZH=h}Jl&z+))e{1>be!Z`L)?;XR{H%8B&wq{+?){xYhIFqWfy@ z9}YHOP(1zGN+jw)VQBhc^ZFd0J#V@e&)6zn+%13ne}4fWR?NWz^N;Mc*>r!0yZrSpTlCLO z?f>BR(lfv2$iag?PrP#e%xm|n{bKv;=7s0Mv$y~F@!VQZHsIKi8*eOnaw{)xtDI9e zd&AC+R?X@A4jr7!|MuRWO^UZX{%#GwE*`gQ>&+!wN@rg5dvr~WjeFgWSFep0tXprh z;`Bl@f6JA8+5EFRz5TZ2pF6H!Q*+}$^HROKn2tY7j*G?Z>yE7cJa7HAYwyqPe|YxV zAD(iq?rfX=Gk%^8|17O%+tl5j!*lG@Ctm(_@^iml{H%7X{9SC>ZHxK?YA>}PTgi55 z7syo~_;C2>9(9AnZr`0dSFEq8m~WY+zx~aZspW6qNSjN%e%;0B78G-|!+*xL^>yN} zkILA)@O(P`zDPh_@b!;(3tE=Sn@`W$Q6XqJt={IP;2o{%_4G(Wd(_NMlF|2xyZi&mReXwColIdFe{ zUn+xKb!}yXcbD_ditdMY?E7ac@6<0p_STNw{;kZOdWP+f@^%`3$doLovxq(?f8ejM z#*?kzKfkM->wb=}dSQ9G`TpAL7R&Ei&v7j&xcaU1|GL6|4}WdEZ~936uI`a{ZyIL4 znOkt{$@|>`Yy_uGz)W^KKSTUQF7L|gN0teJY>v?j;q^Re(4&pTUk-#zPH z9AiHH?rtgO+DEO+?^oTCSblegebPKD*}_ZhzWMJzU!QSts_@6dLC@P|5AyaKY<(WR zv+TR$XQpjGPJDhp>siab=l4y26-YST?ccfP41eFxy0h=4HmW4ujqy2rE^z0;>?6@~ zN0#Z_%Q@Y?{m-%Byt3H-Uc1LGTQ=w4$;$TIVz=$D1pAIpBF}#B%l>eqvhDCwarS*@ zncp9C{yn|+x{2P7jj@fkj}P6uGjabfvCp#iO`ppM3Mcv>dOe%L-75e0y3934PUczi z*gtM|&t?+$zq9A^$!DkcW&epd<@b&wr*y{>iz!bLp-U6&c(VITi&(RLY=_;$vAWKBa?k9<^U3#*Y`-ztj8S@qEBp0lr{>B16|ZDlhIcbI&`n zd-l@@6LR(ze$cD?zk0UJ2f_4hx2z6)$xMEI&_BBA^6h`NnN~l8kNv#Uv6(+OR_AEl zw67nXw}k7RJ3BKbrY?q|!{STjkzk(KCoihc80ROuotH4rFG#x|Q~jj&%Tpcu*iF+9 z7koX|FZ(cfcDu>6=$ZCE6n;H?cf+sp?b)rzDsJ7lG+n#ul$?En+L!Y-7lYsX>dS09 z^{Lrdw(jYzn4>EXAC}4cXO}ed-=B@DXC)<^`!;`7oL}-ospjy_@?BPL^XKnay14rR zw{+kCM)&{63VL^LsuXayyqf0s#azQr@7eO2N4b0gH~wrkl;2Z$y!%M4cKF^Kd(NAx z%(q*!W%rs2;kPx9+E*KK-j?RCik9QM_xbp(3*R>!T`%#{#D43`??<}NDvJl7J8w1L z{>8^RAD>z6 z?Y?YzTsptm|8d;Er&;q)_HMj-&*Jn}yT^M!Cmried7YcHrz>sA@vnRL zPx^s%@H_i=JT*0JZ*HHP02-$TEh#(SRBXTfzs2`k)89BBnf-Tp&Gj3=NZ_QV(4 zKmVNjf8LhQJ3c$_d@p7B{x+-rAN7e{{h$9;R$3qDZ`8bh#O9{O@wk|uKehH{-+BIU z`u9f%ozLC>8hf~J2LBt|C6$|}{#^W#YkS3w_kVUB3#|YA_Vm7Nx59^)li9D&ZnoYRoE=4)aX0u zTpsyR(!F}_oq6|7Z}s|@r&cPb$33}Nk@322ozC@oSexx* z{+!AGSD?QBW2|!i+oiw%S{hFJ*5&#>hbu0+5dk&|7?EX{}uZm-Tke}2PH1BU0v{g$&9ke3tTJY=BPDW z-Ts+;U-#BsmgQ%cB{zRkp3!@)WwE#ap^n~$cwr&2{ikYPea@Sd^}P7?;f?aw|7=z= z_&&ep!DFFc4_gnmmz6#><4g4qvX9*)bCYMk!IteDA6i^<%wByt(i{CX=P285`Po}G zzn>&f{#G`oMzHqb3hm$Vtipe-+nTRSWGLU0t2^@I=EnU$cBdSCeEN2Cb#b#@;Pvo- ztDev9E0TEibcIDirt+7n`7?68a<**lylfFMTQB10wfXbRB#s@dS;&2w@%;8p`@S&# zlI}aU{AbCrFq_yzk9GfV_@*G&_w1#>-HppPvtRhI{IGbO+@@Wh?^(2IA3W*kwVi)q z@6%1a>~{_P=j}E-E*_}oCh_ItdHcC{?|hV8_jISkm&c1_#mj0=WX@r?%&kCi3cn8@qNi`Pd!uC zqjsz~X1h(5Y|W#?;%WcBu?Jhuw#)7OBVcoCztw!}p;c>ie2&zUw4)YQIytN(ES>$<6%MP zitm;`OrAeES#dhp(#-GGr*xTD;onazEc}*do1Ol@xrXoGTl*i}-}xWp=O~pw1#dLX z-}rsLeZ&7R>;HD&jkYVXpMUmw;j^37R^qo7v%M)?|ME`R^_MZ(|8Lc1f3tti`SQ=J zISs3OHru_oC?^xOAUb#D2uju~4X9{02BC|>(lO04o^ zonQIWN1y)QIDb^_{10j4N`d9u-Q&}1A1_Mj`HJ1hNn{m(1oDj|bNpKJ50-?QJytZZfap^7a32K9|$xZLxb{^8Jndqf_5D z=JU>fw3dIJ(fnU~tPgL|{@nezmM<>-qjkg?Rv{_PK^YwFS|J@rh{kI6_ z|J(oZZ2mfX_UxGQ%aZmk(y?}hA@?oTmDyaK{NZSO|Gr0o=I-Zu=iklLPpf(w8Ee1J z`rYI6j0@)W$-K}Hzr`W1S5h&h`g6mQWaFjoQ^M7*)vS6y=bBsop92DmH$J+4>%jCm zg(rmX*IayG`9m(6$^G^*k9xU9y#3Ekf0lT}FtagPqU z&3uvk{$k(st#g)4=X-NZux{h>{n<$d9@F(LU*+#G-~Tnc{q~HxkJrraV3x0a)?Q@y zm|K`{-K{g`zmA30d@1LjY2w=LhVoojplVM}F)xnEA! z<#oBwx!%n?cr5$-rHvos^{%+d+u0YK5wHCY8tlH`UHswwzlZxP|1;zX-YZIfAZ(OS zUv4+gzM=ki_`jd?Z1wM+e`)>My8G%})f>O*VVZT=a$Z$ee3A+J!jH-<|RDvooRn* z@0Q*E&vpJAZ29c?VT~;NwtGd#`uCRp_2ap<>7ak!lL+hC{eRa$kjVDO^9XEyJuv%F`0=fRJ-!uw4(BQ9j`NjO;0e=YvqQKuUd&whL{@m7M)x1*f# zxgS1g$RG3i`}#xd=KBJNCqC$!y`xZndyzid-`VG1ihuqYYb@Js=;i-@Qw zJYU;$?@{beo&BG7&6&3)%>DN7ec2B_bWbiSe|=56x@osrer|R6)@AB)N`7yX)UQq5 z@#L^f;QzS#>krz*5AyQ!`~5mHOG0^4-`)jp&1*+1%fj!yY38Nczu z!O+=1)6VQWci;4r&UB08_wRh1{<+}&Za3*nxtZ3VwC+u>xt>#(7H3xW%HV0x%^f$V zuhF0LuK0r;`|Z{BkJJC(NdLn8+!P1Z!V@k-hRRF z;`PL`ZP5ldv)cW0)`^ALz0CN0;pVTck)Pfl_>>oXta(HJW{bDwpCxJw4>q0L*E>=7 z%$r;_vD$sx>tAii^(Z--o~*dx(c*i@R@L)&eY~Ei7=OC@!w=<$zs}E^V%uH*^H4MI zJH57hddHp``+ayJXl15VIDzj@&8DaA7v4SyH5dD{e6{YspDE5S1i$|JF;zJxVae=y zR(ALJ+*5a1>mByCVQH&*7|Hi!+JnVs*3Bqve3_ebQ{Zn+OaJms?rWCI?X&j(ASqMI z&-X~`>Ctw_^GjRy*41o@^k?Dkx3gz??^u15t-UVh#SzQAqfD=RCiBMFH*K=i7kC=I z`}mHmY@38*!fpGmO8@^AR$uWl_At_Cw9?%FqslV{=|l33NA>={ za*>w*#JJdSg}v;JTQ(bRYU-bVSh()~uZNd3pP$k@ah9>N<@25Ph=@5J_{#U`*xgW-v8$6d0X?}pFH)~86;eO9$)j_^2es)5C6;$l+U_X ze@F7i5}q6FZ>B%ze_;Rh^#3;-=a>KAdVC`P-Scm&zn#5)UU{$8nR9#*pLFzVs&?m>#QU7D zbm_m48FMx9lY4uKo#y4K|Bu87`tQ2+QSr@-YL8;Kz5D!su4_~jXNgbF_&1Gtr|J)l z^1STBto->BIUIA(zPxh2y8qw1;x9iQO|swe&1y4<+#>){3K&-|BI_wU^Om(9U)FE(8l zud{i4{(g4-^$))v=m{$SImu`zSF`1)X#e_0yV=%lJigL2itod%vYm6jDj$C;{r{8r z|9d4*-UkU-`TZqDtz^B^*`;H!F)QlZ{5Eg$Lg>DpLjU%PK5o} z-EJ|p<{x_X?-!o^uiW?2d)Ddm`?DXV?NN8MRzLl4=f1Oq-|G*)SNG=V#0$K; zkFJ(gv!2s&)ce+w$J6+Daubg_+Mm4tia*R|`gxf%_skNS%rlb%)1Ghqu=RG&j{~c_ z-OMVE9F3lL^G$KC)7Q$h80>mS<_Yj8y2yll>okKL>HbRT3} zHD7dC#%=bU4p(XUB|p`}e9E^Q8;$GY z-*3G*uxhX6;p@NlCLM@fx89(`^a(Zpip}L*Dd_GR{;k!ERzTJw|Dh#|X7neJG z^~J^B>~S|f{P5XwS^XU6@ATs1doJ8xT>CT5=7;j1&lCSAE$wbDuLP|mi7}V!bxq#! zbMBs5i+g+bo|iC{e^>k=^X}~8XWMsv-B$eJpL#%9B`p75e{1`k|H1#a|G(A0nfGPo z@1G5C;;zrUy6?u!C3*g9VinGbJlb>X(+h#$O4h&b)n2#IzdPBZdex@l?cdI6{`RcB z_v>rrqfJ)q`BfiZ?-R-U{j>S`jh)^PpRaScW6N>pvrK}w`|s!fVv}>e{WB}vcdWd( z()me7vGvn)KDnKR6P@^?xPe>=rZZ|I9@L@%7pW({bb6+QZwf=Q?@$YpHSKH+uQoCLJ z*5rIy9FLi--Sz)d*(Hu{_$f6vu;PvGug7P;pX=_7w4Ss6b@^F=nl&pQX>Tu5Tjo-| zv*ur{^7~I4EoYZMW(?MQ+AMMW(_;U-w`B~U&(i(_nu>nD_RdLOL;uc=8{UY*&; z+8>(?j@^6K*I%};Z1T<|@ypwM&m>Oe&d)WonjJY?*r!SQX7}`(xR_t@?+RLb=Ns+W zaniAUUs=!`*>8o{?fN&q6!5IR@ujnI zL+G}TC-|cayE^x7^r(M(e7~i`cA3wPFRD*DK3*$lZtt|>@FMMPHM-Zk^XELiF7t5t z|0~Cj795b1Ub*Me_b2?<|5SetoPXm*mimtqtaFnVc|QuSp3!^tl}y>siPlW*`gU6F zOZUgyO|45Q&@AjYmzt;ax(Y2WCa`E-m zGxDdq$@e`x1R5!Mx^?TRzPOs|!+C!lzhBVD09~?V@ovqGvn(2 zE|bhQe%$-_@$UW(Rr~r5<$1SWRGw-r|LoWIL8andYyGJ|8!xusHa~72+h6uKDRJ}9 zskv50SfKkHoBbBnlVF6H{k4e2#U9rEA1EVh=dzIQ2m&Xaj3CKiX+ z<{q22euLeU-EC{Fv+hfs>i>QB&h~`4_f6B^eGH86JIsD9nMW^g{omN&oJZoTf6MKE z`epT>buXg@*ptqhY`bSsvOh#m{_N9|n^K>PRKL5-_5I9f7yl+bjTV=jZSEps?Ed7bj0{OW&sN&(iEsRr~KQ0%hX6oe_nTIhH&p>n|k~62jhRPwEy(;kFZAl zyZL{v#`dVK@61}b#X{wV(i?Z-&bzE-vbSD+`8xZ^8=DO_Vs71WwQesm72~*9ao%mp z-gwaXB=3sjjPLi>unDV|KFTdVxqbiNoA>uL++WrrZ)-QjcZMF{pVznLuRXi#>K=V( zihuo|TMs%9_hx9PmHpY0sdZd5?8d{*;>;J>q^Cu^+$7p>uCpk9{++7L>CwA;3wXBq z+Ru9AsH7s>00t{bvP_^A=N@AGIon zEz13DXxP>7AF0*w)U^^np<19#;G^F)#m$a`ypX_+tD1Er;UAfWjijI zzPr`AufpW`pMIkyU(a5X*wosoT>bQ_bbY~@$2A$}>p!mh^ygswqB-%YhR@si?*IIF z@2B0{!eyt;_qUwe=KFiXoo^L$erDaBvfOl@{g39iyzRc?k#+@V!z)eQ-l~5SKeU{k zJ&AA1>^ECJUKaj-Y+2dktn-iEZ%bD{-tw@sIGG{+>$>e9!`>fwqp)_L;i>B#M!${E zZI!L5{j|*Q$l>Mg@s$Uc+vW57*dI|(5)XdNA?#n=ZO0Sj_b4D%etym26Kwg9uU2fx z{#*I_`<8W&wpBm;k;{9qy>^{wv1|RujlBH&Hmm$6y(=y_c3i^L?^ETuxkZ0KLuYgC zznH(1KfwD1vI^x}X!`k|%sk>sy&KM3wf8^SttLPF*XjA&-UZfP&k&9;{P=i} z$|>pp(ceBFm6xql|MYRv-#Hf5;a-D^|#A=ZB$BZi?$`(Z#*%*^6^{64<6~0 zzU|AtAz0p?RCltx*HFbm;KrZMz`7qg{nE2H?s=3vxvuK#&ebO-{(Ce<{@<1B*IVoF zwa(K1m00;j*l^PNU%TD>3U%x@W*%Nz@kO-#-#ppoi^ku+9KBOJw{P3gc{g4jy6V>z zdROMdyWPhs4kZ3oyl?s`=g+MCoa_G$|NqFjGxe$E{!{z1Z){er>7HNF-Fu)R_RU&t ztKQ{xZ*NVh{<1yq)a9mg@_!ya^v$o0`5aWU!>s=F>)-pbZwP*e47>&Lf!=MT!iDgJQ1{#ie0KgMtSgC>TZCo>lpyok2g|M&gByW4(l zzr8JAZBt$Mr;Qty_Q)=*q*^fP+H%Pk#Ob8thsfA{le=Krm3h}`q} z19Nj7e`;!#S(8}&+uVRRHzw;Xe$M|;cF7}UgP-5+avwfip3X>Hl}Ta&H*>!gb%Ze4x(IkQf4x5NXTPCe_x=1NP*(F`vA>-(^V^Rn{XeICJU)Hizb)m3(tOZ+)>!TpByfyayQ@n4v29-;i^=?UlX+Q;+F^R9InhCkcvEmr$msJ^7b zF3IOswfVM#yy?%sK3H)k58ij;$WmSF+GEjoD<*H1;%x}OX6**830)BgY4^k0X*O*{T~!j7cY>%o6+iY_lSY?qS$oANqb ze*SmMil-?>+UB3M?z`|k_4Vt`{(9|ME#JQ1po!}HnkA4$;1A9ey3Mn1_-%i%L~mmM z&GWnCH*UZFcBbL)*_$)JTU9@_Ik&asP4_R4?+G$L)Z>qq^Uk~b{?_FCUt^Q!+r3}7 zqpt6F<@fGKAIsiVs4dvLdkS~%F;&6!cKPLMo4&VXuRDczxx(wJJa%Kn`iv^B$j<&@OjN9 zvw~{S8t!l#)xUf1o7T)Pzw%70`myr-v%e?vXO#Z^)aTFt@6FZc{l{%h@8!I-`M)mm zTn^jzuR8b7@6Uel=t@jO`}gB@*Ed)v>*&pRCRqPE`;}=#7>rx?z)c z_}|a+-u#SxX8FyVjq*z^n;F+{ocMX4ds$7}_3WDK5Bz>)%B?%UQu?^!Mw=INirwbg z#cmWTyrfxvY+L=lllyjB<*1oe>Hl5#@O)M4|I#1pZvM2JbE5C+k<@#2FUosuB=VDd z%%@*2o8RpJUuW}Q|F|dDF4q|UJGI1g(VDeZbHC+QhyJ?Nx&6-m*WYc6cbPV_|Bk(G zZ1*_jI$NGg^^R8-N2J*o)qn5RJe>B~@cD!EugpK@|6;E%Y|eM`=9$=Y|L?hSGfy_h zqVi@_jq4553wE7aZ7j{jQ}IQn&BfO6#-!D~dEd;u|F<*T`tw}5uyJyqSt&C=JDX1P zj@owle+91;dX%M^vztF1@NV~=+~xoDsbSsY&68Q)A6$3V?ePM|H#WJuKC!Dm?BB3d zyiMZ8Bk|>Yj(3jUOpMuctFzo9*6T;h`uBqDPd{4A-d_HH<;&e4rv~p!*ko$P*POIw z{n@yGV$U}P>WS^~j45o-RK6z8{`1h>^F_rmmi(H z`QQO}aJ=CGJ+^Nu_g??LWbStMVv$RI^4@#B&zoZE4}5ykTz6?l$x5Ncb+6~t8D6}; zE^n>i$Itt%g!y@_9%ac**QU+cEvCm^S*4q7sksI52;&RoTwks+q}t?#aVuD#o@qf)Asq@pX#TOMOV-TYe|*Oad7VZ4YPR{CzI_%ew%hW0|M`QzWlZePaKC=I z_1wYV|5+|`?S6Xh!1B}sGZSOyK_|OdYwy>sJO6iuwOe{vdi&n^o0n%EICk^Py7#lr zeLp9*ZAY!>ebaMc=}|I;#ryWX-o5Pif*nU5?>Vx0Gyl0y^S0YL`NwfspZ_)YSLMIg z)5U|{JS=0IzFT(v&dCR!7Qc^p%)TvV<7K}`OYW5ST6FEU`0_kle$A~e*84RVQ*YPF zmFNE2wOBm+t--?%R*$Zw-R%4~?wziz5!!R&m)zfD`L~an z=CT|Qyj8yIcHy6cZgO&VZuXgTW>41tyDrnOE8UXUY+e72Q}>T5nLUL`+TUwT%fHy( z`g)dma=qapWB$*^tw)bH>fCFcxwh=zjYHS@l6Ut0ZLhh$X8*-h=lydYEj|BP@xH0^ z1I@mN+I~Ba*xoCc^0o4VsJZ=9)%w}p+4mQ0@2`n{d}6xU&aZ6;<^Mfp5=&g{v)(M# zNqqUMU%%H~Jbq0g%l`AB>)U+o?P9tf98!C_r%kivvuVD<|MpVJNo{r zpJd%et@Zma7CMwRuAYDX*TdXOi;mU2U(Yb!D{@}X8$ahm!Ajoax28S6Vm@zqhMcsF zO~mPRgM?Dgd3|p$`#C-es=40q_~jTwZ(S)`(hieK5dG(NI4#0_pCAXtx12E ztlD3T3&Hs|-ww$7$@aYZb4oM*l)=M?7g`H%FN=*fv$)vIn_nAi@oUR+?){r)oL>Hu zyE*=Eg}U_ZAJs>Ut%?NNpIPqzGhck|Be|zXTesS{ZbEz`vrQB~|14>T zN&e@JUd@&Z^3QPEg{9|wF=c-<|A>unvgKm`u7kh9$7iX%IT!hP+s{Nh)BG(jv*+%6 zBJ*c={g3>6|5;9@eTANqa5G`OT|LA1{D#Tf-#_1Pf8W%8*YgR+SzqUrU!VCnruds> z_NMUV*=w!Of6tk-skoPqueyBZuX)RvU->Lte)sp!^4`5i>m03w=ht5^X+O|;;9>rG z_ka59-a5*EioG1U-K=@~`wbtTmw#PXEO`C%o-<*7yB5#gd11fAhGlzxKYsFT&+{CI z%j(mv|5|r&^X`M6Bq-1MG4b28Q;*Kd z>;GKWc=d1g@q{<$E3@oo=Ij4m*I0N?{O3)Hw(WaD#mYbT_STpkJK8+mus8mf!Go;# z=hDt>&Hj9B{)V3u-p@)hZ!@^|e9!(@@1-ma^DIs`R+#V0ju&9|`*HH7!0j8SLVsS{ zUVS+A`?~kD?&hnNd;WW}aqH1|nOD>K?LI}%`(a=D@IvkV^0N|e?i^kibGZ5a+2!UB z`R(rAdtT0a|JtdY_srdsm;2it7MZ=1q1iv48s*>7&~ z?>qHgYR@g}bMK|zy1WC87A^^A|89|g_|HN;{oUKcglhEq`oz zAL)MfDZeEWB$WdyW)x~%zzrvbP32DW_;e-9*HytyF9=XpzIT-@G^nG?6Yyxe@d z?aMaax&Ob~?#o&6_s8VfC&HB>(EBUS$bKU>wA3yhf#l`rz=5FUh=6lXQ{UX|3BW3<} z@rMtd)A@_vCmJX8M#h}kX}iWP{Ps)rxt~kB$=>kC>c%PU z|23w|3bxuF%YF6mllQ92rs;n?;^d}Zzi~UiYvG(J%727@{SeNtop($xwyMxB_v_=y zZ&xn&-(C0i-ovA+|DRn*>OBiu_dWlUj{OoDZRzu$*2MX}FaB`){K4P(4YLhFJG!>; zypgnheht(W|MP2q&8_um=ijd>cwTX`ZeMxsm35o%>|;Ou^Ty1K{huwO=TyJrcr{mb zu4|5Uykpk==T@Jm++BUC{p#OCa;xX&r+#lb7aVcw)=h!*JypNA{Mqnyzr>5Z`S#^( zlip>%eO~jJ^ZC~O*=FwIW_eEwn(B9LK0Et=(f&Ve>lZ8LkkI(-9QvFeu z-@ysH`G2mPm>%)zqE%z~yu=3!_kYtpBmYNQIEhcV>`%4X@x#r3>w=2)wrkIbCzl)$DZlqwlhw_SGO;smW@WG2czs>`@0Zqk zx7=Sg)`E^b*fBZ#Sx@E1TcAd``GLRE4<;FCr;GO=iMaFcGY9txd)K7Etqi&)+60d_tb1<(xvN;Z2V!;6Ps6><%=&nd-2DS!00_wJY)7`zcwm;wsywL5AL75 z|8IFOP$Sb-RQ@zSdA7uh1N_c<*|#OyKR4vfwvxAKiM?a+t^TRZ+?pxZ3--JeOufYa zL67<9%~lRG+kB<8f4fxxOv~RiRJCNmfzdhyXUvWPm+r( z=nl19xcYv~6Msqmr$_fan)KTEU6HeR{2_Vy4_{v&o++y{!)50DSEu9-oqL@BY;$k< zoay%qo~{0R*+xUYB=K})?wbc$x4Dm0?fxgBV|XLtY`r|k>B3t_1l-;hEjxR7`o>Gk z=I~YT`*0#I`r}RRf=lk|_79W9j@_CrU--~u-;s;%UjNrwN^klZBKJph-|=lSIeG2o zE7$bb`ro-Co?jBXuHdR#T>c~9`5o>1)Nhw$XP;|ZZJ&3{>D`u-x$=b{ettjPSoHtf z!ktOE+oqfTf4K4b_4^hF)>kqnzja$)x~hN8+aK5U*$f{$@~3Zmy1HoNh4aUc@qd2A z7ad|$^JjKiQN5rPrD}f5{yPGSXpYK^XN9b|#y3-5i zRGRD0w*Gq3-1>3KoR{YBp3c@g=~nr@En9DSe$21+fcd;N!xDgNtgqwmgH zaQRQl?SIRA&z;lV7+Y#rz%iRQ=9j)Dd;LrGx{vO4Z;sdH{1^LS{(RooqvG*zVp?PS z%y#_lw`MM9yEl{Zch~M?Kbd#bGymDhUvs1`Z+}Df`wiQEue&(!#^YPcmo|srFrM_y z?Cah9wDl{_-oAOK?udW&?RlQb%=@b&Yu-dMZ`$=|W8+etw4m9aUVhj4{Je4Yz3RJ=OZT{(>?`}?xHhb1rQ+uZF-p{~)dyXdA%&D9!`DNGk#}+Ir z^3Kdvzw6uM{`kbYGE2i_$2?|RJb1kB@0 z`t?bLR!dC3fp&4c-1t#~KRI>H)bsD99#`C({^7vE$Mcf;9^}lq&3wo;zmlJA?$x7@wi-E#M|t%rh7PTQuo~Wvf*fMWsGHY@B3MdHB$UH#Mc>S*F0HU*&kkB zRrBL}{N)oTKfjlX*(%M=)W5g1y6*ac$!TGt8g=*m3^g z`+xs7B!4{x+9g^zDf`jh&soo2CCh9*AM@n?50ih_j`e*%)1GS_|D5yj&+^`#tisd2 zevNwmApK*01NSw4*?+tRCpT%RoLZzDYonv(`RR=O>ec%hgUk{pX-+hgyCHN-Z9z^! zg2aT9s6Prn4mb1Z{rjxPt)Ex&U{kD~lv_=KRD_vcm%;w8YxCEx`~TVKtuN2y$9Jxj z1o%bcUZD(Om-p|{tCU1{5p4cuw{l>Mh zH@%j7CkOAGFuQ2OAAP@~mg9ZLe|)Mdm|6DilX!jMDOZ`(>BRxozl+4>E$d%w!E}F~88O`}na$w#h|9@rceABBN?fSPK7r*ITUy&iqK9PG)`eFHPW~KKkzWjro1(3R^=Gs?t^A|53&psh}vQ2LOj!V44Yz`KRP`Ffidy5G3)UbAh-1w*zk zYxwtY%)6Fbc{TsthwXdTch9zX%G6feC|`9t_rsUz$64pqNj&4#+mIl~-L6}lwKb>qZ;LH%&YZL5 zaw@-j?7gu3r0eX5s}1#I|CY!rv0uBLuy-K!F*+sAzCf9uzNKD8?==1Ela&i{3Dn*TDqbN}#}z2Nh$=I^@YZ0~sX z*fW1C`ab_N`-gvQKc>HZ{;mIRc3QpK^SXV%kG*lcw#ay;_0!LPWsdE9zJ2+D-#7l= z`T6wC(fc}U&3Uz;f_Py%xRPkc3e=j7|t7dl}3|PO`GNMi@xaP&- zCeHPR7v0yF+icP7&J_8^t30v!n8Rii9HpJI#8X>NPe$GzvNam)vm z*U7ia_xAb6T)xhKxMoXzNd4OPv!0fHeBAy-TL15@tsn3I2{%vgUvV|-F0&_ z?|&4&-C?*Sy>`ZAmvv8%)-HT}Vn6*g zNSU~O!^=ny1D~Sii;I%hpLe=(w2XJw|D4u)#rtImoZzl#_Ek&c~Zc(%&3;& z$UFIG>=!DU&G~$A+MLPid>@x+>zDipxxYi-x|BQUP5OMJMCbW^oLQ5zA798?lR=KSttE2Fy~7i;G=wcmJpOZvZF&7Vt) z_dm9(ZILfD<@@z=(wF6lZ?6U;$KX$`(U3Sgi!1%hC57f<=E;FfbecbG9f2=c+ zf5yePi)Q?lIwxQ7@J0Tdgu@q>_ZZ7=T=jSguk3&CHQ(I-T;4YO*oV~Diod#}W&7IK z8phZQH=9*$nCfXWW%lw@U%E8&ZXLete`;!x&9RENn=hX`XDus}aN=o_&&+*hRz+;- zg&*hb(~FFaGC!An=fzBJgHId(JWac`?f%;e)Pd>2P^tk+*YaKD&%F>0G zG~fR!yqLf4;#T$5{1^H>ix<>HJm-vG!p$T5^TIj4bI+b1(lY;b^MvC5V>k9KxclQ` zvua;#{o-tyt+{`ty#JnMuKlFho-}3i|M`1AH}tQ1@$a8!{+0lyh2<-QL;`;#J0AUa z!&Oj3l>6JK6$@5GFno2~sI8@>ko119xx-PWOWj}YF)MN|d1Jj_G5vn7r`gH8O;62h z^uFJlX`Ftp=J~t3)#v@ry{KhP-RAqkeAh*_Vv z_!+p^8S9COA9EnFSws?^wsbD|Y?5>)mnPJNzc}7w`Rk*mYv&36{wmptK3`n4Ld z!ujh@eV3{gKWNLy`TW;j_UI3PS#E!o=aODX;#{SCj83U-Gineg12m z{vF^X44= zEV6HQfo!7lrEKrxOn)yIUPuqvvc6^k`%{l~e~)hEkM+3FYUQ!5{^+Mw{J*EO_nw6MsYnt8wZ;nVwvO_y+9pZdAVVCL1Ywm**Y9&g*mEHmLl;<2Om{%`$! zC+-8Iu);^iW9!*pi5%n=;rI0Tap0x6))$3^eWeyNjcPU&1aC8MeZ+72CT(ozgBl|4}+r1fA z*se6{?s^z=`hwh~Y}q-B<^5(p+BUIYn)B8HUXG_X3i3*48~r_WaA(|_8_JGi$EzMQ zOI!Q3iNAWuD|38;Ym=Nz@}3C=JijNd`S+->LFQ8_PonUz1&^1=Y}GrJ>twr_`RA+H*Rb@U%hQeK>S0@-t+Y#76)c&0q0{cgbncv^mZxI+3;XsZ+_@T- zJ$vQ)H3!&_Us#&FytYYiOY62kKFgTzH$EI-jGf?k$naL+?TyDJ9L0~xROd{q{TO$1 zf!&jhijHj3>&jmo7q=I>zxa3E~dd2W% zRlUv35-+y4{!=*rC;hv{wQQlPkCI{UG@9kET&UoXFyDuO+7;Y$ZxpOeSd%dOYUZnE>sOCtW^Ze^iP85+SaMzBi?V#> zw!Td1zfS+Jh)r3@d$yVH#ntu%S?OHvZM(Mh*VJrV$iJ(Aul7dvrcxhEpTIt7k zM~LZ6m?D4c;r9mzvrjb{{?`n@`+Bd)nllG1_V*tYz4OzyVuIv4YXe5}4Tt{ij7xsA z#_!C(wqow@88zPu&c5Ed=FU!aMxhVB4Br)MmhZMYAjrgepV?#T&iRk$wY@K?k^A_! z;rzzCvx`->tbT7=QD-8bQ!QWecizVHF3;jR-pf|!ylM5Vh%2l;cc5JE-OLI#hiZ>+ zafy9}>(0-e;JkYk|97F9`diz4W%nLm!1rqRz1dsOXz5QoYWQj5o6nJj@>_DBOfRu# zm;b`LW~=R-#}PYkn_Y}A&UdzZm%l-N&zIfuxjXCfZ*S|BRo#2tf&Gah``lkPY;`;2 z0{RL+)vVt*=YsqdW1|(_Rvs2%cIWIwie2|@YfOI8-g7Fs|E$`Z-n!avPH~>^O)oH8 zhfH4`_rf{;P4SG08?Sd=?6%X;{@`d|xGuT9&f(ZA)9TU|2fpj`_gz2Wc0!@CJud!~ zTrx|nRbJTo(>I!Cf4jN${|oMQ;#b`33-)G7-&ieodhZP3Pg~z-$a8*kVBaU4xcu)w zlWz()KN=tZx956>{lfV5yj#}(4T+VAmpL5wh5he}Gu6Crt{QG|6aUY?HzDEl8N)Zh z(FS(26q^72{vFr+yD>HY%fj`3f87*w9EO2OR+ax2Of@krmg za9(_!`Hgj{7n-Z8{F`H|d~Y4I`L}sbFP}9ej2 zc3oKZ*GR9=k=u5{_1O)3H*~#z$=X`9cf}j_ds`Z9^R{_xkp8ChZ0cL>yo6=Gxza{E z1QZvSeoIY{j;!8n_Gw2);bxcG8b+msu{XMRee*u_W+J<-$G5iRoQty+pKQ2kw3z+) zhRZS*w_PQ39PB?j2FRYcZ1>2mb=L;ow~K5Gnt1vOgrmHFGlW1O-cFWO%@OTJpL?$(Pl z#%~w<&kA6FyYtra+G~Xkz899~uGA^XnD;99?ziq;Kdg>KG%tNS{UY;hh4i5R;6zO*=lbnJIX!Z6)u*+ottD+ zn7nn{ESp^R)GX^k=(B*YtmDuuzpcm_&!Vi+qc9!Zwh@^$6o0Dr(yhpS=#s8ot~X)cRm-qcfC~i zNTMp{_oB)ROLxV7&HAm?zUxA7%JEw_`p(Z*tGT#{_eS#eq{=JJdQTVc3E)rN;Wlvr z_t(Nc+nhk#vNv7pRZqNne&JT{>(voq$0h!(>jk9E#Q=@}BSf8?HgUl2F{#$)5|*SyyS z?2K=OKdCMVy8l1&_;#!Ph5cLf^N-FK-FuHeZqA?iEFK5M8SgCq_~+qEhU2yI@1OrT zbL;&Mfj#qI8#BC9-m(9MEo<$i_ZD9SQ`fyW{dITscg{D8i%Y-kWwT$m*zVKESav;d zS+!4oA-m`M(yYa~mmgP4d{dIY|8v%bX?DR5;=kI8-Y&NLzH8t02fw4{3x2KS+IDN# zvNI-0Z#R}H%2%o_jQRR#C)4+|TzRX7hi}eLs3 z`Rl!GMK8>MEj*L3*zWxyv7A{8ymQ^>=^8)%Wy5?sLHLWKnCI`N6ZAGg`Vp< zr#Olgd7LPh4eyzKi&-*t{e06WjohD1&ue_HS|7M~1LLm+;-2>E8ppRjd%c$}b%UU^ z&@C27a#=R%UiKE=6ASr--n8jmeY@)VWbQZG=c{|a1xsqnKW+UM`0RXj?@os|ri+qS z%y*8NJ#FPn=3h&mR-HeP&Ul^c*UG0iURU$pYSY`kZ`L-2=3O4Y7SF8X`xxn9ov`$J zb85z?LO#nc-uJUFhzY$)xRX2gkzwn)fa>)JOBdBPtd%goEnmkkx98fK8@JALt}{RE zeq8L#gacgCH(oy9ZMBcxxr)ugjz?_5Plm_O?&W`)d2Y6voZE+k2gKD5JZwwUcVd=! zz$nfsANAruduXFGpM1s%&61jii4&H;eI!w;xVmT+_pc3#i42AtCR~&jJ-3j3_N~RW z8{?1daOl}{uSH>f(Te7Mj`BIj1H%bHwyg{@_t-x9yU_;;2wIww*0hlCGR?O|Ckde+VxIo$)`&n{hB!co`OQV-Ju3K`S5{=gxo>*k>;ivaZ zX4ltU2P|&bNk-m|zs&Y+!bNH2l!*(Zc?u?9-xeqy&yc~c&mlXhF>~9a+ zEk1CIJ@U5wVrkA_3#Dh@>e;8bFgAw$0`uUAg*Ck^il;Tcncz+f9$N zPuY|@u)ST>ml}SDe{HwTmZjI(kBR3x$?+J*Wpx`~i))VR%f2#odA9W_(bBE^zwNle z_rjoSZQkEK2PQ@aB+Pl|`D}VcF>izEn*)Xx3;XP363$#H`*QRq-?4o&CO>-_u}Jm{ z|IZ7p{~Yc6F7j_-l$@gcFh=~#1-lzHm)X87*uSFN=mn#*%9V}3dSx{ZH%q>8wLBNd z|5h>nxUjyL$FHTW$v56+O|JVFnD*8rcd?A;TLVcm{!JEUb2mP6VYPJj+eyCUbM=+xKM5M%9-!m&bY}FTA2k4gEpUi9{b-$=+gw}+P!53XC37{ za}V9S)wgd}(TPvhA0CujS!n9ZH1Dlj_jYF@+p$wY$B!H7{yVdf@6$#_`I?2y$9Kdn z`dD4?YNP*+4eC4cuKw&jzw*=O`Hl=}N{m7p6B*vA{_r~#ulv^Q^}jbaHaf5SSFm*N z@dukjF8!6cue;;B0~_c4`ft{|e#Es_%k48S$ZwugcmKs_qgRjrv0BR9-a9eW@@nh( zjdjz${uVTv_ukZP=4H7ThyDth9klzq?#i|+dDZ=)uf`QR9zAU|)mLh^4G zTeX#UGlV}?usLo@PXF{~hE(K~mllrZPhy2nO=SOjBY|1y)0?z&3x$6%eT-|idm-`Y zCvR15eRt)e+6@i56M~~Z#B|y@iE%#t_2=!+xMs5n3+}%%%uR?nAoBGfqv!9ATYt$XFTkSoYT4c^m?BA9+c{$&S1^O!%U$=fTF+?o) z=C2p4FE;*jlH>Wh*KNN3l=ixLF_pWvEsj|z{dUoOP%7QhSp0eZ=g&nS_C9;y^m@m6 zPzv1`_@G43S^QmKmA^y$0`BK!Z@MirZ(B|C%X1TZySVh z(6%kX*z88@-cyeya+_9e<88~3-u5ucVc+ZO-nH2g3wb!7PY6F}5iYqU{Ni`18%w?= z_VfOAzI5L57OVLe;gwsyOD#Td>Y&*b$Jm0_yAsmpze}B1EL(8lyu><7S?;ZzMH<^{ zbsz3!Tl2-3#d_*=#;1G2F7)j!JFw@5-QN9+_PyP|QS|r*#%(KQelf)Qeam1r6Mg;Y zdqPSrf0Ce&&#Qy!+}E4`?s<5KE1=6zeD?3{m6jr}*_p*H&F(zd@X&3Ybe_?V3pe@^bNXO}XeJb>6LMkXR&u-Xv{DJ%V zHEVQk?krwj+LXM%{C%9YvCMnpS27yW%Bmg>EO2DQd0=DjchNoj6L=R}mA>-WTlH1N z);2cl`np`%2kYJ&&-A&`%mh`y7a`9A9}$;j?X4DaE7!U`0()X+^h5J|5ZLWp3ZN1qn@6HRGXN;f#Y_VpX%i3ov$8}Y(FZu3-<;MG8GVp&FT>dxImEpbC zj4jRk6I^9J*j9hDFt^x0vH!dDzVjhhJ2uD)b?7lMv-6dd{WxoWzeMgcI24sS8}^-k z5x?osO|UF)jPyA}>loPw zp^5yr6&OLzI4)a0XY=9v_5b&Np24o-z@#FO$?|2d>){1UmMpmudB317*;e`bkNfN1 z+c!Q4$Yi(Z2{^uO_2TIMtud)gi6*7sNb+PUV7}TC8SMksugMZ4&vfx@)&1q)wP^jW4d3WdL&6};<;#U2Y4ooTz zOcUK0-fI;!JzgLFkfr_b?b6Hpi&}ja1Utw)$eLoc)AqZ@b(^VA=YdjQlOuz9pl?Mc zD6uLudMGxqyPbdX^20aP$C2&NRUDY!v0T~NYnxVe_~F{w|58`)JN@EbEc1C#a4mSm zVq|PQcdB{rttZgfbP-~#^*#r1DdZG&CS5R=pvV1tRV7ArXjn{21JScW8i4}gM%NZidsPktTC-+AqP|9&wA{0>H zYqj&?hp^?_--W7O;CaEoZxZ-QnGqC?3NVtxn(5VpZzWaPbvH^o*d{~rCj;jMhYrC2 z_U*6Z`V&{ZE>!Sc&TKQmWMwzV`yh{i(+)aAA%)`tqnwemqcge;s&XdJ2EGptmzL0830b zPnm^GV}PeDG)Tc|0K^s$VXCR;2jW{Pb2ylF0G~x(YISDn;6f{_V{1Xyc6XFF* zb$Ki=7|IVH6c$owEKondexg9$@s>YG0^}61n=U#xyw|^IeC>@U=LCnZf(w}U{JOC@ zRY;*xLEr&%iok*(RZuDcWeG^!IRtQg_^YZBv2qH?-a8x@7~i!utE)ILJ@jkfpYYJG zY0G?&1U$4BwKLq0UzEP)X0VV#<11wc_KL3?(@(>#$I5V2abT|yco<}gYBj_Ga1z*KK zX5zI}p@ZcG1HV$tDm64mqjFc+F_kog%U5sn!E3323sXr0yP#euBAa1)fPphap3&;S z>VDg3{FZVkF{fn$sJRrHVoCqlU3 zRrdzI53iEXPs49%lSc#J2ZcjRvtZc-HN2o1nT+J-kRlYZuV&|*qEnhh=PaL}Sln+X zHFIvtdNfZTn#XWI_D~(iO-3PwMh1Ze%r*=HE1Ot66n1gw#Ow&zRq|44dQ4HL&rBoL zT~_DsAzCKj_-Nr}+_$^sY{10;P&PxPP6Z>*4~j-c$EWU9LiSIihw_2)=^oO7-`ZI` z7AS!d7KiCIMMj~HET)={o>vdkpQD79LJr4=yQ(>DU2T&XI1%A4@RFtO$3ym?kGl1r zY`J3?lGoSHZmT~k8t&08rhDhf*C)H*@7rvq zT*G;2hECKLk9*bcZ5{1?UD%lH{_@`5-FFHO^TwI|`}_UAQSvdKoBL{aFVLTBT|OrXp!#6=WR zRU)EAp@ZeeyF=yE72@JRSySK@iwwBt4)9}YZEfunRCcpC^Y?7tv6*iTx}B_|w&&%Z zPfkjbDZf+Le%NB=T@(B7cZ#?DIcNR8TKBeOPGll3l6M3^ZCiO z==`m@GiF}PbIrcKu5`gZtDD6|MR!i^|Mg1S{csy^uH~FN^E_oA)c*QXc%acX?~a9g zUf#OdH-5k0|9(a6?rn$9Tqv6%`6w^dB634Qm`aHM+KCYcB6^D39Ewa&h#bAHvi;xlILycdqTpoMw`Q%$*M^(Vgju}t-M1C==^ zIJ^*Cz-;s7Sc}+1hZ8~##y5}b-InMawk9HQ)!fg|&Tihk;k^C-8uOj``+gp?sylD} z-e!Bv^~wBpKMtIVseC$h-OawApPxrB|25HF?xlv7R+08+o1Opv{a(x0qFd{m-jTCjhmX;`OAbu*?*jvsC@U6t-nFH?8n^?KYaH| zwRO2y@wj(e?&b;0(nIH=)d3-ljQe)Ww45oQuC?RF4HnJ`4kv^cFxxz7?_itc@Is*C z>(%hJ$=_ElpZDqXip}S&PM^xZzwfT^zkR>oZT{(R|94BYo!_>cn~yv^JPJx)UOKwk z=kfQo@%!(sK4Z+B7$Wfa_lw2-W!gdP7Geia6RWpCi%}G$u<2y%>ScC_@L-(5x$Zw6G?y4RT7rP4bn<}H#fx|^NGE6u__!%uq z)O}~A$nGrQ`90g`g%GG%p8bPUeU8HYy5DPOL_ADNO0zy)z|;QuSg-WbxV=^9YGe*{ z^0fat(8zrElee|u=972lTzSP%|94ai!UY38*ci1%Ft6yepb8)1I%uoT^whknUteB^O`rMn?Uk>`vq4hTK)gD{{Am351qNO?P>9|Gan=0`KOgy9k^uIqO`lUn`$@Av!VPo6%t^yv_G?-tXQQPq{O ztoo9nou5?k`^xiKouT)Ce|h=1U~XdHrv&Q*zIzJYXPf29sM+mUZ5el5jc?cC7N^23 zSyxZ3?vaEy!XS*PpjJooAFtwu&v%y=U6;t7%)r^g4@yYsYce!BTUZz^whHchymtFN zt*i4Sjng75H(yBUEZYD7-|neD&)5G;{_3bd=Z@;qwb9$Hf`Wr(w`N@}nj*^f^v~z> z+jj|O9W`aW&DbZ;=JTegz0>xc0e8YKv$th?s=w#C`}p)6)_L=C+IeBL06QS?V0ZW7 zz*)?5-s?5)Jh!5i#Y5o>sCfI8?85D-u!Q5xEYs6g(=K1WESpy7K7X6yH1k8Vmn>Og z@$B5(*THVahO=$#?CupF5p=&Hc`^Iinu|*Ff8WmEe|H=I&#$kq|DIC$WTN{TpY!u< z=ROP#F%aPU@bvWb=)bC#-}Co=4U5hb+oe-zw)<2}?boYWQr{t=1Sc6)4wQ?xH^0|o z+_PXs7mJ5N4(E{rpA4clPhK{OfwP67u~S(6RIm9x4OUjxqKB>GUibIa9<|t%dU=^| z(xdY3&h6@IJ)JXdUN5@nDn2#qBd9pBXkObhbD`p~m2s>7?5q9FqP=VX$79k-ViULg zs{Z~?w^zzk=yQSk`aPezW_(cJcIF7z^t7|HcD|H0PV1Q=pp_(cP((QR&e}7dm!zGY zb!kUoa^$X(mC+M9uPdcB`WPBte7bac+@v#Sd<-}5e7a(B-=-ps_g7a>pK(*{(#l|U zL%+Pt`}_9JxcNFsZ1TpMeZSvXzr3+=@t-f3{e$+`)tdjcLF+bi@G$P%{o$^vLAcOX zPzE+_+)#S=a)FRSW0q>e2i3c~%gc`(J$m!zxo2)K?#Nd@nb>!8bNcU8`9rf5VmMw+ z7Y#ca`f#$p-AjXWnd!4{**(gNuY5YSPd9qojxOtdW>=G(rHU&i9XO*TG( z?|1%qJWyL~CiQ&h<6p1Wm*?Esad9o5KcYnl?w~OFH@w%YDd+qX>e>tHqq;O0ALw=O z>SXa)@Q9_TsOX4}L2|mMdG4(cAD`^?k=I49$Cdd$elczTzhBm(RDWv$C@NWPz;ndh|WM&$E(S(`U)O1ZembsGN@17+FGhYXwNbfOh+uQ)#3 zed_&ixA)c9{G3h}4}}uYD27$Cqc@sU(YQ>YE)K(?C0eL?0J&oT>U>-Pjbgm$Vbw*8 z4rmt$rkq0oG>XA7c@>&1Xk4du2H6J-jm5UKp!xzFmI$>94WL3w!DkgpKLutN+-eS4 zrjmyCa}gJW@cV%Q)CFoVSlNZ&($RoJ3*kmG0*=W+K~3%0%A*epET99E@DfbLfoY*z z!+ZUOW=dPySv(YkKqYJE;!}1L9YjE5QzHC!yZunw#EBr~iDn5V(^)(ebU@0d9d=vn z*}&qZbb$TDq!R|HLqQ%2S2#ZWRlVZY@@)bGrweF&LAfyGrn*o^5>rh>&!dCs*HKzB z3THsd&#?781C^kNHp~PEPnI7HkwTK7!Eod*MT->UzWpt?IxffwDKvV329Z?yw0^rV z1^P7bPYAU6@x}^eK&eF%r2J;X#TX%l#tk6l5q!}pQf~_^mMk!cH+vhMye)K-a{1yn zXKpK1^S_x^`?FeQy@LCv?oUe=XvA~Lum(NR5V=z55tzc1xZXqI6vq*dH4j)7%+&lI zB|lQNGjkM5nyhMR7kWu3%k%6Jt04283$J)*>deesyQHvpnvl43zUIz-7A}%zWz(^x z`~@ORB@NS=+FcK-GCVg8%KK;cv-)+vQT=V7$i3+s3ya?K=N-P{$D|@4$Rd<9c}eQp zzDrvj<6llJ>{+6+oYq$JCD3D%NQID@ZTq zfjOhqfxm992TuuX*tpC0#AM%_tDk;7`Z%@t+^bV7l4C(7iN;ii#aE^k6)wNMG%$+$ zHOupy4?%?+(b|>ZB1>r+gX{xA>n#f}8(Qq|?&?ap`0f7Fud4m07?1tnn|pAfs*1ot z79rn=Dc#ohH0QpOcT}%n={Mf(hLXx5E?Wv38a1)fSuMgi_iOK8`PuuXmWSD>y7n#d zdYO_6b{2o*jTsS^`|qADidg(u#msD)V&Ey}{!I}55Ykco!LR9E73H8=RnTmQ4X9W? zKShd{v9#{>nfICA$1eRmtKH1vu^@`$0B4=&@!fBI3ak#UPUX0vfs}R~I9XmW=ri(c z2IUucX>8?j;Ofu4h3~a0^X|Q!^!`E8^gVVS4J@nr84fLsx&5~O(vCFkb1dukB_^DP zxCTPXvAkgT&TuFg8;R1jBybW8kOutPa_}yxi@RO*{z8i{;N1JDLO1j z^y1#R?{|f#m&Y<;8%Z5n_xpG6XKY1j!?j5a zoGz@5hhNQCc)n$Kn2?Z=1-Ffs+nl*7XoWDNP)8b5NrU_67wj9}Tm6n1Yk)kr!V3UOsp5zt`zXxLd){at)%SeEQ_or3E#%^S1Ys}MCRIJH8!tLB3SRPEUk z^BL#dzhUsZ`v1SbM%jncKRrDbet2K}8_?v*6CsPGFP~o8{xb7i_ z8M0^wgY1Jqn;(s@cD;V{{?474nY+FyM$X=C5%6kdJ4ii~--6Vrbje#E<2=2*9_f5& zxV+Q}5&V!y0HsrCP2RFrRqFTe1C7j^((|X;YcX3u&1;*y6d0wXcdq!pP zT65dz>G?^psahSIw--xr>4AdldIkHQ?N+~cFIeF4>C>l_quW|P{o2*ebdOIIG+64e zfHUUSue!|G^@~;1)VykH_QVPpU&sxRN6G`>)CNj<4qt^9Fz>1Q*bG=v{Ie%d zPQ3XVf8x9K^7h*OH$bhV6Y2&F9tK!$I_YV~!!_r@490Vj4<_2bd!vXjuSJOQ`g8w1 z@y4LZEe1}Oc?_}-8q=jhr9QIu-n<|2r|AB^+D~6^z5G=E%{=gE!(3HHAr610_Um?@ zY+EFBG&MD^arU(u#Kt=u+8n+Gd8$ES4#y6=`b&&*e#-9Ao((LAIvHdiJT!??%+P*R zpZv{S=0WwhHxv1<+o@}RKhNvvt$caOk{u$R{^uslt&D!+kQkRzX5>{hRbskI>164X z>-MbOwBNwot15Qy-JIE%g3smFE?%;1nO9NKrdXcKUs;z_txbLn$=49_qQil&KPU5j z$dD-fHj%|+ffZ;9N%o5JinE6fIlX2$fA;KFz7OWt|M+hcS=X~buB|Lw?`_d`leIU) z?RI;n3Jux#VYle|w_#Kfa^#+%n_zW%18z1eR8OP0#DiBEQf+?==DFhHyJ)ynCY zJbszm-%pn`>Xvcc^!$L|>x^Uk?5j76-P|Lj9kNJ4&nzxWZOQ3-e`13oLIPGT`*Fi% z=jQJ&k^9SAeIgB(g+z+6wK+UKQdShW+Ip#5uT+_sh1j7ZItAcqVR%Yhp#_=>5zXA@ z%%mc47L=WzUF3JT|B%7H;6X$0|G87As$M?JG-G$S8M}jV=oHrl?#B#`c6^>|RQa!Y zQ^M~zn~voEJh`^*M5nI)rsKui|EXLSvwIW1*MZ}1$qkm1Yc6`Z{iz63xH5Ic;?%c& zMb77sd+B`FxiM|CBczJ_wAbRy{!`0#%&2&I`H9KqxmvtIv%5OAa%JAMZaQo6Z~FT` z(RT5szMr_h{mfaG-Mr7}%AXa6xeqO7?*5uS-{=gFOyidwK0CKu%~%<^dD&dnzE*>e zOpksE3GKMH0=46?hjWKrbrIVRbNOfBG_C-uvN^f}*D!zFf0c1x_I16PyZ+gooOn0T zXGzwsIAg=ucNULWBe$2&++*=xZogD-Z|BrNci)Lki)tP})174H$(!=!Ot=;i#2d1;3w=L*eNTghqa`(j4hrLB(gne*?(CM7?5 z@>o>=H2>>+#gp~lbAI7}^(Fr0%?}pQrODsCBDZC)OIfycf?4Sdq*x(x0ug1iXH z&kgxZ>(0+BY51LXBnDIvtbT8(A0|h~D!tdQkE@IPztR4xwobFf)9|uthp4OX zVwUn1Sj?B7U0ge}tbEONZ@u?b>mMW*|C#eTWp&}4aFetNX z#nThr_vOgm_y1eD`hHtn+MGReUw@yrakIWTUqSnw+AArIlYe})H2<;oaQwk7Em38K z=f1tPT@wC`bJE0HXTn_Pd|EY&@!fi6nYP$FFUzd6fA38_fAS3P{2#Y{%`MM-*thMt za+zV!AA^mi>3gFs{S2IIGOkxI@&2p)mY@Fz81EsWBZ{QX*VQ}utdc%?Jk z42>0@Q?zIIPQSG^+gkQP2lG2CZgqyU0j!8J6cLRMrl2xe>Y=;S<)1%)ZmR!ZcXNCG z{5N-Y2FKm3Sy&0KT5PxjNPX}90L zdBpKD?nJ5c+~q%Ae?Q$fvw6ve6Spg0oqq8+I!0mnHaV?)op}nyX7#zQeEB7NVy=8N zGZy}-EX?(kHU6sBl=d5scrVYYzPk6$nVMrllNVYr*EpPZEDBnE@YCa6-=8iP)vvty z&Yvx5j`Fpi%4%ihk%=Me-hSNB$yv9e^S+(ew0D0_d)LO~?74HlyiMhZn#l3Y?|U1IfK!hgEP=hc;!%Io+(+_-V0=;cBKtTFWh)cYYdKxPhOp3DQo zf?^}qW{2%Pbw!@fCvHmkoio$szvKQ{4U_p5gEthYrOCY4KeP3#(BnyGeV)9$)>-#| z&B3xlpZy|X0U38LzMpwPXrtmw2KStjBOdkHCntZpy-fe~-NQXsmKCMH7eC81-TFym zr1PmY^+~6>Sq*R4M_an?i~YRJ|L4M3x0|k3%XMYci?{!K6Wlg)s&d2mW1_vsqS-6X z&N9vA`S9aMg*5w~qFIr%LG2evlLVo}>LcuLjnJK1|^aWv@8jc6^q^gYpyin0Ke# zS$}wry}*t}t;bI$?6Q5jDv$r9&EpmGvShBMF6ArbH|B1(%SeBm9JX)$mF*{A7Ww_o z_MZANw7>GHa)ImN|L+;xIYg`uoIc)v2}^Pkcm*0l`6|d&`1=LJ`df^LdUDS^k+Xkp zb#uzzQ|phhm+uR%N|8}Me_mEoyXShBOj^m9WLhww;RlW~re_uOsxxVth!eo)@ zN?Rwt@{HX)V#Ev{cm4(vPhwGN(i{`0>5*Ir%MKBb+Hj~gGE-@bTD zOH@(*{LHel&ogdLuRNOW&e5^Eugdk(WTPhwZ#t)b|Cf8v@LtB+EcPAyWg|Dfmaoiy zDZEL!eDnU$Jx@v=p3lFwM{)l8Ij=(B&73&#)|nQcnHsA%i%F~h&7F7j+-LXxpB@7I zf9A!rdHMZ|OSj`^0xcgnuyF};o1yWO@B-#Nf6hcSgA&Xs?*_gPpM)0gk!G|yaJPWv zt4lgdy8Mk?@dIC$Z(KZku4?+Wyq8ze;uw0fqXj?maZymcrNEr&%PujP> zpBA~dEU5PL;Ugh8C+rQ9n_pbp`RT_;rp;IXMn-IQQPt6ydLWfoZmaQu?5nF(IT|NT zm~cSi!BU}Qq~_@a2L;ghipQ)guAq+9K2XTd<}UuL;=sOR{sJb2YrFi!cdk3Wxaor2 z|KGJc;-&KKA5~^fxpwH-wtMq-Ure7{r8%?c`ozg^cwQ9lk9%n9eJ6b>S4!Ac=9XA!oE!KG>NQy1z)htsr|(`9g{$w5a24=JneYwHxY zJ9LEe|8ZY2ZbLJV%tzCft49}KJN^6g{Ogy4zAt>i5Z?Sz`5>?Daqpm*RX+4rT->gYvgq|o76U8qq^O{>L1&+ zQ=`wZZJvgy*mRwKd4mtmY;&Fz zOfzPkci_UWevhiET?Q;(4{io2Bh>;9H$meFvjxFxUKeZv)o?loHP*j>%5cACw+B!7 zI=*A63pck3o&uGHUb#w^cA@WHf1cHw`}dOl_bEFQ)Z})pv-!ff`9V@s&%DZw3U%9- zMX9ls*7!8d+x3=P=I5s4XBxh#x9(#8(;_B*N^`CL#F=T^{#2iyw>g&S_>~_~s%8A$ zZDN*OZ~0k%EKE3g&i?7r%I{M&*0a8faNkpKgjZj)*XC2IgE#y z-F*$uy_DJc`}t?D+)o<9+aB+6UN=47yiPG+zjjqfr0$*thkY+ue^vpt>x`-&r_Ykz zP`kqVfPYr$#z>woNk8-6-v0D*|K!tG^L5r#*5ALbdhbs|!FhdgFUO0GL9hAW8hPz9 zdlCHX%)9nqdvhO8>7Q9U^UaytL96`4rpGb|;#P58Uah>1G>teQA_4z*+p7~e71!{j-?z#B! z(2$#=Yv~7#_gRuu z*;4UC^HJFMpMIja4J zfi%bVhuWqqFGl6P{o2#(+bePP-!ciqzORebJnuenOH(}Ya8u=dsruEwRK-v66~wX_ zTiW@`{d}u+@X@@dz8&&?7LN+*{AbzkIPvPD~4QC^{Nmiw=LNwTZ?5q9#34URpZ3<@t8#X> z`S(npz`%(K-TnRFBhUGOIvfg&LMyzO_>LbeXk~T%xkAH8z3x^eb^4IpN%dfAvneP02%l@!mW|pyb z(LYz!?0%*@-$*reRq;3V%^XkKwd1rr*Es$BXLZ2%a@46p-E*54#+itfS4tgOG%>

7avgE{3 z$?xGe80~6*l}OrDY}lG{q(jgs?~aB3>#(&^sz9PDkBc)p5wWLY z7=Vcb9G2tgVo;RPn(~vKYRallkDb{uk%z@|EWf$-b#uUQfNG()Ue+r zFLHHr$;#{QRc2OxLBd%#_%2DxMLs@rZ-T=T&|;HI?iMHhelBU)zwf8x$=;J(MXN8a zmfO7Fkb95HbG^E~``5oO@$B=S%AOvt75RJnbnYT-k*vc*L z*Ymi@joEQ4&%Vg{zx>~x&}%cqLzW#7_9(nEjobL$HpPIfYil$;Jv}E)n4qw6Bm4T! zO;Iu1a%O&ccQ@KV_|Ig2yNQg}S^3lAqPL`%9|a zuZkWxH_0J^^MK?%e@`e!Cgy(}mfaqNm4lZt>0s7`FpzLN2^q+$Qwr-3(> zqZcpxA9^>?Cgx7z5tqyfw_N1@ef7FIZ}-KsEr+GPPIr|3ukBs_PgVb%^wY3s?>W=_ z4|o1~)nv%DykJ34sPzHw&C>c!)o17bcz5o0n$zAjQ{&I{yyZ_zb_``V)GaKRn%Wziq07-_j6O9i7Pzx+f=mHp<&> zeBdF2zn|Z>M6W-k4T~G!yvez_zkdIf)#2yUt8Z`1om_m*^0?K>`&F-XGjDHOJAKaT z$$vH;mzx}&zxS$l+NUQc&F*k<#npZd-TCE`_uiDUcXu?+?-VqvsHvU0>~F7Ynk6%5 z?qk&xJmB)9u|s`<@jiZ$H{1K($D7rq$lU)act5{v0*l9jERF~3S$;Fm~1}X zXTp8++Sq9pr?ov6?AtQoe@aZeCVK_z?6TFIlRc8g zDHj$r*9(3;CY^ucKqK?s6woRtNz0-oXX_ufia&X^di}RaGtKko%_+N;dCl_Q+wJ$g zCad}GNtv)Q=_r?{ukX}jz0$v7IoRQa(1pnR(%=4Ruh*B;k1oF6x^Cl^9u^OUHJm0s zxq0dd5A_oae;zMp3WzB0J2sht(?x{Q>VR^V#p->=^Y}jKZ}XYNyJ^QO-p)m0f}5B( zAIMf)cA9x}+YQ$)rM18JO3l@1pMCSq`SZ3f&UA12H}#tQnPoe+pEogHXK8fq%%}A} z9@8$}N}TC&d4^w6(6Og}pzdwL!c(8lFs%3Ld%yqxzT9_jHlP1?Q%B|&Xqn;G`e$cm z7X5hGUVXAx%GB#Yiu<86#Vx@fectc;y>3qZzn^7pf8Ou^uNS+k~Rr^Yr zW~nTn({#ZOyll*Y>8h_odaZKKwL@?BpM358FI{Zoo%p+H8*7CW8adP_2+US#wz{Ji zD<$KpUeVHTy4#)Up-%(fhXT310vqmLX8d<~T6xOOKT=H7T(@!FHU4#S!a3!)PE#W` z7)=s-G$a15?8ET1ll_9%%EPv)%+7|`@_m`{nFpI#i;hXAb2#m*`T2DEq?t2cs+L`kEw>c^QuFz& z`O_wDy$_Qlt;^Qjsr`P}?Em!mx{chmRb07ub_6b1?bl=RVd1asHfLiv4 zCN`Y=)%N-Kci;Gv)!Orn>x;ug`eGNJRdHZS^iXh1dU~hq`boi=%x{e4)0^5KDqKnt z>L>sW{r_NK-(3wlkfy-8?9BxGZ8h4TpL|X*_x{^<;A!mpsn<$(&4@g7I3@nyypF_6 z`||i-hP>za`Q`q}v}>0dJ6=w=m-RI3&ggsbcT((=EsG5A{LO4PV(0wQV;#$P`o;Ta z4E^2R-Bs>yr|CxL$P^thmA9|k)2sQe@~q8|2hB~cwi7{%pe>)xNVai(^6T~b>FM)p zuRS|_^Ii7D0E_ebJD&(;USBu2=EFhuNs}f$TFNf@`r6vfh1Ue9wL->c56o>?{ki(z zwVOt@FO^H`b}K)UQu^JrlCqvpWA)=-xJpuPfT~%6m#Bqgum_OXXkc9RJ<>GMn=JboMpxtN5Ny#5QtPcF$&A=yPk>I={VcAd6 zQo_xbj&uq?ot3?Aqu<+~pPzfrHp|`fLYUSB)w&29|pvMB^{?y%Z_JymzQ__O{gOztJV3LRHArtbGp zkl|F2IrYI=e%i9Fw~Ev%Ecg_1)fzd}9M~&Z`13+b)-G7!@Sx(vxpQI`2e#+j)tN8m zH(}9?rd1hsuWmbNOo@AT{!E&Vl>W`1k%8xx=P)i&y8@bf&HLB&$oli*WBd>AOx>`- z_L`u6`tm!PlDtw;>=t74Lf9?RSF_}C{QvhmKl8$ZMw8rISC(HoZ~y}RT*tG;G|7I`fx ze}B)YxqBKx1&S8gOL22D{*%MRK|Nqn%-@~)ramij4fkqZ774xam)jkuJw4_)a zR4%BwsVc+~4qDXm`b2BT-B+Ly9*YCpb8oBJKk=J-ms8s5h|?@^tNtF#D>1Xlj9c4p z-aN83>#C5&0bwDbH#`Ma2liI78nb`Q@{X<$DLkFO?`N9z?&?D=oOY=XFU&Ja?fM&h z`)IfLX~lM#N2#AIpU+X=HQ~jlr>B?Z-rg2x#BsIgcJB6*7v1G`&)a^V6TUM|;Ys-2 zU8O&l@8xQ#1}(KxWMHpIOkDV&^3Ud}>rb80l@INf%>nP!vElr3+Q!>2*Y?WK>r-p? zeN#D7ETqu53KVgn!u=b*eK^o--`dVxbA4UxHO7Db*ZemrJ$=scT5~sOqQ#TNO6<$@ zd0&4;tEq~0ZEK3}1?2;?G!hMx%VUbpU+M8KfNc`ewx}jBe0)~m~3Ck zsz2Ylzj3n9syQ~&DvUxOTpRcb;zMRUH)pgE`S4)w?Ah9pk&#XI-0l)n_J6XUENdF} zV{LmAs2dp&$|Ci&bb@lw`6<7?zV@!F+7-(n%;#ls;N$a|e}vDjlIpyWlm7nRUNaRx zPvfZq+yAWF{ch6Ze*3s%{qpZO3;+G~b#i>&&!cg{Km67NFYlW!lQi-Fzu)h_9T)#o z(X-#_gU|cD-{;x zw6S<8Z%8>QRQYUXddh(Y#!vJA|2e+=MdtFkYV&Kq&77;cuk3Bqe7>9`n(jNqEFN#X zQR3y*HGQ3+vRj1u&!gh;I+vIEe%5klt6OqxO=(NY%D}~K{*q~+cr=N`+JjrzTCwi(6S=cf}+U_Kh>W-_aMr8qSb*n;+!plpz(%`4NezB&Ny6bHTYQ6 zXYpVLE1&w0M~mjCRR5Y3U^3fmA_J$3W}{p6?h^|=kKg`r{_Cr&!bkqM1q*IYfBfi? z#cyk`_1&*tnAMs5`*PWzRWUSbOUA>?Ee{X3pXS!zqi|fNxX0LR=W>Olp8Q)|PFkHb z6qdQZG2N+S>ihlw?^zu%e0?W8YD>n%J-^@WF1lCw-1B&!Y;C7OKZ}O~3rEJA{F-}) zjH#zCv8TkoyCkro2fSjV!Msr{Q}*d6zwfuxbN_8!3YlsQkp-=?=qq*EwxqskW#Iuq zHrMpV{QEn7#g^4;{nV^h-t_*B2fSC;!a9XG}95 znohfz@^)A0>o-ob_Q~9S@Zrto^S2})OpmKtY4hO#v)#oNlV$9GJgB}E?i5q9ByvP$$u&d zB`Z5vJQOal2(Uf!RVflT zJWK_DfEKb}THQ zk$K72tmZsf?AAMJqO$w1DUKe~mU>V3y1p*9_FTmG$j+H64or`J-RV7lasO@cQ?Fki zo4^Aq4_Ztby`%f5nSHgk{{LvxlF~4zUBV`PEFKDPK!d*DoFo3qUYMydgLPm2eY=~R z)BESL^08Z_HRo%^=*^xr%YJ9y%DU)GP`M}~RM25^`;Pl8)zT@uN?+gNJ(qu`?dYTF zGya~vsS#KayWw;mi$J@KVkD>^n0al@%;=f(WUAk6{Peir{?yI%`MMDcE*n3zWSFmM z@p6_~ZdUq~>Qf8WRDXX5-V)VrGw12POMiZT&W&bf=bOUDEA>MtQI69^ma(R;gt4kc z?oaOH>7V|dyS})O+1iImMIera&16qVUry1c&pT&#pKHJV<6@Lh2WVZJ`zepGWcw1c zF1@x+_M$UP;p`TD=ErrXzV-dTP?(AdO<@n|8w)S z&AasKI@=4?gcKSlaXzS){})!~VbDZttD)u)Rpfj$1v> zbMsll&7zyn2(Ej2MA(1I`hCAndGi~^RX!E{`Rnz1*0*2Y+`PP;$JkS0632@r!Y%d( zt(fB1SI0hk``c{Nt`qw}OBEFw55JzHqrO$*O-wnv@qwRrWE+{!>P>X$0F5vlVb_p3 z)65)`zT`opjQ+X`jpI!-W-#{gZa%dA__}Gby3Y({yexRMa#b2R6dE?D^P0=3@h0Em zNUJ&gv6uJUb}7>=i}ZyroGs*!C?{OF*}6f&pz>47?g!6ux8GGeE?Yil`dLd$%S#&) zoe%QA(s1VxKffjSwwZSG^M1S9UnkDYG`4kpvzaq#aWdltPGPkhtaf`_eAt|J!{B9YIGu+r!Ampj|Yk+1E~l zhR3Q3tNVQs+1OpVE%WlRXVagkv~S)Uy*+PJ(orr+>$05A!qbAAQXd{_1?}3rv0vt6 zR*$5LKnl~k^S5T?H0|F|Q+>DMX3!3;6S~2+95J_^=4D20UTgE~_SH%8B@28sUgBNg z^16C$o5R*%{iPGX ze_uC2lUu@he@FzQR?eIoVS0JR)BnEqZt-${;dqRZUm#|cBBRiU*$lD|>J~3bh-CSY z&=;wB<3mE<%;*_Ed)Z5J3YwJl)$`}SS+m`z$mn|a5w_=wD|DhGLF+yjvUDXaJvQxG zP<=SNr}xUlx%X#${jxuGl6k-_w%QI@+5)LyZRUm}>s z+FhrV_tW^iO|tCMc@6@XSQkc2@Z#Fu#A^T0Won9Co|4`#6T*-UwA_3hNWHf`1Q`zj0i z7c0*Z7v!Aapvp0$K>g%xX|K7txvIMBw#|LWn{J*|d-&tpM1O(r(#HND9<1e#|Kx9= zeQ<85rveYhgPkn%zfZb!xBPyt^NJp6^K(|0ZJ+-8`#m1i!K^(eX_7I)XTIItV3%iS zXQzvQcRBdrU~}rNEtzpf6=qHzVitEF>VWoROy@EBaJ{6$@6g}wPL?SR$NnEHZQ)cc zXz;)PrQykmJkf`C94uLC%TCL>?@@UDO>eUJ^Sh73xi=baG;!fL&%U zl;2+7?VaN@r)JNi)SroSzdN1E<2q&0_|I|fKW$rSIiAH^4Xc;x+%^q2x*X*(+bWiI z^PbJ$^39F+96vS7*1INa_p;v+znvM=zP>+w|8a2J%tplk*pgG# zAZKMOi^qX_rjmx==es7=2?VHHq%}T1KiM#vi~nUoO|qGVnapGVzK=!i?LU}ZD<@6f z7x`^@{^UQio;_?nS77fE+`{7F&~w>3F<;m7-dx4Z-NvjREe$tI8}E7eB8=T)jz`DO z-WrSjhM%MVf0=YHKh7O|Aj^*j&HPW!+y9>;owvi$$jE5Vs)T74bDmV}Yjpm*uwCxc zq=P-E(q6}J6MY?oD=ZI(O9{9c9gIot0s`jRO+5gQ!L^6%-W z&naj+UzFx1$7pfjV@+>*x{Y^o@7$Ns?v%O>MgQ=(v3Sp7RrQW^C+^yq|r2-B#lx?0hmWjy~wt-!~!H&+_QA z_`07@jWRE(=+6X=hMb>gyVm$ir~14Jv-5VH^k$RP-}9j<_Kt9O@1uPuMDDR%VeCs! zFKzI@ZRW+5{OR!SWW^j?vAr^tPflO^`1i{E?Z0wgrXD+aduLs~r1kG2=~s{6Emq^6 zbENF2|ZWRwrM?@5`LAZpkDTY!J9{Z+iyqSey>~Y zmzMYA$o;%sCllMgZ?%=%*gf%4d*@n9wKdD!5Al3l82f&r-q-#u_BIdLIpSY}X84X} zu1$HfKK#VHzqe~T4fGjU9;pVd%lo*awBhk@5B0?bN?r^Pf>=2xG~5TRq|fyIuG8~P;g7u2jaj&WYNfM@g3hX%*l+%3#(UaZ#J(wpqN?)a?Zwwx2d=Oraw zx9!{b&|l54Sz20d!N!9-dwtXE55MeX&&esM%RZBT?oZ#G>=s+a#sZaw=h8O(59OX} zKb*;#81%r$eTIn6g_kooSZF+&s(f&#ZQY*=f5D%BKA(SjCD?!K&+m83=l96j-tuNU zpK`d3ck}Bld%j%qzO?41%-i>(-Yjpcz3r;05#jG%8oP$gWR;))`?p_oYQ^_L&~!>+ zL%hCpkFrew=g0Mli`R>+;`@+1d%kx1mYq&%xj%gF#fEK+yq>;wchoOa+hDtY<&*ww z_!b*~cY7trL8D#oUtbIC6EJDzdw>2^V(+ih3;OsVW4h%fw}|x4r~5-tQ`$j>q@tArBxh0**o1X=ZA1&X~ScgR4d^< zD(X9|_LjasedW@rnv=%*pE&Lmoz^|)=R3pTVC9+S3#tK^4zIV`6Z`$u0k<~}ITa1Z z*q)oq+%0H$y=?Qoz;(vHB}acwy2TUg1sXhWVBr%wu=!lDzpd))>+7fQez(i}(&fv0 zFZ}s-JKt>H&QnSy-RV_tw_Z;r~$Jk5p5P z1g^`R&j$@=ho0G!#FS9-@8|Q=A08e~JX{@0b@)5D}}i_OI2n+IN(-DC_mYPsbi{8l``}9`@ve+ws$`viZiw6YY%_2E=^` zUF%fYp?l9@?auJKr_Y{A_mSp0D*1ppif_{7DBfxE*Z1nnudO(Fx~$$v;IYicbMl58 zcc123T$J12?(6fx+4s-4z)dC3IIS{%b-p!|Q8 zn(&+fIFey5$2vD9bxyV8j=@y7c|8~DGztefQbtnH?x}nMGabMl}m}J6>@!0DR zX#1Y*)YTU`lJGz0smRZ=v$O47+sh?DFk0fnF>K1VitT62TdI$PO?1to_czTLIcYX8R#{Z=d1SC`hj zzIT-;zH;4Do4y(*clp{anHaq_mt9Z-7TsbS2NdEm!`drOD_YRc(+xi`s&)>7gQWpgXdeUI6y;p-}cp< zvwZ&J&4=sp^}5UF6m|VwJ$I&Y`l*NQ^6ylaE4l<2WL{E{_B$Hpvx=E42NVEx(hPem@&DShJKR`?YI{dg=-1&F!b&{kFoYX5 z-hbWm;0^Qcw@fqsZ7C?(y!pwQZQ&D77 z_mqFpHByZ zw*TXto9Wk|v)qV$ZkxT{_UF=D48d{+7PEhu?PR^aU8Yr^U-#ho<92%6tL8RT9c%6^ zFEILSFz~T6SzbzQ^~m%&US!uIm-emaP1=&F`FL>Q=vy02#IKQHzuo zr04Irt>kqlOI6%xbrgerYR75rHyx*qSOpI~UR?Fy822%yE9?7iF$9|(DlyC4pV#?s zn#PAuL3klV_S{WVo1eUj!&(g?;dq8vc%Brl1yIC zlAXVPEZ6F~vuHu2+u{vj6-$15-{N+=X|=&qk%<6#ewmTV}MTCKaLq)kDlj?yqU@3@t{W7Ky=Uf z?-4ibL8Iyu9L}nL`2BvrGfS4P)9mvRW}Bz*+D325VC-bu{_MVO$_arE(QXyN+M?p( zv$Hq;v^6pkdVPI;xZty8OchyKt3EOOSIFLVcGsQ@d8TupPy_2@;>9nm~FIo!crYC=%85*(k zcv)p-gCf6?i;DQhIVbP>gGtH9J7xRYEw#VP zQj0Gw^Oe?#-L>V+vFY1h&3L9CvqOO2{*S>Lt_L$O_g-J<@2=V;|A9=S=Rr^}SoizQ^qiOC1I<@oHCY>pt>d-0vZqyrU~|lLO&R(2P9;=MD7<^XBy(IpU(Cs=9J<#QZa}&H06{=FWTc__5;b zTC=PxCsr~1OT5uqWBQJAoe!f0iw!Cg*;_brX*}peFyi)!n zUk;WmDVxvz{QO*dec?@>%O+cMZ;MS<_dj-F zp|khm(`#KD>Sz4q@!Henv-;B=UY44@e+{2s^1m9VJh@)7XU`&;*=z0Y{BZu0sg(YNOx znzwoOS^epC-@ko2_GSOkPnp?y+cl^C{>|bvW2(e|-zZxF)~6k>mdbuOlE`%L!_gBn zZuT&g&ye5-mGW$iTo0x?PR*Ufz&Xd9X_?){Fa7rU+Z$!!=Yd5i7JPc*d1kJ)c(=Iz zwhzqjFDztU<}uX$cpNC^2w?a=r~2ST zP;`Ma-yhcpFD@>gVVvIgiShh-{p{NpmU@e;*6_<%1h~pZC#U%`{8vfl`?NR5T|GLx zy|PbGpqp>ws~u%AUaH&H_KM8R*LU{dHerf`Tn|$tvzdfbkF@u%3=EX^Wf2K zT|1||U%?44K?$~jm67X# zn8yr9CY6A>oB<5$&z1KlLr<-3oaOHDXRowb4oC6o$=!OpID(h^9c^G_J~elXDdV5b zUsz9k_0!+3$8F*7uxP(3*PUrQUZ>w!VskZ}m8rJrV%^)1<2alGTiG+*6VUq?5VpFi}9FCOSPWIt&eH~YEVqZ6;({qUXB zVReV!jvWWLx?PTo*i$RTH}7`Zc4^tS|1V8*FEpuLe|OWK7gCIm)fu*5pYOzY$FydR zE0f9tHBkO#nRIpn17`&&@y{!ix8MSe1%WmyH`a52*3fDFiZ1%~<>lg*_y7L>{vW(x zj!B0llu2_R=Om*zn&VtZWU`m^pr zb8BGQvHY%JUputv7Bs z)BnBp5S>;y(Ls8?-HtW?&T28)n3#mPHvInaqd?e4ymFHi=Y)oAQ1jqe?^dnOhLx$U zYd?Fvnss&(180Q>skQ&($M2j@3-S$`Y7h5>Pk60lw83sslhgvo4DKVXCawM3^Islw zU-qaXwQq7RwCZsP6~4f+eJto&p{0 ze@t3D2k5-QeJy;BZ#$!E@P$w1*#G zNZe(gyI9|KLOjRO53_1dTUvw4ixpd0y1KhVS$1$dkd&0vVzM!`WW4`SbxMWk)hNAC z)548TF9pj@@7=eRpUIjTdR6!LN*1Prb%_#U_Kf>g|D@Pl`LR=|Dq-?6mOWlI1{Nto zNw2PkGgWwaaQyl6Cxpdi-8gF?kY}z6C>$+s~iWM3y?d{Io5ANNo-}^JN`eeZ~6$hqJ z-v*|NP~r7U|7Jdn7JJX?`t!&8i&KAmz4BaRO=?~7)-T_RZ>a>NvefkQ&eh8-Q2r(9 zv}1pMgU}(#*r3!OHwD3sKL=I838q_aEO_$#`E4Q8r{h>&P%ZhrXs^=h*NPo(xH<$yK|AFyN+IWYI}(Mx&)0w1I6{ZeHP{;n2R zP<+0ETX!QD$Oa2H1+yKJdHKI39y!gP`p)3I>$c=K5s*{K4t(Y~5f#P#ATl!2!`C-d z@WJxQ{zDaX*MA@BV0vzKyZD`iyA#va*$v^H=2dCD&Wo$gPGN{! z?kAgCFC-wa!0dnsKjci06<$mt2Ubri(|zTwf64XFy{TISwZENT_#?Lao&9nzrg<(7 zE*~D|sx6G&+wE08XF10^&+?9B&Y7-dxLH<`ubz+}!NEM^C5 z=RQ-sVBb-uF|TH^g&8P@6&P2ue6fnYy}(0c$BhLY40Ye$L~cF#XFhNE;v>6HWYvOJ zFHUes75W{8f&6ftn|0rhm9}u00?mC?NaU`A$$B zX2&S6Z*Noc@bn6Csnsq8`|`R2mw8&&V>{k9&2Oe=}9>~f3kY+QS6YTlv4u`gCPS^WmJ z_85f<#2G7Z2VJ|P)Rdl{u54=?8@aH9W8SP;t6V;$%(pMx_Ok52@*A;tH-W=-gWrM^ z(i@lWQoOcWROL~0Oa9M}ubmzDH~n0?etrJRAHM(dUrWsNW>N{*32J}o-H*DwtMA{m z){3ZWmU}k5wiB851;BZpn`0IL$2C_ z>vyd_s2Ce>4q*}ZFAzLtn0y>TP4Mx`!T5mbh6ZJY8EVCCaqK}t+85I zaGxv7^!}V0%}3=94h-fC*B_}qu*|c8MMl|y@s6RNPHKBYooH~`((`A}9^Ezhe!|t6 z!2~p?|=z|>$L|;P2MUyIPU*a4ty|4ol)zi>dO;7)=hV+ zI@<*NqNhatdwlriZRx-`9C|JZ|}IDRI4=c{-$%+ zq|aaeb)@vo8c!ybfI?6Wdu~J6c7^WzpD)4NuC9-2Bhu)#>9S@48^#5q-=Fm(1 zNE0D|KVK&aS7TK7W5bZ9Fpf$Ac?xq%4B{ zKu4l36O_34MsrI?SNKKEy1F_qhUFY_m){vZcLwEU)9DRj2cORulV@OXVDNNt4DmHT zdZkPw>d&;rE{(Ih9tiDNWOtNRgXff!V*pzhQ*q#v`Q1YQYeasFEjqIN#0`hd9I8UP z?{W>k7$lJEdiL%6_wej=-8%0t+dJ+TP3hleJ8|n)m%89T>pt2C&QahDkp>m2 zXV-o@(|GDowAcluhVJtZehJ@CEPeHRo@GQIxGMMI_|mZbLsOci;D;4m$_R`?_bmFdSsfp>eIzX^E=;7JCXnQ;-T&LK1_O49(u>} znK{%h_fHXv+EI}Ba_4UUEhlpCtrOeh+iz7B-I6c3<*72$j*dG@ z4c<$(|9fE*d_G_Byz0$YmuqJi zY0b5H#(Np&E4Z*6Q&2T${OP)Lp0PpSkxZt4c8nc7Jx6!mad{&8)FO7i=p6M6S}U~K z%y!4Su0P=vr~hScey1s@o^D|Ac<@K)!lA7l52|YQ=G|8+$ySW%eiX$ZzHrYTkAmQZ zx4-=Pb8gz+$qr|g4644oxWu{l>$T{m&!<){YR$a7xK29Az3%j_8zFzh8l$hBQL4zk z7`?CCK<}xG&h%Zo=AM$VzYWUyjOm;K4CT`#PES`<;Lh3mvM*=b=Zha%Jv6LhmhtQl zk!!TJw&`?BT>Ih9v5guU=k&KP?l_R~CELnn>g|GG)iVC~A`qQFnjw3&}9(sFren4I8rjF^ZyZdSi{rN#{4{$?kRbPYH z!KW)+3phgpXUG53ZR`HAwzW{V?zKmO;Q6n|x9HAfJo8+wr=cW5T6(wy`V#2-Tv%{(Wg!y4+&WR=HupCP%0~EV_+>v z&)B$zm2vNiY1g%agM&Zb>zwJK?YTkYIL~+C7ed97ZQJXa^50jmSx>%hedVC(S@#F4 z(zdVO1RXt-S@qm&q1nXP!pO&t9o)HX1^+#+xWfEswREN8Co zSaxW;ow?|Nv#tfQyz5-^Bdxt9W8Jqu?Ay4B7aT<9j9d@isYRW>b(v{L&W7T5cPw@6 zEoIw8;!bOX{4F{BNBP4K*Y%Gpj@9kinY+rF8Psl`(mj3Qh8+hcRCWiND@a|w&gblx zxXk&htgcF!Tys>vd;g8qJ^McOa=f;+n6{7Kv2J08%(W=jHQg8XynAx}N6P=zeP*u< zEOi^yQ{<0NdVBfw6=6fG6Xq-j>k`{aeQHa3-t}G+ocr+Y(`J4CeXaBR>w}i<*&M&* z@4~nLTGxGD9}w#L=6-#i_9LfRi&F3Q9eelk--LFC(`Dy2{bYD>@7(5|cDA#jA@yO7 zyEZ!~RH}2dRJbODTu#4w?(3$`ew}AOPc0K>hy?E!^EjZ)nNb+keIx3MSmW(4$-Y0P zz1}si()!uttJlwd{yCS$b3__IcV zi;4r&)|m{fA8yrrziUt)X>oe>6s3mcOO`yCdpUSQy2k^bsooE+ZR1~<9#gOL=dEMv z%!eNN9|Oe`zqDTd_;+IYRE9d=&(1Id8v=?8D zBi&D5*=z9Yzk$j>!H)W$?!V)XwFTdwwlnUceiVcF$I64xexJ_Ovs=af?v;h_-%aNA zpKCo#T8hIjH%u#k*rjIvxa66j&P_w-n`>Tddveg>zS{Ox9Dj;;dd}EiIVaRx!uH?S z<@*lSf788mwQ>y$6C=m5yS~pB`OURD#U8^_5P7BeV;tv(n<|hd*HqyPOj}njSbMbK z^I|>apTA6ljU&c2D3#gHd&rtHhpr{-lZ+?r*E@)Rj_|C&&2Jud*{vA z`cKc9LI2=n@%?Mxc+9*1r{Qq!l<3sd?CcvG_RMql?P^v857!BdZCwo$Jn@?f*3^uh|_G z1|Le8L=M>g<*WSh;|%MErRnnw&X~T*|MvW0@TVUSdGl)Lf0_}uv3lO8wPh&>9v!sZ z`rSyz(5$BBYrlHP)WW&xi$h)1ZSq|_=bJ2AdUs-8j_I+EfBQTuv+C0;VsCr<|G)2b z;@J%Syh@kIB{#HRpD_(*`EYtR&oN%z_hvbf`wkxe9oxNY-^-?rJCHR-l=&|-;wPSns-}-e|dY{DI1i4q&)~+@? zH05lK3&ZpmVzASvR;*;n;k+FsaQ(BH?s9OWVMQg2%bjbU^ItE1a@*>9DX1Nx(D=%s zfvKW&hqs)Kn{GpTTACL3l%<@)Atf8x`FBR~`*ts9Jz80Ds$-6x);1CUwZ0!tE!+NR zmzeU#qVrBUcALFKU&>Xh=JnT~>2UdXXoFMb-6+%LpO2Nz>by2F_QS7hCoK{x;%|FD zDy?mNxc~3Ld#yK0eyumWsr~*1C=6E5=5_u5?C{24KbHS^^dQN7w^ndo(fXwwTwO=+ z^d_I)8Yp=`wNKEnZsEiGcC5dH>hv%7)qMQ;?P!nP`@`pNzt5jv_Ih?&R7gr&FpG*( zeaRgrQAR!K#8Z9WL=Mb#IsxuCIWU>_Fu1;#Ds#4!6)I?KSoz?sQ)w8uAKwp}#8uAs z{wg`8m7(sKz=6G9tIo5my6ItkG^WY=mj9u1Prjatzdi9`_X+iBnwtujxyW?C?NF(o z@1QJH*(!7~eU0?G9B-M-^Kml++xE79`F!r&&8Jc2Hn)-mrP{U~NJE&&?gH|HXsbTzB6{GR7IMgDif7G~rbqSTF`3 zoXNjX=J~NnAfX35pvK;Xq)2Y2XG}jPwbVyWNo@Pg(UHIHM|yqP1KvMXkBZx+#AXX` zwEp?_On*iF`6i>%lIl?1O}{FCCtq7CfB5!kcH?V&?p1YOGYdnXbZ=&=xc+OpwA_P! zW##`eY55YdzZwqAG~Ckprps{dyrX8$E9#C5tU2wStGxT+JiqfRGHN)1Y(c}w?aI;M*r{gGXt=%jaQ0K71q^X| zO54RIbmVq!n799@-JEkaj{@0O-|Fz>1L|9dbi}`|GICJ`U7HudX9(RXK;_} zpX{r){A=90MYGe_$)`R%uO}ujSwF6;qd~%EcU{WPZL05PtnJIu)6qM?l+1kY$eYLS zGPZ5u)oqxbHl?`e!=1949q(D^oV5)0^SnL1+I)}PCd-%g{p*fwZ@<4W_QQK^!-KuN zr>B^INWUfLyrlk2+Rx{Dzny+I`o|VaZ{jw}nf93VI%iK;-8vtZ9rNeQ>juOeKRFAO zbijjDDh^D3pkZbe12J%aTAU+*p`1}_^&jU3rXTh?TEBhVS<;^^U&U)6`||yzW9AQ! zg(Yuzb^iOuyAz*=tv>Ore-r1DnCV4Rr@Hj)-+7jKhrup>uCC7;lfK)$@rZi;?cElJ zdC@acT_UFUR!l#4q|8-LjpcW%qSX$?nmsjFom;o6?+=kqIK|AC9(-SI`-5N87JqoU zQ1!>p8^@o9trreBUt#c(&o52x*ym%;4AGu)aUNTCDNASH%4+`kjkAM6Pw>IRhYz!a zeYhTctNf@1t|h?}rV|>fLCtY}|83v_wQC#!4BOpWEBS;MFuXHZ#U-8-xSjdk>rXOA zZYAGaR>-&2{NbYO0@9a4Dx=k(&)F2EuBE{@xmq_+@WN*EJGp|Yd0Qt5&)l@&wP?BS zy6nQ`LUYeQRnwaPCi&&>IT=SkE`qGz5lYO{?-Y=ezshVLXCSt)uJj7qj7@^eHha2n z_5QNmrE&W7SFEN1(vzo&GycoypWfT+6#eL`{k#JUugE?%J^fk_Gy}|J0x7mY)s+IH zP=Pe4)^f!W0mP0B194c1O4ZBwnJCSF#ywprN{An0=1JBs5R35#mP z+y$%h`|Ej(rahXZ^rz^Lan1kdTu;QGMolRVZ=1JWnQ6znU^yY13#E2UHBG(k+~1WR z=0}Sq-rIe!bk~fdg-`E2pE#{tEVifDq+MD%bDiD4m0e2PH8Q4M)L0`b_wt`mnQH!# zOR~mi*8Tfjn)7^ShU~?oAJpf!zV*!hxATDUc0q~A{B@fg3RRhGw6(d*@0N-O1qW}e z_?TppdFjZj9bMhsit-=D(?MtXflk-{?CWs*MW^iDmuKu1vzKk&nLc6C$SLv8{4$?G)>-oPhjz~_`mGV&w2X4)ft5}ybo~SGf0}!W1d{wyFIb? z!SlEIHB7ztTbl%4$>*`Kxhfc~uzMxh7*wcuR#5yF&%S$gs|15L?#o&I2b_53`8F`^ zNWU-NFE7jbVbyNe+}^WN?_YlL*}3mxmCN?6wGp|WR_wPfx-;p6@V(o%`+Wihvs^c9 zoz}Pf{c-MSxou^fpyeKF%A0Q8R}bLa8!!1yL1{*KS&=Klbi>|P7QCS557Rb|o0GP* z_WVkET3>8-uX)Lv$;aGg#TqWVy!l2+m8ji~-50!0Zi-y1WR)26{N|0CG|{kk8)N@9 zuH(LPm|5_ot4?~6ePb8L1MRRi2R=Nk&ke9DeB^Rwp6%_V2KPQ0$N6@(6DCh?maqH4 zXj}DVgNCwX^|v>Te`>29wu(pXJqJIOjDu0PDqTF~(RbtjhtF+({UQBVoQTx667az@ z3H=T06(^W;=Xdi3?!9RGjqCB+djX+yl6Kf;?QvsLd63DX(NKL$c4gY*jRFVG`!MY3 z`5>USUHC%a;iOcf>cbhk-?IN=d%W`Ad#(J}M{h5mwmWf|PTV_eYk$687p|YXyuVeI z)x%tA!P(&d&kpm%e@t{MDKYxDIbX1B{@s+gC71T?)@t~kZN+kZ+YaXJ-i)9A-}bLM zrY_T5dpNhx)KuTob=&Le&E~1xV)?VPHt*TDptbZPxA^^n=**z?+j~xQvsDzHR$ifV zS|Ymkde9@v`k z|MJHE4__B|bl9{(P;%P$JmKD}Mj z$Mq8{x7dS=?($nRUccRbKdel<=;8hU|IFX*`Rvziq^YS{@t~3Y!Tb09)8p$T7cQ)K zeI6l_!UitL9eULlDDV5s{dvlVz0a;ei-sai7h2JwpWpmfYq(0lJ)sP+d>+aq25Bpo@ z&y`zLF}J*buTXw&J(p2^^vT<&r7!;%JH34E&b!MZ^sX6h*8l5b8!&52)q+nCwp>;F z9IO0*H~x;KEqCebYpvRH57vFniqtkY7Y|P;rILX^3Lsip+B|5 z*B!~-em5;J?c5y8877&NPSyYYdL49CYPZqtZMlcPUXNdYe;K`kas3Lh-#U+TFiU*GM@hfBcxy?|N!> zl+7wx-?$Hlo^(%FFfzDygQH01T#9VzmQ`Bnhu$UI#yQ??sG70$VP1;p66?_Wv8}O! z2fnJ!XxyhbV|QVXcihuQ@_KfUl(YXXEr~NWhgIUn_u^9rIuB}1Bcb2E*JJao0UDq_SCJd z*%$X#mv-&Ab^^%g4 z3*Ty?owB2HW+o*bm8hHn3!gGTOgq;K{XSlwDLE z$g)1PeBNZ$##gQlOch^u#mkvT6*JcSoVoJE_fsEt6keOc*tvdNMCqTJKf*PYA9Pn= zmby}918OJd^Zl2$m-~AmOEP+9)SSOZZ?Js&+}xcjx_MT;qf7do+m&K#a+hRoZR$O< zV?x@FMK8~udgWSOcv~aO^tL3U%?*8?j}vVT7ChDHxW3bNU6{4;$tl+v-kX|j%ZXbq zTW9;x($4xr+sUi@{i4B3u`Jnt9^F~-wO_qJ>G1+r$+?BK-4kEj{t~w4*XP=r@|2zD zo?Yx^`QzPWpp)jdRq%l9%wL_t>JMJOUfq(J{j~huolfbz9gK;Ii9i1RelI94o*tO< z>B-5Z(p7aI4zlm~dM#S_)R|eP-2Qeymz;{B ztVsJiH%r*7+o{ZGQPU%Xo!5m@#FjjKxm7tJ@Af7r*RJ z{%@$X?ti7}cHfk6aJv(0Z^SPUEuj~6(pLu$*#jJhT^CII)dL05hWqCR)gDM-k6=Vbi80_u$N9kWt zQ&G{7FZyUY-@bm|ClPb?KWB{3JD7c)^5ye+`^TG3>ly1e{eHh+9&|h3De;YWZ>cR* zns>kOxa^JxP23-Drq5T@&^WP3_M}DKpBp4>3&St{Zzq^ z(%|F81O#hb{-}s@zI*%DH~xcG!Z|(A8fbL~#yf>!I+?8vI~o239xC>5Uo>^nnId+^ z%1QrzxaKQZ?NEG{vxEJlS|w{^Fz|CDfh`X}pqcg&WaF?HW>bD^|RXyX69#mj_R zW!IMHc3WRq|5Cucd-}BM+}AG7Jon;$-Hm0c$ZQVtdm?@mb(NX!p}p2YGwNe`GJk6v zIdsJ4ljFPV>FoDRa)QimzSUAPI94IBQRI0~uYP*IOzvgBqj}1EBJ>&yjy!lagHwxX z4pYU>9j8Q2oIf8frj^uaRPo_~(e*FW&F@t(f4^6~UTACin;VVObfaH+FLf8(KVAR- zYuizrM+lwoM)nliE2a{HO1O?;c zO!g1I`aCss7Rn~}?*BMPfwMw`aigGXY?kl?-kmYL}`#Qcc@vYj8EjL$iDnuc#TkFSIx8~XTF}~9C_WYU++`N|mRAck|HYsAyOe21MO=4nhszp+o$UwY>;E*qc#&a}b>&1>Gw6h=Pty4p z7qQm;{ThB{W$^JuciwKl&v)}j@T7{_-{0N6z37ZHzirF(xGGQA)u-9!^-7y>)A;;+ zetpurJ3C+8+^p`_CzDw?7koQcz+H|RUH5nXw5&+|y?@8U)Y9FZGWwiX9GEUP_I^m1 z-pcWTdkIGz^Lzc}E=*UaGO&JlRrTB^<<2vvALX3h1wz#}DJ7=G5=l1|10N**6RfDZ z%h7Sw-PhYTX}wdq@~ppG*G|s5b>-CD{)$fj*L&~mSbuXyuFKExBa@ciIhR%0=oa_( zg8j2~L5wjv0U8bAfB9Q~T)kEOLSf%ie)~Tgy57a@C|G!PlTwC$^fsTpr5n@F%WcWL zd@M<+nT@yUp2wF1jm#NWS50-5+?ZAQX5;aKmrJKVx*8t8b;mm6bMtJwPn_`R-fr`y z6ja)KEYRZUV2BF|X*qPrY3ujXw_5a9EsFjIUcmBHXot!>wV3obnNJ=+b{5AdF~f1atAU3thzPblN@UFqpZc=VJTUc2t<_T6-= zQfb1RIX%mqN?@3@ewYy0Vha>c_|@sKG$7dp4EiCP&`cvSS%<(OCRZ*S*c8@+v7 zMi!(6A+)Rfec#vD*H^czJ!>s0F21U^*tY76Lo*xi8U4@T;yGkSL&yD341p{kj(iEO z$b8=H_x5hkL&go@Y4#1y2acZ>?=Pzp{pyrFl_5|6#z8;hJ>WURJkaEQu10pc^D?Fi zFE6e-?;nEE-uqV0HCR<~t-N$jn^>u}MBt&t6GJD&9o-uqSB|paZ(Xe1w8mp+5UhHQ`6% zj@x*O?E0}JzWi?Ki`(1R|4gj|4|?oo+44g-`{VyN^^&*Gt=$$qPaHI?vtSixN#`ym z^X+vDN+wr+{yTA1mCc)e7LNsGpamtn-`BUM>v!kf}MB0QWC&ak&&??YK}3}9$lhxV*h z`OkNnSq5~Z(JjXVSAMCOYO|hyFOlw2FMg%(Wbri6Vvn|l{^{~^?>9+2bS%9(%cJi6 zVve}nJb896CeuC!)(=wU-mA-;9$C(qKAnA|gCOG{pF^27v0DU=#ObPAMcA4-Eeey z%$ukuTN5iU$cTufT)cLqQ+TR}UCs>yo%nrs9_>nb^5n^i6{QPQ}ZI>K`*KN)|6#G=hV?h;YFw*w@ z?BhLA+z)Qvyy?X--DCUzRDI`F_dWY1mD!z`R2-O|u_$#*Ev~;?J%xXZ*nw1LpH7D7 zhwjV~+aJ!x*lF_r*X#9))3khNn{B;ybVuRi55Hco2i4fxU)9X@HRK*I_n#l8w6i1e zlX1?xBS}x||NoYEVVJ(6I6L$EySqPrJnk?2eAZmq&hB26AgH@o!NB73_aEU7J>FcKz10IWwG@R04j2$|AeRQg@sC z>;xHYOie|V8uGnkdzl;eUwKhkTm*{o*$%%S)x7`>bRRhSKl+V&N}%4vZ81~A-&aRp zcB}vMqj2lQWu}SI&wicKUcZ9lgZ2A8!v6Muw_JUhJuyb^+mfY`_0b+R7niPoc6RpT z4-XHA-HzOt#Costx$K?t`?Y4ux7UWHsBuncxXqFAKEIOL*w&a=|BbBwZ{h71Q=L^D znBJ)=m}v-G>A6LC?1;)XzH#Jk%S}1X2@Q)u16*ERPuB=d@I9)+Wmxhciwt!cjQ`jLq`Wk2ZP=%9!OR8sbn>_wUtdjKQC12M}B3z zqU95ZtGq3?;2jUmERQ}tx@i1F?aJzKf6fO=uUOgGrrnxvz;wrPf=<|Cjt>5v5u11< z3>ZLlv(@&~`uk;8hp#_&ppp41+uUcZPa1dpi4@wnCvdTw)}lLmdSYJR*=f8j|Nc6b zO2N}>m&#g~t=N~h`>mOayL+_uAse9;r7Ta*_s3PA-p3&AU)%qG>7MY{76qW`l&jqb zu6}**vcB#53e%?f^W}2`R$YJlIKy+nF@XgPamjg4f6naabDDVF;OO};cXk$gG1ymr z)7W<9r@#k2JA3Bj^`N2Ugw}@ePX4OreGZ3LEIajY31~v|pNfKlgKNDvi-j=fgoaY4 zPY0sSe&k$VH}~v`x9svY0;i|x1`9qAR`c1A;bC@mnti#_qnll3*8`t@O1l{v9@{Ek z_v7Fq*^?2wN;JdPMzOwJo_&2?*lB~wELlQ~!M}wg%hy}|liQa1H)8U#6FVm`fLbJX zU&l57^KD;ku9teba$jL2bG{L%QPRM``XQ@evQ1@&Q0JoB15c+%ePs0a^bCx9u)8}< zd1K$x>?`wZE02_ssK~%tWJ2d@t|rHovwmHapHGRye=pCPT@? zL#7881FR9(Q;?>usCT6Q)@)jCPJ4 z96gR+c}$~Sj z4STF!Of2VKv^YuO(e@wp-G4h>>OYtIm@}jat2}UH+W+^P^uOQt|3B)peityiVV1dW z{=BW3m)YF=<&MrUO#X08I)8?E;Oxq6IX8nq`$S!H-d|ZM%qM5l5xF_-<`-S@)6-&i zmnB|U;P~o2PxrYQj!aX14%D30bySy^o!{TGNO|fVsy8UyGn#( z%$%829vCokJ>XlMAkhf6)8p`1=ZDhQPHL*ZS@sx)<*sJb2LS&$WpfU;dk^AAhg%U?urkBfqOw*6r;=%d!S?ZBXu4^JxZf=U6$ZPNFF@44DkB^T(o*Eu!nD15# z+I2j2`_7zyH#a0YuTGue$+S*r!q;uz{!iPk_w@Pyzmqpl2%GdEYx0`s3ZRLz#EReQ zAHMaaZ!oUh>%4VcrX5qb>F(I6cOhdej{|JOxwdw`={haN?-Y|^-16IfmD;6GlM1yy z8-)GcDdc|n`>snlJ4?L5vLD?TOKPu$~_U1rbo+CTF?*h}npe3di3-!5+7i@P7H z9v90lYL-5D`p5bmOE^3&mlr5J<*%tu+4)gH;dIt-*~PyTmV69I{<-evjaffdUo|>Y zBHzd{-=Zh$r-_rZ^Hkoh_3QO(58Q2^t_vC+^=M$)(Yzqyj2Yu))}1avo$3p3EO>Fg z^Uq$3n+eQ4rkui{pboKRd|bXJ&2##K%z$dqpSlE&*l{+ z{`l}PVqeWpSIL_yAv%#RCE^?RxcAG6rK&jZ4^`)XIm5$2_`<*YpNdr&)s-67Tl_WI zG;vk;hX_uVLuhMtY%8iOlTRHz9S{+daAMEV719E|FSFYfmm6GO_^VG$NW0&0?eWF# zZ$jPTf4a4Nm6P$l->$Gfz=SV*tyX?x+y_QjiQCZ5Ru*#XJ{kN%#0@&0pB`VCFa5N)RUwcjMS z<=kvym#bhfG&B@To~}_Scruk&rRmq^^LE*RZb}#R_x~xHTL1g)_Q!j_-`lLAELL=P zSLxxGmzQrWdg{d^XLDmwPs*D+JB8EF&2ijc_jgTPdfHj#1%LOuGF(5XrY$tzX`W)E zaj0Bl^liJAd4+#Be%Q`FHMLFI_NaSo%SCa&J$VHdWr^xCCe?Ro%|KSNg+!g*km490;DeYGq{jAD$e+FaBk>{+l z-|KJR`D2CLj@6G%z8Vz%d-i)m2FLSt%NTXv9}@M=GrMPI*Q9ElrZHRe?lc>T7q_2; zi8j{jnR9+N@86uox%l}0e^<gTe)EmhUFB!=JtNW}6?vlwsz{qyjqx z;bI_@6<3VdQ>7;FlrFJ@t_J%k-(94J>CkWbA4rgmmkkP7RORnlkb01eaQQ>oOnLj{9Y|bJKJg z?$>^oUAOmJ)YSUxvE_*;C#j01mc6;rxGr|L*6*F-aTSdGwqGW=N}jCv`}O+OXY)4Q zT=Ay-K*(Os491viH=`Kh=b5m&#<5t(ZY9?VSNI!AkEltWnv!*sj0t+N*_+NP>h`}XAen3rL$75(px!gJlU_|=3q`$ue1 z7JQ^^?dRHFwf9OhdnH4nt}SSgpChv`Qu>`+R@|hmN@t3h8RLH>=Uje1l|k>n+{t^^ zRt!gZ&u-=f*MJ=itOe}`8v`m?d=?*YWw|b$)b&Ak|A$VAb!I1YY?@ox>K7Q?*t?!# znz>yZsQ(blQnT6l;)RYseV_6dyzz7SGp&D{-Nasf)nCzz{C9Brzx{RdL5}Bx8X*B| zu}I~U(Hhq#tyy?P&GkshM$d`7R%a*Y@M+aOo#b+7o_+nkO2L}nZ@2Gwx9fGtzhHmc z)|Z!;M^BlU(0XyP`)bo$kB)X<+)^3g?=P|0y5D%uy0=T0|Ib%VDLx~<-DZ(a(3`2H z&t_>e{dnrj_PA|Yx%4)j`wt%fUFRBmzvN&<%%vA@HI*5hb#~u2o>lB-J^jV*z2?Fg zos5*?fX`QSx9PVX`|sWPacZJ)^1b-nzwdbCdvgEmiE8g=$os{XmYaI05_I{tEhE>1 z(DQqaeOs~kKx(6&bkZp{TcL$YN;^0n6is;_X*ttl)o%8gU;7#F-I3XR?4LtHl1hOc zhe)-u%}U9nOScqlw)AzG2wse~6Z*N<{QksNbMpt5^5z}2pzY&R8rE!&t9q$A$D)vF zvbuj*{vWATXb4i!aPrA-M(L0%Xefj#<0)SV6-Xi>M|%> zr6^X`(IDo}-Szpy6hX__kQ~ivmjHKG{O1-gPIB4@C8ENBAt9|aUv z|95t%J2|P}h4Unv-!1+am;BBULB^Oqp+Xh8nUSt-26D9mmVxO#SJ_)X)O&pDVevSS z$<%h&v%pJcQP3@gE8Ew7`aZ3<%;n3qN84TIu&}Z^F50g;Y2x}gE8m%-psV8r8+oP8 z6f7+zugBNh?%AVLu5^EUzI@fUH!H8DfKHMNb^O+Ae(%8E@Ap=xmD zp^};I?(=3ZnzZ#{bpGC@_jU+oPrs#A)nUKKCS$^H#f!g<)6a!WoUm+7&fIQ>b=R+N zT6T?T$3ktxP^Lyd!|4zDzD))-iauR0Tf!~)^N5kF*|{kl$s1)SK6x*k;i9xYKEWrloyu=-Y8#*0@O?tvy_=T1V_x^USL-+(z91Hnlk;e&d209Gw7I*UbusAOd+})E zzlZn4H-7x`-S4YsP+x(Tq)IoD&*WgBBQh^?h9< zbU|rH+!Ak<9UXU-nz9vR`jncyS-NA+>z;8-n;U$h+rz0YL+WV3>XezF`oj^lfy4N< zf`KX9L&;u&7t%Z*zltr4wo|I>=FpRm%sFiuS@3%9^PMlYDV*XFQfQ2FbGVkWNo;la z`gvDps;j9zS~fc`D0^GK-7k){QCl}@q-#yG|NrN6#gmEdI;V7EcZrnWuibvC{`uT; zFXi*T+^3Gcf40lH_m z6fU}cZByMfrXSTWgL9{(mAP`FE?Ju;Oy?zQ23BQ_o0O-N?yM7@oL%@_Vv`By1c#%b zqlYByuWfU3aBqAaZmKF2Blh~1kn53rSysKDMX^dzPl6*t8um2sh_IwC&``MjH7@t9{UH!g z^3iQMyOg_gJNIoC6O80tS$G1}=`M)lJQ0<5YKrD5Cf7x?pT3X%yd~)<7mvJMOlm!J zvhHkm!nZd!PhE1=>sVozc18j+ncTq0eC7W8N3Hij)5k`W6lYI7Ws#Jeyf)50ex|3V zr-W&i2xu7YlHv&qaownviOTNNo<&+4iXTwj_5W{A=Cv-P^Smr7NzqDcp6Bo5UvAuQ zaplp6Z~hCs&!kr5zE+X3*}m3CcH=_K=@;leL7DPTc z?H)J%s^HUKE85lW&-6GT)yOIQFY?a$+2-e?1fPX6Geyn0=Ct)?;2Lk2Bhy+pbKFwO z$^K|{slf!eap#3PbYE{OVaYBdH+7o|F6*)`|sWTJ(b3{w&&-^xvaHxozmak?fn1W zU)^=#Wo;EZKxYqT{ayWg*V}Ekr+m=g7JF@y*nw18UGNzIXKGGG1wIR%(D0+)zWSuQ zhMwlJ$E))L_V-V@z9h%Ua_#pb(YrRqe>J%C%QiO5sQDP=!gA!n&9BO*x9V!Hw|8In z`oyzCkC=1}%pUlhv3uuX+I~8N@yJrs` zwtxI~u06tHZl)7_@4)=bYMzPz3}-)`zxxgMQ+B)3v$-eqPHc$m6OU0ZDF6F&so;ZU z`6;{ZDP@E1MrkMp4fGxFUOHD{fopc;hSSUJfByWLwQ#*_zHUc02+qL_vu|TnRYA-vUHafoHKL&DzgJ6OQw5E-*1tQ?mf85 z{{HcD_i52`|LJoKG#k@PL)w;MGa^-;rzn2 z?;N?WZOXVlQF~Lx^!|wX9WU1QFJs!l@hm8g`@y$G2eVBR{O&a!038qX`Cj$=y_bsT zg~&54Q`)*s+faPJ?_yA=OKFGQ^u?K1w&%4!D*yV^@_`3v`s9EpQ~J3%fp$+GW$#Nk z$n;g<%AUEB#%T$^zP!|`{`%_bjxU$I&vrY=t=RwT)oQKyMVEhneVrbdc6OHOj?d?; zFYYSM2A$*hs#m$&$?aIDu)2V#=u){Sr}q0>d75mmShoM?v)NYOwi~AMy1OozUcb9! z5yLvqkbfIp@ABQFZCfZVufKl#*t1Jxr=xbhN-Si(5o+|60xm%anF} zIm}Uzy=7+lFRjnt?_2NRZNoV)o+)aD@oziPi_ow2k+eDXn+jDMST6CvFQ2E2V z-S1b`L@Vw5Qv81J_f;9CKfimM{qE`P)cot>?w1Z_=3|{@qG`Exy@SLU%zV6305-dJ*+XY=a+HL|xW z)uP->4F8-=PprN)t6uNJnOv54T{FY88kNLCtLDDj#&<&iR60*_ZeXggY^bQH!b}#M}3~R zL-)O^mS4NY*18_KCLb&q!~JkmgV|q?$eqhxzy4P8+Xhs`F>p?3P-dz7|F^uuVXbS% zXRA#!m*%{@>?IQ?YD}bezi1}qbP6Inf z>e&+C*=7;DN-}5m>gQfHJR85GKvAylN1~{PQoc_Yd&NqId6u%r?sfU(sV!LCcJx%r zx3-IA(fnU{j|K0&AA0Xv_jdoSKV*;Y*|f$N>xxt4)8@?l|IX$&t~uH+dGX>&(CEyK z+Oo-&6TN=FTg7k>$02mruIy0}y}9oF*?UuTBlhI3I{WEYli0PR`Y|@yY51I3&H4Bzim2PeBPJDR1NFI^f@r86zt<%vV6I+p`oCsr{^n; zYwDk$oef^J?dJ`&gX(y;|Mr;OztF6%pQGp4ls*xqc; zFFyIV)hF{JYd>Dw#=mk!uHc&M^MAReTi5Q{c-7YJ$rpz=w!fvPy`LBD=2O1QP+V(= zRzvp1f6J5QAHH_Zx4ybLXN~3J^VJ_r%W!1;bt2H`#9#zxnr# zs{Be`>-2TRQrCK#m&;A`@&GGjcEt>3vdzH3o0T7)n*sXYgpNlt5tB@YPy`y zYm58J*%KHzL)e+3{!H8Ws`&EawVz)ZnD>D<>w7dn&$M&ib1c=^=kk?pxzRrJY$P{r z+}@|RJ+1Lm;xl87k6K1+PZgUBeE9n9_#AKh(AWJZ_w%^L1@@f{lYjS5Nvb`zVS0VN zPvo*2IvGWdkj0=&c3i!Tz8G|Cf@<0>EtJKe3yD|^>N9Igl>*l(!K=AnkJsF0+4Hb_ zqTyyq#yn}k^&o#Sf(AbrX2x2$H!y84-V-s4B}K zbmI2hIK;olxt(u@Q7Tv7&ZlCjA5*QYgBfjfb$u%v>Zg3v>Dt5P)BW*|FN@r(Z34cB zZfnfu-FSP$G|--<#kOLa`B@)Vl$m&%yrII{&9s7$|IZHdR?i@~cJq*Y`H?eA7~WYq zT>~95$qd@u&~QQxvY`e+9uR9>?B1VrZjPm7@{w2u20>@gSpcWlWA^Ov4W6zSJLQ9I z!Gi;rw$Jlfe|o1WoqpKc2e?0%e>E`^%wH7PBR(#2{ zV7M>y&%Dm&*TSnt+l;na{O;b!`Q$mf_-(B;)BjB!=XPxWtP^qD<8l_O^vpuVvjwh~ zXHUJ_{q51Vz$;nXjF~Dn#7jIR2|T!_QLpzV77 zt`G)(4vqln8{KmiArl*-prqhB>o5HHKM3QB71O57)WdDOx@-LOcD+#YnQJA=%*LZJ zy%Toqot^!~{ZILJ%o3mQBEOT#erlxzx7q!CLuvcNYEM_%X$!m%?n%FPf2G;NEgC86 z%8)%Sj%kik^Wvy>c5&I2ec1N2=#1W(ei6xsyHgSiB(8}kzXkx ztJ~39mVcL_3>TQdW=XOAiMEt6qsm$Q68dcjBQxuai7( zyDN&REqh&`8BOGB#%aI$gBkRvi*SPq3D6Fpsh-n8Cy#|#GG1miVEv$V{5&71wGH0V zzed2KJ{;U+yLGKxQt~BNOcs0{ggf2uWQ0)h&`@U6%fdOdAxdG zpS@7s!Q;QhL!J1aFZYZzzJBd*QQocuog@EJ?=36zIO}a?Z@xh6>n;5Ua$CCIB`7w` z{&z2|yP#l|iUZU0P6pNwpIPSJg|>ttTk{}IVjM@(Szcl}9n#Q%vXqQ#%aK*jXFF_& zmuNfce;}XJUXtIm&hGxFl$c!xPpsNI2SYX<}Chp3XxmZ6}jF371;AhPk5y8}x>CzHO0 zhC^A|HlYJY*;EY;1*5z7AH6xj`D%WUj3LPBY614^d8=6YUG`Re6%Z6$_{N`!nOU>p z`itG?PEKri4%$Jw^Lv{zIOvcq$Y9*FVS|9BrKN|z|M8v0&nHZo(h{tBe8b**>TV(x zhMp5asli>~gVsm4^J@|_Le|d|I`I0+%EfQ|nV6Wau+4bHcn-YAG!(RwD)drXJm|bF zzXM-Ft$wCX1Yi6IiP|hC#<$G#X3y^K>FEiQ7P-k&tY3HVX2)OC#94EgIVU*qDhSNJ zDL0pCUC_PzRm?sc?LU709L%)i<@+4dgk5u(KqopWI56&!x}(Ae>dO=|E$i0gdeB## z>fyLv1GU#;Qj@uvp-`qX@s@XY%p2zF?rpeAvy0fIW zSZj*dEQgm$yY)7`wfa6m`oz>HY(@M#+tTiais?;p3f8dPQ6?o5J?-gWfsMxswfEisX}M0Xe)89UWxpr; zEqDI^Uh;2T!3~ZnjtvZZc+wj!6&f58nOGjQ25wSj5pZB&WD(Gq*uYc}-Y738ckJK4 zx*c3Gj5}5xjOY8`G5vadZ^i|-buP>MRw@`sGaYujbLW?4UP<23U%!4;L`FwV-|Fp;12LhD>!Wfx8Y?>$etd)UjzKR3GKhK;!oQx1#v=km_eW=?npQ&HadY|i{ zXvhAH|6=Rc=LTB|y4tXbCf02glnAq6frAwCvd@WynmFw30 zE6;k06*xlt8W{EzE$`t}ZaC1+!g0WU$>CxSNH7JIGH!PaRu`6avZ`9gWx7>=zD~y5FX?PqZdG@U?L3{a+tXm+hY0 zdjI9$sGIz~KNTAsYFRiAye`(W6oPmG5?U*EHr!sYLVQQm!Z&Zm+3i+=AJz)Qbl3d|died%pFcZ(MISVO z_{Q!$IYBcHCF-SOpDn2!Fsp2PW}HM>7W*X|b=7eD^6JnC=k zyXK@a4uybpMy3x|^JDvXp-}?1GfVwJ+ykzISGimjcW50Ht*H(a-LY+Dct!aR>1yeE zxA&K>X>csp^?JKcKt_P+xoW|kp4PYe$#3VsPg+@7C{wihQuM>^VqgD+{aAZVeHDnh zcR!?l{&mhr#R3iwR2Z2)OisDc2TsK>VoHDm;|}rA#^s7Tn(|v8NNru=zr(3u?=t&0 zNBjfhXPU=*&n>@PueBhbvVCw z6_dYn!B;+c{p0Q*-yN!xiLH=sZjIxy+AC4^tME>p_~B~_Uzle#ORS38V;PjC68gAT zYqeoHOInP=+ZGdiIO-(7t9m^c$O z3B1yK@Zi?e2VC1lOSC_fx$e+9Alo1>C)Xwx-lWY^F7U4Qr_`zBdmom({dxVP1S5-` z-iHSVo8P$d%iEo~)*V-PV($I*%;2427vFlt{9G$w z!@FO!VE6a!&iS@s2J5bC>RhkhJKyu%{92IR>7oh@_NyhAoBYIW?Um(S zCl$d-qrst(Y1g}tj|A>G?l@e~dOxoG$6K$0?M{{P)*|axa@hpF;cA?7U~l#JO}hQ^ z_UEpt?_KnJ*`G%Y*15HhUgd9h2ozZN)#S&u*IR$Yoq4>Zzws}~%H<#{*NUunW@z-{ zSRx*MXHDp%DTbcVf<(jPz;(OK8E+eIi{6*p8@pKh_+LBrm-pto`y2niaa&T*;lUfG z^7r@7-uv?8Vx%yz;x3Htz;~synh<80d`!C2Ft{egl z;tNmM2{y@6L*(cz@ z(5S@9aiIH3N<4>xK`CQ=L#EQ58NPo@44{??%$U@$zS*RzP-33^<>!0XU95k(YrCOR zgM%THw6yfizPLRV6W97z9sTYr^|_n5Aou$&_x#I${_OKSKNTEw^R*oq{_SkqBM&nV zTqM9rkLqbHNV!%OaBZ`}}LUt$&x4?r#@lWD!`=#K2e~dQPa= z1ynJDEd~*4Iu0*v6BEk%9avc|9oWruIkgK^Ew5zZIN*QuNH@43#YU`Bb6|+uYJ99l z5M&BA6@ID?40T#+aV@aY5mJ{y5++D2z_jr{0|UeV|C!T2*)u?oR(vy;vnJ%~-GB3M zs;jhYSkJUcQ$ayv#g4~qzpq`(ivC)<{hGa=>#Kbqw6-Fuk*LEo~z2jb_ z&?P1)C^(^kNs&8JZS%kL@7sj))F!7G&GdYBukzgS*E>(|d_M29t@E>c>2=>N`(}70 zD1jQ(vjqw)WWL%yoWH$FBnEu?3%q3|bi#?Lrtah+$Hd7V4Jn?Ke#A_fS~Q`GL)7dHeji;9Wp)_jn6F_}sm;t-Dv1 zQD}n(=q3VL3$xiQ9twtx_Z-xo9oh@;IYAgV-5bv9-!xat5#*fE@E3H}$o(0c+*KxA zW~pGFdHL}9bI_B*!AD8ec@7dc=d zF2_nbd}h&TFhAe!j^9oluLhDN&#I*`Qj<4@8?#^hqp1{NL92BsaV?Y+Hf zj6xY*4EYMPl@9+!^V&@}P-K}Y+0+Y4YPl{qheA6j zskO7uk3~&t-Jqm)l&$A1D5x)}a9I?jbK#l(GE4$wH|J-r!_#G)08#(`4XXk|hf33nj9E1(* zAD^=OTo(NLR}AO?_)->)hR>57m{bz9nVv~A*m2y*%D5?saODPD#@hc{J9QQoh|JbX zx)CCG`1@4DH!IjqF&l=kHO?{+;5aVOXSKk{Wl=**y#d>iZ}P%r2J9RjG1FVhcFt*t zUb|zi;g73c^Q1LCFvb4i`*Nmyfqx@&bJv_3tBoHNTE9M8__?@{??IBd=K@zy1i2gm z<>)Y_gnx^!^9zJ!+22OaZWF3mwmi_S49yjJpWl`6bDG3HJyxfQcOD;f!IwtctGixLG3fqJT-MP2TE2<7W4r7nSEbRkZVSNKXLmR82o>ut8A8f$61^ z1EWo>N+dthScbzgj*N%aQoW19S}W|or;2o6*M1m%bA|z^zwThgQ8A_V6sNQ7kBb`O zHD03A(^EfxF0xv8(8}Y07AT>lF1g_+q|ms8W5)gqPY*UB_0v5XST?mb?6Vhr7(MN) z_2G&XSDt9C?>E@A{zGgaz~Z?`;%UnIOZ+_26V- zk|rWT;l-fG%!bH6eJ)dumlsNW4)eHTe)Rp6g`Zr&4yc%_dn#0T?kVLvvGSaGd~)8a zgv68>xk0x^JUp0u*%LXDgV~$rH{ATBBE$E1ZT{gOpS8q)97_7uFB|Q_RHnEo)yw255+D8plTqq_BhKu_Yb~aev~rS+=y>mj?gwo2c%u%qZkB zx1r;~Wi`tlkAmEGCqbLic{Z1=Kw*}?=0*^v=ePws#P{#lNCa!0S8 zDya{iYA8`?tYbAiV{?jg|Mqiyu8R)N^f+)1RN_{a+%(5)I;i-3-?&IeC>3pXK!}Q(d2#5x3rSu_!LHn>rxmQw=1{Zmy>m#uNUXc zGCe)>^XC#VlZn0xjaxv;-)zd74i*muX^t8DH)uAbAMV(|i!gXXHOmvbkR$H{G+G=K z8kXBJ+ueG0-d5lZXf!KK*kVmR%XFWDgO68!W?nAp6~-?veVvU-q zxgJ0(bXaUYkz(9vCwM=9qutHGu-0fhEfa~qu?Dlmvka;cr>XR8bQ=~!WLFyd1d18#Re|4{^o0^(NY|-lxHy3**#`0|%i^l;Y zmUB1NJA~aUBw6!j3MM-TD>^XRh-TTevv??Ea|SS+zqK&&FuXGNIPi^S%lGXr_1#}R zLqkQM)jT@9J+yM$duwYCCY1zVCgrX}GYoGsuD{N`{dVq;hS~F1uf3_gZ5E4^&;o{g z6WF(VG_dIS9r)ww%U{sVuQnU0p(UoQ@O^$#{i4PDy-#ZuyzZLUab%9zYtGr;Oezz! zS&p!!p1ZhJTR46#yc=g8i?fV@?Dx_AWimFU7WzlFzKO3+?5fP`5B;&vo zrnW$RzsT>&U0qyzzUE4N-u`V{(exW-LJEyC3Lh>%47EsGA#OMS{0FA#=Bv-odX^zNs$^I5FeTrZsDL^w$(WK72`T=6;LyVEb|v43^fL))}4Z z+_PuTuagFv4ci$t{Lfl3i5%FR=W#;;!#u{-f&z>&x96XDGF_>`{&T3^zE$tCwyulV zw71Uu!s5xow^uJ-!5h7aJLgLtlb!el%K--mb;=o6XBci>5#pA#{78eHo_QNaIt~LF$-`m^!5 zu3tC%&Cv<#{9?8nG$br}dhF|qCkJLOo|3ih((KpDKhn~-Je_j#{DJYHr|o@aX?q z^#dY|fBIjuC5B(QFE{=C&1?}($DjXJ+9@~vo>x9uc;j5J=6Tg!@`_8XPb)i@i%zWh z>CG6zdinPsg)ljB*#}1hn`1dNdB1C`maGq+_>5I9A;Vn#TbSVMqN7J&AFB8?`Ad6C zHshWhN^RBm*XMqaEL^qLMzlBX0c(}!>lvNQb0jX`W}Tk&`AoXvng3eG_mqv7|1en8 zzS)&AMymStL_3}kd#RZ18^m^JUA8vkO(rk|G@e$kGEB`E&#PH`ZNnLlXHLSub{RY_(4QA8YQ*xQIK<>fuvbIeik258%RYbq z{@{sVUe4N{hx@)a--x+raOrn~=+l~~D*8wE|M}SOTv1zFwSGNoL8r<7#6o9An?U>H z+IpAyKTFwj-PxegkR&QN?_9|%ySzU&bAtYdW@$3ToOc&4KaGsl~^)O=jRUQA+sNw2a{%2Y)&8giA_C z!#sw~%a*Y{efqSG!S4C(T0y7Fa%Uv7yv~K(4tC8uZ@KcE_OEH?QDPOQGhVh=6*|<5 zud7pC;B=s|J;daP+f#E}=gt=|y@c-8p5JJveCp3ER>n6W)6_dBxwLn4#4Xfk}^gGG%AgF9w zlEsD-o&T3hOw(9jwCs$)2P>r=zZcAHW6%>$tg?}FYe;7^&3>!EByzx-!$u!eOiMC} z956mSbt|Hk$iQjh@4)@w>eZ`l4Et(-mz}w1Ar<&W@b9ekjqcIj4<0ewSk$eM=C2ho zsaZYQGTvs#^Lek1hi4r+Zhe21@Sc3zQ|*7NEDfDqJ@W%DUV7;vzT(CE$su31SJeCu zcAaHDeV)Z({+TkLpIy)9?Vj@N)V604T1-3oF4g(E#@b(OTk+aP^u^yVGb??~%p)29 zaHt-OU+PjR_}Dm6_;Lz!@tcOzf=AbQTJtd4*xTRV;4rb%;OzP9mTfI}w@QM}zLX9q zYiB8%+whZnmf(Y(|1uN)Bie5Z{&KV&PCU%y%JN~;CL;-^AImms9oUg~>F~E~&%7AT z`;QMSUjAE<=Xk-)Pw)BES4Ph_ywqEKyuu(qQXoHn1^0G4C)dRIrHeL**U26Wo^Pff zrtbYc?PEopXXi6h{DqKet;L_?!u7<#)l0NvVOo+0jJgz;Vac zfNh)9#QfV196S~D_?GjYh&Y*pwce9v{|Rz)(g{v^bmny2D#wKFPq+E+ykb)pHBabb z@(!oYmqwOp(U;dxJ=+!Yym9G4v9;FkZ11%nJoPMj+N>kRPru7*H*}lts#s#b>c|iC z%NobB=S^R=;%{>3mAXw#JLZ({O|ta7x%^C+&h)zJc6Pgb&s?3a$(nfW$a`MK7^MS8 znOd1F>?O;T92oETEk3yqIV(ZfB^->p4dJ1oSKS-V-?Z81d}|W7xaj?ykLT;6TyyrX zQrbS*GpSgqO#S_;6YqCwG2NL{Qu|hSQ&s4MFPGDwIw~!5t24DydnEu zuvZ5zyJg^8nfF?rU%F9}p{fd5P7E>CV%li_A90Dy>8fhfO@2w9_a#km2a=N6}NG_Wsh*ZFv25+9&-h{Y*)cXDT$zB=RS={O%fr5{TK^I|saXa655d(1vub>5wi$Fn{i zmtX0rFhBJ8{NSfXpfiTQIPmjHJ#Y$$jI#d{C~(?r`Tg5b+z+m1Jv#BDqx|LU8|zN= zbN-B55TC9de&pz*DlWu5|WZG#b>MJHhvQfxCf-cGdz(CZ&e!*RDxTPUvDiwJrDdhYQn~XIVD42dQqz_g8;s zbF2OAsaubKxt_e)cBj6HpLv%BZ~p7|+QQnq_j1cr1o%$U6ufXbM5N+T(8_JQ^6ai8 zx9uo=Ct=Ptg=61dt2CcCJJzWfmnlav++U%Slot?Ycve;L(DCo3kD5NpiTsi|#xg^Q zciy`xs~GCu7iKNGurBM){Z&U_@!Z_JWcT{8)$iYN9{nn&@rdypN5Se|h1<75{p7P3 z9~#2b1&q-+NAbhY=kq~3st@mcKCe6P?yjw??x!)M(vl2m6i%G)!n_Q;??9M z6Q2g{>leDd-s%K*rnigcYwl}pAJ^+UtvDOAU}xdlna6)j=wVp*d&~4+6CbS8jC-j9rf>|LMY>(Y09+{mv48n+i^TP%;R#}{Nma< zw|p6GmbY0yxAJDRdEPchR*kV*sN=2KzhbNJcP6+g2|i`N@k34LqFr@R;(8B-?;I8~ zj~?t)pSgW@)cxRD*^pv1Oih5RmFNDgWxova@7e76a>;wg$79lkM+DtXw61CSHQ8p{ zGSTujocN&rx<>Y-wVq!zLm6wHiO9Qs zDR@-=o}IUibJl^_Tonh#J-rN-Rbj#l80K9)eE8Tz22K-q2kx7bP786lUo3y3BEl=x z++^om`0=W7z&f>^D?`4oHGQ>DSD;T(rqm&|V?_er#ACuQ9-ob!bR0AoGoc~lo>Fyp z{!*^6AYqlG55l!8Uny>OsEjK6Xnn!>exgepuU1jg`g0BLXBRDDEu5R4c>RV7Bu%+G ztZ|v97u%KGZ!7lm=TFX=b85fctoZlydExcg@`ux+^D;x+bUb&Jzt6j4+$V1@H^1hS z=VGI$PoKWHyfZZ*y7p*qd-c`7EzHyRi!}U5e|;siGN4-M_5EWm)0T27 zEvUDN`q|ZAw%A3sI?0t|z6Rr-y+Ny(encwYwMg%AcRk(yarxcAw}%4voSyBadg`33 z((P!km0S-tPc`+87j>=Q{Oh1{@H9i+t#|JoPu@{|NGxi>jp_--yUIll6j%MYSt+zZ zjZwG4J1$iE=1%cR24&nYr*w4*%Q; zYCJdgDSfz=z5e0N^!Z1ZPLJ#IoUCS=YG*e(n95X!EV`vY6^KFTHT#5U(rW z9(!%5`#bxJt=GBm!29Z}Pnw74^(LHaU&xlU>u1%>DF0A~>x=$&XCL};`}s*j?``}m zxgPk>dugur=FikmY4_LJ+?sjG`@q6~L4WmGKYVdY&5d+zV5$h?WtVnlQn?_*xcS50 z#adZs&ir#UbbbG9#zzUx2@aoGTzKqD8kp1jix#`vwSIV(`B4K@JG@a8;9>iC|NlSn zBJ{?yc5RRaNGt)8?gYwv3*< z?A+Dg-`}s!i3(Bqx!mi|zrCB@dZ<}N&7Zx_CY13{RcfP=XXFB#ABh6qAsYg%g0+}- zL>&L)+PnS4z32Af%?VYrr<{8ixlM1na@X!-`Kc^BQr#!;<|D=p9omI29NO-}aCMcy z`{(<5l0H_c)~J{NzxD8KIx{GT&TrUruI5kxd!5NC*7;Sd#1d76A9;M6oWKog{4zvr zPU{t%T6--rJ#qivZ{8g)Z#?c*Jm!@tzf%~or=oDNQPKZrpMNUsyOU(QDfxJxTWZSj z6@iOCJe?l@Xhw41!Po2e&)d-B(%HSY)1|Y!)^po37p5P|(Z>{9mZ|fdxwbE2VQl&8 zld|nAIaSm5N*_7(URZlk^VF|5_a6Vv16oAxXXbh4U$gq}i{6nd{#FI*?$ge{+uUB{ zfAIa4(%q{)zW(37Kx}uGwOW#;C}WMoLSf_nkBcTBZ_F0mTv9!Cxl%)a@Vxitn&-cN zJ(d3Vm|D%X$+FSYynPP5?s)L+<9%uHg58gYmz@qR_n-5F=bb8JHfZv0?Z>2SPle?i zCz>a#2<}Ub+;80D9(H8I(n;$lKD)x+I9Ii?OvS+AXLG#e#n!#McPUJM6~vP7{$q~_ zxOJU#f1j*v)t8J<@2}tg_s#swJX>k;m;%OiyI!qoxh^6t*m-Okr($>a)z@X5lU!1# zPn{}x@uqe8JD&iiwyP(E{cRFm(*M6J-|rpac24y6#KnD9t32!9x2UH*^xMp!H}A2L zSie}mn{G|yji}e=o;6#Ku1fwFo@ldJZK}@{y;%`WTL$~nqx39y(^fy-Nzq>Z)z7*pg zho9B`?_VD|f2>L&?)llmTwm!&r#MXjDXeZRf8|L~NxC-0uRmy#SQVy?3;x1{!` zUHuaid#jb58`sa0H&9!c6S88a?B1&_-TbRM#kE7ed=u4Y7szxK_+fa;{*Td9<$aD@ zbDc9kUA(WqmUCgn^?9F?e_IrO7POnP+AQW<(kH<;yP`qs*JqS!Rax1sv^?1BdH8e4 zmhC@Qc|X~;Bz3m(jJ+nt+jLiQJg~a;U(A&8*>%<4-+|Y&{_WRg*|np2hGCxjUe>PT zj~SH@+;*DsJPm1(w_(;y+ccd|2UPfl@@u7!%@5xhHD}}87ob+9LM+RX;4hE<%34(A zg96v}L&?QdTNMYUk1Q%ozyAGxe|+Wgd08%L$K|KzZse%^eAfKp?(+Pa?{~{NYqzuW z%k}h1n@iPiU$uHQFZZM$yN<~JUb%eUp-%OA9jx49N94`6slNLCbb9a;gX&_#b4$ld6#uGo$*JkHfyz{|&D8@0&X{^mr4)wy<}CeM>(nv+XRn>KIo8 z?$X?jQ1D;I2pK{W{$4L&;_@)%+(e`2Js%&enYecQzMZe%M%8T0?LRPEk8#h=veVw% zQ@s-OSN0_qSMsc{lQnDk{p?)k>5bR89@tJb&@VGF`uNl&_2j1Cst>oBG(J~TxNteR zIs5puYupdw)^GnOXW>}*PKog}D0A;rOm6l(aF6BD(f6sdohyC>o9CCjv3>JKmNP|_ zv2w#avy+J~P1)8WT@NBHLvlnoOC%TtZSL28k3FouJA8c{=k1Rl4)Z_0zW?7=MT^ra zy`7VOM6Hk6c}YN8r6?lKvEN@<-S5b8`TCqav-Ik2Jz6h2|HwU;HEwm^HqTd`QxY^C zwDJ1a_xJag_;%?X-%|=&8EtR(g5yKI2>%iOjqld3?pQimtzd@G&h=qR+dH=T*Jt@> z_gvgNXGU3v`BIJ#OB~Y*9dqlA@+$@M>u2A1ZQDINf95jYFX4|){Vlu7{XmFu-TQjo z%baFYSY|0YFz%7GuyJ8hxv&r9-Fz>n8C$Qo-k$tTmMyNA#p8g>fyX@$uN{_^xK(@J znZdO{SYh3yhF(q&m9NeGb{&WLta;MsRi+)9*Y&skM#Gnvm)T!mU+*3l7q_@5>drLg z=xsTU_xIIGPu=<95ckLD_Wvc7-Fgm)$Ja1!-(C1*MZ`ppC)-?7{pZ>243xcd@gvI_ zrX4>PceDTae01+d!&?<+toLpGmeCkd<{?rs@qVLr{5nV9*E#!i86U1Vs3&80qj#p3 zmCf}@K0$WxL-rf4yS_r>zsF&P|1pOrUi|lNc6=MRJ8O@8!spMQd$;#!G^C4IYUhLQ zHn=Tmq3_P5a=`>LDJ3U|n3SrNhfPZTV>P@}Iz_lHf3B?8x{8}+3Po|VSZ?_^97z&f z`c%;_FltMN;4DDjTdv5O4oIO6%R205%SQx-z#T29TRXktw zi1@?3x;N!loBW!vH)e;MtH)`VsYZ!Q=T7=D#>vnz$ zs;`I?c%PH-rQGODkZy6n#j-i3N?9Ba=3135Vwk7pF>m@-&|Jpb3mXzuCPcHi{1H2! zU&*j$?b@UE`#TA*eEtdnk|aJx(4y&JnuHc0SJ-g@{*+f#VOaemud+DzweIvcj!2w&vr}V_SEX zylmo?He0Y_#fre0S67EW{=WbJ-iPP^exCn-&&9%bCv=(8829YibEktLe%ICBr3a7A zUw=nw^3QvI_Gh{$o?2aft10RY*DZCQiky6PwJ*gF^XIn5|LIL}t8jI$;j?`c3!lsQz3;$KBWa~{aya>u{GZak~$#k z{aK)IWx3Q$a5>1$u|wAI&$ky6ze+wHdRip8Z|4@LlTD3@!aA-CmMuH>VXdD?uCA=0 z;m=*K*BKvq@%!Cw{lt)OqTw-(?((%)ew6P>J1fN_ZMH_R`Ldt&*%=1{`D(u1OqZ}O z%UNvHZ~IN6?&s-vr#GJW>i_?3sk`;5`Olx9pL>m*+jufV*vlt4*f8##epT>+QiC_k zk6H3eGvaF#Y6A0?&vXb+Z7lj9B5(IYuSq@Z#&PbLXUZq9ddlldygzvH)UT(%Tp5#_ z&Y9GwM8wK8aE&B53 z^ZCci?f+`B@yT?go}RWa`}(@xuDRv+Du4XG|3B|d{H7F7r}^2s4X5|--BWVG zy{~?Lx$OUO_WeI>Mww6k|9k)6%xYbEhi|LR6XS^vyErl!W8S5^GU#h+9^}7W_CQ~B zvLI{j-%Ce#uBoGF*crbAIYhU9>txDbwZH6M{DaaJCw*-z%9B~V&7MZQsAlI` zb6W7|`u4TX>*w9=?VR>>ci^){3mEP(HXrm(>tSI1(6cc0sEPyANg;vP!j-8ZN^8re zgg%_s&n4Jr^vgQ3P6w3n?U1HQjTiZ6wqCNkQ#rLyRbldvyJfd?U)$UX(LVf&-{n6B|b+{-Ot_u_PS zv)=2V+glseIKOmK>#++18eoO3lB%sac;k(Mp(7PxQ`bS?9GH~lE_W8Ko+ zZ!g(?Yi98{kcKp0ySZ>{cKMOpQF1Blyi!v(bc1f>iA* z{B(esU*dOs?blGBxmH^PXJ)V8E4DfPyxF(?-|tmhyxDMAYU;!l@4fYQvK(&bH(#`8 z@4X|RYdWrPl}T~2@ZV%7K;j$m+^E- zGVSn@d;9(O>t@>`7RKE(nOQU%o*z)y+^sTUEo8d*^y)>Sp`kNAck?LG_LwJmyb;)jxkwzjcalZ;J;f!^LP zm%!0c)Y@?8_N(nuPxqC9hMzd%{v7t(BQVoRdP>Ysg$?sn5)_&ITnZ}p>t9$O9-h1@ zeWU$FCX2uObxkJZWOaik!(i(eQ-_GR=7h{{Q!Vf8n>A>E^3mUyIIP`+a|&*B#JVV1>V4E?=B}eqP1vwcC64 z1iEG&cbBhCaY_ld?^8;jcLBi)*9JAKz2a*FVqk<$-P9wmym5Q?(em9;}@7xKhP|DOXM5(RLp( z6{Z~(RqO6W^}1jCmRY=a4S0Og$n!xP^XiTH`Sr~jAAI8aOQYKxe7g*das6Y3$7RLik91hg zHB4^%X!ZGS`Tg9QIBC&Gd_oG1TZAqglzqgkcdgDyaG6VE_w;|yCS~n;XT=xDu721@ zyClqwNo9f&W4caL@$R)p7R1fvW!zKz{G8jD?Z+P*z6+ILoM#j@Z$>Mq?t8dkgC}Uw zz!v{SH|C#kaRfDr|E%3xy!7`Qbs)D+F$R}Vlw>{CwIw_jAg zpz^gd%ZEvmgmhNRweaq{D0|%^ZYF3cnh%TH&V5HW)_U=NjDDAU=o;5FD8`<-7L$#tUu=OW~sQoW0|s(+r5^b z51RQ4emrb{u5_;F`_Iki?V39RI@Qvqbe~?*?JifjMU zHO(aZn$DkR=KC{4p3O|3ckt))`T1|AbG+@=-?t-BQt)8Pewz@xpHC(qV@_Y}l-17h zF38O;8MMUanv%lz9nZ{VR8Ke`c`s{SmT>%>(VoXvQlamT*)NJ*Jv~R4b3(&aPLX3; z1!>dIU)Y^q{NQ)U}^A%hzVySuM`)b}#zc&HNk`E%bo&M1aSH zJJa!Pu1RLl?k@-V#3yQae>^0Xe|L8NKFfv6w^fAu&$GF?LACPL%H;(QTgA;@{hI&p zi+W7aN!4TjK-c#er=QCSaXaSDQnK?|6DxPYpO44QwUqB?2TkXQHxg7)UM{8hh)?JQ zC(}nqxunnPaSxQ5PH$St6fv*9X_Hf+IXGlgE-x zQSG~d?_SaCjAx&!dMfC$Xf!NmRn}Hy6w1(LthC^0KYZ-ooPn=K3|h-R|+xPbEIvuTvKh{ zsL09~Q=M=^;K1Ly{U0Yx=(?e4CK(=EIyI3|HI!f4j7K|sok^ViR_mP37LoWng34|x zZ&tbF%r879Ie}C0|F`Y?Rll5?XJ2pUXZtne@t@gwyEq>e%g&!y{m$~w3wL{?GVs+g zf4|*++%3WMD|7kWG?xbNpS9m^E)SHvxv}zd+T-f%4-XD<_7|&zM(i?}J|)lmBcZm< z(@1jJHzsk$m?Iy)HtI!OjeOHB^GgJ@B2(ppA4`&j#WtSByV+i^_2=T|mS!niQWSsN zulISXhe9fo$bn$DJbCi=Fx>cl_WLQWS?Rxqeszo$3%MdO9`y(fj)U)wRw?y`G-4 ze&6B~mGQN`Z+cu6XMFwNt%;IB0RaK`-S=)PeC*aEYkf`NpK8*tMcsOr+FupV^!c=A z^Esn!tyd58+sD-VtgrcWQhh>4rQl4t$|n=6x`k?Eihi!!`Tbrs??F$sX;WUPfp7Ue zVZtcOswcd+n`7PH1FnCj$^UIcRJvw`M+zhl6lOV$O@?9$lUF z;`>#9Meo`Fbe@^)wYpzkXFL^da|AG)-zJz??V;e#(V-#8)w%#_vC4d97OQ2mUA+SK z)vr3=@nGNcyH8wlmRIS`hAdWDosd%!E_ibKei4UIgT0&9M^wg?G>E(NmMJbVxwC%C zg}r{Rv&1VJ7T%q6BieP%X;Ja@+>cNCUy>JE$NJ$>m&Ci;{o4*QoD?yr)RNkJfcs0( zHA7)`3&UmGx%G}dn|a=8#{bZ?J??3K9}`{p)4Q2=n0&jYTr07n&SXd4!m11FeBa+N zX*kF$Qt7lK_mZA7gZO%v%FvJ_?^9SlfG$h3|Nry6qS6LGa5HI+aHaoiqqh^gC+H~o z1_lNmI(_o~ThItq#NH~?XJ=+M7d}2_s--UY=JD}<^KbFRXH7x3&q+;<+*4ubXYr6F zdH;{B?h4uYN3^5Yc-Q@S$iC;pA?}Qb-8)wJclRqxBOU#SUtf)Kvcb?9zUq^CY zPd{Uu%QpMeDo+K_WuECQ+S;m&LK&S69J-$z0)m5+Z3Ird{=mOZzRkf$v&TnLx!|XZ z{<1f35B09UDm6{=WK4!rym*?-yQ7;|JGC*?_I0tc+T*<$Ga&v4B^{$<2 zc;|baz-(dD$YyVa&j z^7B+Fotef`u(4(mNB&HQz<>WKGR1@p zzh7_9#w&GYMbACOH7bYZffrt_adU8Q%sy>D^RwH(YhEHdbd(xSf7`Y|KAXGWsdxMC ztGiBo11;8XaadlUd5qQmNJG;K!*s9f>tcKFVq!c9x>T zqP9I^T1-1^7&;JZeR6looevdOU3KEErTW!M*ZZH`PBX1fGy5B$&-dc}p{wgzLO3iw zOz2{}zv9obbPYksZT0_lU@`E9?YKdt@dZP>599HmDRNF zt*VW3@C##@ermnliSEN}J9|35xaR(GbWnP*d!?aJ{*UaRr z>a~7?E_33y`@yjNZke=OpUlk_UAocR5>8A|JeK`q|DUJ&hUM?>xTU5{I^N7@b>fEp zz1vBSLi3;a%69$|i+`s+uYyVGp1RljJ)ilOoELhjchLpZq5!pMND`R1&_*IJRc;x$st+a z?m+Y7_FW$@xH}%+ZZ%V_oBjKp;J-h-e1B9tk1lQczSjNchIoDTv=tgJd9|XfWM49g z{;i$V^-rw-ef=tr&(AIir~go#oP6(3)!M{~mtt;Slks+CxE^?G{mK=u)o%%u&Rx#5 zcfOtR`jgMpv>W6X9{I6)iiUntjap99tek{C*#gT&GfXqv6&p@BH`ir&eA)HlvcG*V z=u*%b34zH%|6kw#|L^;UpEbw&>pw9s&`}cepKli{Zv1Dpzy05mJ%7L5Zp*0s_w)I6 z`ODgmwyss5Wb<;#xHool=rzhG{ax-_klU#g_+Zod z3T>k|$I5>S2Lx^4PvYZx*xOme&pE-th(+eqyCvyguS-ZpwdTD#YQHmH^kcMsQniw! z%&qUcw|>5zlLuT{@c#Sw8qChc;|93QhB}Zmj8N z?w|M0-9YAW>Z$$KbC%@Q9`i6iC;Rx|o>a#@dl%~6`NbjHT06DtP~4AF3IEz#dm`rM zxaWoayP90R+Ee5H%A>2~XXVeSh<(5O)Vb(&x4z{5ayk$!cpx@w7v~$QZ z>IoXBF~+Fb3LpG+Gkw1AuStJ@e?R{0?CjU4XCKYoC|CLYZu#r}m!g6JE_d#2%azXC z@sKU~kbCW!Io1lY!LMu88n2saC^dNp zbaXr@vJogth`wj==F9uuReS104jWt8t(nF0#>v6ueba%klJ$2R>K5{z_-?5fR;yIk z{UA5)$-hu0LzN#+?-C!sPkwu-<*x#xF(cQ5G^rTSx=M2e3m&-Il9i=o zx$KLuMMPv)Om&C&>2Bp6`JP3S<4<{Xq;~ntsl4fJ`YLhFx>!5W314nxR(iSgKjczf zcKXSTT;Kg+X4*w_*{88ZZ+W-ldxbok_U!)1ETQvL@0YKE9!k7I=<$7@#S-f)<5n&D z)*l-9`y|(cqI+Mvo~oqIH;UdE^DFk(j~8Z!;q}WzS|Ww_o3j*Dt`Oj!@`Jto=H~Rr zx3bsgit1GgOnl)mS0(Agar=LUM}k~({En1c2TpoX*!gLmNoEkUpj}|(JILV=8TCw2 zEY}$oK5dUX^3F@OX}0*y<*o&sA`?|*Iu~BLD|&U_JB}T@6<+AMoS50(;jQbZucQ?e zv*>8D-}GOVVawiz-4EI`KkvWeGf_{4yDV2`S3k0U8Svi2dXZqqfwz-XrCPLunEkh~ z?K`3Ott(m2$r$Ow1Dw7-cgJQG(^M+h+ug88oEB61*^3WsSR%;bm&r~r}Pb&Yk zuItad)k+7>dS1VNPib{HtEYW!n4C&d)RFI(9DXUyiI!%pyW#M(yKaMc>MGvLQ=V`8 z*LdK}%C)aoay(GyDSo_&Q%*T4Ug`TxBjWBl;w(}&6H z_kMHgm$Q}9v911g#PoVhvQ3AI%|vU^ylw-Fj<-YI6%oal?gws5whJ9nm2i32o_aCx zfzkx;1sys;af^N|C;ilU1)KA!CQ z&NFr5iTp^HH`f+zU(>Cj$0(lmG5Xc@(%0FB>PQ>Z4luEtVXDZ?T$$MTDd8gv_m(@e zd~3dJ+2!%><*Dt}!ISpgy*y=yb!6ur}NYbPge zbNMd**v2AN-u#+||EJscecqRDW%w6cDYAX<%?OKg!4u!dewKML^ZkK~)w>PW^>d4E zHB`2$KKre$ChC>@rsKha2Y#1^P5h$%t5=@=i96G>GXguFH|QQ?arzXnP+sYgyquU` zOooro&YGW}*m@_l_n+3=&10N)#$kWmUn!Z#Y~U>1P|Mk2q3g7xB!c@NM;(|gTS>$-~Dj(d5U|DFD7C|pyw4_Ufh_Sx%YN)4xf*8dHZ zJHr3xkvP)|p0stf&$d0gwTt6{(ZmURf^JP=ST|EAVgp0@y-M|Iy3xzZC3o8NoH#Oh z|If4e8ZG7uhj~G(bvEcT3a;4Nq5J*ttyeDYE#B*=gxf(DLD4AoHQvCs--2gW(`SwLg_pn0sn z8(w!jC=y*;bF=%TE6aS7L%rPJzCW;EefY=oRYxXU|BLYb_e0r~^U3B{KZ;kyESuEv zak|`#gO1EG99rJ5+wHiLZa(``9i7d%C(2hwQvRb}t?+HppN5L}xxc(#lV>&cfR4SE zPE5Et3OJl)X1^2XPJB`eb9%(aUTqIj-fQ;(9kF8CnmoKC~(ox9|8D_-BZ_vMDQ z&V+O1Kw z<^CG$wmVF0=c!zAfnQVitLvJ;cRL>QsR$hcU2Slu{{LUMCw|^vKR`CGHI#B@Y>eW* z8TDXVb;@Rqb>CmdxHr81Tw-f=-R05tuXUg<{Y_tme|491BC;8+<5;dMZMVGs`dRUk z=O4eOoAYk#W4Ywsz*O-oVw0MXLSvYK0rvyz@^>aS2Q(YDGbLzu`2|h9eBJ+>l+y~1 zMY^iGEa59XUL1WrtAC4K-3o747wdxQpE$+61&W1sPqNbrvI@MlNxN&gou^Es-qYV2 zg@0Ey@0qa8Wy+>~3Aqyge<`)xI>S0k`ueQO9{!UY4|e%iuaxI}H*1mFtU~K6{oGo& zb}f3B&X8DlCm>bop39_@n&sEF{F&^yis^^0`!9oK+lBdS1dg;zc1+7(6m7Y6w=UBT zwyhZ#m7F>Le>lwlRZpho!$HuAqkql9L6cSB)YFo+cgE4&y?TeX$6ep4x4zidV;d-G zy>WHmz81wX{f0N#A%(x%#>iKC?JuxxgV*x0gZ0{5Ip3K0xq?3X5gGr;$qFFfB z5~>B3)=vCx=Jvro)i=p+bBM?%0nyeu8nv5^yJpAv-i$Q;zdm;93jVpD zu9)Mw`F{B#v&@J4O__FFF$VQa-n>6#tnEBweM^<+EvI8QJU;Bto??3Pn#!*!s~GHx zyZDu>EI#i3-pqEm;;;xi>?{Bl-v{f=ggn|A*3I5f@iEDnL;PWMPS7&HxvTgMA={Nt z2r@;T@te~9VQ$8Lu|1{plvm_jKf5k=_qFey_j*#58Y`Hhj!gSob$8D3?z@}To}N{l ztp4Af$f+zIZ4&x%LV7zR%f7F#w{HV=%l?a zJt@4Fn{Tf@`-L0Zx{V?i`ZYRy{{Cv0%;!5w3Q61js_xF2ef{z2m$}bWCcFksviwqS z_^aN?r!>KX;s4`loTuEY&QGsqH$7W6{q4n*jXRhB2wW`Vu2-{f+0B2lt8=VfN@|m9 zCHiacY;o<}IqCFOmZ?Hp|Lzc46aQbWw13j)&@10hW_>z4HClPb=fo9_GxGMuExQ&h zdSGkD|3|Y7{(Y+~Iy335^D3qvlhc3oa$46f-E(uhc7NHjFG2+?_J{nR&B$o8cWp{8 zWc_QIlEcE=ev5Bz(6@>T|K_#DUYK)BTZ7pC=wiOplN|Jf7BI}~UVP4OvV)j_hf2fw zyPND66{I@8esh(j-|5$x$V#_@zpVdPIxdjxtwLEWH)%KH^~pJA>DiZrF8=+kHCs&7 z^WRFL{q^f}tS5E$<*ht-P0l}*VSilY0-qo;4gNZbBa?hz_D1--GKjyPv46!!v75jD zJkJvF$)6n6eRbN?*glPSQu|Hc3O?A@a?}sB{99)eAlMUb7g`=fXakymNVb^MGh3_ez_Shq|o?H$$@dtUXRaKDh^D%95Yx8 zX1+=N{q1eVmI{-shuHyr(k~Si_T5UlYWRP(!Mwo9UTR+Az8@a6Bs9NT3xvP z*`I414_Hs#udaz#uQR*G{Lw1EFFX5|K)jhKeE2ydo*qnSio>ktjcCO3utH;yjvwCG}Mv% zLA};J@z57jisssE%w#%@x~41S!s?tLIm@&CetGQcsK2Ib=hc>Zm9JP^ZA9Pd?t29~ zmGtH_jx_zVJ-ObXZAstuyZSc#EcJX`cV+9W|M}dFdhZm{dTs~iHy+~nu^VZrNU+ z8cW~EO&--VFZnN6zMSoPpYB@m1AF82Q<$`iS;Bvug!k76zE)!v#H)Gb; zZ&|-5SDtRJ_<0?W3QE!Zy<{%-r~IELw>$~&(BBqZb-bnt@!q2Yj;;_yGx_^PuU%xo=RQWa_-q}p^iSO zH-{cJ-<`2?(cxm<$|-1Fh?=`c(>dq{jwP?>3{qVR~{QBIF=JNLZ^J|M- zmCC1et8cTL3oqlI;N^ZI2ZC8oeue}xf?OaAIue8Z#2yyT2@c6)J`FnJ;HP2=yT^f9EJ_nSYFT!0JP=~s z_pF`aS-5=0S-!{>S7WELp86YH`{TiGj(HJ@^7;icUl@n0*X~K!!|tyAdirZ|)*enP z-7ke(QiAnmb}%mf{nT9GKq`}uUsLAEhbfDpX={Nq=ZyFANjs~IZ!c6n&^?)f^9ksP z(VYgT-Y9dXC^G6curo1sFvJA~HOX&#dmvs*O_}Ry>cJ_2Oi8UpSMKSZ`K!^=74_St zFEZVHQs!f}oOhVvU4)5|>-qCpo9+dbI2^;CP+^>tN@ ze-6L>8PREY^ODcDX3+exlz_#i0_8qA5o4AQfBw|`_*=IlM$cmIIku-ez}Lt_j6W%` zfMFhw>NL=yNR8V-+qlf?Y(U2ixiWpKnlE->^{Q1Hil6&Av*`azRFCWMS;*_v6;mn2 zIlhlSw5r+@c3?2n5L3<*iOF+YRx&Yusnc8YCC1NHjALx#Hr4emQ%U^<$&`1HRC z44g=N4WCFb?%XLX{~^Jk7cB?or-SkV?k>%}U3Q;%DJy@SEreotDp{S7;qO^t696ZW+)V3=q7_|UWo44gN*8CVOd zd)2F*nWiZ!n3vyT-?MDlu^Ak}iw-C^}8EOSo-Z8!`99i`N;mm@>IU?xX! z&4JUWPVq4AS+a!X^eXnm=KL*tt~FG<$%TYj`xUgc#E_nZ4legBfboqyft+Jb6yMp(Qu3T+T)y1^=~DOQ)~BN;{si&EQpIJS~JY8+r7X z%bxvz>+CIlE=^^h#tZ3DfRiT3h7*QNA_t0d4&00tI$_47)NopeuU(LE&nofHE zBw~{Y+^*B0#Yty6R6&un0Cd^Q_NfZzK_kA#AdBiA2(ij>u<kUn|aNYf5(302r?OJ!oeWRC4zx`eIwezhV z(;ES(CrdylqRDp67guK#Iw1&(yp#nueT6nKG3qwCD+xNYTrbm@I(6z-(BOG^c(}CE z4jKLy*&EZ$d943dFVl(`{5@(3kZW9*zRtp~aUSmj#QSDLxA7zTDl>B{DtZnhZ_ z7C-$^74xCO#w}&LOL_OvP90WO)?-Sqqy9`Q68h=;akn!2Oz;vzPLxeYQDb4@X!F5}ADYrj>dySnFj>&e~Qso2D-GGgoZ? z(z@8~`^1Xjfd^wWzEN{vwAp>UeXVli9MuJ^If9#AZ-_NU!^ccz%oiO9jnvdv+7&lT z=-^}4k8@hj_DybR; z{8vnDf8Bx$JHK4nGtbsG-P@yqWs+)w*s`;Ww#Q9hkuSLMYm(Rj*B8@VyYEH)Fmtax z{d3Zp>z>q$8IY=+>~yKg#eJV2GEuYiEzp9_q9e=@eP z+kTQfbCE^u-8n?byFrQZZ^DAs9Kl0dCDs)S9{pN3S6%49(*L{c&MMVKzfbpLQkig< zWy_1T(V1dvwl>J>H)T(}J?C}lXUm}Un!mrS&Oj@>1#4Np{9XQJ>#4&}?ypvk_|hx1 zZ)z_mIHN9UZg9`MAKY&F@$tp1f6})WFJabvy#aK?x|p1e*<6;Q-Ub_m7e{sSUY82) z`?~8~Y31DPv`uGSm{bzWs;trb~m^#--ut16q*wTBe%9_eE9#$(4l?{PjA4 z_eJ(!?wNUYA_M0U!4F!|Q{sJ#H2Vw6|JoMC2JslMi>^0rwl9tZeXeu*9__%32zBPaeg;Eln8H_k{4fY-P* zFzuM?)}4!Vro(R`fsfH*oqwk7%>1`etLdn;?YFlE)ox5G37SmmyuAFHkg18*z#CV%UK~~ zc&ECDc`vuEo5+reCgX_>v7nUk`ot#Cnf8#FdF$uEed^btv$>}YYhzr}_Ub)9wN0l} z?Q^gG>h-DTmiDL=&;3=g==0<6N9XI`u9|-(InF;VI)1h#$l*`87=QlOu0Nc5>9m7F zgL?R}4-fhG#k{op?a8E)Ajs6t-|^@Y^PPg-FWK`ZDkd8Tt2i*)xE9#-LJJQMh3%Y2 zx=(M)y(hl^u8eN&Jh5qyw}0KaDSrCJ-zxq3^844GUiP@Alkd{k*Hue>pOqcW_qP08 zSStsvSvDv${*>12?zo>+8x$1RJA0a#f=ud{?Xt&b*D5h;3l~@?&nV8}iT$*9i_^wW z4x8TU3N2u`r=%nXuFgRTRH4yUY4e5)kKF$3%IUr^Tcaqt&vfb&8C|W}*Jo?V@4sKu zS82O1CiwQV;xn&(r}p1pe}3YfIVsWajn{ii?!6x~?=>j=64;qOeUI74?(%2ai}$Xt z4^{lT&17BDTdTk*v_Y}4*uBlLbO*bf|6Eb7_ZM?4PP`G)X!z_5F4IA}9heGPwn*fj z+qyu}H8oa_@5{#>mlvkQulQk~TwY_jckSmN#i_T~J@=mfSbXvR-L1uz$6P?APnep* zeeO*sj!ajo3l9%Z{>G($E&SKoJG$HEvUnUg#^NT@`f;}N+z(Z+7ae%Xn)mqNxrgfl z8P1~|WBQ6?Mr)34+zCNJ1C^ul;*C-NCjb4qXzh!;g}=WP&V9YBqE6Ocx9v5%?q1zF zDpxnJz5V1rsJK;l&3WQ}b?3ino%-2XSpi}DxXtz7_j&Y8KArma!Nx|F35!{dusuDf z>G-7llQEZ#yFK&H3%OhmApPUns#=qu==5%K)Jj(sKvQuf()Azl0 z;iq2Z&i(wymSxgY_4E!2z3zY2a`WG>oSF>^tO>O&PpTd3Y(#%gK6T2g=4UOhT-;^t z-%_W5c8>a~br6lV1JS4;EZEsbMK7xqtbv@gsPzUg-X^c;&~=_xd2CJQmn;o+$71sRLat06ia0KYt;^y?Mz8 zvL`ZdZkcyrr}C=8GXeTFcJeL^*S{yOUEi5L!J!N^_&xXWp|FVzoJ%|o=yY^29C2Y% znQ)us%YDYV-3ANAb%kaZFHHR-8(43*rG5Lo<*z3^{`mUi?<4j6|me6|e76nhi8JG4; zJ?|+y_l+%U@79g@v_&)^*QXwdh+G(|T}?swpUOeJxwK55uQV zpLpa8cAkG^8|tM|88i(PNK2Uyr-1^=?Z6*ZU4BJS0@~^9@PGF77^M$e1Qd7beERNK zG9zE!fd?zT~ap3TG~+9eGI-bFE5^aso6OwzHSUEw<$h zV958swMUF|g2OS+4u_?a#ddu$0~JcuLKEs=KRYV(R`>Vg|9UUp{zzt zlvl|&D}k`@YNiWKLasT9GViu1{9_Q#z4R4d`08&tTK%h3+jK==70+Rm$x}DjaiX*4 zsKdIAi*MzKUEeWNx!S7m_(i)jky1z4@2eWF=w1C_UYNj($NZ0ud77R$f8IPNprpZ3 zdsRZ9(1|S2L4;KmH!Q&ADCnd`7Zw+rsVX~lqDm&js0-S^b`1-Su}?c$=lN@f<@M)Y z=RU7J?XPt3D~n&6U`+qs$~imzKn2-LC58KrxBf?iH$vEa`Reed>(}K{&TEq!4m@ep z^Rd3s{d*eA#yu963%{uUt}OVL#~5=YR4~$b>alNykFE&Hy$OBN;9}~nwX!oYIZ_04 zhR=1?>UXi1Znkm$jhRt6gKJ@4>MEXN2HzJ8*xj5Zb9&k;``mbQwQuE#`Cpk^-8&yx zo#Ypl&+U7#etv6!)Qa8fFCQu1 z?TJ6iF1@ItxmN9BO#JB>_vTGD_N#xsES$`4@mFd+$Ky#UGT_{+aF^wYwI}xjBgSPM z4~q9mb~&v1s+6+pcG~q{T~ij+{12Hgc(C^T?vtH2{Jx4Bd>65I@+zNpyX&2n)Spk& znN|cH%D0;A&l(>OTP%I#9?D|rHU*vVm_2$u>h_-X9u=#Yb{I@3>$Ld)X4ST>uffY` z=I`>TUTGdUDdWfSaF&fJ!N~_>k3DM?{`(`vn`N`{4`tUyWlwMI^IK<-CKu8N8Y{DV zAabDit#HEse?bh}|4VFc2bI(gOezVLOi>S}^>zHvRchXoHqT?pHK&SIbD#a>%}tIv zZ7BV-^TFEIg3TO$ihH&$KK)6>f$6N^1*QsL-(EwI&o^Ix-z+dooMHCr$;W4@BP~Qu zs?98#ct!ZvsuM2`?`_+m?eyMKOX`oNs)K%rQ;F5(pTDaMK5>|D3p-YD|F>gK*f#lR zB_9~X*P*UMzOB^oy6oPUs@zlG44!FJ$HoWmbZI^xA#-3_Y;S$b4u?V;&%!xB%XGtX zvrlYT`D@4iui!0!vXb(=TwD2qlz`o{+wGNC@6@S0`I0o|Qq`&4)N zy*GDH$quWUf9`AD$;aQ1=FdKDQtos8X3WfIeINf=F`YHcdKatb;!@P?Y!^b*Y2_| z{FcGLXW8ks!GfR6)g_MKW6hcqD)M8x^`af0*M8}}IB!~^FUJm+EjQQuB)wYe{*!0^OUdr+jgCKymMgzp zk-AavXF9i;lK-5C?LK{#o4dA&bx$c->caHH|9|M^0EYhEH{NDlIvJSt$#bsCqbQ~I z(X;(z-W_XwJoWnNcPcUYp^vszHeH$=GLzdUb;*lVeYSvk2VO_-IkVKhT!ZnCx9Sq( zrSlsknE#bs+Q!DTxXX-&#VTO9N?~vil+lhK%~o`X|9W2 zZV}K6J<+>ECV$Jm+moZ;Pmh^CxpZ!Y-OrQHo$S~@C8!JfO?usKWB1}^y(XwX5asQ_ z0vc9tWBOt7^HElAsLp*Rj3u;cRVYhn{Y>ngkk268W_QanFgV}tQv1feb7%bQ+w}Iw z!k@?Pt~&J&wuH9GscupBvG?At4E^8V_=Pg;4~o1jvvf^n$*vDvo0}X`aiin)48w9Jv#d75y!jeNplpmnC3?;>##kZs()2s z!*8*}%WZbZTkHCFzA%hgc%)>e%;vQ(ub)bDUH7?R){c!!ZoRR*wBuQI|M$d#iNc$& zsLa#e|2~58kC^JQ{=lR?ek-FEyjW2cdw#FimT8?jk1rqAlG?vp=ul~uPvye>6Ay65 z9(!@p_*2h|uibGgG6g)>olm`2#~uk34M<}dFz*O;wP zHRt){`DN+?pbMZs{hZhe9%Y`Sx`6e=s#RKtnNn|W%dOZ@;k0S_<``{`j{I#u((TtO zZHo_`c>Cm%zg3oshu-Yg`c%EHi@{ty_I{3Pg#O=k8w0ZSW}hqlc%bgr8k?v4d7NwX zc}?`EZGU#lD(hBFUqsK5*T3gZ*<`}Sq9n+Y z-Q00(QR$)WHkGOi?N@0{3D~*&GpL7lcEiJ;PnO%;e@gDSyj1SiRpQ&{ud68%4-TXbX_|P1K{2uV!wZ*RevwZNh>{Aw}mb|NRc$ zoxkGc+nIB3&b>SLZY}HH(Ad!RyI$SCY8rdL>hG%yZ?|uFvS-`e&wh`1o%`Peh4#G* zJF8o<;X|U{XO15%7F>+K^rrmOj_ho6c&VJgQt>i5FIc(6J;1G zwr-~In`5@&VQb=bw6E!Mm)(hzt$BA$J^yve<7d5$jGGk|5}0}qF=anVy}+=0iDCkW z2zO({r6QKhJin_BhnIDQoL(y@W4hFBYE;nns@j#&YZscVx*9U;N|xH$S6im0mEQKa zdS_2>bo6&`lq(|Mf;j; z3p@&Z7SG5ns7*R!&cf4pNZ`VKqnrk5#yE*6SO4SxzF&5~!t*R!>G)sspUe8HM9pT_ zm?)f`&LVZ~`TpH%GaetczIIXGX2~*>m2;J!@7umCV6*DSORm<|3!FnO*M^&>7d`#d zf9=GXrw^k<*TziM;$!6d*lYWx_v34psFIC;W1~VQ{(lo!+s))0@aROQbm|53kF$RM zYyHJyG-KMqWmek`olTWv!?Wx`pjH;Gl zYvUE|&;9f{rYoaauk+h;+_$!@PB|*g)aGF6xFE}6!uFN>n@jRHwSURkwf)Mv zuRGdPS>Jhq`+l#jZy4}gIxt6ZLN13x!)h}N7mIVp8gq7ghsirDR119I z{NUOxW-0xE|J~}8eb%Ou+zpIr373>b%L|sTD*EJi^~qOG=_gMM zFWx%r&U)+Uf1WKp@tl|9xj#*v(%fU8{5!@%s$RZDerzCIQM;*)2vV1AH*&AyEZ;P@ZK$kci#4K%EjR-aV#%H zW@PTHyf0;>Yp}fNrMp{$`fh$#U+Ej!%ZwY7SSkVqO?KwE9pLzov(Gwg`xe1~AlCYi zuYP_yIbZ*j|7UyiOV+t6z6S&hSVS04e2BaKM)&u2J3pyww{D%#XI<7|_jt3M0PAa} zPtrm$$KOY6JX3#Vji-*0h{rVb$XQ~Ob^JG(Og`_dAnA7U(ka2Z33oH>SDINougWk6 zB_c*=2SEd-n28#Z_3aHUdkr_R+bSzK{nvL@`ZA9>+vD^By|1nB*H5(M^tiUHwD?J5 z!sIhMwarhrHi}u;r_L-9*jJw!9@_Ud&h!V9q5CuT*~v-8HJ=Vm3f$bMbRb@Kzg@-i zuKdui2N)_p|FXQaE^*!EhVQ}kd|TGCKg*6zR{ry;?&{wsOo=`-bYf(8In?Rp)cxP> zZ|XStcKM9xoqxLz==c__P073c;NOPMOe=2rlczRsY2wgeJf}8e6+;5|f7ZfHGd^`T z2tIiFd0*sOWq|;v)W63{KYf&UetJ4GpZ!SrmsfQyJ6t81+8Pumh%ovr`jTC^L*~Xe zmUOwJacq+hiOyW{cuQ8@`bgoNBO)1%!u<1Itu}aWarBn`_s>}owUf^~A2{AE@&C?o zufwmT>sUWJ|K6Q;c2*yc`DBe-&Y<9HG+=UTXcyF+`8k&9?92b$XVQIU?Dx$3v!=lJ zfnP84*@yeDOqlxG{Pgk3lUJ+DU)a}md(OvY2mF5gm@v2C%+#D{{V3+DYoB~So$2p< zK5xSI=ik=qT}e&;Q1$uW5nd+A-e<+0J2ja8%m^2r-Lq26vnF~ zo;5*xI_op>B-(`rbdw=4J7-Pw0pY&gCWk05FDR2`oV6ss*h*FH~Y7l%7S(Usd zYNhglme~H8r4(Y+UQx2;tQ|F&LFnEC9>Y$ z;>9*1Gt@rb&Ha77AnH=xoVMjUw^YLa>hIF~%@%c^nd!5(Y13c1dJdnKJqBre&z#z@ zH-ByW`{mvZ%D2}AuDi1>o_D`Q<)7bQKfS!d8x+E{Cw+lsU82L9>$f)DU#eYyW=`*? z4=cXmFvw8`KaJ)?Aa3Pd$w6Q>UHZ`G0jd*SRZ+gr7N>RT-J5+nK+gn zhV%ETlTE!D8C63*96$TtE$-LbcL~B1Ux@dMbZypYYfwz!SkmIK^^R0g&GXYsvM29L z<5OfkQnIG)Pu%{xy$gP>?K8RhPwkvw*N3}}O%s+0#z)L3nRAj^#zCY{vO;rRTYT2v z_J6FLaciSYm-C+6`MS6v^PGSIi>lVfu_4t((8;efVr9eJo zxo%~jdYP-H8?^6jU1iMu=?VYqr^g!@^mW5UnCAF8$p4EzyF+WAP1)W(|4y?xcK=>4 z>#S22<5&3P->I6}4^qOImk~2FG zx-ENV)XktJyj_BEUSz=9Eo+<(aC}&^OEzp~i9o`pbBqf=y?xXFv{$pA&AII1t(KQD zhd)VyGb2Z%%*m(Os_omymsVs~?rWR0Wi@lRGV?u;b)0+Rf3GTX zb2zZ*V2@g{*1I(c&0br!&FA2F{ohw}SMQ|BOl#jeH}T55pTDQkyf0GTecAG${zuO= zxfu7|E(?9K$RYZ6d4Jt@zqbGL>~B~<+;u@`^3_A zhph>=i6`u{Sl z-~TeY@u9j~sj{k9*X>;iN5Q={x7fd(+<19!=-EBrb@qPwU%MkLO~;$pdg@B43HQob zqJHJrtjbGE_{KWfbYI(!WlJBuzq_pKPxbUypK2Gl@`I9t!`C^zMYZQG-5S!x!nAd} zLB&vaUR$WDDT{)Yg+fT0&TO6j-n~I)Iosq9T)D7022?m_vKX`%`Sn{bN&U;WjPqUN z>SWWDhV?&rfBHrR&OIqzck@zxjqit5p{s9wj=$1=>q(H)f%o6*Bp!a)DJIdk{QM_V zzJ2dci1$6dGo63=?@td+{@>yMZ6s=XCjZ{9($^_ZPfcZFw5$JDBWbgF^_r;_vrma` z{k!z$y2i$|!uEy@dvlLiNy^@s)K`>clw9VpC+>^mD>dT^Cpn8H%W{NsZd9Cf&rbby z>C1`uGqralXS~XgnIXLM!QZW$YH}~$uU>R(YF3O0-ByKsW41k+OvM&$#iFP`1B>e=20pN;r_?yY*ORhjZ; zsZsq|p5|i$1}r>{My?67qi&yMwS7NX>S~r0r`R3NW0|t6759g)kCRmn>Sz}TyuZIt zs=dEk$?uQN!xH*Qa4@qWi+?UofKr>@%kGd+~}cKgFm!YA8Le|c&u^X=;_srfHr9-m$;@7iE*TFf4) zx%G5%@upqpZd^UK@7r!hK`Rvnm&JEh?p<}VQFhfS??&F&=FHO9UhY`Di*?)X-62!I ztL062?(RN){)#<$jVt@InS*5mr8Ctgb#~>8X86l*a}JG^y`fnD>iCye4wZo{6<11y zj_SruUcS+`KJ|~pRE_=e|L5Ob?R|gR%h~?|s~^`zN~HxZ$`lk=-NN%(Sb0O})zYHM zepa^z^|lj1r!1M=8m51ij+nV>z5dsXbqgXt9(;fPqEnoOM!lVUUg!>Qf5k+HH{L!?bJph8)hzzV|+8zx;Q&xQ|oh z%)$k%elvEQ`t?V>MnYGy&7nf)is57)sb&9LJ|E>e!Jo0~+tv>|pQ*=v&XB3IsQ(f3 zIn7C~C~p1B>#m1?NL_o~cHY$P_FoOAKR;*6rgUC8|83Ru-M>mX&dlQBIWzl@dM12+k>trg#Uab@Ziehf9tQG z+@f^j)w5sxtSMiwF-Fc^=k3z4`tJAEMz)`p+ZfY!ZCEy?{&m*sr;lXCkNYh6AzimT zq#@OaWs?k}@`25N*do7fQ#ioABX5ye{m-1rfYc{GWZ(>semjoVB5P+rKWleWyCWH#1D%U9Yab_Y9v|_?OuK*FvA%*?0BnbIaK& z4?3DycFs2s{PC9|dW={AevT)8=un4Y!PUz?;AE%(+*Wt)n+(t-bC zIcil&8&QuFw^i8`*-V>OFbxkb!FnKtE-QTR12Mq*rOOQ zPr!g_kImh+UNPZueKRE-Ryi!&WA`)hXXVKbubvLmy8;F*n*Z;NI`d{2vR>9$ESMuvB=^ zKkfBE}JY-Vf&Kx!U!4Sz71!N)leNX)h3)l_dT`X)yn0FHBX-tXKX|jAp7`1L&0Ol^Go$_ke5KP) zH0Wg~u~{R{ur&Cs`u ztdU-m)z+XGz`+z+wCbej-uaV$ZI#*~?shdr+hxrmmDdryGqu$Id;PiK5I3*(lKaV7 z1vN?rhkp9l9WeeOe(dOnZa0OV;AOx0pB7(P7kfL?N0U=7R=Rtw2gi#Y%O163ZdiCv6h}98-K)kWx@GxlThRSZ8omIwpyLZ zYz$Xh%Xuuzw6^JPKA&p+jzFm}dztAk(t`3V@Y-9rz}7R%hDw@G06`QBeg9)5}WyXVH9{c7{KpP0_0 z{U_@7q^#%JF%eb%0^f9lmn$KwZ{{dSdK zGg0;6u{qDY*czL+&C~q7ZME9hS679779?CdD$X0)Xsd?{8*OHe1o#p+>@;z z7N4Ddg8w=HeDz6}e-!-tu=APsllIT;=Y>yZ)_{v~)lY|i9)8~W#Kdlnea*U`LO&n= zdHC6-NJ{>kyxsMh&OZhJK2$1M_JWMa-sb`;`tM9nUbJFazzmB?(+|h|bamj&y`R33 z|9OA$67^lzXD!>e=;8d|iN4|Sm#^?2FP8JWY*QCy(O>_rz4qoO1^#z;xa*2Jf4B7B z6aIQQ_wr})_?kdJ%cmls^VMsn|budEDydawGu=pQ~=s}s!pHUfwL z?|eRQa_;uKVSF|p4&2x~!z6POkGx$>+5Ot`mzS1$KdCkPv-sn3`($<*AsOEw)h99! zzAcYYT<}=Lu3anT0>>e%u)=pI8WSQ8>rC8Luxr}#kFRFeJp0HTZTm&?iA~$qdn)Jc z|9m+7^zHWhWrj1%51#Q1l-b{EmEXwFZZh9GzmX?$pSwZ*N#nagQ;K}mESY$jg*Cp* zggCqT-PiE{7f^U6c&9zSiZrvBS-*fff^4}-3c=f!ve*W~S zJH7nBo!-NORX^VD`Fzf({$I__-0gRlgsu)NdOo*&*P;pS57cCpV(z|PzgO@1-1>Pu zG8PvyU9xI!_<#MCXlc9t#HX#Vuk_7yS~yqc*(P~L+5VE4;7hZM=YP5O+p{L*#>WSJ zJG5rZ^ElRT{&T_l6b}2VZ2up&%eNfeo7`_}wrua(?|t(ub8Z+|zFxC=N%;D>b9*@B z3q+1y^q;~Izb$8`@&U>E`%zo=>YTW{Ov81SPZvi5SL*w1p-&zLzD}K)J86be*3G%R zZ{IUphfF+pf9>?v)!TIp(#qGKPJbH}60NnoWQKmPle3{Z z9tQuO-lZvwOnY3M8a{je{q*$o)7v-YPj@WUPj@`Kh6~(zS8;o={;dDh*l3w2CXQd9 z8Wz6$G?yuJG56FND<3U5Yd2Gv+qnFr2e0`2{Trn}_{2`MKreY@^h!FR!or zgU%+g{M=*AeWLvJwX>O4cE8_jHvDtH=C$w7^Y#Cf&F@t#o>TFtGx}Ys;m7;w^J`@| z`8V=?d~*MNSIK|#oI4N94qrLn9?Cq`Y}RQzmKW_7HGHajC9iLpKWU!Vj2erxtM|!I z*YT0^dwS*j<4LWahgZ(|RTs0#Ak$)M@cn*oF*V-hMX!Dro;YBHY#^g8?Uq#)11FPU;ertKRIdVrm0yRQ$T5Py4)AyWVJ zjjQvQMovmKXW?OVUQi-F@c_6jS%}>8F>==bz%Ue)B-@`TcGKMn5)t+sR*(nRo8}elL1n#iLHHyY>oxkt|8V znr9XoG`8dzPmkqzA@O8?e%6ioZ-kdWud}!|YelP~|Kt19KOR<_-|ry#VEOmS&x7Xs z_XgXS)nDQ%=r6VJ5!DVmvB_{Mm(BXt zu9+|;YIk$$MJmv0dXL!wGB7?K8Z{?mU!}N8v`J|}O*C+0TUiN;c6jsF3=vVsd zM4+r*&rQZ>PdDzE{3_yMq)5V?Hiq^u$Nq56uXa{jad=8+R!#Pt_syKCDn)AXHGkM_ z4NlCok+dvYVqE_ynfISpgZ+9?N zx=o>13>P>oKlzD6HvdEEpGR?p4@!KB9+w)Jtn|$crf|=O%rf&2_(EN&aT$=0CEJ*BK}ORFO3Oy?DnCpLrpt z*Xs3Vn{xN%rat|Zx+&qRPUObAOf`MM$ZLMLzfA7qE${MC7TP1We&+K&OYP++_XH`- zc^!7!D~s*Px2CI`g|4lRZO$voJ8zwO;>YdOFM9+zKSw&H31mcWo;>HqM=AG))odAo zYL{lN^gTKCk^dLNS|&!n>dQrnZ})FWIT_RTsrdQ1sfNkNcKEw)zny3O(Bo*M^fFI1 z|9LuZZ*47>4*s>oJ2BN^U5~W+xlFOLmv>xe<=r-MJJXuV&MT!-ez$bGi{OJNZ}08h zZQS_jO0fS?kNE|Xo#%e`&NHZMIy1}GJ8DbD!&^bi{pPydRP|}t-IH4J-|N$_8F|eN z$9cMZm4wSWgM1(CyzDIg&f@sP&pfYo#v9ttb5-=XyJJUp7i#|#sF5q zb=(UzpQSM{I52p+IEI8>nX4@8F2AVWus(8c)z$?En#C;24y5bEs2*M=t1-K1n~uox z;B8M$L0MJdGRKp=dm8_LH-$E_naeXi{#qN zErNgLfB!zETWbFU>G0(dFV5y{pWoU$*8gnJ=Klc+NY;wfLXK+2!|cSf^gEnE&C!?Kv88pEGv-l3HK4 z=tn`-qMFV}m#4}brF7W*`Eb~%_?eGrX&tCJ-le^6!oJ$yZhej!CqV}be!5rvp0i?Q zv&p^s|9_uuyPa2@vMb)dBc$W8u%CtFy8ZvE%J=+yHk&Vd>hUcnCi~lIzP-J@U$Am| zT-C}wpH6A_bu>;0Jo{&!ZS^VX{5=a7ZF<~iz023&BE$99eG-OFepWA4Dqk#Yf0AE+ z+h|>emCHHR=`l+2wO^;+UhK@hSeBWMXTs<6_V>Tm_CBAL;plWHxXq6HhOE1C%-m_~ z_Z?e3U&C(BJdOB8KPF8`cjOS($WaNMc%0#=UUaSMmXDS%nsx5H*m-Ss@h|tlxhKPQ z{zxoVI3#KQQv8*pb>{nr0ydzUVnU7%yy@YX4^6*Of~YQ4tDj_Kiaca9I?Vt3S57kkB>zP{c) zmFxQIdjWd4pBm2GSF^~*?%Ay=v+nKus8{r>&)nr%?WgCjpH6SR@48G>b(8a9*{5e@ z^CxQUJo#4LFVZNCt$$yXS>C;XDW_GY1~S$B_;Nk<$(iquK85j$F=vIY*511MTCvm@ z>)gxnv#w=|&X^df{k^u{u`{DNKXh?t%K??(@!aBi;mep^XKik7fp`_MeKH zw=J&xZRw{&7sON6@BKFEFrT$g)z?=~PrP_|xZV1)T*hBVoA-M@pDQ#^JJusvbV_si ziTwS4)sD;8@3}ogX!f1p<$hBG7rR+9N%IzUyx^0yI+CJzz|#K5gJ#y=;)1pWw_@p{ zr_bltmr31wB5hT&;?0)ed*WL5e?Bz-`FLFZ)5qiTr+>fSuYWzRdhP5xCV5vPzTHcI zAMw+Uaoxl%t!MS+E1w8je!1XmdaKv?@v*+i!T#1)1%E7`TQ*Di_r-s1i)AmXZD~EL zr?`FUi@NW3%Vh)mbc=5B@a?euekXXAneUpta=Q!a7OZ(+Z*~28I-`(`^(-G1(f)$n zZ)`qu6s#}!AtP|#W`9rQgc-BegaS~9Xts#h=3+>zLbS1J z_Ow$Ezt&A!7HwWOJ3i@O%lC@EMQ86tdwngtany^EQTc#3W6Exqivj_D*I8eGUr87{6Q3+it zukIOM*m6H)1)Ih7YrpKRetFH@Q{khwIytqH z=2sf~J$C-#sOKoq;;AMjBl`Y$X2f3ukHC+0bK0(+Ut%Ne6<2ySwCGOZ@sn$}-;+`{ z`5HCh?DF|_yNuaCeZBrM{=%G^PbWVe6^}nr`~9wZyL??m%+?A2Kb_W}n=@T6_Q}@k zaorDDHukoe=Y79hKK=Rp`g^`Uf0z{Jf4`k?zucGi0qexLbbz?Wz2$t3ubr zOnrZGal7(s!^Wn?dh=_)&AiOe&hXvzz{QL^))#KHU05i0VinKzCl-%7l%E`zuP^zr zVC%cx=jK{l|8$qHz0%*edUDdrx|m}RRJ88(U*7O5Rh6HS@8j8&NrntUzFR*WF5UF` zjM9Qy!T94Ey^p|K8`?)Yw>xXY)ESWZcST4ulaZhRG1TA<(OwdTX2AD=GGPDwuZtHo$qKncr@FP~1u z&M!Igo!S16`qHKwI-lEY<3u`cXdGvob-lhc#?9=~ANS?ODg9#6c1MqI`z9?cKl$hC zbg$_S+;dH~tgS8oJN9Sx|$n?)vHZS5-rv6FVG$-fKoP(#QM+Np= z&e&mi@c6lxySBbwlDBu)M3!K$si%^vF9=-x`|E4*GabACe?Ck6l)b)g!9Qj{Nrl60 zypy-ztMWGZ+xYw6@ApsT|Nm&0DL!ME=X}aS@;!^N#)QM?ZNJaCyhzQHvp_6xN1l@P zC~d%Z@2GuZ77Jmkt}bY zxJUVH`Q6g+%9l&0KicHbu-NkdjpY8LN00k$`tk4hs~I{EuF5mX@M@_|*(duU<n5nXI-ED@0@J-Q^IY=qSZP-E{aODTPhs+ zYHs_ge6HP*g=qm2{8z**K1=#6`m=G}LZ2I1`xjjKadgHXz4UqD-mGfJfwY&KA7;P2 zl(2U5@4G7kmAB_}-nzf<=C&0d*Rp7*x=J&t_B9ATQ03bgyI=*!hf61(uiDkfGoNqY zrblTv=S+^AeaUrgyz6A^O_z6Ws_u4X>V3QW$?=`@mtNU(@Af@8=(+9OW)JsdJa^n!%+%k%)2%N0 zOL9z1q5mb-13T}z?tMS~?W^)h?(Xq|8L|Iko@gzY+wWF6#d|7)`?oCzQh2-B5_uM% z+PY(rqCf=y)X%qUC2q5y_ZGmW3%&6E~-wo#g-j`^qdY^+#aun;MStbT=udJM`e0k&Fx6hwfRrS3sTwjI09x7X{b&#jrHe1PBbi9qkPWr#o`XJd*Y;;4|x;w`t=} zpUe&63Q=F|#P_Y63+a-{G@iXwd$;`3)^m#_exBWJT|I5ZCBE(&nP)tWU2P454@BSg zg)R7|!MJW#tH6Wx>vm}_%M5Cq74^*b;2w@ycV3&XEtR@0ZN4R?I*tVRvmry z|7lNo%4(_JCBLs#A9+;A`Nr4&=V#uHpBJ%g_*q;1-<%~k>&~s-)lYBf{|h~E%v|bC z&ZL#^zIOJX2sgfe^>WV-SH%NU_sD-aeAFoSX1?n>_HX+D)r1X$f&W2 z=ZU$>j~}dypS;{?o@MwfU4W5e!{=LeGVTqlk43QQ2S0N)^jo~}jK$d&x%uZcxK$ei zcAT029#o2UXXi+sNZ$42-|zR+`R#r<>?(OTDeP~*-LDKu^Sn7eGYk~hOM1S#zq>p? z-3dGjRe4aH>CgQCf0o}VK5xtUG=JaEH20)_S?g=Hl?znvHS^oO@L8b2_>SwtI(gTI zWM&!O0<&Ks!WshIImbFBr#+untGD*s^H3{^>M0EMDh7=~=icq#rq~l760h3#obhb_ zmQ^1vt}tS<;XB$hr@@r{@8o7yOAq%sdVVY9#ogI?{{8;$fA~jXCG(-p4s%Y1XZQ8U%%Hq*8AITclhCdza%Xlp%{LT50bODP4ndaKKbmD?ZM}KR-V|T_1j5X12-lmPJTe#NV894Q-qP4{nH;jHhixlJp@ z*TpSec4yOb?Sn~|)OJ4WUfkT;UpgzLu1%nU!G8N$=Z)t%|N3q*D(PIK;impAqruqE zcxQUB^PGJa+J#TTXPESUKm5>UPWHZnS;wYN`@4mwWqZ5hLHnGsk8!v-A0A@H6(w=D!NxU-wtyUq*xToPG(z zLzxZ14fVcfwk%-$cSP8K$EKgnD@9cggfh?ZwYj-PX!pG<%5%SEAB^2y)|=TN?izRc z-8ItkXw+RiB>aaPiKF)V*Y-!jLwzc(q*r$Lp-k> zK4rpkN(fZR3)Su}3u2kV;J!G3^TYLP_a=oL6Y6d|U^rKw?~Uhs{d_+uo%k~q+Dn~& zZMBK{ckknxh1r=8$c?w>xZk#mal_ZrVt71F<_w5XN6IrOt}op0IH;N9`!-Emv% zcHa4QHk0w;9wo8qoeowk7W(|^<$F(M9;vtfEnNBY`fsDaR;JV?-VMPAcG|w5QpUUM zq_BOw`hLAX`y6J91=g^f%l$i{dS%O>RXQvd+kB(9WC*%TX&qdlf7wezOl*Dmf(bgh53GqieSVH*@GO(eM`Ct8 z4A*y0l$>w3pnadVlhLo*gYV66rSI4!f8H{)_2|r+d2V|iir=lD@+EBd&M)e=2YOtk z?u&47AJY+YZYp-@5Z+$&HR|W$GXI(3^~)Wk@0?L>oNEah!?$CyJUIRC+3?=UZ_WKC z{3^}1R^>di`D0wyA(mzSQ~q4*)t5YcMv_TXkWu-7_M_Of3b~g&dv8s1Wxm7lVbac> z=5Gu`F1*+kM-UzF9bnp5Q1%eC9@-Kuqx4AoYh?eY0=VCCXRHInu+Q_Z!`P5*c}u<~$W zP5`GTPiL>^q>?B8HXmKwGm_a{)kQm>C@#6ZJzszQzF(_WGetGu;B9Bv|8m)Ep;OL% ziy7_G&PbRqy!cC-ah-HV+xqig4R#A%V+d!8F+Ff`-BP*V?{=?0J$uE!s=FHTrU!1u zY}WX+uSdRbS6Ij#{m1Q#Ke-+KX;f!1M|^GVnQb3+W^hh1usUkZUe3qiTI>HrX1QJM zoDJ`@cgZiWoU`GibluOt8iDc`{|SBD`d?wW$_{^}iSeEV$BOsmxW1NtB&{^3xtQ-* z-&5(;C(l$_pI%gTuugEEQD@xo8gP?5$WcN42iNYCr#dgCot4OpNZ$NgikFpBb)Vw? z$OXs0aDEm!wzkV6+>=%NbZ{kTlKRt~*G1QZ=H7U*{HuWW`t_%O|Ji-VRIOXPr?W5m z*IuohMHM%nw;xk;IJs(%@7rlbmG^CO9nYFqe7R~=_|gAe$#*gK(-Y^OyR4GF{qME) z{!?AEpB;N-B)B7a{q`kUC(@D(Ov|Jz|4nJ^HDCN{1xrOplm(wpyZ`U?HD8Vg+}xn~ z!(?|$Od!XHJm0d(!MozMCyVt@JMI=A>SVh5&e30>sbnj~bsj5TSiG%Y=^TBrCRC{L z+p$*XZN-Z0i(=;Z*e!f|N=x~tcwB|!y1n0`cK+RV`)T~~KiYi>pI@I>n<5@tGSN7- z>6!)KBhdNfQ~mA#hTPhk``m};hf3F}{~wRb_a)f9TCw=aIqUZ&*BcC8-r2d?xUp%P zId|?K{?pD4pt0O@C0(-|6!}?x?EQZ4^_E?C)n+JGf(zR)X6-O^8Qx8FZ?VWIO}--@a4udSVJ-1tcNLExt=j;UNN&R@^F zRqktXD3)iM`F%Kj@@(PxU3FhsXSQ74w|f6fucp$;`dIB*?xRNg50iDwO{|PJb!Z8KE6+T|Jks)bI*R*;qgA@=r2{j>Ra7i z+-`yoQX)1dPTgbh&@YksoR78Yf#>VysC^DQs}WGMRm-%+mDS{x^wl$)TYH{;_uhYJ za$)t8s*j5V9wgM)t*=h`cFO+AyZrMP7o9n;z$EbCS7xHC>VdDD^Ee9Zp1QrB+FASh z+FEV}PDX~6ou3b{y!}Qu`sdE#=T^cBxwp4X{eG`n|Jt;Izvu0K=luNfxc}+Q^m&CT z`fvY!I<3!oS@F0`anGHq*K0X%^B3}5{P*YcdF$%iC)zj*9xeL7E&Slg;a1LqM>Rk1 z?XB{DKeyWYZP}%QM5Z|wDerbX=9`{>VS!`Oi-qk%vMMacr!9IdhP9a@yX%LIrZ%ID9brAbnV79Yp*$63tg)(%`~}Lx+we9u{*Ce zX$G)0D%_tFttug;c(>Wn^8fd+-^+f27G&@V7VOVF_&a|?e(F>!lpV;S#zq*;S`4i<2<8~X&bi)Le~86^fkC2QJ%=QJt%WoGU0Ew^^dY!`dW zZ@+{#ZjTjVnzJi*=j+Ln&K+{!QCl`orm;T%d&BQD6U2Sza%gK#j@k5QiBm(eRP#xz z`Gt#rp1N+8duvN!RKb=b>MLuCTsoDXv#gImLZE(q?A_Fa3VMf4{V_N$;=y{}28y z`gl~_`jK?7+O0YdfvXJelWhWj=SxbgXI%G*-~La54|C1#x7)N8)|(w*=a-W?9M4%` zb|FOi!Ii13eS5#%I{h`fV8PsnzuT| zSQ&osLH{-Tw!pf3E^qaFf6rgxSNH#M-NKXnKkx1+Ol}ty4v=NuQu}Oq8^erZhq}*k z96x<6W?ue15wf~V+JAE56rHmlnrp<|)qc*J+BNfA>M^cg#{1`M{9eIkvBaWq$;9^i z5$;Xw`?sGcdQrU4E>+kpyiRb(qb}Xgb8qMG-}}exNLE}D#|OKX!{7N$W1IMAp8d_1 z%zx|do_dF}*Nl>Wb8h_!`xJdzd*i9-gKZAe6cSh}rrqIK?dHf*ant#8uehV~0q#eW zSI&4B{r+Lc6HV`9rxEP?*8|? zKg$NX-x8EPKP@-wxnR(&@V~69(@$m#%U`>^q%M%-LsaN%wq^NoCyi^3Z47FK{}@k? zWSPS7d}(m*>zAjq)7FQSEj1KOFFNNU%WEdPy7Fh5@tH%nPPg1WfB28>|G)QqpZ(pk zB5c|=8)+lZEn=T8``e#nWS3E3m#bLt<;~5@H;ev0o1K5>S>J5G zmXm&b_F0vSZ11nHpTGI5ZQ&yq(Cj0Rq>;<5ZMm=CR5*uZ-~J){=F-w``l9Q{=JyG|9{IX%?v&>Rq|!l{r~&@_*VH!gT?a`R&>{?;rYiz$; zV8??d?j+Gi+po`2T(Cd2t7YEf)yK>dRrgK(^XzWdcRtkvQ@?~gvi{S`m6~z0O7}Bw zV;5)(h3MCFVGU)HjB!@xh3#&8pZ3U^pO*KopU7VSXTOtvmCNz^)AMvc+t=!wl-IPK zXR5yO-mlV1Y^Ekt4ELYvcMc` z#YvPSfa?!Sa-knyQTC4-RNy9R(^6!b9@bI z*XGP;u>bL(IkxVJ$Y)MrwG&UL$LsC?`z?BonO?kg!he~j`PJ)xex0|-^jzAa33e79 z)n^V(@V~vJ(v$!6@8=H-%uZZ3ulxHoJpH>gZ^4`TX3p>8Ha!f_*DZUtW53L<6U&~I zXI+|hTHu5nOU1XT1)q*H{MugU`c(MggojlcjC`EuejIA_5#d~yoV4R<=cB_v-|Wc| zlrTteSi_{FHhc9PjvwzG?3kqb&o5-Lc+B1IU$?kqG0To$J&sm<6U8NUy-$BzpPuNv zV3AS+ONHy+&7uop_dNOdNPIctK0SlRN#DQvvt}fgnJi4*%UAUA96yKD@$I$6RhPUL z#@7Co)AzWW6|H)9x<=v2TR)3VpI|i1P2O@O`r7Lm8oM`dw5_?7y{UF_T+>Ri(#zBS zsy!C(xpaBP{o2=enRCBoCp~9b_w!Lk?$Q!=Va;0{M(%Pxhis7S)zs2r#%`h72UhuKC!Qrn{+h0sru8`lM|mtr5Fm_n4|nZ z&-6s*&af2Tf_kR&>UZ`@iriYOX5Q79a;@u9S&pst!<+kRtJ6OzP33Ye;~wd zwps2RO+hu~r)NL@=8^1sIlJubt)1HzR8Bdwkn!Kw>+#mNZ`@#US6;bj$4ujNE8g-g z`>+1tdVcyDyI zUgEyuqA3-ZvIFc=woSS%@NmLinKO-FWa2-6`2BG5gDDHYhTPYWKm07+oTW`IW$PXzP?V+adHr+wYL3Ge4$BN<>7Li}BvdK!3~C#~o(w%UN{i|6x~F?yWPD6d5KKeJh*V%y9jr zwN&=$X}ZNx4{k8JOV_l^mK98{i`z1D#%lf@Ero#}S002#M^}T=h&uac? zzTf1t`mZO;-)ReSd_2C0DY@G|%|PwPIe%rlIev5ItIv72eCD2q;_u{VzOkS3G&@~O z(pCP->x}=`?}w`H`DohrXIB3P{q;|GFunEP>C~|P#1-F}My}1zq?z~_kEzMdm$3d4 zb6l`S_0ZMp7N1t}v}RhetyK(=6f|JkV|4Fp(WSKc4DOQ)?(M1k^ljJflw`koZ!Z*G z>M08J-Rk=H$c*f#*In&huI+T28dY>yPM_=BnyY^;z8gzgJ=C-9{i1)Xazf7YnU;~D zfh>bh6*qTeI;Z?waUzScvF5Lw^U|=ZRvVwJ^ged-^!|JAeYQ{7=fArCRAa7x$5P=R zFTO7XZH#>UsH-|igek`SUiHYYaLU&Q5e{C^pHyc1Cul>kThsmQC!8_xzXn&wqDjoO}Q$h8f2{J*46TRvRTylYjA#hSMU_XP}CPRTRbMXpW!^I_q8NvmZO))nrH`RRNh zD~BURj8XZ3arg46(;Qh2u~^K%vVZe4xybpu)tbXME^>I#V0N7S$;XdIHNReeI`>3f z+41Gwl6cE?r&S9x{rmSCUJ~0rKXd=J<0s;8&wb*4(zx;1#~luzEo93BZVRXWH#;sS z;3zFqB=`HtzS;Y;mfiV#*YcoR|3_}M+S#%iZ>7tocXroJdjGuURreARzjlV>Pdv{% zB-iaX%lWkJ6k<>G8}YunedEWl;w$wiY3WjEJGKM;73(XzQy+3eufzgB_I=WO_867^Z& zT1`vgtYsgymhYS2zTug2kMU34v~&^v!}}}#@-HfTb7N!EPT@Co&UMLbt^fEgvYvF^ ze|g_?t3`h`8yNlyH2EJF+^2D{b^gN*3;w@ZK6zQCuU$S*m`#nnq}eQe(15_ySh-*g zD?1HOR<*i%^)=6>!s0)EzHF)&(-|XI;xwo&NSyM9h~LmGLJgeB6HY=+dI1g58XtreAXxpH=#E zmug?i|KgkLSwG3`J>1H~`R{2@`IBDHelZVy9VgBYt*f82NPC>)O+8;I^-16pXo$b+ zs`RcerP9|=MNgI&KeK;Ii}!Sf@ZIloysAE*nC!jCYwEr;R+hX^pS;a0S`y)^eqiTK zpF=-gqR)ygU_G~EZ^lKX<*a@+{c^T4k`XVh+Zh8A1sXQU8mIMK-W2p{cjDnTzu(fg z?~7JG)She}=xKZJ_l-aO-|ZgF*ke6^F6U=UhTiADq{8A~AF(|5gxCJM`KAs3FPKXz z-+~75^!a^Ys>4|MD4Vv->xt1y4#BNwr=6TWmkaI#K@A=`u;3=<9ik+v|(ljI|$aUl8||t?2GM zeU_+eJ2z%*SswrE>AV}2n+{CfZKvs*W!j$_DSzX@zJI5aC*P9pPvv9%mQ%Un&Glb5 z8e;MUyiQvGEqqr{S$rvZQ_`&8Hr?#A|qwDV@jt zbnbE4@{(B}`lbXFY2RnR=TNSi#((GhW9Rb~PnW;kc~ZJ=>CO5V=k5Rh*}S=y|0OSD z-~Ov%(UzChy>cd=k4a*wm}VK_ko?AB@jQcb_1o@NX4DkSob<===bnv0id^>cQu9xk z|M3#D@5nbfD5?F0+tA>A%j20w=^PSFPo*1udfA^!v##E&I3bdyhvB@=ys#)XK@Q;u z++sQt{KM^Zy;q$(^&oVOhU;$P>VoSZy2 z(rn9K#eIqgIGyWWADf<0GV7c9UagJH(Xz%7CXc)AORdCItgioX*>|#i%k;haxe)D3 zo8>Kx(;Qk}&v2Xrny4@UEr2{AaKeVktsz}5SliH<;|Ghy5>B%N`(wY&^I82bxous- z(H6lQnX~JPlCEnrS6}r%(b1r=wqsr7T(cs__tj7P0@GR5yq<+0QB%vk9BIqOWp+Pb zzO+pze!E3xoVSh@Q|zkm(`#q{KYf!S-FVL_rv~SYLWd^j49BKq{yBy*o&{z{LxkpZ z*3HQ96fd3Jub#!T|BHaf^8XXW&*tBAzwt%r(`;tlf?p=H=Iq)iTWNVl)vaM;Q@QP3 zYlgS^G9RnrMVucMAU6`jh5okR_u3~Bs9Ug={oK|wp?WMYieHuNxm^4i8n;Hx4T2A5 z9=zzK$;eWn84|r|g8B@HpOqguer`RMu3BKK9VBS6^>^u|b&lpc-_E}5eI;*e-1;ql z_iwciP1BjLl5-^TX3TZ9+NW3ET|ce8pKns&X8Tn+XZ5ds`?mIH`OT#_8T(X%doP^& zRB0@1zLlMi^IW*iga-omCp`FZgFXGy`C~H*cN{3poyokbdYQ1}{0SXhr%&g6YMcL> zbEAD=t*X|#4gZ{f{rLDe_J8Xix5v_tqEF^ueY8fWJJztFTjGIS!HSbA@xQGjCtC5f zd|pxex}fIAohOUM&+zPLm@aZAf2zy>hn5ds3yPhWu;0PfzVlRTL;d-$=BtVn6V`&4 zyDoSFYHvL;+@>ue$f$bYqCyB$&8tJ4pJLC3cSS7fT-wC2ZL9w4-!uN0l(+0L`?xT~ zeM7Id}sK~PXi~26}a){rrsu{}#=dXFWf3-k-#S z_SbD5NORBLbJ*+ShmTzkr}Qs>rYL7@|NVabexJEkp;jd?6q?z1yL#66P5oB`Vy?uq@yg%dOS=zB{7B*2GIqJ^JXB zBvVHH`-_kAKNT(bySL^O8xLnBv5+>&gALYRYas6IQR= zb?W9(y?9;UxM)!DU0)L^yqt5LiiO#spO@GVEh&6_Y@LHzqt(yPhgFs>TKeMj575v~ z*QNT16_VCvYhKUxs?%mXXF2Qszh%?iVk}oo`uFf>TFwJo|L)>B{89L5 z`(>f0-Xi&*DsQ~obXw0S|DMgw+V6Llg0{XKmn~=LEq?QG!{>9>r#GLsn=P;W=~njo zlc3pnW_G?Qll|>1w*}1Pc<%C1;F&YQI~vJ71Eda*aE zUGn0~tm(Tg!Qa%wx%SrAFX|STm-$Ypq=SpFhQq)%&HE&nwFk!{D}i)9 z*2eAK6@4b?vigt2PtupV_C5P<{=qn0fbrL?KJFS>CeC%->^eao=Gz4F)d(-0c>hUv z|NSW|gO}9*|5x<)>-E!;>2n0jr4Pt6?eU*)_vy=J|KpeL)qM8d`DBv!lVj3(Q-u9& z6d&4pegF4%_xpXDZs+Z`OuBpbK3CQMjjjBQ>#EycR0uKOU=o#M{J&lKrbPI|#tm%X z!yf{gK)dpq9zg>Q-V8X8Gp>W_Lll=Zi-aj}0Tk%LV?-C?91(@y@^G&dee*fO= z_~CWSvq}tiepI&*u&;@-OFt*G^Tnd>C;NWC>s})```tpvcaslKl-z7E$y26%Qmj|g zL@l-aA4_l7D=e=2`BXf8vm9^!i9C;d+lqCy zv+oCT$K0wvtKZR3CwZviz~=Ki?ajoda2!3*`+V|~%#)dhYV$?*gQo{~JmUI!|NlRA z>o*&k7m0n|UH<+Z$A{(fsa<{mu-#Yb~`{vqN z_XQsAT5&Xq_lUWQsC&r%FNT{6Lr`11bWPh@5_q$2!_x(CmyHw=Z{90*)#wNBMj~4HHv&<|85P8KZVkop<<>*C_+0 zJq6Cw66D2#Qa(O9+Q%^8zJ6cN>Ax;w*MHj{4&U;K2c$tmj_K^R@`-iPvQOVcCaXtm zb?N?P`T1`7eO=Jn{?#H<1?lrD(>`iNxLxXbF5~m5G^YJeQcnAiHkQlH792$$4WPBP zhYWoVEV^x+YJTiv4-b@nJ6$Q%!*lV_e+({o4#;>d*bWn|)zHk^8odoRUjoZY)%-yXHBf@ z`h8Y&=9oV3c`5U&l%GXHxb9fFM}CKa>S0TDxu8vzpP#Xwl=Htl!)WRfK4H^eD{B}9 zeC(FJ%>9u&P0~2+#QXjK-*KpO=C7LH;HBE%KMVGCeSd7Srq1&Ozoe0iT-}dE_dma$F7~W_IyKyCvQpsn z^o1XGy<=cC&}H}i_s zuWxKzZ1eGma9+;U*Kwe|ZKt%??=YP8X-dNTo12&K`SYRmjbB=GaL>o+SmLj zkTghWu=#Sq`Q3JnX_oeLc}*kFxUQb_Yvp3u6LYKI?aW))`T7xP1A<8W4E6xK{R?-b z&#OF^*^)iC^YVsYM}F!-)@<+7dAOp&-tX`$@z8ncjnyZ2EKUp1xC~ie1uipF*g&Hr z&5Nf7F>!nlTI0eg%+R4=+T7H3?a3&$a* zT<}d&Jaa)LRVX@0II^Q@e%-H=p)>STW|?G8s(b1kcu4-X;toY;iN`blXfyVmca+K8 zv8{2L!mSRk2`a0LtXy0|EY;-Kex1uP-JlJWNAXK|2Lb@KlylEzC7M)-=9yX`$E25Oi6rpCD^}It+LPhoyE)R>*p(Pi(DgVnl;7d z=ab2KJ}n)`a;7{wK7;FE?T3Tx+)~dU)ZD9l&be*(L0R6CeR@jwxt-%C%&mI&bqz}#i<#@{Z8E8?*I-N5O zrn$FDynja4{ofv9;&@`U{r*a?hS$5Bh5mCc(Y*zV4N!Sxz;X%H9WyzwY0c7p24U4L z_7$AZSmksJPUQEwdW)S}l50Qht7-U@x}$Hpz>!_SaV2K|)Hj>H#c%SRyo@RGLH)5c zWlekENb+uql)UwDf|da1s;$rCQ$9U8X;k$k!`;VVqnT{JFM#WyU_c&o-AIoG<1*RBWR^6LtY7&y(Aiu7^a`|s9k z!#@f;txI1$`SgC${(7GU4ioA>9u>c|Ds=UY zuOW5I=avWgQ-|c)p=i83@{~50Z-DMt%73S}JDrWg| z$>c}x?4LzNSX6wu=)Uy?(^o~s==b|qN3Zbz{r0+B;D1%`={s7B&n*Aw^J2#0NSc=R- zC+2zcPn`DE`Uq(Hq|+~9=7(uZX4}>7G8Pqbt=nACu{QFU|CSdD)~9Z19z3kHyzZs& zHMaCK*PTrdDwZjnQTUiCT$ZEh{qgJJErIJ7w=_Syyrp2z$^3&shKKkcFUfrPBjd??coKHf__5$R?Z=-sBp#kKQ|L_ZgMI)1 zRa<^Kq5L%d|F7wHDxc4FT-%FUf<_1rg^t@GuQC(ZRc3)W&xmkwsPz0CZ z$8*;2k1U=3lrwox-)Dj335fy^d%kCMI87>4Q95)`g7;|J*;z+tEnJeLqhv22IzLOs zb;&Gqt@$UX>(&W)&nV*%zAv!et7%d-p5ug|P~Zto!Zsp42EvybWl#ou2f52pKY zSpVPo>2RIhqr#g?|KII?Z*^Ju#)o5H+OKKH{J*(5{q+BT-|f@cWpeXACH`XGtRY|f zWn#|$zuyWU_v_oIpObMsEqz%_;JU>z{}hE)8~^rxPBrOyTgZ`aw)BadK)*BdpLx~q zCJHLM?f5v=VyYsKXaC!xoQ~DM|9n25%(7!WW8DQuc1vTO`!%0?tCyX4KYi(^`8A(9 zgYz1vgwOGKeQRsB4|B}p#ZOMI{(8OU$jZ;De>N<8JbkN8m+cTdXE zA0=_MT|4je+Ej3qOo8{0l#~`oHDzpZ=vbj!5GP`3F+cjRdE}*6*B`vy`e4b*j_Ib5 zvpFUy3rx6wR{YDPYddXae)itY-};w#W7{{eC)=}n)%4e-1mCZEt@~Z`@6YG+<&{nT zX#35zx>}pW|ML5{+xgbHMfQ7D4qIJJUenHUeSO^(9j=rYt`m&sjlO(;la^m^)~UZlFny$T+q|8yZr>TKN!?m zzkNW+_gTrS`Y#vVtXQ-ju)J_xoMm=Tk0UuUG8dH}A1?!(!L| z+(P?#dmdJ~o^*e@c+QF+o2I4jdGy-wo^r&5%dQq{r~ed{W_&w+OVIDKqj?**f=(Lb zX}siqfaAlZ<9F80P-EJ|vO>&avf7pd7oR0PVqxrK6a02iJao2N_pKh&VvWp4**AF^ zofT?X45}~H_{nEZyM56&Z03eVTU+1yb(qfGp;bBUpQu+j)9o!e6>ZMl6L>Fs-8Q`T zy5fCyv-7hTOe=&QtP(u;p799pvw)IM67wv6E%97rax3f8$0ZZ){r}V^U(4DX#djiX z{*P}b^EvY8h1Sdq=j+J7pEFS}F7tkX&l!CyCOgJ+)-vC_3ymi}o*DY*Z^5EJ4=du2 z@wb0gwfdg=r(n^ax<9SraWBL~j`v7D?l@cW1hl>B`|M8#+2uuU{R3_8ykRieSEn$6 z>thXQDT7M6(w_}Gc^TJzy}V1Cu`kU0S@5Ax0uSC;3SNUOwq7mu;7Vt&wZFwfmhk?P zTkA^$c?&o`EIRSqs4n8P{uz*^iAA&Co&J&N`@Qy<&OOx|?ngKZR`Fav)CC@k zQQ&XPTK(E<^Ul3XJ-<#f%+*!hfAvdeR=x4UkLsp3yB^-ynEX_~{%5x`{IyXeDL{{7s(vrIbIbA3K$qh~JWUwMc9 ztEpnmyGOqJVji_ensBIigmz@l|Izm38|QnCN4K|s&ONf-PI2bo>^JerNdTAV` zAoKBQt1!pklV(4Q&sn-FY&hO4t6uu*T4%3)k9FX=_4ZMhGnngtJZyJ--OQ-_WBudM zhDZ4IIBA& z$5GCq&UwwspDb#||60^;D_-CXUt}d;aDcICZkWksg~OToo|dl`wbwB&IWqGdJ7Zy6 zZQ9!P@%6mH_9qrE*wXUT$1eZU+8K@QZJTB7e=*b+-rG}|XV{X?|7f;zZl(6m-t~n) zX6gIXT&{a7ta^37cJa<%GOIa7#Pn*U|Jj*+ao#IlCo$QzU{WPA&r7pOKKHG z|AfS)F1qDEF9o#dUA3@wN_Bn5PAVaL@4+eYrYy=bDUfyVu;Us&i<`*IF1Axe*lh4+U>b(cUU%Hnol0 z%JZ%kugpo_hi4fX&oN5!9Bh+h^@yB5W39%I4KpQO_|)7lY-#1_FY4L5bxi+|Oqb#d$GC{72h!(6KP|53O8 zsjb)Jw71`@S`FHaS(y82{r}VY`%k!v$38LJWb^$_aqRDlGwR;$e17NIjNIGXbQ?NP zZQiNP@;T65R(1RRI&0t3AFJ4Y_?q8+^61bErTWtcDtHTyzJJ&*-tpJzazyHrC@9(K@Yvx=yq$1tgCuQo@#wYt~->Zwi*GoOP;`w-v z{G{c5QU7n4?m4=rV9q1=J-aqdd2ac@TYVm@>H*2s5{u7-^06%XmGm@Y=T=Cc?~*g9 z)IWS{U5F1$h2hf%j)$s&o&_ccr}8NBX!2S5SxsJ3I{W&GS8c^9v5#3d?Ymc`e7L(( z-sg_M3K0&LCFNV6eqHkHQSOp^pQZ&Vmv?ircEqb5_-GkE$(Peq&FEjsGNpuLEV54Q z!WyEO=VE`>&&0$&sRUOtNKLZx)Om4 z&u4r$nIax>;=#W}x32V^8>$7*F{iKFu0KcY>Y{>qlcuE4N%CnqQ1^cC_n6W}>;Av5 zpZ|8>x`Q$&6~pKfraI|s~{xF0m{#FLYgpFC*h zFDpBJzw)`PZ_&Q>jD9WG0*zmOU3|%;|9$%EKl1f|9*SKze96jq?s|%icu(jvaLW@s zL#1HMA;CCLQfBRxAg2cL@IDRS0uMpkIhs=vCiN^+NU>x$k~U}GmXf`00`J!F6USy{ zrzC$h-L&(U)ysRpX{=IqHiyKtrUO!Z_|W>)2nBMK8jm#wCAzq!Jp^1&->hu!;9Q=zzK1@ApY7+nj9X{g`;TZ71u|zu#`J zcZlA=U-$8-`007o?==1GelF3T{mr3abEOK4#jGb5dzZ<(8Q)bu;OQ*K^k?7icatAB z@_v}68!aTO&i}Lbrf5!scAXs~-^{aL3v>!La2&{2vsj@iG4-(h|39DSuKi;Deoyef zkNx!)mQ(L%E}y$3|Ng$8DZi)fdw4oKj-~T>=A3#tzRB*V3!M(i+w&LxtF)M-S$S-` zX7DnL=pOo|An*{-0SAe&gYO|5@lzcD{@(h_i?eqp#p=4Ejq!Ac#U}1 z27zXrW-JimKe_D5tZAWz0^WO7d*&{mSGCHa5llkubo3Zx!tKhJLa^}fvKUSK?*6v+4Z*~5srOoo+!+mFN z+-SQ0?k|~Fn^&AyJ}=E)dqRDWX6v(n8eqGx|hWckS)90)$)-It384atG@1*~6XO>~QQ=Y%~>$I5CtD*1y%`3Z=*=Lh~Z_iGjxmH&vKAPe0 zXg%kC?*?Z5Dbe$)UTM1Xe4ZnI$7?t zB(@yFKzB%l-IrS(qr{?MQzJCzB=ZvvVJ#DuQ06%XAJ2(+Ehu!cTh7^MR#BjF@X_y+ z>t<@4eFSPx?-mG{q`fL^R@?e^q5D64tfuA!PwG9nBCA38nL_W$9sV0KZw6l9YEk&) z1gL4dpyJ!j^tv+~H_Z~ZImk_wl$H|S-2 z)Y<>%)9IpPlIc6TosWEbeO-Us0?}%}&-2TT1lxq3hVcNoAvtl zI>Ddy3R~Cj|5ufHd71AWi)C-wm+yYJ%R8>>rRsOR)tkQj-T(jZ`=YN`!*}0T+yD1O zg7xnmkNb+%0z3}4e*DZ?@U84Zw$rJ_jD6oKSBI^A^p5kX2IIdQ$^C_|o+xZo+-_L9 zNa{h|+++Rn`r&bvOaFYko&WCdT~4<8A6;GdSvuy_GBYqn#*RbQbm=Vbc>%M_bdHMJIy?kY&9lN`;K#t_@ z<^J>EiJ41IzkKxO^pA%piuZMVN&T~7zNpjmI_6BlvVA@Be9yg0-ktvO(B*j=i9(z8I89n&J+d^eqzVrp~X zR9qnJSz|jz=WxH$0mb_Yrj}DWf+zW>`?CBEpZPfZ z#ELF2^Yz|3uguVSfsWt|gT$ul{RjSlj=lJsFMIyfrn<6R@H8_w_2CD%_~?$ouh!uOB*%RWDDO za5!Aa@3Be0pU>7gr#g$CMW3}e{VeMSGu!Qpi`{o0JrlC7MaRgsbmfW9zZnC~^Y7WD z9}fTBe4k&V{qX-<#ku>IbZfqPEzasSVS35m$%&KK{N3WZe_7Ly=MQ2YD_Dv$*DgEI zW?*-||5wY`${?fY`Npa{zs7uC+4E>^YWw`dj~-0tk@oI(YdETTSHMDxT{=3+{nw4; z{;A^eH5XGarZ=Z|{%bDNkJ^wk?ZmPfIZICe4PE|S#8T$@t?<7VXV({2<$e0tUC|c* z?TdV!`-abbFNOQqc%^nc@4U}^%;w$HZ}D%>KMRjx`BN<`cU&uPHEfJY4|FJ<-o3J| zLQW3xA|EVgtTEo{Rrcda%c+(Arkw4uAu$&HbEoSv)i|x?J1bQ@m7lR*XyMxLk2zj1 zWSQl^LNM)9=X-gtH!Ih~FN-_=XOZ%OAE$QwIjKJXgzok`N}%I?O5WZw&C_8k{=`~- zp_b`=e6Pj7-6dzIzt7&c=#=vX$FK!!y(GQvtXluK<8_AhkG~i1E8V|-U+z<8+>~vx(|0!JzP@84Y_}v>{kwk0;lgvJ1^-OHm=}Itc7W&TF`M879lqiT z%MxR4zKQqUHEk-|y7rgN3V&OU$G_Q2zwsaKvEW_({<-p->5H!i&fk_f*Q6(ZU5P-$ zg=2ZfQLB@W^%Sag9|~XgKjGYp;N^br;xc#H@$1a_aFyYRxtgKE!AO~`ySuigncf#z zF@eM2-lsbst}>?;SX}t`Y_;M%#`-y{Con`T=2AHDex_C3_Os}N84+z9FSwca`Td%j za{x56R(W;#ua>h@{aI$2W{a)1m}UBmhtbtRU3}kqmRWvgme)7@l3HG=*}q}u8N~&U z4VN%H(RPt&DUR$rDo}idF{!W0@r0Ad?{@#F%kc%jW-o3r+9&7rCsXc!xu@X)|B`@x z%LI=V&g=U8drjL`|KE107sc%T&VQX5U@!U7ch;mEiAmNq?}YD4UCaJ){hC!~wWWEu z8ONN;XEQ&64nyb$jZB>0@cgCkmDdcLn>v-RG98!k0_}HDTQlu=t()4L$?Lik z12fg;w>wBk+nWD5weEky`MLKcO!h18kMu}#4mgu;X!mNz^4c4Jy80Q;FWnVVf8BIH zXlo;=HdzVEp;;D@>x$YM_E|i5?I>&_^E7*x*;Jcnr{tMx`quXOuRbfu)YV|W^MXO$ zCgw=r>l>z}*QE(u_$V<+!ih&I=h>N=>prX~obh|%N2xG*Mc&o&>z2x|PyBi@;^-01 zZF>#RGYB%CK(F$KvbZMBDGDKb-!)-dAGz*Pg7|N2l^It>6D|SC6c9 znBI;DOo_?+Hq5ptTr{WrUgf22xzV6Q>+Tc_otyk3=C{G$dCz-#1+R)(aQn>x9eMh7 zu}f-B=kk7$=e3|~1TMX`|8s1^OwA?R*Z<}HdG`J3|LN~-4y;*m&ir4l{((c6s@XM_ zTpa^9GA&{=ohFg}=7^N4l&eWnx0|Yj_KJTW@!z^f>beJ;5cq=X z|C|8~-{)&)_b4&NnG4JklURIvmYAz^LiiyS21hSuyY*=bjSfujoY}s;+;%K4>VefZ z$v?^s*?M9ei30mSUdnoUYU>u6tSq&QluV!@~a@+m+3@ z?m7Rf;Ktob7o%D8&$i!)-NLsdF?M@tY;q6R>%1KEIrg{o&5u<*U^>W}nfHe4@~(A{ zdcu25%ww+yUW>iH>&~}3{}%kdQI@>>puMS6!oNR1!y4R@3tX2>yH{tn?zzc7e`l6( z?ShDg$KN~di^nW~@w4}{;*DMN^j{=zsJZbzRQ&Y&zk4Uff1PJP`!A8TkmsdVk(-i>lQ7TotJPTu|S z4RQJn|1-qo8XCB9~K%I+n%|usxM{kMw*%gsr&UZY{us^AxB+ozkl4_p8958?Cv!jeA)#Z4+^?3E%g@H7I|ga z|C#%BNA3atgFoLYpVeL;E_IrDTI7lI^SFK;e;xDDYxB>;56sKo#r)p1JaDnwwZEJ( z3l8MU*VI3*=Z!n`YHRTCLo+}L!=r)438ie*D27v+@(F0#oA5bDpk&NF0d-K>S?jC ze{(**@=&Mr?%z#o7yFg(&&!UEN>AAJ&OJs;By7XF-=#8nYo1PAcBm)3?(Fi7v)MDH zOk%f7Ugv9+v#p9KlwRwc(#D#V%NoP}LAn*>?~Vtr_+^wV_FZwPXa2q7x!t*y2TH%& z9JVWuJ}w!-_A+Yv?DiAad8as1PE)m&`N>m$>D>|gRfcnj|_=w-7G zO#g%yFuc?EzvjjS6%uG^h_XKXm2r(-;!j?|JKhtVOdA(X(e_|nxBKVSsZ2*qBC=M0 zSo_ZRUd3bH%CA?$m-^4QGs?WAa`ME94b#?Tz3u(uJxyn#eBBR6KijV%uNQ~1a((d> z&lTP%U1H-=eEn{&ult*IyDRUMo^3YajmzA1zIU&S%$w_9Jd(Hf1>7qBT(Gx_qvrEj z^KU!kN-j9gbHI9UW5JeE$o|KT_H>HLVAU!n<)T_<%OIx&VV z)v8*(qE~`@&7O@85zSA{uPOGd?iP-yIxJs!exB{x&{vhu=c2Xs-3Y zfXjD`LSxrw*fhRuyd9c5{pnxz5BI9ypFA;9Irr%Qzwhf)k8}vm>wQq(A#Qb1;L9!s z?ShU6KZ2N*EaKuES&rum?tlCD+wJpQ9k)v-aIjHUO zt$TXjJoRzuzXox&DMxqme3XtiT=CR=+tv5mqw05;zO+1tFMTU87#hweq_^;u2r>q!P-UJx$eK)_RLYTG=I^) zz1P=9r~l=5_OO3CCHTo%^ZQ$rwR87375+UdtilwFSw!e+8iSDo5%d2uS7gFU3eVER`T2NoP z$qdR@Xk=g#*sf6L{GjRS_f?^**CfbJIGfj|v~Pi3MCIY_<}F4Mo18oX9B+&6WNytr zyV%JRRHiY<&Z+rukp0uu@c5IUJ-fQm+jOqS`Uq7V;#7~|F7jD%b6xE2qRJT#Om#w_ zp@^q<%kNKpy?%cj8^2u4x~v_dP4(4tE^oh6)cx0Y%4w0MoK?LSJPkux+C?qa98Bcw z+w$%0ZSULLa=-86`Tf;(hP6tJe27oUb=%JyY(#Dw9ZY?;Zud;D%O}^%*w;j|e3aHN zRwz6CN&1m_Qbe8epJUU%!1{88#ij@#>Q z@HQS4Jrw)Udd7>j*9CgF+Q%8T9NYiz>-w_m%KbJ$S(UG+PR~>c2xSszkT%cDsrLsR z)_Zevdid5;E41FnX@6M6leDkvfrPbK46}h@8mO+yzP@g$({^ity=gl#AFkQ_XP~j*Z;1%{azA;L(XH=Jx#WxPyLw65@Yc20ZO6_DG_BK%*x=A-|F1%OU;Uqt$FCWgZa8pis`lFd z9~L^dpIX>1C$(etZIysF2A8;q-m1vfM}OYm-Mw8WYKsS`O;{Z_Ra1spy6WtKh^s;} zpIIEfgfYq}SzP+?`~BV3+m|d|{#JTtyX&UAFPALl-TUl8_hcin3x4}*r_M4}pZoNr z{)ayYzZn@X{M;p~{iZuybbsR4Ue?>|o|+%Y|NB4qnayLjJ3yTp!&Wj* zmD}^~u5sNlJ)`CRw(u&2>0O^fr=0tg${BKG<)SZ}mVS@gUnRSISAg9A`vx~v<4z?c zd542)ptuah*t_#y{8#MhZDg6!P;FQ7SIk5{8JtZUPjCh>^v{v?p2WcEqR4p9!R?`A z;`B#-40882yjW7ERp-2>h?O_?`hDG)*$!XYPOwhgWppw6XntvaQ^e7tts8-w%@ zY2*EtU4B938UhCujaCWt-RIdh@!poqVC|O2-`}xaXfzKgVoFC*Z;fuZ-3B7>89;m_XQ45?3=#zNBy$@)3?}NIqSfr@L-;B)m=qN zueNQ?vOV z$*CFtdAIxhlY{K?Q&z9vw<|_Y?Fd`r_j}d;4}_jS>ek=(JpaLpW&ZQ^!edLPetCaC zeqPNd&uTgLXPwJ)W8@W@R$EW`^QT4|RLEQxjSXQr{(iUjbiHf-R@VaW{XP47+p&a~ z>t35BO;fjDU+jFId&l#GDkmy3yVqC8e3WLg5ZhsM(C!a^@qU-0^PY3Ec~lsf-c>dh z_cwytPaX&AK~1XkQ&OG{ELjs7@)fcL?Pm8?GgV|9X1d75#HeHZaJ_`m`hRAo{A(-} z`JO*}cB^(O=l#Y-+jp4s8=IXkBa&+^#V_z~ znVCf=6x(k^85F*HyZwILQK`p^cf|RBX_ay*m@5;!_TW_^nL`%q)O}}}Y;>|{ai3Do zypDC@l7&7iD))ZcSo7x7Ved6hPfw3tozHP+!>419xbDAtAAL0c?|hN7|GV$p=e(Xh zuc11RC2n!nKJ%Q3AP*%?W?=nr!)Bd~D7b)P;JjkND7rdfBc~$QqjjP?&gvAqNo}7t zM_{c&grkCcL&4g0elgjgq`Ji+`@O?O>wuU7d6jG#ry0CWH>ZZjX~x%lZ0(V^k89(X zk6XvaQT_N>?`FFc#d%HQ&UcjvEpWbyXB<`NS< zSjd}od)wMA*|+_KguX`_FV?HBHP~gb&hy*uO$_L(Jk(Km2fzj57$Z^y3t zRB$psye+yvl%-vD$Lmin|HB{J6rbPvp^ov4FRT&xfSr-+L0v$KEUZ!ZoJB!kr(JdZ zub1lDZjNh?HrpT3(%Hb)ZmNDsp>Fk#w_W!|JMU?!IeaK{%@-9j>D%henE2<{>-F0Y zA9Cpwk123m!@KcZ|MO1uc@sc~V1SN%umAUXzU^`5_z$bY)=skj^N|0VZd1CNT9wd& zudLkbWFr)soI8$in|0iah!K>zE)~)BSnBf1VD;C^GNO;A8mmfVH|E`T`Rx8VW6jHs zyC!RtH|*Iyb?@m@O-A*1w(L5@|NPperQU0I3;)S)`oKF;I_#cSO|op}uSa74zu#Fu zsbtNmMnh=iO>j6XynrF@^O>7UP>~}tjH1gEDu15V-ydNpqtw6^(xF(VEPsTdJavF*5Nw zpB2iuw`XV0ogEkFZ5K#;{BX~j$V1i9hb!0V-zr#p(3I6_dsOpk(H+NjyjyWJ?|7f= z+mtO&`*^?H-Wyf+a>5s@r5oRI+}ZY0M}G33g1CqEH<$U&e)C@H(KfF8T6N0DnzGmZ z`+w-a&Ue+d^Q?P3!5tP*7k4@X>xWxU&tHRzEI23JpwYrQZKlAInA1L6kF*?BYtiOl z7T{1QTAk1!(X?G}YUe$MFuxCYVmc8CNA){a#69e<|FhU1Se15U z^RCmsb45$W@RCB>qg^Z;H#xWSg>hWpRiJyZ^h)lW&?$xai+-ETkE>?794OZK^tIXk zJ9AMC%66zyC|Jxap~$S>kt2@*@~apdDH=xQUzpLG#cEG7bfUK zyK4*12{j0`s0Q=|UF3Wza9pu&x844|I(xrvF1&sznpM5`wa$vHH$NH}nL*c(ZS6_y zytAY5aaNA*zEyR9zg~Y@zW?{!+RWyYI={cZe$L5Y*CcJ4^<-y%=Rctf*?m#X&Me29 zviVr@MGsvyiitn+(vh7|)EYp5#rBr_v`2A&9@UC}>AK!+GWiCyn0m?pH{1Cq}Q`r&zZ7>vpdDSzp)2YwrxZUv)0TL~q$-SjXdv4I|eB)AMt5 zplDyFvi^;dSK9oh}avfP&vdHZU1h;7ZfdP;AH+Cn>)nDUalclms_&#eFZ_+9b! z$U6D+4tf`l@c*xVc6N69dBN+CZj0{QxK8YTMdRzJ+T@%F320q3NMl5j`v(3M@=N)+RPGELED)^-!zMIVND^?cUgf(e?YK+Ao-FZt+O? z_~_`RwbACYt6yALXjJ;@ih%dAx^1A*@_iCJ9=1s@$-chsDH~5XNBz&!@oR4L>`7?| zI3-}q^5xg!YY(d4HlA4fZbk@CIHSz%EM7iEnbg^)*=el>J9Zo}<@ow)>xxCCOCM~# zug&}OZppO!-Um__y_RA(D!%;pS^NL^55IZ0t6LpZb$RdbQ2IKN4XXltMDdLXdB&~L?QJ<)x4|HuEi zzf@i3u?Odl=M&=2E3NH*3W>NqpytZ6Lm@&6jaFU_<`YW49N2ifbRL7;jgr8J(ejTz z?EVsEkn!C-`njUZPX>ji^>KT5T@kX`up87W0Syd&n^E)U{@TCn8n2v}ZKin)Bnl

r zmCG9vowZxcGcUOa2@Ab_GQ*+q^|ZMAOcz<+UCu2QO{rY{;yQ2LI#vsvLoeT{eDHa` zZ9dDL>V;kFbnCVq-v#oCf*hz-d3^CX!^sXj!Vh>;c;sXFbYF2kSm-N%$E2)&@~(A- zSNJu~Sy|Nmso44N*Xy?{K*!5j{{Qp&wN0b?yb7mvyIyI5)^?|!nxc6v*YUxE4?&sG^%h^>s6XYww8HDon%y>elI3%sIx87M$bdzws-U=jHjC(776qoHP3}IQ z%+k(C_{U$L!6G!-X^+xx`$4zK-E|qU{miG|f7M*P zCu_3&k-Qy;4sHzNE)YHNwZc%1aX-h4eEF|CpYM*j=;4St~)7wyU^n`IEi^l># zp#uH*2z@QRiD|LFJQwrTE~z`MXO#6tca!aUm$GNmJ*^Nn%x#$!U?yNP9;s6R#=T zA~`4ZoKf4sKi90WzDelC<>mc@p)*7J|wj)+|DtXkh7(H+(LI%Q@@=!4=k3#_-`?7~IqX3}W7ATt<7Zx581=F^9I##g;dUBk4=zh> zfq7d;F;`dWtuCdRdzVFs<=xXR3NQY=W*&>jf?G4rM8%jkFDJHF!SQqwCt8BHkNN=b@d27LxKtq`=53Hra4htOKsIcST z{@g?BGiSI6*B*R-qSyO##;(WqdrIWFHrGhzo!U5MBLANUTuXgtn_c6QX|h;*V5%OZ z!^SAIBAP{`;riOgNAFE?h!MWf9&WQG@0{ke?R8p{UTS--NiC0Qe)zz%fkjD;!SK(9 zZ=6L@>qRmrc`Dk7OxRuiUaZ3@F7ffP-fKxhO%E-f&r#Oj_hV7GS^TAYmCt1@pH2zR ziT!@RzJBHZqIj)zRYPxw_o@P~n|^a~JUn=$AY9~nX>TaY^@!I_e`n5|xpa}mk-~@` z<14yMXJ?yl-*^8d*U~&$^Ka8YjdhO(mKY|H1F=#u>Wo4iUQF)}XElCI{Pe8f_0zve zLa)!MXjz}}0}U-$G8N1&O1kc%+@kKy^=HM52dQ^sp0kA=mW|t8wzlNOg@d~~f82dv zr@nsQudE}s2Uw;RM1NXt|JO4hJmJ9qw&nAxyvp9)$yDEU#HEx+ML>^H^q`itO4I^4CT8I9zgd{kUX_%I&g*|FNsnVy{b7mc~Bb z6(Se&!O2(ri)@my&I{8Wk1v(>PISL+daX!m_sd=D3RgUxD8DY?gDdwsV~a44!-wFF zp#XD^0EYF_$2<3`H11Gcz`v{_BW7P|zm47Mh_H80nBDKp6Oggs3^8u}=2pJ5C}7i; ze3wS$peyrhzs)=|$1+%N=My1+`G+4A`NXwmn` z4%rKpdIa^kudJ`~^Xp4EXtrag)22`C+&)IP)5;R&HTmR+hlf*NU0GRluk!iH&*$ym z&us5hd0@_Xr&?i!#Or(AJDPTI-ICh9X#4KIzc;Sc=0`%zj0BOS3fFbh zPxYD`99;arF|sM|f0W#{U!Ya6U)lKtWe&crzfiSfGBj;^EVu@WG}A9RGgv$nBp7QS z965Ym{PedI!S~fY4CNfaO{oQJ94b;9Gp;naNM16WIeFgCGu00dwSuZw?mJ5pCOJg3 zIENG+o}}u{q8IS%+uLmQ%O&3n=dxsVHE3%j$G-OsnG?1#tTiY6&aquw_Ztfn?52bt z6n`DHUOGAU_kzC+?7oMsSkgK+y<0TdOif*VI@iS$+!w>dxAU!0elcTg(;YBCmA2-fWa!b7ZRck^jefrM)*VJFBueZrNM)e;M{B*JO9CEzG`QY7gE0&|W(c2=j&m5?|qt^IJWdYMv>Ezh=i6+;6OZFV~zq*R5C!mThF(@y&DQ4yGNu zf-jmJStr3a|E0j;hvqVeERL=E%eSr1Y@)y9&f5o19sadD`FP*8aBG)>9`~IK_}D;A zo!<| z^1O!Xg>{adcPdLZEm`(vYgmbVh_`X!(kTovv%b8%d^|6{58^dS4ySyz% z?$tk+wFg33%9BNpObll&{J|M+DC1OG+RDZ&<*~c$t(W@ID~1Yb3XKJvJD&Ic^Ej&HTE5 zn}TocE+b*e>-E!rzu)sOjfkjx;J(MbUvAr(jmPD*jrZOwv^`i|xZ<^zQ1+UqcO_moav$q9;eEGjS>~FgIbFLvu9qHZOxlsBye{Kn zQC^&{bI*d>Fztef!>f0!P1?T8g;V6v#q(XygKwWDGo$ zLBo908thIyT$uWHo6XMBs#$L|mx=nGm1EuJ#-tJ;*!bj#(9ZuiSTgS^EDd?EQX_xO za{GTj9v7X~-G1Wb^7(p`)qFq6yaVl1xVXr*=+jAc?iiL3cM037Ek5(@V&Cn4ZwG3U zpEhOG{xmyp*U7E~P%~+{^wgr(84gTW9S(?YDY@IbQSQ{Et|xhQzSpfb_Aa94&#nRLng%lcJsTR2GR@#|z;GXCCkSi5|J@Q(@vt@Urhl1M9p?&zwJdjXW8GzTyqnr)b*JWC z*@n_L+&t?fo^M^3nb%WV?l^bX+Shq9JNj~Yt?Z5*JW{}MXWfgnPCKNtxyKzG4{9zic)lplSPfhQe`p6yq*-hd z6H;j8;C!IQ&n@TmVZx?V_r5zed*_KvQ*nz>?w-iNIlQc6*wy4v0xi#MQ428taaII{$9=PHI>~MK5O^>O{L*;F7KJO%C89JE}EFT&EI-I>!DniMBGG$bxsyZTUFd^KRo3Ql5S{)v)b^CqjGOf&mE^iWkAIc0qzNFB=Fs3!Pw4TYk zK=jB}t(fhxiZVY#dD;c|9Bm904O6ab6fC~9$F(@~=BA~h2Ve5r%uznG2kQ6$V^Hgx zCw-d}xN&4taDa*bQMsRApN);WYu&w>fsT0~r%!NrEA--OMSqOF#m1if*KfU@xPPtx z9E(7`{eOz0HUCt{m)%VLXC{!kl~=;R0kk7oyC(nE7R{`?J6$Br%Gz17dK`kO`y-UIjU~;j{cfFsWk^C+MAZX4|m%5 zV^T<@0VE+QH12Y1V5*3G5@%pe?3zN-lX89b^@+_{Jn-a8qrEOxcaZd6hFfm{bBbvlQ@m zF4(BuvbwO9>tV;Umj<~Y7w?YjbE_==8 z%3AycIxh#BRF!2CIgs6M9j(eJWWe&_0#mlEP0pKJTfN2Bou24km*r>%iY9@pEO~NM zwC~P(FM8nP%H8uH9Xrf#uX8=7xL5Z7ionHBHXfJbzNw#^dSP4s{k*lpg>T;(iJu;}%OA5A>YLB9sIQ^xLC0O!)`C0Pt4eop-PY=OymbC_ zfpsru$tzi{`^qbAw&mZU(sK-3mMd8#Eh&F@XXh1pi!}$Fzv{}BCfrm6m(q+v9g0jM z2a;bT6ltggT<3UjkcD5Q;>3-O$?Y}!a{Bk|-BlSY&Gy@yNkt%?<=kC;_r6ziqK;i( zEzbekI0w4C+)G!=y-!9nfA80@h`N>L`S)^Ir8qt}GG4#U%xB^7;PtOb-g<%&&BiS1 z4eKX=*pzT_k?S?4KgXo=zvz6p=q~S>mbUD<%j|iq;ONL@31M+(jyV2znOpwZ^NaR`VkTJ8V3?y*IyIA;1^^U~SY^t{@Sm^tNC6bud+eVjXAIQ7@GWrb&3n?W7FI;O+7o}GBPcanpmD{liGsi8aQNGPopK;+Pw;ZTYrA$kHnTlVcz17abi&m;`|IoD zb+4Z`6>;ViIQT$*kMC@=d5f!FE}j15PVxDzznA&V)pBm<`+DZMi;zO&EY*f;Ul!3L zrfm&tohl4Y7yS8F8mL_;dMJ>OQD6GwC+@tU2fw9D4?hCUv4zQmhihzM+P>q*awevC z+(Neb+9yt>fa9u>12ldmAOk98LwXtV-hKGMkS)bwbbDLwWO4DdQ~kTY`hnd+lrj|@&-aBa;C|50|dU@Mz@`Q%C8_6fAU&)?y2@IT8HPC*%cFU3jWU;n$9 zCMIqTbULuN{X(&+*RErEp!GHmX9N~7$Qid!VpeS2<;q}QUUA|;GwYxF{JXnC|7|sh z4ttV*b#3F7gZGw6pXcy37EVp}+xX=D%4X(->8Axx`1wyb6Eg9t^|z?qQ%@gYiRUcI zouGbQZp-ngzYi~$cgVjxoNdvo8@cI7RCWH14F|=tudkbXW`?2jt!=r{>!fx#M{Z1N z-TJ9*g^)zrxjCBQ@wHpGzN`E5@%Y*YLNnTL{}XP|XzMg7^jWw2ot6J|Q&35Mr|7gU z_`X}^ew#=2>JiGE6C5H~ViaSphjN6!F1WLN6@S5<^=w@aZ@9%xX+P2XpM3+{@$2RS zhc>QbstEn(-(qTzwr(=ZoobFb{yd;k!GY9ArAgxJGV! z&P}C${T#*FU9Z{$W<{=YO8ByHPR%8I$YQ@)-Px)~erm!GBiw(7}$x~KZCzBR4Ly%(DlCpYE9s<%hC z#lNfDfBm&I+tIWemTit3E^Llm)MIVs`|oFJ>s7Zul`cAGc#b`fm{5K z*jjo$?DFIH*X!NAiof}bg-{0K4oOgb%b@;Qhv(-U2Zq)pII(_NCrN{eZ zwLyE$_dMw9Ed2KFZZv3~=s1)8TZ?NT&u{g2IO-;PQ1G!_s5r-+EYWpI7D=KgcX{%*O^JWIiWagUnDW?mtM#tL-@Mw@%*8T#h4WOXvg z>3q1*;Mu0o;4Dz_<9XSp%@!r~+qYi%YIXO`!nc#3__r@P#eA9j$?U6BxtDsnanCa` zij`V2b?%-__qxtMg+=p3Sg)rZPP=*`!f1Z6Lsv=uERD6Vrso!C$93-c`&jO0!N!>l zr=FK>U0nZSx{`W{zJ29_cl8KwoOXcO_?y}!F z6?MGy{R!p%E&Fy&6n8rL`~CiW*Quah&F=U6v`f{vg&Kkwk%v*!0}ltELiH{RvjXs0NFlW9}u!z|HtO*^#P zlzG2zUB}V)pI2aiEThew|AL97b7sD_Vf=pQ!siX=_`8@DK#fxkWd}x^Q+yn|otZv5 zG4OM1H6%0na6I@Qb1CwZ*QH0Sj=lYxCkjqK-tRG4^;p&K{Q2T7SN=7Gc@{kucpQCL z`Oa_G#+u#RiUjX2UXd>3^6hq?*VHn%AG~7Iq!oYs%VfRAWgW54rF+38E8_pa?=mTuc>vF6y`Lt8{$ zKYa-Q$FX*%rrRGot57R5F1@;Jd$AmyY1Ve9&G{G4dgQ-`>4)}buBW=mxv`NFr2$iK^Ji*ye2!_Ukv?X*_OpLD`{BNUpmuu_x8kt!E-5vQVk6~v4oHUwm zVd;EH%R*&_!{zgW;cYg0==cUOla9#H|e_oD}-Lbug9zIJw za`o9#mK*&>Z;F{sK8PL5J!9W1`lJ6?)t_(v&erGR-WupyG(KVZEP8Z-@1&_yQ)8#M zC^sZ49ohYU-|jdSsDuvf7nTjrH_knZ^zpe6xm z`shGBX!Xt0leat?Se#TAFje^Z^=)OizoT%m&5k|&Q&d{g7-f=-&wSHv?B@5G_%L?T z(}y!zjdCk9^1SCr*DR#&*>58n))zOyJa9OloHWUNVzKb-o0Nqr&z33rCCJsGwy zpOk3c2v(Xtvs3$f%ZkoETP0tNW8iH@7xW z>+aH?%Fjj3aT^jEMUR6{Z;>=ia@qOUM}biw;9{A7;led{xB6Ur6Qwo#^R<)H?tI;} zx?uGits9BYwx2Nl9Qb7B(gQvfxiK|U-fYY}?xnKne{rGXy_!8s@8?h5Ia_zLpty>~ z2@UmxeZCvwBh=JXyn~!3Em2uE>!{D2SyK|9?K(N>m*&k&7K>X=&St2teN}2Ce7W4@ z`^)P02frTmo5DZwt}>|7DPqxRnC^Ut1=Mg>c);}_dV8K|M8p5$(EqbCtlr&fcs=1Y z+wxQ9tGk~xzFf$;XokV(j}u=`dSSEW;8HfVs6D4Pa_JkLKe+hb*~xQ%{yHIU7Z+iC zcvYmcUt(v>kFd(b-Qk;E9;@$pqq6e(-}rSRA$J*KXHP3&E!Y!NacECWgZ7l}`{yR# zUG{z=i;Vn++7l61@^8rQ>D%4AwI)1Sx4+XNU2>we!4k>qrvCFMCAM~T-}^S*KOy?# zp4#({*=uKa6|fehvN8VIeBMqwHf&AA!reB>JC0O<4)vVmt@rWHc^4rD#vfhtEsK{W zObR)(HgJDkZIdFSw}b2ZIfr76((f++boiO%rkvyc&z4qXd}loZY838gnbhlW&v147 zoBK}!Brkl*Fn0WX@4v3d;m64^L!5ZO)NdaqLk9=e4|jK$M~iDno8?Ss z599A>oalOAK+jxQ{ZRY2&G*Gl)z|C24qLM5-nrNIuY^bfhmH3OryU^a*g2|$0 za=+OjQ=Lclzn6N*{a^OWl_lcj;pY`yo$IUg{msH2XxGfC47_l8j{(QgMR)w`K89Ss z&leM{)!>}NbCqe2lAHH*Jzdem_Qr>PoSLeAI(4p1<)*Lvae3^*9uLBq_WymGzqYjE z^|iHc!`Up%nVsX}=BSrHarB zO(ucE>?ZbJZKW2`n~Z1OOw(++E>gSAnMox;lu6{kYN!5G@Kodjt_N9+evCFra~!$- zjILeX@M7P?!!?l)6?Ljb_Z^RrEHA4mI((HaCQpH7ms@1UpMd{Ye=J^ij$!}f@AA|2 zW;_Wz%zy0ux4FmOZ=3JPTJS@@K0h%sP`;^r+QmtmjD?mg7JTDAV>-(V{y%m9lieDB zrr7Lwx=sGC&1(OyIhTxY?uu$WJ+y-w8KFZ`2H47U5mc@t~3M_wO$+l`Zza z+xc8>=gVcYnKl<#yK$^IT7Ga|jdqQHwDC=j`+w>Ml-s8tR%nb;XPB%k3B-kYJ~z-Xh!ug&htbkU!IsiLlK-&aOI#y!9K#A8Zq{@yNmygT^G z8}p*I(LJZ4vhON5s>$p>ugdx9eyZB+Iq7P1T_n=fF7G+wvuY*druSF>8(upxb;r94 ze;yty&~G|v!JobV{CA624etwoD10|w!64@~H^J&=$CLaSoYvf@rPs6vxkgm>y_}=7 zn9oaa$K0KTS^L)9i7Q;pd|Tz@*Y>5S_@B+V`|`8P!o*UmHe+Xo_W5}`W~|_2v=O+w zDs=Ud+}qn4Pu@{$+`yp|vm>DD%ZtR*Jo0uic0V38Z#ZRKd{<#X*!|VzdK0JD*EpCx znf8CtlDwXly()}C9c4_qMz>Gjv|T^_+E?pn!8z9+T&z2C!(ONZR8>Cyexvw_O2BHC z7)F_}-VcobKFngy-s->c#*VI=1&KwG9q*jZ8B9C8T$($H*Sa_Lvqtc?7vaq5Ki<^c ziJ1C+^HH(N)VPIvz9vU@zkKH_eD;ZDp5EHR{2sf5RaOxz=Fd{7H_CJ4%Xs)B#*kIv znDdnH)+-qHWx02*xf!`Hzu*AqvMC=x6A)soh}iR6tiC@ zD~>@#YVZGx>$#tto_esge#8FHGfW~AT|PBd2!!bGZO0Z-1!-|{?gOaoAz^d z+Nsm_ck|;ee5!Z&BBS`Ng(YdSgICh5Id>nk-Q0guQRHCqXMy4n6#-#TJN8K869tb2 z?*tbx#2Fu7)BcgUcK1j2y{9-9*7F5&)V^rhw9d`W@ZVZ>)*svVeZFDx@?Oe(jklry zx4P|+%Tar;f@{GTV!|2kcwgW@bNkc| zd&^T1sq-%>-;BC;KU;0-pL6crB`03H&;E7(-DDl!OF}8dd7U;tQX6%Q^wPDsb$##u z%j`?G(6`fCJ^A-(p#Vl1CB;8Fzuz(~&B?x_)HqA|1L)vVg-Oabf4^KVde|y{E#`5K zW)f4@SKg1mUvHH^HM?7Wve3#u|6PypZc+fP^s- z$*26UPe;k{n+gXdE%Gt{bM%Kq{JS3#>$@20_N@H9r|#JLq6WsFYp%L}>e$$DZmpX6 zvWFZGtiIhZxqmyHSHGTpeeA36P8pvST8vx|LRC+E zy6E-b@Av!D@Bevber@adEt!{79vo*Ke} zUj72L-w!daoRr6Rc>A$Me~!NAX4Q*4#M#^XdggKWWzo|$pZ|-`vv8eh?!qK;z?tP} zPU$2DwUa)6e$&3ay!;dt=SOD=y+#9`HT=#QbjRw#e8_&UdUGnJNxUTR{WeYZs;;RuQ<*vWN8pBb!&l z$^4q+Ri9p1r>mc`J$$L>Xkq7{mw)#x_kkbmV}+!**jS=vsl{hfAg7k|GVeL-~Uo|-tRXQdq1(IZ<&6* zdF`z)uMInw@J-`%Tj26eZo{!(49mHvB+S{NAI2$D@u1Nt{hUm;>b}a)X_*%mIIiI{ zU3L6v*ZoS?ou6LZd2RVHHALR9tdHeUv%}^u+lqYub55Ez$xqP6j?dEFlc}_cf%St` z;fx!eLLIhDaZEpySO50*bYksFe51HxqM+$np9${ulk;NRVm^cwct7~JcG8n&iYr53 zP1j~emz9gdlR^CqiY2qh?H*-6c{e3vCH^nxH@sGUnrKP@MO?OO8Rc2g3XImCrtDs?0mXq~y)S%uOlY8}IxHe7Y`QDOWx2wCb$uHswJK{nrnl zFO+85oPK^9XbC!Knd}+k^IuYmzFrNFHrJft@p;AKzE5WkA6X^Xzz`P_qOxOoRsGA& zMzw#RtO`AG2{dx|K)6wHt=~jptN4WH9t*8S<}cVJC8W@(1uFhjYjk{=R0K2`^%?(c zH2u58<5bjtw-Gf_?`WpnbIL1`)>JIR95`1{Zrgq_-(oO_s9%( zqt?^?f{b+s?2UfTpR@HmuUEp%u5E@3pUECyazA_b*dGxY%_A7GA))chySumhGGMa{7hWt%))3fl_)wMQ1F!Y96VCQA%C4%S|IhdJ z#cdW>abWuBq@bx7>Gr+aXV$g+%(Vv;n?NHV6C83u4bjEc}RBQ_#JBVVT~V&DB=`@4*b`9XET6{g02zJ1d!?~c!XKRJUnj+?!~cM=2V6h#Nd zJv@A%LGCLejNuITca^Trxw9isZS6bNpOfN(V>@P-`Msa3lm5kTVuILJXDgMHq3qL3 z>x<@ZlxgFT)tQr36E;7yCT#b^e=k{$w0Z7dovF#`R^z|^Lj1Pq@&~^{k3Ik4_V{_! zVZ*tTmxB5Gs7k2={J6V*qKBX~%|GPf7d29ZzyNADg|F-z@ zlhadOuYK*jJ>TQHyp>^W)D`auukBAvpL+Z7``{&Sk9qGrv&3oV()(^FPn{CtdhoI6 zisiWi&IkYh^vPN;DSUivp6mR|XOcT#Eb4x;Ls|4*JY_2LP?X*|x1QruHgkLU_1G|p0XVBGU8 zVza7{0%JRaTn{%_&-}-Cce}EFSbqJ>gjm&?etPSEIWAZ|v*S}<@=Dn=PL*c_Q`P66 z(bSoE`rb)lpKB62TTGQcR6PEFi}xYd&flhYU;jI@R{6?T2gaHy8*A*2McDaA?s1=I zIN763I)ov9|85tvSx0#{ylri**(Lbq_3U!i-b>%3em|3*`}b!)H|x82B~Ev#a@BB& za#igf)}NQwrrzcWyIA%fsy&Jt5rainKTRvCzLH7E+YUy(d-QMtBdp_~tuju!Sm#wV{U$(CL;B&DESy0mX z;hf+yoBPR!GUcCJ=h$0Ba&t~_5CyH8V6qWYWfba|(_pt_C({q^)w@rGWlB^o)p_a3 z5GQHinY@~fv*_;c%1veAkrTHr6f9EjGugc5{ng@J*$wmRzPx_$&VS2Bc7KD?>-WQV z9N;)KeeY!L4~lYL7Iz}%$IB+v%6VBnS*gcp6Y;x3bHmY&)3b}cpRgxq{(JaH>1Jdr zr?$Vb*pvOY_pcGH-SDmS{{0^B+3#&*Zmsa1R$g~YS^qrmf7OO-=XWGAoeeH}a*KIe zAj5hI!S53sma;t9^>|^s+@v#Se11Nb|NkOW^v93K{ii{vvR(=Hp9(ru?UL1F#pYlC z+5{P6&f0AMz!y1lw}D*SIYZF07)K`AS6`=eSKZZHxpL(eruEIo`(xD_Ba|E%ZKn09 z9d}|X?P%b7U}YCs+@Jsb#O7mBmeUTavM-zAe^6D4 z`@^nttm_s3WyX1!obs5ky7%k%GWoxYua}zDecSK+%kSsof4kRki~P`E#=2vENu)UE z8|4d|)BS(GTs}YN&@|m>6>ID2`{yiTgYVz}`Sa(d*cFSQ%rfCDc*Hw2on4?rpQz%zxfqJ)0U? z_ex35Yj!8H@@lunGKb@VG^fSY=L+wRr;RMkyJL6ZY9Q^T6 z?rH1$q~EUYtQ=V?JH!9x7RB}7F%o+)=gvyE3!5ieC^K+>oSnDJ)6e>?N##SbjAeUY9@h%XD< zcJE2q?8{Gmc9ZzqFX)kMxFvsvM!D>4wd=kog?}9A+ivjfgvraantk!V@7vs*CSLz| z{mRF-J1(+W?u_0jdf?$cMzIGX2eP~DZYws{vV^UV)0MRvbW4-v^87BHyFofLI*jifYFs<{iT<-6ezQU~=DS!ZrtdNA zF4?1Q7O(4G<~5OxS1LlVhpA$TN@O3)Dz}8YyGlVV#n;@wo4EBRq|d87w%O^&sbk(+ zKNhUbs`&Tw`O`(+dXu(nF_~$U+QlPjv}9FiTg<)E`c1{p{bug^l=`#o`*FqT>29E@ z-W9w|?50wR$4f23LA!HHXSgwMEfl$-#yNq3wO}4IsAcKIa4l-a0nYg!^6u}u%k_Z! z`qLA;cQ$7&wf}f2V_RXPQJu@IGX_i_gEuFicV$!iqtoep`1zD2d-d|Jz0sLgedfsw zo~zyu7Jb|AasJv@Z`Kb_o}@%IgzFbw^J5Yfnqi#YSM&Sr_D{d>|Brhx+1{Y$>(y}3 zqVw!Zi{xWGFR!hQPM9`fyW*1X@9suV(~HeIosyb*Qhk1n61!YQ!iiFsSr<#|o9Z6i zdpTFM|MaFq@0Hh1dg`PyVcHbMn=j9%df%(k)HubxQDn2LapRgR$0kjT@xS(bP0zEu zV?OtHrO9Y4FW-4giub`&#lH>5&n$fE;IY6D)Tmm{CvNS{w9%P?$-?eoC}Yj%XJ^}f&68E!9tv{p_ulZ&Ze8{ypI^%Y%$^^6 zDBbX3qHBy&WY?2l@4S~}PCk~(Qc!nJ+Av5=r7=s@An%UF&cEMo=SutAd~AuSdbu<- zD<3q3`l|Zvt*zVSe#rg@U2U|t>g%eYy;DU&D+uR)__yL%=UKGmPO`SR?O7x?yXUt72J8*}8&`EeKXB6nxcTyuA?kD0{5ohx=!-T9!ZzRc_Qns-ZU zZ*8bv`bO(c+^)ClO7gbskGc8Zt}^l8-cNVi&fkrVStO|AMDyCS8`^K`zxzP^3_=R-_$m@1}T+{7fbLJWLHP4PL)<56<^e!W`#ZM*c1 z3+4A~)%$Heb@a$s2Km|jd~)FK({0bs&)0XCt6Z|T{NTB3Yokwt*63PKE1FsBRq(g| z{k^@>+w$&e{g3-=elu><*+x*K1XhEFTXRJNL{e%NYW z>Se`sPAO~qE~qS?Ds%m%%*(q^1U&uEZSkGr(P@}>L#Oy!z1sY>KlXF6K3Mf_dH7_< zqK}M^53N&MUa)-6pT`wH3qM~^oW6FAL%$mP$))Uf0%yHl%KAN$Zz(6E_Vcg%Ja?si zpZYuf-^N=OtR|A%XW#T(`{a;b`GdYbO0!pCkicNzxo^L`;+Zex)kq|ms_@xh5TsYM~bg-)J3w~fDOOUf$g9bu+Mwv{(OO?MPdeR|II(~5?OB>&9Wcu3DA zsAQvwSGL;YW%HGHR+qJg*j@2yj^dqe{B8HGEz?teec!y%=znCHboTROFLha*4Ava{ z{O!q!!^Qt}UYF_fZ@o3)v#VCaV#bay^19Wo%l>?FI`Ou3lAze#4R5Xm%w*_GE`EH_ z{{)lrvHb@B<#dnjkm9)FpQ9sQ{wsQ>{__{?d!!sTdkHBtE@07UU_W|zE@;V!E5kG& zfdijEe}3C6BWIDwxG(Ln*vyy`ox2M{#4E0OGpPs|GVZY7x#cVWrB#vJOtP+=s5`|S zKP`(+s9e-f8u=!c-;;qq%p!8d)y7tNH0id2-6fuwR$;nJ*mg{nAoZlzIF&>u0r;w6t5( zdqob^vYc53nxq4@e^h+VxG;gXDEyrw&R_IPQ~%C_16+ST$IeYjEehX1JEa!f&Q@i* z8yt4}gjw_w&&g_+HY7U#7MMP3)~S8Jkub$mPf zP8qFQ`gwM@+~k+5?|(isPl2<@_5c3p?iZf-d#Aca@Bey6Jaf@>hPwOL>rI3wzMtKG zp;g{5Z>B7ex$tI#(b=;RfA{^lzsUX6lGa^?Iqk+v6QhK)Ki;$5y;^D!xZ+CbKMRKX9m(d#QWB+&tM=x3{nV zyNI{$pec`}k&E4r2h7#I7q+ZAW>wC`QZU#0>C(0JrhaCt(wz>j`nG-Dbo;fxr>0Fh zF8p*U>vw7UOY_@I=B&ALtmyT^^C$Ihi%;oVIsNJkTkPL)?bVg7pAtyBD2m22i){e1oy>#3B-T&au7n-iC8TXo-ao&1j#fA)N} zp84bMSx0ZayxOKY`|gK+y1(@NY4_9doU_dD%$>;bKzVnZlKWQcty@-R2p%Za`D}i6 z%JS!T<8Iln`{B+IuM%@Dtx5EN;qfM}2e152?VlDQ4C=du_%tw89L?F$&f>9P7AK3} z3l{By9gZ87emqcf;MtH6fi{|z5Qg$n)gd1FCPnID8D=Z!vFGu zvOC^P8{HD_?Wr_MJteZ+XSSK|gHvl475w}8{B-eo+whC;He6C{c)Z#&qT%)5OWtk_ z=DUQN&$Ik8lUI8FT~66P=U33YndRvoTb&;?pBC|%)9I=Ixw|NL=anTT%FFhrxPG~v zJ||oKIL}Had84^EA6rRY$(k{jqxj}G`PvnQ@uFH%@4W?Ao?)s9*}t3hotH_-QC2Vc z$9}4ohOVnO8LF}RdY{tp<-W=!bNt1+!||K#Zs-4eRQ~L!`_cU`FP+*cB*ys1PA_-w zbtV}m3m%S1lO}mR59U~Hlm=Sk+sJ{mP;suq0%?)CS2&d{)Q-k%4Sq7aN_mOJ1Geq2 zbT@++b#U}L1f7(Voc3pFN!FyOSHGqA)n2}Q`K?Q3MT57;0wz%@-)BmVHxwJD|CsTi z=*$d5qvB^i=Nk`(3f-CdLokG)oplG-1O4|quJ*5rbZ2P1|9y`i+cp`s{Qowd`y#&j z9f*5>cHPZd)^~nsYkx=l`}_EZ(abIT#Gl<)zMuPJ(V2a(ZXIx$`9YL#qQoLSwKHFg zD>xr)n5M6(sdw3@s%TlsiC391Jv>wX>Go;e4NrRVjPscOF8RE~Eted&Yb@k?ps~L0 z>ijvs!+&Oe)>%7gV?T>trsX|9g}&EHItLOR9JmYQ88>SQDKvV4dQ~EVpi#0AaYl{@ z5#Qe2RG!hmH90Zw_O`QK_XQ3<6iSV(bDlZh^h$(X20uN_Z$G8q{@;yz>qAVAojK#P z^Yc0D)GseCs0XZBp|^yc>%otxclWLTOReGf?&`rHc6-B;)~m(M^r=$`DxX4wsPvAO} zPlqMwNyf zw$oJQ{;Rrgmh)ci+HsV5sn~-ZW zh|*Q>y}It3miycg;=Cf($jU9|@&4Xk+u8YdcWo8jtysk`YZWrfJpbLy^ZhOy4|ezm ziED@)_`4u#U*EHtxlfNZ&z}&>dixE_jHks@{6X!LqN3}kQ(kSVKRqQrPGkPTB{J7v z#z@}#n6q2{=O3Qm=C&?<=1dlM<{l`D{hRQAQ|&&VxIa_Y+}L>G{Ii_cP{9L%j5@F7 z|L66*(Yy0nNke<<#*(}(^%v^S-Tx9A@c&Mz-~ns1H=5!668hQY`jgrB?NiFI-(hUT zByvFc@v$Hk2d0053mD$XdtU?98j6ftncB5QeBN!zzP>I^q;Y_D%_YCzXDF-&HlqwNIWo|5Ke^e0^q(`1H*}_or{p%J$?ud1+~& zyycp=9*qWZ0aMibYs)9ExfA&DI`_maFH{@qKb-bnwrY~mRaNz6Tya`kaxeI9x_tWH ze*IgY*82RaTXXjQ<%o^(MF(~mr81kpn^CxLW6ph%88N$JMH%;Gx@B8;W?Jamc}YL> z@?gKt`u36 zpU-=I`PbeBKlyWu7p16NH=lOi{N$sx#iyUxZv8EKNoTR|Oq07eCT#Y)_v>fjT^`N0 z+oD%G%?vL-vER2-Ye~_@G|KgpARmA-cyYk|!4-EDie zUH^JE{n%IEh6(S#&3rvkaL0`t_KqflQo4D80T|3 zu;}aQZTok?P;psQlk<-@ZjmD!nA$n=zKbqBaq8FK)a<|WO1d9?-k4CTBH+*R;N6nU zcSjguVS3Ky^zNU$GZJ+*rG z@vC*mlh&}?O37rHsYw5}BaCgUR9xgXqu=T4|GUJ+eQ=n$2PdkZKQn)`;o{Xf=ilh;Hog_a^ZnFjXi|xuY`wsM$ zi~sj7`u>K?NAK>sF77F2U#(BSXzV|6ZBK97qD%HuuALN}&oM>igsbkm#jHF0Eh2e2 zCpe^l4i7rkyA+h28XI;wHLC8=ZhHRYNyI~GCkvJWor1|ttV$J$i3`uhhr}rDo-E~= zmlb@{J5PgElXt0sp>x-y4NE*H7S3eauBpoD`K;0KXnM{2axSZkb zEwqhFNz$oS;q(sNry~B`vna4{$=jQXnXR{%+?%Hxw7+8a5*78xa z_|LuMCHbEfkIg;pdu*+Dl<3621}|}OD{uBzeOC1Mp4>C7==Bz3)#U}t%a?Hk9Tq&GShwx;D?RbAJAI{=m}y;>QjsFr@p8G*4b?ZJW$Wr|klR?XqJ})}J z{^Pavd!zX6qSsD-bSYl`zvKZ+=KQVmZGQe;`|0wuoAY8rr%Or~hn?PQtpD}QkC2+2LwEfu9nPsoG?+>ZNN4XZ^h zT=Lf>JIZl9)mZ#B<8R*V81UZOSxOI78}{$~9H;5A;)c`CWxvF--}1b#=mU)!{>*tg z`_sEe%Wc}rv%YIT*$FHuEefolLxwQQg z%TLc19Y6JkJN2+P1OL)F&u#{8aFv_)H~r?M?QxS=$%MNygnR$=i1P~B`M5B^aaPKy zox1NdL=P0MZ~d)vXR#r8gOo35=sS4HO+L;E4mUvC<9HOM>ke*o;AlTu{-CLrnVnB3 z_xWUhyNyqoTnkb@2<_f`$xHh&7v}_r5J3)+C8f7T*L+NPs@mZEpsck(^vG2!nM2#B z9bK^X-3q1y=O!?49uWmCQg|!-SXIB!J*IF)=fOk2q{AL$m+4OZct0j|-AR$hYEQQv zj+^vliOZ$oXqq1@b3xrC)XW%TXg)(=Ori3L<;)7xb%Wy=brnv-|8=C zT|R#En3T-47p+HG%ZyIW^W~fL?&jaKDKlL^%~w4ip=iIM&Zno7)2LiDXY%c3^JRDJ zzVvn##{=o2uv4Euzu!9TA&fg0H{#>DDj4W7#a6(AH@4WX;S=9_O+OxmiC< z=rjavzA6=1zz`=J5~>!)is&USBV z{HoLoKCUyXyMgP$jNdM-ANaS~D@F^2Uk#g{wCgUTZr`-xs=EoXJ$wFsUa~?qyViJR zN?BEL=8RhrM!}obs9NY;Z}z>nviaq#+J-sTUoY#>Ik2^Fm)zs02WL3XGn!m%Hf{I* zT^FK`T>on5n*4vZYRQDj(J{)r(q>aY6KmzgC5=M8?p?J?jW^UB7;U!o>$|%%8M-^T zW{Ym!*voo5;?>pF;a1ObZ*RLM$; z;hf;$BfueYz&!t+&itI(z-y|yJ)Iw2Zz~813#}EcXuguOGx=DhBIl`#W^Jc_8E<%M zV0j{9v;DujGfr(f_U515vDh<9+3zU+H~x1+z5QjsZQ8<+!v}78=v(f4dUUDSg92N3 zMl9pq7rhRMZenK5Tc2AllNV_|7gU0k&TrtlDbd+=|J|;4{CC2?yPp#TIogO(^nl?{ zbDaZ<4d-us>iaY4i?0so9dI#5+@P0vyW`)7-P9sGNWNBiHgOE(Tl zf@)vxntk`JKL_26++aFq(HUvB#=n#JCM!qJ$%qm_&fj`KOBhcHH8Aw;^n1XRdc03IjNyOY=k!m` zm#%D*-1$_yvHRDbP%HCW@$Y`V5_){vs!UDh?Y$lQ6O_DtCUu@Per)(9uWCcgmFH%a zZy`g7yS{S%v`cyvV%Br^Ij?m6~hv!akfV?1X7!}FEkwR51sYZxQY z$APsVqG2_|zLd)~Gbert68XOCUAgY*A8r3m9~6Fi`|JGE|8=XDyyJgyUGRZxfsJ&Z ziZu5R>9d7*Cv1+I_vhcW8J6oVT1{A(Io<1e+VAPY2QnuJtab1CtJHV{w7A)4pZ3aogN6iQ$2)>c_^m<#j5hZ6@nvsssYY+hnFyM0|Nm+J^tYPXtGDKc zfw!tm6v~LX6V{hqG1DaTlFOz|n_O56=JwcJs)8IWa|qPdib*=-kLooCCQ}avCW|`@ zL_|cU2p)Kz_kX_5TJyzh)rXdW_5vxCFxh^W(PcN2&0_t)#w+EKmbOeqU7eft!v#qr zP=RXCqS5gC@WibX7&u)x8NUbbs`!>Gq1xVj7u?>KrCq7+%L}MUIHTp!!170x;k)|bCtR6VxAdCi-HA|WNKZ+LV3awZ^R9UIe%;Mx zAY(NA8<;98)j%`)D-N^Vk=EpTQ2N7R6DZF>;%X^Nfj_hMheL;)PM$ou5#*(u8yg(= zY5!ZFvu^tm6a8?of226JoVf4Lck<6#liXV&n>KG=`+M@2FC|E9eW)eB@ z_``zY6Dk6yK?{~YU3_{2q_Kg;L&1)71w&t_L2+?0DCKV|eI0glN8#c%H#a#wGpoHE z(I|dgU8Zjqi-*E14wWe?rw;BuKXIF{brj>C#KUdZ{(445&6>*~tQP=TMxh28@bkQt z)6L?s;1-Bob|eS9Jso7*0x^yi@*no>vAMakIDMtGh)>(k$u)YjH;Z3&T7N?YH10P= z{X$WA{>#~BXMX*=#%b{IpNgvLQLYE?jH7>E0)>$RBkU>{(3Gw#)4Q`Gu?wy|YD~Bc z2`v=|rhP&OYI|R^*ZltWmdRX0;DCAVO%3i@f84n)m;bz_rVZX|Q^2|8nfV0fV!!EW zzO$kkW#0L}y1rij)xQ?ugf{OfKRp{*{wO#w-r?_B)6C+bkOkU!H_Q28-vn6d{o$c- zftPE|fro6YOh2BTot;|yt|;=_v-+S(5l@m=ujvAnCr;`U0_LdLn#rrLeO<2i_3zqu z{z1XPFaKMYOgZDpr1HR+k?VmSb80Ev;ZVi{b4HE_S*-Ks&)45w{$6Y=Q#%7+_J>l{ z?$p@ZkxHDZb>`=PfdW&(qHz*S=KhBPnbU6tOPS~8On2**@@!_~UGvra?tGTx$M>o@ zFkSU-V5<1~DfR9D`}h97aSMtqWZ0KmZIhNJv4}66{Zh2>}q~&xVyi8|Ixemj~_pN_OXAQ*X7w90yml)7;5-e=y}2| zXkcipu!latkF3)sx#lvAv*k^fWpdy(z0tLblF-TQfN z-|bco0R;mQ3%mW-@BMtf`|WXc`~8(yJX!7wHXYag_aU|Ae);=*dw2e}Zpwf7J>a|@ zBU8JGK!beH?Q%h+5PTrY`bh7?>eZ{ieR+BL+uQB;^X9S7W2~^@Qk^GQSNGBP_xtGj z%;)+4_ij}_&sy(VCa>*mi+!&J#}>qQ0u`){HDVPl4?b#4;W+r%?*qra&!Xjx$2V4l z{rLH-u=>-v_}fQk$H%Zgl({W5xBifijR}yFs~;=RkYCh_QV8q=)4>kM%wza>$+M z+h1v0e{0^}Z@$&PZe4%->dm6qxp7;*zD%vI`?O|l|N3?R8=i$mAHVy^{W|M?X!eOP|yuPn$7Q~PZ@KUlx}BUUHB@94gX42|O490%gx zJknrfa%E+C;3~a52a-e|Tw~JbI+!n4SyX$yoaO#UnGbi|EasIr9glr1*mPVuM9HB) zJWhMtPvbl1lee*Oya{`7nBV?R%!kGOcDLThC#hfeJDmIGxb~sf92*}Ss(n25!TR01 zz7N*%k~Poe_lVTVFDl;G%+Pq7i{n7P_Orh%93{FFCU4icv}oWi<6rytR;5qD?!SkI)}y?pO0ueec;%ve5WtH&i%q% z$BMQ`k`3+wcBSe+3iM3x{AX-`wVLycA8513{hpVT{q1hPd0(u4-RE$w-8z-`fBFP! zBUDf#;AhPyYw&@5X=5Ss(lysG(ma(`9*;Px@&P%QlvK)$jM7o#=1> zw`BAF_j4o6bN3q^khA-x^8VjB_XREja=#Bhu)Zh1KBoI=+5O~#S|%2bJ!={m{-iG5 z(+>%BNSPtaof2Tnv)TB;IqQAJvyUgwF7JE(+{okn>!)Vz`~FtTZC7Q_@e${(Ff~QXD(Q6mrQ$Z_WsQ|gPI@JHEQ>! zUq6>yzc^fwk%?v9|LStv!k_=n*9a?u4TTmvjh~oyp6SlrXT)>=9^d)rbvd8Ycg#PY z!zr*rcS2)n&h~i8hT|XaoX?f(zWDrkmiLNz42`NB90#UfNKIBjn3FJ(>F|dtiM;>& z|2EwEcSq?X&#VJJtV%o1&A)Trz9M zP7whKwtK9nLX{YqSmL}K82$zM+;eGg5N&+ea#m@b*YzuidMjZj(>49mZ*~U?D-=X9 zGX6;1TPw}TR4r!Ec*l6dX{0K2gE8al>2m{j2{1CTxUsT4SZv}apb!vvfMrwC^(#Km zB2&SET=TSIw{B)&;6CH&;uu10vmxdo;s@^728UEe#)_x!R?bBZMrc|<_UAz94G2uw z8kKu0ZvFMywblrCaR|KdJ@9Mk_P!PE42_mt90#7?2u)VnkiaN>g4O)~1x^7-9%SJV z5#CUzdClOdkU~I^14G=V+%7eT1fK&eMLx$0A*D1pEUJVz)V*A*{7TUwp^u5_gJs#( z85}wh2Uv;@o;5E)s5{ECCcewmXD18C6cK@j={d6(aSBKX8#KyHE}Vl1z{YzlYvQY3 zcT{l*%vjvOP}3J{s>Z}EX3!W@@R=78&<=)-ucs&R2J100sj{*>P`$k@17Q=gS7!(b zH28B``h#5xYRjQ8243$mmax@VpIxcid+*I{eN^1wEG%(b7ov6effdjACKy5?RTC?TXUVlxy&xcf%uv86}gnx|1N8xyi zj5?yASzylidip)9MRH6m&omSm<|l{ub{=SA^-&PNQ{#+K7vOW?*V2f?Q@k4-JQ*1) zPHxkjg~*D?UR}n*@*vph9QFttc)g8=jaRd_T5gS7Z~nUsIhPNbgm0jhz+|cR*a9g~ z_i{$;|8h<6tAK)mJ0s%{ncZ1FEMhtdOgkSuleU7ZYfNIgrhimwSqu}47Z=9??`^&p z5m^k`s|KnH4D*GG=3$S(f!Eu^xh8D&*_B_b_TJnM>m-4S0ZK}IswVtKLq_2fhLXn- z6(A)QCA^46Vl-+pGX9t{;WPFKL{zd!S`g`g4tf)OKn(`Jwbx&t-6xA&98*x@Q#axF zaL#b(J86Na02?T&D6z$Z4=2Zg2Eu5!TM)(U$GcYhPc)I$ztaD0e0szcf3flkx literal 0 HcmV?d00001 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 0000000000000000000000000000000000000000..c6e9f20480ed01b50f41730b098ffa4b1b8d9fe3 GIT binary patch literal 106895 zcmeAS@N?(olHy`uVBq!ia0y~yV7kS?z_^Bkje&vTaVk-lx#rv`bGcXT=+a{tw^we!ge@e*WjPf9L9#|NDG)ZuR+}3=8fg zbF-}6Ai|LOM<;Q^a&BWIty!J&Oacvzj0Oxxo~3ZjVP>vnT($G#`(Fj(b9-M+d${HL z{{R2uRnPXGf5O1Px=7~u*(XaD)-d%N%QPBrGqCV3*r()ea5h!ra%)DThWxx|M3{m3Y6!PO-;6&e%$>gZz{W=(CqFayHjkL znJ+Kbh@{TlQUA&F&+PO$HzuF;6Z!mR_nD3d$4-kZ*%bbF{^^hVr%uzFRox~fHAQ7% zxPg_?1=H33p5FzH?a$dK^i|%O`O|og$|tVb$$rbSzsMBd%(E-j@!~9=m%OduUtFA+ zc1d6B%Y#Yvb*uKTzhOU5_1O=f?-MNj9#n34W}&WWlGL}k^5MVYIWvVG8QUKZog?1W zbktOx!}iGg8PhD|rQ2=)uUWGsVEs=irK4X99^bulw6bnS#Xg18u5UCwc;~FQzZyK} z{24yM`?Fi~Hdy+lRkNS_&C+$9eFtOtgYJJb6nJJZ6(=;s{be}O6>qtBd1LmjgsBsz z&zN$w#LeH=X7bVJ5|ReKy}oN0X4o}vp8wGV7EO(0vVPDK_J)_qL8F7| zGy}Uzqlp5G_5rC4j3Nz68<^j)i6+>uVEp=k!-DPBLCpfb7^dq7cnjEVSeg@9_OLvD zz&xWxfrCF#QA&k>rNgWVb}3wu&U_QBpK!A}2L|X&;rOU1xP|Q&hp6Mx0M-(LtcJQa zKd0vlv;*u*jT#frf+zDA6WPy$U^X3L-B)}3h5ZO@Wbs7 z&3?$#@a|*gKcxSVlSLt=rE!H)TZieRE*6E09!yFu*A!blG=q9gl;?Wz7ja9v#3{}< zwBX@Ad_*GYjX~TD10MFtjgjp)n^qpUl6=cR++e$b^NhkX zbsIx;*V?1olJt|_C%A8rFR+kdUf#R;Q02o#1t)i0-J!cwxLnz_KqK(#_(5oe8)+`=grJS?1dcvkq$Q1KEp z?c8>RL23Gqt{rnLI&OIG@sm;Xo3zC9mP&PyhUUyIjUjGXYOyM|N_&+jpUg^;E)tmO zeol4z#QsV8liN?s*EABb?b23lQ(Ba>_)6d|jqjT8SA?t*yV@z`dv;OYVzYqr!Se&` zJuj&As6+_Gbbb|@DwHc^t{Uepq-v@*%y7^-#^y zpe%1oA5j`Qt%8nTxi#;}-)r)(M z_!0dZdlpn4Jj*KoRP2=H$)l%5Pi;MMcj9lQ^Ha*F`%k>Dd0+2;1mofb%LSHfa7vti z$h9?``z7~Vu_7@sJ)3PG8kN?pST3<Wte2zTu{l1%2vyKck#`|KP#556cvtbV@o2W4YrEOz+ve=Xqa453W^vi#>)j?|uKKrju1LRFp3MC?#)|fRj0BJ%*vC^*n9NXCf;kuj;8HBt2z7pwrSfUw;j*fvYhCHvBIi3cce(EVx|4e6@Ez?t^D9q;t zC1`H6Si^R0$M&h;quz7diOd(6&ozJfzN&pr_x$RE(9Ut@5iC-1$?dR)X*E=ub(Y}=LlO{@Z&uE&l+k5Nubum?`GY@UL zlDd4lw_duwyK9&hTCwIHJeuww~1&K2o&fb`7sP{$iTAI^LZ)59CFETb?*1jb8S3&Aj*@<;${%ujJIr?$x zCGDTu)!{GK-rDo3vd{gT|7ye4maBJPU$O4yhO~^epKmq4UQk;2de`;U>(9sAPkKEw zdVk{<#ni*=E00wkt$c6pmhYow1*SJm&-1m)zxsLd*YdWg)?15i z<(K~1b@11NU);Q5yu7_#y|-sSPz$_#%#rjLXQ_7deroYyA+ka(lkZovX z)VD9+jF%YyOkZ~H*_^pAgi7{WO|Qt?QC<1{@aKuo^W!;oT`0f!{Lxcyx#{P=TYR6h zH>dvR717(;>g%K8bN8oJu3dY5(e|(5-SN@Cb$;Jk7ajY4M`H2f&Gm)%<#evN!tJoP#6;$@4MAG0}@wKF=}h?dx=oKkD<|E>2J3bnq2tSh$nn$D(9QtE`2JFIgNEq!#)ym>CNn zx1TaIAh+i#(Mch>H3D2mX`VkM*2oZxVRW4>ucRJV?|kAhRMhC&DE&H#M&~Ge6JH+}OYdtR6!KVWSN~XCy+WiID}e zP9zzqP8)rY2ar4j2`#W_kc*ogmyJF+TtQ)M$7Scpkix*=z~JfP7*fIb=5FPd_tzuu ze_x*xdF{c?W1Be>Zq7(>a$;*rK9pkII>E$Kvr#8#)+ML#Ny|KMwWKW7&Nh9_RbP^K zOV25YORn9L{K!R|S|1R3Q9&9{{s4cU&0`qUz-%Ki{ z?RStJ0Z~@w_$Fy*=1C)UpzN|UF@d( zylJp)whcv?|#TQabW56xLvVjH&btmhQ|~JeJQ^M zaS+HX{&co?-%mbX^fu{lSN^|Clearn+{@CwC6{qe@q zhG!p+{d&3dqkB2uZ*`vdmlNE}?^QmpIq}H$-^c#@dHerOQNMl}ISdCUbb#ohCGJ`A{*Uw|UQu zEsrlZdmq!6p7QVM^!U0d?-w#pm(BWj<4E(uxxW9pLVG{${d80z{Mk&KyQOarMkmim zcD~^Hs@2c-Z$F{J@=xouk3&48s~q~ske@uBWVym|JV0@uNOr> zKgV6Y%67-Q!1TV&BKddUe^WkmecQK3Ykz-VSaa=z>a=r}8!fGWxLvDD=e_$iCh@a} zUG|T>{B8b6uh00fWulGL?3%?X*>PK?Uw2K{+x_QY(@mR?-7k(^*H8bm$+6_c_d5%i zr_1gVI4?EhSvUV*AB`8IPBqpC&WYhnv}7I*fb4(S%8{ z6Habw<#moP+`02j#pZ*lv+ckAI-#~eer>l&&8M2}ohKeFxH?ZR^6~O~n;&03_?Ps( zy)Bb_LagLpckkT7ogMe>{;?)DoRvA4;;gsq*v?}W$4`jYS{;41sQ&zespfx@7O3BQ z=CixvMXcd7+j&2J?9cAG`gHfl!wz4vr&s^_adh=rzjarBCWTj@y}ZhJZV&&SjIYPT zKhNqbycpQy?D&Xt_qH1+{;uZz`^(_vv24DC$LadvIh*?bx2W4ZRB3;BpzFay@%giI ze>ck;YWz#>+;ihb=6aX)KYQleMHd~(x9eJHvr@FsM(W!GU-P>r=M7%hE$=pFzP7Bi zdZS(T9Pe-5DfL^{zY(te;$fG4r16?@h~L}%pKs^P^m^=krsV25-Ny@~d-*pPZaUvy z%s9VKs#MN0*=o7p?=^<+k8*g+Bt6%U*%wf^_2kFy-4!o4RGZjkKRNY7F8lnf%^TR) z9{$$xRr2n8*V~8ESG-$zCSX5nuHoXt|5Ez+izIY^m`!dKDw`cT*RDFZO0T<|XMT>T zjr0r~n{Br`WF{{Dm8DwuEl0WCWqP;Jw7PwlMeP?o56F;_jjL(pZMNI{vi!^62^03` z-29UE<=+m`tk)LxgWE4D||jXUFcx?^1MH{x?&E-e(yDtJk?e^})&y4t1pI;q!pPOB^2zWn6L4-+@O zJ3njl?=^rY#ncYh!hE*|({^$L055ZR)+TvGavuVcS&w{GQs( zCB5^{S#A3n_Vb1uUvw3RTgWR{}+2=};w-=wg@68A{W@r}OkyI?v!5GPt zqMNZx&~8!mTHoohH8WI&(?T9Sn|f$P$Qb! zFfU=c@ZggLGbY;2-MoigIyw2) z>u!DKYcs#Dt5dnY(>5``(=k)OV|7#he}!KqC#=FxzHmuNjOzY!b+xW+S;{=q4Fa|` zTW!7g%l>US`J}J6&QM)>dH<(I_kO;Zmi9}k>d|jM9$G%#H{#scq=4y@&+T~m;D>U| zw(OXiN3xF|Uk*;tHjrC;Uh46ey(zDs?3-tLOppJp;#eeD(dOW1e&nu(veP#AeO ztoM_5c(hJ|^@0}>AxF=?513Q+uv}w9_=^kA7qeO>OGH%MTzPb^&;C6ZxYrwYvi8S) z`ZGavx7nibh12I>Z8!5Vdb{sBuf6TpBhq^ex}4{w`7JxPv*wt--y?&wlebTNbLd@f z^~M#~clPDmc*N>_n!nGA%YLu3bpgFdoa(drn{sTCtX_4lghzZ%R_a{wpP!q#Yc2X) zZEx&(`}OEd#@EuZCrb0y-WL8D`F+ve4eus?xfNfs^{wUa<;5)59&T3K`{CvG@RM)c zA3u4#u;QA?>WBxQyO+OX8ku+u1JB6Z`2OFWWl*-a=Ws=)#3xx?SVfthn*eKIZ(adG3#<%zkw& zf-h{}wjC$CB>nYnW-gasW;m@nIn`g>tv2GvDJ%U?+wHRD&Q@>ObAE}A{PJBH=A1$u zD%V+&ea4wXBC!4S(09 z^`G~%dAK1vP@>taOWaOQ#_B1tVk3;PT9m{L8HxzP;?@gR9HD)ApT`{UfRz zTjinKUNWU7mmB#ub`$Js|J~4#U6SJUSl4gQo1-&i z{C+(WSYB6i&G#dY5%`QDz7_d z^Q*%yZfBtOhASJkOtjnmXOru%t($MpH+BEsQxtaZ>r*e?@E=*9cRvfecT(W*=5K2Z z_qiAGpF5%}|2(gW`}2mIM;^?cB33Niyk0hQ_6{|zeFq6IF{_~e)rkJ#p_-6 zZFV`& z=zO3mGcEXJ-qFP8hivXA$v>ZP&*n;7u6=;dUd8b9745#2i?yFWQfOZPcK)IXKLjPU z&j0xELius#nU5c`C!bz9%l{AG{b$E5DvEEmA7!uIIwS1QF_!eXt8ewT#wbg|*f^~IsPH?H|*AX@i8^GU|pREeik z>dRh=`tSRmewKOnd)uJu0>1~9c7C-N%|4zu5}lB{O3b}`rLUV`n&U*xUq5brxO6c6 z`qHKz*}5Gylf~Wl*NDwyjlJFEX0RoEFA{aMJPXd21@Tb{}&(#%CXM<4MYkHE#||zWG%KL+ zYv(S>?;{Vl`E3+y-nU4;ZP(^?6(6jGFKgRb9ZU52ct}*ZYklmxD}N3w zzCY_qT6o;G&BA?7>?nPDbn3$5^ZJ$Svz;f@ z=1cxaX7ApA%<8Z3siTH$YdFf-{r}z6D4ugXB89`gde4WCTTaikyYuDf+XP|deO8MW z%ZuGPWmWc3MELH8pDibs-I?@qa%bw%h28pcF6-v!7Jd}XoZQYnTh`8|ns?&jTK;*} zFU<9yxzwF8m0LCK__G|YZ~ev-8*As<|0sR;C*$mG@o!G1k3F zS??Y5@K@S;Rcz%^t)CqFUH)zS&kH`M99FlrljTsGo40TB!|c_9);DyyOttojG053& zd%k(${U0wrbxZmGcql4v8E5i8<=%&b{7K87m{;56&w02eS@j@*@#I6?IGocy1XH8z`$Nr#nv zy;^eP+B&{uw(>s{&zt($w6M>wbqTxo^W%Ds73+Uo5Ko`}`saEcWq$*y*}HZeKCXSP zqTr+Qrw_eHC9bwVKIOLNOLYFrhncJ8Pgif)^X}`>`Hi)6 z$(AzqdE5Oqa`Q3eW6!pv^X{z7dehrH?~lpzV{YwrHkOC_PaJOFY@gltPVJM1{L`>~ zew*1(zfP=O)mXx{m@)=To|WQEw=N?aBAn0LQKS7k42UtO|6dBxfEopTP@ zz3AWZj9=mS&V=}w#BN8<`I>VpUO7wes_5F-#p&;OHD~c7v$C4nVP-ab9;Frx-;UR^Pkm+?N|T=YoBh+8m#UAY>~sFkV=R#7j(q%B zJf7#~WbNOZpK0(HCEhma@#DO2WE@+ZChZW$02TQ zre>RU>FY~P*423nO@C%R@m;vRW5cAKcYNdzum5!}@%`NkjB z`DS$R{jKy|Wd7t=$&tXCqU!G_axEK{&*j_mD8OC)Msqk%R*jk0`p@^F!|-BTy~>`#d?k2Jz+H=UbPy60(= zp55MCDTyoOPo?{QJh46Er`EnAZMnLqk=C-?ex05AQL*;9jO6T`k2f_87WIEyBj{gu zo>lMJ@M!*4+l)W$C9As2 zJas)CRQ!}wqKdYtcJz6#=safJ=sUx1cZZ%_lOw;Im9AH<@Z~3Gws#+H%gq6B3kD|Yl4kR~AvG3e5Y2{DHl&fF9)=r zSAGopZSZq{<)8B*F&kdS7!`Tz%j{>d)W3h2JvKrwE%B7*-w)yUldkfg-jlhx_lxX< zlUM$niD;WAAM)dTTJPGQhn4mDZWIW{{kfI(SpV9SwF}=Ix}d-BSgcLThA!5$*E4x8 znzB_!*P65Ee7yQ|;$#n*qd{>meD6GzHZ^~Bj4{~pO?U0qxpm)dD<&heFm#ouo^KkJ&t_ z|C~$L*NG|HUWvH6;h~!H_5J(Pd3S$utncNumpgOS{!YrrtD=t^=la!!?Y>oV?Z(S> z$qk`8DQll|+S$FBbXxxZ&oE`9n$kMCbLnUIvA^HT{(Ol{&7J?Ag==ya`6^^B+-mUg z#1~zm?$u5H`&q9aIuI@H-`yP7JwYZ_SS_x7fmv-z;U&#!f7TUV$~2s${NrVY_UnhP z%hv4prEyQH{NsaztIo5n_dOWL^CbIDPC(hOFH=mnnf`n7=G57xj~TSfit4(J-xyr^ zm*4*5&1CWQdlaj}<7>alnA=M8=v)3R5@*|8p?71`zGwZt`(nOb4?S|~o?_3ROD;Qe zcoI7r*ENOLZ9T1;^T*)C5nk?0_kx5j$@yn{qI;Un_dhcFVIXpUVftbBio~bwJO>Xi zUVheZZtkA~n+UQTq!6iJL8S^kCHYPk$fzc+R^$1?wz==kT6uJ=b-nH=M1y$#pCs&*8ep2!AYDWFFIn{+%r8Jjk zscgShbj@a^VBGnWCoO(hTy>T9IOo&h$|YjIJnPDmtaRt_3BG2L+?sN++jRf@dbq~k z^kVRjmb(kG*YnrdL{uDQ7M5qbx2wQgX6~&&B`dF-%ab+!tEO~DwAr_=YjVx2KIY@M zFMjEs?7K5AwMeDDFMd)A};$akA>gR-rnoI!~8?Zqt(5< zX}RA{s=xR#!%*h&#_$O@1bZ@z&y=4Oi!UqT`+RKMjRz-Pt$sZ9JolwrI_%%ppKIs) z|H`=E{QTK2nUms^?thA3w>;s$g|MW-{NGmg->)7#x90Mi&*vn~uGvYayVq2f+LgY0 zJU!cNdF8bWj&B}_hwnL;|7pX~N&b5tMZP_H^xML!YhTpn_18RGUnsVE>ayxVNv*eQ1L zw|>qz^l{<&Wfm_rJ$%=l6MysM=B2B7i?;r}9$)(U@Q16t`{qs5v%CHApydtE&qh@} z;WDoiuSY-la@z5uNOPU^n_x8t| z=QG^Sp8vDz=tm1_{hw2M4L4nO`*0|5@wL!etD_U=TbezVURo;9Uw7q}#TDh_tFJxF zeOUv`=Wsf&+};(NyY`yBidnS$?oT(jcK%jmOj=A>Q zhE(S0i|zj}&NF@Vr2qe`Tb~%XpZT8k5l z6tgcsS$|Yzo|xQhF8dViJc*QtBH{bG?4!3jt)G#W@R->wa^IGtOZ}Gh{EW_hy!9y+zcOC+*4dQqd|7?(*OyBxc~{>#_&Psol6LQOzsiTg zeKTGxFtg^Vdg2j3``(|Udma~FP@nxvnJpye)+5)7ryJ{%FP;~Vv#Zt+vA%!u&*)yZnfphSJ+`p=!4w{yOh+46p_x7Ty8P3LwVPUqcwZfcRVUZl;pxn~Uyr+2@2 zy4&K=H?s(9o2rsc2kSSSmogMxY}WL4PWG9j&i(U`1^ViNoTt6sX4&P4FP~edpFdY@ zbR{A^y=KXBy?+L$ea&935G@ja^yY_$dA+vVo=4hdI}6Tx<@G$=5DrTCw(EZ8>z#`H zJTvCptmU8P^8DZTB1Amb!IGixi*CxMjHff-nBI7^S(QKgX!>QRg|`pAPU?HUyrkan z@b~1zlwhVKsf-INvW>dDeO@PBQ$HO1;7WYztv_j}`TrDLEji-gU;TBz9K(-{zTzJ; z3hL~v_d5G8w%y*bY<*+*Egj+P4_#AxY-L*dZ*;AmaY-}(sg8A3&X>d^>;3cWCae!X z(5SCxbuT47eob(BXZJjdSrPkBwdz$9QeO#5j`D4L_<>6=bc)oV& z-g^_Cv**hP{_Aq+)CZbIMrP4@2$7?znAarS-$=l zvi}Yjt-G{%|F8CL@%o?g@Z|LNqlnKRax#(X)gU&(5}cf-}KH*JK@OFiHG z@uPlNa!j)Bv(2B@2>y?}(OjH358UkSK6XTd?T(XCr?6KRsV&lV$@t+s= z9*_SKTk%6NUA?UI*M`GUAGdydntXK0`DuIJm!Fou`{!ckr|2_3gv(cUclt zc66e>jjK*p$elNnMV*Ts-17S#R?Mq?ek(Qh-_Zq2=EoHtUh6we{@>4_6W3>*Q4~pC zem}Y;=bGtdsioobJ}7-`Q|&V?t4jL7y?nz<&5kwO>WY3neE9k>Z*=@b>7@}@KOU}K z^WO32?Uy+z56U;6n_K;jX;c53gWsC%vgEhFzvVvH_}lg5zmdx7pVZPnHGY_*{;BQC zlk}gFkK+q_-(UQ(<|Fs=XUFG!mB~1Jo>NBt$ZyiUSwM|EB$P8E&sIJhkxDi zeHgImi|-T_Z>uckN3(l=sj*x&^o%^kn)iA4a|^!V`kRz-@xfDV z+uGcH6QVxeeRS1+ZqDV{-0MFMDZFRWkE^kWy~J186S?N-LNhiSql%vslXI8&*-0Kx zPCCACW7)}T^Y`aiOiA}qe$V0eI49@VhO4Xjc7Ju;dg|d5>)B?a(iJ6A_Ojn9G-7I7 z->}5<9p9l=KhwH4`E9%0ubNrz|7SV&{V>^f>*v$Swvw_IFQ3llo%6E#!RCv6ljoIL zy_x!{ef|5jtasn6?v9-=wS{LehTb6`R9hr_|w#rdO-|BblcY^?bE z_{ZJ%4|Rw5E2_eu8s!m4W@xM$B#I&!%@ZCl%s!|F55S3dEW zADdg1vNqDls^C&~PR|5o%Ve!Xl%DXd(N*%j;}W_ z`}5FV?E8Yk`Tc&S7pDbyW-EFbWhJih%s2XQ!=-&PtFYX@UB27>;%thz4;*s;aq>|9 z!Uukq_*Z_Y9e*ILWd`)(vo3*UBPc8ZJIwh}4++bJKf9KG}B$l}#Y zR?FH+7dxoM)xQ0cw)}lUpmvGC-;xhslxvC?O=EU(|5JC#y_Yq<@ve|s#p~JRbYJhWRd~#Q-70^7>9QA#zwQ3E zYx&2wFTYj)HmR+B9Fe2Of5~4Xlm9%6(i8b=^3*+QnMWsH3B7+*fA`_e z1IL4xE3YhAeRQ$>L+u|~`)2dhT5$La>T|X8?VFzy{=ekJqNEG_@wWP#l6yY2C#>1X z^K_;2^`(~;gLq0FYCXAmRDavga2Ge7uRk@}B5vKQSU)|}>hJmN9^QlN)2{!x!nHpu z`PJH@J(B+=(pAov%;v6|X!)GmZ~qg|e!tg_)^C0sWoPH#+WBMpW6Ap;V((pAtiAn_ z(Y6gA1^1^I{5Z*{x35wt`)AAkK0~{|IZ?)mXSWu-={}#dBwyxFR{Pbc8{Z!^-*5i3 z=km=H2M+H&cl-HU^T??4?9~_Vg}x}67rJ?SyK)U{JTJG~pL2|RzFFQbc=512FC*Pq z>|^A7zp{ck$=|=OS?KMqdws5c!OzI^i;oG|%m2Fd^`@9}<(osZeBQb?-;jH;*-a+r z$BFG`Ycg!xKfZANlA!Tg=8|G?c#o#tUk?70lJ`%w)?7Ovp8eBA|Fy(paqh_MWf9%` zZPpfN%RKoRxLxd*9@unf0vE#)lAdgWzc2Ivr*}ndWp>YH~jxol3a?K9{iB|a^`qPf4 zk^JMkdtOVk3(a1?@~`NK2i8$YxjUsv(6dnVt^Me+OC9vo3WUw>RWwo2WQCq`ud z#+S<9PTt7)-0E*WC*^WR&4u724$DlImGmgzTe$A$$?L27@7sTV*nLoEO|izdM{Vm( z!f$1tIeK*ZHmSM#X50QOE^_E!U-9du&8+VaWIlk0s$=biUf;1RNV};$SM=M3UB|E4 z>bdO_zrFd{sbIV8p6eA;&arSWu`#i;sdRX{sH}F^t+~hN#(%o`eZgGQQ@_?cG|$Pk z@Zfp5d(P&Q;lFAgZd6V`eQb_N{*zxR6;C{7&lS43G@R$;BXhG_mw7$CX7NAP%2%D- zB53Q%^Ye0?=bVpxrL)cVDgFA0Ynz=g!~54$fo&23Ja3Pw=)LXEOU&N8pRtc$fA@wfC%bsh|7u+PfpNLn-<>~?6wC>~@g%VDfYXyt z7urnPjveXS_wR^p-(=sKT$k-LwuakT{;l}3WooGT-an@M-b@m&t_+-Bom*7-(0Kob z6F+W$e{gYL%;$)8cDLkA*ug_n*W}NY>M4I)S)10sa_#zSQ~!2c-rPInNN7E~Z~L(i zFH(c||9rrFQIoyk#o?8{^XJITe)E}gu|-$+`oEFIUu33q7wLz4*k{$f3csAG_e6YZ z`r9XGC*PT@ou11tbzX7qt3_{@pPF>@fVWTgMK-0)6{^}lv)uD0dRxfme){=r*4YWm z`^!&$^k3T)XD&2n1@9rf+fQC4Exhm2JyS38z|BY!RFv2994PbhWrnvDl1h;MrqzrX%lPEW~JQH$a=i{G|SFNl^eIog(A zyVmei@A>(=B3@|oUH*_}qg+=z+^j^!7gKeFAuDZp}N_>8Jht+zi-~Gq1bx;d}boi-#@UICAOe^8RZrUdxtD`o1@5x~)QD^>-k{eZ* z-&kM#`g2kA{fI9&L;QCO+RH9c)9$PIaw0*h*k%2X4>vB&efxOF4`eLr>0 zBvRf;|K|DhBl%d>9BIQt3H?&5&1_l3C8c-%IUM>yN_zg&%)aCo+s<9q-1aKx()IS$ zZpRL!S8CgR%K4$;X5+%Y)ZMRNCapHtrq*!Tc{88nnB2o>k0W8T z%<~Ni<#Jo~Q(O1f+&caC;W?gNTs$o9_v;QXUY_Qj7JqEH^YwXK-oIKNJ%3JadgrXz z=K}jnUW&NyNouUTmF`>j)A7&sc|9Kw`roMd5NP;f^6Ta7;e9v7mP>uJa{F0la9%2+ zMua=2wlXWvT6{M9z84yAj-LKku;-ZAzK-{Tu)R`4R;D&=O_ z^Y*o#3%kE`ryKRzKQK|-_G?8%;)cWGufJGtcp#fJ`Ki1cH-G)3NSDKY@+-C99J%;? z_T@x(v6tFUl9HA8an%)Vl{jIga&)fWj&i;p?{-;9rS{c%Z=NyU+amcfL;I;>d``gU zc6olaYSwoTYnHjUo%r}@vwO`Uj%e9=QT4|xOKzI2%d9!G+IN1;o{l;ji<^%Qi#Xfx zOHGlJ&Hdc;;*e?Iv9A5^9V0E=KV7?XYNOpG@8XS@Z=O658TI_zvai=7OY6@|8C|&* zvN`vCe9rk<=T@4=y?vCoe{HgnSaR1FQ8WIlo7>_m<}SST@lEO9uaUuhr;820<_Khe z4qsDYa8s)wdiu)`%~tV;y5sAPMDnD(IUM|Yvht^6rjp0^-1>6zX!ncb&69QQd<>6N zZ?VhP+h%(sM)unBT^X}omUgbatHiUVb@8^1Q4YQp-?FZ>D*bu$GJ1!uMiKW>d-u%t z*Pq<^`_INCEbfm=_;*uF_J_8p|DIBt*Uyrt>&CB$*xKFBd*;~kykwE>5l3#>uYbxe zuQu~g-`?%}ZO*-4Er0uy`Pa7n`_h8e-KYqDpRM$Bx_Z3lN3k6iM`HKXTn>G*^Ulv- z=T82AlhSirb$(C&uB?jMzbU;7>(?fnfB&cILdWdAKhk}Yb83_=y!Ku7s$70p{cn=W z;=_EhNiRN&?w9J%P3Aj3XTsWuk{2z|JH~s-e)1paBQAlT?|)X#&#d6(#>M5|3wK9IP4}*`OnJXa##GESX|Zjx+>W1V z{Ws2ki_kj1<5A{q?)iWJ$%tOe`%cjbDFTl!)NtLJZp+BNL+d9{cr0_ zg-v|%HLUZjc3R2ToN~9f@LBpX(Qfl(=Ukg#AG1H7-2d~c+n+n*My>bsITQ@;83<=~CK; z6I1p0b?ZKHS^xQ!%Okzeta4kf%CCo)x^*nN*Uy@FwrTkVUTx_a6&%aVV&6Ky`SVEo zkHW8%#Lu04#~azj<9oNrvOGNKm}SJOS8KEH(_**jecsYL_jzyj)_K)%@VuB@{J--1 zKZL*E=r@bBx>Xx4C09G+=OO3U_SOHc&;Mt7JG}nZe0w38`g^6{V{h|rugj>uTwR~~ z`&s;-zPG*m|1NvkzT@K#le%}u_a^3vwwR;l}#lqw=^Qy!0yQJyYc^gZ%>~8VaZ$XeQ&kR;}acAPIvCH zxbVZ=toYXL(p_IJ9skbUShvYEe#4iW9j?-)ir0>b=f9rByJ+>^X$FTLZZGs-{Bh#G zIwKo1(V9KSCiP0K-KoL1FKcr8?Hf1Sy!Er*hhENJ)2qHFC*hIej9X7%3Rv&|amslk zdw25s^KNY4O6&6U?eCnIN{_6*Vpp2B=y&zCAM+b7rlzmgzV&T=S+(RgJ1*;Z(89&^ zv+=+GOwWz~J9GYwjBDk0YtNS*KD#}4|Ci|fn`=L3-`}7BBzr z|Dn_CXKu2~+xd9i+g*i;^>1(A+x@qDe$DH$!@k>n=bzJk`XTbN<;l354Of5M@&X6R z%d&NsZqGLr|6ZH#_pH*+uk@nX!-=eyQnqvIa!rY|66>7Q6DO_B^S4ebP5xNMne{$C zk3SyV=6Z6P+?xtMmA7IqUTOaqTO;&U-0s1FW^cP>nZmoa6`y_IWgK>X_0O|eZvIiH zxcIyiCq>>KJ0m~G)cn?yFGm=bpXHeDQ~y;|F7Alj#`FDVS#Q4l4A=hn^2PdXzy65X z@19@r{Pni|UuXSq{I%o9{mr%?Kds)LJ?|&y?di3jrsv)NvHbt>xBCBgufHGnd!6~5 zCrxMT|FQ2Y`hCm(kNEcezZd_ns6KnUK56&kbJp9m_4ghCS-|M$J+z5ee{jY7%CY?)6B?%sGO zd1LFtW#*HNY9DQUxW8uE@_v6l=FG>36|?0&3P!uc@7}j(QN^Qmz31i9Zd7!{7Vm$5 zuzw$W|E6vA^ICbfEI;zAVD0Ct>ifSfsBM2^{OOz0z2?emGV`Phtm9Ao$XH))w?h8% z`B@WR%h#XYdi(ybZ{{AqYfgSSqAj<7?+;7=yZhes*Btt$|L;SZ>Ur_ME$46fJNNy+n{Q3?>%Lc9yR$X+XUp;Y`QJ94zp?LIv3=(6HvRu! z-md=lUVgsWv#T~5f<2Yb?A~2^d%XVbuD7rEKjyui>io|2%;otdN56TVufFxIl==VC z^R>6W#s4`yf6lgNd||Jb{A`Jq&a1kW@+#?p@Y3=fX(wKsyCIi)Udl3RrObo5OKo@b zCFaV{RnvZMlj~WqV@5)=+0BYyS1Ua3Ck2%UL8{^wQs8~+YI|M&LoChv29 z6AG@ThUdk9{k=cu_j3EM`}6+4(_KIF+n;-7&)?pDUH3YD+y7_m|9`)IJnz?`xBh#6 zgc{GQF_qi+VeOrw)4HF3zpaS>vgrKXxWC)v=Rf6+|6*xezwfu(`CC_}9ls_j{ZacF z%ly2`C*r2%FPA;{S27dNxv_1UjlfIS4WGRETc`I*WUoHi8@cv;`=tNZ+AB}y-EVf! zcDB0}{z)`@Z$uNB_@j43`z<^qf0!Y~?ZG<{4N2JX-iEN#^M0cgDXuWJGt{?cvz}S#tf2 zo8|xbpFfT?Joo3edrx_9`sEp2-+Rq3+W+nQo?rj?c3su)dFK0yf3w#=fihKlK;=6Z}02=&AmMD{_SJ4UtI6qexd$vbo~C>JI2TNH`Kj4ey-xv z)ruob?lX7pe0=WZ|6|el_w)9=?Ry)0y*{P->+HJh-`e;8+4AFwLJ$ zo#*l4Vvh@+=lmZp$iDm}{?0?m-P;~rir@EOhlX;geaxClPWgAgoEz<`9~-$R1t*9r z-zf@_OXn%F|01ID?a@`Y4}lwBWH0~jU-zhfR{l);_ri9?uO`<2JpJ#>w}-dwHq|~~ z|L68fYu4lPPt5OpJNWka{hznq8rPp+KL3f^nWJv*e`4ci_FdgldvHg_{%3C=FS0%L zy!Md%{dM~uz3M#0C%M0=Cp>eXT5-(*x3kWdT|;Yks9X zTzTEr#jch|c8Z*I;bd2Xui@2I9y&Q+U%p)4zkGsn>Bpsu&zZ|V^|k$M_2#zm+4D#D zu|H4v4q8^%ojUin`Z9X%^qt_)NXLF-Tfp{S9l?^R{Xqy(yYhH%!eboKg!qM zKYwGo~yxKDE| z{cH@yXYbo#liju7rlqz1u6y2XyMo8Lw?EG-dYU2)3WbI2)=j++G`C(&IjcO=?03$Q zCF`fj*VOL%db!nIY+hAu*o4iCn=f4!FW)%v;Fb8dn}ttHEwB4kB2u(OZ(qrkpU)QW zS$UTI_owTB{^ZR!eJDCR>dqhW#oKP2xbmqR?APw?g=Y@Vd;a*2mPNmTSUM96TZpv3 z_3@?&J(GIp*Ew>mTfoPD`1%nUiCT6^wxIUyNh+(K$+O*k^gxoh_^V-j}~Ejvsyaw(Muy?eCrGzboX9 zgzPyMRR8s){_JCM`!{}`K0j$r{32@pZJzv|Mu!*>nrZF+4a_Z__F`w^L?eepGnHEuXK8(Blk~f&-sdf@^|v~ zJYM!T*L+X$voFls?`+unsI@-j_xJZd{y+O6b$Awk1=pk9bHq=3+8(<)Ye9DGMc&nB zH@m`nDoP%%%`H2c%JXEy?C&`bF1_7g-d}RLd10^kJi9Zs{Bma}%Pe~Ke8u+qxHpm& z7T0&~P3m8Gt#g;{?Ygb{{G~U#rsh>w-raxewcOQh;cuUQN!*ri9o(g?vnxlgE4{bk z;iseP&Hg;PqLzK<>7j_1ugdRz$vM~@J%4ST%e@;PFI|78Z}YcixiFWk(JzjFf&b*U z*Z$Z4_w3uM>3a))KaAg3mbCZL(ewBAy?5vc7Nqe4EsyuLsjV zeh`1CoUCD2`&Rq-?a$H2XMZ{t(Dx&0`dYicpJHr2Mhh48cIQQGJsGwq>B!>Q=Wo5~ zI=$mWVt1ooRl!TUpz~6Owzs}5m?L-TB4^PF+ZVG}sI6vQGEG4^ZQ+cN+{xFLWG_~$ zeY*Z_&azJ*59J+vxK-oD$Lx(O5=FNR)`+*-N-6DD!qWdV3a;;) z|KUhr+>zNf_g_8IDXV*?`m;i&)^_QQ7Y8GgXZ{Z{sgdknUsiIm?Bde?8rEfYwkDsA zA2~e{+*dL2M}p7Yg^#n9do}+qG?|^WWPQg;)oAJa7JDME|6BUed*b|0Yy4;Jc%ol( z>ht#a9q-Pnf0U5lYi9d@|Hp&nXW!>_JKvwBrT3aAK4!1!k4e*O{(U>V|6lTD`+xiE zkNfAumz;e*_e=NI!h=q75r3rjn_k{?(fM+JrR_O$nV(^wlWnekIl5SW{fy7k^=r3v z@VDld-Adv5`{l-kliwHW?XBgn`(kS`d0+jUPM$5(uh|wZyjJ=9j9m8qTiG?+C zmdy3-u>GrHoy@WPZ_M4<2c2ai*fyZhJoTF#8#bYyUA7du|n56#w?cpNaiN zN0!;yE?L%nV)4}T=GHb=e?JJYNn3gR+w#|Ze*N)u{e6XJo}bV7|0w_NhP`iC{q1M2 z7q@%dd0T(a>lflGM-TJQuC~oyx4*FZ>+Ji*M(4kOT{BT!#cpGl`n`Iieadye=a%Q^ z@B7xQzR_&mr!98bKQ!hYTP+s;Q!M@GGxhqXM|OOiJoWS+N$u*tIX?nE+{~P;`Q>u* z=3jPKmX<5dGySQb_j4M1p&8%Lh3(fR^sm`!{QG=y<-LsA37rmXckgRm$-flK@VlG0 zM9I@xw}SI<@RpCNQ+iux_W1u1>hqr-_qHT<#))6*xn^8DIRg)`%451-H+^T$?1-A& zjl36rZj0slvoJ~DzPFQS-#(}HKeonZ{1&dLxpY#^x;DA^!AILE%lZm0eGE7IQTyk` zWLZ!5b?p5-N7~f>)u&Z!54<+m_3qgB zT>0X!#>ez0^t?YFUv=}_nbq^`{x9OVUp$wmr|4i`J!{{cc|T-q-u^0H!~A&F@%XK^ z=Z^p1_}2dKz5P`pmnClV_j-Td{%7a^VJ>y@2$(O1NTaF#s_Wt1I_XlU5 z0WAP~U~*pSdE)OIaUHjkU;mso?{8cF{*9Fz&PzRyOBdfiSN~6m+NXu<wy=;^H z;F7R*eaGIq=MKqAkJlHs#u;WwYwP6h__FDwU;Lk<-LF2lv&-%Ob#wDUW#jx8Cg-Kh z&Y4)$9CGXEleqK0xd-E}UjkvO#KXx5oIz8un%_Gs<)yKc@=)E41 zFpaycC_qk%+s9_tQG;TeANqCQg17De={nzPKKGY-Wmlgze}D2b=I{D{^RMTb->dqf zFY-v=PNF9C#FoW>MExz()Zc0ByrFkxbJ{)KHfPI6|F@i<1zOJ@s9Byd_wmUHyK{4c zK>Jz_9bmtuDzU*gq3PK1HM(6Lb8@>|=3R2i-}EBlhH$#X>e)ZOe3kAf`}=3&*2Leb zW_LQJU}pi?)yFc;@2mR6Ue@mT?@0DzKdFrh;_J*9;&UFPDBnAF&0>=1=2?7?;ty)q zrw-@1A3%doinlzr`|-MuhtK}VR#1NQc4fHSU*WrFx8HX%DLmq0&y{^_ z=aZ^e{PsUI;@8XE|6lk12kV*bd;aH6R@dKO_V@O&@6Yb-uKjkr?%21xkHw8*gMKG2 z{L0=O|1+2O$HGIdQ9Knj8b`xp`)+)57P5K0tn@e^pS0mq<@ql+I+r}0^gKNG-0qzA zjmM5GGi6y^DebrXe$qMiXp6OqrFPM~OL}^FR!SHD`t?Wg?XRE8**)Ty<>oNHmrqw> zULN`UT5|R#$H?mbBUbNsCf0Wzj0-3i%=zDCe#&jnirU57mftvV^4PwdoVV9go;vP* zbnwvYISQZG?5woOeiK;4YUVY){_R%(_ID2@K3=nUxXIsM=j&hF>>kdI&lk_}eN$NZ zjNNo!QO|Szt{AJ5lo%7`7gY9Ko zd!fPcVFD=0m>&Bky#GhZl^-9Z!v7_tDT|5Bo2747ut_jKIa}uBmDTT9Y(T3aZ@XT& zpDM=S!1^?zV2)-5YdnJ*`%CXWsi_QG8xJ_%;w>>c#xA)(sbS&y{#*BFhIz}$)l6UW z=YqbPM9Ig5hwF2vho%%x>}K8Q`0MEMc)ed%?$VugRYz6keYyGSwC~1G=E?FuFRbIA zJU6LlL+r6HhmT!OTg+=*ZF*Px*wI;iJAUl^xoq~cqv7j5Yp*-q_gu$#7j)2rXzswuxU-}6h`@4wq?9>ncjcG@j5U+&1yIjiSy-TTS4{!fel z`SSJa|9&Vf{u5twR(t#XKM(SA=l`vas}s3g7+3$f`Tf0rPrUah{JyvS-v=4~HGGzA+>!;WJJ~R28 zsg(kJ_x7>N-;tX$c{8Y@+rjbcG}Bz`Yxc9Mlja1!-TqC&?$Yw=h4W3n?UlHe&KsBB zP-fWKo|n%0@0OY~|Ch+iH@_y#Gd+4M^2wiz>+1KV`-&-Qfl@A-|iDz5c zX6H$rm)d8=eRajow#mBt{#{&IW%B)K`}zZJ{ySfBy5FBHS3ma&vw7eCD~3PhKR;Q$ z?fd!K3%`5)Dso4j7){>Qq5TW0IGw_U#e=Tr5z?ED>>e~wPt{rvuee{J%A zj%J3xF<38m|NOsY>wnI9drt6}`M)F4YS-MWuYI_^nXOGCq08BE>)OYUa?ZZkeom^p zE;(uUv5FP5*MB+fyE8%G?$fiS6-V0m7h8Pl?f>%9e%kj3K1aZVL(@F!Z=F#+yu_pA zWT+D`?emeMW^8Bxt-oD-c`P|83%f0`e z@#pAQeEPgCe&@fnldnr`PXByI-LAU&=#Ql*-~XBVUjD!N`sKOvKQ`H$Y+N?q@ag}D zvG(SlzwcRYlm5H;{O>Dor{DiR?XCUqE$e4k`K8pKm;U(o&1(B+PgZM9wzPY~IL&Yy z-rqOu?tlGt$MdK|KY@&&OZzeUCo<+XhHLf z$qNlWyFY(eb**E1{mu-Trw%enTI(K#nDg!WC0db`KDqDPFPQ`B?c4k8*HjqHU48WO z=3_@~>a*;g7Jm&qv*E_&m1onAl}-4+;l(L->-*>He*dM& zbNcP=?f<{r-mWiKul8Z~vHv@k#a!IKKHb+)+B)mZqo4`138oSXa->4?(9v!Q>`D!Z2NG9D=TnG^xEx_67&|E_ttn6LF$&BessYO5j+DW=cf=bio0)a={QSU>6FkE>%X4kbQ6#K-S1ubOZ3=Z^7{ABk7>{vY))-+xxo=f=_Kb6?-f z)R%t=eeD1C?~VLD({AlKv;SYL{pP=&)AtnpUby}5i!Yw~`)}4AUH)h0Th;S57r(vT z|EKo$?BlAxUhn(Af40>V*$cPo9)1kJXRmkj|F`vXk0{vpzP(>_^Z(Y`i{19+zn9Jb zzwPAV=lNeVWwg)F_4pBvefrD)jqF2C{N@T#+7>m^=3D}ToSfJ46Qf2AhoAax__q4{;j6vC;W8SDpkWUus{=|61Gr>-jwWic_Dry|4NF-M98n{?DcFZ^ivPWWT5O zlkxe#?@MleSh(lG$@6#rE%yKQ=xpg#b-H+{Tr=aYwtyV zF0jdd(ixxL@yO<3yiC%iuYZdRl5 z<^LJ^KRe#;K6X2&w&GRIn)9$bwf z`rC?*+TZwK76VhJG_sAE% z|694r{8r=dckgT;*9$DppXAfmdi9a9qv$5>^?Qn5J)0!E^8lavEWaJMN(5~Ge7U-! zRfF$eU{y}^ls~+Wg=;@%!-|nyO$EDu;4Ij38$KT&C zHRbY#msj5DZY{iJ^JBICpReL#*9|XEUfgL~q!_j|W{1o7FxkF}G~fI!($UG4cYe4Y zU#-9YRmva#U5`3%AGiOSyxsomCH*^bKRM@b{d-sbr~7U8WB*&)>_2~aZT{!jxBPp5 z_TIMtf2n@k-|hBazHc}G^G*Lu%!K+!m*cnmJ$(MR_3hX9|8Bm`zvrvF-?7v6g`ZaM zd2>_$u3g!ml?UXjZug)4QFCJAkKGggeYjnp^ERWv`q_)dwK08tVIn3u*QsdRM%YD=5?A?~sH|5!eSvvQham!D03ahT|Hu~B=@5VRV ziY<26YfQ`62G>qG_vQ_#-TbY&^quvhf6qnUN*7m)%`<&H^TJQ7^HU7hU0b13-i@R{Ibpdp#Aia ztCX7EDSrv=`o81`akl@qzy9)W-|Kg8J;USn)&4ndlleQi{=xU0^}n7x&#!-OU7z;P zKjX!@^s~S1jP8g0pCA8s=I#4`uCKp;?}PO9{ur6ssZWKE$8-O5uYbqi|KIt^?``rD z_4T_qb}zR08Td@5_q|G-*+26);c;~(zDC&xC}tp63f?R?G2&D+;k{WSgjKfrUp z%;BiQtrKrZnai~DZ=G`@vN%9bZdE$3%)YL`Jx5p6#)JB5b|vR$1;=i=JwxoM=;Kd0 zmouU*)DAB(?JREWJE@$%@WI84`GOaZzn{;w_hRYGh3@Wa7kt)r`V-o2U(3zD(J^tN z_Qy2aKgwn)j~s8dn9h8>c;77jDv#SY+>*aNziv6Lc}e_~Gs}2Qryh4N-1^|-OP5bE zJtYs%Y`>%&XQ=U~n)z<4alAp7xL@(5nu(hit{0hYW^JjtuVVh!LjC)3zo+k;|8mLg zcJo9wL+TXo_L_oUrpF&eUqAjk zZ%^ItorM$sKbl^1@7w-AU(Dw{IsWTYc*ZHlhB||X%zXcg&pd9Q_sPHh-}>wBI#1RL zn%xwA&68DgA#r`1`?@@vtpAr9Pg(C=)PH8OUDEmqA9ub#Se$)pj>68J>uoJW!F##C zMm}~u+gfq0V)5;|S7$QsuTA%S-gso<@?`$3?e_kk*Sx%waCs-A`TxX+3x8$pihpEbZ<-Ful)OM>i<9P+v4To|1P*4dHSDx{r~g%?{_{t_I7^F zO?lb#vySKH|9s1z=fD5gr;px~!q3IW{8={t&(_H@jH0U}Zge^GUap^S{%`$cuO}t{ z9{6t0&6mE<{a^b2H~DSX_x}Ag*>Jl2Yj2yU4QuV1*7{D@e?2$f<-T-j?We11yOZX! zJ#OZ!HMJ=!eH=JD+vIr3oNTLKCv09Sf8&yz|69&x`?9l~lcX)R>(5>nxp(}ys`F2c z+s^|FPSn0|mcO;{!S4TmSWIVLu{7ZnTz*3BQ1}bYn=?-2M9Jh_n)_*^M$E58dMnia zUHe$peE;jGY4c{~-2I^|W0ZTeLg2qg)1;4zVtXWiXM8_B-+1L>`zgy~Z)D5;vRk{K z=ioi*K$rAeymEW=PNWnZnRLG(cE-&Y-P-m~x&C>55sBx1dCBLT_}xzHC&K9;63oL- zo^~yEo~~or^DC?BsNTAl$v;|}=g+k}ve(Zeh z``-A7hbRALou2-0&hm?9aeIoEtlOgZ^GLArjTGCQFuO>*57N;;BmZ8=t$DWp{iF7S zaci>Y$M1B1@*&{k)AqBk>@D}aIU;6y?b+0xd6Rq#H)yusdg=Fb#(MYLa(}MR|GYo{ zeEq-A+tqDfdK!N}w)67Qsf^PnXRqH|_IvaHS9jtoc1(DE$1Yp`RLkGwe#`v5p4a2P zzni6Nw9Vkij&GtU{_d1lUzMP)@qlfWf_Td*j2hWTD zWUjm>F<<6G>$R8zLf&;&lTU^IIj68I<)pN;P1Vu3JwK0}=hFiPK;@;Uw@uGi-YR>` zo_}lI-wV07*KVKxyLR5Eqw`G^W1&AF8w^MK2PePTD8V}(}xGcjvSgF zaox_=aPgx{di$O=@>gNk3f=gDJ-x3PiG|R?D#Za|(4d>)Fw@O~T&R)LUeunyeeKqF~Cw7@^db?Wm zk%O8~_0ePL-6uqgY;vzyeAxO^vu=;qW99jF?Rs$$$ENem`*9_9!%rc58yBmxK+4U^{vm=1N^`}15hEI=^?5&L3j`{Pix4aOy-|pbN`NDrM zUVj$f)64Atu-mRK`QYja3*TIMzxV5}w|n1z^RIF*dphUrnU^beZ|B^-TM5kdeOh? z)qdhz+ppWUoEO`A{^Z9mfsYSdZ}UIL9_K1A8ef)p^Z3gLo&I-OXRrP7P|+yDqqhG1 ztYa1N=cVj=($DhO7`Co|He0{+v3}pzHHq6_eVA`2^Y+8$@1oOA1uU5>PQT%|YP zcPIPoA0pwO6_y{}v~J_(UF^NjYf82lpI48p5K3n9N%4$4qV6l~asALz>;F6JRKldU zMy!6h>&eR77PnqLKkX{CzjJ|i+XuljZ`2d&S=cf_C?himZ=0 zcK!XL)BQGL%JyGB9yz{lQrwLa>B&+PFKUbZsR&N2nWFq>#m^_%SH&0C{rDuHCgalg z-qBsY{(kfGYpKQh`Ca=gC$0Yc_TaOW_4@C4Y;Kgk>;19o_?AG^nBRf9@rt?GY+L`| z%1(b6`B>+5kD$HLJg%9wiF>+Yru4tLv8DWqS7oEKe`bHHBwyXX52tivj3NRpz8q41 zEZo0Ns84EJRAO@7hb{75>AwFSWOuVZ-MsPW_3dU?k_^>yf32{)m;5$z^ZSE~Z7swP z9X!7BUPiQu*aY^yM{lrY1#iu~Qu0yPhu_Au&L%fv#nGPYtijf^`D}O9EKbqQ*l{#u zPL@Tk(v3GC`93~ed!uGc<~1{&j~mp_yC<*Uv$y2g_N(O7OtZU5D&pQnn{u{%+#GmW zwBO2Vqhv%*?4rLD9tnBzpI>_0^_qW=nOkk4iCe|b2h(%;=RIQ)s~4@=_blF2(x|xq zgJ6zDQSWg-``^cw^-EdXHQf%c{2&sMWzmuQ_yWK4?J7gDeWz7-cdF_)>p#2Zls!dP z%lzjgezS+4*50jn;r?{hj?=CmP1C-WJUU->*QPJtwsk?6K+m;yJL_+AkEc{*)g0MY_@aH`*>e#;!qS&7 zYu8Vk@$&q!kM289UO)VGji7Y=t6F}$AHVNeP2cKYdeQ8oSm&14jqNryPTZfAuy~ExqrgL3cXX{io44>H=keLD z5+xsx_=)qM-F|#AU-hTHziO|4KYerbNZg*9$BMZ#3SK_&8r#}T;qCb&$}F9_HM01?Kc{z&Kfp!VWbv%I+rOXxWwpILpxR^i->E)wZ_Sfc zjmv%u$@4Eg+{Rb8^I-P=Nq4VrKKAq0&*R%<4u?%JW;>|uwTIKxwilZ|A@~zCm ztw-B>=9&FtuD$l%J6n-$7KaX&Yc6`7YbqmAqMU11`HMg>xn=!=Gc(%xB=;sgT>RsPiT#sXY2h`s^XKW+8K&x|u}FV>+%9Nq zRGaKz@QHJE#fNJwY{?{5aw81ADo;kLlX)KD92tvOIcC&4+He?&<=*8qhA~4QcvUtn6>z zKDXj={mYZ`kw^CLtGyh#_U^x=#q0CpZRebybBQ8JtS^FNdecsvoKA!AV zuC5cvaWS{NdF#3U{0&{~`MuipXPs-SZuO>H$;jsjO3r^$@>9UO*ro5plt)LJH})yU zTHVclF{L=ywQrNQT?Gf5WlvF8_`=C+cbpcV7c{?aGz;* zZQGtGa^^C6m3U4q=yZPaWmbr1zR;|Oofq5Yt!G-ot!yh%^0Q<4+tGr%V?>VY^d{JvW&zr3;7ysINbcVi#ER%6x@*L6bxuDG6A!ALmbcS#`}aZ0KRH(}%*HG`UMA;%{kL;0`_3)8^WT4(n(ytB z_)KnHT~@Eusi9@!AtC~M7ijD#Ji1z`XzKaUJ$VoIJ&NCRYVQh-MKM2qy9=`36iH>@ z$l55jYr0;E|K%k&&)4rOKKH(4)-u&hze{tTRDPOy=8W-qv-4;6yf1!s=C_<`%8}wP z3-y1PUSszA*CsFXWo6tGhvuB3KkNUoJbJ_4BE$PuC(Pl?C1$>xFBq@z@k!luoOjD_ zZrPWhHh-gA^)DRsJ{|Fo{p?!n#AfnQ@ydmDRyLbNWxhOl5hT4MgzwYDqdOM3Gw@ke zc%E=%+jhEqrOeHqI)oQ;CdXM0+ldO53E zF78(FZ;fVkMR~iI0TEUdUsrQZ zYlfVl`zznSieu(&IQ!_}d6j}6^J4y9mdNnQV1AGqbE%Ra#r;w$g1dr{3VUU687W+YoIa6FV);?y_s zaf`P#7k??6U=>?Oxv^5S*`$U|Y|ch5&5I=keyFnB=|;VHaB$&;tc6pa?OWF9-nX3R z0CVmX$tOYzf(A@nuS_Q%*ci>dg~j%Sll*~q$~7#{Urg`aX!xgjn#+BLA5*TRf0)H{ zgWXQRb#~Hz=M6{sS1w?-{51KOL!VDk$x6Ms4>rW_e*MC^|4Bf5{ABB^j?%S4EA#`? zCOGz~UpQI8a3V}T^Lmn#-A5E`GIq1#jSGD4>@Y_>gMWfkoNlp20^ZvHng&6%~ zmFtul*VwPkJ;m_%;@-Be$A5*+Ubk~i_E&i+hu0AmPY$r>D!w;8aa`?5lK-pv4_j9A zKUuZ>y7&u*f*bt4uh>sEwwZmEuDP{rpSM`<3D3V>yfN~OA5KiRi)BEO5RrBue%?>8j=dB)9AKInGq z@eTI5uk5QfFe*&v`4Vvb)&d^gil<*5yVg2fV*D2;Rx@%kSo-JN`G-=lJ zHKGlrE=jSQp1th8w&2>5YqBN}R&?`8E)wReT&Z*Flkyg(hYRxS*I$fkW83-Yaw(_H zb&d}zrygwJmHO)b*L%JEg&=Xh>J;TK4^}c?QH`&->1)01q}@k{@1E!yBzb$n+H)bnCilsSiIWc`ZYg-wKHXC-u9U~_ z)&_p*CEZp=Dcw&F_sjfX%$TzLo69`|fd>=#6__iZs91_dWn7S#`cx;tzr*CgWS^o* zIn(3{8q{eY$fi+H6& z+ReXNKR7UX#bNciCtCe~ge>%fVE9v*5S-f8PzxM<;mSx#6`_5&#;8;)+ zG{n|!Z+w6R%ZMJgGrKhzMPBesl3pyspVE1&B zXpGGDJL)wbq?mQHe%+M%@M02A#s$XSKfQ0JxII>rp2+L>*MTkff$H>_(o-)OS{Cc< z+@hVxoSM?bCij)q_EYDFMau1bHk&3oOD@onUb1-kwSldt3nR|j^gX?S4qYM4Z4tei1{w{TL zvQEWS@rNPD^)Cdo@2XnA@cyKg|1^)+-MUoguzn+he$-d_=%DzZcQ4|Y<>D>|FXMl* zOYgG$)L^Tdvri#;Oy=!NcD0no>wDccT~w8PaKTXEz3G?e$9nY>R{Q*(x8|UQh3uc# zAq&p7BsLiEFnjQ9V(aQ$-pXWW8o|vL#kk4i&_QK~ZE8E!CdoJ*Hx0bv7HejhxFovb ziQ8?Vzi<95VzztH!1Jv@k=xSZ$zt)!P3=ZD3>G*2{bv7IF?qqn^A8%szXh@{V7wvO zDj$8+Tegb9>edEtvm1W z7U^r}R-Y7DTXyT=WSt5X^|$;7)hl{gOC~u-I$B#TVXxcw>u}voW?j3Z{3nzz%4aP4 z(=a{emF(23@h7M7oARxU$(S+!#_>RjTML@q%N(0+@66dSmtXkNWeG3!KOX|8>r|Mi zEpnDijc?u^*L;Sx{n3Ypb?!%P&PPse+HtPd)elD$r(v<|M8IA^Cal~ik_GcDh|m@ z3mg247Cww?j_GAD`7lAHz3guG2KSmr6aAhXa-X~D&%&Q7R!;);+NycCaY@*J?Kg5* zR~K~5{7d*g1@(*Z40R_}miyLR@x1Vc%Strkv;{BAp_B+7y(Ysf=KdgFp*^c6U)Gc2 zIh^p}l|b&Rh8HKEikx~iVGg68-y$Bls#BY6Lq1IodV6AG?Uzml&8HLhUUELC!un!+ ziBr4Gt)=m~yEHEJ+1+CNCH-5rf^)};%X(5NC-omZk^k_BUHp9F^IwXRTil}$KU(fT zC)_W0iSM_Q?+ys$I*Zp{W?C9MVcJB;c$M(y*B^?nm@5j9Bd-Udb-&Ec@=ubRA~yQ4Bcq zc~0rdgtp2PQE9o^x|i*5Eex)Ye0*hLxYn1wY_e-OYgWIWTyjax#>{7?dl9w{%V!^W;x>g zRtFC1$7YG!TQexy7$nZvamBRqgr}xoQ9+2+&lYk0w~vnB z+~nBpm1_|-IYfA>*+Z4~1?~)f%iFSLNS<1xClh?)OeC3WhHD~uUyP~b9`F{1;TvX^+wqh_kkQ}|{u>6a~cNSQ0 zWslh<@le=4_f_|YE2^_Mr4$})iTi3{=4fA4Kh!%<_4wT-R#G>&VNnr2Tk3#QKug~T&`ysyYu=2=YvvK7x_Kpzi~dQ}$S|FOdFCGJ)SPy0`q$jG~maZ$K^Q1yYt%>@^_=9_(Wt#~!bH?ro_ z%vUF-pCiSM@?q5>Fjh&O1iv`lW(q?SjhuP7ly~CjiJ;1d|4_ql%?(VT~loDGdsE5 zkLifB?7b;%4B^k4gR6eDq&o`y$WYp}Kv_@S|F3_cRHv zsLfun`8T(`g8Yj7E2`1^+T?7EQkqXaXuNV?Q1D;pG_w@O2TO$a{J1LDdC30Z%F|{= zKMwEraGPtiV_Cqpn#pm$^c~{v#dF%rvt6}6CT#UJdRvqG^m`&qytxl9GTNp-Vg3|>6PIb@9YsS8`h?A9`#Qn{8Azwdg2eC^S&YlaIv>&2 z=c1{fSzx;ENwe7fD}vPmkEJ~cXp`HsF7QNvqz%K~5*L>r?y^xQnfUqSa+vk&N|q|v zSlo2e+k0rW-|Q4G{ZhtH7t|#mO}Ebox4)WyQ>(u4{)Dqz8r|8yOvz-vlAiMJZ8h(= z6KTKZ_09doD!1V5^#rqfckk|97TjodE_Kq0oi9_a9AR>g^h;;pkPw}7iEqDMs*|^a z&f^&g4=%6 zmPt!bVBh=8HS*6B1{~w18J+Z|0 zoBotkal7F8t(Paxv`D)loU??-p0h#h?#gvr?qBGb=l@JIVgD~-7T4QHxcPIhFy8c? zYo<44p}=1U>k1*Gje@&9vYe;w%WY_DS#NTFCG&|k`3ZTa9!ymKyN|6ruKCBwh3~5y zcJ5REJZI}6t1D5-b6?Erv02A_fnkkk!OLBjOr5fWbEBF+$$VPK>L2-K*RrO|QdhqU zNxclUHhJM7mYVTst694Imra6l855!5@aW=Kj-9){UDC5&(9GswR&}JG@6{q!e%{Va zw@w^%)Q?x>-}&Ig!sR@3pG-Wg{dWpuQO;G1gX{;iXTHr{SifbVeBG_9@jq09%iJ3u z%9|ZfmpiqId*-Fdk1jY$e_POO^hbpu>I1~ZKoQcjtDJmvPP;mtBX>F-iS9~deo zy>XTE%)POECpRx=eCE7h_6eP~4JE;B{WdTC-8>jCTPO2^VUK~_Dvd>+ji0ppeOj?0 zTPu^uYUc|9(-H$S*A-tl&2j5$38mg_vH2Jr z))c>O0l%H^o)ETN1|!qTBnHuf6HfgphuVFD>I@Dwyl`fF`=wLHCWImjnRwGyNs`*}d-`^L9)J`gi+vhTryjap}^j4*J`mL40 z$K(z9O-{6h{dqE}(dLy3L+zJl7fH(t7xbU4|KjW~-JruDSMZ^+S*$_q?27RbPFYUE_7ml=JTs&A4w5?|+%)I@gRTVBJ;*OOq#C z7Q5TL{k`PhqV9ObDRU>O+WmAXd;c!3S?6ox92WUq6SO0ryix}FLWKXV${)twTPuUZ z{x5Vi+xg?dJc@R_W7_2I;uos*-EO+aOzDzBPcgJyxnp?uk#PP+vk|osMw`6r0YCoE@@Vd;Ae|W{V~=3&9UcCwk+(Q z!yp^`$Wd-vqOrMk2E!lCxmT`+-|UMy73iw^`O zHwa4~JefB2s+;we6Kc!n8kt>ko2hbpiTc@0_6O`Hep4p82idnMe>dGx%lnAG{m};p z|FfSt(K zs{5%QbLN=+^6b+oVZ8EExwgSx@eo&Y$BV(x`lbUjYtzo+T z%3!e-yfS4;$`&`*Y`T>(J?Eqg|F?_NZfX3zv^1vTrlS3YKD$Z3RAegC-DC1Q+CnWoJR>+4XMNeWf7+w{bSe{FJ$ne3FzvP4qJ9nQ|>f5*#^C!9UDF|EM zIJ;z${DELKcfK#L_-nRs_P>#3+Lypy^T9!V=F29}ub29GZu-ZH^M0tRFxu8`b#S#@ zNJH4SBThfu10UZ@rjc zKSA)f=&gnB>bX-6Cf8TF#cs>ynb4Zsu+Px*_J;mlDNgMscH3u^o=N_~U_o`=gHz>CY`c+sCe- zvUQuuw+k76BKxEEZ*kVQ+H?8eFGl^hQR$x+d74q-h=8>7AiiJ z-)hYFL6JSCYJFqBnNZ1=x9{Q>8@9^byQ20}aKD$l$)kg67ZbSmT)5O1`(Z+``2JrD z>LxN@SRwzyp)civTT|HQj!hpX^~#*ua*$tl(*|CJX$7=*&D(= z?#bi_Q}u4Hb(}W$Lz1{SXNCt8@2-lg`YB9{*!80`f+cN)wU;*gJI;TSD7-5u>Z61I z{!4bn3m4>naGrjBX7ry+jW$o5m~|U+ADp<#|6&5CY{kn-H9s`G`L|q24NvG|)GvA( z|71eQ^qvm)uyf~MJP}{_f?d<8lx3PaKnU{T*|4HS!TWTA>HGZjL z?khcWYub7jf4 {N@(kxVo23a@VqYpP--{3=;%DR#qQQrZ??b82PPjlExw4|edpf06Ai!n=e~Gz z{`)LH)>}Lr%?j?1lr3YLyB9A|ynbL~Gv^9Fzb3Xwh9ZV14gRMuDax2UkcjxP@Op!{ zo&~R5%`BTO*-imr7Z`L)UwBNp*jRImA*D&&@6*z5nTGjhuT-9RnC-sFGUHKW++>lR z7Z_g|=Qeye5q5v#!M=5MG45qf&a;a?e3(4r$Ru}b2kU>6eHS^J+e}i8D!S=oDc~aY za>D%uf%X~Rznhj{@V5SyvD}9#=lVBB@T54)B*cu%s4O`p=-KL3a*?*bod zhu!PWPH}DKd$GiQ%abi9&2D*KYSjJGTK2Ws>|*dVom&^#`;0gn#O6F%wr`@$3m1Of z&y!jDe<{~DHv7Mw#3=vkrMcCm>fVyB@Nb=BHV%GH({kL>vidV7I!2Zq_Lr3wR(r_oq=LIz8RHbwZ#SHGXN3Cax{4`CJvk9Kbt$t|fl9#*R{JkUtJc2^S>Wq_ zHFM7D?<&pACJIGL7IT!;1Kr!sy)j~V#H!Dp{U}2qcPWTl(ny8bJB46`L+UkIC zO4H=`3$8zz9P@Nhu&i9ciAMGY`8~HTaPlqi39>FaB5pO&zF$}Rs<`c&2A&kp{UzSh zOB|Hril#m`SG4@y#v?oHy?V{Y^Nrh7uKUUUd=a#c-FW8m6E6A(lzeNocW-DedSm{n zXwy-4{ZC&5mOr^L@7qKpixRF0obxIjrN4Qt`_-NI;nU0yy9|`LW3ra7KlrBNv3HHm z{D-Iet}Qrr`18KEjo&`o+!Z)jw6^?MDl#Q}-zqsaY%2>G2?^VXU*HTKhFNFG29F>1B;kGJi5Z_zFzBgxL z%+pQ2r4Op6ldwJAe+*X zB<)QbrnAVG-t=4N%EjM#{r83RiiwQ!CVSSlR4Tc0aScF7A-+l+7-+*7Ta#gmqiL zNS(53_Rr>woOHNUd8w<~+b1W2XBVY3w)Bv>lBZd6@h%1a#1~{kXQYo_NxzpYR9i%IOKfm>fW}U zPP0oC{mYss%O6yfZg4Al>7KPXXZqg{4(qeGq@4JtVelupMp78;vs?GGFa2!r`SGA} z+O{oyd>=Zk>4}IEmbrXeBn?CicbwA9 zE>e!+Zrh~B>}J^c;QA4T^h<}l1y~9`r#Q?Mc+~sj&J*W0t5u!nn4I@n`SIzSC!`2- z2$vjmpKEq%)|E}-b`B=5v`fE=N|i-%Zflt=^TewyXPVB%rgp!#T9X(1|Lv<9W69(>Dw3OG|FZb+e&eWT@#q=fYk$3W#-lQi9+@%>;)4wv-zGx7aVl3Iz z7_-S#T_xJC&EI6oL2tF4UrzEp{o&8RU-eSC!T*T+T(jQJq9-2VPdIBu`ExhTm8pBF zDzA|(u-`(YpS>z;-s@SNf1K`S?xy(;6V zdi6=qNs70x`hVDRTJ8w@uYfy%c}EYTx7Xj~Cn`liBwFaadQ~`%LKV-$Guu^s*1@ zgB#778VU7B5!zaGMB>ex<_?>!3CbjeZ$T)sX zL;r-z4pw(*POBWX={X-2SJ&2GY2*2&wt#Q{pBCFs<~E;%tG+B`j=SY?(ch8n?V;vo z|I(Z7dKUxP7EG1NS!NS(Eup00zcv^DhJTlTuC}~Ax3bUYnT^pFxh-Om;%h|La^A{h zUT!9%Zda6}q%O#_`C)U;hlXo%e;P_AI;+=inOw3qRN5jXQ<>BD!xC@+@J2R#&=P)d zKsj>DX}^mTlAFII>u2WQ&Hlh>#(cx~)w{iFcQucNe`4Ik`eUxumM1?Pl4n1Az?flT z_hjzQ>nGNKP@2r5|5tT-&W8&OTb4dvW_IiH$7Ou>7N73k&3@wloT%gwpY}Q_QKnZtNxCNaef6G|gC&g_kEaqhJ*qgL?3?UKh!@CokCiW19~w zX>-e!GI+9L&B03w$J!qkl`W92T%=yXc3@$Ag_`V^Lzzke>Li4_p>|DO7ob zh0m(!NqQGP}6Rl~Vk-eSAHHmVA>)PGw1W8uX`ZT&wF9hJ8!eGznE zo`(6B%Vs-PY1y-z@@<-E#VMEKdiP*h5kuY9K(}=czmjdQtJ&o&x&LE+vcU%>`Txn* zr6=3m|1Ma=!gqI;`UU0SZvky=3diMc9dPcJdFjyh;>5w_cArdlyf&4Peg6N~9`+fB zTjxAKaVaW!ZNT-tZn+k&uR~L}9(0!8d2RNSM)nPo9@p)^I^D@G;dPKRJIPk`@~~Os ztxV-a?XnlQR=?&dQF+y{&hkQ6zs%e_*%ubRR!n`tc(YNaV3M=+pOt^s-N`=UKJS%- zxy4Tz+m}C9tofJ3yd`$$^^nq!am}`~xNmgvU+}h+l|J==aciglzN7AMzN|X`WB-l! zrcvHsWaR_ipU*bnE7)B>mut&R9t+ROO`FeK)t_F%w7%0>&q~d@Z|OQ#kI3{r6IXT~ zPiYO?eR|5oRvxwaYDyLgr?P6Zf?;O?@Njp`acS_AE(+SIX`A}Q;KGL|EKfJ($S+V_ zxqlM-#GKyRLw3LBoqovZKk46=d$TUL&s}to?bMSG3e2}pFqSaC6lA*9=rMoy2F2zC z0m~oR?6cu0+7UXMYG_%h) zlGi*w+vUOKzU^P!?#b-6`}4wi;+K{7FAn8>xzr~2YRQzgO$8S`#mhXFtv|RS??b%$ zHHErz4L*K@{g*4x-ArfVzt(b%G21!Y{dJ=$tKVzZs#71BPN*<$aXh_MacXN%?A`1I zu5zB&GoCCdn#k(lEGd_c>Q_Oi|W!e4vCEAq2n>YD@mKQ+G*+iUk{$=e^He;4-Dq#WRu zFAef?j>@_myNgmqo7Azr@LJ#BkG-uTN!8PFKrA`9Cb}Hza>H%cOM9i}@8C zdm(`R#q>VS2KQ;dSbVG2uvRgpK5<%I7j)f~x9;tx)qv@aSTi+J5;RHE7ctl^rx%?aU-`BmxfOm7^ixx8!HovYQoKPD>feqmklg!zke z?J>8POL?OTz6i^Go!HQ8_GFd*C($1v+e#Xg7jjSV-6-_$V1KX5iwVv`wmZKp-?d;N z-wF3C2bH~Rnc8DsZDG6P1B{1;lY2N+GEU%s@5p(XmQ3XTIM6-wN z>Nd>lIvpQ0_~=Tdbco-aHQ|xF-P>6fR~BAm643v0VTsj5b<0VsJ=J$!P*A=l@ndUu zr_N1@=9m;#tJn(<+|6n(DgI^LIC<{{=0%QXsUIf?PuIPmB)))mCX<-V7KSgASo{7x z%x3Msua;2I zG|Rjh?3FJVe^kcCHYCS0>CH92p>*>`Z_LRlzE=*q*FSk?n);P>|IvNyb1x{e#~qWf zk$c9>RmJ$^;QBc`nQEuV7uB#S#Q$~Rdwaqyo=ZwOV*g}Ufyu#pSRWRL4(h!2NM)!C#~J} zjJx#31m+2c{pVeBQjX4Wp7%yz?YiA38pWlGZasWHOSyJs@1oad_Oi`=`_@%nL3*OD z)QhF@+Z?#%7Wk&Vy1}8(us5Y~as4N4n+vPXU+|hc=Fk^|i` zr!IBQeXu3sVRK*EPt{v3`x@9k2wFX5F1T=rdx@FD;_lvC3mxqma`Gl!k}TiN%+I*z zS65Cf%Y%zH3gNO*LRL%SYp+@auyedx$U331W>Qo7N7tqg{(r7)-OO)Wo51HOR->}@ zqTiyq{Hq^qse`{RZEZAKP}&`*ddp+iu_aqR1mB-r&HJ}0yeuee{y|T(oi$9n zr8$p(t;sgIUXs(!uh7m`^&v}H+&*`g_yTc-={&!@tCm*xzHwNWGFkjnoOAq!ShGdJ zb`0wEfquCS%e22s#oA9$oUbTt|3Kg2b3;sN#?$DR$ESbDka_Yr_HOnSy?q)se}wHz zQ!bb1IF)H}+fBNap`JMX8sDD}6Bsg7<>TU;=f*c*Y?S?T`=hd-HIs~<`st4>fol^N zT?&&dlaMq}co)&;n0Ud1!Q?G_uUP?)+0GXa)K?@d=ZjUjC7Dqoa6l^~*~nWj_X5`u z_qp3b+^Vg7cVrYVaJ92=sF}6o)#NwJVijypOyB#XGeMwA^~FU-eb0)D3I2Qc?td6K z|B+u-Uw@H+Q}LOH-G5K~oVV%MYNINKH?Gs4aJlQ(XX#vS)tV2>x4d-SJ9Qt6 z)tA&JnWB>ubT__l^n2`8Jw@=>(!vHY8%xd!^)Hy8>pygy))#z0+peP2@Ce)2>1AK# zEg81Nthr)eaK`h^bic1N%`P@EpYYsxUQs>4?`@yFRZ*SCSLJqY8OuFWHctK+>M;M( z;=|V;eA_8LSN`AIJl;>2PyYyitS@)|_dIq9_t^N5wA@XHS8P31m9TcnYi_AY6<7KC zOr7~A99~|^w)YvU%?pKwt2Hk>t9h*^^v7=EysL7PJ+AC@S;2>(<7qd!`?5v8cwJb; z95v~~hZj|gUeBsw>#tc@cX+wc+Y?RtUu{!=FigG3oFHwa5^hm+(@k%7iuT)GVu$Bn zI`-S;9RG*F>F=i;R*nw2wnK?q4&?8!vH-Rx^1mkB%PyJ26!-C9w@pc6^Dl2}i&rNY z7qo`|WmdPm#K~XMka3XliOQD=!Ul)^nGT%hUs?C0q-6cUycdmApjeZwz03ILzO_r}+7~yUCL0Zj{?T zcznD3_%i#bW4+C@#T>tFNIab6H+OcM^Zm~Zju&Q`X6O9-^K)Cp$E4evQoZ}{r2mRL ze|M2n#e-AY>wA*+{kpnd%xTVP4JQ6Ct%lF*4%w#7ul=?$a&y|veZSvjU%h(O`fhq$ zq5EuqM~+8otqiq~Ual;hVN+S;-&wY^_5icpzS;xDdGt2VE8aJ_DCzN6g6;lN{q4K^FhkI$**vi`f+y}z$;(>~7( zd#3dg8O4lOdbt{HYd=1AdVQfDW=}!24<$6v@^(MIs=PoSqocxHld;N~; zUvZE3y|0&>VOw4H|Misb8QbM<%uYV{xAn)2-o}Z_?#GJUkG=c$!o!`Xub6)a`yJ+Y zjNcs;Z5;y6Rt7NToa_1a#e8||mbg5Ht6$|d(Q;sW#Xlac5 z-cYt->i1oY>n_+nkSpMSvyfe+r2NRc#xnP~!a3Ckz=BZI8>$z~mb+2ezk4>*_bIjI z^Vzkdgb~ zu-oF;$+aaAhcJF`09&SE#Q#I`t&+5X`E^u13oL9NSl>Eo=$E$?tgry4&tYQa0cKn} z8p{fMCP%-NgR2JF5|9G2`~=&Xv!yUYK*A5?H2C-{7R1UteiF6|5}pv@0)CO1>^2wl zHZc|#-vtFTOn3)-*I{P44DsU@JEy*b`2b>->$Ao<-YwD5(YaSwg~k?LY~z()vv1!% zZyUduKXMt_*{?svX#R_<<_8B_L-m5?whz8fpZBdG@yYk;Gd6}Zznvamck`Ux?>BnA z2h#0-NA7;V@As}FkB=OE-}v?Q_1iPk=SBXJ{U`^G8Hf4cP`^^}=F*IRLdM^+7VP!T zuWCjOsD|nV^D7rTX#AGkZ=3eM=DGFT6^r{eUElX@>$l1C|ETPG(<&afVf+5SZ?`2J zWU~HxB{=upot=XJzwiHlH}3nc>#o061~0$GDjt)tb?erRy?5W-&fkA`-JVZgyWea& z{b}9)r_-X}{CM2I{V>1%9Y2eQEY%Mh*+1pRY4org&FrcDJp2BZ#r<|xzkc=@pEEcn zQ1R)c`di`oKaTiy)qQA|mq=zdF*CdMbb5TAyIkcG*|Hmng?<~l6@I*Mx6kUg`Q#y! zpTTGKVu8*3MXud%<`kdX*vc(_Z08{v#u?u0>;Jx9eCG2x>+|y}9&vVlz5`7-T}PPf z>NO`dzG5|IcR3@WR^#~a>iW9UXDgS_>oPLWzc+`Iv0RPCXXC!2r>A({Hs#*lmip~} z{r@w@=WT=ouh@T@*q>te_oaWgO7Th6=_yVd=B{sd&?~-`xjfY@?~X*_>FN6CWvX5* zocZ(f^XHj+F4XeZyepo6y5>W({F#;pOD=B5+-DD(`OoFt-DP_D(xpi^^Y{NXQY<*9 z&6R%7$T(H`Nqd(%VTstzyEUC|9p#r-QgdM51Av< zK0P@(^8=&KwZq|VGe0ZtQK_ziXG3 z+2+gtPWsnfx)l4V?}yBDXfCcj&Rkdj(mHwLc3nvg!{kauS=*8q0lyEh+a)Bn-@NyI z-}jr+_dc+3@1F59Vf9`;rse$gKMsHU^ZER3+wXUhWh5UQJYM_l=5pK2HSKpplpDBr zFSg!NoM87R<9);DQ}e1H+)SUpb=4}Z-S2k2E)&?4a#D!O`Bjq(YmOX*B~g%9H0N!6mFdmybtdA1!)(W&*S7CV6_2mkc(?ZZ z-HkpBHJ{I#AAhj>?}Cpf!>s4dd9K#5>%$@L$1~r2+j50-i?+7*<8x`Z{0{K#cTTiv zn19pq*m})?>wG5Zt2mea{H7@Le`C9R-Hp0mm*=|`{c!J>+bjIP-R{f69j|p~K7BfO zeqD|Ap}g@ii20f<8!y)t9tDx#BruV;N9-~fAjYBDrAKlBpzbne%}0; zx$tls@9D{fF|BtC+7DGUEVh4RxBqQ!{%uDEe#d18wu`?>DgN?Y_I*XWW+$ma zJe=nZ`O9{3Zex|ymOu9Uy(#tdBXd>FT&wLtd#2xoR22*Is}8)^ zuPu)G))vaFnx^xj&&=_2{om{LN5kKJd}TZ5#LQRm|9@RC*x#)4aId{*ge%Vv&Yu{#nn|Z$Gn8o?(tD)gX!*|v`)sEk&U;p#8*gY4acy^2A zV?7m97%Jbavl2PSEORnJ=ElY}^_j)jYm5c&eVG`3=%Y~C49neT5BD@VHnZK#-F{c> z_)J4}i|3)WHM;Gok>^h_^dssTB=cD?Ka!8%_uJO^w zhWqhd>1(PkU1}6Ep7!C_G$-pxb2Iyj3f7gKnxbjlk)M~>=O`X8FE~N4#nJnMS>dB2 z9;YSs?&Tb0u4IVU7bsaND`S89+&RnVZPvTJx#l}xw>jYVZI)^FBg>p~$IC?@Jl*@^ z@^b(8pEgeXR?K?WrbjYPQR4a46BCtpXZ2m)4XKqK;D+3?EiJ|*0rqF+w``T-VTfMeQcGTb2n|8=Ej3hopVy2 zT1eZMtn!uEw0`QeDaVvmj~1+P*`MaW?i0hjXy$dVXDxpFnejQt`wd9#hl|Wl_daE3 zI=^d~?zbnie!qG7mH*Zw-Sbh!(TfYUtS1{Qv1u$6I&`H&GREvk8jt-IKI!sFvX&1+ zR_m;P73^qUo0XTlja7Qy23OP1IZv;(hF#ur?^To;d(GsnDYKmYZ&jT>x-MP6YQYh& zwR=(~@M^8zu<)sJuFKJg*)}=L&Po){kTv-xp{=j`Jze12qaQo6;8F92m;cL;KC^=_ zmgT;Fb57O#RTf=WX=`hLZm8a!z2^xm zbv4$#ZIt8Ql58N=*M4X7m$2V|C(Vxx&Q2~!;aI}CsB^uuSo!A5n__tR1R5)tlo`M8 z{2ia9?pOOcjx*}kty^o-ZnOP9`Z@gOrM3Uh+^fe#Lby_Nkew|u_M%9VcW{&`=9ew(tWVA2K|F-N`gN3UTq>zb*V{es1AiD|0P-_idJ&vtM7o%|CzN9W%4Fr)EE6{qB%% zYp|?nQ|*@v?~f)_=htapj;x-04xEy{l=6!NU$}a`Hd5!C=;ayWHW$rygy^VGpqjhcjf)5 z%I!UUyKe`46}hm!t&Q#XN8|aMo_;<5b&Awr{BKVCb++x@Jf@~M)2502zTI`(=EOTqb$*4N zr(ezME$%G-cSaXhhukRUFOdD3{c_&#^%9odnP<4gGA_S7_KfZKz2NzqOL_I%5188+ zNF;S^Yv4cn!D_$C`Kh-L>6uojUUa%2`7bqE{AT~NI@5P%-$OR^-r1E~`t7>ucCp7b zC9{7oV3&)i?Y?S$<@ecz{C&N%5sApDazR3)+3w>1VVvhTf0uqi~gNaQ|-I(`5T6^)#Z1=F$yY_YELnM4SZt$xRN?)9rP@8)?Io=f@!N~GJFbKm~BXY{{Kz+-!xY);v# z;^Qtwl^t`7e{3jTEBh{H-qQ1rwqGutyMNu%b9LKa)UVyRapQvBw`w{RF4XNz^fWx{ zxP2>!&4#qwk0dg_UzxaEsi1N83*@ZV#Bxh|d)|iQht}IJd%hvrat`}V%?*4UeVtGG zCqH3+|0Xi`;(N~j_pjYc-x|IB$YQxxDO1*4HCOEg60WUSobz9B{iau47eBIxS>1mA zUFJ`ub?K|hvNK1_Z>hG|B~Q<3KHYxf!Ka0?+rr!8nQFfu3@^KJO@88^3xe{G{=D$0 zmx+Bp8D1#r{K+dZZ~T0kf8CGntET4f3UXQ=E6ea_X62rK^w@C1n~4*|`Z`;-c}&*+ zyiM@ol=7+d$(Pfrv%U%&UeF8q&?K|(UjCM~cfW)bZjd*6RuLWZx#z50@fzK`pK_M> z&fETOeWRIheb+v-p2O=Nyx>f~(fNDM<;2e?gU@Z%Je*bvau)B*sQVY+^)tV&1yvnY zSDEiD=Po_}KJ;+xO}Xj?Gmh=4>93ZQ5N-LFeE0t2bEQ+Ja=sVh+?Lemwtw0F_hKl@I%GlyNAdfx+~e(>(2`pFIS!F1w%Euzt>;*xh;W{%>4w^NCeD&QeLn-s0}Q znoHZ?Fqg*(G_AX8c;lXEeVD&qiFHgO zVz2j?^8Pn&wEuSfHzMCRK5NWly=Ark&!=r$KhE&r>3FPfnO5&MSHPO>9;cqmIYkA& z8v=Xw)!mx;ZMkawmfPa@Z>#F2%J{EYkzXe^hl9bgXJ6>Hp_c; zA4)R61TIXzSn|J!&fP!l+s=hA4V$|%&+Hb>FkU%3dRA_9cEx3Pp5o1nCI?G-yqy*E zN`!q{_J3fS_u`9>MckoppWiz->|MH2?b4DzroFRsORnC~NG|^q=$jsfcYJto=h;!SlnWC!H2;;dt~hhhtn|k|v*IU= z44gCVKF1SzONbY5mr$=u(o7rZb&#yW2F7liA?=ObtZ&q47M-2Em+AN4z zpl~S0Bd`LLEl=9yKTFU$F3(o>`t->r1tWKB}azBjG5Ul@v4$rgO9_;S%b_u86Bv$Qia%GSmqjj1h|VRPZF z!|d=EZ=)^G#$C`?7g7|S+}5+m;p>LwvL7F>I@T)hx^i3ev;X^le11JqQDA!8e#^(t zTISot3*BG6SNoraOQDya`j+5R%a3jE6)`!%S9E%f&7R7Xgnuz*rAIYx&)OqByIm}P z|NGYJl9DB+m!;&FUD?6+WRB%kIkVhbDe7|ynn1%9Wj9l&AAPpI=BaksdE4(d;{Sb` z9`XFTfBlDM`5V&rf4G4LK8{?ycI{YrR9INpv0eW`!CUseaa)0I)P|4mAC@!E+crHb zzGuDa+T430b-x!X{+U$hZNX=05chIdwf@)tVSNtMrcQ}=y_v9D&O~e9-z=}MjAqG4 z63Y&(P(HNm!MW=%|Gl5O@9N>ut-0U&Lvyz!I3EkkJ9*<*ZrWGhoAEO8F=0u{?6ZB> z=xv<2*!#_dkBV(F@j}OZ86Vwy@4Y=u;NY7Dj?Hg2JY2uYK6jV>{>d`)^X@$S;lZ)a z*+6FZ+{5cvRo~2-o%QyH=kXh{-tT&kJFn7iyD56SPRZws;6u?O=f{0zW(oIZ+CSpl zu=s(%zV^qe51#SNj@?k6vHWnbl+)Wrsf_cn-y392AC`WbczB=mv7F=?S02Cp_tgCM z!NvU>KlA-ra(B}M>y@j^qT__>c%CyCuiLgUbn@Hq&-?d9-}-m&#GEYtJ<~p&vwpv& zna@h&zDeF4i`j=tYGOmE1$AtgGAHIBtUDY@Bd#+Ei;>qFv(Aah5 z&NYd+%;@CoSJQ9(@$J98w`_9n3JGbmsd}3ptL@#Hu%27)#=Unn@1mE!rPT=lAcS8GUb$Q{`xMbE}`Sfg`oRO*YV1l z*!)pA7jXN6z`ek`r6u$hvGv+MI~>1jN6to zGq&dA(T(S3**u?9{3bj8_pH4?&*t~-_ofnCdQZPq_xo*nGkbe3Gy8W3!RmzUiqlum&;L6~ zcg;_C!S9EB-{S@{%#td5H<2mq zZTRu32buN%S>nZ-E(9;1eyh4)A|<)~-JXqGH-DLTutfWUn%nyI>-%?S*jK-rUgmvP z){1eHclOIYW!pM;XrI!`Idux>)i3|*|Vma3%!T_%&%U+Ah&{d zeed3~cd6UTZm+xlHJ-s#Z_)g!ggXybew*mLKV5m%HhcDK{=0XD?{c--S>?UD_V+=z z>bls9dH&X0BNiK3PGGCkV*kgXnagLsX44V7nlB30wQfiK7%Qi(O#eT<%-v?bf`ye) z&JBa=zej>|xAbzQ-`>P;_-T=V5WmT@(+mCfZ~pON^RtC})-Js`FLwWdX;;_u5`uq*EYtPJ3PHH&&E+I2>`>tu_+_fsa=Y7sR-t$0CqlMjVnopNJ z8@FkA{7JR4rJ^a$)~TmNem~@|-@*KDT6A6~f3laDjrrY@$+E>~44saep5G;I^ZU)_ zk26j+v2wR`?S8R*ZduemzuW9~9~iwe|9$MQe-m(CxcZ=NKmz}Vx7yY>OGg8Ik(p8uv@=e?$xVT z4Fxj(n>v->ag{$l)#r5|$MaJDH%}TnpBi)je!bK>^M#VIl zCKvN@_m61VI^J7**IV80lAj-OF=PJT^x1*}!Un>u+m0$&&zY~DcSiNYh2Z@7bC+5c z)R_PI|9)rM?{Cbdh7#}B)V3~oeeLiBi352u>E@h8{c~%UoLwKAQf~IowAxPPyV?FX ztN$_?M*q6#C{q0XiW|p|2YcQMoT_oY;Q4R!DgUj^$K?g&`IKL%8|aB#urGVRjAveM zv2xCV8JxSZTVm4SZuG#cHg~xPfT1~-tTw2-}61XtT4N6vcKKU zur()aS58{|=VE`|7H0nYJ+hTgCYohkSz+_9`u^{EXXn}8zV~h0c2^nxu6=*s=5OCw z{QOz@BmJFEgl1b7r`0%!EP8r+y0m*7XkzMU{_RhP_82JGUXa-*xi|f2FY~7N_g)^V zexScPe_MEeeKW6gRP2R_#UG}9tL?Ozu_qySaqYupC%`=PupR@z-YQuLh)07 zgyFHCn+Aq;4^Ccm;(yck-(TPC_J>neyPxbxWVQY|;p(o}|L@+rsFv`|?X;g!y6=6% zuV>4AZZ0gmJ}X(_hxWUe_M$&h3KpF9spad-ZQLd;=Vpn#75>crZr|q1$K>V9?{IwM z{KYM~Z*5iId(`#^-Mv+ZTX!u=dOt_O0pebj6O{wTF4lpS+f--TCcScJ8k)FDvi-`uck9vSn)V`~Uy@ot$v)-1Y|( zocT8P*S<+!C6#``GJ9X)<6}1O=Kue*d~NJ*v-s~$AFCUre|~;Ff5WY3Zy(3}+51WE zjW2&r?p={SbDJ!i_D8#KRoe;uShsEAQM>x`*ox5j8z-+7POM02d93ts1;^6~vn=(5 zPT!P$U6G=4CG)(z@Jpuu>z|w6j2FAs`eVx7JM-G}*l*47NH~3)|HrJE#q;IoRLz?k zR9EtChDSl#jU#=X4A#+X(~mAcxb?*e9vM!K{ScjDf7oxsX0<(LuVn1&YVPdS zdXl&8mz%+n!&BLL?Jl-m>3{z-u#orfE#|{gYPS@f{rlwo*ozk2o$2`OmQl)y2{xMR zE&sf5x0kxUc*6dlXY=2jj{mo5RpIg3pYLZLI42!$@kpuS-OlHa4$pbn`fcm_+O@U+ zKF=5LER37`*dWuO?(f&@x0U;CCM|!M_;T8`X*GAbYfm%xyg%eym$D?k*Xv{S=8Vra zpEuvSEA7to{I%T5t)`_n7b~o}niI2Q?>u`op~ns%B4>4+e|Y2LwT%S#Pm?F5b5H=qc&8 z()>2d?+srMEI5{Bz3FSra!F%$6T|!?8n1)_xoO2j9;wp|MUxe`}ci) zeN8dfdPdO=#pet~81MJ>N(kP2Dc`r4)9&BL{^+W-IqjZ1?LN#3KIkeQE7DwYRa`ep z}zhFuL~}ofA+LaPkL@r@{NsralJ?1?BCHk_lk4iM)x~6&tIC>T5cgf)$+o< zsM<}wvgS9{{O8H^_Y^GKy!yT3y}+CAvTHgXq@DJyyua)3+RUGp-~Mc{Uc0d>VSjk_ z_qfm3zR5}KDP1Fbt74k(-TbPWWsUibTz?O4*z32x{2p_D)#ce`3eWC7s@lIps$zK^ z*ZRt{*>6^@UUw?~`wqhbyFVSueJ3V=%qzsvPIv_~o_@^FLPRZjN7W{nkYN%@p=mx6C4)oqw=i6v+6U=D9p( zntD>~C*23@=dIejY?=vQOUr@vcMJK<7CAkb+xjNI`}hZgN4t;L-!5KTc2Ka_Y}GEE z`nzW}wKBpsI7qL)ln|X-)~dEnHvN2lL`wa->Pz!Z>TLROX`jQ{wtBsJeftiT9%`|9 z#H8ggHS|{E%$a4M9vsfSbwabyd_rta8SBl$ORxV_Wu*)3tm#O&K8L$JCPtg>_9b2a zTgNt@kB&}Fz83KRz1)if%&ET3xo6sLnr;5G;|K4VBb~yJHwx}|D1WfVuJ4i5ghLyT z%Vn!DUbj70wfw{S|G(Ck{d&3lx%`|*+Ua#J9pC?QzuWu$Ud6XRpfxxeWv4CF4ok;@ zrqbPI`1|jbUXT5@`~F|OYdiTqzH+TAdC|bk*I|*DWLqlR!`1a-e%_1HwYSf|7oYeu z)~xi}ip7VvAM)LnciXG_>W%9+4zI4euPOQ7^2g?Hm#&Gg-B;NyYxvJ(_q&(>Urfj^ zuyo(ybtt?1by?PW9qFX^Grn@}e))6#yDQh^1$nn!wJ>{TeJfTz_F8IgZ1dkrPV1|R zSG*7HllajZVBpqlqA^3vSI=OH)V95!&qlIM_)s#hKD*=o`{)f<&spwKI#%KFf>R@7 zL2u)oj2~K4*XMJp+~{5C<52Tp*~@J!%QoA_eqXfr!k$B6Dc5c;Gf2$V{Zn8&KcXPO zCJUq$KFIwl z`cZl~v^Zz4?zXkf@!OWII(seZVnS}S(}P$kJC7-smRqPQ_5Xi;T#n;B&y+2#lOu8; zZa1E--TPqIro<_s=6+i?Zrk50Z*3>HTIocZ-0Q=M_TM_nb)Pftt5oKc`2Bj_ZojKD zgabCe)_qX*`Sbk$HSO)LKPUK4(7UlA(YdBo>3FzEMts@LR6~L1$2%K#?Xuea=hJCr z+1>9M+#@-JDrS4vv$5XNo%z=8_4&WHDQ7;G-9D84_J(IZ*KCLLWxG~?zV`R(u@K)a zpjn=0C*IyavPHb$ON((&!>@Z!&9~-nn!J2Yc5vI-Tb%df1s{Igv*YEmWPV%O`Y-DC zn`Zz0{`Bj%t(nz1B8T=#C?9XIZ1GiO>5uJu7@W~#SnjaVR=QU9wB@`BrS1P0F7^F3 z%~IF)*IAc?%(;7-l{s8?Mo#!_AbgN@zt{2GjNewyO;K-^W{BTg@Hrzhdt-mB+&(AX zbH5Lk@6Ugl`gG&vI};wLE>H__Z#(BB^yS;LK84yx9Gu@?9A+u&y=8Ig*wvnH^DW2D z_FC)M>3vaQ@8;j|kiDttAIBMf)n}Za=UH0i zT7UHB*Brk0Y1(#|YxaL0^1CVMOW)Y}SjKACjOVqC^D3Xs{Gm|u%j(1xy>)v(_N4z| zGtGTD|LMhM^;-)c`)xXydic$a&ip>d^5<2vIQf_8_F7+^Exr5OwYkR2<8wB<$&1E? zRs~GhxTg5kFUHViE%N>+=jp#=zx#5l-UhYKpYy6TWp-zBe|LDKyiQ&AsHdR(qIV)p z?=E;YznLWLc0*BE%V|xpo+Vgzw*2 z<7*G8)iyj(I)6U!^723I?RS{II8-06^zU7N__wlwxra{3Gxg6?9HNV-&$0U4X8Q1f z+>3SWGu&T(6n-+d_S;QEPqX~E%=&e6u1m#LJZx3!jIZCkdoJ4>`@b*!AFnUdx={D9 zRs0d3#rpUw&P!RI?Q!~e`~JVODXt3V`uRRC6YKl>RYl;(XS>pMJNN&2s{gq9mh~m^ za+%xnf4=G1ApcpW%JRsyTP}uM3=Wo2wJ@s}j!5v2+vq z;B_rJe{bm38UNq^|F{0?wQJ7`-^`0IK5JU`<)ZuZLY5D2+PI&cpD!;jmuf!8_yxoI z&=h4RnFIY|AJ<+xDjqLWpTcqH(o*lor6zYf+4ox~)ZVT6eD>SA^1a&g>YwNQ7T@=w zRW@%5bME5D6R(z+yx8YKf83UN|QPleUzCqI|_@KHb~eLj1{zSO!W$_)2K zZFc?QH(RkKY58Jh)83oxzK0uk#%}x;z32Yp-F91k^f*)p+rB@1Ztlju%5UsE+uN_? z#1!((PO#qd%CbD=$r|UjoA2k}&Dwr;buZ(OW52?Fe{}9GJJq9Qo#nN3?TsVM&u?DX zqg=<7eQUGP=GpghALdrP&YqofXvVzi4O@?wZMm$NA>SfUP%L=l<*YToGSeM3s9zVGYWj}J@iSQh+#*e>69(A?d#zRy40+qN{~%VXnf<&n!Q*Zh2A ztl6@cWzX%ulFlz~EZs1FrghLy=Ak?_Iy;WwQ30 zlcKkqd@9P%`p3-O7VP|g!K8V%`S)Kpn#mSJ*Kss9zGgJjVA=D_fVFS=2J5+pY`@V28%5Ixsh z_Uhepmi6~PHuUG;m*n1l#5N#D?m@4urcB8N$B#W`{JHlkpU?gHI)mxU`Pz5Ih0Cp% zKK)bhZ}p2Ui40Ttk;nXRCyL&_A2iv$YN5BBL`V9i-A@+83TX4q2)UY}o0x8Gd1Rxq zd(RrqwVm^v#kx|Ls(xErtahW6Kac16rmr~;-vm5df3B#|;hHIE_29gjf>wQj$~5)R zT<-fHLXM@~RKMl)wC>)*fO#dB&?Pb#wB;0bJnPl||1JN&*!@7izaNjw-`6~GY{sNB z2lk!~4R6_=WSU#g%*puZ_4mC6{e68~(xM(8p0d~IKy9pNMRJ?-bz298(-R+BK0N4L zw#jn&*=Jg*by0I;b57Z?%Q+ThnOD7D>}xBU@_1=uHS4QiGKGBLH5m`&UNFl&nBMqvqFi0z=Y{QZo6Prp?3Il= zdUo@kC5sMkYiyXa`(5J)ul)}}L+RP#a)0K3+o-&5qqy1MyW33jw|?gPlQKPK!;)u` z)>V%r%MLo;jMsWARBW%TcbezbL}j+WXMC+o+rOT;y<=IQ#@528(T^+NPWy0;pDC|w zvFDtJ=dK@ao_u9t+846>;GuU-MH?rfcO88PTSNMv-R`G zPoDnl{@mC&s}ml)x?f{oX*vFWu<1kh3EtcBuWzU_{%zT@s&dNS+Ary#A&8G(_t{D= z`6@j9Xr920Py7CY2FiK+*KEBV=F9o}B9~*ayXzsJv-8An)}3)aZtG{i;nu$5cOucU z=Q2K59oTsI)Z5+P*Kc)9RsFWp+;5At+}(uxJKDF-o2qH|^wZK;!CJ51N-sB&SU>fd z=&~A)7o0b=)3-$5nw1-QwkA5v+J28tO=``_x@7L>DTmH)pB~ye{eJ&uPUmH}>JIa7 z)4Xc@_f89(&A_MvpP z^Z)M*{vW4Lozh-@Hmr?Z+wFEoS7G8ixddZ38L>Lcw z9lX7uZA#A;-I>1norvG8iORebri`Qq!POJZ0u>W-oe_6-Z6R#)y z41Iff*_tiYare?p&raRhF1Oj;-RltljSYXLr{&u?Et1*%Zgzb3@w(5Ivkl8O?fL6RuKy97{ z=~w^9x$}6}-u$kW|Nl)>va_y5#cy?)%?qC%emlYNc>DA>`>!r*k6+TH_FCu8mhTPz zX`s=?yWcPQD(;Be|DC7sZ0#ZDngvQ98CUO1?OwYwK8xGTKJok7AE6w(j^BE{EH2~t z`{HMO_A4%`^OP&!Vg7rA`HqeI)OX3*QSmR1@BOmw>o)ebt$p3si^?|rcCxv2`|thW z+8T&$ z;%6H}rQggrxzw!Y$A=$Q7Z>qu;oR_i-=774^DchfwyQ3ZXVJ}V>>11jbv)}PSM}^* zj`h7PR-JJ*_F;P5`%Nz&{+*wGe#Z?ync5$5za_JC?p@k&vhL%Z&}}=);iShrWs1`XLi*yp5RQh33qdf!)l)(J0^?l4F4 z_hikLVs>jX=HIg0Ogm!opF5WMLRNX7s%CF5oO*Wce38}<3@LI+6*G=>?EEp|?2Vk7 z8OzV~JEs3^pK5yR=kImAWkaG@W$Xsde-ytJsDarIP;J(DZe^n`G=jX)U&9Y8-lK5~2XT0ST zyMrRXR$_4eki{gK(3*ZaM`l>e{zzw2G&r)M9MHOtz3XBZgHJO3;^q}rJ8 z<*uW(-OTY5JC^^j{N|afom=PInwxiWUsAt>0k_rd2}{pDx;?dcVoHrSr(;q>_#^p{ zOIh#d{#hhzmRnMjEXZhYKjG@+4r|_HR-~Z~r?PGRWx9ZjZGiSc-yLmQkv+7)vOE2bL zU=d+)nfOXwNnXB#YemO>`4ACNk)uo+9)S;-LniQDvgXvC`enkrH)^xr*8ckc=iF>- zlbcXoUL<#; zz0RZpi?GK>1z+u%c6HLf$G2s5)2!3%bGvEDZL9Y9j*inik2qTFyRKblJzG60=C{^ajTncckxK*F zxq5$``?mbW;kM`P_coifDG9VWY|iFSz0H;B^sr~e_uCtCzJ>^TXtMNN;d!6uUTydG zMTGmhFW;sda?qBJWHE~N&Rq7~Wn#-AFIOFpsFZ90`IQfE9#LVnSKOHslH~SF@7Cc} z*LUCD>8LnG(6urA(H)NEXLiQ0Ges&!Ji4nblhsqunt8Xx^Qe_oMv|QG3}qJ^xo zruG&cYs*LuR9SLMVc%8Jr(eFF-2HORqtYpvk9BYDLEnB)+giOX z-cNOFb3(39@|vB^r&Twj@qx{c74j*IV#A`QwPqr$$<2hxboUE;M&9nHx z-*URyb*`tD)r&MvRC@5KG_dkmh}QY}3-n)4Y+CtIw6RX$Ktmo|(EQu}-);$IeY!qj z`rGHLf?3$Q9jZP{th!e!6WTY|Av82JBrHr#*kXm+N@+Q!XE$3!g=YLYEO`D^guC?F z(>EORwzrq75XldZp zA4ZB_rcRw|^wBI{IiN3NcgI(unt8j^kG{X5w=MORpH6p&t;>e2WoKd}vt3uD7PEZH zn!_ljy@utnw)gWYjgRG}@0Pe<`!{u;<^QbA4IWJGZmoy@E%!aW_^#Njx-P{AJJaQF zIDXcMowS>sWRO)V+E^!epqSN&rS`(7Penymb5riuvE}B*Ih}ocZKD2U-=>wpEWaEN zygRdGPweH4oaa6-?G-YUkN1U6y2~Hu{FQAZ+rhR@+hRVZIkThPkM8U(Ns89h&D2|R zRaSLNuQ}(Z2DzwB(HpY&CGeeGGMjT*>Dl(6YW1_HvX?M(zsgKn=qt8n(b)~Nb^0F0 zD<_%0b8E>+4%=At{rDHhyVi_z44oTRSQT_OHz=t;`g#BJKKI;pV!w{>`dzR&J%a7M zaoyfW-bx2~{(Omg`?)0Ps(5DB&63KEQQzJFiI*Ney>nak=M%!0MMN46ShB3mKd-qf z`&IOOV&topN)w7W5*UuB|9kW(Dbk?zplNZr+SUAf{j)BW1*DhB?ktX)S+ZFxoUMM- zw$EzGWpa}v0+#5p3Ox9lbR;NvcK_GbSjm*~w7@8LyVPyd1M0q4Xr0hsa%ol2sue2) zHt_~=6iH}Ln)XJw;kmF-u1}kx=DBm{rWFf3TC_|}OE+tA^6qUx>EGRMU7YG#x=tr6 zky~~B-pw=S2eRwhd@?xYT^#7SYqDBOd0?Pe=ewG?I6+5SmR(;|RaHOzaSmmzT=+KW zW}ULytqDJzwmyoNmJ_&}D0b$aIwBHn@w6?K{3Ce$YcM; z$P@d`?Oeh|?+FBQE}WRLy;_P{^-8mnfF#Q;8(ya^OXc0wJbhgS16Y1J9C(rCyhwGL zM?}7Kb6#MvVQERpgv~cKA4{-kUwi&S;N-REoB16cpLxCO^}5MIe}7%yFBf!@=cdsS zZuc*u0+}9RpS?f3o@MyUW|o0|uFdvkByy7gpn z)U~_1$;+l+XV~}WwYBP5{rH0ATO{YbK656hvpYSy!)ke>@|&{7cLk1Z`PsnQ_2F!X zz}vUJ(Jv=IoBFBANmb!gyU(qpgFbU-3Uq6AT*!TWXU4n3%NaQ370xyH%8N7>yQ%$S z=j~pxaO=Eu9nnh}6Ay6d$9l-_{*xOt*JFv~`Ai>uyAJ#FJ2LXG>&=R;646~%8P@fC zv$Fj1`Fg)jHMs8o_MrOZ&9mi~dmc3}T{{0}N%;JS#h1BHK7GpGRvu`w&R}zk-wM0= z$DaB<(U(=Q73Le=lmo-6R^OOYl<+_U|zB|H?bs-}dRU zNFU#s-2cw}U1qS}($KH(FV1TFuIdx#CKy}%;O>`C+t^=xmF*95y=Stk;MvzNKaSnD zxU;D4vga3Rx&7K2uHIjEW&9LvtZ-*|%-S(!Vost%n^mXeo{eJe_r7{#meX7?TmBZ$ zcVB7u?N9zLw^LfgE7Wyr{f*r*SKL&jr=@XU8lU*7Ay=TnO zP4G;U*1i2%P=D2)=CjOY>=M=YKdQ?5F{E;^7pDF-vbtEA_2Hm_W&e{7n?Gi&1KlV2 zDIJlW9P0P^n4EKDpN-X;Kip2c*rW4wQtsZ}IJL{@^b?C4701KpdV9Z$nZlGNqIG+>uEUOxG2g!S%-UcvYn##WK#i*Tk$p2VoVhmaO}2i2PFUfh zSbv%vkEmclYF*;PxuPGocd80`+gI;@G_&NHp2rq(aKZTHL6+i3O6M?0rq%2mJP z8@{hfbG6uqLsRM=v5T+zPo&*EexdVUP2lE&s6Zj_2TatG3$SUC)1BWw*n8qs>KMifbpTM>1!ZJ`!Uzwn%nh zd)PL4qgtN#8dhlz#x2ToD(CZwlo@>g5^EzdB}w?0&FT}nb1u&`^J;ls=HenfSITAg z#hW7A1TIA_>0rIx7T?Tym`!QNwx^v2F74qmuHpNNByS&#YqZXh+R$0|in?C!|cvhyDo=(mXb9udOaS+W=pHZ&0gx%>As|ZrRl-9 z9VcHtW>u*g&wAufF{O%tLXVk01H{CkPr5&eGxBa-Qe{MyS z-@KAn4Id}2dvdtaV_)Ns*fm8=pX680%+)GsI(WtK?nFoZ+}Z#SvqR-ow{Dm`Uc0B^ zq@+K81Or$^7{Qc~aHF8-7RN2MYm6Qx|DGK8$$)9W zH8I9xmTA*DglDBXRUMB~(Yckj%2!&-eA6k<6X%b_-~XYxdP0qH!TR?}!5ci)N)Jmza% zdfP>>rq3y4i>-V*waCSGnr^h)?(+A`K$j_ZM(OHqR}#3+F=M^g-u}Ya4BZ$3Mv2sh z!_(h9v^pGICEq@K-mz{OrV71}t8+hJ?i16$`uW(p|4}bf);(Ch{MwF)xl=Q08t1Bs z-qbjps9R_9R$^xmkNr|X;oK0r>+{@w=PgUn3~*H3|LCyUI>n;2B3|Koj(pk9UA1?W z7~Ly4g*_HtzU+E4^QO{Dw>A47DQjFxny2n^@5uR0Pb2;{{{F~PwxEi$lHua9_1l{} z_mxvg+dyj(tCY!mPPU%#qm znPdvxt$aTB%h&bwu}yy;%l}`{D{X$Mzy441&SM8QXI<5rWmURr#tI?-zl%9En3@=l z@%5i~el2ac$X8c8BmK5pR^ip)_T@8KZE|=|ynnV()l}@j@~Y_%R7<9$C3}nhxoT$K z`&KV~N|0*wzFAQg^V6otIAwk_*=lCQHP>ZcpxBdr`J{LY)Ol94Hy`w&0_C6;CL^x zk#%y7yrb49EnTBt?iNlK`RT#ux*vYy;GACat3&>a``k!~0-e^bV|#46Mp?|=DfS7!_kLGDegDwlBMbe%mfbk`>hE{m z&F#$LlQ&J#oTL&hb=6*JMO#Bw4cpfZ za{pH96qbE>6Zhf4OhKIvo%Pj|)xDi3|8X$N+ur)YCi&IVFCT8r4(>|fZfkxM{P;|SU7+F`T5ybfGN@b|DVs9FE1^1w4dWV*LAMa1P_h`hWxjAORT5IJNxJT zn>^`_OV*x1t!(M{9Dl!DvtDxEr1XvK#H>>qp^Ax{j(+a+{J?&D*UL@IcfH6zy?4v@ z=Bblx_w;kFe| zM{#n`sSed!8{DfBE;3cE`q;WaCGyA>pF@F~-5;Vh3(TCr@+a_auI}xZ&yw>Ot&ZQ{ zXB8tW^li?}^vlne&+$pOuXGGp%VI3{Kv*vRlB-ft^XH^z^U}N*->lrMl;wPT3w!>l zQ=j9!`Oo}VdP#87Gu50;?{{)ZZ- z%renfv-g`{UfI~WpZ)qBlP6zC0cLOETTI3cEa4d;P-G z-~a3Op45Jwn5K7fm6g`jgP&g(Rlk_mzw2e>v?OVE&8;6iz9oree6%e#7C5s)<*@Xg z3+G*8WbV7y%$QeKw6@o7-ODBOWu|*hb$x00Kgl|25wG*3;4Zb`rF$oBt31QaGi8Q$ zca+!CpB(y%_rwY|IDNV_?RupCx~XwLPEFpjTK36WpG7gB4o%&%?%`#|$KM?$d75pS z^=|$VE6d~*b#;$h%9>ZteR)3V_O;a1*-iClzrD_u62BawcFgx~>EUS(`m3v(w{2OI z)NNw(`b*ix@As_?Sp`*3eR%F6-lkK^`2 z9h(a`MAl#OR1b`docU2`s^}hRo{MpNDh%gVJnH=NY^Zob){)H{{&p!Ok-Ec`OowD|L3{A>WxoN zPrsa3{Z7&2>grXij_|M2zu{GVsD)Ea(^iYgktN~P*SmB5FJAjLAuII5vUwUz&JGFg z*Q2V^k54Z7^Q5?i=T+1+#fIzd0XDP!rq9l+nYw?*btQ)h8c|Kx9AqY5p7ld)@-%Jl zy3>UUnY`tRi4SrgnQyLnQ6K;9LcU+(!8bc@taatmJk=&%G~u}ddw?wCMY}a`6lP`3 z)+pP&S0}NJ&#cb&+)t^m+yC6uSup2w&at;kL`$<*Y?~|Y@U8D~=>NVwQA&u)l4nSoXPjwzn%Tx{AlT#Q%9D-hBP`w%o<_e_zLM(wzG7 z&#UnLUJtVj&)fgE(T&*9ur2RyRujj``ma~Ro3Bqe(Z2dHOa2SbAVmg8pY5w>$KC&# z5p>mC+qjBjPxSZvtlWIj+JrCqrH7pvZann99NT^S<>J^I(XBh>g!6xX+_|$Y%AjP! zL;bqMhezxSau2=NuG5az`^hQ%yKP&ZXK}?+`KUF5g1ITJ=hc6d@Niz)>6#v}OTXDy zIkL*oR?L-c%JsFZlLJzo8}KYp-?3m~_&+6{KW!5Bw`V&qe?C_u^}zBBo(~h1^gY;1 zb+-CCm0s0k5_C?Olf3GMwtG>K#KuFtPEJ3BSYKA8rKK5se_!`~x6`)X<1^FeY3{mb z5S<~T^7NsL^!vr0lhsZxx!v;fq<@_f%iS3JUoRFf+SYV*-m%!Cld4L(=GUXQxqG9Ps{#{8$*<^)uk((6a^-vCclU$9fkQ{XZ!fuU zdCg1?nT-bo-5t1!&x&nqOStwUx<&f?+Jdl%V!J6ns;xTy{m5&tDRupmv0G3$cLEE~ z1ch6dkNYn!-t3p0a^sun^q2!i?hVJ8YJNCo_5OZ%Dev1hmFr9f%{h-|Cfk3M{byoz z{Uhrtsc6p)D;gZlChZDVxTWwebf&T9-N}cS27*UJdGhsqPt4R^6+Us6NZ+22&Y4VN z333$=8V??lFTYdx(@Pn%wl16-*?|xrCI*Zg7z;mWhd6} z|5qjKqVlp^On1?uMT-{5tGs&njNks(3*}u;r$tvq1U23|b=Lg8OwZ@?Z3l|}d_3OG za;D;5`Tg20MNhqoHRlI3i&qH=vM8)PwEX3*bH0L}{=$JQJWSQGx2I%C8?;rJoPWxl zeJeA3_ENu`LI;m^GljdlICdyrapm!EX_0<@P5+NfVVh9=A+}3Z2{VKDMYu(;vQ1jV z+v6y*Y4c{+MLn!u^H*E7>G|0A%IqugmQ`6Grvu0o_{gy9(Znffvv)lVHY?e`e5t(K&9{oDxt!m` zOT`EZuoPLJ&P@9@(dmx?D2QsxcwS7FFFYb>d+YhO+j)x%PV0b9xoLHft}Wpay1~Mc zCe(Y_n$v!ZsZ815Jzp+)zuftJp0OjxY@w%5TFSP%T5Y(&bXD}RLLvLcaB~ZmGYx#v zCr-XwCNA$1%=J8|?wEwtCZ#RQ9;UlT%?{=b`Jc_dZ+cpvWy`raRji5r&HrNZsvf47 zG3uSsS*Y*(+bq`Zf7q^qYyUR|HZ%X~o)_{@c3+y}tqFZ@vzMrf7Q{$CXkE{Ee%)t- z%55G-e_RebdTeuDrJJ~BsS{ICkHNL`O3y3Y`Aqk@K4N|{S9$%t?B%6fWdp@aTv{@k zH4+pUFSdqAul_F{ChK?qq}_C_%$R^wmdyff-f3z9${QP=W}iuVpl>|+KyAy^4?EIV z&YAHec+b=l*JG9?4nJj9ty?FSX|}{q;>03FS(jZ3w-i1b-}Ya+W!;{X*T19YODImb zH`o7jO!|pM4mSEfq?YJy6y%t%@$hf)Im_o?CREkG&5mFEp>;;Z`~y=SiiXDsuCefu z`?vr9_y0{x_N`spd$0C;>`j9cEW7@GyM450(av`-UMReI!I_kQ$6J4|OZlC`_MKB2 zIV&AE&CcR$HhOjUX}0gLRmtUd&+5*<@MhBCjO^F16t9*ay|=yc%h$=4-^3-Qr?d*c z+|#49>)CaJ%lS1U?t87z+HDQ%lwVz59Q5Ym<5};evvqAAKb-x; zS@j>2)xk0W*;)4PZ~ljcg>jv1%hdOndzjs(Q=rlQ;be{*GI{y)UCzB-esS5O>n9=? zJo~4_P&T;Ck_1Kq+_m2E)Gjg{`fvi^B8P z6}H?hT(M4q{c?x9d~Hep7PN(GPZQC}<H7aS-R*lU$9?PQp*IP}Wc4eo2YP<@)=S*~0hL zJN+9C7S%7D|I)gq;4J5wgl}K}u8r-9KA|PrvZCNmmWWoy2bcRPyg~=XtJgUpULc+xYKfHNyOuSUlNZ?{W`I^Z`7JPB*UH-Jw8=1_@kf#a)5$)yM8{hhEFNbQm+{CR-DuogX1adK zv%=RWroY^BYIVJ6qn*G8Yvar__ima@z@bu|ZBgLafL~6a~Dj7(yY!EQMbBl|G zhu`i;LjQ^lhJ2pe3nOIsjJuA@Rj>KD=-Rcgtc(nfxc4_UHYX?vT2~+Hv%F)z?;~&2 z-YV0B+bm_uzWzC?U+1{tOj5~}jR!>jmS0-pDWrC-e&6@K?@y{Qxf>gN*JHL5t^Sj9 zUY6NgXtycvd*N?tFABDQ(%-UKdzFpZ@~(5Ue&uSN*<#lCYr!}BFXz^&H)rHzM0x3+ ztI|r?I(g}y-}&);mvTJLs*AqyH}2vWQ{p$B%3rqsg7Oc!duPS}oqShgmh+kSD*Hqh zwdzvkFnfVq4h}OjgR(ux{2~e?UWqZM89b;9SSf+XRqaXL*S5w_q zX?ZO^-LJEx`n>ky_4ZNjdK2FMd(^zjJE@U*`S!I})mG_DG+QDVrrV`)(d|&%l%$7( zcHPcd?ZpC*41JSdtl0MR%8jc&(sQgc)ZZ?x%IDttv9J4f)^_i+>Y+Xhm`*Jf`e%JG z%q{q`)=C9!|0_WbB@4|tUn@zZe0V9#BB6itU)+|5Q)N$DrdefhOsF<7^9VXOPbTZm z9(mKHZx^hav-iol#M`qZW1pNZn0Wu({3&l{M6pJds?5y3^TYkX`Fxf*hMdoqC*D1p zxlwzz(Jn5{-Niane^nm8WVn>2saO8j`iW`B>yK_}>1<=T7ktd|Md8iV=`+18+oF9N z**iY13SI5ek-#*s;1Fk0*2NfSBfg;jJI~kZd2Dek*q(FK>4pIZpLK_m`CtC}AIt}Z zcJr!fAcvj?{$idEen{KW=ynL(#*-U!X5lD!(2qR1*DK%JN-CvUT>~9na10 zdtbO17y4hW@ycP_O?q1%Tz6?+_v72P(q~4M{dbl-{n}@BjDOq0M#+vcUv4$6xM#UO zzkgVYHfqcFo)A6$z}8M&QKwqrjbZSnm#_J+U%E8ck2CoCvqky;ch8plyZ&RzuQfTp zt{biB`YJB3$~xD5mQ2^@_fNhymaf$kTT<1!*{#{zZBE8wDUB5quW>3qb+uDCdG_qt zq;IQwzVffO)=Cv?oURu)>;A5H(H&m*_kONc+Q;tMzC?<<#bJfxm5FS95*{BGxy{IE zP-1=gmHnrJ{S?;8oKt*Hg~ofRu3ehMbmrqWb&y0Qn>p=C%eN;<%zporHbS9KTjt6Pg`Mrw`B63;NGL+Pd0l;uynNa1e?+-< z?9K<@Q##6)9I#yUsAQW}8Qa}`e^z;@N?w1NkaPWGZH6z)o)4-En(d7q%Dhr=s#++2 zC|xOcs>;It+*oP0)SH*pqV}zxJ?+hlZBDF@tnbGCdVa~ws`h`3-J`iz%4?pmvOnTW z|CIl-#P#cImyixKEe0XRO^GUZ6wWA|`mu&{_ZEw!rk5;iKngYq$gap z%=TZ(zxaO%{u2I0u;yZ8ieRneKMvv7`(~>cWSc(XXmsYc-Qu2^c5cqi!;`r3>^`3{ zwtX|fIbLQNfBM;3qCMQZA1jHzb$X97acPCb45r!ZPoX+54VUI2{&3-YGc%*1kmH1#ALdVF>QnPHe9GMEbTdFc=NfZU z@BB!GL_6(;e!m=q)YknROtW}l!mA@%*{nyTXmYxYm zrU;3xbm%!OTXOjD-G}Y+deV$%XPfK4;0@wuY%jbZf4t4f-|pvE51f9a z-ilCox%roT{HDzwr4>FujAcdAEleMk?X0-lx?*mGx$mDc_0?L-H=9~-Q)9XwUUNF$ zh=*&pfS&%{PtL#6Y^J*?eNlhqBK_3m;74N>mLtrfOp>N8BG(@#`u_N^n5CoHewm$u z|Ae@vlH^SZuaf!-144A2dISxGy`6WR6+G&&KX5sR1{1R~L#g26$GJ}T3J&u=3+W4R z6^!*ai~TYC{-3sYyI$)F7A3R?+|YRc@B^n6>z6InS1b!Ark!6*ZwuBcE~N0Gizd< z__Q=XYW|J(22Pd7Wzi?Rgv_?@EdKdDs6WLxt!)3gjgn5+IoXch;J9bLcEVZZyzTq8 zG~9MNrxh|?pXVW8tn;ay){n-Wc4b+5q8kkMY}}mwXL8z;i<92YO`0t&`Azh`2+Mbe zqYu?YN(yFjEq2>9(fv%PnbF(Rl0}nvRIXlGy1r`Js`&?1ByX9A-VS)Zmg5LFQ^b!S zZ}aP;-*hBfS4TL%Jbm)OIklwVsTt0f_O$qZt1UnHcFpWq=f_2Ar#=eaOGsP6x9i?= zpFK0wo}6EFeVe`0(wm_rH}{-h7JOXopIl7w<$%flD?Y4Im=W-_clx16l`U!q=4^C% znWJ=((>u0tJ@bV0+cOHAl@;ckSem%3|NKWsuGd^2^0OCDuKWL#Yu6)r>y>wpPV=_- z$I32#S^T-kh3EhGM4E8_>hAAUo1=KsBj2yEBsW8O{|?1l8a?pIw1jw$n3|7AH!Tpd z&f&(uc%wN_&ySrH^MLfaQBhYhN! zcHckSZb%k*@_$X<(r0<{dn>=1+`9N=*(aum`O3RqH0Exp zw%*paW6BK?ciF&Y`u^hE`4=lcsARc#?9j8|Rj*z&A8R^ru0JekkrUG*nFog+y0`53 z_cHd+^rshU{xAIgI`h~LpHeVZp| z%}Qjhe(`a0PM%fqDsP|nv+f>U-0FTu;B7`|z~xZLB&dfii@}P+S|O>=zFb+o=bFVT zjwwIXyHdo2Exo;@uD+S1Rp{fF@J%>v;`!OPj>XT})oSCce`J%(@h`JFHw)yMT~)7B zxZl?CIIf}K_n~+ErvGM&nwl(J6Yy;(!_WRSYpsiZtMt}P4TRXmz|@xEuNmPWdGEru zFI{~~20|>C?%Xu7t8Ej=otnWf>*?dSU=`yc+sqoy+O<{I!rA;gH(rz%&hUT0G5Mt2 zzHd%Zx#wm^9B$k8<8jQp=)yLk{|lOIOFmsOIn;Mn;qS$~<2w{@RcN{uyBU6Yx4=-T zvte7l%Ec>NL>OHY3^*JX&7D5k{qoBh2U)HgbRDo0l(=8UAo?u2YOg}ZspxN1|@TX5K|eu75wn_wlLcU}sE7;h4d=cgvP3eEF9B_loYiuj{#{vU+Klx){fj($7r$wgs=u37RSJ zHbZ|wx9eOtZGN97$AXO-LP5DNV&=QtUTIOwY`bCWq*r$=KP^3w6PJ97)B1+a_G#Xy z>pCvHW8gGjxJLPt!NvDLc(BK&I=G&(QRtsymWVd~F`gP;4Md{O3Gj|=yJbF<-)9tmytZOAg zp`pH9UQUbmHspQDmc1+V;3i8OD=X`yi96Swy6t#A=j_t$vrnBoYF*%DCaZK+(bM#E zhVqWIpno?vIrXyqyIef$#T?GBCc8H^XlZM=rihD+cfZY<@G-*TUWdBMoPO7~S?x*- zAYmk+&0=um;pY->Z|hG-yX@PyvQ_KXFTcCHJUBL1Hg|pYQuDIs7u|Iq zuedtpF7VKXZ@a*?*P3TQ+QId^UCY#Ty%wTN55foG9CT z_IgLe*7!RfTP80$o}mg|tni47k!P7(5!)r+lab%v<=om_uJ`t1-jZuQQAMfdlhyZj z*Sz#ovgW`0XJzEK+P>AZG7=XxG2}nYZH_a@XU{87pLBSt>&v;VQ?A?g9s8jcSeUrc zW8>-5r#ZDeF8mCiB)fB(LHyhe7D79m%B7#Twtfu>z2%a=wbtvk)Y~n)EZeu0J`b{8 zJ#|Z!G4r=AYl~8D+lCyP{A|@3zgg3+J92x?2??5TR?BYbrL+sI?IO~`Dm4!eSu2LA zNUQcpcWzs`h3kg$^~T4Vs=4d zi9?Apmo{n?{yH%=JWjK9@8@&Y?J*g~W)C?ov2SAUO!xEaTVdL;pmD`a)(NYOj8Zz- zMH$aa&$F+oef8$el51i2vi9zb+~@L8^6iTk3*NnZccA#yV(s<6?EJfAzbwE1ch;rL zRokRFOjWLD7|R%1HkD4iI7?xrOLai31G6SH*BUXo6|v0?5YUwQs3|#rjpG)>1JyQi zl5Y%OG#9M))?r!ZH`nS3Q*(m!q+Zr4g-b8wC#-4O@UXqu!|Cdr`x73@mauGL?ATE0 zXLyk@knJGbM>Ch(FH24pzbY^Hx^?;cY^{$M@)Rz2uqXA-&op=5F!6YXh3c)vjk}($ zgeJyfmMZ>d5q_@`6^{L6I(5a|1_93Iw;Hu zX`g>wDgUhUJ^_ZE-2WvHEq_u~a{J_p!kO!R_Rhb&@79?ucVA4inX+5xt$x!Y8#Qp1*h7b zCR%Z*2rciBUNqP0u5t_?P+rYZP#URM-D;pf{2)uQPa-78_t9+^S%C}yVMYHBX z3IPUz3m;X#{&G~>T6ss{?UmPRQOT%grnotDmKkkdcg+{Eeh5^vToDLZt-P3RZ`vAE zubkmn;bJ{AI{YVeh|Qe5PhUS-^g)Rqm5dLOW;ne8?@;Y-pJKGQ_Go&{)QWb!MH`$&DAM zBuT%uzvjn%?Dok<2F^zp%BNqe$dvs4`s>%aRhgf?>(jQKFnv(aEUuO;L5O4uT!wED8&%eg^B?Zae1Q8~ZWJOE=s` zAz;qa<;7{YUp~p?+jRSJd(XFH%a&br_h0+9{%JzfBtV5B)gA zznn1h`1bkroG;J$W1f7A%2Jw;!oB`kR-<%txb#+m;=T&8{It&U2R0S!`+u~DEWOBk zJU!UYEhjWBY4&Y~zYGjgw)YsDUansrF zx4c*X*KU2MmV5Kg41zv zCjZZc+`CE=}S?9GS`Q~a9eOL@7uy8bF=-+&~ENwa8edUJ0#chfg9u-|UZXM&3 zwO7fJZ@1{y;4hDNnepHKlP0cT^`y1mXT`(g{|_&@V&Iwi=dTbSd;&E;zwmPM>16k9 zkDqy(Cb+H*HGLT9>TtU}CUebdh^ zdkSBF_(+hEL&rvrKR2`Q_@&I5LXwTSYs-qP<{B#AQn;X9R(8oc`mWN_cMGKsy;7Lg zV!cF9>6Ltda$}Xp45`lR-y`ks$ou>$o8JHK{|4VBKQ88Nzi6x@@pe|`FL>qM=k|Qx9Buw(mh0`5e9MmIl>d6Ug0pPb%eDLVGdx)m zy|Q$d^GZM2}}>;dsxu+_N}x{ zKl)Q;b!N8J=FFe*^l=(r#OZB1JsbX5C9LC5Kh`CG?mTfIQ@7;LR{l4uP z$IF$MC#RIpy;m|ZLz-Le;wt@L1u&AgH#ODo zx~cm~O2yv1_h`A__W9=ahCvrSg^XSJy+qw>A= z@*ms}KYqT>{4a{>laz1!zu*6+@x4mB%Db845^w2*1}~3*ug#|!Us}DBoGNI!X@1MY zqmw6g?5|iIJWtam`eM;4DT>N_P9_MAQ)1)lVE$^*cbz=L);OyVqX4YH^v7T#o_?$KC zB#$-ub;q4ozpDB3+H;4Ej;n8LC+{zjUtuPlX20}L{8<%cruyQ!KIh&OYnSRAy4Ut* z@yVT*OI$5<{yqJiW*n}ov&ebX<`pW}{bozwDY^f9?U!rI^!8e_Ej`~ zlDk-=iQu-*uOAkknlyX%>_0BI8x|uqL0(8VW(Y0ZSKOn2U+dc9c@Y`OvabYw*3K}F~7SrRGKo(%Rh!o zu6I_svPa#*IpM~7$HOtRm7RXPT-3~!89jM+-Jd^koE-bDzguPc^v&H@n8CRBYo@yb zqwsl|wLik*)cPmHJI|i1b*yUfHs0D#+H$Q1SB>FGCQ;x(`nP*}S3AwtG&vkyyuCVQ zskxGh^viWO-Y(XjSA5xgeo=6s*~O*oN6#MrfA4DFi_$H>#V$=ZWm{9O8=uB>L{7}T zKzILwxj8u@!k^slY<~Xyva`G?droG%*`}X=XZL=om)~rENkXCSyp7Yx-LV(1`pTCs z;thz3`D&7uq}RLh+|sb{#b;uaZdIFX40Lx`+t*VioWUr<=&Il+5V0*sa_`S)vkzX% zxwmJg-kuLmQTu8tFTCG!*>7aDSuSc8|x-IoX*p%Mp=tP)4`{#2jLZDK7b+3$d z;gyw>G=oC=6*gZ~x$PnSN>lR5#F_+G7H%oWhC+TZcOS`uzUpK4(@Qsd7t3x*GvC&= z{m{%GXXnaQb*^pa5t%=kd)Za%x@Ekpbmq_9hc8-cYD({NPu9(uzh*FJ`kkEkhvA9qs&ljacmzLuV%|6> zJny{Sdo`_DH4L@SE0^nJ6@KiVr2D$y_VEn{v#zdCY+!YNX%m+E;n(ygAN$)Ywq6vu zR)42{&0Mjb?UR`quV^xHwmF1{i|fYjn)2;-zP*;V_T)?V>;M1VQv17X*T-YhPpiJZ zx>|B2(ETDazl}rsyvk)i?v`Bk-BS42?bnmZ{!gSPz2>+7GvU?i*H5MD^>#d9a{5(k z8NqMHz5my)`Df}jtgLnUntoqAe%8|krC;p~*)8V$GVWhoeg4dX)AI@gcWX~!I;Qn! z(%psruD2zHV&|NjX16DjF*bAZ>XY)h-NnU6YNb;x*O=wM|7H-$-@?l;q_h8{iMw{4 z!P&Tbk!JEwmqx2dKkaTi<#VRvTidbJFSouO&j>yI+h+BHNUm4^Bol5-|Gzso@7C3Q zv32jS?(xX{W@5GAjfK_1U)m-c6Wl@?COmpEzeBNs@As$Wab{DWJU(CPVtnq;!X$9eKPnjfFC^CSubcFB^Z7Y17hI2%kW&8k$s)B~W47!) zUumhDr10N`aWb5o8hNn>*B#~G^~~E6Y}DUuU-tjP25G;Eyq#-Ll|~AsG|v0aabv;d zYiuTAw`O>?Et|4C`mWEpo0EJRo9)&6i2?;m`dacfn2jgPOd z-K08&y2?E9?Z%VqmWaIIE)qOwAE3bHesLD-`gQ`Y|WYsGJ+<$EZBP6efz%W56?0w&0Mu?>KcxO z>AUyqeZ9Q<_FYJ$d+sb2rO4an*S;Q7PY`&pXOE4Rp5C-K@s&?SC(1Qz9KL$>>P1iW zxh~h^>uu*&zuWof`(ye4Gv?L*t6a5m<-}}{nfbflnlW*jxk#(Xq#fv4y>8vI>G5?l zpPiqdzvsoe-S4>OJzlb9%anf~+U>u5m~B^E)&F{7oW+d;@vDS#eQxC^2G?w`DZ1vf zdjtE7kH7yW@z?9hN=>@K^L$sbOvt%zx%~eJ4reLF?P@EZYEwFE?zInlIrqFM%HBNx z-9L*hABr+8Rs1r(tjt(lp;Y_E$}*j2z3qcyo7GC5&x>;ck~cf5svP;Nlh={`E&bn~ z6nF8tX3HDu{{M>8Tz%xB+=r>nT`!}LPr1Oo+WxJ5okDDhgf_qSdC8x$G0S2+^`q9N zCI_9q>2^=?=Jjp%j{omfn6KurNPX1UW576(M>Tq%Q2znh;&Thy4@FKqW7-#)Sy^)N zO!kWlAJyGNWzKos6%fpLc=gEj{qGVlduM2tb{)7NATj^>;&_?&3l_~WsomDj78}&Y zb~86Ci)&Ttfuw`A-|xD&%T;N}mRxWQdT#x0NAtXjN1RoEzg~YT_3(I)r1Fz*t5&ZQ zvKPo%QDS0le%ds|NWt*m{Qp1Yt)5N^KEhvhc|r#0R;(h~{`!Fbw$9;RAR2xdTBTsaeLA%f%y+BNRS~d6g(~EbXU%%{sT?+f& z%`0Q`&aXbPX!p6*%c{l9FWnb2ZLvs||JYn(T)=~RJ1;M8$AN9ak#qVN|E!d7H)J_qXHot+xT(5Gj_3IN;}TEq zeT<%)`(^r6`{niP_nWWcdUL7kSNa8xgo(^5H@!Bo^Uh>@)9R4BJNB1A+=;+Rj9*R# z>L<4{ED2ZWI_7Bq^L_pQ-8-lB>iF6JEvfqPu)Vo%f>H7Vb3W$B+xLCd1+Du@-lL(d z-TmqF=jQ9KuC68T_kRDTaK`+8jj^NXksWW|mFBKn?`hO5k@?PS zw}gSgfx*+oF+@eIf6Y(sy}MtBdKcv`xl)wUX_owu&tu!#TwletZA)(d`7!ITglBoc zDO-m>&$s`Z{BQG1!>Kv$s|4F8^2u>czwP`#uG`&du986!%e-3&%h%=2D10Q|v}=Z9 zKyFU``D z2x*T7g{Fl01&27*IARNriYCovesa@7$Am}yWV?KwMQq*Ar?z4zlL`}~qK>4xEaNDw zW$Bwy<9EVHy7)+G;no7q76;cT?<;0Gh6W4%KdNNEsHdZ|Wb>(Qll$+R#yxAOxL$o@ z-!}Jky7n>EN6+tKFRqv4>;8Uu+ONE*jf>~r+I8n}*M5a+^|NxvQ}!-!xR=i9xW#eX zv^Q}s=X0t;I*f{w1ut$EQO6_o6^~=L zEUeqgcdF-MhDGM@xp`S-Zmzao}EIjpK5GBK~Gs8Y!A_txy|mzMd? zZgOom?ODBk->h4=oR-xu(EIrJeZ9Q|+oX&a8p#_M6ke!}+nme2xtEE?f2`V zmt0l50A38XvnZ8+Dxb&Ze(QG@!Y*G{D6$@ZusW}RgF)NJXqk9f@U;EK1rtrCC~e?p zF{+g{v0U!!QDDNU-qqNtw9WdsRoCCsmtWraqkHJ?ku<`m!sk?9N)_wkzRPj_KTppe4G?E$G1&bl#lLHfL?`3CS*@)M zzeGx^g8$Auc4+U{*Vn&a$p80w{(ql-+iwvYR)Y>sH_5$aa%*RC`UQUJd4Jxd+jGY` zANo^%t&?y2)tXl;mva@GHJ-D0%v1IKZh7^Jv^5bMo9aGx$InnGc{0)c()zlutL0Rr z!ps#i!^C4*nuHGCk(u@ShQ!@7CSRvGCbXrCJ}sQz6vf_A7UHSCoZr+)Aos?+|23i7 zXV0w>^xW#PVWx%CD&0xB9vfU2@!r14xYA7J)|>ZjTcp45t?}<$_?!9Y`K5Zl)}QhX z2{d!NSFG^&Vpw1g$Hikgk8b+QFW$9o?-M45h$|;lZug!#EvThEl^zQI~n*qA6gXQ(-JH_WOo|$Q!dAN<&asTf3`(`irpL6HN#$?Vc zv4`*GRlj>VkTPSC!Vz-afM(djkL(T3@X-{u&E(qV8d+n@mT^90)&zij* z6b2u^@#4!%zI(ocs~>j8-F)&`pQHKo^(7NK+*c@B&W$yyeE0q-e{gz0FhgVQ>$Te# z{X4{STyRoaO_9m3LmOHc7c#4yERe7NV;Eb0x0K~`@#eI%LiR5-oV%E-&lV`04>)Y` zs6+Y4J)=JgFB_loJTBn3x-@m_)MMV4_^#)&oqXrJ#oy-Rk%S6cPd*l&i*0?{Z;JJE zqP+AgH?7e26?ao}=RUqoEb<+j)s@e3-af-I{4Ks|p-D3;+FEnvo;ygla(vopspL54w6BcQ9O3N;bgPfgpK;McJTz~irl@L2 zXsGJ@L>u?Ny}Fb9TK>7TWa!`gYG+!q`LELzKEH;IeK%Nz_@;`fH$<%VI(q(r@Q3Y( zJXk!_H9l_NlK)7y+3dobE=wbZD9Km%GnBwXTh_B~9`U`!9)9b^zom9og_Z}`T+-Y|2$Z`K-wjYP;?x?V*e!jJ#{`0w%f~%}%zb;0!EI0)N%X<1)oQ0kTZoKONR6GPYGe?ln(Ilt1xz<8e9dx*M(n%x#LdDjJgoIL$NJ zZBuy+jx|;|OZBr_9{c;!zusth(UAoWN829i%}Hth{krm#azh>0=L`kSt83Smyt#Hg zW8?FJth1~9{+-I^Rc>>ryIy9s;I;3n6*AsGl-kM#_N9ESJMX4*ctgts_Iv#byC!5k z{>%NHHCosA-tIZ`*3I|Yy3VFMeDnT=R~W@K_e5EgCvb?I)75PG*AR5b{_wk{k3Vm} z==nH$=e(s!vYeN4IE;H%`DlcdoOAQpVz^;X{O*V~tr0xZN;M%9vNN)@QrVS$Flse) zEmnQUZ@NY_P35Egnn;(+r_$iNV zd!;zxUFPDhXWuis>NkIpU(x;WH3tiW!hS_NvDlmo{fY(#PHe?<=f!yo7hSv1-pl$W z&ztq_*C)-IT-j{*H$1Skw+%~fSbRFx#wUOF{JMq0Uf(YKc$A}d^YPQw z3iBl2NJwkHwDq!7T9mtF=IhzlSg*SA+&yHgc-b+*$wXQ*I^1RF%bz7T|7d4A-gGok z+ugElTTYsAWBu}@rtga+{Bq?#PUQVG<y*bAL~UFNE}ezOzR&E9)n_Rc9W&Ntq(>fG`D2c%US`ey9+X@m@GT3lo+w0pnjbH>+K zS2q>a-rG~Tsn_hpvG;Y~ttaX^oBupxd|qTr-ky(cO<#Fi+L;U6x7^Cv&AT7GqgAKy z%uHkVZLjAUrE=wUzuvpnj7@z-Ym4=s16MVfHaajQOMls-*m=a5sc6c4$t}D4 z&nWBUngxA5Mv8ak?#128_kSKgD_~N@6#vf>u`CNp5~W`L0(hE<`#n zy(gu0qWx9iQUU)L9*3{USp=`~*yebALiU8*V!lMyHFCOV@|HArhh1K_%_r(=pwsMK z#{>47x9^;1V5fNJ(6*iLqi6A(o!YoNBPq@J?5WHJmOphvX3RcWDy@8;^WOQ2xPb5p zIZwn_|LU8O@pyBF?{U3}fzI!xSotTtX?P-aC|6_4XWz6VYigAfJGhKKz3iL(ZmIvb zbqnV2`K}!2@^YE5`aBgaTu#h#RwlXj|INyOn=O z?AnDE3rp5)4!3;ElXmmrD&_oq?(_XFrB7N;^8NbkUi-S@yPxsI^%s{rD6RS487iDp zkmO+;Vwvz+@{+_0+sCTErmnAX{qo*tZ~m^AYMzH<-fzEOwdmZYlI#4pIyIxIn%$U%U8d0?!284e`D5)Stql8Y!+Exdb!Az zjq!`%9|sNI&a-DuK5^~1rsS|9=fT`gedV?aH?`*rmo{x`e3D$4vO)f{B}ZX!u(0Td zU$bZCWE?)(ZMNIS?AGPEj%Vljy!v*tGUoy3vb)Evm+o4rvUj%Eraelk?2+eBYfs*l zSko{|bjgi?P(xv^1YvD+rA58%;x4+2;P(hKPK_Nmup$s`*XpCH+kZnW^o1? zTOyhySG#p@uL=75H1=rK6=Ck&lG}>}e%97@_n#tSVEXXpn*V>lcR%^{?RNh3vjGd_ zdvkYvzh7UkX?Esb>+XLcUWX39*r_z3gG1s|3J2GhZH}`RXtkbgQ7Aa{gXgN9aiZ_n z@=~R3p9@a)l&jXgP@XZt(Utj$a4pNF^7Wo$S5o8{1gB?{54l z`s<6uKdsEK&o5MTTP<4}IVtN1>q53bciBQ0+ra-9AGIhe-Kxm=C{R5k<#T33z~X3@ z4pzP&kv$eq;(zn&=)aTYi&7?U$-mXc;Wo7;uQ8&;}mHY>DS3T7u#kjR4jjao}<6w z;bcyM>_0xXya(On7C!{lahpBV#UB6O&CI`Xshv=qB**UD2{xNIS8SeNy195|OX;TT z8>b3H{F+>CwBpywqc`2w=T_hS^kjeSQU2ELA>L<7t)?F{aWY`E4P;r=;UH%3`tC&G zs|3%FekY!7Iqdt_wETB#!u_vDdFWfFpacf30PZL|K_q7N>Y+_P^kxnB^>E*0@=( z`k3q6bpc#@YF~e-?slGkYx-gJfES7J)o-^Ry!!fg*W&M@jVUe+$#WHh6f`9MpV60m zeLH;ag6`Gp)S23tA7<3qIv=?|Ra4!={KTf0=WAbDwbS>MzWe;~ac9$=Q=qeV z>)+g>s@U3N)0h14H;A;{9yoKaj>kIRp8;;A7hMk=#UFVT<&8x^WUB??-A&ClZgn|K4qz!&H#*!}(Y3T|>Tk4Yyu=+!Ft8%YsS!KVDBR5oC}&mG*n8U%Xq1ao6K3<(qEW z&G>dW$#ur_+n3L6D{cGu?)Wk@`>Lb9mYLU_H_yJjvv2*xJVpOI6Z6VH73XBW)R}Fp zakAs_vg?bWYtA&7{wf(HE#kd9bI~2AS@X@N-z#@q#mg!%A#+2DrLcVVw}aOkXR15x z-J5Lq_e1ae3u}-6pJWvge*5(NF6~dJUfvVsztlYckFw7x*|gc#I{W6ZtqVN-bDQNR z0hZO-y+SsBuDXgd3cR&x@18X|es<&`Z7F(27}N%hpn^vi~iV8OZ`ItJpfqZr(83W$>Oy>yy}~+#LCxGYzU| zMDCuMSv|w@aGcqnvzC3=EdLyOSNP-LG5PnB?#t6<{F9IOP5t}E@t5Pjt4G-yAB6Wp z24`mbm`-2f4jY`wlDcPTRp|Fi?K+nh(_uFTE#*~aMmax(jFLbB+_`OHhSMaQygL@N zBJ3LV&i*o*`qRrLig)*cGYg*{U*7n6g39eX=Enmf%(zbo9{O8-eEqj6mw)!1zx-oU zk(7Pio(Z*A&-cAGPVO*MvtSo93);9_Z5!)_JCpbtou2#6nj5N}l~6IS^wFFW-!-B^ zpfwc=sR9y)7ukX)SR~#Ke{(7%G}JX%TkZ1yl={|X)oLu9W*nQRt6o*Ov`8wc?@i)~ zt)=~kKXeMbIbbXB=H5r~9qAWbU&b$3RXBB0dSv9x7hSIx$lTtS-C_3Si?_FTX5z1) zQ{POEZ4;OkeMw=$)4T=Fw(mXTwRCh;)@%>REW2c~KFkHYQf`WKf=jm|Zi{w(+)di2IZbb#Lx$Tam^W3JLcFSJ?^4);u=6>XP-jLm=0L z>4cb=;;gk3&X!i?TAj_B=aavC`emWpRj&jN@m^MHe0x(ca>tgZ=@WCmy*RkOax-7O zp6si+j_pg|`XBXKwQzE5WTd9b?j1WKRO1g6PLa74^=4)9Zim+6FH0uX9)G=M-|AUq zPvo86ZTSs5tf(gy@Q*d{meCC}#efsIn zu*|YEvptPNKuOO*u)~09&vYJ>16t|gLH_3nJP)+x?Qw>h0Ef6_5UCPg+&Gl0KQ&YVIAGi*uXZ#| ziBC31XGz;tr4@4%7qvZ)5qCVcz|n*oS`w{L77$(38`SL37_u`)@70?(OTw<*Wqru> zk?rQY7oZi+d5UQq;(j|6WLtC7cn)3o%et`bi4?Oz--J0y3T_grHv-=)JD*};x%K!X z$40l=$wjSY>_6SA7u!GBm-+canS79&l*K{;4)vQA`xd6PrAwSQ-`9}%sOP!#c{8>1 z9E(BwWZK#q+Rr|(+jQJN``fRK;>q)0Rc1Fo%d@xKYi79gcJ|537lf`~;B0P~YcH~V z!xiEXv62a14 zH*8EkpLt^vrh86B>-_eMTMW%^?LTww>(&-4L+uB4MqN@D&3;O~ROJxs5aTfG*sgeM zLQZpqqKt>QO02?#hs=g-@qZRX#i>mwi zE`Ge+LPNdwi)ZIDfs!jeZY14nb`0Z}7LPLF@3G)HgY1b=W?vg1pG9E_7zEVn0m?AKRZU9#7K7N$P)44_0#(Hw@+=Blc6{{Z zU%N7VlC1sONfoLyEsr_d$G+f@nkLaJ)$sjoM$zT-_ew6F6U&wtGsx_YKWrxXChV*3 z0`C3n^8_DFuzl>T`}dRcr59Uf&U8E&zvh)x;O9DJ%@U3|6PGM1d@RbCd%q%AQg=FN zw~iFk52-(w+{`EE#G1^>$Q93j8hd?)n|@xwrOeGvS~VMWly@9lRwW+$T`78Hu~UR> z0CQrT?sLbBrqjJtt~Xx#T6{0QhM)W7{~b5H1GZY#H*R%zUh_)Q+{HRf^jeJwqpQM3 z!2{ivf7)+7s9F;5+hcNMN%LY$^Hu(xy>q77NeVc`m+!js<^9%^0n%%oZ4^FSabuS? z-`M@Bs5ohtgOKpCySjP59(K>okyUSLng7>y&e=Gzo5uq6K5W|l?jeJIaqn+N^MF6U zr|+m}RJj;gT$H4CGi*_Alv_WjO®K=qFJLN~?3om(}x>?t^V)UJ>D=zD!x=Uu#v z{PypZt@p0waN%LJ5kFnFbLGvcf*Vat7hZ8&aCEAhcX`>xiNVH6+tm2pM=qazH7xG! zf@6N(0Yd-YnW{*wQnV5BV177PJJDg?VcB^0_pE=|yXv2_`x%y7?AJ0&vCiu-b7iyqNvG4mu z7;WTNf2**NRK2BqH0W`=fWv&N9k*1&?wzUDEiwO;{^I!m?MvofyZZU}y$@;KY~S|q z?QbtmywJE$^7h1IWa#fcg$COtaMW1=#RRby~lqpu3Mol z!?FEl%IPVQ{aptN1Poku-(1*3=keWx zYr|E3{@K^y!1_e$c}>e9j=1GTdA*_)=YKObH+(&}osH+n_G3a}#)(4qH-7y8bL6Pm zwz+F&=e@mX^y${uxlVTkavPTDRSI7C=6i3Otk2mC|88hr5qNX2u*&A&;R?|s#WhzX zgFHMsH~bXq*ly*TV8QXglF2i(_;c1emGaNWZk{|VeEpW$BIngPUp_5A_U3lFcHDvJ zg8LOUR<-Y4x8K?G_DV@r!OrPoQ<^sj+8%gbYk7qEX7ShB*?#k~mh<_2+WYqJ^Sj}@ zI$vCW===I(A6sN^o3s48H?PuGTMF!Gh$z-Oaca-^pD}ucho>BD;nWX!u&wj`<9$p* z?^rLo6*YIn-*^1-ZMkfTQGP~qdGVGxQV)8>HqBb6w19(S<1w|6xpU__nJMNT;1<+B z*jUHhD*0lbXYm39=DXctHCLorR_@+C+sex7sZ8a{V3xL~hL`Sn^`HE&@vYl(@z< zs}+abUcYYtwnpb(qTu7B)@O}&?F-yj5GQpu)#?89*&!*5^W~iOsb%p+lY?c^ z-?&571xL^MWu<5<-(#>edmk$&$`Q}%IL&cZVPEw#nZNOe*dDbWH_tV>@*_X@UEiRv5qQqNhiDMng0E;T(k$IPm*tM^^}t^ON3`=uWJym#->@=G#{ zWNTzr9nQOZTZ&7C@RY0K{WW!I$H7$-h6)I1opKBe3{tLDmur_0Tz&ySkB@Yk)MU;gamvsyAK z)O$wo+r00W?ZdySJkbJGn^=7>9T?LYSFScCCe9lILg>lWt6 zee(K$|Ff@t&KMD%`}hB2(c{XUo1IS-d`oB%)mYpk*nHsAJc84(oXYCnD$1%W6oQiHmv+MYqk_Pj=~EofIGP*mIAkKG$CP3$M4n zcvsl`Ztk71IL*k&$h-0xho>K4k>RkIutUZ{>%jV_-A~&Xx8-ilv;GS@)8XX#^V8Mm zRXD{}JY`3o1>jeEoWtzT~`|Z8Eo4Rn#cMA-6p@D*ujk^3x>lm*FD% zla@w5Ke6hP>ZX-8@m{OApEjBE2e0e;^XzZF*ZwaDjZ~6bdfr(_s(rg3?)F9U{fs+@ zo;wJ~*dnbhAYLGg=aSARTZvrhYR*P*{}?ziv0 zVgLWU;MxVR_61I!{AJheB_VSSs=ViK`8R2@oy+f^ERxsrXZgI?zkP{H`N`Ec)!K|5 zm_)L_d}36u$_Uo-I-+pG;&#|&r(dGlVJDXP&VKUoxO{nc(^T#7UH;2neDj@U;<;(_ z=IAXE7VmaEezNs?T((9o_vd_tj5n)RttpCUGBVzK`=(XWqU<@zAEtE3ywG&8j=KHj z;@NKY6;>xJ*HZ0Zwg*Ef5XYJIpt?85m*F~3`0q_5YRyz{3h-;CY1 z77d<0EPKuOS6Kddxji@hjccXViIQ76aUmN$@5TOZs@d|+RCDDmsjXI?zvhXAitWC? zYSL1p>c{ml&OE1N^w{4P?UC8^nsd_K)$vYEc~XVfc7;DVd$r%YFf{SXFZOx1?aVqW zeuQ2ReYNge(#?QnhTNO9-*bOl^OF^U=_eWw_xD%7RBKjKu(t8od2w~2^{+P5 z_=etfcKUMyUvQg=dF-E4=Qn?+;x_mGb)TP|ot=85LohCYr8YBjWylM@?+2aP+C8tY zjaF}F=g(tVHk09pdGRwJ@$>v?+otACUiC^#(jdXXf;X)__x3hZufxh;TsbYHgX@>v zUecW+{r4?H&ifPR4=jIPl&~bvCuV2p*Ba#~f!t`=;@&6jBo|3;zF8;hRUG3l7Oe>j;{PmiXr%$uhdH;O>iUm$zi)0QbNbdQn zbta|B{=Z?b`gE0>5!?LYPVA2VxARlJQSz1VPi`14U!@W-eH&-ewouX2i@VoNOg&mG zwB(4R;qFPyI!FF(d{Z_v_(R_u!>{F+)NR!t&3$p|lIfGP>JAQNKWwitp5A+$H*k*j ztoR?P-;egPEIB<*cQWW4&`qz`?Y1(wyFEXC=lgxX=g7sTrnZ7k1D(_>ZGK8`_Zube zX*v_N_4YgQ*}e(be)HLj#r@No`RyVYZh35AT*q(wWr9xZu8_Q4FVz@x7JJwIe!G45 zr`=XBeiXl}e17Vvc)ZSJb^o%`!@X8K?)IXhq9=2=-&IRK)-!SbpC>YrTeGIlseCrm z?{3P835t0K`ogYf)SbOLe^Y@Mw{TE;jNHTQrUQ}%3hOKzPI;uVoLF{X#e^eS$G-Sa zYIavW{VP|?a_)y=i{af?-|C5KU<|o5ccC3?IILn%^qUW%ug3p&#tN4yrY`rV$peXhB@zYYS zThr~8=C7-FuiJ85bkgnoOMJbHLkd&9w7Z|Y+bQmS@Y~hG&~*z8KAvyfGim3#_b~=< z`R>NvwO{wkO2Im4(QSzUv#Q0>M{?|Ko1gD|K2Q7mz3Tjkc&WcHm(Q#6^0R(x^7Hci ze=0k7?mVYf=NYj0lJlyHz~=hJD?DZjqF)O&yb-?#0bzFhV{eKUQ&E@|Y_*!l0*>!%+M^Oye={P*#=y!8aW8_(Tu9Ok$8nc#le zplPLl&e!&NtqiG*Iqd2A)@JPWPBI^+ZTYfsUaHxvW5)ykmaX|7!FqDzEFsa3Mcdz+ zo4&i|BzUy!?!61kt!_zl$X<)S`Xpq6fMyKmbN@h@Q(^7J)rzmZvQjGUheYnk+x_{& z<^C7bwO*UvnQL{4#nz=x>P~Z2_zJ0XeSKT!Z|_$A{a3PfGgwAtceI;zarJ+RjfcyTm+gdvxPUe2n9;`RD*-b`t`~2$@J#069 zc*T&iJ-+^Jp=R!?h{yZd=InkhKPTiW+p2XbEXR!%HJ{n$-!c)uW-_()%v@`Cw|+U@ z$H#iPpEsAx&$Ern-bCEY-~ac}Lw4Dc z2{qqtroWS{`aN;}&W+OZUdKdTk1db2=w;@!aG0H{j)Q^$v9qm*ne_W^C|Wp^U|(qU(gG4 zsQL2a$p==mn_UxqnC?c^y|A;`zLRg)*;RU9ClOb#||F^y(tqxn^V~^8UpMJW+^X8{tk0xE2x2@hSe(zuXZP|>&OP)`+Q#!9- zZE<98z|m-(PS35o%gil%eOh zWrJz7fZ3l%8CcvKXLYKY2Auq2D{4hJ#{xWB>GKd+3(>n=brXm^|gGw8+-H> znR)GvJx(yb_u=_x6QKF=_~&-XT$9_UZrn0>w0NGg<%;|QrOxAy*~~e=xf*%rEA?hC z*7!H^W5u`9#)5*BUDwogn9l#nkUo7sr(1aS+3Qk%haPJ4WG+`NoK|+8`PPSzeUYZ0 zRyPMf{cuO5yyZTF?VtmsabT~^YGUS`}aq*tK%16H0DdrS)zWnmaX)STn5mO)?!G>;U+?GQFK@5LpFjL3>yD4pmBp)e z1>KAK-4g!sLH>b83Cj=fOZ`uMS?KN^woER6iEvf8#4poVN;wTB@3XErRpjSiSukO_ zw2LIqBB}m)kH1ZbGhQ~sG5_b>@_U8SVwN+T(+@0{Jn-|%%H5_+abj&z9I0xyImN>7 z>dor?GU~;$0Usk zAGwIznwy8W@yoxvvt;_r7+uk=_nyC-yY%|qLpN7c+?;)NSGnt#mQ5-r^`F*Gb94W) zb<4S!wfkK1=U!2MyLguPG?OivS2*sTFpkwaTszh7nqXUro@J*f-PDz%|gz5y?@Ys_*U+*v<)Y3?{blgD_NfBb3r-g%WR*m|FW)}HBOg% zl)kN7_>=m)-;#50>MmHnV&&A`&+jU{?z(O=YfA6J*u=x#M`znzP5!;D#Pyrwyt@;a zZ)a^kwW!IzjH$?a+arb?zn>qPLw{90j%S{*)TH;Y6hod#{rU5Y?^nOyoBHq1&qEih z#lAPP%T3r>{CwA+e*1qFJOBUteK@hTwe{1r==_!Uz9gPIZ;`m=#=lQL?*6lQrY3V$ zxMAzhvonp||9zUi--1DO=DedT`5AvLJ1pJ&m(A*N^5s9yo8Gb)PuXZKUEy4NlmCD? zPsp?%54S7d@2T&-y>n-UZ|lL!%ML}oI&Yoybm^Zzv!WJ!^4t6UUiHIeD*WG+Yd)P+ zPuy?)ZpYzV99*}=Cq(rZMcuI3Yrj{yU1Z0mk}uZ}l}*kFU#)d~Z*l%L*@L0$E^kPB zU_aHV#rJXH4~uzucNgVFhepIadTH|bsCpRp%hTDXp8jLqe)1{%pM$1J?qwq3JLmpM zREpSE`pz$j^&lgA)N#gNSKc}Ez1iUWb+$I|frS?h>(09JweBpBzx!?8_68LhRa4We zb2&a;p8xO4++5xVZc~{Tc<*{Xzkc8CD?gg$|2SBDeB7IrynM{S$2s%L z<)`aSF1Y?XB)RJFsqDgayIyJSe6gte$<*+;!pGvvd+yYJzx%L4=6m_ir_*zP`S^R#H@>Z_N^YcjF5@p4yW`L;pQiTYQw~4-?;(In2zH ze`Za4b!^wP;H`RUH+t&$FUVOf=$A1yh>yEt;dR_9VSb`h{+nVg)dUCOitMm=Ow#u% zd-tzf;l;Y9%e?aY-0;^g%HM`+#Y?|kDS2zhtEE@|)rZE<(K?nUA#wjiwdcF&FYN(N zihf%xZ*IF7Z}rw~rroCpAJXrdrOPubzHxCXy#FWu^sBB`!zCgU*km%!G(Oav{PfMr z>(&Q21>X8FJ2+&?SjHF>{Q9djzdY>Ni60ZsUnmxlXMT8~k-6N1@5BDo(_-a)W?L@4 zC|CGY>2j>UMosJ2t%jHbCJ)mXnpbS^k+C%5nYZX-^q-8fls_lZ?RkR&?RI{9fB(L3 z*uM=fWtZ!DVwm0jw%t5=yr@2)`_4xn|LD)op)HGb_^sb;n0qevi{-E1x4F$v{@0#y zK}n8fA`{=17Utt=b!zVK%xugJ-(b>&s7G-V5IPb}p~|>+U|6L-Q6-J)gm0y#1*8%JsIt zUs|f)?rD{_JurFx;_Kz-59#i^ZzlEUOzy|or{z6&)h92xxVZUrn{RrF#_JWoswF#q zM;&KrHR)&-EA>cWe(!s#`}QrRO@2}$lRLw!R_p&3e8Z&rXI6IjAqE!V;#De(7sPDy zPV?Er*}=?rpz+JDTqCp9cX|zL3^Uj9A3J4uiT%W9*9q$9BT{&W_8 zUi#h5%q`^31Lx(h1pzt5Zf>t8(#Ke_Df&rQb{G=F_|etLAsQ+c1APS^6HwVqy8-SmBe@*VMp zjF!E@Gd;feB<}rj;>^m)`o-bXZySI6{qwwM+{K#x;oFPz*X#2#zn$xQ;%#K*r$;j^ z_iquj-~X}g2%G$OJNIKBZP~x%K9-kucza}O-y9z4s7-%k)%K>%xwBASBf4x`-2!P- z(d9c0R(lo~&fpNk1l^N-cVS>SIvI9+Q*;;pWT_Ni`0e`5o@p98f+q#IxG$CC>)AeQU!4BtpB`?!r`{TM^3P?O zx57+r>#J{R+Z)+m1@@JnX53v98#YZRtFZQYZk0b<8qefCk1CDq(%-AST*hzfb}88C z>DsIH#f3&en^R-XnOr@&e~qATVCwh%*XG%|-hY+TY4($mNy#zrMQ;8TzHbSIiy8B} zg)dYmr!d<~#rS^gXXVdzobAG##H)E^ez(=eh~$M0w_iy_Ub+w>@bd19ojWbgwni$Q zJ({VvM)_s0{D=1MZNAA9{|bM8ddt4q zUp=mL@|9dVZgtC`tWZ$=RYOnunQU=Y-Kp$hJrhhr{~oHCAiA}kld<*HrRkT?YKmO= zzNo5hp1}2f^}k zs!I>}&++k?TF9gJ3elvnxnDeAYd?QiJMoQah5Sj6)00$Zi=T{t_RlkH`noK^T4%W@ zvsAdv^(V}<3NZ92WnOIYyMF%!KmU)pEx%YLd z^Gz@Hm3pgmG9){XNzJiz`m|j)IO?su?U^g;m9vlRV#+WUsCuveeuwm_DZl5vW!pJH z{_vvsR~H1XXY178Y1sGm&*sbKR=fF+d|GvLhR4m9?z+Z#Rk@K@R%{fSey=)x(w#W@ zhHbf59^F0ofZf62=@XB+)$yAC(?4BFNKUzuCs1K_x{)We(ALB5>OPZ0E00IUXHQ!p z%k=W}1LN(QK}&C$&5NC~$tx~yvhw@sZ&rQw*i_!P?J0ZRm3o=0T^r41Zil$%_!T~! z)o$G$E$_B;`Uy?tYp0$(ti3Z&>WW}-QPH7QMn~Rz2peBWGu^aJ$6n_m-}N}}w}uX~ zK8F<7S)4wr|8>&jdCK2qzBbfss+$(`>)m0%Ka0p`}g?k)%~`K4Bf=kSU&kn zpP}Nj+b#3<*UtTY=B{Lwx!(T{d$W}qcltP9&g<_suh(&%I(vyvsKcpM-G`?7`1(y2 zKK9RL>D9|yUIwQm|J1UL|LfQLc;yw5aM4q*$jXT0IT^mAt4>ld+J zPft93f4X|=Z;!%X9`?*kD`HMe(v6T?evzWpP$zI?mjhrYO_oS-$D1ILieiQDlDx^N_NT)c(3?n`eG*bC7~y- z7hO9M-X6ZoYSTj531{1i#6-7MFC5L)X%A1y;{oge9gx#OEf&JYqxKH`ij5)^@qyfFM669&Ue)Q zUy}0gpL%kIv32iihZp_zkG}jacmKZT>w1^9*Z7zH3A;8$Z{qELzk{#qmxcd*Tl{TD z;JY7hn8fQ#PYR`rPg&pR)&KnX>-ec#p2YQrxVNSBiG101{?^;5*cH6foUZcLm%p|D z!rd`FZgOYoWQJeU{#s>EFt}#;^0VK}%vJNH8`R>0^A0gpxowd34Yqk~qI_)YxxDYP z-ZzZopRqHTO?u;E2pUwee)??VakKSSg z%bt+%YYAI`1=j>ii_)*|n^vT(Saafz`>J5u2bWY?d5wCj`IcOra$?1$n?{DEYpoMo zx+erY+_3NJOi_aZ-Wz7-PfwiEnsD*&r_`s{u6`=K=nxrW=FydP@%}YKW1c@6nH;&z zng<)RmTLZxnZMEX+soh=d9`O=UJrNtroq19X33djWs~oeUyHb9kiD&Ox6hQTrgs9C zW(h6yGAlT^qp{aex^2al@U_CnYMu2eUNhOQZAs|&JW_4?-t_0k?RBeyimvIaN#2`y zPxaBIyanqod@}f8aH4!_CYM7=Xr-0){f9Rnf7y1f@zb|g%MV5B>--7$CTn!~W%zT^ zt$(AcLforXpZn%rSwB%%R5~?hzwMLu-LcC3^>uUCXuSJ(T(d0nev-hGBiwe|UA3F# zWquu2zWcA@;kku;`|DMDj~Z{8;xB(&62CF2 z)%wt7;#j`#yQy|H6J22j}j+S95ahvNMHd+a+DU{Mwwc z&}PmiXL*B_W`CX;*W}x_SAThyo*H|W;pQx>kZntOr)ja?HC-7QG<(W2=_6mdivC(j z%#b_4cr?QE>()s@>kkRqdMudU^m>ifUfYdT%{;!hRQ?wKdf6$zSmr{S^P-np#&)IK z7V0m4l`T|Svb~VGqqu0>sU4SsLKkaYI4nFVHMDY?NXSy}Lf>6y;y0c7`f{c4F|YYo zvLshrdMzk8MOd3ikiBXx`k&>oXe5=H82)6*FJUH0K|8 zulPNeV@o*dzU5@wRKJ_QZtA-Im4w`9~kqK|0bXj5-+6idA(*LPnl@}^*bQL4-9)7HOS znYYZbKYNJ(Wk~YlyQv43>MOhT6gW=&vNZ40y?1@$#!-4bj_)n^oYxR>n431w;PO<_ z?UI)sX!EO!|KR=iucF2ES^mf9==x8}wajj8yMA73kv{o=!S?&!H90f6=7@QpExS8o zeYf1HWc$;*t#5)Z1c@!(@kia@-VZK$33Unk7xD~e?$uUlH^{#IrE@9vmtO74)yG9n zWuzWhrvEvwxScWCsJ3oG-PM_Ey3b}^U9e)U?xfH8|F>qpD*pc`efxzaeJ@`|dRgDN zdXo8-CEte49%+eQGM`@M2peB`_Vn%aNy@BOd8Kdd>1;g6y7<)V7gJv}OXrx_m8C^( zlip!At+M1_Bj+#snxfVBj8-1E&%BfIWyzZikEqAHEnZEZd&k`7gy20A99^|?_jZYVZ*^&cG1O|XBRj& zf0g~bPr`NIb{q3gkN@%>&DG#@*z@#Dnex)5yah81lii$)?4}31W&8X)+5dk9*Xflj)T+7OQj$AvTETcCEy(0wgtZCV zlrucM4ObtTr2l?7UHA0l_=9=8vdm?%nxXWq(TZ*2qz4ztDzu%^? zw!Syl;!DrF{_^>nZL>nIUbn0Mp8fQq>-85(In@?3?e?;H6sCUPuh3z4RkZDU$HA_~ zoeLHmxM?$6@s7huzPevxGd5OCR5<*{^Kj44yQSY_-+ePUaQSBX{I_L2#xt2_O(^@| zIiYdq%0Kx#KC=D1wtb&!@G_sm$4%4So$a2aR~pWUuX?$3Q_4x9ogWTy&lQnXXnR;y zzAk$IuUD&0*B_G(>a~md`Iq7Jqpi#SU0|LzO+Wr!gK93r`<2CeBoD|;%6aq5H~F8r z{Psc(hJSxPpPy{MTBMs(jQPah-}nFPEuUAl>Q3qP*h#ZzYnR`vT+Y3?|H8lX_W$?H zt`OIYdBO0Z?)Tg0ht}`;1>X8Mygo6i-+8wK$0de|l%Q?JzYRXf_^L(O5{>!%;K z%fI`ieD1h>{hLdBzTc~sec-kb`F`=f=UmY8>G$vD_ht3Mn5@NgA|@PInty+h@wtQzlIlM$n*T7I_c590;Y@?> z`}>w=RA+^${W5vVQgFHdul+HJV=g>T(+v++Cww)Lu0OPUSDztsu9lHz@}e`E{IOz( zQdYk839c^xd4P+b$9PZ0g~%_9CV54M{Maco)!KPp#E&WN4;EcJUHG_f;{V&-VopCfeV`d)t|5!pGxZ95B0-cILvFi$%w- zXg{5HGkt59>Q=5Zlbx$tJvL9ACU5yFq%!wtv)%SsP4i5< zeBFx%^8+2{=kaV?KDTVvoSIK3t(S0uw=I9aQ{2zokUpodZF0^d`I+LWOiS+6d_L=E z`^3NgSFi=20-xu%8HUL*vTOECEK8a7Fa6T?A3OdgK9hf$etv%Kx0%|~-CVaCwkhb$ z;}Mn@Qg@hp_4rj|n@8rcQ7vl!SUc_0Qf}<2{G9UT#l^hfMNf-A{*Tqk{=Ap_GHB^U z<=?N@52v1ZIsNf3hAjL0KaQEFyt%P);d&vt8@4YNG^d|{oCeGGjc20%f3Hl zNN)MdQ*hay=fttw5|8pZ>vw7zH_YDm`u+K9({=T>T|JvqKP&cSpJaydi}$O=`)ndV z%(vko6tkPV2>((lzwHI#Noxk_js`1}RRrAx2yYnYpULL0yF8gE4 z8l58lAENo@9#)I>xihbcD6wZTs^4y% z&M@oq=g&_;3neB6WbFTWHeXBOq(bO*d(Hm89xa8ezoxE=Yw$BmxEo)`R`aQUN&7Y9 zx54!)r7IYoJbPw!PkP0sm`4ZO^c(C>on(3bJUE%*?MLlRXZ~4ynfmO-^)-=?_htQf zykyx@tA7!e470@iZp}H$W(a&ERMDP6baK~H8s$X{8cJ-Dgo6k-< z(*5T3zg>~_8*9#dJN|LG_YQj%_de*2E;^Q5~Tvlma7 z{d#?}yO~Lla_81NAARPBO&6T!W4$|{c|pbXi5ZK3&ze4cc|vt3vxMA>FBa}Q_ne&j z{hsl}$$O{XZP;ONBW1f%p#A>4-S73P-#m=9=Cz&gI5DmM!$J0Q_ENSbQzW+vltkBm z+dTi|=JR&i{`P;jXfLq;-?rz=Db1-||K64FFSgnjy}gRtMP=KICrZBU@?!TbzxW6L zjJ7EAe!aiqV-mZ-%?s)2`zNXA3hJ}nJ1G34>*M!U+0!>Js7w90ViA@S?b5Pqb3vp@ zqK;nB-ST^t?iO#G&!1taYgj&WYxZ@ktBUpYoB!zY=KMc@>HBX-{s8tI>A6oo-YGtR z^2|(Q*%zi#+}RtF*Y97N`t-P4Z2S3V%i6OFH*mh^JoZZRrk?G>ceY#mI$@zf+5=*{WM)-PxUF+U;dba`!QS)w&R?W19$3Lr-JNr5U)b%c z^s3pu#@(k27+%#iFOd;=cm3+O_Hy^~s#y8!>jP)+`1NYF_1-SEt7+S<9`I{E`}6bj z;RVf}=NhK@R=g5rGZ4A)NPvI)(a-HOZ*Ts2n_<3E=~F&)~+w&@iUYC&w}N(&(6(# zJ+I&Tokit?M)pIQn>xRO=EDz%F28j&ye)CNO8w)t^ZD$rbiDAMW}jPnzxKQ3f^PZ8 z_GQKUr#-nJRQ2P0|Bpsa3xNWKe-;9pU1pyTV|d%ieC%Z1>}4m{+_KF3@OX7rL{)B{ zo2%)Xjgle}IkzutSfmkuccHq@;=~UI<;Gv)*WM0qd(Kll|LcUJQqc>`>U`QPJ$HLF z>O3|yU}V45%3S-ZZO1;=2Tt>E@HjAa>K(r%crB2Pq47$O|8tLsi9d=C9eQ(fbL#fH zW!%$Gs?}`%d*JY$e@0O@pEsv(UdTA(L-e+snKfT7y5|)ueVn+AscJ${bj^O=h=0fJ z|2;fs`fSOh-|zR|?+w53>+#1Nw)(minScI$-_Osiekd?{=c6v|n7^|ZtFLQ~o?B^V z{xD5P?z~6x|KIoj?{ei)nB6-+M&n?n(8o2iW!J_{tebfLin-0R&jN``k3akT-~IR7 z?WcE&&zJsY30oT_YFEl)H?RKRPtaLdZgn4;|G)SD@3eEBslhzkKvyo_`i1h8)@{bZk@C~`ovjo*neKk9zs9WUnN^6# zuBv;VMQsb%WH>AyJa(AG$JbU`dr!CA=;o)FU7Y0?#k-$;XZ?O!?dGA3Wek!kZ#R2h z`(yJ(#&%9c-7W*)l3#&e4?kW0cb~rSG5-mM$;Wm~=9n>M*CB%$p#8sg0iJ(YEuuoJz)u z;0fwgn?3l7i;L$jnXRZQrLm!C2kQ>qSzqsd<9BpRH#6~=aPs-?`+wJ6KKyKl*@3fL zGA}>*eBQo%{+pWuorYGM&s#p1vHWyG`R++!20?N6<8sUuV&_>~UIe;27M*hQVVHHF zVN3p=hi;PFP8&Rz?cKNbYy9@SxdmEmuN2ufO-w89atW_n`uY0(_siz#q-0bxY}opk z&*tOD+y3jRNuO`<3l#5#}l_f221aeQ`$3fa`C= zFS}2Rx4b{`*Pk43w@7M)e-=6n80wDQycu5}YvP1)pjl}nC^Z%Zrl%Bzw^ zC6~^pTsm@+uZ8j0A7QNn*BexX?l<$>-FT^3oNLlnUi{?5L_e)t)BY%D^5ih5?XX}` z=s8<`{Fnjriir#-4s`r2x9Pmqw(*Li!wI`DM+@llBdeUct*rb1-0qE zQ;fssYVe2I>EA9fUJ!j&dq-J=bVu%k+m;{Pc0HNf{Z`G3`5Iqi`=Qwj-mkY>ZSOHx zb8&l5$^4?9Dc7$~l4rAA@-lMfO{1#AjGIhe&z*Pgc+;=#%E7^_iXQH zY37?h3R5Hx^e#SgX2I@LCuL;n+Za`j8Ywz^&+~|`{9m~E9Fv1<{Ita{j`d25vn5QK zZo{i0sqrnR!TB(!>JQgr%wK+GM)-U_Y+z&J`d;b9{A*AAYcDve+n?&mzD5)gy9Az^xTz2Ss#I2K}`E0(3 zKTS_*zTv8*nDC%?V_sRwWPgr=*DttNO;b_x72-YqSyUl(%X*&h$v#zc zwVS-2N=h z)~u@bsdoHF_uOk|clWbMBy^m0V%WeN;j)!^Ytz@)*LQond7QAfxV5B1KCkZCwjDjH z6YE+cYr50VPb_@gIgdA6`)f?z561Npn;!^2W)aHScVecg@q?C@%-UFrDxjP)oG`u*WRCSYF}%JjsYWk)RD$5ix`boJqVDr zOG=5Dx?t~;IWad5o#%PQon!Dt+V6$L;!7ERuQSelxW_0X$;?w?%*UW|@!{E1XSkXT zn7O~L`;=3+=X0It1cQPLiU(BYh?xhzV07^Kaq|U-enbbuiif;rlX8yfGngf}ecWhS zcRX0)?ZI##v8ZciQ_jEtHhp^8<_*j4FxI~Q86FrY_-yt3`;FEQ91b+Dd+xBCF?ZD? z_S?_Wl1^%!m?Rk8_G;(x*(qgjg(8(!rzUSIf3aUWNA?6C=PpxO{<_yUZ&-cgTQ98= zoHFIjToXsO)UV}1woB}n9Y67mfAXiBPi|k$St`Szbm~}}n%bfzSC5~ZwYGR^#SQVG z>2Wu&y7GMLPv%nzU*ap~y&z65WPst{)`F0MIGq&oNM z?)iBuJt&9yHbc%ak7M0^S<%_^`!;AAOBX8dm-+Fz>snq*w7kJ~`?%;AI}aKcZT#`~ zef|Bj{L`l2oVNUFbG^>b{eM24ei!>tw%6oeiM3Y!Ed{~f{wnKl{d)5}JNh!m3!!f2 z&MVK}a&5^}o1HV$VMc*uMFF!_@wR6tIxZ#VwU-p8u3WSBzDjH9(V(Dsy;V2YGS7|c zj9&lzi`O)sEBhA*ec7=nPBKr)Nw?Lr>--uu)pIEz1tLIy7p06Lf>k4CbHjiHAL#?6%iy65>{3e|E z#hdy>gvHG5Rzk=8z3e|e_T)aiF#C4S{&<_q!Es^=v(0jgcD*bMmVa;W#CS|oec@_AJ3f6`P_c&Pr?5u z>h^`{S%>4#$d=!EcyVshJ$hx*E|UM^Sf`+xJ3q5bBM&$C&cUnw^1J!)KaZsT*A z@&>!*4v-D|PQ zIUiULFr0a_+V^%*uC(Y=#d_oEF-4s+wO=kKPB@#Y5YzeZjPdynS!Iv?Pw%?@>YXiL zYO>++`|D!M0!`wYO=ka*I<#l9eNTAq!hX~L9#vw3J&8I-66aUw7d{Yw+wx?VVORNl zR-w1;xsRuL{>?0ZA$0UWbx4m}zuem{pWpw#ZQoxyeQQAY`t>F&{{Oh$RrTr43IF{O zx2|`w$!(c4&v;MnJB8Rg?y{v@)UHNEI7Y1bIhpNer2VozzrJ}LT*g%S>7@GI+=)jn z{{Qv)m(lsOq_g`itHic^F%-6noYyD|D*0*(7C1MUg(fF3WXx*0eZXw85VM$>cA;J6 z$I6+UGM|6FUTDRK;G+|Ch~sruTv2(kie2oC(;cy?#&7xBK<= zpC^F^kq@aFGWhX)H7RJi?a=?zwMp=EyBMe8uCpF~o$UD!+Iv}byegihFSzP2KQlc0 zQ+iC&oTQ#dwoi{+cgw&0WOrHb55tTp2ajppEC2n_-7@#iqB$QgO#85X>$Bzde^>7L za>?6Il`T%L=!BwC{ym%P*R)F~tH+yt_%8jpGR6Ay8GB|J2u6^h8z~|l{hMOM_eu?Py{}}M>o8Do&-)|mI`txS<`CHa6TYS#{G7w(On5*W+ zpmm@}{_nkCHA*3q_Wk|$`~7ZKrmu_bmA@K0q}Ij7&XBP6R6c*?=mrM%nAx{FKQ@14 zIK2DOf5Wm5c?Z=WFYcB7FwN$#S>fASlkL5p|4Mq#-D)@adC_f4_g2AejXBXA58QWl zvh!v}NXDoNJ>a+bb*_BhW0kGb6e*}$NmMqx+eee67Rf6q?J&R>5`DN;Ug{uF# znsevdJ*G3a?#T0(r`*Y?J7W3w%VpM5HL;p0ydQWDxW(%0>-$ICDL$QUQgyDX{>8%f zJC9!W7@wPPn9o|rX1&X4`G*|W*V%tMq5Sa6ysB54;q@B>a#F!_AL{W@0@AB?_#3* zs)cVF7+#%sIB|r>!C(S|{+sD5N^>%I9`BL7d})QP(!8(v%*kvLXYB)LFnFY1a(6xS z%c%6jYPJXc^V6#zNSP<}PPY89k%4syliUQZ*=J`oCMI$%*zUSY_r}}WP32t$m%jDu z+azl=-#nIn&-;_4`G`6B9olVCFwDGkxC6 zcd85S)L1UaOWTnXnq|No>f|98aGW=n3$ z&zIp(oa>wFex7>jr!H-!y}Px#G$Zn7@=E*n-0x+%xo>sVrN1=&U|7H^!(nmZs>}-h%HfR?*f*j;zZn`EFEStXcDeo3-VGq19Aw(4@9U8h$n0d46_j9R_3lhSd4uNi z6${qy3kvm6^PTmC!D7ob`GUjp*ZH^(oc(cFm#=Jd)j7d0TMVmNf6VmsfB*E=L2ubGfn1|BZm})thWrmmOmM z-@yFw9HRr@q#Oq}HJ&t)2|w251;J$RL2%l$r<>n(!U|8F*mu=KoK$2RROo6BP}Y4gIfW>#-pOd7KMUON1j&HT{9 zDLiKvANRa#60iD0W*-%t?Ql{2*^Cn_gO{64+7M+eAiq6*a`54Bw(iI0@Bhe7ocQJ0 z`FnC=y#GF3E^FBx<5a6SY2UJCQ5LiMdG}sk?q9B@@I}`6-=DMn`@HS=?S4$Cs(iWm zz>mxG|2;YIH+=uE(C$~SPBu?9HmYB2Ua!4g^}+f12J<3CKAc;)jQMlw`8 zX5^0t4h&l!FP3czVv+H%;4}EBAHVO8?05HvyebkI8#?Frws_oUDE0WqaDC1SCI^NE zf1jD}FFY@-?pN|aYVYmi^7Szn7Q3E3b;*z|((YgSpEu9-y_ek1IJ0fbU$eab#VIk> zek!wr69e8q5K=nBUwT<K2z{yhsrZzogK-vPaotscJ041m_dRw@Jo9$z^|*IW zI2d@_PRH+%lj}6UemFYb=$?z+?0WyLmTRA0i_V`K>}PrOqmbPKyMuzO)71Ruz4OT&1D2#1fQjo;?m*Y68|tU{G0D@i_PE1ws~LXi}vk)6j5KqANb?s*4~H8K6o?vtBiS-ec(`02%k|8jC{KAlj0deL2;SKmydIQZxHmy_S_ zJ9MYyvhSgj%Gb>QK5d`3X#18wYuDPpXB7Xmp}$i&tYZ2D%RTR39y`WRU;U+IUdi`g z>Or&1Ppwqe&HUh!6y^O&_(=f6Pek`{@35VqTx@ zp9jtSPi`dlmws*k?qU6Y&u2fzhO%i3}xaTO##aFSG0Bna2jOQU3}5mEn;z}JbHqCY4Sske9z1C^Gvq?oGH8Z zkM2o%tKjoau_;eZOiY|A_>E^moxk}p=Ev#&&)xl3)GKYRWY|*e)Zf!=rFO|%ko`^9 zZ6$+=xwk@A%wNC1AjtdH)L;h&1&=!uUNIJ2*vuAiyYhj@x=OYfVZI&D*!Ob-mQ}?df(Gze&{Ze>HQ?3(Kv3OU(ME7u+r01IO8JG`Phae<2vA-ypJ7{|Eg*}wOkI{wuU6I%XHN9u{c=&yRD&- zc%X5|1BL@)vVjdo4blup-Q4HBT_O<7`%v;hZq;?Rb&Te+i44DW82T(8aZI=Qb?`-1 zam?*b^?5f0Bg?#IH3aufT+Fvj>QI~8k8_Wd;&>CJH*}=Twc7Av;r@ip4$jACoSj^^ zA-ndl{?s0x?FOICrR_@CtM+kx?E7fSu+r4(Jwwg`GwuknHQ)4(X5VIi%4H!?u)vU= zX~EqY@&|(tG&V2?S|kK{IS4B-c&fhQX2@J}?N8L=a#_{xvae3qkK zvh$|T$rU^2gc-GC6 zetNcu^gLEIP7(^!wixC$2JiQL?)wwaf6saT z&jZZ-C94!-UL?QwPoK8h_7juDq3B<%+iHI@*aW@b`ZGKA>Z{Oou>t)Ga=EPw&+V|i zZGBtH{O^TBZ=VHrch@hvw0T7nJ44Pr{R)=-J@r=%4K7+H?Af@jUrzhOTR#~OP|EBq zQ!-p@U3%@@D(?&X<{o|7ce0(c$N%37rUmuh)l6G_O@r?KX}Pg{OjfN)9LeT)w;!Wi*73ZeCjkm^YXG%w{LT2u`|rvcX+x$Nq?aI zqZWa>8qLQ4%k&sem^kmx;&X_sSTZ-`OPG7fjlhFU37`8|gL1M@F3Y_omu!<`l_qd0 z)?$K3t{lVLcb8s2+9TKbCFqEy_U<(m462jWf;bo$-e%?APG#*@On#N@z?Ny0uD$u> zhZl?c%U0^e>

R+yuoHW@kf-Svg!kvA=DLMOX=gc?UnqIQ-UGV8yx2z&JY>X4jUp4t@^y{S?VwS8} z6sMK_^$OQtrWo;_?z_u3&R*FW_eg!^s<>J6g7hu-PEJX1+uQSYzDjVr%ISH|lT}Ss z?MnP7sulYgUUK2d^ZWec_x=Cx<}sf3_!ymZVSDR4P{&L9<^sl)`2~kK-&yRNw&Y^$ zV}_21Ck!hl7AT)h+pMb~y@B_~Gf4%$^MU?MHA*jCU!AQvaK2&c&a-__qD|`e&)<3W zombT>t{cnvZz&mUvN>1`%7~qZ8~vDXZ7ntZw7>WNw8+n`+>_1w%DRuYvNc}u+ zX^}PFmIe-v-^-V&oD6&%G)H8;H}~VyG6LVf-OjgeC_li+zC+-O?+%OSpU>O#Hy0`| zJYPKR>yE<5bHu}&m@iF?lCS^sQA~G3txUu5KMZ#~b~4_T;XZM`VLM~qpMY2;J%uZN z+ZY=j^EJP_z~*T_KO-*jKZoWH`4ID>?UU~H8L@~2i``xz>HRN9;OUth>+(CtHyw}! z_2rui7aVRhn;mL*>i_aRoBkx;jEa0=_t;P%QHr-g?oipkdkn09@;cbr}I^_vHj=6WBmq8?*3W8IOWsUgLOF+!aq$0ZW$(q`LpGX$3=Djw*! zIVo@cQi9j&>Wk@{X9|AEKXzz#)v^1J5|Uq-&zbEzs}VdQT4;G8(P8$%^yHgSl@$+t zCxtJPuAOru>c2rpi-F1EPPbtD@Atcz^kZYr-*+{TV!E>Tc*FV=FN5nf{~ELHKFB%g zz!7%#u2b1hw!gn}Vps0Rg7tbGFmdvhrw-GevD9g0v<(toD=O+CYk5j*$eQJ-{ z)q}3?LhMsEN~ST~nz6y!gu$}D`@JjY<(DU(TzYvsyJ61$=Ua{H4!nEVczF?9+4X`3 z&X2GDUpevU5JO72FLz0Qj5_~{iL3d%3muNJU0%0N^4GQRcgyF4HjGZ1J^OXn=gIT` z9AR7-|L@cEhmROv9b2I?i97Axp3i>I`)6@h9bYm3hmgZgIQx$Vi9>+tr>CZ=o8&dt^S{eJ)c(_*3mW-Dy}{dipT zYUOgXXpwuYAu<=PADvVEZs(^f!Tvjo7f)<4ax3y)6zh5~C&}>vpNxS(7w`Iwj2CzO zHB-2KVC(()KPK86Eu8}|{e(l3tjo12v^@to&z{qJmR-jdB1g|~`l+_=K| zf^9|gy$iE9e%Y{7a+2>(hN^kYo0Zvn^y}o_Noap~39j(+!xx-&FwcLi{^|J6>ATPN z@d-(XT@y1~BeShGv4p|=oJJt?ly84>ayTz%#(3{8&&vOKc3=649gV?EC(V4`Fdcpn z78t22z9&rGHI(tfyC$_4Ws%=rhbot!Suyp^8XtBatHn)UlBM^67Wpl2KkM68^Y3`U zJ!7lQYfWWhw^w{j+HNRx>;J#+`;BUUl~lf5Iz8ptnVD;S^*ZERnb|h{+16O_=a2mf z-MA@B=1l*}oby|9!M1dzUymbavAvtP!o4Z)z_$2HycKSjdbxxa%r^13d}-5{n+e`P|H2dDD?Vdi6z>CorfB7Gq(;l_^KxiKFvS|0jXo40^L@i%+l1!J9p=1$LDe{B*I zEsy~Zd*su{uA-f9_ureCF?UMt1YhVASCopOA< z$=bE`31~QE*M@%EZ!;ci9si>nyDKE^%#4RBoo`RR`uh61^~oQgb(jZ)Gv&)}B)&Vo zQmrvFo=+m9*YxT&hL}%J(u2yF?`~UpnPGcl-k*sxX8W1vnY(96&De6qWah5hR~+K4 zuetHOVCE}fWWSYbZJh90zqo#Vx{CPjEQ=j;p75G+&ED(ita|UnBHPZ=xy6}0=Z}BB zvi!o=T}QGMYaNfORhB%wTlCLv|H~>SE?LzjmTd zd3DsElYf4_USB@{S^l1n$I@O$Mnt?=a#ekP&8EpZ!j zlqJ*jFZbNn9v5;~sj}_*+10dU_GRxMO-&-ZSMBop^yyQ|rza;H_mvsf@Ha8>9Z_b# z#hPIpSTpN?h%k5;Kl}Ilz4#T|b2gvP$Sz_`xL5oA z?uJ=xvoFe3JYbw@QJ5r=5np&z)cQi{H1^WppXdL-qhW0pu!k{n)d`hITmo=d7-)18H~o)%5cV@)?W zxmSc^-Nd+dpUEcwN@g#(zx;pWr`0d7?}$FKMo{POfp-#o8%`u8?RaxvGbliEMPCWe z^y;)#%)zIhzG6O|^5Mb3!~>GO!utDuEL!lZ@QmScjy${Fn=?|b6owKo0lP^nCwx%`>PTAkX`vxQ#; zGe33LzdUK3m1l3}D>pk@yPEg%cjx)?q2(sbv8{2lqC{;Y`sAkTq|SQMzIygK>!@9w zFT%rSzf!riLha)DoP@8}TTdtN`~Ua->AK%<)j=~YF`#jEGk1qC_5VK0i+=?zZtPEt}#ghtuf&}@yp=9?aEMwCdLF~wwA`y44V%RTo1?Z zx$Cm1M9(YZK}YMpuvJed|KWcsZK=M~@uYoV;I;!>?AZ@pn(^yh{jC#qyVXuQ&%b(d zV)Ept-G>5>=P7!pul?BYan07!q5~bvt}6K5HV(bI;nkD+*JUi1c3OIVT3j3abmvju z1s}wwsRk|z&7PlBv#{UQm|I-0{X>-Z8m#qo{65DnylqtFuzrD1JmV?G7gU-qTfsH zdTjjqkW5>O>d9~smGot*nm5==w#$6j)E5&Qqu*@gdTqtssL88t{JHaT<)`_g7ah~H zFMGHAD4q1?oMT>=gYDZbmzS(Kaa+b=&T0MqF*|IQ>AHVVWwQU1%92qS)OlZ~;z6U| zfdqxwoA>XO``2gvj$=OiyMFl{k0(Y0s3TT{!Ja=W^ro?*-5&2O86SY zTm9BFHT#gzTY+U|m5h63{;{Q2MBLsib8f}-IbURUK3~E9Wb)U|PcD_$ubY2H`^koh z?kDED-%}UgUpsG&R^&yUy{BC+yH({i2h>&W`~BpY`Zk6P_v}yKzBa#q_dIG_?d87h zS0)(c-;2{<8u~RT=;nqE9%0WltCyVk5*xbJW#jj@+^?U$96F{@^Y56|%|%=O8c%w% zH}vAcPnp%rPShQqoN{S_$joJ0YIO?rpWbPHx_8Rz<X36BR|MT4UJ9?WZn%w>Q^T*?U@wzmNSk`|xEg$^2Uvk-Z z)AM=NW?ktYt7pZA$7{#u{G6|&KTki~d%9O(YU8Q{xuILva*4{km@dN0@ALh+L(!x- zrzulbC1>Z0{`ndnW8~uYWv0U8U%rz$|9+n~<%yipgJXrk>$TTTwzT?Rw{d6OC+}BF z3wC={PJK}wrN1|Ll8U?Ov$n6Emu4TDoU&c<I8{P{CR1>cSC9UVWW=8E%(oSW=>guK5&ct+r`&g zU!Sf!`TFXr!@5R=36uW(IdE+Kwdqz437@u^T(IQ7KY#z(^;7q~^t4{lH{qUzti!s? zuix&s+?igx)NsMuVe!m-T_wUDJ1MVN6 zj+lOWu|`?}G?2Pm?%#uE{(CP^>OE``y5l(iSH<7epN_3MzGTNLgM-<^BKBc?8x}kY zj}2e>JH9K``KMRprmw-3-v8A8R`E_+>^4_1{^rvi4h7=ci=#c`17r2`pI$w&f7-PZ zr<9KNTm5NYFz;}y+2pY6DxR6Cd)Hj+pFMxwN28OkREsL|y5<;YhPr)u{Y&4W_o9Qg z`4#-lr(Or?in-z3x}|=eWu8^{sC?zAqvRWP4}VmV8#Sy8e9cqsE*^46E*k ziS|pbIxV}}v8lH2RqFjuj=n#9yG+l%^RQ1lBk{2OT4egpR~svJ40%MG1Z6^CK z{@)_ExZj(?_~a++q`#Z1*1Tz#pNFXNg>|b#eNX)gxp=tq(2QHmaod=lkN`PWiPZ%{(B| zlF9mkLv=wy%6*l{ouT4$1mkCl=VZt)sQtA?|d=7^O&o0k~_GqvTokonbX#O{BrBtCZ?;~Z|bn7wm!VS>}~6(cK^Jor*38S z3eLTEY11E__dUPXd^mDv8t1O+@}~>R-kxO)ann+EFsoePB(z%cvb`st+|)HzH$98) z91;0G+s49EZH3m!jlxM)rOCQc4)=X64g?lmUhdC-UCrD)Ja7NsGWKm}i~s-qKK=gR zcjez^OfZ{NrE-(;gx`s0-Q7<4t0Er%5{N&;oO5xXFO&WAC*dbu*XM}mv8)SZTxXr) z*ci9ADfRr>%FkQhg-1WXX*2uwQRRL6UQhgT^mg{sY2US;-0MFbTlF^j*WBe|lU;7R zd}W^0e&X0J-C47@Z+T!M*rR78Gox&_XspDBo5AcS^>03Sv}fO=T=lJG{E1bU8t3hB zYo5Pz@9W#9H%co$ee~JO<0C5*w=v?Uj>!h=KIUsdveTVwmhA~Nx9fX#{Q~paDaoQL z$Fsb?d={^|T^GFgRmhTGDpQ2US1j|Nry+q0h`M7H3lqwQwfdpL^@} zefDLa8|=dG+dTg){xh9*Wl91Q-;qO&Q&YL0G@I(5T)5bur%#}-b)sF=z4JG-wVtlp zu6<{EV*Ae!t==}nh)c6SIe%V>*h+U?}nX6 z88*H6>w7}teOdjrtfv!olylnz-*6o5J!_ozC3C~+)!M0n51$u>yyJ0qmRW3HcE<42 zt@5~_19L8^?DgCb`|H>77stPxDoy2bFq^yXvrkb*;nhmZWA1*hMKi>;73`{N4zHfF zR@!ia!}g;e_->@VJPSTXZI|2r)K&lY#)O9cTWj?I6a*Fh&E-2^N6q>uYy0l9zrAft z$DTbljz^C;lob|kyi@ght>1QQ8O=VKtEQo!p3OS`c!i5&pwLX_|B?TvOv%^h{?@qF zS$Wg+ypzYccD`EsR?CJ-x^2TEuPul3oVxWuBI=lKXo7a__rsFJ9_& zocYPB}H?lE%r^k&=%rKGnAFmsTZ_x z#Qm%4R^-n89Dn-$#4S@wc4i;Tt!i6Y{_)6%H?dC|EBjCWEtda04yGU5p$}Sc`vPQe;I#px6J!aoorqzmD`KlMDuRX??KM}NH zWzUNJ6Kl(-=G;}-xv64n&U#K+d(lOOM|>|Z-}2PjX&2dd!tTb@rRDGMJ&gU5B08h* zaYN$*JI$$^mYS`8Y~53i}Hx=#7-PQfgojU)j53k$(PD)SZ?bk=$ z`drewXF*fXpT6(^e^*w&Ltu$&+T&xrpH|2J^?JAOcizkE>*KvW$~q3G>|6HbwCo?| z{s!ia1qro3i^_e{+}zw6Om7|V5Idm#<=V{fFC2Cz!an@Bg%4Fd5U>|MWwzt$(vP82 zmexBd1bxouu1$=bWb^Z;nPvEXr`b|5(yJ~hO)3kNWz2YQn!!JFPRqKo=6Pm3CUbM2 zx0(O?{O984gS~ORCssYGxq9=T)3(t3e4oXQaW*A;XD^!{W*vI$Nl#g`wNS1YW8+I1 zz5}lnGgIeIj-24#|M0+pCzBqW4q@76<}jPP;Mf5jK7NjYr zCv1ppQZP8f_#-qpc(KBEpH*2Ffk%P`KVEV9lD+QJl|OT=I++~w&vZx2a+!Y>>b@=Z z>v?H$if3hN`}Ra1pdH_?!3}Y-OnX+YCfI(v^Kxi zJFf0$>UY+wkKUiz#<_RRw%4Uc%fqgHtTSIPwhC{^=YI=!3y|MiNJy}_3Cww_v9Qf$A1)#IP7)B7&GXa4^=PejjtuP(JN zZ(Ss#ca%wlZCRq&3gMt>8P32=GhD0hhF#g^ZU8$jZuZTq(#93=D)gsO;&octGV%A_?Oc$ zcSf+Pq|OE=;erLS78ByO`yNZ1YMtOJ&n2dgbhUmv?-f6mH}1wR+Fqpo<;L`EA|!BB&Op_ETRr?Ol=+y3;1`Yu7x<1s9@zE1b!` zu<_#^gL7-6w$#4*%HSfN_wmXzd+!;~UKlT#K4(@T;-QDMw z{DQf2r)pL&eR^`utG4q;-%hNX;CBAWfhoI=UQ)N#-m`IMdeOq?S7uqJ{QUIP&&y-& z_4V=Z`$V7Ia0n5;*jfu(xb$Z8`DyNQl}lW?_01iP6TQz*u;P1R>+?SZ)paVX?h-*s=vhX$LUotD)DwI?+2W^@AYc4F=F3TZAN&@b z`+D;h7whF+;m-392faQKGG~WX)Bac2{rP2o+)T9GbbrOYNmHlkaGQsmdi!5<($wfL zWkvflo9@Y8@U;<-)92=Xb$7?%dveZ zkF)`{&N0tfGV@U=8<^J}}zzHYky=j*4RO^H=n%zB=YJ?c#3 zzpFou*1k`>n{Q|sVX=Ki_FZMAS1J-L>e1Wtrk3BUJU;j0-+R>_u}3}k^4oq1_<6Cv z?#aCJZ^u0k-1&ZS?oppVACAtpzjk@Indi(^4O2qRJ(dMezWk(Rqy5B>TD4)W_BSK; zdCiNhT7K+fNLJTsu2kQX)A=m+Z+`YZQ1au{*uLq`?sJyhP>C%1?K_oSEJo4U$nE5> zO-}~L}{+#9#=*#p@MgOoo8{SNauO?TJ3ZL+Dm zW$-iMXW7mV@~l-l33JcPn4-x)-zelYg&| zH?QLcOPTPL^Ro&+_4XW2dGa7O^~s5ec>(86p5&}g`!<>VUEgztOB%Q2YEMf>i=R53 z`Z8#q)q=;yd_^zHWKLY@&1OHj+^FJ%NBJd|1Kl504Q$()MMXtFz1e*Jlyv@{haqkg z@{*VN%v5rhuiYY3e8x~WiEXmg{&`7hYwK0BLhB8lZm;@uN@(ZX{r06vYwm{D|DI!; zH^tg+_kpAFv73(EWPQ4N@ukN3q#Fna!eRjxi3!VZND$^ z{pkT|o$|HPD(-4;wX0Vw$^P)*;Gu~@9(ETVbE~EF$Hsn~zRfMQU0-Ukmp9VcQzzDG z_)SP}Dq4_#H0$R6XUdY6Su?8l2f8NN#KDng-m9`x|0Wk@&EE)B6dMe*0&AUT`}rK=$R@1HYH<*?(eM zSFh2w`*NQS-(5HHxcz>=FcJNn>`k1m*DwDL&z9z{TUZo)GE;fQ-G<+XZb-GS6tsSj zJ|p_CsZVIn{=Mnj{(Lz3weQ-(Ljr$47k58>%k92;g1D*S7o|m)t9M)ynQEHVu`g%$ z_fn@?t0P;aFDxrzTPLY-mY3nQ)`>}((Q2puY+7oxJFWAH>RREwSMP;hTAMQE+GA5~ z-u~@N*oBQISDilcOVG$Rb5h(_yHlP!8E!J<~TplH3W^q2gPzIkSE{dO;C zp1Q4SnvTxN$)YoVdz5A!uG}>@Zim9*9nT%2d^`0nq@6x;WozC?o0=aV685{^*&cOV z)WFy{SWGWQtWuS+EY3)r`NF^3&&@R-tmJH-b$)xz zqN<1GCoe8Oe_^t>-q-Z&x_22*K~7V z>Njr+TH^mc#j?z6natW(N{Ne~JUA9IHS7C%pJ|73d8>{7eBq0(dST7<3dy^q0XfoW+p`n*mQG%}W>L&1t!t-d{qo(kbe~ICX#Z}nM)QWX)8iK3 z3*VG7;mgE8P3P2*R{=I%f6g@JFw!=3k}6N0~1 zY>4Apw|N!UlKQi5l1i*b3+nscZI;k&$oXH_{b27q##&`BQFWgg0!Ph1Wa+YB3jU&_ zQG0z`n902-KcFY#WKYf(nZKAt`Ru1hKR>}Y-QYgQT_FuE8C~5 zGcM10=D+G~>MhxfU<0nM7U2!8y)8@=VnRZmob%TW{o16M+MJqy^{Df&lU&dLd_G@Z z`s?%j|8KtR{h7j3&@ATgwqe(@#@6-@GF`LQ1nH2a^Z=vI?H5V$5x7etx{r9RM zZ{pVIz2!gVcYCKMuiXA}qv@Avo%7kg>q%BI7f#yFdHYm);qBA!1Qz%{)G%VdMz$e9JF#m#2NS$vV5D{EY%*V5N1X22^etZ%>M@sd=A-HT@=rE%{S}kYyyxn^h>UXc|OF6A9pU-))xWDe%yV|GW?tBxK z-$&_P-L>uwa=t7nI7mBx&8n1 z{dFdVkB)3`k>2q3KjR~h?)Zd9lbOFMeKbA9zisW7$!fk?B|4M#hKR}CxI3Ns3-9E# z%jOdNFVt`R@#&J?bN}vk{*tn@FT7WJ?hP^B=s4r@&D0mC{$#z*ezLGN{8VOe-D2?; zk)1n>p~r*los*Wmx@()0{$b1N@2^>YsZYP7)33kJ(QkfLYfR~b%bPY;Y~(pFr~M*x z|MqV?!$3#G1i!k>By*{)?Qw0&9Qm92k!;KJCi6~RWtUW;$I@eaWv#?|M#<{JFF)6n zo1c{5w{`WI=xDXg;jgEQe4TBuW&59RHx_=n&d;@U(Xap8%k}mb#+J(b|2g$z@}Dyn zb+@Ik$7&yz9q> zKdmr1R4B{+JMVWZck2Cr9`VPb7IEjM#uRLAa|`QT#WL*;UA^oc4haJavFOJFaTvK#7ndXL-IPI>y zAGLbty84gX=T~m$J1g`lu<~2y9k;aW`8zZj<4cWuzf>5%dTO@6=JoS18}o#{vdwwV zChla;Q}Mj9D=@R{P1)P}>84s|ZIU-8nuQfrTeN&fo#GfScj{@hYAT=RFs)wr`; zRrc)oaO<{qSVsQF`!VwhgITW0%~NMTcXR!^`&;9!7sg%|>fd*dJ7<1V)w%NAO&b!= zPrFsq?*HOM@4=mlPP^?MFI;$a;|za+ubG?X^EA5OluDf2V|!_e)*>t4YbVmpwzkC=X+wQRJH?6$=@5{qwYeT-SV>!CGvFB9}XY^jXrO%`mtrh!N_xHW`xAyJRa?(y5 zy?isxTuf_QG%3yjD2-_D}z5-H!Ky-4^UHeq+s*{H{G!+y-^EUSO_$rz{30WE2Xo`AKKt8WC)O4&y6W$(BYtd#ll8n7`+!}K z?N2apYXslL6KWmELt4`p^&w*IwceA{22Q?I|~ zp3%yy+@F%y<(D%r*6;s&eV&~3hs&qDz8o)0cFRbA_DyB6Km!Ad!iOC!O&8w1d$lor z+x|~GrVH{hxeH0e{h1?LU9~sjNa+)gkoI{zUxRE8&)t{!IJdhl@2#x4 z?fq@ht8ZK9wHdvAFLUGjqFeI!zdc|5ZqK32Z&t0guYQG@>pp!tvHVTz%Dn$#*`2kW z|7PSVH%*X|&$aE`@MJo}x3lYWH6>Xtc_^5?c(rlYmCvTTxBk+Pem%=KIqYPx8;8id zsX52&-p9{b{@vw>Scc@mqDMJhvwytQy&oN&_sRFW-qi5ud)x2dm)TgK*K?(2P3)SW zRw?zD)jOtkUG7|UJx;gk?2hHqrWZSYb8nuaF^B)%zt-2kZ=e45cw_n7oiFWX&zWQM zfh|ei;?e#q9B(^PRkFa*km1Dis_;rk(yN_SbuUwYd;csCdmEGMBqZ{43(GCDsyD*i z_qSZ~Im!L@2J;KQ+4oK2^*65BwbF6otEC@`XUyW?fHeBjx zS8XXftM;dzfklC#agJbsveX|(^>@Cv)ARP^y#Erj+3UX3IWMomB%v1(a+^->Tz~7u zixysf{_2HyH71H}j`2LUcGpVBj@sHPOZ8HGe~=k~W^T4(z@ zKFK>S*c!Cq(c5mbJs&me|9hP@W!Zek_0F;@yAtQ?>D}UfmsquI|51C9lWEDx z$-ntMA71ZR;O!UQ^?3a=Kg)T7iV6&j91aSLS*|p!P7Kwb#^$rKQ1^^z&cnU_w}O7_ z-hMG_d)9Yb^UB>%=4<>qEido;jJ-lPwX~depL_7eiGfL=!QmeVOMLFvoRx;*{Y#|o zHa>NXj*s7ds^}>H39io{uKnK>wST_7(NB$wQ?2-c`nFx+Dod_z zz7_rI>%YId^0#=Um?ur^5*AfrZ#)`%#(Dj=SIb>KefqR%?V2?=UcFkie!2Lm)YW0D z-!<+`s^9Z#mi|xv`Y*G#&wN$7FyPku+i@#)TWq=7ka5#DvFWUI=h~ySogZ%8^LVs` z6$m<&9gnX5}AnR^GRB=gV~a6{e4@?|q-VY~w>UFO`D|dsIF(ow%m2 zAe8i zdC&S$v}K#i%p)_8Y^!qIdNsYXpf$16U*XFBfbg#jG5c5by_zF-(|;RhQgoB`8mo1m z4<7qiTvx%x@%He}0ud%q`B>z&V4ouYqbT0(*9!EOyDpFDR}f%{6u9FU@w(XJdxy8B z%tVJ3{kP=8+?#W^@NVO}-Eq?@@XAWTTTM$`4I)3eGyHybe%TZbrb;D;+mmCny&^I> z_s&_a>zT3MPn!SwabqZzXndwRL2BVzM;@M(6YqOFYCmU3dx0t-1y80e`kt5k)!ypdv@vzEmJFV##R4@2 zWZ)wKhm7M7z1D!(P+?!chDi5Wf7llMKoTtExUg8ZU3()q=|bFVV#(N2Hu*k-h&;lz z3Ve-MINs{?D)c6I)g14t{hS-`#SZeCCde5R{q^oboMBl!(FoZY=jJgu{eCl3NEunM zC(?ZfC)813rwBAM*xnLcAf$?{O5oNVX0R-%gkWG2=x|Ulb+YzW_E5f|_&qS`{+-z~ zm_V-fXkmEkt#m`!*Tmb*zxRUQl_V8Da1w`TZZuYFxL}vOzy;)7c%%p>ToZi93{H;V zU}ofSQE=!lV_D(CiKLgY#q2vIOh6TW0|SeafJg6zbC)#JLzn&@)qH(5}n}W0oDAVhx3m-m#|qu>lcq@3=9kmp00i_>zopr03Z39 AKmY&$ literal 0 HcmV?d00001 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 0000000000000000000000000000000000000000..9688dae9980e490f09cf2c88f1cc40a81a8f8604 GIT binary patch literal 58339 zcmeAS@N?(olHy`uVBq!ia0y~yV0ys7z^K8&#=yWJ+3VuVz`(U8)7d$|)7e=epeR2r zGbfdS!J~8PH1-hTn^MQ$pIIUv<;1e2<=(`&rPnW13s0R8;KZpY5R%iOvo68aWfpJD zs;1RJDY_l3+}%ls_bGHY?a~v=S@DLU|HHSvpYK_}pa1#n-?{qb|306cTYdf~!-6}> z+$<|Mh%jXS(MjB}oZHw)YgVT`lRyI_qXEN_XDM8Bn3-!CSMB`x{#Swc+}>By9&Wk5 z|NsAZ)w8|lpD-}6E|NKZ_Q{fkHB7z6GK~h@3@p40_9=NAoK4lZ+?vs-AwTcgk;r$m zp6!U-XKSo(azpfKAOnYpS@aAJhM%^x($k8C0_C}6QQ z=F7`9BB^tC)PM5)Gdq3Gjmc;IL_WXSeWv5VvC|?;HiiG4fBNJ8snfJ(Rkuk=O;K4G zZeV3}!F091=XXJ4`*ZdQeU*1+{xqJW@`-D9vfr}oFEYh9^X!Usyf};JC2uSE7Z)d{ zUDDV3@?cVZ-KzcTZ`jXMefGoW`vgnB2bCM1S*UB8B=v2seE6?;&P<_4#`ecU=ZJST z9W_2}-yYt}3YSpQQ>>FC#j$9L}>t*o0-u}|T&>l;lE-Z|^-uLjRK ze}+%+{_NJg4VHds)$Hefvvgf&-@#b^p!?qp1)dp9#R*Mue;H16#ar%O-k7~BVd{kG zGo~CZar5`JnSAuQgrtFQukRX$8FtN^=l{67cjjhRLypgujIZ*W-$gG;vlCy=;9$K= z|4HX(%U&x66=lYZ(@#5EZf>?b^M#e$!$IgmqfG$YD+hLoMN=c0tRJ+5z2RkY(CA<~ z&A_hGXrjQPeL!jhqez3&2Ie9kPU0v`oMLA=^LKk2Nr$^vJgDiQ2d~#LOO;m{BZk2 zvmY`wy!)8>59vSTWKl?IXklK3GN%@3oK-qm-lWyRQYgG!O0z0cj)feyF)BSe4V^`&+G$_KYpsH z-D7LVaQ;~OL+%exe;od?`N!-Zf`8PRW-f3tP^}PH#98Acw{Xe@4-4lVo)vyGRJ;UD zJGUKSP@2A@YscJ*jvJnP{A3jUCN1&2rBWTFp*eF)V~AUpTC9q#(q84sC$mzdiv(u6 zpHrPav44{OJL{G@EvSWtFVvkK{_2Ql* zenkJqo&}W$&$7xt6+2~l^5|*NQ(I5mo%mbn{FL(P{uA$O-q-sd!MJ$Aa)BiqoD%0B za%~Oge#!k-tVm2u&t}_)Mx`|?mP@SJ5w+26V|byHXXMFX&jlgtO5$ElHFCb`X&JfG zd+$3sO!_lCHxVh{ZtqJDMD)k9Z4U5UCP zw|erGwO56&cwgCl^|)z>$sdzz877%F7gVyMveh!*U3_!#&x)mMMHg=MTpgPo674s8 z>DTXm|McHn}zF3#Ts$Us)b2e{K7f=P%!1eQ$n1 z>W0Jz-c`J8JeqCi+HSV_wmG};D910hSzNaGdbf#~tNyK>E7C8PCv!gzdF*_Q-#5i) zkMAcRE8lywUd=4>(e}-s9eCE}%(dC2Gxi!@H}5yUpI&#qDeYnU#*>R~KbpOC>e36F zZWf*KI;?eV)v2rlv+`s!_8$GUiTB#EqiK83YR*2tZQ8cTZO3!|Z!zBDy)k=>bnKeF z7IhyN9zFam>s!w6=5Nh>KJ5|h%zVv#tNL<|XB}Ve_sTD3dDQaf=XRYtd9HnK{BDWe z9I;WccDruvT37nE$oY=VU9P*o?xfy1d`J7v{E8KoDpe+xJ=IYa>%P7Dbg1fQ#oI4Z z37Q)%*05dMv3=_IsQ27yM^$k~r z%nI2Vf>4kn|yY zw;iq`?mTXKT`|&?x?YN1;#UQG`}z9U_0EfUv@hlRq=^#UGn!`X_TD;uT})N#%tM>5 zq%NQCt(UHEFW24M+mYMz_Y_}3_-6G-5rv_Jk7dHo$({3FC%nDIS}b2aUgp2eLA!+( zi6$5A3d0{oOiB*Qyrex(dwtV)wVT?N^`DMUX_&HW%E~JpSEiXbMdjS`+{a_zTCW&u zS({q5-yuFgLE_AYvp41%>U|NsmgY3m+t_;3i;T^ewJ!<&RggMWc4FO`e_NDlj((ha zN&BaEb@~lZozuIuM;&+ne`4e&=_5?$@K2j#?f) zy!%ht)~N>GT;BKI$L;&=wX*W%Kieg?i)@=y_od1{m3nG@YJZqY=$(}%D|1!`t+^HR zwNStO{n=W--?P4Jztz9>|0%=!1ACk2vEN>`b(ME^f$2@t^L(xHuYR8VwY)8=_1296(O_FtJBWE)x; z_3g_y<0ZyF)0drlHfQb&p_08;(<}0JR9AjK{CVQ@{CJLC7s@X_fArK_Zu+_J7T@RW z&8h!+MfA3|`ueE&-2G{lYu8?1wEb&%cYO43o!__CMaRD1kyyNVbA93enX~)M&)+TD zUHa1O^|!6o|H^;8Z@atKT)^0%qQ5%-OWmu#=KP!FeeBNEw$z6Fh2(s|s5sunMcZVk?lazLEl1NlCV?QiN}Sf^&XRs)DJWiJpOy9hZWF zf=y9MnpKdC8&rEyN}8=wMoCG5mA-y?dAVM>v0i>ry1t>MrKP@sk-m|UZc$2_ZgFK^ zNn(X=Ua>O75STeGsl~}fnFS@8`FRQ;a}$&DOG|8(lt30KzyQRZsTG+BoAQdG-b&8T z)d$Jw8|oS8!_5X64-&Ns$gD`siEzoxP0cIL%+IqkH#V~YtH+Q**l2^$8HvzoWQn2^ zNd~IZMjzw>Bo9GC3oIJs;%3KXqYn;OP#D{Bv8sgLVPJ4z@N{tusbG9_w{pY#lacqo zzwb$N+nhOlf`E`mhDr-7+h%2zu+Fm*)-wcn^kg1Qa5z=g;hV%P+O_IN!|6X0w?0x) zO>S?v$&<)_QiqG9hhZb9fRm7u$6*gC!>F|Q%j@l)*XF&xa^_C8+vYpQpKF)>x_@yf1zTp&lweuaVu5`8VqyN1t!+vXIS#%GWX8zm8-JR1~aB{O)Z1{qrmL+?oAP z?>59aKPNPD>+MLejyu=B%XRlAEq7<C~l=-gRqpB|-rt(!jX=me)w#nb0xA*I{&f{{ep>HY*K8Pl-IIFh*r;xRk z+~OP8WG?Q{eqft*L`kl8<6X-ydJ>PFzZ`y8+?W3%*5XIb4==egOO3i2w@zN4k@#wL zcjI2OeKT%My3QBhceL@4I=8lUQN*JO`BQk*{)*b$KWd$>XKll>x^RN{oU)m`eD^xrifkU?oSKf%#Ki}? zJ{=63y?4^RS;2LukM3hX|HNZm+{dN&?XUm7`*eTI`>)RP8DD=--v6}y{O;0sdp0b; z)%;qn^qGbC$7_G{?3HTXp3~h^`v2bi^HTecT(Z-^iyS9)vf zy#Gnb=`ngQw%;r2zMS@qLruEKaP3F&$GOV!UtWsLmS=b;wdYe?JkOUSx94Pv^Y8UO zzUR2G{QvTvjk~6p*GY8Ly)E*W-(Ruy=Et8-ke+iL76S}eslV_ z4Kk-@&rz=2wcJ)C|6a@Kw|>vl+0HE8TfEOMdyeT><wy?{7a_Wa%=J{EdH?Xfg{H^0_rC!N;@egLX;!F)@UfgqKWZ(Ng`=KBn zBE8Tzjq%~@?VVR&Zoa-@& zes{?OmFLDgrPS^w`Itu<9teH$lsU}sidz4cpHG?l7U|aIiaYH48Km^%&6VEx=!x1p zUhli8=6rJl|K^x0=ll2l2-mUMc6PeO*Q+l#`@c7NfACuHPHp|_+J7krtUiaj7k>#@ zS6tY!Fv)+;xAb1alPfDvcUbQ}aq`2&jqlFSy8L^M;q$YlpR1z2Eik?LZ&vq58M||LRNlEo zGI@D$$MjxjTX6X0s~s%sX1ng*bn14&ox^Vp0$%e!P|hj)sUIV~AtGU`Vz+ev-1T)a zHhS+k%(N0W8{555v^zA*kY}FxnSC{v8Yl4e95cP|B!8zIGrgNsk4^N*Pv3*VEn zB0~My2F~5gUA4JJA2nYdZT`HXP{Q>WpW2c1BVqf0$Ru?1SKgE}Tu^6l;Tdxr>#+i35by!fFhs9Yl{NZc& zoEI*?9z8nGulVa;MfL8pet%v|tCamYn0DDB@xt--*Up98n68>{vpeNz=lUtr-#mV8 zE`DojXq;W*Vcoi|{}|@~$ho;vef~j(Wa0F}7ulCP&3`*TUwEUr=6%V(v;$U!jQM*b z;#ZVjyOpIIx&Gw?&0q2_tm3~1%=n%dc;`Ds>^+L_X zi@bYOjo%l@21@1SV9{KUvcy3RB!OgDM>jEENJZ#ev?$e(MqPxu&g)f{w|7yFLkI~zG=6v?HUyn%d zG3auhm*%(Z*vy(^`hJfL&Q9Jw{mr3oQ$^k;1+FiaylwSKbxm>CeW``Bc`F|$gzuL)AtE7|^DXPw%`erQIX6QlFD}~fY3othxU!^F z_jz?j$rt&i?=|CDqHnf!)4A?HwwJy=@HzFb@Lt-2(mxLCimQ*!bNI8SdgA}m{w3it zC0pNG{$5_pa_wWfZ`H%g`Rga$?Emw*u-+!SD^%yh&!d;m@kegVR<07Q;#xF4w(xbj zd|g%6k#vcpivGu>A1?lJv~xM{X&cM=H(JjwFG~(u|5|A!`?1 zxbRE2Yy6rOHy+wYoS!w%{n3=!uZ~6Vh3(t6<7AhlzuwKvxSt zuMz*Cc6^DVaQY+PYgQLe&UWcscKgV>U25B0H~5wP$oeJ5VVSh}!QpL=Ctn=k3Z8FM zy>)%i4x6<3=ihw1*=auK&jV>@pC!k*<@x#M7ER!lOiEThyncgw&d0Pl?R%0G{M*EL z7VLg7*;i`QC!YLcjM*Pvp83(<%e~p+!WY9wi>&XzdtUwG!Bpw3Oy0b!BffM;@2>ce zby?Lo{wd$Tu8DfKOQtzndD(6Gc-r3Pqsz_phH3F~Kdv3~uix5#oc+>reKs>McAc4* zD!PUBs}0rKC-1X3vQl2YY-eUq&dUp`&R@RUR&27h5Z+!t=elHt#hT*ef@?QVzm3hU zsg8=y<==k)WvSV%=eIQEDxaVHDD!Z0=Z!n_&(G3Z@W?gk+4_x^lec_)IpJx2?X`~8 z#{G7S`or_CaIHDEviHl)r`s#OWT$Vp&*pn4`E<>m*QcjOt^08G+J#xs#U5=h=5I^y z{b)O-xK(ZuIUw)d;#pG~-D^QJA=KEr3PV)*%rcHhdy+Rq;;G_SvVUG7N#iGa#u z?e6lD{>gpq@v~$6uGTy%*D$v~c5t>lQ^H5bc->{9`wb69K3`RqtQh`cbNG+eWFV| z`H%kFr2Z)Mc)t9@!zqu1S9yMu^-aarZ0eh^Ie#qVKl}R6pSSVR z<<-xEbLR4wyzFf5nfuN8;~GKkNAW$Mbn{C0e7ZQ}%k`bB?nZ>w&fv2DAX2pDnNd&C zDf``Dy43Ig-ux`(``-2}{&So*506{M9?<1Tag3aDU)sWnn!kSB`f%xB z`t{W29@)AbHIv2N_kR(a*BW=*$=zU!EK7=+P4$l4bv2Vxr~7R@d`|!NlbV#^HAi>9 zI+)1J-g&g*`F-Z;>XuT4_Y6Ndwd+(%_{&wW4VV&vne^X)$sIjvhiLDS&M+D6G6KMn{x7nOdz_%d7fWox)~?Ug@W)8{(zuax%ZpL9$$t;kgCTT-UatC|bZk1x0r&W>A8Gd= z3g%zs*SCM9>HkGQQm(o*S*3sd-H4AOWxcH+uWdMa@J=gm+ z^6{DvGHtH?ckkL}-?*T8E$Kq|m%{<&*F^c_(vzx=x*xab(Tv-X8Z2iE51Ze zzV+tR>8i3rfiDzHWh8!YPOHCm#?G6L$4V#uPj$OF=?7oc;HXe4q zdh?aJSnjXvx4LFOd(tI~>k@YT*|oqeh55-D8y@SA2hQd{?z%qb?EVWzUtWCnzSsD? zAmx5E=f4{VAFbc$_rIm6u!B*%>2Bkcdb3k9&lSZa51p6Oud#Vsw2a@pw$S>+k&g9! zFJ82>Zr{`wYjf1aJjX6^ljio6UnxnEg)bg;vrl}T9vxHTdh5x%!}5osk}1Xx_GkRr(e(9DlP7Pf7WF6@c7Kxr)?#oCg@%Y@yg=f z&pUY@OZIBrxb7QE6!vJ?N!rbMJX=fJ?&Zgq3pPl+cxElRZ=2M{=DSIOiyvhtKDd}% zU2gV8T0XWe@G+HZh1 z^wWO^Yu^j_5Y5BqIpfzKbGBW=p!oMcS~0Dp03NYDQWg+UHj&K_iGi0Pp#WG zi|_WWtBGeG@Y#Rbd91Vj!-LKJCr(-0|GUcgHB|A}qc@jq|A!_$;dDz57BBx%aa59D zug*no-$seWzkcl4B&hxOT69&#tyk`!t!#Fqe z)|0v1U%t#$=UTM-#pTm-=V#uZ=|0^@M!oi0iQ49?=i>MI|5_uct;3RRd-!_cg!!j8 zFEc!R#qRz*(`=ih9KY&o6J{?zcF*F)N&n)!oRSyECoBKyu&J^5z4_ULJf;3mfq73~ z$9zkuwDqjL#$6$NErn}c@2YF&Ctt3N)=`^&#^J?8&DRfJnCuYa-g#DJ+2MsposKO# zE1TBb=ik33Ws_q?;>xc-MT-ry+EuKm-2I0wbZkWYdjheH-0lddxq`7FgP^Y>W7yAQL@7g;ve zUXU+5EAPWsd~n~6Lt@QBcEMI3uN=>P@ksNSmYNM~=;q$8+c!Rb?LFAc+{wQ2E&Kd^ zZq8wf*Yji#Ts$7$B`RhYzgl=2{g9#MW+l z#`Ay6JI9UhORS6%bQjgmy>EYh*0++%w>2C0e-zo7_y6{9`4gLuRkCjW+-$$3-p(fb z$g>z>|D6G^mEV?FNnbZS!kxS|^G$kl*IrVR3@dWmr8}gs{NSmggt+u>8eS4ApRDVzbvg4$SuhMd}mHW0Ou!a=q)#T{3 ztvI^C-#=5}l~_WzQ}3S0FV_UajHb_fnfxR4zTKPHh9ir&Ts+*+FLk`?#Dt5dx6Qcz zV@vd$Cwg6fSMHo-FWbX!@yYNj-``FA)Aj9M%5LR9Zf4jP^QR&P+F4Iy;6seGfinQhGo<>^BZu@n1>PN-e=Q5JBb3WeGFj%zNHv7nA`LYim z0zSv^Y}ooz;orxA=hh&{>w_G>MLKxfhZiryKfCfjTNTguV2zQ)%@sej${xG+?zWp} zc)2-z_eZm~Kd+Zdi^*BrN!!Or{7hW@B0Ma<rK~^j{-bZ2A487zM8#3Eq7n9j9E_g=NWvntCj6$KX|mL zU;Wv!32TmSJ$6gq=2QRr3*yXj(pM}_9G&WOmS1W0vi0wG->S*Dd{keqmOH3-`>pnq z|9c!0KDNMqV3r~F5J_gCx;H0PVal@$MG_1iDI-yd8obN1N3LM@|&w^7%h?>hH( z<+o+Fm;P#eVe`LZDjL}Z_f;6&Az+Ue44nw8`Q-Um*O|>VE!*{4 zB0-cp@^xiXKmV>LpN`k>yV}7MKX*R+=g7_P4}QPjTpZ;0q453k6AYUed~~uHws$UU5@*|8p?7c7zGwZt`(nObk2`Ye-ou_ho7{Hh z@FaFLu4@Xf+j?3x=a0dOBfQ+1?ga^5lJn2@ME5kC@7FT^VIXqXi zUVheZZtkA~n>|0mZ>+UQTq!6iJL8S^kCH z9nflh`TNMnbKdPKSZ5JDhxc6TxwdnfTld#Bz31`tn;&~#s7n6CX0K>Aew0Zv52Ze|n9-`AKi;-3uizCM|sPBy@Ic(Vib!TkNyX?YWTEJ0;k3Qgp6m z_Pi?`>z_YKne*WB8u5Y$tci|43#2uEz?W%uN42 zN(o*UK2zqT{37H1T!y#G<@EX&=a-L`Y54B7Om|)$@$=I+){FUlA4R{N(tTX?uGe<%kF4b8$l0}@ zIIZl`c-kx8h^myaB;S(x6ua-}blJI&E0d#hK0Hy9{qspB{nt@m?fx3oSPfza^cYM@*c@L-n zv8;L%TIi<-QYTJL_y-U<^3<*vKJ*BejKdxbKQBj*}Sn+?BZ|zoN?&m z!t={4UTS*yt~)3G=E==VSMwM%KCj>Z>RS6lZH3yi%CR-u9yb?nnEXujtH64TYt9?C zAA32y`H?2uUo#zh;c|(O*&8*@?^QfL;~J4w_Dt4qMo#Tn4*!RfePV8Xym{WRch>x$ zQ+p2=n#Fz6luk}r-us|+vD@0!ze>84?Td4cnR~rfko&u0Yhg^k%=)O=`)5}FIlQ%% zy?@W1O^w~N*tc4|m@56L=0M>J^W`UBw5{h2ufN~?Jr`QL+zS6~wLky$ot=;7OmI(M zX=4@tq2E6H!wc8Bp>MVZJ{Pimd(Pa*s^&s?`_lU}Pq;^v`2AXwc$#h7BL4q5=cNii z9j`CVt!!xRf6%(#t^E5MO+E{c*q!_5n-&}vnJn?~l=lC#uAdjAIs5P4_(J*b-^kAg z?tf3)kyX?=_6GtvL zr3620&pFE5tTX4)sql~KtHWN1UgpWEIiPlW`nDZcPHqvrf9v6$(jA^%sampiKi-uW zZroXL!*B1yeja~$-@O<6m+P6=_Q%YRv)YFs^dV{fIB0Tc@9w6FYjYe3QYdZhN1P4^j{3<@Pu9 z*IhLKfAaJ8q$Aq&tCneN>mqxo&-?!z(Z-phRD)oy)%aPZ!YlQ-=B zT-m;r);)WZd&r9YZT0g>mv&BCQ~d77=C-)!UORcG>&X2+mp0?!X4fyDVr3GmtJlcN z*;?IvaQ}LG?~JZM>4RnGOrJekc-;T-3N>+NEwi^%!{sh#>iqh7=ydvXOUbM?n~m)* z1&i0uc&awLDN{t|6+}J0bkJ>%}d)4oLn?DU2skF3A^Y|8^UVLbQN~#p3r+!QMc;5xJ}J}F*h?+ zw-_<8uI*|ocO2QGS5#|c!|L}pblKVbZGR5%hx@woeAyEI|MkI!{%02L=NXGi8Xn00 zl*rxDoBc~_#|mGQjcn^9PTn}|_@iU(jxNo)pTf>QuQvvo(0 zdGr7OaVvMO|NP{;yK1rz&##P5;tl6}pk%1H>t}lZm#6JIQI$3AGwt?o?e?udx_@`! zf!w?o^OxCb+?(s?&UUS|V&9r#!=H&^Hg9{+b)NIN+3UW)?DzS9nfi}kuKp-u@ily= z+5VfKnCa#d&G?UQKK3)G z=1=!kiR%&FKeyOqAE}r6_8?}4`X7tetap-(61!Zs9Pn9p$t`s9!a(;L=!fk%sl^Y*k ztp9o8*kk=adQN{PY!=>aReK|G?T=eOK5b08l%lU)lf6&9NxFR(bj@A6wA{r^b;m;Bf7cz8|iH2?n^%_rHu z&kDLWU4FmT+3fYK%-&1d{uL7|4htQZvbHt+;TnG9i-=2Dw7uou4qkqon&Ce$8V?X`Y z^Of=Tj_M6Yjg#_?0;GlSt6Vh??2xnDX3XlX{@=@#+3&J;9}{P*6tDjH=%HBw4PtX7R)4R{P@|ek|z6U1fjxFPl-ur4pw0y#98NYp-GdFK+bYK4Qa$r*HgQv`LYE;6COlteS9kDsX%VWi1Z&O)Bt^aMjPvX}0itL(~6YNFK&wBJ~kNECI z{afaid#w4?*R!y&ZQ=dO?XNZ%pYP~vI_13Y@!k0IQgw-+gL^7|ojY@FLd6f2&(EgH zyxhFnw||YPw;Q`(Rc|kEENg#n*^G)?FE@mrlHd1_`Qw_0)BhJ;t~k zp?t=Nhk3$zi$C$D=0s_{>3iZVe)W3X(_bHM3;Zhhc%&o$&gJV5#D#CpGA>K`c*h~euJ3rcOx`Xg;>W9ks|M=t1CBjAc&qv7_Uq5<+2r(I@3B>Q%zw=)e}Cz+7mL5` z{`L;KgoA3X;SQ*lja_i-R&=;!R%k`f9{ zpC{ty)8+q^zeXnic^0K7^3y#1)76-%|39VOo1MMqnb`gswRTSa__N}N+_n9yWBp4m z&p-8evE`9=$#qi8tR!c%XY-5Q_k6WE?m+aGI;rV%RxI)Q^HF7Lg=wT`?x!%BlW|{T zIW}9{IsT}4RiYxDx3%Jnat&+z+49Lt_}L$7K5~6{;o$elA70JpiwxoU^>Fob*7Baf ze*!(G(-zv4zSz}c71PzI7t?C9`^QnU|Eo@REVZ*g%HVxl`qS4*zRUk>_HR`DarKSV z{HTZFxjuVa`^+E9Kkj)lGq2cD-{^VG!I#s$UtY|#&1je1|Kq`fg=XUai)ik` zRq`|}=9rk7-J|I|Z1y(~9yRw*(w+a|y5X_q^L?|`*6bAHtNj?+^QE^r{_Tb+?%#5I z-d6|AE4PxBtk`h9^7o#1bSe)f!m(* zO&=v3cHg%((e<@ukGuXNcIU|F6ZmHCO7i*m(!Ijt+>bE16Y0D)8>jhy-qdGUw{gPO zw63i`PG7d#e#~*6d}(&#{pRnr{M+LEg62ma{hn|&&A3r5>u!abxco%ZlfL_oCKO%> zUi_H*&XM%hZZ##Ai}M2?h%(F0_!3?40MWo`$rFn%k6%&R6orsrK|3zRD=vqGi+~G?8bByohmn$(}`|7;$+*{`KX?rh4 zCi1VX&bj+IX-V}y+iX4St>tC!RIHs|X7~2%@!^1P^`bD-D!NU@b0tT^&NKqKBn_lR=u8a_(=A;Ez-ZG#P62!l#5G8|NE@Y z{^is{iLQy~Y;Cg>W_bU4DzHsLfamQo6}`8gGx^UsKt`<^rI#?82l{rktnqxfyCrja zz}u-WCd^cc(5SiHbVs#+zrhJnvx*XtJuB;X2;9e5GpBiU&650}hwdp~XV+LR zT>ax@acs`dj_1=K#AIFOUmbHuMOxn7D&@ydzRB^@ylgw|b|oJ zb7=dAt%+@``;V}`Ugw&x?HB8L;>5+}PKW#pcC z+FAOlgp{!LuV2F74gAbLZWX+nbS>`ZlP`}iUJeVp`{mR1ADg`6-aK`UeYf@G)7JBU z4kx#p zr}y6JT>Il_-=nq3-t9d5ZA`+73(L<-6`6J1?OJ=jukX&y>AxP9-|Ji*Ggu zU+es>le5)#s)zgCmNS_*`9h@rgE}tVld4NvwtD317uP6we0WjS{p-gc&M&_eENi~8Wcjl8BkTH4C7Z_m z)SX|F_)0vwD&=hKr>lK`ChlN=n!fzAJv0B8&C86B+GWH)xViePwfC*#scUWh^-?}y*NmH0$oy2j$o<4)7R=Y;=RCBK*~&42G>@uC}7Qyz-z>ljJY zoOHE3{5<&N{kyi=XBKvw`#jOLcxA+EEr0B4Qx|FkCa&%#xb!bkAb141D)4GT83q<>S>m-Wa!|)V8#%%+a%}-J%vXNp<#~_qw}TlqNfhe=TK~$WwMp@Tckf;cFfxJsKdRVXPKetcn>4zXc_LZ?;t{cuWpC0fnSLp4r^J?`^1<%{;;*p~c1xZ;u@Z?-vG$aD8vrPa}}xxT-C z6mU1avn{h{b?%Oz6-(YaKED65B;`O= z9{;`huVuvEoLl^D>6#DI{i}D{-kIqb-q`ry#qIoxFIV22cyszk$%|W)zs3G7c|Cda zTb;nouVo%h*PHvOBV5Pe{MX3I@pS_Kx2h-Cd=NI1s|n%x_2k#fgHw~Y9bMVBw=Zzt z`_r4*#N>}Mg~!jEb^F}WXN~$+T-6^p+1te>bKkGHaWG)mN=?jh-vNO|4t+lK#At7JnKp&$#hwaZ>9+zW6nl-ydiymzk~e-ymwj58?BVq{@DI z#3riu?X#cDXT8ZIUQt|H`eW#a$*Ye#TzjbbuH?|(+JqR3Hzx!vV@i|F`9CZ>+u- z-_|HPuk|&({q5h+Z);x8iF$lh@7=fCR%*$H%RJ%}i|cK(f3#kgTqS!x?t1)u)2BOZ z#DmYXZ8WPt+QmM3Ez|bWnSAyuKCABe@zj0JQHNu8He2m1mGWaI7oJ${J3nSmN1ctu z%}0ktocFZ9-u$fQcJtY*;HH`Hx=StB^9rA>VO_fIVN`@}%bTk!y4`-xyf>C-tKD1#HX`=a-(*>wmXRGjC(Z z*7{|q`R#m+T4k&AyDR71Z@1s_H2)QEeQoU0n2jIP?qB!)r0>7))}ytBg&n&qelA;m zqIl9?JDgs4B~Rz*p*Ekr?++S_&-=9b`tx$OHFvL-W#61-FMquDcse1F`ZsiTVX~&)x#q-_V?c(0++Iu78{0#QI zeK(cAZML0uYO-GKp>JoF*PSq{TXXZ?w|&{$vhyphU#vOJIj7*Wy&+qeWd_fyPpbcK zl?X((8|`EApBpom-|p#Rd5vY4ei+QZd2*qd+}|Hte?GbYCrkO?iR(8$-h6cVbi1L+ z^tDePEt78D{{GFBNN{SN!Cc-etY*E1KEuqbq8D z-k@6Anhn@r<3e7V`-DqR|Q?U;D}>q)$eR_~o=aOmOoLjT1d zC+@2=vN02_*>h}CuhiO|8hrb*Ca2%NakI@^KkI$yT7Zm9x2Ya_4K7hS>4B} zzmxgJj>paK<9YM?&zZQo4f9RUZThyN_SG4e@88yZv^QUT`dnQ2mbdY5zZvD!ij?hB zE3R2{e%AW`clvMb`*l=)V#c-dJC)bV?)Gla+5bDeeq-(T)p7gxKJ&kC`jK<4X2H*& zZ=TnG_|{whHhKGL{hb-re@)-tocDdp@!s^auD8$seRzA@`#&r1-(6Sv-q*dblTmcL z)HQpVfB(O6ub;KaE^pV{zPGaqC)Qt;zPIo1>iDX^Wp{bE`^-P5`}D`=WyO>B-bh&e zaqHBS?0pF@FTaghy4^ne-2316=9~d%*E~fRF~9BwXEQf0_;P7u#gvpUw+a$pd=EDH zZQzz%_cckXpMRTf*tRbT$BP3pw-WcjqOzd(opCli5ygqkuc7C0^v|P}4 zf8{Sfbi_XWDsuK)mSLXu@4=d#7uA-lvsvT@?MV5xdi{qFO5Zwa>$U_YFF(IOs?*=S ze(Tle+kf2L`|0fdO@AMU|KY!KoS&sUH$1NBXxY`t_BFpZiPwDmwrT%I`@H)1>iY|; zuSUn!PT@BD|1anaxI^}DNnAGiOIl{sI3{<=M19=+Y^U;XD>>ijPcZ}0!Ld;ZS9d;dQVy{*6h z->jL|mQyyb|M{i=#=dWx?TUUclmFNIR&;&s>xyf4HpYH?(Y?L@_XYLb_}@3>ch)|i zUjOM^=>O~be$t<_OvniG@efX`rzU1t;Ui%-f-Y!!7-t?@qe`m?>6U*%rfA4Di z|A)Ws(6{(Muj6NJd&U>`ddbh0Xz9GFTPd%S9tbZjuQ@y6;@pk4XXcp}Z3(hGVC`F7 zAbWV{Hh#ZZ)fcD9wD)yR=jmqf-!H}gGCJiIUjR3+zpsDM!_3FZ;+dO#jfh+0V%O-)}9avR4}Ovnt-4{CYIC=GKp6 zifwN$$8OwnQ}6hvXHA{$4<2vKzkKM}>`#aHe2BK$SG~6PQ}nvWuMc;B_`B07{;YC-j1^Bueaki*B(B-zx4M+`#&q+&aeAse^YPQ zgERa*)8k8SehZcVoqF5;|I`1azfWG@n{?}Q_35|U^Y(vA-@os#x&2?=+neS7pLx69 z?)y^Xc{QeT8$YbQQ*zn&^Y70U@gL@#zZdtfEq>0^&+|V=8rScOc0YgX%CzIxM5RA! zKVzAnSNTNTwEX3==l>AK;QH-GE&UWx40Cwn8;o^PM@|5|(H$-Mi` z?AgwCx57W^_Mc5I-UO;k&fIO@d+zPs$LFPfpSrhs-Gr}S8f~)QB+e*UbNL+KjpKjI z{U$6t*Prt)bME8$r#By)acZNXm9q45DSw`%u+QD?r@9M8&hI+^Lb)j9&94tif6k@v zNlL%}(|%5W`caF|`Lc%h4CnbKYuz)v7xVwt?0Y-^E`I-e>s$A@&(Ge@uRE!pBhDNC zsJZf|YX7Z$uUyaH{rA)V@40W2_kVqsdH%ib_x-;^Z>!t9_PiY`|78aI zlgP)r&zyBlzAS$I@W;sF8vYsm*P^n8-`Cu4zIGnmh8AMD!E|B!MOBuqFLg@xn60h* zRpDipWi+KvD*oYNmXK$R25)|UF8-CGm-6OX)BbOXFCSa$NcC47(%yDby702*u{S3l zv?SJ4@bO!n{Fy0JlDq6rx3x|3^{_t%0s9{8a8NF_FIiK`DgW-*;U>H4$42f+!3pBZ zcZx#f(s_#Pzlf-Odvw+9L*T|2*~`DnuKzRtS@g^EKR<~3+kVpjzheK!z2B#O&$a); zZf{q*{K1FjKf9vu?fv^`cKp`8KMvdd)G$xdGqZa1KInAGnq_Z4A7lL^nlB^%A<*vm zVf)(Mk1lDeX<4WI;WCO_`g7KqkBri${9CW4Jgt0|*R$DDE&t!uPtNn_SkIqtpAW?$=B)|9a;2hWzs;_RoLI z&1t?1no2u2y)vnTW)^iTIy-An+^WwL~znph^Gh5hm zCYdsi+%-oR$9+7yIc`eV{))c@KNW^ z8()q-jST+tR8i+o_Pz4c-`2g(zd7&Uj?-$le~6~+;j+CUn8V_~a;;jn+{rCB)0Nd~ zxBhwX`tsWOdN!LrTzS2@X^yPylzIDhbocTs`x2EOckh@Xd-b7yW9=t$JgkZZeioOHUDi`L z=ovJ*VSe2~&Rq%p{F7dHS{nT7Gv-;smw(hF?5sV{uA>Ld^Bs_)N{s{VWITzp+u;l56JvzI5-WdG>BKBl;AwtVUJZ^G;AkAAzf+Ag{J=j^(| zSGBj=>n?qJT>s_MN7=W{zO3S56~FX`=8&p-f+#&hUE(TPFHS}xtbiamUGuN(YqV6CaHeadhj={{1*oIc`VI z$JM<1Ew-MND&J`kw(HkibKB?-53Z%JmY--|w&%;%pWOcT_g}U+wfERH)lc~Eo@@Wn zx_;N*FN^2x{QKs8yzR!gFL##b*?)_!w|bJk&&;d7_hst(KZie>PCM?q#_orX+&`t7 z6rQ8Y*Vo>xxLxFI2@MV#{nA!%vY?Ig$*q!)v{o9;n zpB^^lD?Zv9@#15);L5}guOD5PPOkYSC|mZ3e|c-<+tq3*$4k}B{scZ)S8FfWwRlg5 zdF8&VMj~wW zk4hg8&e-LA{JOtX@Gs{~v!fpIf|E~gH``ZOA-w+Y?#I&&?SD1f6+N0{|G7|p)8B8V z>utM^|9DX<@!`Mv`?}MA@^)B!S-bMlrgI;TUX5LDd>QAKe^_wj> zF86OTZY#KT{_0*&n|01{jnUD68*Q?WaG$q$AX2>XP<(RXy34BHMJJw<|Eb}g^Xjs5 z)~*hlsGl{P4==vVFPgYvVe=FJ+x4G+mREdxBz?T*U(dJqe{PoEcDH@p>Hhia8THtm zf4@cZ)fooc{Mk^sKmF|1>-qZiZ*TWK`#Eo(>8GY6>R}&`{PsUqej$JF-*a!bhVTEa zaqU>>yld&a@<%UMTz#Z<|B=r9KTp=rkua~@R5a5*Id%PfUU@D%>s+JOx~iP8`^QUP z+6tVX^+$Yr&FN+f>GnSt4rli~e=R5S-}B{~`~$vwSQJmb$k%SdkXQ&w}d-&s4lJtVl zZ}&+2a6TGWCnYRdQ`;K%F?;umZ_9uESTfnS?ANhp3Qwvvd}SXmdAz>vL-ofOlVv^J z*Rl8e@X7W+t8{WW~^%KlnjYb^YEbb9{$uW$Ts z|GO;zmz~|PcjMnPJ9_HRPyhG+cCy~iqU!T!-`h$Re?0hLwb5t8Ul*@Des3`En&j?( zu5It6-e0_BAG7^vq+O4u-0n9YMZUe9aGG??AH? z&p)jxl-t(-wsnQ=wt4w4Z|48_CO!N2uQi45O3Oap{&~nhT}l4etJ3=G_S4gOW2%y# z@^8HUUM9mQH;%m$ke6JCg0}FSSuYe4`m#e9nUu<$K4jSxge$Jd5v9{6X!jo7b&_odr;z zVs^*Z{GNsWo|k9N>bEU?eb(szt()gQp1yuycZy`PJ;T~Usl%(}r`<|({y59~Te(ae zQ|yu6z3XS4SwFWY;z*ZNyydaoZ~Hzvpa1b%MfuU&kJDxU1mEx7e$U0E@Q8~&SN5@; zPpV$=+yBsr*O$DnU;q2Z(eu~$e&(I5u3umBR{GfYPxtoKK5MUg_3iLuamU!W--?UA zvNy;7%;o)&c$hVsr=mvVXn1VjjczQdB@ZV(56?ZfyQO{O zu_Mb&Sr%7H`z^npbdEjRVy$ASUG(mfo?f1n(#5}i{ZV}T>!)&d&-2T2b0E`Q&o`Z0 zd;D5T^QLcdU8VPm5C0Qt-n-yk(~WH^ZEn48 z>G&+N_;JOx8;hTCdC^c_E$s4P_tyxJLx&4o>Yp_IQpN`-cxd0zSp-vPtBuovQcr z*0JUr2NRvm<$wC*YxVaZbDI~x^=sh2!x70s-q{>(=cVkXCOvDPqdVux%Lm)bw)R4U zXVZ%lN1JXa{C#v;U%Xa#39p;|-XtOaUpZf&E{pg%dt-l1#`X4- zeui#0^fG@XKF-{Hv2EtwS?`uUKIH2j^XE>*VcyS9I@f=SUUzLT=2efb`KuBC=7Xrs zmmS{nyYJTg{;}#zYBHVRw@5GH~Ze#x}D2TyCvq!9r?LM+b*N}WU<}f4*&Dz zyVuvfH!c1XUvbrYn|?4mzwMNb z0((AqncumSH$iRY7=gb_o6pIFaFEEBV=-}3tY-ObCmXZ+|5zH=+*+Q*3xCAV{%&z)y#-jmK7 zKesjOO3dMtV)6eUgxJpb{pfIb!s6qx|6Vk|Pwd}6?bEu~$K&}V|17up_UY~1>+&&S zH}FQWVZZ+rXdb)H1MRLCeR)Z9-$owJKQASb)S$`7dUac0_h!?@_pP#Ze)V>~m3mxw zw`OU4-YM4Vj}y)Hk1CthNF1G*U$eyj|HM7)=}LY-Kr0VfehcU5$*Y}anNqU#h`D5n zT3D}7#I7G)hq4_XY~&B$-K`cox$DK`g$AG9mmgM+G~C*;>3X?=LT(M~!|dbjM~_R* z-oAP}|GpTR-p2;)Z!VdyOA4I4@X?{!e->VkzEP4O;a~RS&8;Utw5OF-)JaI33f^D1 zYlHgy8Hbl|zqOj*ZnpfsYu~n5_B5vdt+^EZr1>tl-HV^MrT;%VJNfygs<*xGZ_WF7 zV|jjk!Q0pB>kZS+KWkoJ^Xl88kEdsWI%S6c-lzKCu=^@G-(cgi`G!ybKh)iC`T6^o ziKe=M&*^X>Qhio4(D*WL;DkyYQi-@Wfg-78Ul%U`EXr}XhxHl+5fWv&&!%V_ko zMB4b~7uyQ=s>Qlx-()9zxopa(y8D{J?2;(yqR@LkdOrw8W>4C?>HWdxV)s7(P3))N z#?AjPxxv3??D;#c0Pf8BNb#2aDLP4Smvzl+Juv~A&6j(Ph{lgFyVPLj*GzC>D|Wy`HmR?JaxV+uqya zefH@;VjtI^2wwN%{^iZfX7tLMrWI;>`0{to-QWHQwD#Tjywv9_dsbLWuVF7vvAh1& zM*TgDeEQyo-+3EcFX;a))S4i+B+5}&tcZ2uR?E%C8qAp+A0ECM-hOby#)6zNR0_iBI`+N2I-jv@n&Hw!Q;;Fy?X5CTu`jg-GsN1LiZoL1061wOHcG@{P6sr~jRQTioXR=I!r)t$jb+rsiMR{9_wG z^+fKk>D*mo!d~#AlhJ?eh0oUUe4RghuB`p7mz&SdlZ$@Bu4(mb%KKXWd)sX- zChr6-I;$wYRkw}*missN-@ekNdUE%@=LY{Md27Dyo4}#=)$95Cd!Rv7%w+$&=B2V> z%fj>GvfJiz{x~YcXLe-x3fi`aVek*M07PZ{|Cz2|IKBTFgWcYug<&sj13+t^@6UNx{bAo* z$Iq2PRyR7oCtYm{{;lpLUf6wUk<;sxtpO)=j9O*)xwd}DI^Sz17FPC4*U#n(+g66D z8$OEucqz&)^QY7N)|;+{zG;S=&-2F=uANbF^W#Eay<v>YPC0hZP@oI`QNi|J;!Zpe=ofL@5Ytm|L5KRTYj5+{?E?a-1BOl)oY(= z<9{mI|M$UL`(G#Y@6=W7`8%(UKlSH_%R3}))Li@=x9jf>`G0TD?A>Q`LOr(l`{n!} zKMNV_PaZiL7dO!^>)+|bpE~n?mM#($uibTQZEViduKMf+dp^$ZsjIQOsbuNpKc7$O zdY;c@^Iv-R&#KFAd{_M;rMf_^;=hHcX-zwA3Hh)|*QN3=i zV%O)E`oa#+Xx(Ym)z>7N_mscOSbT2nIWviuC3#P!?JJ)v$X)7Re!s^0vG$Y6$2Q)a z@#W#`!@q9p*S!1IqaJ6ecPW2P#gF@+wd?on{jh3zo_@uz$NX>jbAJT9s6Tn!KKpn1 zi+Ur+9X=V{pHKcj!$m&a{AA&H~TGTQyiWfd!qZ#>e=#@&M&8YXMVS{e%@E!3cp(O zAL~Ba@4IMTsdlz0J8;$xuA8aTW6esrTV;1P^y@QNJt@_d^P<*&4hzT`j@Z$u=GuDex9|V==luP+PY>sBto?kx_V%}RAMLL{Ja6;oV`=@S+W*=0CBLV~|Iy#} z|6BUL((14C|9?Bf`qSO!qvdV;-*fVB)qP#Nf8*aPuj}7s?D-?EzWh?=)KLfPRhPhv}A_z z{lC$MFS}X8W(Qcu6@9zaWf?T?wHp`upD;i^S>aMbD>6-Mze7t!75Si^G@RM>fZ6itF7l_!wAsDB@?a z4Db{eNy%@_h3%+4r;a zxBc&5U-$7F@BeS%+r$5#+^Q_j9^d)18;?;Fsabo{#@M2VKl*+O%WvQNqtgHGzE`c`Xa6g!|1ILlK6=$HDf>== zh~1SUrS!9oJ9q4#XIk(>%k87s;%~OL+23q`Jg~_=SH+lrI7(-FpZC_YOL84&s!YE5 z#hF0j1K7F5VNnPkMel`nxB6Z$b6d`+qr?P0o|8|N3O^~wto&W!Sd(o|TGtT^9)=6Rg+CP7u zo|(S)S@rgM+kHks_ZquGb!yr_{(Stu=I5%9rPE&LRek@G-FI~Ry8iN*`nTU^KW_Tp zdj9vExAi}_PM3Xp{MV=O!c&Y5bqNod`TiH5dECD6lYjj`{TJ>!Pu2>W-4uPzlT~ve zaebTnx;&e#|3*!xtamQzKQq}bY5jzcJKrBP&b~EAVdu`}wicq_`hzDnevN$0dXD+y znvZNd{$81Rd0+JTN#~e5mHm&)UCFDTcY5=(;%49HY&x3<4p{nKjSx$dKplX}+K z3M`vt6T72k@noCqgbw+6huG&l-q{qr=aYB;+DjETI@~qQa*OP`^zWtH{BPbK|MQtX zo9)T__v)Vi&bt-&`C0b%^?#<@Z~y!62N&nvb5Y;!{}bN7b?@6Z~RYxw%3!Ay6?Q(bMhtb zTmN^z|6P7teC6Yh&d%f!k{?r%hm?7rJpr1{PkRxvLM`!{R7<{o}|ah?UuGcD5Ie5gKO zeQ$H^J^ue|U!GA37VS9M$GZ-)6P%TmM==bi;Qx(Tg+7 zn%UQP8%xS;d}1CWo@w^$(JB9(P5Xsxk6!oQs6mByE=LP-dNjzIS;Oy#~9aj|2^~l)sI%ud3V-t zt9g08QY!cHQEm5EUau$l+n6@_UUSZPI{7K{{*6=TU**4hEdAf_b@dW&j_&&Qb858< zAHQgP-HkQ>CLccSmlK$r8+Y=B=1mdy*+nbnzMInjvuelNga5yC@6WD&6+C}eY~9bN z#rBe){ho)j#YC;!_v_W${C~5J>kA(>#ytNP`8z=EjePZ)%GYOif7?}lJ!j&^RzrKMKm$RsVd-tF_2^|DMkC5(4|odd^AwshK$cNn~dp`2aOHi+Cg}W6 zTIbsy6TkcCb@?;xU*$~Nt>^afL@egNVzvERV)TEmtn;`1pZxhQHcu~$e|f$I6q7N zM-o^3S@ZZ;$KqvPM;^|*`p~}C;?~3T{^e8TYi4k)zZ9`4c>mvDx3_KYUq5HU%g?cE zicVd-aq`sWvcI>!2*~8GeA@hGVa2r%y7Ik?|E!Q+eq+ng+C8~}{c_izmA3Hm-3wtt>@ib5&a?SNZiHc$J72@dV1i! zkw^cUON>X_nXGG!3Ktn%^{Q+%E;cLqe3#WqrY8U5^Vwe--@nZ<$>l3M+ zJ?`v}Ro%PcrJ0(k(&BKPA0L+;`k~?eBk=Rd>#OwL{(g9>=ws0#bDw#F-M@Y8=hvKm z7I#miwpb;+d^`K>$$Rbg8yCc1-)a8hwcKU5s|9b}x7N09Su1(vU-tUBZRx$6*BHC~ zIZ>gN@~!owR#BkcrsT`_0=1vA^4Tu66-%@D@$<0u_JWfMt`9G8_xJeQ?DDe6x{=k| z`}5%CefINAe>Shz6`OMYm{|P%TLIliXKZ=7_*X^$7SLEM|F;J*9O^$TUP~8rpFC8= zxWVIgMD8}T$+PtKvA%X(ajbtiFZb3HA9qz`rC(0;n4UjnO)KB_8C}7=%g?gTw!K$a zQ=x4ppSx@;ueZ&clbV$svH z?f2}MbLnugWck6b!pnGr<7IjUWUk|hH{BgrPEZOnb)5&))?s;3iA@j!T;0cV`-}kNg ztY`hrF*{ybEbINlx94a5ejk~6eA(aUl8tOk?I5m8OV_Dcy*c>uVR7R#U(z3cSbXQn z>w{I-I(qfL?qh%dyEcFB>Eg`!Uzze8BWzxY6?89lU7!Ex#MaBI{r%hX4()V2#Tu`0zDRo;A0zh!KZ|=VhZc0Jm+h*^*q19; zdSi0=3-ieP@#*?%-yS`c=FdFb;>;f3xjfDJS%P!H3+1B!=@$;#NPOQ=dU9X!mCNT7 z^=|(;{JFFuCJ?5Uhz{@Y1@wt6Uuh#fBVrJ$!lCww4*oDI(A3WS`m9`n}07K zZ*~`#+4iw}ZJE`>*M}9a^Ub%hn^$p_YyRi<4GYh=>He)bBf*z^DR0iphYNEOo~)O+ zp|SexEaq?gPvlzmUw(O_!StuY+RxwqWyU}M#+SvPxqbhnlZ*eRtw}kS7ysz-!jc;o z4(gj%yba$}U3JYO-unNIX~{RZijFB%9PJEWbJFVlXWsQQ9=}{%kg#aOjR_YoAG3eC zd0D#K_ruqRPwZxhUff&ozJWO`aY{Pd%y{Mn&Cg>rb7V4NoXD+fJ3VNZ+N$jm zKPO-4SZ8NrvGw!mHlJLh3Fdozc_QsKFFV$BS=N{M^zCD^*4Zj?i=pK4!)ou&8!C&v zpZ6*DyKn#TbKd>uauLGGUtTWUa8!_Q-)W)P{Ou19ERfFJ{4!hS!=9O8-DiJX*_wFp zyZ5@02G<(5BseA!RM>t?$jz5e~H~^J=YIs$+T@nK>d-y(c=8-QdE@8G+n-L5n;8^SVQ?>>r~SvVOZyovXg`S<_WEL#5aQ1GkEx?y%dT>FpMTCnc}b*mya^Ix&DzjgcEio^Air{p7# z?B7>=IdbjYe@Tni=f&I3I6v#0#m6n%=Sn6CR@=ONRIc|$h%D1*H+!?O}CPf&k>ZI|D@!nfOoM=--#)Yjx=xVQ;fB`oBd)+ajt9M zCT+V44mQi4qOS0Tlh^JzEj}-3e%;1T2lb8Ct@s_6{OtR~9qiKY!HcI=MLkQnFSAU^ z5msE-F(+cvAX}eSOX5H}_G%`i-wAe_nl7ujjFS&wZ*aQH=x_a9;p!?QVTptUiK0mg3Y+wjoWcVP7A{awi*a>yas0ks=*+HV z>?OVu9N7`E#_PNy919$ccr1@Iwn?h_KmY&p>X+)}dH%2NnD4p&dCt03>-YV*wzBxu zymi&kL8`jRxP)jA@ptjDu;&8 zl2T_r&-r6mE$o&{Z)3)C%8}lXJhy3gXM<)c{Lt=F=nOf6VC}XuiM|n zSo`XWW7g_Jp0GpJA2*!?pn8EifmD6%@@tG9}|*X*yeo-Tqo6jN6xA; z`f=O79FMdO?@VtTRoi!9X0pY5(rA*_y1gN z7h62@`YCxykIxY`Zx*z_Nw}B2!QbbG>O7hHA78ZDy|#XS&0Qp5^T=K8xMSIbh5nU? z&;OpXJLh8iT$%8s2fsKD9-sH;<*yFeUl-h`nf+fmVP4XOy=|}c%bxbveX0J`!7BOd z1Y`Xb#UG!RSG-u<7yqicaQ#HzbBuq4m2HlC>3mvWBO;%=_FlGDB41N}e8y((0?Eq` z#tV2%a@itT^kt8BHt_tNvtZ7KDJQZfHt^Nlym0c6O5^#)H6SA6g zu8$PYA90FbqW?&7@lm_l8xC@I+&M`hb(zx6g3?BrGP_bIp8xF7zpY|Zx9`a$=0&la z;$IvHzw<@dnBC~o!F9hhF4x;}-Hu_IplDt)>8;YeXYcq8_e2ROsh?vNEIR2gXns5P zLo5F=5zAi^(zcz`)}3>#QCj?G!*Z)ytMm`;u@xr*l-l)5JNP+lbk*68wI1F6Ml=8D zv7PSn?Y}mt-#_9R`>#XXN$hvPr^~v{Yl{5m9+ki8{oF%P`ETYM?)j(W?T;`7o?@%Z z$j%FJ7u)AmG@Y+%i=|d2-!hlK%U20JxN|Ay#+Ax-J0pCbTNbk>$rT#t&;0ykk>v`@ zm6h#0Hn%1nZhLa%a9m#Zt^$`Z6}eBg?@ec&K9LY!wM*~db1BC8%~m!^&Tc<7_W$|S zsFVA#!`)Dcf4-SZ^tYp+AejAVGGEcF#Zz+cW}o1d%@JO2cjQBsBS-xQ*At5VcXZyp zIL@Bi+xp4Xeb)yO<6lR5b@{G#@>*BExiT^E&Wly?-Dyb;{% zDH<22${%HnFRc;Kk2o>y&$EvT&AN{!%a~0O-t+ImuFbj1w~Ce>RI=c^#lvti(1}Ie z&W2SxHpQdM&g%__O@)f`ye}7@KAF5u$gSV*kuuxQMZLU7m43fSX0PRtoLXt)xm`k` z;zs+1Mq4kw#$FwxANp2a79Q+z`vj)=dhs3AvHzej_r#6qeKjZLD_8h+%6mPTIj3^Y zuau75U!RwM(XI0q5yLvR-c**lTJu-Mns9-Ri9g z{^+cK!)cz7+>YSsz7<;(y$p`Bmu}s7+EKnLYw_$)dKDimGZw4Qzv1ft=gzW{iGeax zx_IqAS+ZSn-=6hcp>yUcCdPALtE|tP{-2pF@YA?m)A9hzrJD40htu=ARJ5gET=t(G z^jRW$(!{#@ccwQcyghq2`$Kd5I_+KC1mtas1U1ucT~YjWh`(spvy1964;)XmPO|&4 zYO%iM&1QA!D^r`FT$sxHOH#R)`QL@s6YcvR?zk?X{O`lr{w~4A>wbTzSsz(_|MCq3 zyS=gRd9TSFNsrIm%v~^djl?sS1!oxxCmp)na*3r~UqR#QvU8ovu_sUH<;w25@ZyL& zoAs87iflQ7UH!376rMb|>^9HJCD!l+J9p6A-%I0*Rw% zbU0QfSf6jT@(=1}2=3_$-l-|}RPp}}*S1N_wqAZ#ei~0S6cgQK?V@%p-{nh0+mkL$6Lty8i=21L@H}&6FKbPS#T5IpVx5(*nY4aR@&5k9 zTTc3eX6Zhm$**?Ca5kS|+tfAP>Z8Mpl_6_Byf~kqC&lbr_UVmA(!uFD3eEf}E#}*b zC3|NliI|;Ymi}^ZdaT#hm5-T)`ggs#w7ywlMRKsw3k~r<9Ks(}-8*G!a+2G@UOXOf z_TZBX^{bc~t1Ql2zMr|ea^0+a9-+Us?@e{q%}~Fz^vvs@HRD9YD}Myfc@gyf>|VETDcrG7mFC@g&}5UlV*PiiPt$y#E}!({<>}83amkGF zn@-J-DAHM(|00fAKK5j=I=ffw^poyiWBfioGn){)=SnBLlmF7BR`>tQ-kEOd)xX)$ z-e0xn`h@MrVk-hx`PJ7yOOTpl_wVK#ra8i$jmim3j}tzaHq4nY%i+Uk7n6w-L=(H6 zZAx5jTsyg5*vEmf!%yMs#W0^a9-nmW%7Sxd*3T1p%xwE)f!?EsOyN^LZ0L`Fqbc|5 zg~JCCd)3-Yi_bRp7&x=X9(9*?VteGN_3MaM?5z{dCoa0jbSajK)PLIZaA7;I*w@@v zzQ3QG<(J&qpjnynsqU!WnvdSGhKcQFN9^)!Ury&e`Rb%i)uz_IH&1?U&}d!1(ZR25 zMYdr-|4+yH^@>9KA32Ko9AvDPdD85&DgHz1^lxpiPKBGbEi>b&n=6CraY_s zZgcHxS5Rfpb18>?vp!FeIg{P-x*;d%lDh3JnG+nwB_($q!diHaY;`j9n7%D#;$>4m z!xL@mf+xB1OUGQkAlX;ld(Xnz=STa2fE^m*6*3bZo&Bts_d|;NNlLrb-y4@0`Tz3X z%{~zFZNtfk2IbNpo4l)?OWz!N-e3|Y#r`ElT=JdijjI(3?R@QTF08NS{pjIu^{F%V zOGdD`oYj1lcH3813)U-!2|ix_XNB{WQO*Xx(6^8pp6_|8kqkEpn7Zdjom*vR7lAh5mPru1l> ziI@87UmF+yIMLY}yl!#D)AmV|#g+1Zy6{JZT64`NaS-jB= zA-xYR{53BRd;i|E`|||eU%_jRx;D-65Qu|ByuCMEp$ z1$63HcOzWvX`Vq@(QLwp_RSsq;p_y2bBgSzvhd-%HC|PI!O%Flp~Q znHS-8r#x*+EL7J2c%Zntn75KcuCVyT_MD=ROwqmS+bVXo&e_o1?%T1eBy+jp`*ZC7 zpS-f%QMRsO>amWX=N3C=u3~09Ca~(+=Q))tUHP6X7SG(dcBfC>7v<$OGaql6c-`>E zcc~f!^#fto)qgJWukq@zP`G6^r9I`Pd*5rRk43tdzIs(=$Ys?XI(kP@m%&aqCfeBW9rds;Q+Tt&ual?! z$-{}=CAI4#yBD9+OjM0&QT;P0og+SQ(`8)9kbqT+3B%c+}U zxlzGS#V_NQ)k+P^qIqu^66%7w^kaW5J+Gj_=4!+;TygvG$ z#?|!}CuFN99h8f%RmeG@zU_+J{t5nN-1jSOK78`*n)l^pu->D~({f*JI@&5yx%Ki5 zk$;)Oo6l9hI$>D%N!jMXNjsyRwx15@|FufLApWnb)lj3whP}IZ@%mzexjO%XML9C~ zGWv}7o0Ly!cqYSdZj%4v@|>9bcVA}sZ#rz_GR;W+O+xtRDQ2JNRdN;9CgmMIpZfB8 zOk2NQ?#-8{_p-@H9eF7Hf5IQZ`5rT#MqFr*uaKF&>$=8#-@Dl#xZh3bVwA6xwHCPX zVpk_$O;V#iPp72aUn%`tUnZ}9G_7{(i3`)?qc^T){(s_NNB3zS!K-ShQI8_93atA2$McRvm6H+9SO9 zhjQev4Xg(z`~SEUUN5BeufsojW8k9rKb}cjRvymQePwn3$@D+2H?$v4efe22@TcLt z6Po!)x#Z)kiaCwG96Zkc>B>g_#*IaflicSYb9~$(V|F2Q-ItYp_FJc#*IM%1ZL5B! zUB6Q2kA~QjPUgy^{Qbjo7u*LuSamur%qN_Kk?r8XIyPH$uZz;Wb=x??}-{wX5yi?H| zx%A&jcg?i>@xx)BRqceLW6wP9XLq<8POi2qJH>71GXMF5TVM0)-vG~!!!{RQn%>MKJ%6?s#D3hC`%sqeC6U&pC zr_DFb+H%5S$}4p{vBDdR{N~&7c*GfQN}nUKp?%M%MS8Vf9I}2HUT>0w=YKh^S2U~V zhTo>UzkV#xy;&n~Vs^#7GxkYw%?9p%p~_b$PRgn7QvL5L+cf>tdpY$l3MHw+)mtos zbX@CtEdO0>UtV)@`dO9RZ`CDrlMbl;)!e^m-Hro3mfg$OnrEKrSl%(w>X(ebrXS~O zpH?a>SK6I??(*lIgB;Tu7i(A_FY&axW_ApS%U)J*GVwdoLE1jnt zSykx7?-XuUeaxxu%R~9vUsLseZCbfw^1(;^h6~hXUv+WM{xtdX2S@pj51Oq&UjI-W zUwQ9@M_uxFsT?h{ms)POCOlf^EB{@}>xV!M-_dSczORDw&&8%#9PoN>naeTz2=htv z(jPY_|Nj%$%=;*S=f&$7SO4U0*_;EX|8iO%U1N8&O|Ipy4gbm4F^l6jzP5293B!qA6X1vMC!Ok$mdkg*5R8Omh`DU0fWzL?mBvS+HK7Yx0S!udZ>?cbm_t zmOli(u`cn`YH*yt%X*s_W-~2Cu%jY{kURrnS%0cC+tvuT%PPJQ_lhAL*-eUjh!Gb>7 zloQ>b!~acO&VNX(h0o^814VAd?)smTY-*2a-|q^nRQ-L|Y{FFgU#;G@XB>;nL)n@B z+MYKJKT+iTeCNH-XJn4BAF;U-92NiJ8B8i={$NBR6~L8wKN_ z?@~409jr=u4}Iu~rKm@*p@XdYzK6$?1ctbNu*C`Dd@r3GeI|v=7u{&?-)FG=?8Nt< zr}L@SCaBov97xw`W!(30V~0LdSK-n1Q9|Z3f-kk(dx^dKF^TV6NAHZCoJ!1%%WE!$ zt?QEAeP#0eMeFKHDx&;2`-Kjzd-dX?x?S$irFu>7vzskC1?MG&#nygN?#}i4wNhBQ z_KKy*>_1Ob{|fw?aIDkno5hs&iYfD&Uq4f*n6ZkvaiP`u&y_RIPp`2%nEuXb)?z2$ z&r{5{O_(ZsHt??X1MfDwXI6DZ_dw%Lyn@Cbwb?6n*E4 zAMcYm<=C8)a;kOaCF=uaP8NP-k9>N3eN+&8vDdE?lZB#xG-XUywp5sXKYPM;38mFB zAHC;qSnwpk-s;BE7I)50cUkE#jvrdpb6+WaJ*e!L)Utd=Xt?~}O}evxDBAt{B=IzU zUr5(_ix%d;KOP78JqrBxEpGDh>9Z3p9|dQ|G1$S%^_B}pq;O1 z(@g&*cl7_>2>7$5)Ua{Ae(th=i_R_zi=W^S_Hl{V$M&O!i@WxyX_VGDb@~3AAZYhQ zL-NyPoxeYv(w#qk>T*vhO=$jmDyreJb47yk)6g%<{rHB^^Lo4ZmDzcjZzf*+VJvT2|I$6KXu;*rzZ&)a&PsO* zsLM=V9Q59_Yp%rSI|rPv3m7cy<6-}?Vt%0mzx9Vt)x1?-3KiV%FMDSy75wDl*<_)< z2U_(fJlbcdIP1Jn^0qHr=y%?4xXS-k_E5(1V*4_NH}GH_o!-Cyav3$5;?TjVsCky?Sd2wp`qiYYe{YpB!yV;Fol;-kW3XQwbqQW1) z@xpSe8owPee73r5sr@rPOg-(zUis-%@iQi;ycf#hpKjI$PT+r&C@z_znAGgP(q_V8 zwW1GCy}w;ttY?4XQdmrDP!vaU_>ZJ^=iHkfH94h^d;IGzcGs+M)|%M5p*#Okh_oc% zkFHkrmkHj{5la1YRhZ@Oikm)d%I&D*i2P-7g8x+NgvSl-I)(w~D_=OBa{l$9q|hzy z+r;1=Dek$uzj*rnd$6!ye(EO1djd6mOX8pSYF1vYJ~mfm_x~0A^N%R)zG}t(`A*FG z9F||z^Rmxf|6Fi2v zrXzBr=QKOM*vG|^ImH~#kCxB-e4%y9;x7-MChPw`@zgM@W~1}`CleN)ZM6RtdW!9* z;|I$J+}tM*FTQ8|Zpn?m?zKPGT(3TxCjNj&q*=M)pda%>M|Ou44bufSA6u?0d*Z+r zEl{TL=D_*apA_YcK1dXNnRvZZVfP1<6)#>cSro6%RjR8tVQH@ndudI=d^?WWAC6A0 z{kf>+!t#DI_Og`O%kS&hEN$N77xzYqT{Hcc!pDtk?`%9MobWee!XuY9yQv>8K7a6J zn(Z&eUkCjSJ=yNRI590nQ`GpYlfuap?FG(ml^3QzXA1hURKEU8@biYE`(HjPt`qR8 zP0*NRV>D;}(fK0Xp9JGLS3C7Df3qr?+4hex_m9Qyvj0`*Ke#iO`=3g;Y{w+#XqhV; zJ#A!Oxr^1_>Wa2`@{m*c)D(Uu)=vKZIUS*9DU)|7yvcsksebeAvCy|RVL6VQ+GTSb zR-JCI_%QjdlHZq!>%Ume5c)KKmBWJl&v%}CSY>zqA%(73F_dHm*yzZt0|Nkew{eR+==U)n~ znbgW3Sd`#xm~7ndzUjt-bvrID@3UF@C}_9b25h(4o{dhq?a_i@a zjjWz*)?MMHKOOkw*liv;$^VSnb3Lbwk&l6i`NG$E=@k>$ydE54)@gK?mcAu?=75Lz zzJ}|ZOAnr{^E2+x}BUAbs_kzR`Q zuHTu-^6^?{jz+g8ZuL-Zxref3(d06geDOQat^$sGZf7PFdNaR{z*dCoVj@ zaiMZucE_Iw-d3Ahdmc^JfAWDb{?j48%A-oVwM1v%dU4<|o7IG(P^!cUzb#$?&4Ru?Z^KH6h<@56;d z%a0@(_T=)VSZrvv{j1$+wM0nLLw)Oys;;w2{jE|jt9ZZ`J*ePe}7Os=`AL=X_L0C!6d&rISu|_pZ;V#-)76s z9kbi;l$&f(Vt~H3s<}|tz37AS?a` zmVDVbw}h=|{YROY-;UnJ4i?4#zRYRvagARY&RxJZi9w=d>Xp(R7 zug;^YZoEsUPa$s=kBNF$iSYj4OOCWLTwqCflo;O-u;_HJO@a!?Vs~2bor6_pZw}}B~Oig>Md_rZ17=It6j^S zHwud-yvlYNI?6@9JgKhQo}*FrlyiRF(dO8SEtkb+=gmLD^zNstm|Tn9l#*SA$$PfA zwtQyV|3z5dtmx9^!kZ3mg{RF53s3NEyxlGsuss>j zu0MOqr{xwQP4hWq|4dS5QM)iJz=-O!+7V+S(3I-w1 zcdnaxLw-#)?{&Us@}DPW>#62G6E8h$|0^V2MP+eJDzoi6@fh)lmdttH;eVxkQ&PP* zeNkk8Ddi$7_iux)^`%g0dx5GWXB6)|SjexaZhhj*mc=y_+{3E6rcZJ<6WRT##ee@J zMZ1bEmQN-}toZ&du36@~?VH?pan1IRnm%~)boXSW%V?CYzqr$-UiaKVmNfbQIc1FJ6c}eTuxB`=GAP{Q z;hAzeu-(LBidEp&2}y-2JLgS~*z_g)!N!FaOWZwqrpV|_s$}ZU{V}2BLi>|$^^+$q z&Yz$0q({f{lDlm5jiqPV*-qWr{%9)y*Ow~#JuaD@;a7fsIM~keNWq<@^g+oCziC9eRt)QW8aqQUN$xO{EimG zB7v$uHhaH(SZ$ZnA@uHtR?0(R>&M{mBaF6Q}qL zgV#^DoWbw`w7%l!{Q3K>(%FmVB~LDvlb*#ttNRT1v#w_>f4>|ImOtRxKWT=BWrWIS z1&-Sv6wlgzQT(~c#qZyX=NGny)qdEpl-=itqx$!?6;GXOZ!{eVpWDiBwaIn=qbWS9 zh9~Sl7WG`&%f{0osg&2(crW{t`soB!o(;N8<+n}*H8mbH(|_oEBBMs5x_G_dx)({4 znPWdKos#?Ggu|DokC)l~`uuTOubs^)-@Dl@>fc1!t8OUpeSBD$w_@#&BFk3-n{F&p zNqhCxh+UYW_Rr>#4Wd-RHxE#Z&BW zDgDTN7yIrHPj^;Ky1Qup zzK_1XON#>@FObMAUeLVu$?lG+LQe6uS2k)-`hVr!yUB8flgxi?p34^drrCbu3_JHA{_S_Ev#Lex z%(u4MuwQ2Q{_uR7_=7nr4a+CD?!9{J{^^r!@#-Q=pUhxk@QW6nyzAS9?DNY@ek=Q(6ksm(`mOOQfq}t+!PCVtq-29*&J=-t3v0BQ{sv5Xzwbd} zv#V0sfz>|CZi`ucIjv?TQ1-z2mHyHAAO1VUj9k-SIyWROyAsS~yY^sP(ZaY*u_s=2 z+W#yJyx6Y$v-SK!iHDb$U2*NO_{c4P>F4qp4-AiG1*^;DOiR*ybCEwxpth#rLcQq= zXLcV;`IXCOyDwa>{^Zl_lHE_gF7%L}_Qe0^gYF$4z2)mNik}!=IqK}Uufy5Qmc6@@ zxo+=_<_CGJ_*g9VfBrmY?^PX-8N##J&+wj=J^EcBW-I+l72ZGL(Smh#6*4E1yp5)OYS%HVs$TCn z?KkVMBQCPHw>%ZMYfozNpIx-#z3C4X`#*x(rE|+y3A6v;Vn3=|S*vmC_rcJg0mtpk zF1Yrax$8N{+B|6yEX#DB*SvRrGv@}#Y%8Ns;p6KHg^d>v7)4l3NM(?WnQJiRi+6jS zhdM*)FPBTp&mMXGl6_yrm7AvysvKskTT%Q;(7H&V=GH0qrH|4aPc(5$cX3PVifo+t z>4n|QPC=aytcu+J|E4BaHN}2OtPUyH`_##2rf}4gv)|d~=oFT2`xl!tRL(7lzj1Kd z6V0dg-&=WdJN9h3%V@W4im=hoZxT-;9g@W(pL#}%TYvc)V%NX8U`EcN)s@7Ks>vR8=KJu$Q`RoGvJ4&&K0(ztuOFOUv)huI4o?+?N>_{QmV`w_i5` zVxI8H{}!m!NKcv)|7$|4`@9z#?qWq>62o_LX#d!m`?KMAqrQjXQn!!nwMkDL_^f3n zEuZrLj=0el#Y7irkgsjtTw2EeXx^mfeLQdf#5vE7&{Xugd8)RVbDQN0@q1Skf0@>O zVzxg&^@rk{N5Mruj_CcBx#TD&*=Hm6W0(Df_H~a0)w+Vai@qK{Zl@x7vq{}%gSPBZ zXPyrt{57{;Tu|SAW1;RRnK!>>M9TK+UXSfGTW~PJz(b(<+(diLmid0?k}lkwe|mw_ z3)P!G^7DSo;jl`K2uKK?%qe~8=r+HQm&x<%<_2t;c=5;ty-z2e*lkp|-n4phVA;fj zZe1HMTo3lo(LA>0$I&pw<=nErZn*i|J=veIMqK~Xgp>`wvA?El5#`==qxHq%WgVRz zIxTy04l3JkxVA_r@~O_6qWL!j{64HyKHp?>oqvC$x3=Tq|r zR!#9(mz3H2);Xz9ZMFU5@aN11y)&BAo2Jhc zR{tj(n)Q--sIRNlW#te@yrP zaYKivicc(}f%`^K5r_AyWvAR4ie)cVuG=U)|MQ%Usdh%^7u58qPVcw<^kHVUj>ZO0 zSvj*O&fBhD`{^WY`Nm<&i{NLQPH2ez+OXV0W%X;lvJ0+llj^?ge0CyUZt;_<;`Im9 z&sTh8*L2JAW~=-eu5;p0`-|YYW`>Kn74@S(obcbVsCiA1TiCvkBxj3e5TWwSPvwGhB?4sLJ!Zjz%s%}j6J?bKDx2bDx(9Q(! z$6YdI7q0GYn{mZow)(|t7xv1EDP8QpPchg2+St2)j)?lE8!al{v1M2MyhxE5Mw==TxWGNYNG@NU4Vm>ea_<`!oMf25{T#=M0dHFb4W+~e=$L9?X zmRD`pG4f)uW7f6&mBKtP=ZQM!H;Ye`{We`vE%V6zqWDGOhtjJbiu%8%3V+=q@z$;| zwL55rg0uARs#Sfl^A;otM++?7(7N8v>t!;xt(UJ+FSl8Qz$D`xtK-Wytost`?p~FEKNIb8|Ee=o z7Cn%er2ECE4kz`0iWSXk&i?r7t;~}>NjW}K`yJ}$8M~LP_q@Hk zt5{wxd8PWo+P&8^&Kd4sQEc$Oq&K1NkUz5kQX1i+V(Tf+;=Ct3xSlbw z@M_?k17CWz=Xn@S2&g&~$s^W$l2P$;vBAsc|oti%V)|N4fj6 zMLtWal+$}&U%yWmxSU;1&7wg>@=FX#a!v0^KD<{4SbEU<#$a@-|c$6 zuGHsx&jA621HIj|+051MWf&wLYAIp9>YK?CbL$C%U_#=Cf`^B8JfBznXkoit*VEI} zr;CX&o8*4Aa(JLD|I_2tr~Ci^-hZtSy*xsbqriWrO=VHe?zh`EKJK$VwkC4(C0lp- zc~!#C)#f*RUfJnyH$_<8Pp8)DRd3>HiN<*xXXe@NRzIDqY_Ni(;NxYbIj_aUJ#CLl zhPa!VWpEh8-AP~OKYyOwuMfd@sur1N_~op4CU_%n_uDKxcl8KMj)M5iyga|_y6fV9 z`8&lc-Fd+zqipfuzS5l&f1d2t-w;*i%;7W3B-1(PbI*YV$_thYL`~q|#Vjpj!m^{V zPh5et|&o=U2Wo6p^eqVL$lb0&m%#81J*Yxi!``EC{p8G>@p5w|M5!qHj z117fv99N8(Z0%lM&$&MN&}|9E(v$7|NP24 zkmZ(8%uyl#?wLT*eu$-C*ctD~|7xC9s>R5te?U7?M5Y-c0nzPnSK$F`#a*7Q!luYZ zu3%x=!@ftz?Vuq3Umc5ctxk|r!-x=|wS@^wEJ+}x}PGfUyY@9y)>UXCEc zBsq^KLc$ni1EX?)I7bCzocQ58+f{b&+V#jg{#TGp!2!mN=k0zU(T@Ll3Y6FRpXZ(TFI^H^KCde4&G!3s8^7PHJ|2DV)3h&F9KF=%mIVFz z`@X(Ex!>08;L`y*cHA<>7;z^ zmx~|!>;E`Ei;)7S0*9#z4}N=3<(YC=ri!KOWP%Y>Do4$y$@AY-UEh80YN*Z<;r$VB z%HD3fofoSew&q1ve*XJgdAr~4>Z*S{wWQ0oG}ijvj^yx-`PLureP}Z;seQ;D|DuWi z@2~6o-&ws_&>S?^UcdIG_wKHvwbzyVZ6;Ne->)sdel0rxZt3T<=F7hY``ca()%g=V z*Ph?zgTwWGzu(>7nsam0k=^fhz23!V^Wi|yer;#~%5wbp7bN@miAK8|qdlYiuk#k6 zPZM8WT3UF~Rs3k>^SR>5$NPLYzTf}<-^XX>`z`f$JZO?I%?eo+U-R+k(mm^TJnFhp z`~7b6xjB}Fr**dE{b~d#U2*=X~GJ zEx%{^_UFEDx3azeR<}zr<@ObpMFYO*xjUv zqess2`nQ-)4xNY%3bSU-dSre7r(a|Rzuk`nowz*~AMbtaD_`35`QDelANBu#(m!Y>G8vM~Lim~X?GHI0VR>nx`vL9`3y<$uWg~Li=hEb~c^1!I zqW#z0+;1e_iG? zQ^-bk|F2i8AAdTn@86xDwDZFu?!`NIMyka^Qx2>0gWuk){C&>SWh_X16)#`Oo{{Js|>+7C; zQdSk5{+g@%mouNGifsDa(y%I_xHr1}+xLB4yVo#L5|Xzh5?N~MB_A?w_}%4np!$Gz zmW7+0#Jr6BtyjYiM!UOsPt)?3-*Z|EJGTqyQ%k@fpM#(kF0 zX3VfCTof@|C-O{rNy(HHxw;<@E57f(ADb2Ixa>!li{yD;h+QjUSZeB5-q~1~a=5=? zegnV#Gtu`G*UH*NKI1g^pLg^6-iDA{ zkITz%PCLsb_I6Lu1pVA|^K7G&X6t-B$@F{2<38@V$RFFb@6}zm=hLYztj``a^Y zueT|?k=Xtv^@6Wd(Q;@+ihv^WN`b`;L6-%M;%j>r)bC!Gz*6yPZ^6SuTh3SC*-=>A zCvx}c>FLM6-Oj)N{H=O!Zf@qobNh87uQ=CmPMN>re%kEZq*qr~*7hCeg61DdQ2vp8 z{6NEvk5T`CJ70CR8942Ni-88g70#fNn~(jjrZiLpT=(?IFvT_W%Y6HmgHoEcgG#gZ z=66$iL1i$+h6EPeCNr!*>{xyZrVd=PDz`U?KTvMpYl~vv6-B1FhU*U(@Ry(B<>ft? zXasem!3>Tc|BPm2+`3kv^nmq8!J^!F6w^xkLDfJ#mjnYO7Qv2}Xng2$V1J9{oQ&`t z>({MYccR}W=7+MU?e+~X?)PcA8=3t^3Id0C0g1cfiKnm0G0mMj_u{>aV&%5gwRiS^ zc~((#^zX(kUuQ<#{1>yOe4A@^cyx(t`caj80=WBc~)$4&eH+v!#RSrl<^+m2V0 z=h>8XTjbo{ew=%|m5r42-rcKSv0veYRdp)@S@tljzu0&;B+BM%+^(v^n2P6?A0Kqx zHdBq7T5B!lfB62sFs3x$g)GREPHh7rq^tcu(!-?hM^jmTQU+SbtdQyjpbZ zQT54Ft{>mt_Um?C+kZ9Z+~(Qax+j18S>+>TzBR45Bnjl?#wt)LQf1G)%Dn#n{C!o& z*PWmExn;r9rK+|+*F8OIe!ynyuZKGx^lfhDmr+9s^$;PZxQ6gc89a$Pw?5ZiRo>s( z9!Y zAEqw3-uJV9f82z7pMt_Yts~)9ELbG?foad8_Vf=;uV1|AxIO>Z{)n3&?>vxP%jA7J zYJRV^_}|4`N~IkjPe5$i3u>&$_L%&imz|yM{r=~==925)6Ti>?9Y5vu!K5}&u3PX* z@I(K`(7yuLa+%^9t{=Rx`R|ggTTR8*)ve&$6l;I<__X`aS%3W6Fe4eB5qbnbc|+jC z;g{2v@7FmHUJ~-Z|Lqrc^SYat{SW+Zo>K}==O9-!E_67s|F*5jim(~JxsI8gqM~9q zKeTM#^w(1VKs$R@HMm*Eqs>&e!)Q&x?VAcp57ZwtbEGYu^@!={bOY;Mx3o=o<$h-81Kb}PAEv*Ra(KWU|HG-~iL$(A>ZdoG&mY^q|L@z1 z9}nBT_o&aSP?}fw>t)bwQAj*WRI*g;c*L%wY|q!X;5|nL0o z%^5o&a-|zds?|ompG2vj7(EB68{ypLQe_dVrEqD7}G3&Qmg8zJ7 zU$6V|ob~%XKVF6JkIK@I+@wZ&Tik60xL=qSYC-U?q}cAcdRa? zr>`$#_2QoE%dB=K`h0Mm+k4S`-^bof>z>>H|9Nmx&8sUbRsXfy|9Kd(H7j(bZQh+7 z79S2UCwJ>^)A$(Cvp#00(S+8#{eQn%d^(~0_>#AN@ASQIQaiujy1vis=FVTQRtNvS z7M%6J z5V>0bFb?Wo!0vM$+WZ)B}~Z=1WBR?Q6NeEdY>n)`wDXTN4#(OGWjcJ(>ao7{#XmAuow4-*Xwrc#eM7wKN1=qE2>|0v3t_1#CBP+ z$?E=n@qZqPuRQDYuXz6Nm^sDgEFE($cCHTWa zhSud*f4|$EKlA1GJx_I0bARdY|MSV{-u-{ywr6B!YJQyX^Y8n5`RaE&(>srsNz6Oc zedArx=dt9B8nF&uo4PO?c=ku!bvOZt=_viEZ z{>y&Wy4!B1%|5dG(<$xbQ&TiS?e^tn@=^+RD`tSYXtFHRw%dr?#}$;_+5hI*ERQaJ zf&Dw)uP(fNdHV6&-z|*Q`sQvr`M>R&%O$@>Sppy253Q11)yR74QE|-kC5@6RgWjG@ zP7Ar~Y91f6`RvmVhxzqy3w>V5U#syp`1*uBKaT46?S8jwbD@ zTiNS>-21*y-1^N1=g#Aof97t#yX{KdgGTm8?Djt#b>jE!Ia2-q_kH>K|Gq5u&YidB z^Obo=cfF8(U*TT!%y@pF@w+{r&;9r`eZNf2!qApK$^Ev=mU$~DuxN2Girs&Hx5n$N z&c5P|pALoZeASO{GWGt`dR9i_y1NLdsUh<3Vd1$)vzJc|mOiyTl&Amu#P6F@FYj3u z-N4V5rE}+Io>J1nzwgTTTQBpUuNPMHu{*w}zwXQ8rDoMvrt8H@?S8v0y7T+z`TuQl z^97Ty$CmeQK5u7jYbm~d?b@|D*=N6AkC$IQzpiSN>deJ`R$AvQ9`l$s>ssf0t^9D1 z-P^`4Vgsvt$^F{zM_t8ZMYO}#2xyDx#YAkA-+$#{m!JXDU#EuuS$|)A+x+O|)&4Ku z`=zGLi~O8Xwd?-9d+%H;OnD+?kGx>5kmcDV_P|*B*>%Tlub1zL{XDO9Cquk^=fA=q zpO$}{n%wwpx!u>5n_6wUpBSIF+5F_ej>mo8-OnmNKbu;iv&P@{>y@DDcXxJv+;x4g z-ZJ0WVUO;XT=qQ}Tyaq}JVx-wBj?7Bx5hPvpFl(UL4PV}(J z28ExWIO9Lv&O0P$xNC1_!m(98)lQG6+{k>jFI22Fx8=0lk3Zf1H4{t!|NXA7T7JQi z{b>E4t!yv^io*MG3-xx|DSGwJM$ zz8&@G(s8|2&rE)a{C?X1KYdMRaT9+@`t{@-`yS#0Y&p4c<&$qRC+ zng91q`qdt9y`3t#Un4iC-4&l#|LgL6w)cJF3+L;9Ia;2{(k9F(_MqIRIQC=J!J~&? zmbujG=Euu$&)c^x=k%Qpwv^|k-JqtY!I!9p&$o8P&gScT7W(=d>-C~v#>u~L)IUpn z`O;ou#w&5gZ-?ZIPAH~wSBhQ#*{}VvP3oS&?RWXpA@uvM>zj$`hG&DzFRqUJ zx@wc{XMWo+0e@cl*K1X8Rk@u}_igihPt}U=Ht%Pi-+pAn)@PtjrRTFvr}ey}Yd@_u z%2|A}-ei@^n=Zi@>`Zf_zw55-HJ|rIwYKP6+=qP%X*U;wMmQEaY(12-=|>7Tmp@1P zxjB;6YwVPct%>`-Wy7IEvQr9XS#RqI&-=JuGyUF;L?foZjtxgE_HAvAx3B#?`+m%~ zxlG$u?)mq1z5dMSYYlZKeO~@;&%$RMU4A&;{>OE;sKka60Xj-8vIAIUUwP)wc%(Q*{l`MxgdMi(e`)iSXZHfIX?I3J9oQJf-9Hov;B5s@taKR z^%GjuAFVa}lOeMIk=7&bO+U+Ie(b5g_xoXMK}aM#V~gvA^~7?fGS_ zJI>$zmz;67QTYMuj{}BlxVWph&hf80Gt*f8S?*TnH`|_=8vdPFyJ%zSUH)Amu13?^X?;?d-;#A;`DdTzP#y$ z%yu=a<0AWy+_)hc+dEf~H+CwsZ@9BbiP(H;Yw1NBb7Ok4a=Sk21}A6zzH?(+=Jy$! zpN73;X5(hAJ9Er`zjpcynUg0J4w#8=o7b5E8luWtHtFyCpX;}LO1ifUFY~Fluz3$9!yXDB^Ux&(lUAo2f{oc=d?%aH@;jq%&__EqZo$8Y$-k095`~CK^ zndIKPZMnAF?^R{Locp0B^XqZ@e}%t7!}kga+s;e)WF{SaU+{t40kJ9H)|Kv`ce`Bs z>Pp9RPtN^h=Bx7vp8GiBZD2KH+*{=v?LT&&*zn@o>xVlUKc#M3dET%0<@wx+|4i?0 zTD0`dp(*x?CtlYU%>L)vSMx8dYu^jy)!ZseKRxi2>(2Q!(<(XZR@`AR>3N^}-PvVYg&UH81lJH6zK*S={jyQI3- zbia@N@_k;IcZ9|CiJ+$Z#KUziHk%bvAD%ipSwC%){+#8pf4*3ltACsDDUD6f&sINx zd2hr=kv035w3@5>Z#~&KTa7!{&gFxv3EwA0{ggdF-s#`EX3W&w?eK5@irBK7sunJP zws*y+oP9Z`_}t0ahV}pcBm|x-_5O12`#$p@rJ;h{8QWew+5hiUnM~aO6|5G~Sw9Zx zL>GLCZxHUvHMPmz|M{HtWX+U{tE#^reJzicu|BWhl97Hni$Z?0um^7mYpQF(Z)7S@${n5 z^)8RSE&qUy{rqQZexY^yr-x~#0+*EL%(gU-7P}>*bENdb^!snvAD!Rjc7P*3i2rTc$An|+zz+_P`_$30Wsj(qLs{}vPNA|78;X!GyK z{LFyUMt}L%-{a+mmfcl+splayFLEV-}C?f`0v^$ zS9nBl<=sn{g#I+Wo)(=qb6sNR&GYtqj!iOuUGpYczT5Wu9pk?zMBnefFHsmPzb5L0{r5hvf3k7q^c#B^)=!`KZsKyW%~N#JB72tCyMNL@`L4Ebm&NKgiHolL&RfUN zCaCRlk~eax*xgt1Jn`ESS$Z7aDjq1!d%yR)p100_BQ05-OP{yzKk#^Z$mjBH%Ab$h z|J%4_gIk;ak!Sn2c+I_TyXmOwm#OP(RBdgqGyORxo$n*B5;5bo`$F)bX0Ek(Y)N3y z<$kY?tnRn-Z)6=hJeir_#^B_O#*^&!KMtlm|M9rr-~UIy<+B+-o|*5T`S0{fiBClf zu7A4|w)N_3%kNpztD}0abAO1Ey&3oH=%P^W>+`A=w7)Vft*D;4=qR{j#O&7ae^cj= zMVj^prcByZpRd06*{cZD(yj8+WAndPx#|2g zN!ftPPk~^sm&4Zcm6n+tZVk=>DgT? zt|y->HQW$7ZOQfpAvYT=V?dfgJ#kyX2i(i|%$oTA&IiBmphoiB^!J+AH{ZQ*RbtPy z_47LB__`%55&XbZv&8+tzh00I(Da4?gpS^D0R#_AHhjhp9>jZU_-=$mLH!qZFy4V zYfy(a_K#0tbV*t0IsK)kW<6aOd6SdwyjV8FA}z{C+T8k?-!0X$dE4T1>{I>iuS$Hl znZOL16ys*vbEwqm@RbO;^H;o2X`cTWJNa+5V5#Y&sWaDzKbUH%cUVZ2`~B?s|L*U2 za!~%rBGZ1}?(>aS4-Y$NyT6|GMe*Uusr*M|yZ_78SN(7|+?<)Wr|Nh4F}uxtvMZ!t zgNM(uq!{P@+r%*c+Ky10tCjh_&uo4~wbIHbuP6yKOdnno;s=K~rZI}GL ze=B!9I9I$X?c-j{6Z0bfX8iflxB0O<_c@(;_txK>v*R~7yi2b*{q@%`cVpRT!vANBQ*HT=`}_9AHZ zE#M)?22k_TK6?N4C(g;2X06q-dDPf`eNWcu8(UvKm|=61`$Oa6Om1dZ(1?*zgzB%F zhnv=})%~~p$EGA(!FYqG&Fv>a1|_hZ5(G`&_{=a+Ogl3}kljp9HaS{tI`@@;>@%}G zH-lZ=>R`D?EA6T7p;Fq*s_l4c!cO|@U&3m)HbJdl{Z}s*h zT}*)thCSujz+QD{N1L$Dr=RcHJAJ?5w!1~Yza%bR zxl(g4>r%F$Vy@7h2Jr`>HWjs;d-v|OeOLFbZQqJ3J#TEi^VZCanbQFpJWMbUe4trT zTwGjbwCC`hxsMNgOHMl7%6??UwU5m&(_C^U9$uq9PyY9sD(9U2U;7Nx&&lN5OHQ1) z;WDqHL6x)5=PYiHwJ)Y`qTLl>Xu=WcA=U!d#)nWDRhR6FN?7p3hFwXNV>4}wll*KR)gg4>_>IvV}wc!0QJ(3BNtF zv$J(_SFew^zI@@+>3MMt+`Q94~JG#4kZ-lfd@x za&+h}nAfmbe~$ViuV5}j?4vDv`=UU&Y&=S9#GFhSx^@x~{IZi;>V`Fg@1h4UWwHQEwqygdX; ze+CLH5py6^|wd7HoI<~M|&G7as{^0Kojn83`P=k@e)>-PpfUTuCx zrnm-v#uZaDCUK{SJJ;^(Td#a;FT?sHlP?)cGAaiYa%3o;R*Jc88IkkdMoY6mcb%qr zNl4I|zM~u*jQR&!U3jN2gsu;li>ku%8_r72>b(bsbbRT4Yh% zE3ed$TXsU=LpDb{&#hBNZ@MoZ`>t*G_sivvZ#JI~yVhVS1*;0&7NmiOebd^nb$%6V z{J!t|-jA22@7tnMcRjZJXx{F(V&O3x-=)6+kK|q7_igK>GnbZnKjyFh;j9z2WyO)l z-d@N<`Z=H}BlGqNzq6SdegCpQV3z;kFsJld-T=^0*y(Q-K%^)=}zHsS#e!m-KE>^$sO4K z$rUuncUhXP&-UAmBT0q-CQaX?qPPFgr;tN6XS3rb@B8xww5-Ilt@Tg3{qLJUo~YYP z?tZr``%!GsN!6EEqq=I(&NBUYr}+HR>upc}P1^7uJTvpFfr0zRx4vyViZ9wf`nS~l z_+#yN{Y%%LKXU5Z^F@0%S7>{MM~j^PWEUZ0b+Jt&&i+Qgo{vikeN6i2Nd+G}*Ryuk z^W%KM-#YK6$p|cV@AbTXXJPk&X}u2r1wKqS_}v~InpL=P)s*gM1xz2}b#?8>v(W_>hB`Mhh_t_tPK z>K@IHUDM&J+kf=$>HMzMV*iZPYz=JVc3s{0(f{@yv%P%*@i*4|SR#`Ut8gK}XGf`h zW~jL4_4r9PQ$anqM|;}+dd{_d+r^yVb9l#(Gu-vjzZBb|7#j5ZKIvayyX}_N($6mz z_b;=3nP2l*I{8?SWM}y?$@C@hO~1F-|K5J(*~*_6-Q{ItUf+xGpv*Z00FEj-L?&QdE-{bc*iwAo)+=6$pMerNKw$1?SQKB~s+y!*Ly=i2S} zqy$0p>7i{5-i=FFiyvn#pCin_ZpYSBi|&`~_&n=-*WzlpvItPy~ z(`&iSylds7*E6uVmG-V}&}g%7G3bNKfo@%<7} zHJ3qSd$MmVuRoH$2O5_IE%jKbb}MuF+(*Lmf4JOqniYTO^!$CBE$rWJxqRxL%;80U zL1S)bmay8&^MOX@gx-I-?BD;s=DGFKZ_WI65^p-=_wD)7dM&T&+067y>))+>dx=A$ zaiM#|(eK5p=k?~_Ey_BpyU+3c!l~Pj*5CQ{Fk;W6nO|D!UkGkxSKd$wDuHd>Ocf9A zzjw^F*!Rh$>nbyO+mzrQwZLvGvN4?ndP;y*NrFHwId^r-XC zzu)iY|7%vh_uTIL&I|u4D*EmJRs4DX|6l&if7Sn!{WEr&FwJ&r*m+2LPTigdZz`5O zz39swUif~|wd0QtS=-NDC!Dj@Sh8`V+X3zm6XhBcKb?QHd1dw~gUZ!9=9Asee|S_a z(B4wq`Bv?x-Ph^+|2#eTD6-^1BRkvrygNzVx<|a$m;M1y5$%8Jo6ogv-G;rcM;`ru zn_s_s*2ODVM2erC@mxCNP4fOPm%Oj~+)n(Vq+R{g^^beQ_S+p5sVeb#XH$PT>Buv= z)CfFaW~xr`bz2Z8kYM<0yL-Jb|N32fp9R;Yd~KSd9Y5iF`FECnYsF<>z>lIPft#! zc7Lh=wY*;I_k{z~W*VQjG2ZJOS!L7v{PY$zWrG-w4Gi%gzHHsN>G95oMV621!bP2)W_ ztW~e?iZ^=ECHSI|p*!H-=h*x&2R22Yviox_`roQb;9z zGI`whOR-%>==1aQ`jyYS*Z$O2`YCXB{rXK6VslHcMW)^k+wl4F{C_4l#Px60{eJs+ z{{NrzulhtM|NFAs{wOoQO~9kSAKLAEKr4Wjnyp`*a=-T9(wpn+|GxfsDtzB2mBZ{z zb#tfAaCvh+eRcoJPl68)i^|`aw%tDZ=Rv-r7H#Y4OYVIBFT=Zud$uvtY*0tztW(He zraMz^ZayVka8Gz|`t65?WlwkMZw;+)d!25uGucnZ?6YmAbi+J_b^57&f-CY^Zg3s8 ze&~9pc-xsb8;{E+EtQXd)N7)2FX3=p@jYq%Uq6o9&#PPDnfvSvD3x!~b+f5isQrl3 zQeGf$&que7m##_IO$@42ay?2_tsPMei zXSXlPxoBknFZC3A(rL-)rPr@N{r_zJH`om5Y5mo3w%{{Oqb{?Fqtu3?&MK8}LcZxBJEs%}< z`u+8ZCvWr3JbX<0-iNjwPp3s! ziR${-{Vcd7Sh|Bjzv$cL_i=O8#H5^9uI=^C=9qh+776bIfe7ik!FG zSgiA^$|B6nR#ey5dr|V8S1)8POT{kk;x0}O-{`p2LaTL`)S`u(f84oYsO7e`zx(1^ z;YBM`yKgKi-aX^?p6oL>v;2NF9PFKMyZPs%1wlatGHwSr);Kp<&eC7=vhpE!{0_eb zD?Y9|$*s3zLC^8ma>cjucE5e}Y<51M*cYX3k3mztVaB;TCG6{Jsw5QE*3aAXe&27e z{RgVv@BO}vpU38Hx+C{S`+pPN`&^R`6=moLu_j97Cn!b-W;GW%?CwpaDz3MYo8-Cr0sUCb-_rJ^TG^)u%q-@?-?mL=-oNWaJNLJhhrU1BZLbl1oFhb?$tLDT_4~c2o1Sm& zwfxw6r)g9EnMTHZDV-Q(DPZx*+~&(wzkNT_5`Ev) zG$brM`+HhsN${(eVxKO(T&;gvno&6*p5uxCik;J*#H+A&mhbyKH??8I^4fz|9}5~Y4(PB+dhZP&NT)tqi8aJ z?69YN-s{=-f({E;?#noB`r(9fzt8n!-1>VAzTHfp51O){^X+~9iz*9=>!nX0ff|MT z6d$lw-2bpjGIO3nw&H2=b<2EaPWoqidi$fFMO^-|Tio*&h-b$6nn^5iE&azbXStJA z#FWX`+wx4BAKo_ukHShjH?T{}DAm=he12%|_Onlp-1~8~Tm1Q&E%LVYO8d{6-Ch!( zbGCls)a>HAxx0VA+x>X+{J%LfWupKT5rExP>NCQF1-3#dJ_9JP@g=&pX0-qx#7;ATrV4+w=vFL{O}ie zF{Lf*`<>_Kx!T3o{e4xNzkbd8pFGDGh~8JV%6wOz{Ce$nzyGFR_a<97Z111P@q@F% z-{sYYwX8bg*`P)~Xk=fVDJT8mT4S5g^yt&e?f<^KbZuI1YDmrOWfj6+w~y`0p5%W{ zP1)cQN5xUoOUeQ4|F#}j-yAY`5qQ|gMZn_clgXE7`&V$E`g%Qn{=J*)YM)ATcVvHF z(e^94M=bua{_R7X4QtMrGdXuU{5`v8q08lM;DJE50~|dJOrQnGa|#Y|zN~$@w{ShX za`Nt3i?=4;o$g+}IQp&86Q6%-$_A4-Dj4l1y|}#{Wa9x2iAG0;@-p`8p^)BG5`eT+=VvPBYD{n1aJ@3F*)$^}9Z%hQQ+0Bw}-12ux zMfQ~~$1JzbeZjVAw$SFk$JX#ZNV@hx2~^EL%FPFjI)FyH-DDVb4?OR=S$QN`@*P8( z*%^J^1A9eNcU;_VulRs<$Ap4R$dH=b0zZL-hbQwj&Sc*+K33whV)7bgx9#Rf?|j_7 z^v>UB>vu_h>+-q6{h>={+AXdw?!EJku3Vd8B@G(JlxEaDpl;^h^>ZzIc)3w+N8P(q z%!?`y9r^OuC9dH+qeVBL6;oUTzts}6447S;1QQs~t!MY2V`2DhMumq@OziLKgX(MM z$XfSTC;H5@vD7{AH(|x5^B1G~&vAcq2^B zjVqahwjPw)Tov?Y->a*J#m~-wh7@l#Jz}lM?aAGm_K>w=YhR4$(YM0@?&q0Vr|a)*lHB-u^ZL%6o1ZUb1q}!O5=uL( z`l@9W7ihiNscf4nZA(pbbGp*yZmNoz+K6b+7IVvnPt)jMUYfxo)ufsCU6-g!?kIGMH>k6jkT#eY46S*d4r%~AYxVhVwZ(w-8lSBR`>yEap ze}A^xuRHu`w)(F(pVP8=*-L*sd3JQWK(XZ|P-eQMobVr(%(WO8#a7s`KG!siHU$-f zV)x$gKIgP>Pfved<$q9H;mZwB|BPw&jmoa`O|zSK&8w>Z7|W^rRHDVf`Qyv0znNOv zK+B-OrJ|dH?xx6`?1OW!H!csF#upV5T=ap_sl8&-#|=`AUQVEGAyT}@!JBjpnA{dL z39gvZTJGPm=fu0tj@wiF8n!P8`Nd&r2OGi@ZE6tTC2AzVxA>Ld2mX&Ce+9zon0}cj zJavD?0SS%;vjjf;T{_F{^u}i#6^!#1KimZ^BS0A#l**Do6?9TV?mbWe3f=SLc7Wpv z#}DrrmQw4$vm6kog9;W!w*&qM+9eWZLpC))4KZM{WQuEep0OhjnvLP4tg zzdYusE1i0t1>$gt9F{*~Qx3Xa`PK&7FVoJQ`5U~r4?J21wv3f2uHm-{WQ8`|lEWPh z_nVA*5^hgL^}`3@g888N9)xzc1PRa_mc+qr+7KIICOu%?alTQ}{lNVo`J#@uU}pwQ z2d#Raep{)Rfk9jqbQZw$!*Rl%)%+E{Jjto*3&RDn7k%z z(B8e_z^2Miv)}*9%Rg_l;b7<8`%Bwq?yI$)bN}tzpTExRe15Fm!9Zfl&eu#vdrdT~ zXB~QF*6SVU=b^?^dTK|bUZIudk;^-@gI`_IJP^Ffy241sfeC~r++!&-WqY!+l`(Ro z>E5>)mEW9qw?Fw7sHa)7`wC03?)BoCEVrB+nD*FixiO8!qXCQ(`dJkAz5lPKkh--x zXVNw8Ur$<}JbXOolBIdGlFiy`qo3;M?*~n~&u+skq`(M59@R{Z4i0MPzek4EEN(va zTE-}~G5$~8{(_j`Tk)GbO&geOR{nTfSkM==-QE7OmyiPEZ=nW>xqFYPYp-6TuN{_K ze)9jHm!dWW8zwlob4L8vU%EJDhb-p=1~771&7!bxuigGL5m&Q9Wnfc)yfq>>=XB=ErdRPCnA$?~3oZ%a=;GFs2&y3xn|(u5YF4d>&% zoi=_-NMqVz`S_?4ID8UpnK+I{#68?TFL7O0!7S>eV*+R9t-xeXdJlh-ShUgZ$o6jt==AKka~wGfd&S5uYFeW6@2Bhr?xq$ zvF(p@rdVyUe!3f&$CC>X-l67JL2pkq139-n3dPHtrR8a5huFU`H27`K1{Q z@7|wc*7+v4IFh-{#a?#cw}pSI+ez z-5sX3^@{GqCw)cxi zZ*O)`keHMz>L^+CPx;c$lo@v{PAi+rwO24J?JKx-<-@Bf8s`dLB&GcbG|`^px>~_n zTe6p!+qEcj??s%q7x?#QGvVH3-tix(c&f9qmyKdYm1A&^PrfvoEB zx8FP%n2VmR<391_(t=rY68{^&3&}m5{e6dLQ1}DK1HyG02UcHuEimVK{#3=EDWwiw zMNUSED}s;lmULeBRGfFE@6kce29`-)3$|?wTeWIc%7F&P-E-T$56uiX61Vd}qA@pT z$rJ`rSND^JyH|koRbzhbr%#h6sHV9cev`L>)$E|zvEQz5%fcoxa4s=u%)Pzss;Ykaa@k+Hbh5htwA$}?uSb~GpD4KD z-?K~Q&DKQ4Mn0hh4DS^z^0rwwzGB;d=lj-pukDfV(>-pm+}1c${=WR$1P0C{Z47Fq zA1iE*oioe7rvqwCRD7s^)Ty3wZ%?J$Y<{~R4t^F7TRL7|=tGVZL|#ao@{H_7~7h4Q0_ zs`|X;*E7wmkQHn|_}r_n)A+blZ<>twpyTbi3i@zhl{< zav=rAmeewA$L#!UKDw-2zux?nT>YQInYPv2Hun6Uc>Q(njq|40W0XM)$1bKFYT^9!^Z9)D zmWxxTPj~P5lD&R!SlF5f#hEi_9xJ=N%y+W!Ig7&)a&|R4eCAjfCfP@DI?jK;X+FdG z;(y8q679J;w@hbXt*GwY_&1N`L-5l%Tk9etQ~#dkG&>jf`1V=mEz%-k6Bsy`*f4UW zasg>>Ey$9pxOeeb&5t9q@gsi`?Jc6XU%>F>AO=iB^v(0uGuczmtu z^ZE7rLc+tPOW*JPK2IlV%ZZ40S5{7bv-!MT+i^*4x!Ny*mM@n~?lHS@{LjzNs_N>; zS2SO6*XYm_k1d(_=f~szNwa6KUd^KP_sL{`vuW(SQYy>mRjt|)bJpb6@T5K=zA!^6pwC?<{|RujtLj<8FWc z+W&sDS@7d!@wkeG4rj{W-MM)q?{FJ$(+9!W2a*52T=rMBw~r4$nY;b2+Hv{%Jsaf5fA!kOleofvS z8~Sd?{o>c8G{O7*VkM%htPZaxyX|*#G|%d{i|1_QRE}71y3lkFN_%5@Y^e zB57Z@=hK5%sYmp`zqsgpZGHUu)hva_{B6HZ*|;a|tW@R8rPD*QtYbSb@D!h5eY8Ze z;q#|YCBlw>zT7RppR40+q0Dz{j$Hn=HIYU4Yrjk83JCZ1$i?m|F?{7`{no@WX0x7! zTVbK$VJ?#iW_z3~l=AijuZ!k87jd~wcg@9=3H3Xl>?&5#*6zOEZMw_kVOT(ey3Ec} zzK##V6+6vL+AX7w?+DYm8S*$J{)bG@O)iafecFtlzqZdvSu(Y`;o8%=<@YA~+x>LW zKXu;zzs`x2jfIch7K0*1xF!3=r>E27wek&JcIZh+{5Zs||H4)Dm|^+5I~@|~6*b<6 zHeTG-BV!qq%bdSK&!hDFlga+CTt&AvZV)^Y@S5rUr@7_-jLR8b@!9=IFf|aIX=>cwuv7+mnzL?M_@DRLb3? zfyKT}N|{J4`jUhjUl%X@+!-$D_M#csVpa%mOI1M7Y~WVihL<#K4p@x*r7 zWv>Kg^XPw>nLcmjw26hxa+#Yw+*j?)ySr=U*B|`zk}78W{_-;UisO{o>^~3Q&gJ8D zbl#jUeOlmZT4L|6h0g7ddXGA>Y+*RQp1tCRRP@$}XFrd5Ij4PQp4@sybbI;ko)7=l zi?sf?o3NFQY0g9SuvZU{+)T26I>ln!H2X>48m|9-`(1{8w&nrd`c8KVj)i}|-#>fL zwk7_+;T*Xu-_IDI=WtoCCvfh=1HI^NKH)1FYo5=mUbnHw;zh8ZrR&p#Yg6<0|J_#e z<)VAlMTr-wearVfndE(Bk32_u%6$X>DNP#$H?#klk=!@&e%)_hK8pto$@jnCDejLc zy&8JVi0xl&!9iBRg;Fbws~7Yh`{Tfr$RyG7qHOn7(@p+`;*xJ?Dzo~|);X@Wzi@HS zioZ@(<=O%%6B`+h>+6d=nD+Wri$se3(qJGrw-`DH$Sw+Gv9t(1AZgNdKGh^W=X=PrSQ_20d%l2I2xBW69s;!`H$KMZ! z`FjtRM!l)~ez$zp*;iLrKYi4#AGY!02cyL`|NnmX(z#!BS~p8F&atBL>_fpB3`h6W z91(P1abbd@bK8pC5AvG|XPkc6F0XgL?)TcAUPpsxYb$V^(R{PPulnZMw?AgGB>6Hh z$;?aO@j1$H?btE5!f9tGZ{_`)QvA>3W2)GZUlTa*#xGm%Ru!&4WzvHxQ?~`->*Kr{ z9)~LyXB}^uC&EAN8;iz)%lo(|XEWFoF>buiuem{T+wA0m$+v}H)|a%TJvi;NJTiJ; z%}%bV4Gyl3`<`8otM)Z`;8$H~c!F=^`SbSw_hdEqh<0SQT^C;{sxe~<%S{6@y_gl# z1pn7O?loUko59#;QvB@B`KU9qjAIin%>U`gZFsI@^0nUw&YiuNrXyeRpwa8i} z-sb3WSRdRM;h`{>^Fh8ibH+yluH!sjX-~y8*VP1X+gau5TPB}*Mo%o?ZQqaBxhgMC zOqgJ-<0zx~>8~JHm+}ko^b`7$ja4TRL*C9@gfHQ`TM+1 z8k=&@o?>&*VqQr0Bk9Yo@s&@fYOVip(OrHm+u5TZOKxq==3zRb(=a2%g*)&FxAT)M zeq9!$N6m(Yo!^}9mt6MEY7$I2;1Id9XlYmBJiFRmg8e-QD^}?1>W1cKWoNU_)HOKR zA@;z7wY~Vl()$1ZvXn$^4+wmnvE_)&XPcZG23vFfU7Br_+I6qBs-3|+{icH|oAxa3 zJz>wz$UH0k$N!pcT2Fk-#(!GEnwfv6*u%AV-L31)a_?Vy zBh$z$aDRt?!ml=VO@IT1$PLyAd?2_Iie{M+~X z_3OiNAJo1+=&g?OW|VsyH%sI2mI=qK&&l)i?|AN%@p!u4ozEYdY*#$rs}X3M{PuA` zL-g+Y%-AN*n!jJKYh7Vt65ssTvE%XSwBvoUM)m({o-*qF{2y~tf`h}~ut;aI!s=s| z&*v<@bT+X`K%^x%VZO9kPQdAm;wypfD1mBsGm3%6ZfB=!Cl_qtY#ds}TI&O0$}RCvH)U)q+mQTg5B zIu-$^hRoUdu>m?Ao!f#7TXg5yY>GK*b;!GE_NU+P_pfK;aQi*^*Q0Lzketum`uj3c z)*oC-FAb?tI2~-_O5nYmfaEXTA5ePi*swYx6(e`*ii<-(qW_4z~s-8!6FiTRR)X z9+d06(5oxlnkl{7`817GmjlnRZV&bjFdF6yzy>ey0foVUih@4 z#pl>>!VfC_&c`LMGfZHg=`L`;x1{Rv@&10!{C6#&PB%D!{&E^{d;R*JcuA+>$FT$Z z-_88G>1OzpEuTG(HSSKm(#rlKWu>E4wYA-+2T5$MVxZv_g~l7g7xp_DvDtjf*?Ihj z&;oXweKop!%kTV{@Ota~-S;k)xf!p^FI&4Q_Wqy6TNB{}Ck&h}$_|Wjy2oaL8->Y`R_XgUW2@^V8}9qRn>4#<&+!33mE!sgKxjNDDdE6 z`-*(43NBU*_f*f594FRr#)1`*IN|3>X#ukAE4E`KxujN>p4JN*17C!ss z?Zrqp)3ek59P6ECEMX=aXYT#0L_cKPv!_ob4IfN1p2jv2+a^;zb7`xb97g?Xi6E=NGa?UmafCup-+7%e=Y91|0x(|ExC z@}=~UY2JlLrGutkj84;y(x2O*9A|M6c~H1-#g1xlf@2igpuxyh&|Q{ebf#&?x2nBC z@$vfB|U+ISl*@E-9obKo>Qe36D&v1uOqUncM;$mW6fq@f+ z?nSuS9xXRorzbZVn)cT?H!$6?s+}n`%^~FQZQlF}XXj?2#})257AzLmwu(GBDe@?) zA^ZGxu?HWDk_$4I7j(JqajrP5&G+#vWV8-Erun~(f%V7!g~xLj3pFfgJJGgXx=C6| zZ0V*=L07knJP0hGbik`4|8LWQwhPzvk5oBNx%_(W0ltR|L3wXMFUuBT)tP;B%;ss$ zp6I~B$n}9kCZ)-M>=jJ4qS)iAA=2hHlhKeI$U%86x3%e{$lORrz37M?mGz==~)AS9CV>V{IeKIGd_*xizCvLw??~Ba!cB zJ=+nv&(>JoKkj~$HSUos&12#nxe8W z+`!7{g6V31&+mf9_UG&q`YP|t{AoN#ftv->{#j`s|0#_X(DM4=Oi2vryMGN$T5N`S4%yoS8z8jO~wy&Jpiw zI%=xUVSD8LjA@qf((ShY*Q{9*u>Pl%($TL4kMG_&T3I)vVxPik*EgCTymQvuUk#pf z{tTbs{n@Q~8!Y|Os@c!|X6d@lzJsy+LHEBI3OqBIiW8dR{xY2CinrXmyfJ%M!qf@V zXG}R-;^yybGx_Lq2}uLrUf(qgGwhl-&;N0E@664th8&+Q8DHf$zl&ayW+%R!!NGc$ z{*%tnmc3RCD$0x*r=NDT+}vz=<_jyghl9|CMw5|0SwCnAd&A4*pwYo} znt@%V(L{ko`+(F2Mv(@k4a{%YL=)^+Fn)c&VZnCmpk@JI4Ab=kyantwEX@fldsrSn zV4l&Uz`-A=D5b)`(qYyFyA-ZSXTAy6PqFZ74=nr;WFdI2q4+^fg>(#C_~G`4 zWn=YO)C#vNxo$uZm`|Jc}C%xavjmt9aoRvN{YPUk)u?` zx{V>aYwgi(N%~3e6Wlk*7g)$JFYn!asPf^Wf|EP0?$F(_cZXPv_&Ry>p4kT;fBaNY zyT{g!;ry}mhuj~Y{y6+&^N-m-1plZr&0OGQpjsiYh_l8?ZsC**9v03!JS+TWsCWsQ zc5XYupfr6)*N(Xr9XCAp_{k{xO1{70MMdSB>))QZ-ebt2*8Dna4CQx#@fKetLCjELDA~b#z*kdZ=b; zP?kfM(=4@IC0T{JYI|LG1^(*#vVteh&HGqvoAzY=VCCYtv>9y|u4Jw=;mcKiEAUn+ zx211!_T|VgDOG}hIrjG3>1iux>q+bF4%!ycu@r z{D}UIJqs!io@JGPDt5~9%qGq`+@I<_tkT2vJFlr?zT%QHTy(+u?uj)#hd?hSEW#UA=QME&ZPtB0<9x)OCo zZuR6VYp)7l@xHSA>T%N$lRqZcGE6dUE~sQhWvgYryZGkfpA}2jiZ0yhxjHsGB-(HG z(z%Q8F8S-xzUuLc$qR$m7DtOO&t5xw(eCi?ZE|bU7fxRizOp=4{@V5{&tJa3`riD2 z)D4LbysLQGcr@G2wcTv^=Hx6YsTSN7MG6)tr5P+q7+w+m7e_-(tMQdt>$%>DV=U zE$TilJbL(B*0-GB&EJ~&eA*-0nfaRgR`ul^&pN)`@0DN7@~Gv{&+R&Q@?87e_}vn_ zIbx$??RMSTwXXDSk@Fp!yIgmF-ATQ3_>T6S`4uZFRjN!Xd#a-<)_r^P=}^_rinm{+ z5;QkjtYN#hWBb(aQSZ6!MCJ?3=bFEKU)4UR`+oK}&fm_TeqXuXw|-xJH3LTi>l>~L znH91#1b=u7=xG?NFnSTW!Slnc2lE7?PJEhhYy$s6y@j`(<=bAibRODuSoLu7A?ZW* zZaZ8>+zx&?Y(vSx|pienTIxA zNnJkOTQ6PTUaq^hwpvZz(lBM$l$BRHu1qs=ipsg=xsS)bwO%pS zvNpA9ze9Y0g2b5(XK&0k)cYcMEzN1Bx3Tr67a5x`YhM!ls~~l%?8Le=|F$UA9Q`=; zlJ-yS>hPCqZ|!+i+2?-Ff3@Li%hkKDuUL0;Lt4h#&$pUiFDNa1z3ckw_2=X5C%v8- zy}xmbV(Q`bmB%WNR=ziP%XxDq{M7lg^8XHgZePaK$&`5TfZA@~-}OiSocO&lfAh^} zpU&Rr-q{-7-P7&Wy)W-!&dyy|Q?hSfTbavw``5PUy!+dkw>R&9{Lb(A+^T5~Jr zYoUJm`?Ixvzh`~deye}$|5Jwd2lh74W52y>>niW;0@ItO=lNRYU;RA!Yk6B#>#aq% z@=O2hI{53sFK*s2Uf$lW-rKVus0CiW@o>gr^S&LjFEh+?>~0*&(6r*}>+0M0!}8nn z-N$SCO69lP-K%l>>=LTIcIyV-tmvy|_vT%(yCHb>V*RDxDdo#!(_ibm?Y}ZN$TqYx z>f4uZ#!HNUrY}49Y|h*lLM3~xrdQu+1F|CRrG-*$Jexqz`jMSpevm%3Mf&G|RU``DeSZK)0U5%H?B-Uu zPoJ!AvS0FlMb+BRzmClJtN+&L-q*iJaNo?T-B0KK*}d!gv-i8znbhaic>Ff_x$(o~ zKg)lezc#;mpG!@p-Ofka|FUnI-_4(UzjmMfEU($Kgf_IknHk35$n59A@Id#Ldcb0a z1$P-5R19w`pXZqM_H{bPAN6@}7pJFiI{1n+EZoWPV^OlDRn|hqmn;qnQVabU%#4MP z+fSL9kY@GTHP1bI@}-q4%QNyjuCqS4ec;NKl;2mbY>(Pz*#a19;eI*63l9Fs&r3l{u1?T*tR0UH#6FmbZJ1zwU z1)HLjG^-#NH>mcalr&qVjFOT9D}DX)@^Za$W4-*MbbUihOG|wNBYh(y-J+B<-Qvo; zlEez#ykcdDAuw}XQj3#|G7CyF^Yauy<|ZcPmzLNnDS<3ffB}d*Q!6qNHsuvVy_KAw zs}GXVH`FuGhno#D9wcfNkXezM6XBAXo0?agnV)B8Zft4;R*xZru+avgGZLZG#Mly9 zCz1?Qr;R?y14tf%gcevd$i>Z$%SIm@uAnfsEZ^NnI21WKds{XfNGLnPCp6oz%s^2h#^Q)#ce`>3ba zF+Rr4l85*vIcg{vFdb1-QB+YB<5kW)k}V{`b_)r@w=p^r0aA z!Jg0OZud=Zw_bMlKf-Z4uAkoS{e5ovy`6XKe!sn)yZx?N_rHJaLEl<8gM6`bPUGkE z_Vv3TG;wd^*56aG`^BQ}G~KxAACbL!cxL18JvF;4?pHkSz5V;%_kEGOTXr|R-vTmV zUSqK#v)&E8d;OJXlDD1K-(R!)QJ40!rMF#cmcs(*hB3%_ar52o73|KbzqbGHrJn|$ z9bV^F{qf6-2b&vr9r?asiTkp!2kU-Sl>Btq-oHc6 zxMr!d+|R~?&0^=H*GK2?Eq%1nJyFzJPNwFlMAqh$`TLIa^&QvPX6pXsy;OyGPsEYM z`rEAA?^Pz(bkteYocqYT_cQx^-uRP;_1KT{2V4J?GWMx0HQaFhn7DBCr&|wCZ?p*b zk&>9|&gU)jsdv)BCHo&eci;2jQfhm9f+^ppq)UDN{Cbl6a{_*e$Jcyx+}F+*Q?}w~ zvM_&Jnb}Y6Fw;}tPl-3j%uN4Oa(yd%es5w^FaJiZ9Fd4XyBk@X!!Jb_pUU5Pu>dz9`&9c`!C^O&b%M4C68ZEH?F&>VD+Us??U5gzPY{Wygr|gwihlcecPGdo2b`**?@Oo0?Ydj@`K<9z z-yKi3ANZ8yxMjlQrTLFeOyt}a@nK8iR*fhHwR=lo;qr z@{F7lN4Fe{@7dm4b8&faj^6)r-nh!c`fOE?a?fq)Zoj9ReD85*|9|Ua5wRzv?Ri}G zm+f5MAHV0;nb)*=;Mn=NbvIJv@87;&B*C|aL$2=Q%A4Ele+V{jdVkQkI)d-1@41OKcQ4CLG0#8n zv}}2?G4r!a_bkKbo1RmBt^LX-WB<10^5@cfi<#G^uc&?4BbfgDj1}9ir0Lbwoq48q zK7ZeQet5GdUM^vq{M^tlC;fk|F?|2CL6vXgXYoACbNaSLe;(<@S-p|#JTJA=XrKG- z&Cg8UIKJLsn^b#G`tI|myLoS?GR*M3q@?bC{P$!_yA!Ne)H0eQZGQa9O(^~KX6wtH z4)@HyywIvJGp(%1*(zwMJknP`XaWp(Kxvkd5+~9KK+LYDj-1d0fKYE*=_1WV+ zf~T_<=jxnYIpe_t<9elQbGyW4-!Ps6 z(W;*fp*&kpD(BYyY*?Jp9e?|2fBwEH9Cp^FkFSS4;d-|#f6K`Lvp0uz#J_%Ctp87x zyFd4{s$XsCuQ!*i&sJ~v8uISc>A3mUk{iy|?7b*sJLh$bOwx)Q!pr|gCi@h(+cSn=SC5Okl-V_78$)yZ?x9IvcLD;a1Pl-s9)4+v-V8FJqCOy5^>&{1*ZF#Iq7? z=VzzJTf8*zV0U{wD}O%!TX)mP>dqfteCjn%mU^rmXB4tVxcElP-G@i^na%t$<;$ak zI}>DFChl|Q{3yBcs=i&j_@m^wE$%!kE>hAHUAFmZeUe?ti18JASSG0d>3R zvR5b54NlxR5-IcRR>`BY|1MNod{ek@J$JX{2EUkn(fZfKv*vEy7Ug{{+p41LAN#Iz z*KI|fF-JasRGR3JmJ>4dXxS5@CpB2zY41IHe8NBpKKGtb96ede9Dg%JGTC83_m%)&g@HX^3}v>zr(h^ z=Vui?^8LQ}Wn}WYDbMznrrS-l%w6|#>#}I^iffE>+pG9L-JE6D@ok>{`_G@wURj{R zQ1E~)eS#QAWcQrBmi`xE*_Ttr<BXjk@?iZ37f9{+AUc0J%l*Y1yB+n#cyC;03M zT%Y{V-0p78(#7``@40y$6PI2(XU*GEpIdKE1lGOsfA%ATr)E;8uW$Xe#qY~`*G!hq z+mSfyr1H*1x%VE5sQ27@X}F|$R{O8XmUh?LwX2IBhkVjh+c@E=dVIm5C*n`%>+Sww z@N!eDu2?N z*~5qKJ94h?_;Ez|cvru_f8*K-_CfPakG-ruckQIDZFWzr+$>|WtxuQzx#)P-`S0PC z7R%?EF0bUD_WM&+{?|^u`wR`UHmF54ZJB+?EK_=O@1n3ACvNB4Y^4*ExQ{pQjeqla zd9OLUQAtd~=Fr7|E?pJlzQd<$B^Fiq*;-_I+z(liH=?%q(#>$*GQ%Ilcp=5kB)#D4n6%eKzHS7=?c?UB>V!(!e0emWlIyFcgAcDEO4 z&i5>CT-9eYVs&Ggy?0if*-WW$t`T3J~!g##O3qPL~J>nx}KN$Ww-Cz zHJ_F0`?uLzOx{#KXR~C6?EJH(dFO8kcg!!<|Fr+z#4lal>$cgZ|1aq~VD)c}r+&?^ zx1mPAC)V4^^`!f5>|(XsUvcZFqIOxX%X%k&$1j_-^Zvd$?C;U zV@^@|y-&k?D)VWuuA8<4og^>DB4|Dm@Up9m%A(F+3Gt7;|(t@ZNF0F;eN8+ ztnO#K`aF%_rpMl#X1;&x$IJEKKK^j{_2uXRx0eN~!BY8!k5cuIJt?>(-fr>r@nJpl z-if-hyHYy-?PGs`Uv5t4ZToXCd2EEP?KHG_{pRVnb!Pi)*RI{xe*66wE9tkzxg!00 zK0m4G`*71GNA66z?{xoqp6nPy6k%k6fMUE`Ra3ws{t7*yAg1zutU` zw)wSn^X>Vj?B9Ee!tQ;2>ZKe0BP%=O>{?3|=P&-ZHy`^Uch1&AxLjU7z3k0i4YSA) zpAhanCl6ik+v1xxSv=n9g9__G$CAezmoCn8zIZWQ>3-v)^gxA8A99LncNkm|=b8UU z`-!gFwq8!TyA@YI9zNb4xL?Ywz9xNfue@07HnH-LKMp4B_1L^JW&5KI#hvr=ck>r} z3a?@+jDVhap8o6*Waz?d*q(7!shYe#r-|Y4UZpQ@2j^w%kHD~ zgxT_sADrc9*zjY)da=u0_h&p$OU%J%q7d6e- zugZD2`sOCZ>-^!eVsn!&uIo?NSsPrl?-S$W9q$WXWJvSdYn%3ci#EL2dwkoDpS>T# z<<~B~#uiy|mN`b^ReLtS*#Efsza=YF))fCfVHbSOR_FJ(_N4lo!XH;X-roNq@yhx( z-SD@e5x0IMUQB&$qa{~bJxOl1SDCh7-JJfP%a;3}kNEg9^t0x+bL_jm9C1A6_xoYR zwGP&g`ffkPwp+#h%<%d3I_7Gg&T2cKR{07p%cx>WH_NB((1-ZaHh4khyE?k}odXf9+n}t264 z)5Xl;+?m50g!%9MFc`gZpaAT>Y1__mzw8?@Zp?dx;44`uDq=3d$9JoO3jS# z-_~e8`}!yCu=z}@rnhTcelA}v68w0%u-~s=D{7Xk<~Oe`e1Cpc^G+T2BIW0&X1Uky zG&X;!vOYg=iul?Zvrds>NSmkWDm8U!J|9J7KxaqtPhqd*J_GJ9~ zx$j}K{1LyC`QIw;`5cZsuGDsHh41>q>waAM6ZYME?XRL~_FEsFW!Zo9K|{;R^YzIK ztP9!Y!#JU@F*1qZ*~ z?~XNlK0l6^4v6Ppq<>ED&gXb#w|NOBW~GJg{o!*yK6vE*Mlq0@rkGA-%AUOH zt8YQoE3vLlRb!5-lv7p)R=jIko${1hUnXQMoUk!7x&3Cth!d zdbsN7@i^&&>-zlL=Kn8pl3Q2xujsJ>AGd#GVb^+_@&^a>pC>-+=bi5%{qfeVp{CU9taz6KYsg0J`j|A&o`Q>3f_uR2ko{xtj za;3_Xr-s|GhHmaX_4jY(wJ(nfd*!#)ooCq5+q&lC$8P<$Hz%k5jVzwPzH>wV6CY{Q z^t08Lm#1$pvY+bzqj1sb?VT>Z3hibq_HA=u4NlOj&e3UE(YnCjKa=5=SUk5=@1DRf z+XTam#OJ+C{t4r;Z$JBpvtNHNPx3KJyxU$e@vmC3OtuGb6 zRh+00jI=n)sBByMBj?cdb7Fk9f3x=eXj}Ph>Vky@CVCbpyHur3<2FwBH;t>U>y_k> zdwcA*#EFM5mrIM?+sA&tp8wQqzvp#qZ-h%OmFCrN$eh1BfpE>{U+A4 z;gR+37u&@Xcl_z_x4eCRR*l4Ti>}c5YmC!n^LH-j_Fr!nDe;Q^b7Zj1C*81pUF+|3 zwZ~Up^J`yQSDaeoao^JYm|Ap=8CTDXee+Boo#eN!s8~^{$9CgZiNODg6`$wo%z4x> zeV$c~*{xp9=!!32R{vbYUVK%by&z1oC+A9y=XEKpYihG>bn)v&i}q%=H>2} zo!=|3*YjsX;1})H`n!I%2!{Q+yp6s0ze>rX)8(Cdg$=5~MJn-!wy5^@b+0^m+<2w$ zBD>ukdU8#U{BBmdA+?jQww&EQ;c#1SzRcys;`}4&lcQt4d^~bTV$#+P*CKvDb^G^G z^jFe>5qjB)r!@b52*00nmH+gf%+0-DWM7@U^5;xM+dTQWALrA0*Zw@Ltj~9& zKrrslt*poT*Pg6h_~y_B{e8z`ZBjOLv8KJA$#c<^tungSoIU5`)t?h5d&nFOihJRE z=b^Nz`Kx1$!H#dbYq!p=`)*q?W${gLb+9J(^{*wb@65}}zoq=P{`aE4sSiGUn6TIB ze4Ov{zf1G?Yu=h?+5z*ey8KUX@%Ky4&pIdk?{w#lqYdhQOZs1QXwS6b*;Ml4b@j9f z*_V&T7gtP5@BPo{v(r2*;^B|$PDXY|b{#v~Kj+f*bz;i4S0b)%c&Mg)egF1!-rb)Z z>w9_a<<4BSzmxLus_5g!xqfwFyKj|TyYX^eazm(2%G&3gc6Ki&otFRqGfdg2rnFA( zRQg$P>O1FYdGPq&`EuAueXcjLwB)cUKgYu?D7Iia66<@7OuHi;Kn6;H!=#~eDw-|TZ|!-O&x_GL3Z z-aND8s71x+*M`mf&L8L2nANn1ht-`Bu(7GS_2$aWWBkXPyX#x;b_q|Dns~ACPo|Frfd=$M5g%8sK`0Ar2pJiTZ`t~kZQf|*F4j6FVAs5?=n7m^FhpuoELL~ zerv1$I5*+Rmz!U87JhHE$v*U!=XFhX?Z?COoxaCZUh8n}nY;c|K;B;kk9!fUz*5*~{3`_xEOyiU@aS(5en zvTLyQ?$R?~X3sGQ=P~=K>Gx@+HJ@UB;AMS3v!_*4Pfq-!X!ZFu|Ki%`4-aYR6-E46 zyYb=c!;U4_Hm03tk7l227H@vH$ zs(yZ~z~5&Q@rmp280kH`cja>DdNZ>bTW_k*U+_5jSV70}IL9P+gJowG?;nk>xp~oY zp82+;Z2o$e4(j_CEsMXi;nVuF`G0FJx&M~4F~0uPW8Z{7)BYFPvi?54@7P zmWQ+do|LUCzIHF7c-OW>+Y7(t=1jiSsdt}EQGvgf+xG+8COx*lYOR}{MYSyd;Hjqf_Q#v&Gu+Oe|1;~@LknsBpHq4bH(hr7a42x`wa{9t zqZ8*_nmv|YS}M?AcjcDFI`_WtwP)|2vHkb()}dzixVW1KT+cQ~$$UA*TWR;e;==69 zDJ6%mx2@j){vi7{_NS%Gi|v`;uCASHzoGP9OrhV2#hXLq=IZ}AKHs$B%c65y-)^la z?wnWlTv}4r?#1fEm)`q4QP#2g{yp;WG~OIn`G2*x*+)*w|G&ERDWjNOllVTVyLUG) zvv{CT`t)GkwFA7tK1oNk*Z(mWKifQu*Y@qPMg3pCr=PvK|NYsN`41kuOwSkY^qJS0 zvvHm9n%?_O?{*jcy1BmlfD_;5vw`w_nWewVpKiIt2aPr3bIbA$&UjI3B^L9%q->vWG zRF@X3Mn1puF`aksxv53cdXYBY=AJb;oZkK7>28Zd->yYi+fkEYMevEhOib_IjITmm|J>Zk>MqT(QxWi1hTDCCl~x8JzYtd$~fiX!?^k zKRnFqwcYkS(l*;!aNaAg=i!F%Z|P?b+OGSVuXif)^URoYvzC9F%k%%=g-*Tu2Z~QA zUfRCn?_+Th-hbb2C$QjS&;=+OD=@$B`|1 z1+_*ttbTt(mz~Yu_U8b9xUW0Umo4G{UoULve`e8sp0TK;;(_c>iQFB%*}tTAtl%}- z$hJP>nV(zcM#TQyzM>LdCuo%ulxS8-{=2j>OX$D`lE=&*YKHU`)__?w!gDs&gw5O z*8h9>=%~J5Mb6}IbNO2-27z{4toQ%qm@cz1z59f~Oo+ecZOYF1W$@^_dibF>=J=SB zFF&v7h3$ISqS<$DbIqnwwP)F~;(tGR#~t+7&ne9{<3GOn*w37rKiyX)u19qL++ve` zq+aUVgP0lWe=IIOz3pTa*`=~&htHZzYN3-K&dhj{EqHI!BC&&4;?2UO1JCju`!Qo* z!pH331;4%=5lZXk*l+Rcx!Jd)JB}Tj#-o48`Qh0JxA}2bZhUyL{^x~bkM;lPIsTcj zS$MZq?Ty5>KW_c_v@z*Yy7#^BN1yiR{d?i^Q#b8T=k6;;Wf3>{Qqk-pJe+!OX%8k`F*al+3Q)E zy_dB8D<)PP7CJ6vZEN^rY1oY~A}(Rk_LhG?eE9mXZSDFKy+LbU{dj11>-&N~Z%eF< zJ~XZ=em-w+!yD!54SyH(-#Y$w_qV|3J%7Ju)L%Q%x##4}8U>r@oqH5be`eNQ`*A() zakYwF_Mf2DKMm7u*`q!eGaYyT^sH=dcJ9%A8U1sfM1Ib=-~9BO%Ga5<5AV&GZPRHk zxWczHzvbcV_SVVOA3HWmd?p6GtKye(5>sd2Z1D(!H>?W%~LrU$*{q$~sy; zHT-|cCg1X{v5&c%_jvEE`1r7-(7G@AOp-3&R>KDq51#Iq$yP4?bkagb*!o-V*6!{9 z{0Tg}o_x7|ZCvS%wfyTt`PA><+;Z{8fyI}9yxefy`Qa<=6!#xR7RO9~ zp1$)^I@_M(;`7Tkzry*8jx67J^_>3vHz!{l@YKJb^T6jm^Zeej8DF+sPFz35zNR+c z{6$)P?3-U-5>lnj?1Z!}XW6+|C&@iyW0P%>pSNpY#RaQ(?!V>caNY#9SJwPy-*xVG zHd|JI>(c%2;{R^`yXIl_*Zco^^xMAa-jj;AdcuEv^RgXQ?~kr~p`80%R%rDe<>ZVx zi7(3Ut+je{#(2@<1)nCoui(G7xh?+29}(qw`;0f6s>s=$+*7FP{_9QFW##O%lP)$+ zj!RB<4nLT@SXOObnbn)ApW4^go6R%*^C>=0s(;4EpRtdxi1SCyH+{4tGYjC4nH1w)_=XeU*pHA zRR&st`s%A=(sP(0Di!Id>8?LQ0-haL`5 zPyQV6_TVFH&)P+&%ek-lKNJ$48sKR^Z^5K>YT65Z?{q0&T)ppfLfLV3c6tA~eJ+RB ziuKpEtbefjXI|5nhYB{Xdo*OMe$+9={k^Q%-Q{Rs^L5XiKEqS;O&gCHOq~CD@_e~OLT0t`zPJr9zu4a~G-KI6EB0})_McU~E4QEjnD!@m z&P1Ebxq2^LpUu&?_^YI8YqggB?S{wV@-jA8jy;O+ORqUxUg_Px?oQX~8;6d$|Cw~~ zfx7jMAJ073JQeUIY zfByFSDVetNZ(OUx7hbLmJ9|&A?}KQKsgA@)YxWPfp6481*nWTK!nwSWN78GpZXPV` zyL)+ao1W~}nDTuGo~GOOOF93kd69iRJ(}m~M(-S>t<(9FcJDtgRk-1y^8H&I7JZdD zwpjdP^TIX79_>D{hRc3@QMOS@uhCBb9LXHN=IP}j+=C#u&Ur7&pzc*w1MW1<|QRZ_ykGiM$76|7nMo4_Q^5VxMvzIql3ChQw zc@Zs8anylb?XQ^hPP09aZOU@@0SX9o-RY%>+E-syZ zZr2Q{_4yBEBc zW*3^he#eFK#B$EFEA*SGxGpo{6s~Kb)wXTju35JLA z)xu-e>u&G)D0=a@{l7Je{`s=SGb+wbElZAnyl8jd!*@x^r|bVcX^68FaW=fPb9p*1 zPqF8InL}dCO$+bNII^@>@9ja!7o7HfB|ptFm`!e|t>m28W2*RPn$FuvEB6Y?8VR#- zx8{3K*!|$6INQ!cZ2f&r=3n{69i>%cZ}=veJxuHF|Kh9hr(;c>u)cbUsD0r&Ze970 ze-6#&&i$1*)q6(Dy_uW`J6dThk2l%s9we-8xyXb7JD@6LoIi!<6epH^5TC>B*_OqOy^PI8a0 zecB%#4pn`SZ5=LxTUh zN}cfUTiMSZ1dUIv`PyDI_jE^X*8H2vS+7@@rrJN+`u_IxlDBt@-$p8zCu!GhTwL+` z#wEqW`ux3DWA5DD+%|_dX?xP^&5M=Gzb&o%F}M7Y*7Xz*n+=E3?%z(W__Ar`{hAMr zFQzbCe0g|fnZIAZ@3)^Vfig#rUjMhovV!lF>(1%gOXN-Mzpj31BL1oS)aJ5J&ph9q zoC+FedJYcyR$W4=tail^X7MQ+H7j~eCDg)Wjdj> z_+aNPlQ)}u*XfD<^4&M%^3UYw%XPI z>1xd_yl^>uQO+b&Gl@@!uKjua-G5o{#C6-4()&uc{B;n{d)>V5#)q}*P2Ki={246H zU3-*K`d&}^k@ft>b~d}7tgw@-*SITprbJ-&mn&Z;-1r(;x#pPm{-=}meEueR_pjOM zz3K^ z@7k|xW!?U5I@o^LZeGOY_cjmL8@Ji*{P;lDUcD>USFG;iA*JSn(_^L>*Vs+{bNBA% zV=}u+4!c18mKZZPyw}hg;Ps{2jrTaeS-~N>`Jm<&n&Cec0 zUa>jRT~vO4)+f(-cK4K1`M2@CUu}2sY4ju=i|(sV*}@NAE8D2tEcy6yvFS-6or_Ou z{co}g%;ea$V!g$IjL#FYwQqU-`>e=kV|CPFntzQ|&fbaNWMBWPxwtl8=U2|E<##*g z^94;7b+@TXG3d2i6kb!2^YYxc#F#$|#ooP5xIXS4vQYvX%{cD@_0LG zf8WJDI~z9TH)ojFnf=u6uXuLR^~VmLV^upkIs5nh)HRbxc_aOs=hKJeV^uSx4G$&s zORYAuWf7N@-udTn=m#n3`A;+Zl3#2)cUg1WtDH;M+gH0CJCI(fZTBhXhlZPt3;$Ag zzkZpt+FYAj!)52qe3D~w51&20IG11QKd1(}R$TqzV9fIk3FUHI^;28-*W5b&_Tf36 zU0ggY?)U2sFJ7MJo)&*>x%2gTTi(A~9zB0fZhGge*yjTKOJ0h&?@4N`yp--+_tWvu z^?5xX5BlGz_z-CLV)EqDF)}rnWLG&suyo`@R<%Z;qb+ z_oJd$x8`voyK^ttL&B2MKmV<<+-(0Owf5SS-_J77E|<|h>z#dCMzdtJS)OyGOs)P$>B=Yb=T*!;k@Mh!ef^0|YtQjV3%iQH6hEILQ+FWKC+BP6#zP71 z`qy5TCmgiy^PN^Nq3iaiGd+>lx^~IB4Z`mCr~9?bMlM-Dp}V*Dht`jiSCbT8KM*Om zdU(&y;GBF}N{7t4T^nb!S6ogzvUu&3_L_~3XXWc`X1o#Ias0Wlt&vNg_k3W6{?ziHIwU!1 z`?IjVuM=Ztb~TpH&nyuX|;BbHwcZNk^MchBe;a<fM6%kvz+j-9%Tb{RBWP8MsTlQz4vdgQ@Jk+;$`w^RS@Au2!`egpKZU4SS zLF;Z*EPk)8_H(*=eEP?2J1maG?y0#P`ef&wpRdke{J$Wz=l0S0J^8z`Dr!qpdl%MU zO*sGlPu9hb*?WJa`y}VoC|h{#yXsZB{IL4pB$dU7`DBk@c&=S%DtG&sOrND<^u`w- z7W(;1ho#&}Ih$zrQt{iWy7t{3BlBA)_E;RZoxb&{<9xg7U2g=X zt$t)JEw-_Y{`Dq7bMEY{8u9FH**uSqCG+o(xOw>Sa_6=OPn_-Jd*%7%Ku$aka-#mX zl#gBfX%?UYOH$Q+?eu3N{`dTg?q~a*<*nNB{Q=LT7J2nuHQN&onFMf7R^YEV>GU$P zMfFmXv#!laFMhX(mm5{pou8!Cw0KUli+x)#i)Y(S+2beoORYejYD=I9L3`#kj}Zv-etjJ^8cXdHUCz3y zTy9(U(!!lLraOPryAYThsBCu2_*vuZ{BIq9bNZKldg48AhwbN6{5OyFo7}#Aul#J; zTi*9pv+^w)h4cQYA3LDM-s>2Sj*MA;J z-k7PXUsPl^&$Qs@OP<#`uXi3ak9#g$E578M!nU*VW*c5?TpTVfmKF2m<&yixQ~0FQ zmrFfu)|-8=4fKbt&%$KA^7mA7x-t9@Md_II9Dm7UF<^HM)ww0BDW z`0>Irq33Bz{k1vse#d>TnDonKdcH$!#w`W0eV1An@jAvOw#V;3n^b&7?fka0uJ$oi z6J0YL!!B=b{2;O`F_l@E|KOEnt7~Pf_y6Fs@34!!QM2gr;{(mz>)megwSRb^U^kGuB8+;_coRIK#HgLLkG{r}p$sQxX;#+eY>xkx8Lqe&BKLX=llGB z?f>i6+ibgU`LY~*_V0Im?t8my_WjL&@7<2yUVHxip9|k++5g}BvVF(L9UgV(Hrwa^ zR-gZmf7@xjT_x3zci-D%`~T$myLx+HOnNJR@6+kq`zx;g-?sN}Te+WgVJBl}XteA# zkskh!|4xM0eE+uNxJ}vbNALfyI(t1{;BJwUKA+{k-m`PQiOtj7`{m=`2@B@uoE6up zHA}ka7CUL{$D_A*r8K|DTCdtu@FVxm<&2so>*w?I`?AWf-JQc*WY@4 zeDkwIZ=c8i-}ZKQ-LLZP+4rh12gW2h99`S?mB0P}{=bV>FI&?;J8yUL?zEUk@n4s1 z&;S2w`n?^o<%e}IFE94C{I}-ioD_Yt8(B5EsyAopNBk~9 zO;UZO-+iEweR|C?i$}YPKRlDZf3dCo%HLuRTf6SoJ^SuVn5=oLcc#5XcHWkgFFP6i zq($^*{(Gfuv!}y0X})~!l^ZWVuMc~4`C@$T*Pps|W$SBRuh04alFUm`~I}XZ`%7M|4;wz`@dJ_--`RHzTWN>>+Ju3*4JG9RyzMf`L_I@t?zHd z{d*RkxBYJ2?Xs6gr{~E3-m>53=l!og?{E7%SN`vB_kBmV9AkCn*{fM_qV~zuJ zb^pp9=bGorS3ixv7gzf=|7^R<^n=F~`ET{UUDp^ZomZu-d0Aaf`rqmFs$V~z7)&s; zliBo4P=2H9m#wc)Yj>%?yY=HuVq0(f4b$(x<@UU{t(dU&?`M;g1-7c!+Oy`axBI!3 z{jGJ%{E6GeKgylh{Ore0#_gY5d-}gsJ#YRM_;-!u&VNVEWA^@iEMK#A%XxAC@R*{D z-}WrGP5k}I|IdcE@qh20zxD6l|Ie%5PQPDq_RL(FQ#Y^I9c#~z|9(?`W9{euKgYg( zy8iEG#kD&dV}H%)-qv4xZSwZ@zvuMd*!No7F7x-R|F6ZD_g0$nd$Ad z+00!MKU`kQ#v7-Ur04W+Nbi+d8^ZU&@3LIX@q+lb&BD*@wx}FyOunRgi0Q!P^~aol ziOtw`__EUB;LDREDsKHap{itC6hHUHqN^@DbZ#^JOnSXMZ=b-vN%=M(+j!kQk6+h% z##-}WjqsVi|NH7D%(XhTTuQFGsJ2jYzS+LQ4#nRk;X3(SPH)_^Gv<-vkBT?9H*EO0 z?2m!|AF=-TQQ3?COnd#~*9WDV`bihU^#e9;7PsHI&HS$U-LlVRujg*h-F~n1_P0Li zb$8?bZo8gy|Hr2M+jU>Q+VA~)`TS4y+pW*%{!DzhiCh2fzF*1vYJNZ8|7U-m{nxhX z{?&E)JO5sL8~*Rh_3i(krSB`P{w6$c=ijT(_of`1y#AQ7*uJT=^LAy{#ml|tp8wnO z_UU_Hx;NJxEVi$HcE0@Oi^$oxKHb=rq+EA&&V$Rk{eKoc)4TiSPvx;^O}w|SO=?%! z%Mn)ha^Z(fg2}n2I}KNV{TcA-{x1Xln{TT>B;=lO|LwK!>Bh<&KhT(0!_Mjz*LUu% z{u7xTf3y0U=%#sg6ViKad4&5)Z{1}4z2pBmPNOS7C!V!kTl{CTO?J;p8R^Bwn+jKb zJfe_hy1!z@9tro4Ykv3GDY+kA$FG0rea$iLvTt+$AO2Yp)?@ttn&4yE$2U!uJ(7Lg z`*r=^r*m)H|N0kyyY7o={@s5sr`P={dsv`qH)DEi&B1S1=YQMtcK-h_?{ny(Dx&}L?|&b@J>5R_kM~P=`)o)6S)4j^Hg~)K|LgZ_UYGsV-JV-t^!E3* z>3{Fd&#V7m9k;di&*u4iV!t}}+iIVex*waTCpCND4!O?rvmRF;sM&HYYIeo*UvgXO z{Jhk6)tFZ-P>a8>Hl626+vXD=Z*BPX^0DU46(<9%vT9oSPp&_6*8a_J;o@H@ddY9D zHSPbF`0}x}hE#vWA?9(!~0K}%vy1s|W)$)A}rCAmxfbX(gbUl04E5U}sT z4jbiC`+_xhxZhhI(eLpqwU(LLpcz<4I-MO!}7VNu06KQj_U*EASWLwCdJ-7Y3T*&SD$L_>k z*~WhI$~i%ss-sKh-`SV5|B~&=&1_-MnPkd5a@QPP9QX0)=D0~+mv_u@&01=B(9Ph* zwBsMTo6o)4UC}0ZQAS=`;#gGA{eO@5TmCwheHp7(z5%WZGl>mK(n zdnY1OH{t)$-tD>TD-O@kpa1R8^0S4HI(Odqa`b6r@SmrOI)Ae7m7o5$?sfjndH;TR ztKSBN0+;O#!5kLyY<;rr_9^8qq~=9 z*}ts$b0e-E& z(j|Yrd&dmfs}J=XYd?K}py#HQR-=8B@{rA|p_`0sbeU0*FFHfk+ z{?U7NOmW$4{Tc11554MWV`{!N% z`S-Tf^Zv-3RTfJB_(A+(?Wy*fJ7s@!&2#d1y<2O%{Qa(%yWU=&_p|f%{`%AZ&5vJ~ zxK?u@Sm6HCJrRAc4A)&_iGAIb7qRtZ*q)>#i@Ec2H*4!9E41I;lg>OnkDuYQhq)58wIz;GoS(#jHBMs^8Cd&TDKhntNOOn?%;!6}LCP4gdXV2VeBYOS$^%e}`A! zU3}}?s=agU&V0*F_bP00oqo3Y;~GKzn0-I4w#Co6mD0OAMnXIG@44CYbSny8Z@$`p zLfSg+*R4OT%jdr@aa!o?CV$ZW#DCV?{{PO&=j;DGk)CJ&{dc(k8vp8hQ*ZbGzb4QB zbLRT4OYzB7FSX8`78bT&;IGzAyC<(AE#m`5nTS_$0^TbuX>HM%OqZECl!6XcFdeF z|H0P!@;{HBy4PNC_lfzGZuif5+x>mdwz_|mkl)M4^zZ+RyY)uy{!IE{r_p~pfs)h|aE%demDdAfe>whsQ*{IXjqTz|jZxN!1&VQkfJP_xEj@;*>OvSs=;+rn+v zDu17m%ie!0yJmZQzkT)#tJwn8zXHpS3Hbfqxy!ucclgosX+|JmdWQ zi~r`@@6G#tBHiX?#=dz!pJ;Ep|8H`0{2>v3`<#34Ba7dDy?r|F_qN5A`2A>z%(Q=Gpm( zo3GOkw>O>lo9(l3zTE7qAfKi`j-34e(X{+Zf%L}{7|(!dIcQqRVWb%I1v`lRtLeS9x%GX}ERlbd62_Ea&t( zVb$y_Tko^#-&T{3kIrE(**8_3xqiis)Bedy&Fj}+(5}m!e&?Tz{QQzlC;jf7Yy@W;0H19&+xrvLP zxyQzB6|ImBxE}Mr@9oOz_LYAoo%*rw?@#eO|DA6W}QP%T~$P_aC_s``9+>cl+yY7N^?{k0wvIop;Diw#f3}WMz~31gR?9 znyBQY+ut9Y{Qkh_*}p<9qk>84y`MMK=IFb;-T1Y}`+VKu?e}lkTuARNUcb5f{yFhF zE8(9P!q@-a{m*lrX~E0RsrEHQxTKQC^)C#=7_;)9&| z{MG-j{{Oq`?XTJV|5Tm-9A$4_&o#4b%Kk^1$KM>0RTaN|v+CNJ-A4;F-2cq~61n;P zL1TUWr+jORZ6v>cEPVE4m-taJZqdG762)oBGq!v*%;r{AO22(*;qom%W;`cKke7O zd8Wr&ZhQ_rr}k|}z~^SEc*|p-r~90fx7zf(<+T36TX{13B|0A*SR&u5s@CdPWMnf% z|8iM^ab1SwpXg(^-(}w`zWgole9fhAE5mI|e;>|XcRO##&uNQ``RCnH+);F~V$S7f z403YvUjJ4fWoTZz^-^oh?Fs>Fi<}qFl;^+nefsg@jT4_&Z?ij9FOlt&l*xPWMdab- zL1+F<*xWkVM)uFOb=x*hR4h&Qih??)Qp`sB>vaFmgK_%ywW+Jv3oo%gR8{KxrA7C3hdF zC)Z5)sLdwv>(7e6vN>B@9vv+Ic#8jTQj?=Nzu3Q9WsR<4GCLy#q?g3k{Cc`*ebbhm zFP4Y+|uYHb9uGuYv&Ec2#ro?P}+0i{;I)6uE zT-#Ht{f~BiC|LI}rF*^n({9;)P3>~^Cm(N{oi88Xzw>q9+47BlPrBtFx?g#D_dWgV zB{wzRoNcz4U-pwdSHJFEy3EIyKlQ#}eZM!Y?|=8Ys*m4}zyEjn?Pj|#uIz@g&%XBG z|C@e0zWU(veEV<5=FfiIc=ceaOmFa+B7^IXtV`*`8)ngj8-Q(pYD%eLArcVhFh2TlMBrsP>zuj&uC0@2EbuetdpL+2_`a1;0Ms zufJXPwbZ<4|CFNVf21GY|7yJ7_-&W$v!{=L-@E^J>-@cU?G9DE)V^=9@!jjl!};f> zB$665`B<-R>+9ZZy7<0THYg0%|Kc9@ zbS1wZpv?qNei>)$$*FmL^4<}3X*P$^D{esh(5{#pL`?CM)G#|~_+vwNxe z>EOFo`7cdx&(?pucCt8e@3&*+x#uhHT;6^?_V=-q**!Nm|9qgm?v7pVpQLB}``@V_ z|3CFj=k4V-j~}aF`(nK8kNuyM`RD%F{{85G+wR><`MrPtS>N06H(mbs)ZpWqZ{PPS z?_0m;&r0|B;+bbYD)MjW*;u;Z{UmD#wfT9;YP0vTzxT-Ba`fE3!xQUv9b1@kT`Fy| zU$*nQQ_}kd?Ub^8H-&$*ttfbR;^W_&%}>kLpa08qLw#RR>iiqruf9#Lx|^KxtyAiz z{>ydM-2G?d4z&xeEBh_NChIbvr*)>i75~-EZhQWmc73eD{zxFWP;~jO!V6c|9}u>l zU9#o+a|S!>J*9sVk2b%I(VaKXRPXiA;Ln=#SNA8i9i8WwYi7JH`6qw(x|$82Yu}si zf4ukdy_q#{f9Kts_w&fCtc3&*#HWgkK$f`woaCps~3EFvVU*M?d)U?ANz~deg854?G)`_-@mNUOiOm}jf~+6 zy6l|v{QbjPZBeg5t? z8T&Q2p6-`;`E0vo3+L{N(>wpYef)mWzLY;V_lN(vxO&A|?tkn5om`(^|MjQ-y}EC{ z`weO@ozCAD|1&Utm+gPe^LOh$e6O#me!C;0{;&VP^5FJ8&e6B^L-)O#`}omc;rj(5 z&3{a`$^ZEDzxel|=e6g)-QEB7@8tIL_g`JM2%Bg4db)n$^!{fIe71%UR*F{L6I*dw zvijGrhpRR}cDJtGcJp#__a6b*<8wak1urE|H~J1LX7YV*pa16g`^3MD{h5o;9erHA zvY2cBy1&P+$vJ_8XyQfZil4`#YfJndfPzS+&EU(=MTfT;GwzQ#q?kT?Uw6*OmDalt zrLp_px13V1_gme4Gw1Y&8xt4)F`1Win18id^q!<8v+vKe zZ&0qeaWwke*L?Z^i?>bxvvhg>`ij3-O_s0S|MacXALISs(~ZTdexCiNZU1!D+r!8I zuXxTL|99Hkt@Yo;Z(sjEJAR(klh>P<{Xac_j#-&yUe5Kq6|aBHei**zzRaiXzjF3H zn)?5f>bp-DWezM4*So^Cck|aZg6jXzzTC$ye?!h>zWUN;*$Vkvx@KGJ#Y#B0E9h!$ z@eAp0Il~lu<6(3F$NFz~Qj!Bxm)}2h#5vcm+n-Z9zHxrs#sdfES1dlePU2RAn9=V) z2Sl%TbxxO+zx5&-0$y@`n)HD z-zR@#V)f-YevbqEc?@^2=$oat|L3Lt3=)#mxB|_I}Vz{}+BR=SRT2k30D9{af7sW6@jpW6|$leh-+& zZ{OP8cvjs0zv}J&{r{e)z5Dvq(eCA}-SaCi>-U`g-mUKU?Nis6z|Zk~r_(>4SufDL zu0e{V#7mR;f_;`@Y@#HI*9kOg|l2Z|qVy?}I*{VdmQyWqL+x7hIc^{W(A5;BxR+#^P zb^TNR?c%YePrn^nZvXJWzYA^rt^cOi-Pydo-uzzG>%+m%lWVW_=$x}T;r@8C^RA+g zhdy5WtNVV>zrSzZzu%k|tG}UFx^ClC?SC()ZGWcp_Xpcqgoh4Yem?)~KKAo>LZih$ zDgA%-%>7<~)ziNX)%WMCSACxAe4lk+_8Qgxa-|` zJlS78CfBM%uI^XXl}94%|21s4>4yAk`#5o5#m!5FoVsy;Cr#%!n(^b~7SXz^H`V9= zoA%?<()%+$99p{meZ2Uo%o`L9(CHPyfQ z+-){5Gr!flc4Le|-ONv)y7jH4{wM42OsYO7Ja6mYui`ZazdhpredcX+-M`!0%X|M9 z^3^|k$iM#g;KO8eUUT2~AHq$yO`%cl48Ory+MjO8DW(|`Lu#PMGcB{)WXx8DZO*OOiYa+NJ z-+s+KeMP!>{hVL&SzGeYyv?k*wkF-_`D>ft zV>_aj|GK6Ybv-BDanHG?bFFVC%f~-oTmGv?VNb!!*WYV%EE1=u7d@XQb@%dSwVD|P zFAiUNAK4tQDXw?j;A3Fjp@^TwGR)sUecV4~dHHkM^1S}NUmm@+^sjyM?fU;y?Rol@ z58HF@|2e$h?*DPilINSB$-bYRzvX}b`k!CkO8r1{qp1YgVMV#x+cq~p1b(N?s4nmdc*pcN7=8dB=7$5=i}T@>b7;izdVoM zS^Kqn-K_u1eSbZbxYo70_sF#!4>f95JasuYtGVc7f^GJL51~Deblu)oSAi<4H+I>6 zB|q~vM`=v&;ojQ2B-e1Jg7D2F=UlDTrtCS|?C!q5)m46fecG32*Ihg0WVbfG%S(K; z-s;cGXrB&VuI`vl^VbjMH!2GApHf?MQbzvk)tZaWw~m=fZMNUfwMj?FPczyq#XXoqlf3K4N_v`J~{Qu3JPm3b&-2dtQ|MQ>Llkb1* z`CnIkrd2#YU!s2Q6VK^3iT{+IdB=aQy}e&9URd+J!%;1rb*S_@GmiN~-FZt~7=)~oX>|1ZoSFgNQlDFaIorc-} zH+%@FHH|-hy>g?4_oRzk*$;(GUd}x|zodSislgF`|A)-yKE@o>jr+Oucvz-Q&XL6; zv&}5!kBYxHj{k7*_WGLB(;MbJ$)edX(|x5NLQo`1itz9zRNFJ1Td`yVIo z-?~?J_v`KQx}WO(>1SDQOaJ?G_x5DH{S|+do-O@6U3vfS>Hi*RxPyXg+oOQPU+zEe z`(M3G$a3!e+QYiHXCM21=Ksa-|J&b&->Z3hamMNXud4Hk4ur`cyteH0^snd47rj5b z+wN!9%B4MueHA(za-)PSjDx>(c)$8|F{hs~**_rrh7PDX9vCEb})$%vF z-!gx1;q|!h?`G*5Z8JEMao^?EslS!iZq)x2Yp+rWRZSwt= zV3&QUd2L6l{l;~*RY`6~m&VqdWtJ|oxOMVjs=Ju!wNDfOCk2SJ&n{XqSI_eQQFpfK z?=SCRuYVl9KgBM+=l#AHF8TXZSF)Hbtnk?6r4l(=qjBrY3)ipv`PUot?_2NF_F!V^ z#FX3DjgB<>Ec({fdw;=6(IT7ND;6KN{?x47iQV;6$lk`q z?6~!SZ{BKTK&YpQWuzaGf{+lUD z9QuWh!}V_Vzo9|rSG?LQn9**7bH-<96|7S}en-CcFK{#U`4!{_DvHSK>G2*xel zn6kX?_t)*Y`SS6`ikF}7j(9p{?Z(Md)8GEu`cl#2p5JNqn}O0@LWhSe{3{aAV1=hn;Tr=JS#Ke^Dm?StT%H)@YRDBEr;`I>9=>*S?;<3sCt zH&;Zz$vP5uaryDIe@#yhyjSw*Uvuf<;dUnL8l%ER23Nf*8;y(2N{tiPn~`1(b$dy+1M-;gq&w{7QsYcKU2Q=8XZeD%iL0)IbL zuG!bE8Mm`%{r#uYRvtW?2lF5yWyponyJ#_aGf6?mmT_{;r=7=^U3S0^xgh` zc&g}Q(IIpHaKGKZeeCDgoPHK}Po%b3CA@q)`|Qbk?Y0|#aJatH{P}CS%WhW--nwtC zZQZg~^2)#L^>f=m%Vvz-{+y`LO8M6MQL8A>Zd3B*dx6?dS@~?2+KQ!F{P=lTdwapj z1lNZbxchtjZFYHCWZlSW?frT1a-G3E)1S@jb;YKfKPDD`|5iZv(HUD_F8)=~zYVkv zoB!K`7!J_z*X(@mlZ!rm*zRyUBzKG1q*;3Vj=oY{ajbtiFZb31A9s~xrC)aRn4Ujn zjVs^w8C}7=%X?X8+ukdzsn9l)&t0;W*W2dJ$-_aU zuZOgqxuaaeBHb7Bc;n^s`!>E@e5|kk!8cO{8`f!3d`Y)0z8t==Ev~2aebLFhq?4x~ zbno1HK`OQTl<&SL&AvNovd!gw1@6>(R}@&&JJWXVkIix$zc#b({BgrPEZOnb)5&)) z?s;3iA@j!T;0cV`-y7F_*0cKNm>n-Imi7MO+w-&7-$y3zo3**JP4B$a=Lc(EMFz|J zFT8)pV)v}=lWnqZSaJV+CnP7AQ}LwY_01Kzx+_Fk7JprMzOVMVq2xR%`Hi_UzY;Sx z8;k9`dUNH=huxBCzcv^C2+aHRHNZCe$g$~F_nZ0u?Aw0s>E5mWB|o=G=*X0IZ*RTW zblp7j$*q@@jvtRUe{e@8x$si>jZ5d-4{W`uXt>}@T6#?ivvU2>18$MIM;5na9~6yd zZ#e3%E|ag2c;Tq9b*zoaeWQM>Hz%#X%wF?;^=5J5>W`m#+qWbhTG+h)NbqLG;uVS( zUj%plfBZZ_uBZCOu7vxRuU&@DPvoau*XZ6 z_v-gQZCF>9yngKrXW{mH1h4$LB@kT`;j#OUc=exS zYuY63Ebkm!GgmKWV_4UGUb+8YKHgmXneXSA-mmQPTqe{R_pnn%r`Ei zAD&icm&Vg>m}!3Q%ZC6ngHPc+Ia98ton`#Jc_;hB_n@N8ci!64d-v?$zP@wM@vY+9 zZ{bFjTkiKvJ#*u&ca6;yK545TPubT0^(yl}FZDe8x%}-So@mE&5-jU2K6aT#KCj*P zP_)>lR)o(-L`>SK>ei3%-y<&@FaEgtdQ9f$M4n@}cWhF2ewx%9>Qi)5Y`;wA@pJku@cxOcMfqw@pt{rem%obn&&GUuJ^n5-hJx;Nx*{uF6 z{kK1uC3DcD?un82`PM1xJYT(a{ikBQXf;2--`hW%6oY4TJUN>;+3A}9i4uYNN$2Ec zg%__sz~`;oCT4%P;>SHdC9{(+F6~S{@%W-{-M;7Fx3gb6B2inVkyF^x{C<(k_2^s+ zgBNbMa(=yHv&mm_HObhnG&kbIU#IN&_?d?MvPM-$54nHbm|1sX>)CZbe%0B<{z*C& zcZS34(Q$r_s+3>H8WNrehkbmows7M?W4jP}|Jie`lDa?qP+5NZ_?e$4axIs*FB1`$ z>-jEySaILRFVE)tSzZr&qPP1|bNdlzzUKcGKaS2{yW_O@yrB7c8$TV?uQGXe(DDQL zsHj=C(XanxLEi^dQ%ru|(vY=*zYf8Av;dWpx?U&sFRD2j30m$|yX9hH=zHo;jq~~I9G6-r%JJ-~Idb{i z)#eqQ>hX_H>)&}X)nurjvhVk)E3Y4X6wtvzqTHop>H9}W8CLF zN3?q`x4!g@q8n{?eZK5|)Bk*mjkP<}c7{#lbamV|1Ekw;d*-Tnu`DS)z2BzA_JHGUtgW%@88E`V8CPPDIgFj zZm>{WV@5!r;cKlfM%KIgy^MA*V=wWQ;K+`M-Mo%#N7JLG6q%wvW*%dodGF@E`!_${ zcYpaSGvl*wm+j84+W*`#HZ;EWvm@WW_1>omnzP)i0jp_DcWSq&V~8uImZnvwtw9_g&m`eS-cz%MNBG zwWog7-ruEC6q2U#Sri{&uJ(Gm-0;GT3+xL|?|Aau*5=8Drw(HEM^+arZlC=7__CzO zam{l+9pp3XU=5PDJFl?&VLa{^D9S3S};PjS!Yd$c-eRmMkOqX}$$0-inz zoIW2vJ-N8>>;a|o3qHQ{$hmM_anU-)NxUZ+?3!Ely;-olvq#3Np=XVnfA;ZFzAZO;n^^>{f{7V<|w#w%`*iHmZr9tU zd9m@ye6y+vzbw)_nVtG?oM4p|+_pit_U3W1T~j_hX*aXdT(>W#bL-7#3qBM_-sX4_ zv;O$!nA@B)>Sx`)qWJ0J-nO5||Afw8xARQ)(|(>Ml{$817o5v1-kaV~S9>Gm-)CR* zYbx{7TR*?DT5`yJRG$1;!TQ97<$E4JuRRsJ?ZsigK5fH-+Li~&{{Kp9kMPyLQ1+hv zKfuG^@I~C=uj0Eux!eExTY033r}jyM{VRc*pTRa?0*~u|eYoTL3AN`8|2hTdC8>)2 z47cm*-xT=XbgKdT!RZ`^`zCvpIScx{O9M3-G7l7ym8}) zvyVNESo>|SO*-6GB*g!Fr|zpGPMvE!Y@W;vdC~o4sddVh3(NVXPaIA!^zAvNpBt-@ z`qUP>{M{l`>FXTt($w_hl4Ye+~t!Ot6#Wu)w07sWs3Z_kmdf*dH!wS z=9lz(a$zao$A=BepBy=`W5b7OwoljIu=xJsae(}{C$lY9o|gaUbRtRKtL|6IgwS^@ zH~&zsekr4ww8Xx*_F?eP$L$gdUN;Z$=bq(#b>dLC=etXe{bugE=MPR<@&Ae_)12s8 z+{|;I&+7hKGS4_%;h7-&=Si0Sdb65mO`cVL;=p2E%N!3sn>SClt>AV~aebtC|9{-# zojp8dOP9YEs#z!a!E(i6WwVqi{hv;+mp^eytiM7;`O}9?=Pjx6HAj8zH)`6wR9J53 zb@STZwmTONntgo5%(I@qL9O14FXiFtBkMqszVP#8t6x)21y*>q3B~>ll=%>Poc&`{YtG4h zIew!XN_Hm`oh=Va^W<8cX1g#+nOCH-@B&+n(Bc*jpPCA;kX>xOZ!M^OFtA(qFcVBrhpEAiqms zhrv#hg(>m1Q*JOS=e%lY*Hg8ca)Xio$B&?%DOt5V)s`+rjb;*p(;Pw;T_9tG>a-Uz&$gU~w_3OdvzI{(p_~+#Wg!4^u4mW#| z-ebb|!&zKO{+HY1IOm=pht2IJ+lZ5*wb6R?N%m{2h|4sPKNwHn0Ty5rcXv@F3#D8*|UQ_4I@9eQh z7tYvnmSy6U=xNJ~&U|)=OJa5SNwoHUI>1)@A$59C z?yDaSM;7bt{_ zuKHQlyw%U7n_cSvSMzdfKkF>f{9I*se`fZq^*0u;-RU{^H*>D+r%v9E2$T3?xql++ zyRK;OynbR{&cf9(M}nneCM@UW{TAnZZ_~;@Awu@Gm2xMtbsl(!{S@}@Q~i4OZuW!4 z+Lf=KIIMha^mE0$)0+40ux;v^ZuQaO#mbO1A7046h*Q4(V-j!0kBnp0yp`E^TG^Mr z`of>ex9s4`5B{#*`F=`ly$PIM&5hR|6dGlHI&gmS1r?{T7G1uN7X%Zd)%W@7HL^%p zRbFwAHY^l6x&6lt-;JsNDxG@xzf5SZ>TK_h4m@<&u5RPa!mCR%mH8Fj6adDk-a7Gd)wERcr%sN>YrYm6tmhi`T5}{k$uOQEN?Dc*J@+;guV7(uE(D) z_DBO(_BVwS_>E519h@F}YxZjG_?seaQtfy2jONTga@@DQF~co$IJV?5#WXU(CzbGX~#0td`8!cje(} zy|-ttZseZNd?_MR=ECIIlpWU-!XG!vSjlX1-v7?@h_da=6Y6Cb7yHImyr`KZ^KHfT z!+YBzTzlq!TFe)y<^L+ddEcR?NzFBiUs5MevU_>)nM0aE$i+v|jM)VR3lc6jE#?TQ z{lMwM+`jJClrP?G37$;*CN1&z^y~|A2n?UZ#4)d7F`sqq6+Tp)cy7}K^=GT6TyrEdeF?Z72MeDZxTI$a^ar%>os@1iY7uNroGP&YKfk)iO zgU(^kr9KJRJbE@E|K^iJtlOmgb3TS}TPCve&lV_sdC4#N-gN$dC$AP4JdR=1?iE~m zuCT!PDmzn;V#xXDmY;*TWzIQ2Gb)ZQp84m6yWdZvf5mMUbe|i1 z$ZH3O_m8VHnll5m{{&xrG<$jF(HF~?SKYX(GMONR>b28(F`56@C)=nGijNfab%VZkMpddUeB}Qr^vUAoYq}c$?cviCNM4|? zXcYG0BWL`!PD#7JQqs3>@hzxHiB~oHlGr^-{aw|qwRX*QmO*+<#|>fAqGMr^UnrpIr4z4*s#+_l(v1mCDC| zob3P5p#J;BlW!-O_kVdk$1(ngw%D-+0-t1b&h;KVcd%sB5C2$&1(pje&;7Y{Cy;4Y zvCWMOdlvc^f8s1&e!pgt>@&qpQ!EqNch6eBW#;!Y!5y!Ki>2%Iy!Ssds`-+6`dq?0 z(;pX@_~!p;f71WM<&e(tCc8g}9!tE4TU^7o>-q!nJDQ@*cAu=u6gFHe6_)*RgehJ^ z$hhX0Y222V>fuLqe``%#sK0+(QZ)O&iA#O$J}oZ{sQIKk|3j;AX^w((&V(mZ-R1ag zlEnBUUtV4wb7Z;h?JG;w&s;gSY=dI_-0xR5S_gfXN>bK)+82Lx&H;^o8e-=U_I5Og zO7OO6Mb&JcaH-{j=V8u63prWkZw0U%bh*ySG{1RrYxCr}b#uR*c$wnNXUEJvae`mg z)~CVh>{eGc_R9SiTIAAacXRn!M$dM$T|0TY6%OoK@x>;c??qJiyk`uhe9uyJ?`L=P$8=7u z>eb-;?Tk=|b;|rjG~C$lA5+ zsZnwJ*lxCI6RW(O)e*M%Uq4QH)>L%$2Gt!6pKW(@O8hOQHxK>Imgw8O2%mQwD89i)Wm)mA8J%D!=G_alzxN@(tf-b&1_)IM=Cs;InXu zy0pzJDGs()*Ck?WzARqvzxc!^kFW!CNTCP{=S8(f7 z?;157BY~yAI{R5=wyn6loS$h@c+VR_b+$K(>-Vhio-gFoK6$3l?i)wc|8DA@qo%ZN za?X4^y9{r$``>$bzXm3W|CEt`b7-;4JgcHj?wxrj>jT>#EdBHQ7^f{q?9C%`xqTwu z6Xlcp>c2eHjQ^5qeDd2*Pq%J~4W9L<*00dkGYCHSD08>3l<_0Yq$PcJ>@7O`isLKq zo$#3F|G`pl|0hkJr8kyz^VM!qJnB2!PH>O%+n8onGM<-)=h{L#PGvlz)FEd4yk ztR?r0;?HG#^3|W-TmCp=$>F!`mU}~St;Tc9;?~L)JCE61G?lE5VV}@AL;J(0%B9&Y z&zAi%VQ-cBHnG(|_S3Fqhc3%p{VF8+Gt}Pb#{sd@luKK;>DxUKRkk{j#9#T;HTKt( zOBcU#?B4b5(t3*t&TNOw>X!8L-US8H;&8u3&Vb}iXs9gK64*%zRFdzFqYJ4)=s^-nf z`nvT8*$Oq-zLc(8w0z@HKC=f;|Fz6b3Y(wg%zvnEUi*{Mbr;v)D3w!W4qPQKS(DIh zReMCk((7mk-=-VRag~A7?VcvASaftdgStbLfMCxgm$qF^zKvg0d-gCbj&FEjd#=Wi z&#tAmafL8fzygZ{GUfTA@4R`m#?tYt^ zq~p_nM7?=t->|s5PtWMb#6Ge9){Rg6e?FQ$f8F!UvWpYNPX3q>H;?^WiJ_za#wn+o zmdkvxD|>WY-0-B>(l0NYCmdFl)ka}EaSs|8;cdm z?2?}z{h3<-Cf8?F3zS`0zr32)Qegi&$t|tNgZEzN5cYc0JeTj* zA>SE6MZSmZQ_qFhJeVE7Ipgy2MN_hMT{?MG|NfZh_kYe5{e3@7U5{!;UlLa9s$Njd zyKM@yDtpzBL+f$^wcLLxJo%t0`^)kGv*g#y%hvswc+hO=7tM){?Y6gM1hyS$ZMG_# zu=3how);v+jUCZDOb*M0{F^#ORCk|bO@||!kB55Wtw(`IKV~&Bte?ztvnC*7!c%pP zT>PoacceC1zE3M5*9gqP<4*~|5o*b#Ya_r*&40zic{W|D$%Vy21<4n zI(rnKTnOa*u)*Eti)Xw|QM-DO-IrwNZ%g{)`F>q|ynyeelB3V0aQUJe&HeifmY<#Y z{`GV|)!GCV+nfXGI<1WR9&T*VXX@fSx;|9Md`9r4c6%?ew?8KFee39*v6E4Wxp8^T zrLc8fvb(QLp1){aT}g$PA7{VNp>?lbTvWHq{kc@H$$fUS1!&kNEVlNGa(AxRua&~e zwO1@fX8(Dj`d8rBgkzmn-z=uIS4^4LJo}kK#f(?XjT5cTf3BQyetM1F!Sr`dvlct~ zex72sZNgO9vw?T5A9%OfJ+rDSx>s;7djqea@ked;id}XW+;yfrTDbhi#89yJ9t5k` zLHuVswYs-tqiXe(A9wHUbt_akepEhXs{H;iPuVk?FO~hhavpLo^6h)lkW;tbF)q1d zP0>;P+Cxo~od3Q^m@Xuzsv=jKbAkP!_`EGEyRWeK!+mmkTh62QA1_WPPpSVIq`B{r z0N**1AC1Bc~T&ImR>CY@i3eq6syH!x^e{wiIxaPn=z2PYi1u-d6IA4qAsFj@Y| zhE9Gp#oSj_rxx0z_6r?SE9;mo>}zpgxnHO3ld1ewUpo8WZgNc!WUqRmpg#Mj%BjDX z`gxD~M~e&n_|=s5O8=Hl#@AGi6jA?GBJ~jx#;Zi1of6Y;IyB2n{pg33iPd}FQTh%-||9y{kSuf+Z4&F=kVV~?CE!LfV zA~Cu!Sj+F}ijA)B+ftoxB(z)ob3NfOXZoq;jfdNQmDrtH{-Z1S+to1pnv$0A#0$k6 zEW%Cf9;B|4XVJ1fZ~6c8Pp{Zw@8tfTWAlHm_~~u`X%h3?f7kg-US78mZu;acExomB zec+oTGoG`_?P_b_EKRXE;QqIVx#~va^!wQt7WXq-{$ks8J;P_ByLrE-Mbqj(ipsJ{ zlXx~R=+^mrq?J!K_oX)bQ&&#K#Wo)rWp6I%XO5ife&y)&+&2Ns`D;@)y5uK7Go@?xB(wISBYLG@QliZ|JY8h}E2S*Bxa0Z*?eopMHk?jYv+uB&I{niWnZJJv zL*Jd^l=P@u`FQiRX9|X)UZT4DJjEC+i!QL({B7X6c)l?wNxG4W%e*)>{n51p+I}US-Q4U(GD>qfE``S3Xi?#h-*{oURgK@a z7(QEFmel?kAEuu6Vz2yks`%L>$GjKH;h%1nJ5Jz#lPE5kqL|d|zS3sGVYQ+UPrbig zT&!n*;!;>lYfuzRa`=y=cIVui9yK{%ANKgyt?aH@;jA^WbwhXlqY!CHz8_t!>Ms(! zqa&31=c+Kv)ry-uZOZMaV(G)?K*}5=PO@0opS#5p(N2Q?%TxR9x3j* zyT5q){d=&mUw-N)#(M%aeM{n>_-a;O{e66{$nSqE_~#!{+Wpmv{qvoe^|>r%#EWg=>H!kgEliqdiA#?SIqWdonFP^eyUH+zi z`_ZXxf~X=sWbL&^n=0`y3!vmOpQJDaM$$$UzPkf+}?jqsoNY~9MciG(Q}#| zU+m*z$(&*i=SR!ueZJ5-W$~AXPm}e3pLlARRkP7~{*ws{&oXo{52yb`CZ;39=)T@Gu3vIoyg ze+uwRR`8hpI`LX%LTrUpz?TxIi|Z%1>=F|?amlT(*~)H%zkG}Dk4Go>)dV`dxa>Zg z*?QC2;C!*UmmK5N^~xqTPcg2Y@Z(01Ep8x#Gxj)xUyLYb4X13vbrc*TG*|v3tk6z#6jK40D zVwvK9U2^J&i_ag-Vhda@zbO8C+N((i*^53YM|%Adk^U?5$x-Kt)BaBj{6Aeu?XLTC zaPc`tMQ;DH;+zzA%ah8df1YgY`y?3m;X?5KoCWRmpN_fLt)J-sQp@h-p>_?+4NGTi zoOxV1GAAJX&Wm98c|nDmVZRhVZuIm!q9#4%hKgRtu4D5mUQAzet8&eaCY5H{qpNq- zD6sc4%M~u{uTj36trb->c~1jF!QSJNa!VYAHhf_4I&kq=+cySdjZ3Zi7fOvX6N=|g zIyk-LiGQ<){5IFF=-*wL-6GMKbdy@Ytgc=!%#xtz00Y%z zCw~^1{iAip4$Ey_-4-jB@7Z`gJ-X^eQhdD3RM}sjWmX6RpFVrmvA6y7Ew=^#E9cp(C>EV9KNfl9&!S#Emy?zU zcOE>rM^N7KiTkTK<=tJZ`@4Ae%Lo|0yz{iCwV$sv$-O=1VeoVvqb-(ZQMc#cWSN%I}Zw9WM%wrpS;>BZMKG_;89kD||6zaLaGHkLSbuU~#$=R?F2$5Volb60dto;-_F<^B!DIKiR`Ma$zWHQ9be!oUXRx*3CI_6wF%2sva!!(g| z!atk(bzdfSo3(#(armJG^xJui;nv&3~Vw3Yu3+nMVg7182QMSo(K5;^?@`<5V+0lN!PrKYs zw>|P_>$dOFPgVkXtnc6VS$}E{s8zjC@Q%NCjJ?LA(nCyclgtSZ{x?fLhrBbL*DP(H zlF7%Bes2E1;t3m9Owo#~G48NPKM)#powMKBSLv0NBgYhB?%KIJQ|6p_vB{um$3mkM zb5t(*{@EuiB<&TaZa2f>grxRZO zcAAzc&KtG;e|*uFdvj><+-+CC{8SMuIIq9qgTj}EOD?xdKVF)vI7j@)()m9&Oq@Kg z>ol8DSM;{dZs)R&RVt;g@&>xo< z52ni~8cdYS{bi!2wNW|VbNPu-t1p^mUSFP0-xT>wVaJSD?4UMr>DeRy3hrj_C^Ps! zqd%rtGt+qgitK+Ems=?-NLKA&TDy}F*Eq#ZqJDZi!1b%j?SKD6iF zoVf1RkH!niX?zda_-i>VZ#pj9FZ;CFX49qMJyT>hExa*Rz2=uF|Hp%i&nnJ8;xpv+=IaLks8q31p1b1^_O%ZpPB(=e36k!r;+aj1J@6MTsGEIPggigNqEg&66Lbd zU{cI~PMyeKrY~*;$Xr^i!sf*%cEYCes9BAN)eGlOM} z9cD^>7ya>6`>&EFakoWV_GrJV`S5_5spylV<|Nsq3H6VjJxDpGSNg}R&CF%Z4I#Y~ z!Q%1%HgxkXEn4dTGx)iJTgoM+-S3-N`k8NUQr>-PWqr&c2}=&!x}vzMAEEMQE}Z2x zqMJT+&Wu^GC*bL7y;4SZdXypdS{Xafj2#)^v(7Wi0VoGZJjFsl6WqcSwBa8?BoIf8w zceDRfo8xNFZOlyMO~hx4pAkLF`Rir#a=V7fa;k=bMH?KRDGF?R=sc_Xh4UxZ&N+WB zo?n=~_Sb_%Z~o~Yn*AUluz|I6y;ZkdNlJI!QB6rNkmsMSxVD##r$bUHuWxzP`jhiD znx`5Z*v4@uQ`wu7P3rcZ7JkLW9~Z895~mz<@u(X6t|#7_w?9m5d~y1?Z|#@okA0Pnj)y?nBOSFyeJ%YsSkoBZP^oqi)IZ1Y6pY5z8kx~MH;W-pJc7xuj0p5fZ(ETQB+ zSN*Wgx6H-;{(1s8RQyvuxZ5wWcyM^K?5_*w-wLy<#6CQ-#6|CD+>^OABDo&b-&48o z{}PzY(q(@r@YjiqMT?LBJjBarm;0nhXn>s#eYBWf3qc3TK1`9EW1t3Ya2(o*W!%Kaks5A7dD zoe-J3l%Jt%W{^Momnm%0r&_B{mMN}Q|NG=PtE>E$C*pSPNiF`fgTxgc9nr7xcs@U= z>A1vd-?7{n{T3&rUoYAkJ_3Q;F~97a5&rygxz>*bm+fk>@C=d;Z+AdSzsffw~F*4Cz_c zv-r;Se}Sgt$K92{@yhEn#0p;UwyfFtnJeQ`F)BHJ?7SNRQ}OsGdRSzX(x}F>zC7N zb&5wFWB-1zm~eDjipEp^$efU>^^Vhiv;I2bB71wwQ>ed-mb^Fpp<@3>P`h+)`6^-d zA6)E5bt`KXPW?U@`ZM6To!JG~elvGH=UAI3ErMm4&hwh5odCE>>Rw*p&-}3Rvic;uT(#JG`wvGKpZmtECSmr$d9HB(jfu?< zE{EA^iJEsiibdZDh&holv7>j2`~6Ozo`gnyj>w8n>t!xae!YnQcJi;HiT))U7&ZO8 zDr6pME>ns4<2avHxlrxzA--)Y{+j=H>ZfElKUu2F_hIAGITKE9KcaB{k>u_ZTK>0o zU0Aq&`Nv9`*25{<-+9$;OptxJ_Qnf!vzI62b{=|sqVW0-<>QU9zbuzH^I3|0VwH~% zIJNu2)yuVsi!8P(gzxzv{I6B->pZ3KbJl_zHOyXW$Q3oT`@i~-xp?umHy6Yz`M&Ay z;r3~%uhW}yY?VayE6cJ&7gBSTCiIJ%*FIzU+-$j|(lMGn(%{%E^D7fimp*WuRexop z`XsxzUhhng9QnDd(zkZQcd3`j&V>`LYu-3NSvaRNu8jA$nV1j@7Z=8)n4^%QXQv0?O#RVUV18&$x?tIC$h1HX- ziLptdpzjZ-;-8WvG3Il1F>E4E`X|$VeK_)5<-*BBNlde6dEA(wx2<%Nm;CQ9uTvKF z_NcX%OmtWK-f4StV(i?GP2D_ZK|QVe=9MLR8!l_r-^OJb`{crLv#L*u|5+b%epKT3 zWV?N3slR9RMuqO!4!fML87(KAPMj9A`NW*9t1&V3+##*+oxBfO=39NzIrjO_g{wOo z`D%6SKF)kq`>{c=`_nZgdyeP{MHg9Jw!M6}7Qf%KY+cL&-KTLsT==YC7#;iIID=pJN9%_6eO~88pG^L{h{c%W_bF!U zFUivNmYF>hj;g6dzS`7W|Ffn4?4bqE6B=1}H0Y|fZ&;GIDwoCLy5ijDQq~_W&r3!> zQ~u0z>Uc!a&l}48^LHLgVw!#BVWP8LN_6gA{Vfv@^SN}|rRsI}BrW8x?sDF=>s=Ci zY>LK9_xagzuM9%7|T(4v&p^gkf?0R zRO3ybmI_H9;82C0CZ$e5x?Hb|R78Nlr=5``U~L z&f6{?UYDb#cI(5$&^J>~u|`XUKiSZ{?w#q8C3UZEILE76ojANW#O6l0+>6WoxqqH6 zv6u+)@Z!2(3OPyViwiDSveX?iXbd?xzc>A^7>nUD&jSn$7Tuh@R`E^E(^IBWVS9yjfCmV}Bwq1A}Apck9%V$P0yNwGEGpg=?5`ChQFYn0eOO_`p zkKUM(CExhRCcW=P+~jQ~qR;JYjs^KUuvf3YI48L-@Oi?$mr@7+T-w@bwWP8;UiM4j z3#rMc7x`Cfe3$y<6kZqnE#hBo*izVv*I)d~EPaUQ_5+5v31DxDZu`VP z?L~%6%g1T&OkYipGm85&(JnVn5e1G4UN-L4?;br_n(y>cWs+mdrO9WL7D#yTam@V3>?v>K$KZ8x;d!TK z^{O9_7R*0*oV`0r_ld>9)aiC9u9p`3#^){GKgaKoNU>sfZN-EyC6Bb!kNWQSsN1l< z;$SDM(kH74$JuwuJX+jbd+YBg#H&u5c@tEEn24#a;f}m8t7r{b=kEkKd#% zw=`*Sh)>tlI=y)>7S2EYO#D|~k>-;R*Q0MNoWgr_Qjz`>lXYLzrCsBs*i#+`7daei zm$`AilW*G=QRdi}3s-Zi7;41-nrdIBkg~Hu)4pQP#GU%M^NhPa( z>so=AhZVo~%j#84c@d=UcPza7PmG-Wyyn^G_OjG{OkZGmB3Nf`^Q7{~*&GIayY{U( zcPE9%G<+t1PrBXy2@fZ%`}NQ;T|?39i^=&3&l~#r>>{oh$v^BrvX@Q%hPvL-mVCaW z?D2)#Wfd<;Nljp@hD47J72A7r`MP+YLsCAB`9tHHeQMcR{>k60e2L~l;>)ym{wut`!o zO|r4$ZrN?!g9V?@noIL38!X}|xcsQ?pl_$SxX&yT&AT?G&QgKT&FoZIB1Ci|H>u?9 z{d#T3?{~YCudWJRdi2Nyo~V^81%5nWx1aIp@Avxu-Gm?cZ+3*Hc7`uyzd?V#yD-4Y@M z9voWEYH?jxZ$ed-u?}BO$_>GUJv+{ynPX{e`{Cj9tzT90Z_K@!a9C->-*30gbMpMw zd=q%E@6x?{^VUVj?EfO);-w^8@UfS%=kt1jyrjBEv7Ae$Npdu51~22e^43n-Acy0{ z%Vx31htqDfwaGH(Mcp~_?EHNB809bee-g`6nt6K<$W_0&c;Tnz&Rb3m)!eU>*Tve* zkvsFK`arJw)~U-D8#1{a;E-r^b7^3=wt1O$@vL32{nxxBfn|Pf3qA@wU^MN{lMCQ~ zX1^w`#{AyS2R%Ivf(A^boelqXuiB(Hx!{7^0S*&hrl0i@?31^M-*0w2y$NIGCft?PuDnpCq&y(FN+istff{GY` zq(g)m_4lpr+*d1&ENUXk6jx(@wZJ+O?oLo3WGOv3T5fuAdp?TjGN%LG@0UsCsb?cQ zE`UTQi{HQMa)00U4hMgKf14i3`ElId?@wP}U&a65 z_vL@T-(UY`^Le|Eufq3R+2&5T#Hl_}T`;=Xw86$K|SX{``GkKizUVBq0Q-bNu)hq|W@*z@4u# zNoYpT0bzk3H;&s~`#o*@o~7F(T$b-kT6lZ+?{~YuuZ!EetNd^q@7~+_`)l(*uHQW= zZe#np^5t{OUg>_zRKIsR{@&jtO$=Obbas0 zE*tWy{3s~3C9u3=X1pK&DtV5~(-Vph)bA^Q-?=PLcYVZ0rE~WG|5*HfvpM zwrts=GIMdioz=G6dAomnp8r4Q($5R+b|TzjIv#Ih%WkT^{kZSXr_2V_7IruJ#nB^MJN;Y4YYv^bJr=ia-FhT_ z|A$-Tit@Xq;X2XVbUxnuvdny`uCn^PilnUi_tp2iA9+@;+8i*?qA;oEsdl{PW&L?k z$IqJImuZ(Ri+CjNf9doualIIcxpU{H+L+$>cF1f-y8Z8)A5VqvGy3;5{-5fNsw5+b zBYrs^_&)uYq_(?N5l6*G+oBuS);>+j_N+Q|X7aN+8LvG}tG7fZe4bK$agl4`&D7~f z%lCbrtJ3%H%uM6UlV8RBf16)lU1gsthYX>$h9J zuyq9i14!4Gi=UpJ zzAWwWqmW6k`atBLy5G0&^NBrD+Sc*GWe)3(7t3bnbzR@{%+#>*Q_4m*oyZgA<>k{; z-tBxoui|y|{iv*9$7MgPTqMu)LhXuWsi|LicVl77;`WC5HT`neN}qPVm9YtYcGAp$ z&duYs&t|5(eQcW;Bv?iaePTUZ z`T3b=&Hp1c0F-4xYz7?HNY(K)xZzEv+Zb$O3N6#DD?!ZE} z6%?|jGJMOpIJSIHnyU3<>uD>;2K9U1({wz4?^5@fVen$b#n`P`p>H-E=4-xJbm{8V zsbX*c%$~$*dqmvs=gptF+wYn!^Pg|`qGAoStl!$$@LqqbAzxq87r_sYs=pX+cZc{F zTFj}r9;n_oyX3*#ZO8@JffvdTj(&fUoYxO2FQA6`b~T9K_qF=qcN>y9A+<>YPLsRe ztI1X@N2xyyeu2{HuNk)&BirXN8C1$o=KGVe{O+YoLNaPFHy-Y6c&~rBb6IXL8&h0O zKijEqiI7wXa~BuOo*&;bq3(kTx^Xi8KlD_CF;+jXs=8X){?m?!;gf&mM_=486&hx2 zhFnQGsB-kIbva?w+Y71$a$daank~QE=v_o%uFZyDeGwObW~{mSp(5wS%P!u%u@|Ln zO_`m%Kl0DssQWnS&Ww*2?aHcrr0vzW&D|9I#v=FrjoNRGhRd!lFFngMjoA>KJwX-0 zDS;1@-y2!XT77Wx8(wL%BdPp+G8^woKbjdWFS&N@z0lC_ z%&*|SG?4&Ri#Hm=X7Ty&4qp>3_WW*%+nuey+Ptp5a?4x(Nb`sEHyd5G{`q0%hSmX) zaFeiMx-}`{?b?aW{dc6!f+Nl1lg^&$>m#M&_di?`ajUZau#&<0+}{?7_w|oY;k+zu z=B}OwvNEHLWzUbiZ^jG$efs|Y>FfM6^Ma1pNWOUS!oz=$bmc={5B~ML)z9ty_~pUj zwv$MSU~5Bz`2DR0T#H4Tzg5Inz1M#;v$?Vd*cj9Uqr`W!Q#%P8Q%Z}}pr#+MV zau=sPJzBoF?@ijLN1PvSm1aMh?UwZy~52DS9ATe z2e0#-yLaz?>~H_Cf33vcj_2B|<3~=NJN{_d()#uD8)R3)Q`LnHpoYo@=jeG7|0lhE z^~&r0N4@u%ePZwRzwcL_oxEo5Qb7Zzv#kyF8EbbQ+by{1>!aO*ZX26qZWLX*c~jE+ z+TGa3lE5&a-ErXRY$qFMU{wAPBJknS@0AO} z-FNBiyUb)WtK{R|*GJ1`=dRnD4Eyu=oF2*)MC8+=UCOk{_>+QRA7V z5WhSF9NS_nf3{6dlYDvX^OFy4{EYTrY?$eP5HV6Srr_k?P;K@5|{|UF`OHE1kDPvFu*ub8xl&Xm0Jdn;(zr*KNEKx38x1 z&!_48dzN}npH}hu+FI_qFDGE_v;!O_@=SH-%P05#oFO57_u|!*NcWyxN%0!Zbnh=y z*Vimv^ZKQKebokq`F6Eco&Ov8YZR`ni;d3mpJAZLXZ2#i4Abne&f`usXJ?r<>U`<1 z|D)Vz_v^)u4~MuP?>t|d_h|1ERejI0f4{EpU-s?m^?3RBb>DYCJ{|wB=*|BB|2oUx z?RV8wpjO-4soLSkbhqCTs(!Q4earUk>hu1CL&r_1ao)XmJ5q0k7C!HK{rF$# z_M?ww`;V+z_w?e;y>@%IP5T$M;Mz1p!)2*CJ@Uqe%X(%DURFQTo4fptjJ4lw^|E;z z<9jl{1}zsmcG57%=|K7uHJw?Du1O`&D*kxaDlg!%b=sL3H?mf*JykOA^PJ~XwwB*a zoxbBi6Zff#|6k+(hh9rFto?d5{J5X>+mJ{5pX%m&zK;=a{=RgY^J|B4U&qSe<;VQ%K25%oCis7)f34Ozi^n|* zW-r9||8R|3WB+e_6>)}qi_2cHpKX3c`)AMRmwru13BY$C**QtXh zCZIy{%LhBdogeqUy?AxgzfM`+bD7z5XBo-7c=_GK;Xv@Kzu#`JKl5dE{NGob%>IhU z*Az;=*RTKST~bmqrQ*b&U)T5d?frIZ^^s(59{>BUIfXmF-O66Pd2{fZEduweUaxiT z&i^A*dL{5+(!T@D{60nc^*>KD&)j^@N_%T%RQ);2=SMu%=Zf6U+nxI;w&Q@5s(k^0rklEm{wcK^}pA*vwas)nd_`_w7q@)oqtDnJe%14n74YK)a2v- z(jWFO;GMW-@`_59KZYBgwP*ZbN_l9xF#XgA$u-K}!N51-gk=uBm z)xNKezcqbbqQ#pHhaWwgov#V$A%Afzi)jvJ9EgoeHy=TdTephnwb4};?JFF zEZnOF%GRPc-0y30=g$lB%h$Z`W%z4i$nnQvx!kYwKHAE7Y|KdS?(A^=7JdKMwHcPh z%OXDBJYQ#K_hljf)LU;n<2NR`zS;Nto$9^*`ah4iEOqSx4IlW~|NZjiNX(vf>(*_0 zYZ4w;nL4ND)5(+5Qu-{P&9HbnCHM-@wwD|BK3}){oz}-60SC89Uwpg$zTETq^>u6F z_C~FV*p%XV^WX2Fc5Sx<96X$i|7L&PTR%svf6eZ4k6&LGe0lNf_od*!ywYYBjB)(T z*A5s~eY|_1dG_*UN59XR=4!b+^VRL`@2^hje*aJ^Vc(0DwNGv=`TMcIeov0zd8VIN zg8g-O*8F`PKVR$&x453p$B2;q|Np+%-zu&XA)sIT(mVNBkEH57)9W$9Hnrk^U;5W` z<$P>(I5JIg-x94~Fa7IhZToKX=fmNorMl+!zi*!R%DtC%c-!on=)P^|r?kWvGWm8e ztbh6Y_22zH*5R?&Kis^(_1CW{tM6M}{oovXnX~F+?B&D8H3vVs3h%qH{qGXnBfP?f<^4I4pfnVd4y%%A#9Vho^_Q-v9fq{B#`i$uP-#7KdjiySym{ zb!1kj{c6&>UHz%mZhe2s`Q0xry36}UXVrWbQ__1a$tZSjZmewK!@tk37fUsF{e8s# z&-hyTn$MTtJ1$In_|WA*_xpz~o7-|fKB@lm-yaoo=VBDQ^2Yo8{yRT+UEiyBWzh<#Y@kLkhrEhJ2ev{aC?uk7|s8HEa4jx9v zb8Bx;H$OJpmu<40W%c{-2?CqnK!+|O4{wX8*z~qVy}9_=nNHj5vZ9Z#5S`^ztL%`*qZ$B%fC&h^}0a=9IDnjVmGWl?|ompKBc|(+lv>^ zw`%@$n7F@qo_xu#U)T4?U0PW5xYzvG$!bZZv)3QQJe;$u{%fx9^gqto?|tNJO!H^) zF{sCcf`(JH{w#hyr$H|NH$Z&+Vy zrtN!QF~w;)r z`PfIihQH49ecE%+@V{x;o4OCp@>9HC-7Y?FdwSlc&$B!#lX_g&R#tTxNClXPnST&T#qZJqS{XZIz2 zhfi0cxhMNv2~`d#=h*NgZR1{p>Smt(HM90N&sG0AJ-)8;rg+fLrK;;{Pw)?WOiq z1sxI3SikF)mT7DlC!_AZ>)&#I2^CFsk}Tc+x+;7rzg*oX+gBHECpI?x7r&pfoY6c# z@7$jMGrRfxZ{L#3>rO3yFV(;Go^ZCht<2n9?ZxSbe!Y+k|8poQJO9hJZ!TNDT;mJc zt9$WMa82Hu%+71;I1QI=)>SL>>3=CLcBCw~`pBQP+x+D9B1Aqoo7TOGwRV2)v8tIN zI{HSQ+M?Pw8;`G=>Gf^*^Zolyi3fuQjf^Dc9oheR?t7h&8xBv8;z-`D^D z{qlO$*IV9SDvC}fV z*`12V&a$?#ZjWO;es;2cpKpCm)tQ=_uwUgR$6xhr>JIvsqOhJ{jpd&%-_!j05n=&Ju|F-%s2tG8a69eOTH`aj|Ah8> zeE#?KkL;VC)(?po9RK9c9_ylWm27Sk`fALxF6@%Wxs za*;ioGM*kgIo;qU`?+AfI=?y6!qsk-%*^7;{)sPNUbXPi%j*H2-FaT~(k3&T`-t7F zX?WnBF7w1W?nK2y+mJ15(^=ZY8~@k`?0&Z^yP)HTzUclfvo1@f&zWeRd~uQM!KOK{ zr@fGUU*Y~?^-3kN^Esza)?b~fYq951V0ZquiyxjwY`gp7<$vk!wI(*H`(G}bJxMdA zVy^1%2VX_s8xB)x12Jzwu9= znU-DIkKJ5n&0d&JSucLy)I?&jWS7_m8`Ctw(@ml4Z69-6{ITw9-TqJYdx-emQ@zqx zgB5iT?3`G%wLg3Rt^BchouC2J)<%Y6tNT{YA%Ew8*z~vM`-bOJ_U{t3 ztG*2y!~C+?XVPO$oBa8Lg?}GA+g=yD$SNKqz;E-xA@k$e+u1)Rt^f^&E?xd+4{JL= z#~&^2Bf>u?``cOCeVW*RzVv-f3hocq-@Rf_es~bu%Gbf zj`R}Sq;ur*eubarDfjIxVrPW%CFC8@o41OeT~OQQdq5k2Gae?%`=9M!;59egYSUWRFRl7D zingZLAN_u{dcB{#QpA+k?DN4Rnrp2_ovQ z#D=Vkx9ZO)uZ^1{@?PN5)-F(=H^GDB!+lf1c&(?$rxsnBCh{@jiGJLkh_fM|{(b%P zbz;tP%ja{7A7%PiM|t*L5Bl^a_K0x#yce5&-MaJVGsyP8`Gux z>sVf~+y8LnXF0BI%6EseAYlx4!dPq8c&#H;K zx_d6q-TQHm`&{+MHofT!@6|t?dptbn(dpT~f%(Vg%uY&pt>T$)y>fD6(F~AgP=}h$ zsX>4Gr(dr7k`G(o_P780WlMdng6yxTs_5qM_ol)VH=pDP;RemNwC?!}n}d;XW4ST; z+`jy}k5jhK4RQJtzNsQV6FRm`1X}I_Co~ujlv2W~q{Qc4HKKAwVoW6&1 zzslSqvs%!A>1zi=zURXgQ?|%%__4I*MVD`W%F2-a^PYCiPxJ21fB$ve^T0LtZ|wcC z?*2yUYbPRQ?*IJ3TX_5N^&_>b^E#i(@*iJy4%D4p_v7WmbuV77og43vohvDv9DVF) z%f?*8$~!$X3=`vYW^BrSW0d^=M&hlwx><&6(%X6^8I>=b;P^1>ny2pNdYG@l!2&i!7!`TfPx??=Dq+t}{g z{^QD{m4AP!3zvgugBD~8CVV_A9r&#O{?3;MGj8wFc)D(_xi5Qz@H#D~xSDX$f3Ga# zukTdbS9aI4@Njbd$~Xb#7e9Bo>wEsJWVCBixV;55yVuLWeI7L2%s2Ua-^aUeOXr?W zXp-Na7^IVN+xF=@_500^=QbT~Z4keIxHU!g=EL+3{T(lZ3-7<0{aWMI!S<^2r=)}D zHDBM-2Fg$iJ_=rVR9*CY%8i}R=S$l^+{d#$zP`X`rj2QZHIKxMo&y=6;ol79xYT-E zTie2`&f-TOJ)ItEIq70y{ohO8k7u?2H?cbR20TR*;Lfq(L&|MVd)NKnuN+Q3^=s{G zjb95}Z|y5vePiw02{Wv&cE4Y!n6dVu1E{U|F!_~FrRKK9%diupG- zDCSkZS+#K0^{w~To?gH1^$*!SpioR$BJkj4d3AMlmC~NaO|g#_@UHJ%J55F+*-E`{ z%IUz(-h9RKHC1nRe|x&|u$lGIKR-Y3eVTIR!uG|wiUwugvd_%Lm~It)Ixy>5)%4YF z6Xp4snCo}Cb?h(!4IAgUIK=NskvF-1a*Jr{$t4vfFE1x~et)|$>yPusxRE{9w zGsArS3i0D-W*R3yI?}mM{K|nHro|j_HRdN@Y<{|A^Je3)64UGJXJl6VJnE;vueA+S zCiXa3GTo{(s{Qrl3zyxu#^CvTYy8$kZdSXtHd1`vR(84 zul4_D-Kl*Yec#Jo>d0&1$b{|ZbNe$dFI&oLemm>NbKdzeORc2}Z?y})=x-3e|5q^c zf|8>hz7-rD>z1@_|EBSarA~#WwC~o3K5l z@c)CA<^P`S4qLY}Dj@jN!Nz8ph@4<8dHr4NwN?>{Edn1NRkt*(b=@3eQXM_(^VI6Q z8xMBBkFwE}oh)d;WZK>!wo&K6*4F0Mvj2;xFj5wA7OvG224!rCt$;ix!YF3i6#_zOJvg-Rt=P zH1w(?_6#&%_xMq_zMtwD@%=xJzU=n<|M&j?xhWAx>OW1M-_u|FCVAuSyxpqzKyAw} zzjTi7tNZ(_;?Kw9mscMws`tE64-TPbhlZoC+up6)k@@oe<37>r$Cy{Y?+~5ae{2?S z|M9kYkKU^9jSbW}$(~sF#3g!B#nA-sk7<*y6~ZTs^5R9T{cf9=&96=9ku}zfEq2#q ziZ)|vc0SPk{@}yV@0VhhWIb)NoRzR`TTav;zUlGzWUC)EvQL{?eKtET_}abn`L&NG zdFyFD-Z-_jbyJ7yAN{(I-40<#nr4~vc1+Wao+kEd=e?K9=g+INaq50=|NrOtkB7wf z8Gy#8k14jxObY32k1zTCDt!N}ThR8?5n=xxR&Ftsd#Tf7r^Trsn_Sni+^s6wA?ZiI zpaGL_GedXYyn^^|2i8A2^!t2&Xzb}MyYhFLv3q4NUcDRmXWG$M5w{BK?`(WEYe&gR z|KwA1jUQV^%ZvZt^49y3Hv8A7XJ+}IVl*>h(mm>O`-^cX@7%0n_2ZXsu*=sNoZI*N z?ZwNR7QQ?naotGK?&F#GX>}=Q&|D_%Gu?ti;Jepf{ zO7o@oy?<}-|I^)C8CCaY<8e0bJl$>I@0Q<>*in%9=g)I{f1~|>p6Wkds$ZKKQ+hSj zQU1uhAD{_R+sfs&uh(v0rfYFrzP{$9ICpiOW8UoCZH`A$Sb7}hDjq0GIJzsf@Lg~E zk=WJi*KdBo&R3rJ``*W2ckRV4tDn`X-e9<4T6OP%6ABw%Uc7bU#`ddH&Ivy6e@Jy_ zlpK=0Zk1Rf)4OQ;&f@3O#P02Q>S<=bc}=VN9#FDd*lPY(eD4d_gL1{XdmhfM|M&AF zXzVX*(cJQTmUh1`_D|DY^s{yMpOx0!pC|iSt$f34sB^>qLnHqQpQtmR4?X~m&3*K< z`}yR@G4p+aS@%n?#~zgX5xTl7KJM*ecllbAzq97u((7#yl&EBhsIcGnZR3wyrSJ21 zw3@%TyZ2e!kN(&80;j|5XU%)lZ?ouI0?Rf~SrmP2*~6+I<+och@^|df^6aQ>nDGDU z50@?9Ze_3kk|zH3!!+@2TRv>QwCM9WYyG#D;lHo0uPa>>v(qSIMfW+Y*K0N%HRsOX zH{sMa@r(EW|NFk<>9pu8Jljrf_-1Z_w}zhb*H-}B(TiT-m~)M?{5Eock6f35j*;-rG`YObF9j?iQhYH!86A@$MEF$rJ$JxZn1NVK;4iE z+sA7nHy`m@pZDjH`2L8j{hy|;=h?P;!&28HhyL^CS2WMMcfQn zYyP?)US9Kl*HO*zeLU~^U+~)BF`d0Y{JyER$zf@UZdEa_%T>?$*FUt{XtqDsU4LJ8 z!I~eJUai~xZq`1Ir5`Gid-l0-z5jW>zV6Sf@cmtq#%UfZ?{+-y`*G{~zOby%UGDo% z%(7owf4=&ju)9p5i)Y(f%V za_Hyh!}eXK+2PCer}xhhk2Yh9c017hUN`x2fUVBH%JQ#&HqPG0X%f=pKk3XOP5;l2 zetR0Mdj78R^RuH5+vVf7bv!@4OuzD^w^!!JDRcL}nyl`x_wmNMPo4fXi>{>k+q_Sh zZ&&-+Ox5a#!U1*`!QWRZ?^}QNJi^ZQ?Ob=o?lV_b2A>vt@&Ake|B!Eujk!k4=T&7@ zxo`SX!F66d-PbK)i$KE1y+0vbaM?Rnl^c<=qW*OH&#n!HJ^ zzq&du)ab`G$!rFej8%J&L$ z>AO2W?b^h@@9ebwpV#Vv8vNq_13#aeod56cx!#(<G4d-#mS4dfIur-)|<#-m7BvKXUmT$jO=9y%pEv>uYb_TWqFk zf2(E9N<+a1%+lTO+ppcbI)jsM!qrKo<-fb9Ge3B|Zlddf?)MXqZckg2{qgMnL*+AT z0(@fkeq;~JQad2Xl`?ypZPI+pcGD}r_quav&0fpI6lL&u)~|)_a#G!5x>~M>rbm3f z?(m6mR(xpcLmjKPTP~m4wkqLyyZxWU4g7Ii4)a+bvH$Pn)B3gi_f_(LiHmhV_QQ1c zz5~}HZswgy%liJuBI(@iOZ)$QeX>FDbBW*CUWVK870-8fRGu@N37WfJ^X0JDo=UeP zv$JkQpR0c8D*y9@`=x&Um1pPtyfl5^lx>GP+Vx9ZZ6u!`K6o`eUN`Jk-eLWxu(zcqyFW4ve9Sn zS?yh<_y2lbeasy*CO3skO!Lb3bypmH{QKVbeK+d=|2^6TTH$qYcC3reubbNr+>Up7 zU;DoL@p1cq8)s#n=7~=K#LxW4>h<07`>~S}7uFu)R6lb6-?!~T{kfJuS8K~xy;yjo z@VM;hd+-1KeP3^1n;Gi$`r2CW{PhpJws7`d8m>zY(ZjTU~f&@ACx`o7WlE)uhK9pK>GhsNlyvnfcwh z+i&UFE`98&uyb}#gP?>Y%P0QrdnDV+*L7DE^HiVXxBHQ>@zG=X{}nf@Z&j*WbAL$g zx80_4cyi&qJ>4G;N+*9>{wJsW=iB^x?TKGZ{-^3!N8a3h`~J6W+o$eb`|RKE_x+%G z>B5Vy;#0rW{klAV+TTxF{*Uzc|M|4#`CgR}c2T>GJ%66+_rG4Zd)=ap%15W!C%?-$ zbhLcp=J{3OC9@(jSOji-S+_>eAaC;4b_R2Uf2C{UWm6JweJhBz`Sw6>L&Wj6d6glEykN$(PgFKARix9Gx>SLD;(T z!^1{t&FY&c8{>nYG_(E=dRYC?R%FWBWR_RLjB+|}HlMdkJ@`56*qk4qGXK3^@#W)_ z*{z#09v*7_xcmNJy>GYk?KfWfeedRey+`g-^7ec@wyAt){ZsAu&ZXYdr-|L)7?^tbVss{q5v*@BVuphhHo2yq0;}_m)(U&BAB))0r=VIt>4R z9JhZI`@SlCLoWONoU=D8pFjIk+upcg(fgJ6uFpHO?|1KkX`oJ?_X!h&`Q24jk7D1v z3^{pd>UQZ5r)&P4o+=&qt30Lf;B{87r@@VC^^*@LynVu*FLw3FxpsE3n|u6cvYIcQ zm!+sb_u#KhpaFxktq%WEW@UT+TN*f_;`8kLF>D;wH^lYhVvarLw2HQOvFWsbKBxGU z*vA;=o#(o4oIb~I^06hh`~Cm#|Nq7BVtcJ=?Z5e^*5~8){|qab`HoaS-MQ~&u8B?O zbMLJ&%>8!1UX3;3^UeM}{%>DnKt!qDA7#pq!Y7?&DVbtHJeR!qf(u$%r zt~&ASW|`$i{hF?GzwUXrhE49pc{`R@H9q^AeAqE6;JN?4N~tBytd^Cz@6?QC{^cG9 zEi2lf%2LsBQ1Maw)L;6yDp&jXy=jf#mJ@l->-YK3>K{RCAAYG``!hXkr`hxCJEQY< zs@nZL=|An}!h=rp^*4Wds8oOWj4;bt117hGD*_psVxgv6W|m+1v~6)6M}bs9#>TZm z8>6gUw=jgCbc*@*S{2401cVJ-y3(UP`?=y1^t>=m~I93hg8!G4yi_kJ8RU-o@rxKNi|SI_(0{&UlO<3-lG zY`>R!V&)%Tw*^w5&OGmxx0gWypTN>{z(9G!Wj|}y;|U$C86Qi3KRSMGI-lFxcd6>% z{B&!>SG=2C$W+?iuzl{8yA!UaANb{zUhDRL1=!Nwc86O1`KWoIIajVMb-G{%@^vn;$OM&gqztE)@;6>Z=RDc^Yd4JwA~bO z``0tjYSqs>X290>FIWf~L|xD1KhMVU+nkyVpSbHks`sm}ohiQCry|W~u9d0ozP|x0 z7W#jj<$JFC{VZ82cG%*F9pSAg207q!t+?p&ovsb@&cUYWc|Jg_x+sD>y$?Kk_L|dyL`J%e} zXxZN1N4MS1lm7hteE-YK%d7X7pY07PW_@zBytsG!W>e5up#&pn*4ux-y3Y&&6J@QRR+cYroPNejUMHer^WJ4^*Piv?yV@+GjE5<% z=C_T6D9e`hecZdD*(r;gQS8B+zssI{ZNBm|=+TBnDPUA;It(>>?L2E|>wcVA|* z-9PR5*Nf-6s!vzn|9$Vtw>6v3d42q>dg00w;IxQyjKU#6rfZn!KC9?`RnZE#BJ$5 zv(0pMcirSZH#xs=-n@D0HhkKHGZahGAV?q%iYU2FHqz zcNv-^_%$m&#_}(IA^~>J;danE^FJT^{@=e>+29$Nn)~C+mnSE9c$_E5um*;>1jabJ zmKeTWl^u0`N7t^TxYDy-4+D8t@7T5I-;Ck|Jqt9n7icV+RNSU`_5hEtmGA$*v3p&4 zd7S?&xtN)&56yCe|_;(N2nU0o=^SgLn zKbtm1r0c_vOWW_?)Y$j-`tmIuEFKLYB;=J)?du>nQ*gr0*+RE^R?Od%Ul_1X{>ii^ zo$VRRgRb}=V!kzx;pq0ayJxUXV&I&>z&XL;4~Ij={ePhj*TRmypYzcs`1hR!cZ^M3 zqf(X!H5?b-(|hh{y1LmbpIJ^!Dh?pjsUpC^^5kKr-EFS?4YAt~t0ph|ef0hJVwqs2 z{W+dBd0p>S-t9@rvXVRhQ83o%tHfZ>jLD008^ z;knIF=QreXIMkevdOo?Rzc}Tn+4|_ZnAf2X)wjL$kP;GSI`>n%liP1zia$$1;<4^l zP!M?>sACbRGhBWp-~RBoq>FcY89)B4`BQtFTfHu3KEu)STDcfyMj-_-%3xx&2vgV> zo|k>d&urzcT}rNd;`EPh_x)uYmljZSyxZiwGstf$4oqe$3?F&l=Wp^~BJ6ox;SAd& zX<@F76C8H1XzV+zdbH}S(t#ZHvxkn)^=x4AIIxLDVc*2E{C0g4{oRMXN)>t^-A=5P z<1N`|&K9KkOG=F3sdfCUK-Ip-ZL`#sK(ay#jMYK~n;2T(`fa;E@BWJ$QLZ0lqxacZfs+aY z=Z$#{laDT+oV)$6xV0NwP|a};9scO~44da(d4A9o8fXfP=|Tx-wj2Mp%?o3H)%2-2 zbM6<**KwH_ckYk*+Uc%kqxX@`rDoxl6$|@y1B09vc4aP%zW60z*_uB+3$~mp^PKK9 z{Y&m#CH5&hRIZ-=9pYUgBJIeek`T$X@Wt~B_UT7!52uLx@yLAed*>;2M;xfq_REj*{5l= zW_EN?kjA0M_bhD`IVUuva)gA1otmZ_y~KOE-tMg@br<%iKUx!Wf5S{r`fTu8wDL$q z{d$&d*IAyN)3p2fe$4{iG@HX~ayPJ=9W*=k{q0xn16(Q&Ok&C#DnF;qG|irNQ7h*7 zyt-eRKi}{S_sI_5B_<$SEL zImUQxO{8*s%}3Y0v;VQnRXC{m&&#<|`T4B*(;3NqE6tBq1_n;NQ~7+Z+uZc|wP9>L z5(}0!9(U_Gyz|$q)lU!e+nc<~xw9ki+1c6YR}N0zmw33%b9>(19;^Cam*=Pa`SJ1V zxuyNnr#>=$W%<;1+3Cd*R!8G*Ny;*b+`stn#Nrl?2kyydO7^e0ooN4Xt+m9qgKjsx z_;)Njq{=yg!O8H{>C?Od-qZC&J03YCEKmP3Ejn-F_xtttSA4#ez5b*lyR6FAty_<6 z`}3z}=ihI)drM`Ke|&ft6dIbEv-|(Q->Hv|bRM%>=N`9y`@KT9J{iq+*|Hgd&wGr| zP3X7#wIc4ton57`UvYjpt-n9!*}1venVFf#>K-5KogD0Ec{IZQ!vSW{&bQU&GBq5A z^PiW^`*F_qr*F;SIx!)R=?q89cPxomAIMZ;^YduX{xkQJ{;wCwnX}`__Sw&u2Ri(j zz~XUW5{tqlMW@op4GE20#|^6I|Nq!uf1+D|--^{IPV4X20mb6m+uNmYSABV*Sbn#3 z`o@1Z)90UzOrNWIdt0vYw!Qy;y*4U-=96rzuI_GC@*<&P*2jY{E-rrh<8i0Dh<@L3-$ILjMrJkN9`TFgxt&ath6aw@c-rQ{_6wGO=NoH>%X0>aqD`Y z^3Jw6~CcaBB_Bdd~BEZ+!SeN$j&(-kwll}Jp zGPZnNKEG~~bl#4IaVOZERNn1;F1MI3sYlTwe0^MQ&Tr6OQU4R?=GjK)?fn{7Y8T7- zeBFu_7gk?d9j?Fg$D{6Y?$tZqY&zXzC2)4>(xsP{`AR2CAKAXswqw29>@$ac{rP-8 zTq*MXl})MMFYoRyxAour?@XR|#lDO7wx6?AwO`0E&Wj9JjoqX*^=-m|3&Oi=Hz%j} z7hef_BVqOL#E-I+5gG5+j+*X6xgI@T61x8_vWh4Et#ay z7b!kx@scGi3t6N*B)mKPj&wdxNk92-rT3v5hvh=6em(an3tP5i`SQ=dllt{~XN!v5 zFLcg5FRb{4<=)z<)`z9uW@a#ltzG94X#Y*7;wFPkUY9iE=kNT6Cp=`?Kdw2wdi_2v z`I--naZ}&#{~tGDW75@Cp~to@bZ+NzGAa4_<+A_O=@Lva;yg?;)@5trLKjL;R`XpY zewoKV|9ODU7CmNmzL4JsxYuxBx@r66f^$`9YnDO?i{a&62Uh<%U;od{oMGGB{r~@E z2`Cv=yeK*%=&o^oQ(XHGV}6k(iho@$$bXtNq|%oo9MUeRIfN zfuI`o9mbPBe6emmF=@lcoT5jZ58CDLE!!0J{O)nZGsm8PzS{lUr>1`c>q!TO=AR$R z7bn+F)MTEQa^#zmn&i{{|NpL+;#ux$=gIUr^6Cc*veZ3NV)ygRQfN-wZ3lH_{t3tKETK;QpmzekT+uPgHw-rvD zW&ZWxsabpDf<-qM_m*;GZC-I~+rHZ0Ne7dZnWAbQuVl>I!j}ItQ0+IHcj>KjvsI;M zYDeB%YjoiET9MX&b`$2ZJ({hkUiRwYjoXiGj*3(->$7V9RulenZn-@BY|VYLUrWOm z3ckMn{>v-HCrtBFKQrt*E?4cNP*K{L(f0h?^ZE7HW?xCzd)eRqZiM@t&+`9&xKB-v z%&&PmHT=ok@_QFoA6uYT{O8O4f8Wdv<=GcWMw*?Oyg(u3$eW*buU0I6GCjU-V(s_4 z*Db#7|MyjY=a);~y`?;#tbf1R{JJ$gVfM|>IU&!ag*XHcG#p;DM(3;S%YYz__Ue5T z{@l8{EqQYIW-qpoI-R#sPq{Z8b*OpV$$fPD+g(=6*ev}GkBTh+%-45t8i)1At2<@V z_t?)^l*7(^r{%+=?7Qm^bct$TIikwZ?QnBLV)LHg?{;g|8wTxNc&Ge+?J4Qn?{~{r zmv*Fa+}?h#>a>K%^453PK;ZGCFip2kPsr~AKYKCaqh65qPzwwbflb=~k} zrd!)`S06E-$2Rk_`LV5mHj$Tv7X0wLwQjTD;fI~v9iQbE9Z0*8*SB-4-+GmFZ-Kr> zd84^rx##9sYS}P75@G#0V}AX=%7YKA-rhahV0FaG{^yg)tN4t971|{4#kB-4NH`>z zvUi4Ia$A)9p3kA-u}k|F9yJd-IkVV0TQU1wMk8b0v)MXFOqBJaR{mg%#4gg!2N z)_H%fa_PDd&Sfnr9jBG{%OA8Vy!umB^~Lln4pW!>)>b{eIq*}~buaJxRwqxNofx+E zHpe_B$C}6Qr^UTb`FBih-CL3Soyqy##?BS1rdyoZ+b!XW(wS9^_6VEa-pEz^q ztXO=*1i>AQj$Y30(OON;(eamMMY$oj zyc)Bk!aNYcl#p6uwBbrAnHeGX= z@VtE;`_`@Y-Ro~HUnlvibD`+R1AKe7C@a`1#}-9TTQNZ@<E@?>iO=yQcKnN9hY%W`T6KV_r->3(yqR(b|~NpQ+J-lFnM;`zV7zgL%j=D#%ytAGi;%21ww2-j z?NXCZ*IM>3ziG%Mt+1zX?g5?!J1QG&1@7#~d-%+WJO6Xt!}-0YbNw0~-F|tW;nw{e zCf>gV4mol4f4>^#-LaT@_u=2~_op*5v$@ph{fT%fk=WvAvP3HC1Kqpx|2Qapi#TqznD2;TT~Eo& z8ygpYdZR49Sx?}8+QpuT?9V2*SFo?&wo~EE<7v@(hWWR2joZF?TwQ%cR7io*Tkryt z43pR+QU9YKzQ=}!ihioQzG`+}%;hrC`@FH4b9Vo6oM|u}iiomVXkDuSx94 z|6`z_-EKKo=9}Z1MS33ul7&Pzhu^RfHGg$bI(MD|>urHciFzM6W~Or-Nf&scqvF7H zPPHLhfMt!?BY}0d9vG(wnq>tUZPySZ_%(rr(j6Of#odEwDcL}jjXT>dj}zi0E(Z+kYdtZ{MZe(G^__tc!k=sAuJc}D49iqEVqX>VC^*~{5X zxvlEjnnUcR*@v=Q=B?i*F>oH4)bMS^@lP@~W}GgQ8IG2l zWLo`*h}5*+9A?4Vye+aicFWy}_U+XNn^-@6x$M7s!X)KB3#Ttnba%;3hYwVoQ+D|H zc8^{EuA>68EAyUrzP}^lzu54a#GGax<&NiWr}hXcySdD@<@bQ~)+fZU%y_G^^VjL3 zwXGkHDeQl=MJw{L_NJKMNjDdSt+aM(F1&QLVa~C0%{JR-NKS+dSSb8r`EuF)SdII9 z-F^varY|+e+g2y{b#==X?JPU9s;KAetqZkAT2tTtI<$U!u_w$X2c~KzhmU*Jjqb>a zF>?5~gnv1)dg7Xd(um(T76zroPAxfmZ($BQGgpE27p+|Am_y?Y;RQeXY?GtsGcFST zxwLrG<|XU17p~1UmA<|^Z{DVvJGY(<`7sYPit@<6;iliC^K;U3)J(4yRQoA zyi>j$P%-+wB~P(AOWiB(tk!Kwp4ogiKXz`>*9+M8?AbFZ!~5Her?E|B;GEzvNuWWP zLtjkDLyXDCbcT*?>cg@}i>6PUsMy-t3Tof4T)TGal+wv5k)`U9v+Gy_~vKj&Y+k(Vc%}1%a=Ac8c6eOTe4l1EnY)z5-18h z8d!3C4v0#r%52$KYogo7^6J&ABac6x`1ba8>g8p=K~Yg!(c5w+c6D(*<^BHquh)#l z^FsHR30$sFoWf=M{~u_CFhjR-w$?5 zHVckDcwx2Qr%SJw7%cX)5@lKHztQ{GcF&1tdP=_jK5)O`*zdXX-_2z>`u(2dMmH9g zFRtQoc^$bN5mS1l%~y4QT=&SjDcGe{oFo75p;l3)==}n>c^&=EyN68toB$dERBOn# znWGCD<2*DeWG3fgeQz`$CxxIB8a<#X(M^|&z*vV_seUJ+bv>A8kMpF<}kMgA&c#52w2CUp?9NY39i+x8}F+ z0v}8aWPY`zv^(eN0>l4xCOeqTCpj>)XzY`2IOKa#eSv;Jr^wBV+q>LVz+-v_!VMqp z1qEuC`UTc5xqrIkE zUOCJ<^@-`DFZ<+Who0W!2&j4AD`2k<@({=s8B-Y8Vt&1f>Akk`aXVw->Kt{Edu4C# z-ni-VQPz5!&8AL<&2wiYe9#41eWQb+MLmwOaNd%Jvm*D*ei9aVJy|B|mFU)4a^khBEjRxp zJ=Wixd)|QBBq|O}AC(`Zyk9>#xBtHRnYi>tKemO}o86R(F-gmxqcnf*AN!j@ySJVE zt)9ewhoj=-r)yqIx5t=;=uZEcYMk?ccEzn?fid-;i-zyEn(x;yjam6~l+ zp<(ZF;2Dd==iJYH_a;}G-qZSdUjO=SuZr2Ze@jpQf3kA^dg1>4$M(#8@7VxSpX8z- z@>gcDa`g8Nep_zt+B?xkckvIudHcGeyeA61y2|?f^ekEK#brL)av7yhnV#Oh}3p4)NQMc)g_WXGV!j8PZ@}WU_*<5H{ zUdCc@!S?D?myfa!6#n0+U7J&#(v^R^?Cra@!~)f22j}kib@ZlVAKcR_Wj|V;voC0yoz~IqYj>9~FN~^LzdCAD@#5xt zYgfh`(QJ^ta^GYsi-$rT%Y(0_yYF7IJYM-|cWqVcl4Z++8jinzoz`s}SaW-k^!L1$ zzVGWkW`Kjm(} z+sEARa{uny9jQ*!jZrJtt(#_kzb4o$ z|DH_oJsU<)9#c_nTp7IlRCs*t)g{ZaKhB*zS(!Qd?S!z>rM|c9CAGzWT|3EG8Lg_X z?{8#eq_J=EMq%L_XxuD1-sLB+0E(ar)@rG<-PCsJ8R~@*6;Ty zPge6ay7sdTlP%HOji85xFVldBlR~{Jo*?cD-JwrtEOW{7Cugu(c@>=WI@X zoNoS8;^ZdhZR>sd3jQj9#)0S0Kfhse%1aCGt&<~bJNk88$Ysf>{DbE@~C2x>vzw}Dm#{N zzwo$h(J9U45@ioRe|&uW^rfZVy}9~(J~Zw5ey{pi)sFlLZ&~CX96IFm^2)ZQGOz!I zMO9pH+g*L>%IWp7y3MtdmnP3vQr%ZvTpW7B_maecwx}kCquW(?DcbcG{AzGMq9Rvt zfbo^#LbZQ~`Rz{_pSM{Y{`B|z{nJ6qM&oKMY@Fq5zf25l&j&U14rH-hTN|w&U-Pl` z;u>CQGme&r-uin}a+(eD?(W*U@!sz8{FN(KOc3_7IC$gQVt4+hd(F&Re!8{KQ{Y+b zJKOBiTIWug*}vv~S@p;5(%-3xaz}Svy;LK5Y3(0x$v6L_?#67%zV6r6)g@7sbXRY4 z$>|{7yH;j>EJ;2J#>T<7w&iBedHSeZf12@mo8Z40iOIRSYis^|Jbr0kZMEggC6hz; z{{OxI|K$9AKiwWEHf?VQ=jlhD42muNw%=xad41i#&-$H3X_kecT+Ii^m8(_>@h}`I zeY@>;-ehM!%Oe@LwqB3R-qU|ndh>Vb+>qA`XE*po- z2Jajz9hD0qOnS>;by#iAo9&TSP z51Q+F8M{-3QAnY&gOi1cajLa6dxQJ$eZSuY_c2sVQ_PaD`;lnc=Gy(fTW^;~>y#Ns z73MuUx_d!xWMt&2>b`9cY#%M^K6TpJoXz{l1f<@L4uo3ay>Wf%k+c9p!mc%8GUN#M;w32Nqv^*C%Uj_SzuBIxg*DuSHqGFwq+U#(0-x&r_ZFx@d!_(>Urtig0{R*n9F==*v-WY$Kfk=i z+$s3OCDl92C-2%9cW$P!`~KdFm91MMYZkx#Q1iHT`GSZfX<1Fd9}m+te+L_DhM*SNOgEuN=Il84Oe*zTVzh3R_emd>@IZjcXt0yGPmPMa)EQnV0`v3QRed?1F6PX0pv44A*_H#iqpODSXBu^Pejw8w6V`A289=g+>X`_gUMVe7It8;?sJl1_QZdYpfsUhs0i&?bhBOLr;Uz4m;5 z{km{a5t7kx;M?2Vy=x^n?(VOzkK9wSabxbG7S5*q`mq91P3OPgOqbv5X`s*0y1lU( zRLUPxarh{!wjyOc+j$1J%H;%HQg9f8?DA;JI|=X3z&qr_iu95sJ0H zz64%<__OiHJd46b8?Eo%FF4Grb#(g1iTNG}6j?qjdGUJvelebmYoMvs>7e=2^{YZE zW_WNjUoM?#nC!M!I`&AZ@){OJKAmey4X@?)-n;fnc-yi4rf5gr#%(UhF zD^1CLOt)`{_9_0BecPM2cXn=8=nWf+uy zWg$=1Tu{L!)FH;$^*&?bmLG*5oSS&0Bp9xot!_CQEq(WyY*@jnO?NfzmuHLt7|8CV)FbMwd&sho(h-Sb`g!{uJB7^tP{!NS<}-X*~QfaTU7Zc(e}?POlMU{$4Yqw0Z{ z4!@oLUh$S2QZf#Gs&>EfLgLMmR&&1I8CU1m{qp?x=efOf`k`P!pN@sxp9Gw@oS9=8 zoNr`gAtC*vjU~w^;pwTVT5G)Kr@gtc@l@l2!_EbhJ3sPWy?S+5*PYt$ce8|~|IR%g zA?_0p91)Rm<)L)7phNuYf7_;~ZaKD&MPuLFTbp;^Xyq180yQ3%?${CWjem}c^osQ#M7qa;^%^oc*mcM08F6$C;1oekN9dC0lKH4rRFW5J6uVJ*8+1E*DOSP98B{x@WI(2k@oNj4m9OtgTN}7&> z;V)9B3h^BMezjF;8B<>2Z)26KvvqIY3k%YT4GEiU_V)IQna{bnxHR@8-$>_hKIkoe z)h{V7a#)b;Ad|+J!n00o`W^2UImNlD@9*vHovp)jZ?0YKu8Z$y ztYN#C!}H@B>wVE;M>Dy(6OZqi3+~h%$YXi&b6a$OTPDBU_oQpy9f}nPKOR~l+_AkS zU1pwk#mSFG_NR>Z^M4LGyKb4h<}2nzS#v>wY7X^-{ANiv3!MW)LYBD5vL!uKYnIFR z)iw0?@oKu#?(ww^R8yP&;60t(XX%z({rRlTIUDk(20KOk{#&QIZK5s% z^NTMERvg6~C406!xOUp(R{uIIGcPL1G)d`KycG;;nyFxTf~zUIJCCE7>%E8P z@jUyQ^%KfvCNnKOA0Xh)w|)hmy@Q%q#c}=pe~J!X=AQOL$l(S1ku7?M+xf%Iz5f4s zZhuwn$S>tMY4fmM_ukyxJoUPCeZ7u~Kt9v2?F%jU2Y$>mKL0$2N#y>yclD}kwsN!! zv>%uD-u&Sh>)M^-rn0{~8|O_Ft8KhGFJJe7;-QC($A$E3*1dMC(t9Uv&$Q!V^MmZ$ z+tylX9=ZGVdc6MaZMnhzCL2C>X|D_TIj7(H-HsooXBg9GS(mS?Vk*1f$gY_!$kp`c zI*0HpnOS={SvTH1Vbx^kaPVuB`8F=r|C>1&Yxm~Adq2hXOro)jL&KxnjB{oBI-?Ta z?>e&Y;*JFet$UOF1eck)e3%~@^ZI)f&x`H?miDK5VoD7K8nW^|1uOo3yY2s=a__5E zt3$LWiV5DkmOiiYnA6&AWx)oQAOHCH`01a|=grxi_+%tb{gkiT)exx|_~2~a`@P>o zUhTWKHahyY!nQt^I`swXe_#BuRw+cNgPpPKeMP`COOFPYS-uC}6z+Z8XPt6uOXjDi z)8kh?e$;tNGJVd%Wuo?GXIOxp(|_KOFq5 zUaD;6ovi6tV=V2v>6^uEUnZ3aJuFU|%8>j98u91dBJz@F>xW}jhr$H%*X(_IqfOFA zfA<@sJ)x`%L&`Kah|jB!HCQ3K^ICk>OVz!ez4s3UGsph=`g%EAME1kpx9u`T9Ie~g zj(`?paUSVpI9fi(;j@W~1Jg$3h7D8J@B8KT>-+osIV{aq)n8sDuG#K+JUK1{lJFFJCw_xaY{~pex`2wq^mR9WDL(IAMXogE*4zW_+^TO$Bzfi z{HvBtn>KCL!QEBI&be%n&-``4jU&%Cj7ihvyxs4d%2z9whfIEUcJ}hLm9l0zGd8IT zzI|fpc|eQh$M2-6^AfBNsXCkot>Efs@laUFsW4ZOEkj9}kIU%yo$~v&Szg7t5mJ8@#7^4C?TcJ`O_%uoyxKay?#@rZC$=&x^Y zwbkboG$nR_WV{a<+m8S(lzqCpgI9%7sDqW!=TOTV&1QCfGroQw1%Ve<`)dCE`Dv7L zLSUvrVpCxHt1By;VxBylB47U}@a(@8uj~H(e12)6Gy6=-;$`=iq_MQlxc1b){?}xk zs4X75N?tB9Us*e0_dIY1(nH}Vr@?W(!pSJPvAB)8_$GCkCIBomOMY) zv#P_;J#uH!QmH?kI@;&$ey`!;xBT&-`D(hvmz@a*nZ6!s(~aJ?!s1P^pQWq8Q!8%J z7{-K#&l~|Y*6Yt_8gfo>U||txdi1ek$(}ti6I@w#Zk)L7{hi|TQ$eR(8f9P8dFtxU zSJcv@FquPb`l(Z=7G3CTFfRv>x9%27xX~F~J*{ja1E-5m!slman*;=76?RQL_Ti$t z{M2u6Z*v>1Opv|zfm?r1z%-r6MMW9!Z)|j~+$U(=7t(QeS84E}-;4Y0Ce6;@7kPB% z>*ONH1l0o{P$Bc?`;O?|i!Z8Ns_yuo6P|P+j^jWyul_c@_51(rlDe>?@Uh!g&W{hb z->=&ZI=gjO*DTggYq#GMI+9fW{$6aZ`x*N4r%u#FI&2FW$x9hSF;v8 zzaC$od*z|DkV4}Ljt}2ejq>hjgL;|5$LFQ1F$#4EGtS-1vi$eI-|wI9c-%KRbaj|y zx;xi?^ZPZIBkbR9x%}jI{{E?l4ml-DA1T?fIo)YO@+JA$ce`HeMQ%)Ljhp)Q_4U=C zuV1+$aXW9vL$;~kF5EZ(%>dg}96sir&RZI&^1z3w#H0Ub=kw>#{WcQZ`W;j(MLfi=Sz{-(_cDXP$22*mXE=Z`I1G zU3_nn!Eqtvk;k-Ob&qeJ^o=TRVU8=TPp@CHs(hNhUPqt*)YbV_sk_~mT+@HVzch%i zHm7saBOy=TyIpB*|CG3lT%uCDOLR*-=U>yjw0^61(A_=$TV34WUh0rEPCN1FXg8}S zQ=r10@AvELRhJk(R_1Cc&&^$1)&GA&=aY8%I)iH;kJ;$f?&W{96D3FZD3yG|Tyl4g-T7xbPXMv<{yfP+oTy2?>D>Qx4ngw$%s=Kz87EAw}Kl#R-CAn5?#&P9$OPfq;XUt$r zy|N3p=h;P~^Hr-G4>yA6P@Ba+sxC`v(x6=jpU-Urq@^GR!B9Jv_JE#zQX>eA2itK zp`gg3u}`?^?mqRqVQ;73Ue1ydyL4~M_IDf0c9t6bdJ&>|qA0i_TSHLC+uJ+!)D+F8 zn=*>?0u^>Wym*vnvWBp_Ux@G}g(Hg=EO5wuT;D6aV$GV6$Y?fRsg#ri%b!msr{3I@ zYLtJ^W~*`1UBki)7Zy6J+S~8{$owf-$b*T=YO>g2|49or-*9nlcr-h{Wrg5@gbmiq zL5uOLkGp@*3{@}Z^$cx({KeHiOCYyu0jQyHYLoW59ZtIU>;KmteA7RpF7`az&rhfI zwa=TeS#5IZXK*nOl7D5=DXiu*A=uwGR6xJ}Mq>L);lvG@&(F8ONRfp7O03ReZhX@o~wAQ zO27TT9g5Ec7Wd}J&B$R}x@eJ8*!no#z182t6jk@jw{=?x9C&bfe*M2lKAR5?j{k~8 zT2{>|x#aojPVxDbrAv}sRvn$F?0(IA<_fzf^(qO5Od|KiUJIC|uV#DbM<=2kOHoA86{VRR0uZc`9kud7zkuW$Ap|7PCq_OLPW30CMK#dzW)B4$rgMSb4?iZX<=}8=ICodV z>(1K^Otb&i)dy9_oVk}-(Ra@7ch13(*Si$HFi5&9>frX3S6Mku-L*`}supaK(|okF8VAE%<#eVD;B>+GI7~pvo;@x4n6jb1XGmBF4;%q459z z-}nD7311)QW?OUo%+I5Sib8DVucYll?rb~TSpJSp5j37&pv3rGQ%(I_5l`;5N4qaQ z-?Kg_RKZl~yt$EY&fQ(6tM-DHk(}1uekVyrSdnpheBDlk>{h)GA@>T8%T|T*Sv+7c z^|ege66v8J#xh~?-&L#E8S&lSTU|aiDd_n0$InFHpKZ2PJhMIje%_nI;Myj^oatP! zPuc5(EVr9yB(}``(e#dGyUM@i->)szxP4F|=Kc5M^7U&TB^l~g?O^SHrMRp#a(CHU zuZ!QWMd!ctZHnZXXdFEqv_7oX;9!^D;WpmMjm+$?LiIcq-m(1q&~DF>CsNwMFmIOa zhXc$_@0aQAaC~!daeJI-|8ZrBAKfCIIw}_g7-Q-W?`1RJV!wT&!ye9nn&Y!~@SbNc zl;F5k>~{LtuGM})A=@Sw=>Gz>04LA>^768~TOZ2Y{_BraH_pObm~SvqfrD8iRi9rI)Y|icH#0L6)QZ}BN|iHWl7pXL->DfzFBY~-z1{s} zk~hoDkdm3U)!Rc3Wif8C zQCr=}aL`Por4O`p^qlSYm}Bwpxu(C~rryAK;DmmkkJgE8%}p{NJ5;ald~7(O;?|Z- zttGDh-%_d#3Jo^sNlb7}-JJ6CTYPHnQ=yyBZmLY(SM-rB%b8Dn|AS)fDI(LO@2m^) zEKm1}i(0tlM27C~CtoMzv9f#gA7VZ_cT!es+9qC=1c3!V%mP06s<0^R^Y-?BI_sr% zF^hZAoRquiXAW$=v@UkHZP)u(ud*zk&nadJu1fx(w9L=FgCj3u!oK%&%kK%Mi2QuH zeEyQ+jH-N*ieCykQCl(=Ui&G}Il-ZpNjrSqlv%T+Ofw|r|M`9Ye^_&8q3)r{iXttW z70$5M7wh;cGqfM<`6_>U$yNQ+W$&jgdz-WI7VFEkDG|r_pIlJ3|Lmshjn6h(@~Jv} ze0!mZ8Br6iFSP@&3sh^KDk19H9ESi8Eu1xOD#oMFE4EypLkf zR!*LLM8#|}i;g>k`M#)%#f258T6#2}%?uZO)arQZm)g!XYkYbSf|koZed8I@tgz?! zznZOYTg)pSr9C=%bMcZz+wMr%Pu4hRqt6*o^V!;g$xx*sVAAq=RbEd`-PV*Z_nZ6V z+wJ`D)86`fP1@3*E%ZEalO-rP_$u$ZWgk8q=0DxU%Kc^4n)v;32HNWL?!9c<(joES z&5@>q_Z~mLr#d^UEG=x4bo<$@*;B&pZ>Fnli{5W>QLZar{aY1JPMN0xoBP&pinEi9 z4+RPQ{tN11MygGmEWJ=2G>!04Iic=Si{os8$L_MFQ!X~d8jII{yP2+aS->f(#L;HU zU9LCtwtaJsjQiPr*XQ=xd7{>Jyf=SMys=Tf+<2O_j(3A|gYIq9lUFy0teo6np?%#v zr*BEg+gqXU_I|(TQTEhM1?<<71<#s3Oa+Z7I$o`^W^;}&ILKNAT3YQZVbQx$I|URalgCFE`B?`{@Pxx_9-&zHzO_GBqqJ>UDh~F$>IF=rMr|Xr6)hzJ00Dw9D6Jw5(4EQdsgw#vq}g@DsayO+Z&y*Oi>@RmwI` zcUN8fnjRLtXzpk8^E^uhbT&s_PP#VFtA6I0+h6w1TCSL;{;<(=fgsbD8gr(0`Bry^ zN5C9@mm!_LEvm8=vi1`C*xw@U+QQvRm3kZ|td`t$an8+bZB2L+9*d zw}wa7X&G}16JoXqxX*n4%_CCbxTM?vywp6ewN)u6xwkLgHTT#g2U{lRHXgw}B?ny` zKr^a5vQ{DalL8mJd8Vc=1r4U=-riHW`Ok;L{962#d3oys_3!0B_~b z&D&{pzwOo2tPQzURaVaaLRUHY?DP-&$=vyQ(LnZ)@WwvXmsc-Z#01GME{zi1tLAw% zIY)={$UKHiU2zo;TQB|m?zy?(n7dr%lDK{UJtwQ3>=0B=`T6OoTXS6HQ&G@PwT<_t zn9pGSzVWzR_9?wpdPmHiGPP=>XYa}1^giI_{Z&WI+2hoz{ngTcYwnx8`M%EOtd9I8 zUtY}#&Yo#@bhV=F=G9Z*COEt=-?QSgjmm@?mS5lBuitZL*VE zaEATEC2#$gTA$u-zrSjxwXIzL#$5mWxSEfyJKybk{bbqfyj#Ij*|ukYjExPy_~3cj zy$InoA?9+R^#LE%96rkCKQgnHcTs<&P%Zvb)qBq4zvpG);KePCV|r&`~EzVSdrNm7`H8F=BzER8`3#X}a* zpv9A?Pfe?$p8o7DIM?LHpYxI5dAsb_h^Rd~bLO4pFnDt$O?DPXvB3i)b#?cy?(U~8 zO|FxKHT+$S9~GT@6Hs>}(em?@wLca*@3B5w9&_Qdii!i1BGWU2?aSxaMXCAC(fIuQ zeE7#lC+zzer)UH(^O^YZ@$uE)tAFvw#j97>&CUBiZ=d)bxoV~_HOJ>|e0!eZW9RYN zA1;;Tg{vxO)lcvYtzBxfcJJDp{D-mH&c=0|H*ds6ZOu~cxb1T-@@>vlLz6ZZ8J~o& zudaSN$S!{(d;MOu;^*f=`5F@sx1F?nK1Ufed-=`l*vm)N>l-2(S&|$Y9$B*m%q{ci z$ggO4$f>8P8Q2_gDg6Ajo43PHP5Scp=c8<$_LT*|JsG&(oKsqK#LrH#q%nYXvC zon^q>U;gszwyO6VYgH2BnK~ldwo4b@cb6$_vH5x>*sI{MC1@Q>LM)TW{cCS$FUgX0 z(O{ELT4pZN@$t>2pqCqe8b`*?)t|j|X~?o|LjE2dzwg`>Dp=}y;i*_*q{=7F2=#Rd z4tE^qFJWD!s;si(^PlJ?sdv*}PS_aRyL4+tBu?TUCgQT>QX&H;X@Tk~g5UYT?=ddmyF6G{vhdk=cwa_f5UQga%- z3d^|h;o6(=m_Qr=LqWvkKI-NeqGV(SDn0T6!u*-+Y-Ki3q$*N=_G0W zJ=e=Jqjy{p)fZ%7U{CgRaSQ=3WJpnOcs(!rl2cuN{ifNMmVDdYEVKEtO2QhMP$rT4 zfk#g^XEMENlKZ6KyuoQllA0k$?q~1ozDpR~t zs8FAJCShL@pXJH4TUoM9JXM?g;y0*5#z$2I%$W-M9BRauZRx&kxwdE#Q|97h-w)J@ z33Zq=*1dk@duQSE#q4($%KC5iY+%vxX1Jc(!n)1(ed0~CW2+L^2Jh0k{qwen_92_R zWYBWY#fQ&>*35wRlrLjZurXY)$2-s7lmE$$E2r+B_4=Id-M*%F>f0|h@6WYIC;2x# zvi{an$p)GZRA3a^Aj7D;uekWr68F0kr)9^lk-D<-o59J`XTvp94>q*2Jo0ULWPR_5 zY?a;Ir$-h(yPN~kwm}B8l6(>P*&oBS6?T+{*N8Uw0 z$^z0ahk%qTFbWw6AK1e-WAeh#)nP^kPxU9gJ852@v-*4H*2EBF zOU><7-II?=biH5lQp;=#I7}N@7I`xmo}a<9cEg4MtF3Rf+AGdYTqC=7akUV)jPB-5 z5qv9bVy(8lT$E?BY15{VZNl6py}ktsjL&|SB`jAzl=3##<4KEE=Z4wOL1}|=wQ$3C zYjt(^$`(V#s8_2trv|<(xfx^P&h&C?Sbp8&x5d@QOP@~Lcj;euo}Mt- zt@GpI3V{Vb`q)$FHA7vcQSJI}mw^a19 zT(S}J;p|w#;u^DXI;&T#f(B#!+$Fn>kG?%Eu-)9-C@b2-FSm2bl9I{OT-PpHqVmP= ziZ}^?x&TKVPZ4v{gifQAnT~;5f_6CK9 zsy==CG$<@=*|doDD^~{2XJuk^wB1}{u)klO;qPvvq9-1uw+bI0Tj|+cpv>6Fn_uVX z*eG;|WA(|?r-N?`W-b2Sn_gY1Cv<^DW1lG}Xc+}4Asu+gqOe4;3N%m*S%tmz$FX#Q z`xOgSyeeMjT~d=zT)NQI3ql-c)COZPW1T&1aqN{JH0qn;Pu_8c%54A#g$5t=6#rv}dIdB$?e7 ze(=8g?d}_8Y)0p|zuh$jyzU=71TleuGh!;k(e0ttZEs;G9Dwv}kY#L}dt;qzPESx zZ|Q?(0+8(o@ENL`EbG2FdYSeO0g$IQh%)}$cVm^cY$DIm?aI4mx}^aFOxQL#?cw8d zkT8Xk0(Oj5`wpicodxAXqeq8HbZ!~uh#^PXUyci(bNAaVb%mG< zA@LcS<=Jf^b$8O;LvEm?=x|B+!F%sZz&F8r8REzuU!r Q1_lNOPgg&ebxsLQ0Q+f?egFUf literal 0 HcmV?d00001 diff --git a/akka-docs/src/main/paradox/typed/index-persistence.md b/akka-docs/src/main/paradox/typed/index-persistence.md index 1604b1aece..5976589f98 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) +* [active-active](persistence-active-active.md) * [cqrs](cqrs.md) * [persistence-style](persistence-style.md) * [persistence-snapshot](persistence-snapshot.md) diff --git a/akka-docs/src/main/paradox/typed/persistence-active-active.md b/akka-docs/src/main/paradox/typed/persistence-active-active.md new file mode 100644 index 0000000000..bfa8dce78a --- /dev/null +++ b/akka-docs/src/main/paradox/typed/persistence-active-active.md @@ -0,0 +1,192 @@ +# Active-Active 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, `EventSourcedBehaviors`s are unavailable. + + +Active-active 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 active-active is enabled +there is no longer the single writer principle as there is with a normal `EventSourcedBehavior`. + +The state of an active-active `EventSourcedBehavior` is **eventually consistent**. Event replication may be delayed +due to network partitions and outages and the event handler and those reading the state must be designed to handle this. + +## Relaxing the single writer p`rinciple for availability + +Taking the example of using active-active 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 and the events can just 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 active-active detects these conflicts. + +## API + +The same API as regular `EventSourcedBehavior`s is used to define the logic. To enable an entity for active-active +replication use the factory methods on @api[ActiveActiveEventSourcing]. + +All replicas need to be known up front: + +Scala +: @@snip [ActiveActiveCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala) { #replicas } + +Then to enable replication create the event sourced behavior with the factory method: + +Scala +: @@snip [ActiveActiveCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala) { #factory } + +The factory takes in: + +* EntityID: this will be used as part of the underlying persistenceId +* Replica: Which replica this instance is +* All Replicas and the query plugin used to read their events + +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 [ActiveActiveCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala) { #factory-shared} + + +The function passed to both factory methods return an `EventSourcedBehavior` and provide access to the @api[ActiveActiveContext] 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. + +* 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 + +TODO example once CRDTs are in + +### 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 @api[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 @api[LwwTime] the greatest timestamp wins. The replica +identifier is used if the two timestamps are equal, and then the one from the data center sorted first in +alphanumeric order wins. + +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 data centers 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 + +TODO https://github.com/akka/akka/issues/29318 + +## 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 deliver order + +Causal delivery order means that events persisted in one data center are read in the same order in other data centers. The order of concurrent events is undefined, which should be no problem +when using [CRDT's](#conflict-free-replicated-data-types) +and otherwise will be detected via the `ActiveActiveContext` 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 data center 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 data center may also see the events as either "e1, e3, e2" or as "e1, e2, e3". + +### Concurrent updates + +Active-active automatically tracks causality between events from different replias 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 diff --git a/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AAAuctionExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala similarity index 100% rename from akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AAAuctionExampleSpec.scala rename to akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala diff --git a/akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala similarity index 100% rename from akka-persistence-typed-tests/src/test/scala/docs.akka.persistence.typed/AABlogExampleSpec.scala rename to akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala new file mode 100644 index 0000000000..55196657d3 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala @@ -0,0 +1,28 @@ +package docs.akka.persistence.typed + +import akka.persistence.typed.ReplicaId +import akka.persistence.typed.scaladsl.{ ActiveActiveEventSourcing, EventSourcedBehavior } + +object ActiveActiveCompileOnlySpec { + + //#replicas + val DCA = ReplicaId("DC-A") + val DCB = ReplicaId("DC-B") + val AllReplicas = Set(DCA, DCB) + //#replicas + + val queryPluginId = "" + + //#factory-shared + ActiveActiveEventSourcing.withSharedJournal("entityId", DCA, AllReplicas, queryPluginId) { context => + EventSourcedBehavior(???, ???, ???, ???) + } + //#factory-shared + + //#factory + ActiveActiveEventSourcing("entityId", DCA, Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { context => + EventSourcedBehavior(???, ???, ???, ???) + } + //#factory + +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala index 77340336f5..6c15fcc5d6 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala @@ -39,13 +39,14 @@ final case class LwwTime(timestamp: Long, originDc: ReplicaId) { // FIXME docs trait ActiveActiveContext { - def origin: ReplicaId - def concurrent: Boolean + def persistenceId: PersistenceId def replicaId: ReplicaId def allReplicas: Set[ReplicaId] - def persistenceId: PersistenceId - def recoveryRunning: Boolean def entityId: String + + def origin: ReplicaId + def concurrent: Boolean + def recoveryRunning: Boolean def currentTimeMillis(): Long } From 7e914284282a318c79472a9aaa0264a5d3d52c8d Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Thu, 16 Jul 2020 16:06:04 +0100 Subject: [PATCH 12/50] Metadata for snapshots for active active (#29362) --- akka-docs/src/main/paradox/persistence.md | 6 +- .../typed/ActiveActiveSnapshotSpec.scala | 113 ++++++++++++++++++ .../persistence/typed/ActiveActiveSpec.scala | 61 +++++----- .../internal/EventSourcedBehaviorImpl.scala | 4 + .../typed/internal/ExternalInteractions.scala | 12 +- .../typed/internal/ReplayingSnapshot.scala | 51 ++++---- ...9217-active-active-event-sourcing.excludes | 7 ++ .../akka/persistence/SnapshotProtocol.scala | 67 ++++++++++- 8 files changed, 256 insertions(+), 65 deletions(-) create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSnapshotSpec.scala 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-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSnapshotSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSnapshotSpec.scala new file mode 100644 index 0000000000..8eeda55c15 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSnapshotSpec.scala @@ -0,0 +1,113 @@ +/* + * 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.ActiveActiveEventSourcing +import org.scalatest.concurrent.Eventually +import org.scalatest.wordspec.AnyWordSpecLike + +object ActiveActiveSnapshotSpec { + + import ActiveActiveSpec._ + + 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] = { + ActiveActiveEventSourcing.withSharedJournal( + entityId, + replicaId, + AllReplicas, + PersistenceTestKitReadJournal.Identifier)(aaContext => + eventSourcedBehavior(aaContext, probe).snapshotWhen((_, _, sequenceNr) => sequenceNr % 2 == 0)) + + } +} + +class ActiveActiveSnapshotSpec + extends ScalaTestWithActorTestKit( + PersistenceTestKitPlugin.config.withFallback(PersistenceTestKitSnapshotPlugin.config)) + with AnyWordSpecLike + with LogCapturing + with Eventually { + import ActiveActiveSpec._ + import ActiveActiveSnapshotSpec._ + + 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") + + "ActiveActive" should { + "recover state from snapshots" in { + val entityId = nextEntityId + val persistenceIdR1 = s"$entityId|R1" + val persistenceIdR2 = s"$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( + PersistenceId.replicatedUniqueId(entityId, R1), + 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( + PersistenceId.replicatedUniqueId(entityId, R1), + 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/ActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala index e9f33b0bc2..5ae2cf87a0 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala @@ -13,9 +13,8 @@ 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.scaladsl.ActiveActiveEventSourcing -import akka.persistence.typed.scaladsl.Effect -import akka.persistence.typed.scaladsl.EventSourcedBehavior +import akka.persistence.typed.scaladsl.{ ActiveActiveContext, ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import akka.serialization.jackson.CborSerializable import org.scalatest.concurrent.Eventually import org.scalatest.wordspec.AnyWordSpecLike @@ -30,10 +29,38 @@ object ActiveActiveSpec { case class GetReplica(replyTo: ActorRef[(ReplicaId, Set[ReplicaId])]) extends Command case object Stop extends Command - case class State(all: List[String]) + 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( + aaContext: ActiveActiveContext, + probe: Option[ActorRef[EventAndContext]]): EventSourcedBehavior[Command, String, State] = { + EventSourcedBehavior[Command, String, State]( + aaContext.persistenceId, + State(Nil), + (state, command) => + command match { + case GetState(replyTo) => + replyTo ! state + Effect.none + case GetReplica(replyTo) => + replyTo.tell((aaContext.replicaId, aaContext.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, aaContext.origin, aaContext.recoveryRunning, aaContext.concurrent)) + state.copy(all = event :: state.all) + }) + } + def testBehavior( entityId: String, replicaId: String, @@ -42,30 +69,7 @@ object ActiveActiveSpec { entityId, ReplicaId(replicaId), AllReplicas, - PersistenceTestKitReadJournal.Identifier)( - aaContext => - EventSourcedBehavior[Command, String, State]( - aaContext.persistenceId, - State(Nil), - (state, command) => - command match { - case GetState(replyTo) => - replyTo ! state - Effect.none - case GetReplica(replyTo) => - replyTo.tell((aaContext.replicaId, aaContext.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, aaContext.origin, aaContext.recoveryRunning, aaContext.concurrent)) - state.copy(all = event :: state.all) - })) + PersistenceTestKitReadJournal.Identifier)(aaContext => eventSourcedBehavior(aaContext, probe)) } @@ -338,6 +342,5 @@ class ActiveActiveSpec eventProbeR1Take2.expectMessage( EventAndContext("from r2", ReplicaId("R2"), recoveryRunning = true, concurrent = true)) } - } } 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 95e474860e..b9f032786d 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 @@ -287,6 +287,10 @@ private[akka] final case class ReplicatedEventMetaData( version: VersionVector, concurrent: Boolean) // whether when the event handler was executed the event was concurrent +// FIXME serializer +@InternalApi +private[akka] final case class ReplicatedSnapshotMetaData(version: VersionVector, seenPerReplica: Map[ReplicaId, Long]) + /** * An event replicated from a different replica. * 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 3fd33ab510..510099c1ed 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 @@ -18,7 +18,6 @@ import akka.annotation.InternalStableApi import akka.persistence._ import akka.persistence.JournalProtocol.ReplayMessages import akka.persistence.SnapshotProtocol.LoadSnapshot -import akka.persistence.typed.internal.JournalInteractions.EventOrTaggedOrReplicated import akka.util.{ unused, OptionVal } /** INTERNAL API */ @@ -191,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.activeActive 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/ReplayingSnapshot.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingSnapshot.scala index 9f2d464ec3..524901b342 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.{ RecoveryFailed, ReplicaId } import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState import akka.util.unused +import akka.actor.typed.scaladsl.LoggerOps /** * INTERNAL API @@ -146,14 +147,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: Map[ReplicaId, Long], version: VersionVector) = 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.meta match { + case Some(rm: ReplicatedSnapshotMetaData) => (metadata.sequenceNr, rm.seenPerReplica, rm.version) + case _ => (metadata.sequenceNr, Map.empty.withDefaultValue(0L), VersionVector.empty) + } + case None => (0L, Map.empty.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) @@ -163,26 +182,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(), - VersionVector.empty, - // FIXME seqNrs for other replicas needs to come from snapshot. - seenSeqNrPerReplica = - setup.activeActive.map(_.allReplicas.map(replica => replica -> 0L).toMap).getOrElse(Map.empty))) - } - } diff --git a/akka-persistence/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes b/akka-persistence/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes index 91d5ea01ed..3e13002376 100644 --- a/akka-persistence/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes +++ b/akka-persistence/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes @@ -2,3 +2,10 @@ 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*") diff --git a/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala b/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala index ec4559ce1b..91d9b25755 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,54 @@ 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 meta a journal can optionally support persisting metadata separate to the domain state */ -@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 meta: 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) + } + + // 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, meta) + + override def toString = s"SnapshotMetadata($persistenceId, $sequenceNr, $timestamp, $meta)" + + // 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 +66,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 } /** From 116c13677a332978441b020ed683826d8bf77ac6 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Thu, 16 Jul 2020 20:56:57 +0100 Subject: [PATCH 13/50] Port CRDTs from multi dc (#29372) * Metadata for snapshots for active active * Port CRDTs from multi dc * Review feedback --- .../typed/ActiveActiveBaseSpec.scala | 30 + .../persistence/typed/crdt/CounterSpec.scala | 132 + .../akka/persistence/typed/crdt/LwwSpec.scala | 110 + .../persistence/typed/crdt/ORSetSpec.scala | 102 + .../persistence/typed/AABlogExampleSpec.scala | 1 + .../typed/serialization/Crdts.java | 6665 +++++++++++++++++ .../src/main/protobuf/Crdts.proto | 50 + .../src/main/resources/reference.conf | 15 + .../akka/persistence/typed/crdt/Counter.scala | 33 + .../akka/persistence/typed/crdt/LwwTime.scala | 36 + .../akka/persistence/typed/crdt/ORSet.scala | 503 ++ .../akka/persistence/typed/crdt/OpCrdt.scala | 15 + .../scaladsl/ActiveActiveEventSourcing.scala | 28 - .../typed/serialization/CrdtSerializer.scala | 269 + build.sbt | 4 + 15 files changed, 7965 insertions(+), 28 deletions(-) create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveBaseSpec.scala create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/CounterSpec.scala create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/LwwSpec.scala create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/ORSetSpec.scala create mode 100644 akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/Crdts.java create mode 100644 akka-persistence-typed/src/main/protobuf/Crdts.proto create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/Counter.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/LwwTime.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/ORSet.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/OpCrdt.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/CrdtSerializer.scala diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveBaseSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveBaseSpec.scala new file mode 100644 index 0000000000..cfa8a820dc --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveBaseSpec.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 ActiveActiveBaseSpec { + val R1 = ReplicaId("R1") + val R2 = ReplicaId("R2") + val AllReplicas = Set(R1, R2) +} + +abstract class ActiveActiveBaseSpec + 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/crdt/CounterSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/crdt/CounterSpec.scala new file mode 100644 index 0000000000..6393784eed --- /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.crdt.CounterSpec.PlainCounter.{ Decrement, Get, Increment } +import akka.persistence.typed.scaladsl.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import akka.persistence.typed.{ ActiveActiveBaseSpec, ReplicaId } + +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 ActiveActiveBaseSpec._ + + def apply( + entityId: String, + replicaId: ReplicaId, + snapshotEvery: Long = 100, + eventProbe: Option[ActorRef[Counter.Updated]] = None) = + Behaviors.setup[PlainCounter.Command] { context => + ActiveActiveEventSourcing.withSharedJournal( + 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 ActiveActiveBaseSpec { + + import CounterSpec._ + import ActiveActiveBaseSpec._ + + "Active active 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..8c0718b39a --- /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.scaladsl.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import akka.persistence.typed.{ ActiveActiveBaseSpec, ReplicaId } +import akka.serialization.jackson.CborSerializable + +object LwwSpec { + + import ActiveActiveBaseSpec._ + + 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] = { + ActiveActiveEventSourcing.withSharedJournal( + entityId, + replica, + AllReplicas, + PersistenceTestKitReadJournal.Identifier) { aaContext => + EventSourcedBehavior[Command, Event, Registry]( + aaContext.persistenceId, + Registry("", LwwTime(Long.MinValue, aaContext.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, aaContext.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 ActiveActiveBaseSpec { + import LwwSpec._ + import ActiveActiveBaseSpec._ + + 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 Active Active 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..ce65f0b9da --- /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.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import akka.persistence.typed.{ ActiveActiveBaseSpec, ReplicaId } +import ORSetSpec.ORSetEntity._ +import akka.persistence.typed.ActiveActiveBaseSpec.{ R1, R2 } +import akka.persistence.typed.crdt.ORSetSpec.ORSetEntity + +import scala.util.Random + +object ORSetSpec { + + import ActiveActiveBaseSpec._ + + 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] = { + + ActiveActiveEventSourcing.withSharedJournal( + entityId, + replica, + AllReplicas, + PersistenceTestKitReadJournal.Identifier) { aaContext => + EventSourcedBehavior[Command, ORSet.DeltaOp, ORSet[String]]( + aaContext.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 ActiveActiveBaseSpec { + + 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 Active Active 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/docs/akka/persistence/typed/AABlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala index 6c297f4ea1..f67fd19226 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala @@ -11,6 +11,7 @@ import akka.actor.typed.scaladsl.{ ActorContext, Behaviors } import akka.persistence.testkit.PersistenceTestKitPlugin import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal import akka.persistence.typed.ReplicaId +import akka.persistence.typed.crdt.LwwTime import akka.persistence.typed.scaladsl._ import akka.serialization.jackson.CborSerializable import org.scalatest.concurrent.{ Eventually, ScalaFutures } diff --git a/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/Crdts.java b/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/Crdts.java new file mode 100644 index 0000000000..dad9330ab0 --- /dev/null +++ b/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/Crdts.java @@ -0,0 +1,6665 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: Crdts.proto + +package akka.persistence.typed.serialization; + +public final class Crdts { + private Crdts() {} + + 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.Crdts.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.Crdts.internal_static_Counter_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts.internal_static_Counter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.Counter.class, + akka.persistence.typed.serialization.Crdts.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.Crdts.Counter)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.Crdts.Counter other = + (akka.persistence.typed.serialization.Crdts.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.Crdts.Counter parseFrom( + java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.Counter parseFrom( + akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.Counter parseFrom(byte[] data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.Counter parseFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.Counter parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.CounterOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.Crdts.internal_static_Counter_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts.internal_static_Counter_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.Counter.class, + akka.persistence.typed.serialization.Crdts.Counter.Builder.class); + } + + // Construct using akka.persistence.typed.serialization.Crdts.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.Crdts.internal_static_Counter_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.Counter getDefaultInstanceForType() { + return akka.persistence.typed.serialization.Crdts.Counter.getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.Counter build() { + akka.persistence.typed.serialization.Crdts.Counter result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.Counter buildPartial() { + akka.persistence.typed.serialization.Crdts.Counter result = + new akka.persistence.typed.serialization.Crdts.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.Crdts.Counter) { + return mergeFrom((akka.persistence.typed.serialization.Crdts.Counter) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.Counter other) { + if (other == akka.persistence.typed.serialization.Crdts.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.Crdts.Counter parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.Crdts.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.Crdts.Counter DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.Counter(); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.internal_static_CounterUpdate_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_CounterUpdate_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.CounterUpdate.class, + akka.persistence.typed.serialization.Crdts.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.Crdts.CounterUpdate)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.Crdts.CounterUpdate other = + (akka.persistence.typed.serialization.Crdts.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.Crdts.CounterUpdate parseFrom( + java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.CounterUpdate parseFrom( + akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.CounterUpdate parseFrom(byte[] data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.CounterUpdate parseFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.CounterUpdate parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.CounterUpdateOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.Crdts.internal_static_CounterUpdate_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_CounterUpdate_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.CounterUpdate.class, + akka.persistence.typed.serialization.Crdts.CounterUpdate.Builder.class); + } + + // Construct using akka.persistence.typed.serialization.Crdts.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.Crdts.internal_static_CounterUpdate_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.CounterUpdate getDefaultInstanceForType() { + return akka.persistence.typed.serialization.Crdts.CounterUpdate.getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.CounterUpdate build() { + akka.persistence.typed.serialization.Crdts.CounterUpdate result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.CounterUpdate buildPartial() { + akka.persistence.typed.serialization.Crdts.CounterUpdate result = + new akka.persistence.typed.serialization.Crdts.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.Crdts.CounterUpdate) { + return mergeFrom((akka.persistence.typed.serialization.Crdts.CounterUpdate) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.CounterUpdate other) { + if (other == akka.persistence.typed.serialization.Crdts.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.Crdts.CounterUpdate parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.Crdts.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.Crdts.CounterUpdate DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.CounterUpdate(); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.VersionVector getVvector(); + /** required .VersionVector vvector = 2; */ + akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder getVvectorOrBuilder(); + + /** repeated .VersionVector dots = 3; */ + java.util.List getDotsList(); + /** repeated .VersionVector dots = 3; */ + akka.persistence.typed.serialization.Crdts.VersionVector getDots(int index); + /** repeated .VersionVector dots = 3; */ + int getDotsCount(); + /** repeated .VersionVector dots = 3; */ + java.util.List + getDotsOrBuilderList(); + /** repeated .VersionVector dots = 3; */ + akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) != 0)) { + subBuilder = vvector_.toBuilder(); + } + vvector_ = + input.readMessage( + akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector>(); + mutable_bitField0_ |= 0x00000004; + } + dots_.add( + input.readMessage( + akka.persistence.typed.serialization.Crdts.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.Crdts.internal_static_ORSet_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts.internal_static_ORSet_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.ORSet.class, + akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.VersionVector getVvector() { + return vvector_ == null + ? akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance() + : vvector_; + } + /** required .VersionVector vvector = 2; */ + public akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder getVvectorOrBuilder() { + return vvector_ == null + ? akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance() + : vvector_; + } + + public static final int DOTS_FIELD_NUMBER = 3; + private java.util.List dots_; + /** repeated .VersionVector dots = 3; */ + public java.util.List getDotsList() { + return dots_; + } + /** repeated .VersionVector dots = 3; */ + public java.util.List< + ? extends akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + getDotsOrBuilderList() { + return dots_; + } + /** repeated .VersionVector dots = 3; */ + public int getDotsCount() { + return dots_.size(); + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.Crdts.VersionVector getDots(int index) { + return dots_.get(index); + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.Crdts.ORSet other = + (akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet parseFrom( + java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet parseFrom( + akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet parseFrom(byte[] data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet parseFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.ORSetOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.Crdts.internal_static_ORSet_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts.internal_static_ORSet_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.ORSet.class, + akka.persistence.typed.serialization.Crdts.ORSet.Builder.class); + } + + // Construct using akka.persistence.typed.serialization.Crdts.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.Crdts.internal_static_ORSet_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.ORSet getDefaultInstanceForType() { + return akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.ORSet build() { + akka.persistence.typed.serialization.Crdts.ORSet result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.ORSet buildPartial() { + akka.persistence.typed.serialization.Crdts.ORSet result = + new akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet) { + return mergeFrom((akka.persistence.typed.serialization.Crdts.ORSet) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.ORSet other) { + if (other == akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector vvector_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.VersionVector, + akka.persistence.typed.serialization.Crdts.VersionVector.Builder, + akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector getVvector() { + if (vvectorBuilder_ == null) { + return vvector_ == null + ? akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance() + : vvector_; + } else { + return vvectorBuilder_.getMessage(); + } + } + /** required .VersionVector vvector = 2; */ + public Builder setVvector(akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.VersionVector value) { + if (vvectorBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) + && vvector_ != null + && vvector_ + != akka.persistence.typed.serialization.Crdts.VersionVector + .getDefaultInstance()) { + vvector_ = + akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Builder getVvectorBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getVvectorFieldBuilder().getBuilder(); + } + /** required .VersionVector vvector = 2; */ + public akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder + getVvectorOrBuilder() { + if (vvectorBuilder_ != null) { + return vvectorBuilder_.getMessageOrBuilder(); + } else { + return vvector_ == null + ? akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance() + : vvector_; + } + } + /** required .VersionVector vvector = 2; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.VersionVector, + akka.persistence.typed.serialization.Crdts.VersionVector.Builder, + akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + getVvectorFieldBuilder() { + if (vvectorBuilder_ == null) { + vvectorBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.VersionVector, + akka.persistence.typed.serialization.Crdts.VersionVector.Builder, + akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder>( + getVvector(), getParentForChildren(), isClean()); + vvector_ = null; + } + return vvectorBuilder_; + } + + private java.util.List dots_ = + java.util.Collections.emptyList(); + + private void ensureDotsIsMutable() { + if (!((bitField0_ & 0x00000004) != 0)) { + dots_ = + new java.util.ArrayList( + dots_); + bitField0_ |= 0x00000004; + } + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.VersionVector, + akka.persistence.typed.serialization.Crdts.VersionVector.Builder, + akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + dotsBuilder_; + + /** repeated .VersionVector dots = 3; */ + public java.util.List + 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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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 + 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.Crdts.VersionVector.Builder getDotsBuilder( + int index) { + return getDotsFieldBuilder().getBuilder(index); + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVectorOrBuilder> + getDotsOrBuilderList() { + if (dotsBuilder_ != null) { + return dotsBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(dots_); + } + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.Crdts.VersionVector.Builder addDotsBuilder() { + return getDotsFieldBuilder() + .addBuilder( + akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance()); + } + /** repeated .VersionVector dots = 3; */ + public akka.persistence.typed.serialization.Crdts.VersionVector.Builder addDotsBuilder( + int index) { + return getDotsFieldBuilder() + .addBuilder( + index, + akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance()); + } + /** repeated .VersionVector dots = 3; */ + public java.util.List + getDotsBuilderList() { + return getDotsFieldBuilder().getBuilderList(); + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.VersionVector, + akka.persistence.typed.serialization.Crdts.VersionVector.Builder, + akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + getDotsFieldBuilder() { + if (dotsBuilder_ == null) { + dotsBuilder_ = + new akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.VersionVector, + akka.persistence.typed.serialization.Crdts.VersionVector.Builder, + akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.ORSet(); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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 + getEntriesList(); + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.EntryOrBuilder> + getEntriesOrBuilderList(); + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry>(); + mutable_bitField0_ |= 0x00000001; + } + entries_.add( + input.readMessage( + akka.persistence.typed.serialization.Crdts.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.Crdts.internal_static_ORSetDeltaGroup_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_ORSetDeltaGroup_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.class, + akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.ORSet getUnderlying(); + /** required .ORSet underlying = 2; */ + akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaOp value = + akka.persistence.typed.serialization.Crdts.ORSetDeltaOp.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(1, rawValue); + } else { + bitField0_ |= 0x00000001; + operation_ = rawValue; + } + break; + } + case 18: + { + akka.persistence.typed.serialization.Crdts.ORSet.Builder subBuilder = null; + if (((bitField0_ & 0x00000002) != 0)) { + subBuilder = underlying_.toBuilder(); + } + underlying_ = + input.readMessage( + akka.persistence.typed.serialization.Crdts.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.Crdts + .internal_static_ORSetDeltaGroup_Entry_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.class, + akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaOp getOperation() { + @SuppressWarnings("deprecation") + akka.persistence.typed.serialization.Crdts.ORSetDeltaOp result = + akka.persistence.typed.serialization.Crdts.ORSetDeltaOp.valueOf(operation_); + return result == null + ? akka.persistence.typed.serialization.Crdts.ORSetDeltaOp.Add + : result; + } + + public static final int UNDERLYING_FIELD_NUMBER = 2; + private akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet getUnderlying() { + return underlying_ == null + ? akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance() + : underlying_; + } + /** required .ORSet underlying = 2; */ + public akka.persistence.typed.serialization.Crdts.ORSetOrBuilder getUnderlyingOrBuilder() { + return underlying_ == null + ? akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry other = + (akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry parseFrom( + java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry parseFrom( + akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry parseFrom( + byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.ORSetDeltaGroup.EntryOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.Crdts + .internal_static_ORSetDeltaGroup_Entry_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.class, + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.Crdts.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.Crdts + .internal_static_ORSetDeltaGroup_Entry_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry build() { + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry buildPartial() { + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry result = + new akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry) { + return mergeFrom( + (akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry other) { + if (other + == akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaOp getOperation() { + @SuppressWarnings("deprecation") + akka.persistence.typed.serialization.Crdts.ORSetDeltaOp result = + akka.persistence.typed.serialization.Crdts.ORSetDeltaOp.valueOf(operation_); + return result == null + ? akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.ORSet underlying_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.ORSet, + akka.persistence.typed.serialization.Crdts.ORSet.Builder, + akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet getUnderlying() { + if (underlyingBuilder_ == null) { + return underlying_ == null + ? akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance() + : underlying_; + } else { + return underlyingBuilder_.getMessage(); + } + } + /** required .ORSet underlying = 2; */ + public Builder setUnderlying(akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.ORSet value) { + if (underlyingBuilder_ == null) { + if (((bitField0_ & 0x00000002) != 0) + && underlying_ != null + && underlying_ + != akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance()) { + underlying_ = + akka.persistence.typed.serialization.Crdts.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.Crdts.ORSet.Builder getUnderlyingBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getUnderlyingFieldBuilder().getBuilder(); + } + /** required .ORSet underlying = 2; */ + public akka.persistence.typed.serialization.Crdts.ORSetOrBuilder getUnderlyingOrBuilder() { + if (underlyingBuilder_ != null) { + return underlyingBuilder_.getMessageOrBuilder(); + } else { + return underlying_ == null + ? akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance() + : underlying_; + } + } + /** required .ORSet underlying = 2; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.ORSet, + akka.persistence.typed.serialization.Crdts.ORSet.Builder, + akka.persistence.typed.serialization.Crdts.ORSetOrBuilder> + getUnderlyingFieldBuilder() { + if (underlyingBuilder_ == null) { + underlyingBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.ORSet, + akka.persistence.typed.serialization.Crdts.ORSet.Builder, + akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry(); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public static final int ENTRIES_FIELD_NUMBER = 1; + private java.util.List + entries_; + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public java.util.List + getEntriesList() { + return entries_; + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public java.util.List< + ? extends akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup.Entry getEntries(int index) { + return entries_.get(index); + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup other = + (akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup parseFrom( + java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup parseFrom( + akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup parseFrom(byte[] data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup parseFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.ORSetDeltaGroupOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.Crdts + .internal_static_ORSetDeltaGroup_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_ORSetDeltaGroup_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.class, + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Builder.class); + } + + // Construct using akka.persistence.typed.serialization.Crdts.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.Crdts + .internal_static_ORSetDeltaGroup_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup build() { + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup buildPartial() { + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup result = + new akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup) { + return mergeFrom((akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup other) { + if (other + == akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.util.List + entries_ = java.util.Collections.emptyList(); + + private void ensureEntriesIsMutable() { + if (!((bitField0_ & 0x00000001) != 0)) { + entries_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry>(entries_); + bitField0_ |= 0x00000001; + } + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry, + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder, + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder> + entriesBuilder_; + + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public java.util.List + 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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.ORSetDeltaGroup.Entry.Builder + getEntriesBuilder(int index) { + return getEntriesFieldBuilder().getBuilder(index); + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.ORSetDeltaGroup.Entry.Builder + addEntriesBuilder() { + return getEntriesFieldBuilder() + .addBuilder( + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + .getDefaultInstance()); + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder + addEntriesBuilder(int index) { + return getEntriesFieldBuilder() + .addBuilder( + index, + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + .getDefaultInstance()); + } + /** repeated .ORSetDeltaGroup.Entry entries = 1; */ + public java.util.List< + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder> + getEntriesBuilderList() { + return getEntriesFieldBuilder().getBuilderList(); + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry, + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder, + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder> + getEntriesFieldBuilder() { + if (entriesBuilder_ == null) { + entriesBuilder_ = + new akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry, + akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder, + akka.persistence.typed.serialization.Crdts.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.Crdts.ORSetDeltaGroup + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup(); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.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.Crdts.VersionVector.EntryOrBuilder> + getEntriesOrBuilderList(); + /** repeated .VersionVector.Entry entries = 1; */ + akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry>(); + mutable_bitField0_ |= 0x00000001; + } + entries_.add( + input.readMessage( + akka.persistence.typed.serialization.Crdts.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.Crdts.internal_static_VersionVector_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_VersionVector_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.VersionVector.class, + akka.persistence.typed.serialization.Crdts.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.Crdts + .internal_static_VersionVector_Entry_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_VersionVector_Entry_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.VersionVector.Entry.class, + akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.Crdts.VersionVector.Entry other = + (akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry parseFrom( + java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry parseFrom( + akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry parseFrom( + byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.VersionVector.EntryOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.Crdts + .internal_static_VersionVector_Entry_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_VersionVector_Entry_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.VersionVector.Entry.class, + akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.Crdts.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.Crdts + .internal_static_VersionVector_Entry_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.VersionVector.Entry + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.Crdts.VersionVector.Entry + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.VersionVector.Entry build() { + akka.persistence.typed.serialization.Crdts.VersionVector.Entry result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.VersionVector.Entry buildPartial() { + akka.persistence.typed.serialization.Crdts.VersionVector.Entry result = + new akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry) { + return mergeFrom( + (akka.persistence.typed.serialization.Crdts.VersionVector.Entry) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.Crdts.VersionVector.Entry other) { + if (other + == akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.VersionVector.Entry(); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + public static final int ENTRIES_FIELD_NUMBER = 1; + private java.util.List entries_; + /** repeated .VersionVector.Entry entries = 1; */ + public java.util.List + getEntriesList() { + return entries_; + } + /** repeated .VersionVector.Entry entries = 1; */ + public java.util.List< + ? extends akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector.Entry getEntries(int index) { + return entries_.get(index); + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.Crdts.VersionVector other = + (akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector parseFrom( + java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector parseFrom( + akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector parseFrom(byte[] data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector parseFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector parseDelimitedFrom( + java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.VersionVectorOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.Crdts.internal_static_VersionVector_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.Crdts + .internal_static_VersionVector_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.Crdts.VersionVector.class, + akka.persistence.typed.serialization.Crdts.VersionVector.Builder.class); + } + + // Construct using akka.persistence.typed.serialization.Crdts.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.Crdts.internal_static_VersionVector_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.VersionVector getDefaultInstanceForType() { + return akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.VersionVector build() { + akka.persistence.typed.serialization.Crdts.VersionVector result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.Crdts.VersionVector buildPartial() { + akka.persistence.typed.serialization.Crdts.VersionVector result = + new akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector) { + return mergeFrom((akka.persistence.typed.serialization.Crdts.VersionVector) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.VersionVector other) { + if (other == akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.Crdts.VersionVector) e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private java.util.List + entries_ = java.util.Collections.emptyList(); + + private void ensureEntriesIsMutable() { + if (!((bitField0_ & 0x00000001) != 0)) { + entries_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.Crdts.VersionVector.Entry>(entries_); + bitField0_ |= 0x00000001; + } + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.VersionVector.Entry, + akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder, + akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder> + entriesBuilder_; + + /** repeated .VersionVector.Entry entries = 1; */ + public java.util.List + 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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.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.Crdts.VersionVector.Entry.Builder + getEntriesBuilder(int index) { + return getEntriesFieldBuilder().getBuilder(index); + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.Crdts.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.Crdts.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.Crdts.VersionVector.Entry.Builder + addEntriesBuilder() { + return getEntriesFieldBuilder() + .addBuilder( + akka.persistence.typed.serialization.Crdts.VersionVector.Entry + .getDefaultInstance()); + } + /** repeated .VersionVector.Entry entries = 1; */ + public akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder + addEntriesBuilder(int index) { + return getEntriesFieldBuilder() + .addBuilder( + index, + akka.persistence.typed.serialization.Crdts.VersionVector.Entry + .getDefaultInstance()); + } + /** repeated .VersionVector.Entry entries = 1; */ + public java.util.List + getEntriesBuilderList() { + return getEntriesFieldBuilder().getBuilderList(); + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.VersionVector.Entry, + akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder, + akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder> + getEntriesFieldBuilder() { + if (entriesBuilder_ == null) { + entriesBuilder_ = + new akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.Crdts.VersionVector.Entry, + akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder, + akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.VersionVector(); + } + + public static akka.persistence.typed.serialization.Crdts.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.Crdts.VersionVector 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; + + public static akka.protobufv3.internal.Descriptors.FileDescriptor getDescriptor() { + return descriptor; + } + + private static akka.protobufv3.internal.Descriptors.FileDescriptor descriptor; + + static { + java.lang.String[] descriptorData = { + "\n\013Crdts.proto\032\026ContainerFormats.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\005ORSet\022\020\n\010originDc\030\001 " + + "\002(\t\022\037\n\007vvector\030\002 \002(\0132\016.VersionVector\022\034\n\004" + + "dots\030\003 \003(\0132\016.VersionVector\022\026\n\016stringElem" + + "ents\030\004 \003(\t\022\027\n\013intElements\030\005 \003(\021B\002\020\001\022\030\n\014l" + + "ongElements\030\006 \003(\022B\002\020\001\022\037\n\rotherElements\030\007" + + " \003(\0132\010.Payload\"\201\001\n\017ORSetDeltaGroup\022\'\n\007en" + + "tries\030\001 \003(\0132\026.ORSetDeltaGroup.Entry\032E\n\005E" + + "ntry\022 \n\toperation\030\001 \002(\0162\r.ORSetDeltaOp\022\032" + + "\n\nunderlying\030\002 \002(\0132\006.ORSet\"]\n\rVersionVec" + + "tor\022%\n\007entries\030\001 \003(\0132\024.VersionVector.Ent" + + "ry\032%\n\005Entry\022\013\n\003key\030\001 \002(\t\022\017\n\007version\030\002 \002(" + + "\003*-\n\014ORSetDeltaOp\022\007\n\003Add\020\000\022\n\n\006Remove\020\001\022\010" + + "\n\004Full\020\002B(\n$akka.persistence.typed.seria" + + "lizationH\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", + }); + akka.remote.ContainerFormats.getDescriptor(); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/akka-persistence-typed/src/main/protobuf/Crdts.proto b/akka-persistence-typed/src/main/protobuf/Crdts.proto new file mode 100644 index 0000000000..cb95464f1c --- /dev/null +++ b/akka-persistence-typed/src/main/protobuf/Crdts.proto @@ -0,0 +1,50 @@ +/* + * 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; +} diff --git a/akka-persistence-typed/src/main/resources/reference.conf b/akka-persistence-typed/src/main/resources/reference.conf index 99c897d34e..1d3b90a8ec 100644 --- a/akka-persistence-typed/src/main/resources/reference.conf +++ b/akka-persistence-typed/src/main/resources/reference.conf @@ -1,3 +1,18 @@ +akka.actor { + + serialization-identifiers."akka.persistence.typed.serialization.CrdtSerializer" = 40 + + serializers.replicated-crdts = "akka.persistence.typed.serialization.CrdtSerializer" + + serialization-bindings { + "akka.persistence.typed.crdt.Counter" = replicated-crdts + "akka.persistence.typed.crdt.Counter$Updated" = replicated-crdts + "akka.persistence.typed.internal.VersionVector" = replicated-crdts + "akka.persistence.typed.crdt.ORSet" = replicated-crdts + "akka.persistence.typed.crdt.ORSet$DeltaOp" = replicated-crdts + } +} + akka.persistence.typed { # Persistent actors stash while recovering or persisting events, 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..099e9ed358 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/ORSet.scala @@ -0,0 +1,503 @@ +/* + * 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 extends Serializable { + 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 +@SerialVersionUID(1L) +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] + with Serializable { + + 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 scala.collection.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 scala.collection.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 scala.collection.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/scaladsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala index 6c15fcc5d6..33a4a53e04 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala @@ -8,34 +8,6 @@ import akka.persistence.typed.PersistenceId import akka.persistence.typed.ReplicaId import akka.util.WallClock -/** - * Utility class for comparing timestamp and data center - * identifier when implementing last-writer wins. - */ -final case class LwwTime(timestamp: Long, originDc: 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.originDc.id.compareTo(originDc.id) > 0) true - else false - } -} - // FIXME docs trait ActiveActiveContext { diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/CrdtSerializer.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/CrdtSerializer.scala new file mode 100644 index 0000000000..ab193b6f85 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/CrdtSerializer.scala @@ -0,0 +1,269 @@ +/* + * 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.crdt.{ Counter, ORSet } +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 scala.collection.JavaConverters._ +import scala.collection.immutable.TreeMap + +object CrdtSerializer { + 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 CrdtSerializer(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" + + 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 _ => + throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]") + } + + def toBinary(o: AnyRef) = o match { + 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: VersionVector => versionVectorToProto(m).toByteArray + case _ => + throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass}") + } + + def fromBinary(bytes: Array[Byte], manifest: String) = manifest match { + 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 VersionVectorManifest => versionVectorFromBinary(bytes) + case _ => + throw new NotSerializableException( + s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]") + } + + def counterFromBinary(bytes: Array[Byte]): Counter = + Counter(BigInt(Crdts.Counter.parseFrom(bytes).getValue.toByteArray)) + + def counterUpdatedFromBinary(bytes: Array[Byte]): Counter.Updated = + Counter.Updated(BigInt(Crdts.CounterUpdate.parseFrom(bytes).getDelta.toByteArray)) + + def counterToProtoByteArray(counter: Counter): Array[Byte] = + Crdts.Counter.newBuilder().setValue(ByteString.copyFrom(counter.value.toByteArray)).build().toByteArray + + def counterUpdatedToProtoBufByteArray(updated: Counter.Updated): Array[Byte] = + Crdts.CounterUpdate.newBuilder().setDelta(ByteString.copyFrom(updated.delta.toByteArray)).build().toByteArray + + def orsetToProto(orset: ORSet[_]): Crdts.ORSet = + orsetToProtoImpl(orset.asInstanceOf[ORSet[Any]]) + + private def orsetToProtoImpl(orset: ORSet[Any]): Crdts.ORSet = { + val b = Crdts.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, CrdtSerializer.Comparator) + b.addAllOtherElements(otherElements) + addDots(otherElements) + } + + b.build() + } + + def orsetFromBinary(bytes: Array[Byte]): ORSet[Any] = + orsetFromProto(Crdts.ORSet.parseFrom(bytes)) + + private def orsetAddFromBinary(bytes: Array[Byte]): ORSet.AddDeltaOp[Any] = + new ORSet.AddDeltaOp(orsetFromProto(Crdts.ORSet.parseFrom(bytes))) + + private def orsetRemoveFromBinary(bytes: Array[Byte]): ORSet.RemoveDeltaOp[Any] = + new ORSet.RemoveDeltaOp(orsetFromProto(Crdts.ORSet.parseFrom(bytes))) + + private def orsetFullFromBinary(bytes: Array[Byte]): ORSet.FullStateDeltaOp[Any] = + new ORSet.FullStateDeltaOp(orsetFromProto(Crdts.ORSet.parseFrom(bytes))) + + private def orsetDeltaGroupToProto(deltaGroup: ORSet.DeltaGroup[_]): Crdts.ORSetDeltaGroup = { + def createEntry(opType: Crdts.ORSetDeltaOp, u: ORSet[_]) = { + Crdts.ORSetDeltaGroup.Entry.newBuilder().setOperation(opType).setUnderlying(orsetToProto(u)) + } + + val b = Crdts.ORSetDeltaGroup.newBuilder() + deltaGroup.ops.foreach { + case ORSet.AddDeltaOp(u) => + b.addEntries(createEntry(Crdts.ORSetDeltaOp.Add, u)) + case ORSet.RemoveDeltaOp(u) => + b.addEntries(createEntry(Crdts.ORSetDeltaOp.Remove, u)) + case ORSet.FullStateDeltaOp(u) => + b.addEntries(createEntry(Crdts.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 = Crdts.ORSetDeltaGroup.parseFrom(bytes) + val ops: Vector[ORSet.DeltaOp] = + deltaGroup.getEntriesList.asScala.map { entry => + if (entry.getOperation == Crdts.ORSetDeltaOp.Add) + ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)) + else if (entry.getOperation == Crdts.ORSetDeltaOp.Remove) + ORSet.RemoveDeltaOp(orsetFromProto(entry.getUnderlying)) + else if (entry.getOperation == Crdts.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: Crdts.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): Crdts.VersionVector = { + val b = Crdts.VersionVector.newBuilder() + versionVector.versionsIterator.foreach { + case (key, value) => b.addEntries(Crdts.VersionVector.Entry.newBuilder().setKey(key).setVersion(value)) + } + b.build() + } + + def versionVectorFromBinary(bytes: Array[Byte]): VersionVector = + versionVectorFromProto(Crdts.VersionVector.parseFrom(bytes)) + + def versionVectorFromProto(versionVector: Crdts.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) + } + } + +} diff --git a/build.sbt b/build.sbt index 85977aabed..87c6ce871a 100644 --- a/build.sbt +++ b/build.sbt @@ -461,6 +461,7 @@ lazy val persistenceTyped = akkaModule("akka-persistence-typed") .dependsOn( actorTyped, streamTyped, + remote, persistence % "compile->compile;test->test", persistenceQuery, actorTestkitTyped % "test->test", @@ -470,6 +471,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") From 21713bbc30d90abc5b197e4d621312d8cc2e494e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Mon, 20 Jul 2020 10:32:36 +0200 Subject: [PATCH 14/50] Active active java apis #29266 --- .../typed/persistence-active-active.md | 45 +++- .../persistence/typed/ActiveActiveTest.java | 208 ++++++++++++++++++ .../typed/ActiveActiveCompileOnlyTest.java | 84 +++++++ .../persistence/typed/AABlogExampleSpec.scala | 9 +- .../typed/ActiveActiveCompileOnlySpec.scala | 16 +- ...9217-active-active-event-sourcing.excludes | 8 +- .../akka/persistence/typed/LwwTime.scala | 33 +++ .../internal/EventSourcedBehaviorImpl.scala | 6 +- .../ActiveActiveEventSourcedBehavior.scala | 67 ++++++ .../javadsl/ActiveActiveEventSourcing.scala | 96 ++++++++ .../typed/javadsl/EventSourcedBehavior.scala | 3 +- .../scaladsl/ActiveActiveEventSourcing.scala | 15 +- .../typed/scaladsl/EventSourcedBehavior.scala | 5 +- 13 files changed, 563 insertions(+), 32 deletions(-) create mode 100644 akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ActiveActiveTest.java create mode 100644 akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/LwwTime.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala diff --git a/akka-docs/src/main/paradox/typed/persistence-active-active.md b/akka-docs/src/main/paradox/typed/persistence-active-active.md index bfa8dce78a..4b3573f003 100644 --- a/akka-docs/src/main/paradox/typed/persistence-active-active.md +++ b/akka-docs/src/main/paradox/typed/persistence-active-active.md @@ -53,32 +53,47 @@ To assist in implementing the event handler active-active detects these conflict ## API -The same API as regular `EventSourcedBehavior`s is used to define the logic. To enable an entity for active-active -replication use the factory methods on @api[ActiveActiveEventSourcing]. +@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 active-active +replication @java[let it extend `ActiveActiveEventSourcedBehavior` instead of `EventSourcedBehavior` and] use the factory methods on @apidoc[ActiveActiveEventSourcing]. All replicas need to be known up front: Scala : @@snip [ActiveActiveCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala) { #replicas } +Java +: @@snip [ActiveActiveCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java) { #replicas } + + Then to enable replication create the event sourced behavior with the factory method: Scala : @@snip [ActiveActiveCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala) { #factory } +Java +: @@snip [ActiveActiveCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java) { #factory } + The factory takes in: * EntityID: this will be used as part of the underlying persistenceId * Replica: Which replica this instance is -* All Replicas and the query plugin used to read their events +* All Replicas and the query plugin used to read their events +* A factory function to create an instance of the @scala[`EventSourcedBehavior`]@java[`ActiveActiveEventSourcedBehavior`] 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 [ActiveActiveCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala) { #factory-shared} +Java +: @@snip [ActiveActiveCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java) { #factory-shared } -The function passed to both factory methods return an `EventSourcedBehavior` and provide access to the @api[ActiveActiveContext] that has the following methods: + +@@@ div { .group-scala } + +The function passed to both factory methods return an `EventSourcedBehavior` and provide access to the @apidoc[ActiveActiveContext] that has the following methods: * entityId * replicaId @@ -87,6 +102,24 @@ The function passed to both factory methods return an `EventSourcedBehavior` and 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[ActiveActiveContext] that needs to be passed to the +concrete `ActiveActiveEventSourcedBehavior` 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 `getActiveActiveContext`. 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. @@ -108,9 +141,9 @@ Sometimes it is enough to use timestamps to decide which update should win. Such ![images/lww.png](images/lww.png) -There is a small utility class @api[LwwTime] that can be useful for implementing last writer wins semantics. +There is a small utility class @apidoc[akka.persistence.typed.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 @api[LwwTime] the greatest timestamp wins. The replica +replica that persisted it. When comparing two @apidoc[akka.persistence.typed.LwwTime] the greatest timestamp wins. The replica identifier is used if the two timestamps are equal, and then the one from the data center sorted first in alphanumeric order wins. diff --git a/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ActiveActiveTest.java b/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ActiveActiveTest.java new file mode 100644 index 0000000000..38299e1530 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ActiveActiveTest.java @@ -0,0 +1,208 @@ +/* + * 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.javadsl.PersistenceTestKit; +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 ActiveActiveTest extends JUnitSuite { + + static final class TestBehavior + extends ActiveActiveEventSourcedBehavior> { + 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 ActiveActiveEventSourcing.withSharedJournal( + entityId, + replicaId, + allReplicas, + PersistenceTestKitReadJournal.Identifier(), + TestBehavior::new); + } + + private TestBehavior(ActiveActiveContext activeActiveContext) { + super(activeActiveContext); + } + + @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(getActiveActiveContext().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 ActiveActiveSpec + @Test + public void activeActiveReplicationTest() { + 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/ActiveActiveCompileOnlyTest.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java new file mode 100644 index 0000000000..51085a48b3 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package jdocs.akka.persistence.typed; + +import akka.persistence.typed.ReplicaId; +import akka.persistence.typed.javadsl.*; + +import java.util.*; + +public class ActiveActiveCompileOnlyTest { + + // dummy for docs example + interface Command {} + interface Event {} + interface State {} + + static // #factory + final class MyActiceActiveEventSourcedBehavior + extends ActiveActiveEventSourcedBehavior< + Command, + Event, + State> { + + public MyActiceActiveEventSourcedBehavior(ActiveActiveContext activeActiveContext) { + super(activeActiveContext); + } + // ... implementation of abstract methods ... + // #factory + + @Override + public State emptyState() { + return null; + } + + @Override + public CommandHandler commandHandler() { + return null; + } + + @Override + public EventHandler eventHandler() { + return null; + } + // #factory + } + + // #factory + + { + // #replicas + ReplicaId DCA = new ReplicaId("DC-A"); + ReplicaId DCB = new ReplicaId("DC-B"); + Set allReplicas = + Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DCA, DCB))); + // #replicas + + String queryPluginId = ""; + + // #factory-shared + ActiveActiveEventSourcing.withSharedJournal( + "entityId", + DCA, + allReplicas, + queryPluginId, + context -> new MyActiceActiveEventSourcedBehavior(context)); + // #factory-shared + + // #factory + + // bootstrap logic + Map allReplicasAndQueryPlugins = new HashMap<>(); + allReplicasAndQueryPlugins.put(DCA, "journalForDCA"); + allReplicasAndQueryPlugins.put(DCB, "journalForDCB"); + + EventSourcedBehavior behavior = ActiveActiveEventSourcing.create( + "entityId", + DCA, + allReplicasAndQueryPlugins, + context -> new MyActiceActiveEventSourcedBehavior(context)); + // #factory + } +} diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala index f67fd19226..c3d7f4d365 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala @@ -5,18 +5,19 @@ package docs.akka.persistence.typed import akka.Done -import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } +import akka.actor.testkit.typed.scaladsl.{LogCapturing, ScalaTestWithActorTestKit} import akka.actor.typed.ActorRef -import akka.actor.typed.scaladsl.{ ActorContext, Behaviors } +import akka.actor.typed.scaladsl.{ActorContext, Behaviors} import akka.persistence.testkit.PersistenceTestKitPlugin import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal +import akka.persistence.typed.LwwTime import akka.persistence.typed.ReplicaId import akka.persistence.typed.crdt.LwwTime import akka.persistence.typed.scaladsl._ import akka.serialization.jackson.CborSerializable -import org.scalatest.concurrent.{ Eventually, ScalaFutures } +import org.scalatest.concurrent.{Eventually, ScalaFutures} import org.scalatest.matchers.should.Matchers -import org.scalatest.time.{ Millis, Span } +import org.scalatest.time.{Millis, Span} import org.scalatest.wordspec.AnyWordSpecLike object AABlogExampleSpec { diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala index 55196657d3..dc0668ff6c 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala @@ -1,8 +1,14 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + package docs.akka.persistence.typed import akka.persistence.typed.ReplicaId -import akka.persistence.typed.scaladsl.{ ActiveActiveEventSourcing, EventSourcedBehavior } +import akka.persistence.typed.scaladsl.{ActiveActiveEventSourcing, EventSourcedBehavior} +import com.github.ghik.silencer.silent +@silent("never used") object ActiveActiveCompileOnlySpec { //#replicas @@ -13,15 +19,19 @@ object ActiveActiveCompileOnlySpec { val queryPluginId = "" + trait Command + trait State + trait Event + //#factory-shared ActiveActiveEventSourcing.withSharedJournal("entityId", DCA, AllReplicas, queryPluginId) { context => - EventSourcedBehavior(???, ???, ???, ???) + EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) } //#factory-shared //#factory ActiveActiveEventSourcing("entityId", DCA, Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { context => - EventSourcedBehavior(???, ???, ???, ???) + EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) } //#factory diff --git a/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes b/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes index b0597f04c8..5a6490fdc2 100644 --- a/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes +++ b/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes @@ -1,6 +1,4 @@ -# Changes to internal/private +# Changes to internal/private/do not extend ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.typed.scaladsl.EventSourcedBehavior.withActiveActive") -ProblemFilters.exclude[Problem]("akka.persistence.typed.internal.Running*") -ProblemFilters.exclude[Problem]("akka.persistence.typed.internal.EventSourcedBehaviorImpl.*") -ProblemFilters.exclude[Problem]("akka.persistence.typed.internal.BehaviorSetup*") - +ProblemFilters.exclude[Problem]("akka.persistence.typed.internal.*") +ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.typed.scaladsl.EventSourcedBehavior.withEventPublishing") diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/LwwTime.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/LwwTime.scala new file mode 100644 index 0000000000..1342d8000c --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/LwwTime.scala @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.persistence.typed + +/** + * Utility class for comparing timestamp and data center + * identifier when implementing last-writer wins. + */ +final case class LwwTime(timestamp: Long, originDc: 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.originDc.id.compareTo(originDc.id) > 0) true + else false + } +} 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 b9f032786d..27e47222e4 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 @@ -252,10 +252,8 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( } override private[akka] def withActiveActive( - context: ActiveActiveContextImpl, - replicaId: ReplicaId, - allReplicaIdsAndQueryPlugins: Map[ReplicaId, String]): EventSourcedBehavior[Command, Event, State] = { - copy(activeActive = Some(ActiveActive(replicaId, allReplicaIdsAndQueryPlugins, context))) + context: ActiveActiveContextImpl): EventSourcedBehavior[Command, Event, State] = { + copy(activeActive = Some(ActiveActive(context.replicaId, context.replicasAndQueryPlugins, context))) } } diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala new file mode 100644 index 0000000000..d4ad096a74 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala @@ -0,0 +1,67 @@ +/* + * 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.InternalApi +import akka.persistence.typed.internal +import akka.persistence.typed.internal.EffectImpl +import akka.persistence.typed.scaladsl.ActiveActiveContextImpl + +abstract class ActiveActiveEventSourcedBehavior[Command, Event, State]( + activeActiveContext: ActiveActiveContext, + onPersistFailure: Optional[BackoffSupervisorStrategy]) + extends EventSourcedBehavior[Command, Event, State](activeActiveContext.persistenceId, onPersistFailure) { + + def this(activeActiveContext: ActiveActiveContext) = this(activeActiveContext, Optional.empty()) + + protected def getActiveActiveContext(): ActiveActiveContext = activeActiveContext + + /** + * INTERNAL API: DeferredBehavior init, not for user extension + */ + @InternalApi override def apply(context: TypedActorContext[Command]): Behavior[Command] = { + // Note: duplicated in EventSourcedBehavior to not break source compatibility + val snapshotWhen: (State, Event, Long) => Boolean = (state, event, seqNr) => shouldSnapshot(state, event, seqNr) + + val tagger: Event => Set[String] = { event => + import akka.util.ccompat.JavaConverters._ + val tags = tagsFor(event) + if (tags.isEmpty) Set.empty + else tags.asScala.toSet + } + + val behavior = new internal.EventSourcedBehaviorImpl[Command, Event, State]( + persistenceId, + emptyState, + (state, cmd) => commandHandler()(state, cmd).asInstanceOf[EffectImpl[Event, State]], + eventHandler()(_, _), + getClass) + .snapshotWhen(snapshotWhen) + .withRetention(retentionCriteria.asScala) + .withTagger(tagger) + .eventAdapter(eventAdapter()) + .snapshotAdapter(snapshotAdapter()) + .withJournalPluginId(journalPluginId) + .withSnapshotPluginId(snapshotPluginId) + .withRecovery(recovery.asScala) + // context not user extendable so there should never be any other impls + .withActiveActive(activeActiveContext.asInstanceOf[ActiveActiveContextImpl]) + + val handler = signalHandler() + val behaviorWithSignalHandler = + if (handler.isEmpty) behavior + else behavior.receiveSignal(handler.handler) + + if (onPersistFailure.isPresent) + behaviorWithSignalHandler.onPersistFailure(onPersistFailure.get) + else + behaviorWithSignalHandler + } +} diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala new file mode 100644 index 0000000000..67761d410a --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala @@ -0,0 +1,96 @@ +/* + * 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.scaladsl.ActiveActiveContextImpl + +import scala.collection.JavaConverters._ + +/** + * Provides access to Active Active specific state + * + * Not for user extension + */ +@DoNotInherit +trait ActiveActiveContext { + def origin: ReplicaId + def concurrent: Boolean + def replicaId: ReplicaId + def getAllReplicas: JSet[ReplicaId] + def persistenceId: PersistenceId + def recoveryRunning: Boolean + def entityId: String + def currentTimeMillis(): Long +} + +/** + * Factory to create an instance of an ActiveActiveEventSourcedBehavior + */ +@FunctionalInterface +trait ActiveActiveBehaviorFactory[Command, Event, State] { + def apply(aaContext: ActiveActiveContext): ActiveActiveEventSourcedBehavior[Command, Event, State] +} + +object ActiveActiveEventSourcing { + + /** + * 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 replicaId The unique identity for this entity. The underlying persistence id will include the replica. + * @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 withSharedJournal[Command, Event, State]( + entityId: String, + replicaId: ReplicaId, + allReplicaIds: JSet[ReplicaId], + queryPluginId: String, + behaviorFactory: JFunction[ActiveActiveContext, EventSourcedBehavior[Command, Event, State]]) + : EventSourcedBehavior[Command, Event, State] = + create(entityId, replicaId, allReplicaIds.asScala.map(id => id -> queryPluginId).toMap.asJava, behaviorFactory) + + /** + * 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 replicaId The unique identity for this entity. The underlying persistence id will include the replica. + * @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 create[Command, Event, State]( + entityId: String, + replicaId: ReplicaId, + allReplicasAndQueryPlugins: JMap[ReplicaId, String], + eventSourcedBehaviorFactory: JFunction[ActiveActiveContext, EventSourcedBehavior[Command, Event, State]]) + : EventSourcedBehavior[Command, Event, State] = { + val context = new ActiveActiveContextImpl(entityId, replicaId, allReplicasAndQueryPlugins.asScala.toMap) + eventSourcedBehaviorFactory(context) + } + +} 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..00c6008121 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,10 @@ 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] = { + // Note: duplicated in ActiveActiveEventSourcedBehavior to not break source compatibility 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/scaladsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala index 33a4a53e04..1f7b3dee1f 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala @@ -7,6 +7,7 @@ package akka.persistence.typed.scaladsl import akka.persistence.typed.PersistenceId import akka.persistence.typed.ReplicaId import akka.util.WallClock +import scala.collection.JavaConverters._ // FIXME docs trait ActiveActiveContext { @@ -30,7 +31,8 @@ private[akka] class ActiveActiveContextImpl( val entityId: String, val replicaId: ReplicaId, val replicasAndQueryPlugins: Map[ReplicaId, String]) - extends ActiveActiveContext { + extends ActiveActiveContext + with akka.persistence.typed.javadsl.ActiveActiveContext { val allReplicas: Set[ReplicaId] = replicasAndQueryPlugins.keySet var _origin: ReplicaId = null var _recoveryRunning: Boolean = false @@ -56,6 +58,8 @@ private[akka] class ActiveActiveContextImpl( WallClock.AlwaysIncreasingClock.currentTimeMillis() } override def recoveryRunning: Boolean = _recoveryRunning + + override def getAllReplicas: java.util.Set[ReplicaId] = allReplicas.asJava } object ActiveActiveEventSourcing { @@ -80,9 +84,10 @@ object ActiveActiveEventSourcing { entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId], - queryPluginId: String)(activeActiveContext: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) + queryPluginId: String)( + eventSourcedBehaviorFactory: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = - apply(entityId, replicaId, allReplicaIds.map(id => id -> queryPluginId).toMap)(activeActiveContext) + apply(entityId, replicaId, allReplicaIds.map(id => id -> queryPluginId).toMap)(eventSourcedBehaviorFactory) /** * Initialize a replicated event sourced behavior. @@ -104,10 +109,10 @@ object ActiveActiveEventSourcing { entityId: String, replicaId: ReplicaId, allReplicasAndQueryPlugins: Map[ReplicaId, String])( - activeActiveContext: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) + eventSourcedBehaviorFactory: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = { val context = new ActiveActiveContextImpl(entityId, replicaId, allReplicasAndQueryPlugins) - activeActiveContext(context).withActiveActive(context, replicaId, allReplicasAndQueryPlugins) + eventSourcedBehaviorFactory(context).withActiveActive(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 56dcf975cf..eba5e0a6a8 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 @@ -166,10 +166,7 @@ object EventSourcedBehavior { */ def withJournalPluginId(id: String): EventSourcedBehavior[Command, Event, State] - private[akka] def withActiveActive( - context: ActiveActiveContextImpl, - replicaId: ReplicaId, - allReplicasAndQueryPlugins: Map[ReplicaId, String]): EventSourcedBehavior[Command, Event, State] + private[akka] def withActiveActive(context: ActiveActiveContextImpl): EventSourcedBehavior[Command, Event, State] /** * Change the snapshot store plugin id that this actor should use. From f531d1e57dcd057b1f8a9db9e8c0ed7d661cf5a6 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Tue, 21 Jul 2020 08:46:19 +0100 Subject: [PATCH 15/50] Docs and example for triggers (#29396) --- .../main/paradox/typed/index-persistence.md | 1 + .../persistence-active-active-examples.md | 106 ++++++++++++++++++ .../typed/persistence-active-active.md | 20 +++- .../typed/AAAuctionExampleSpec.scala | 41 ++++--- 4 files changed, 152 insertions(+), 16 deletions(-) create mode 100644 akka-docs/src/main/paradox/typed/persistence-active-active-examples.md diff --git a/akka-docs/src/main/paradox/typed/index-persistence.md b/akka-docs/src/main/paradox/typed/index-persistence.md index 5976589f98..a3da54c018 100644 --- a/akka-docs/src/main/paradox/typed/index-persistence.md +++ b/akka-docs/src/main/paradox/typed/index-persistence.md @@ -20,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) +* [active-active-examples](persistence-active-active-examples.md) @@@ diff --git a/akka-docs/src/main/paradox/typed/persistence-active-active-examples.md b/akka-docs/src/main/paradox/typed/persistence-active-active-examples.md new file mode 100644 index 0000000000..782557c848 --- /dev/null +++ b/akka-docs/src/main/paradox/typed/persistence-active-active-examples.md @@ -0,0 +1,106 @@ +# Active-Active Examples + +The following are more realistic examples of building systems with active-active event sourcing. + +## Auction + +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/AAAuctionExampleSpec.scala) { #commands } + +The events: + +Scala +: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala) { #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/AAAuctionExampleSpec.scala) { #command-handler } + +There is nothing specific to active-active 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 initial state is taken from a `AuctionSetup` instance. The minimum bid is modelled as +an `initialBid`. + +Scala +: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala) { #setup } + + +The auction moves through the following phases: + +Scala +: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.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/AAAuctionExampleSpec.scala) { #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/AAAuctionExampleSpec.scala) { #event-triggers } + +The event trigger uses the `ActiveActiveContext` 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/persistence-active-active.md b/akka-docs/src/main/paradox/typed/persistence-active-active.md index 4b3573f003..61a4bd88e0 100644 --- a/akka-docs/src/main/paradox/typed/persistence-active-active.md +++ b/akka-docs/src/main/paradox/typed/persistence-active-active.md @@ -173,7 +173,25 @@ Including the full state in each event is often not desired. An event typically ## Side effects -TODO https://github.com/akka/akka/issues/29318 +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 occured + +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[ActiveActiveContext] 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](./persistence-active-active-examples.md#auction) uses these techniques. + ## How it works diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala index 65dcbf9a14..b0b553bc42 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala @@ -24,30 +24,32 @@ object AAAuctionExampleSpec { case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originDc: ReplicaId) - // commands + //#commands sealed trait AuctionCommand case object Finish extends AuctionCommand // A timer needs to schedule this event at each replica final case class OfferBid(bidder: String, offer: MoneyAmount) extends AuctionCommand final case class GetHighestBid(replyTo: ActorRef[Bid]) extends AuctionCommand final case class IsClosed(replyTo: ActorRef[Boolean]) extends AuctionCommand private final case object Close extends AuctionCommand // Internal, should not be sent from the outside + //#commands + //#events sealed trait AuctionEvent extends CborSerializable final case class BidRegistered(bid: Bid) extends AuctionEvent - final case class AuctionFinished(atDc: ReplicaId) extends AuctionEvent - final case class WinnerDecided(atDc: ReplicaId, winningBid: Bid, highestCounterOffer: MoneyAmount) + final case class AuctionFinished(atReplica: ReplicaId) extends AuctionEvent + final case class WinnerDecided(atReplica: ReplicaId, winningBid: Bid, highestCounterOffer: MoneyAmount) extends AuctionEvent + //#events + //#phase sealed trait AuctionPhase case object Running extends AuctionPhase - final case class Closing(finishedAtDc: Set[ReplicaId]) extends AuctionPhase + final case class Closing(finishedAtReplica: Set[ReplicaId]) extends AuctionPhase case object Closed extends AuctionPhase + //#phase - case class AuctionState( - phase: AuctionPhase, - highestBid: Bid, - highestCounterOffer: MoneyAmount // in ebay style auctions, we need to keep track of current highest counter offer - ) { + //#state + case class AuctionState(phase: AuctionPhase, highestBid: Bid, highestCounterOffer: MoneyAmount) { def applyEvent(event: AuctionEvent): AuctionState = event match { @@ -90,14 +92,18 @@ object AAAuctionExampleSpec { (first.offer == second.offer && first.timestamp.equals(second.timestamp) && first.originDc.id .compareTo(second.originDc.id) < 0) } + //#state + //#setup case class AuctionSetup( name: String, initialBid: Bid, // the initial bid is basically the minimum price bidden at start time by the owner closingAt: Instant, responsibleForClosing: Boolean, - allDcs: Set[ReplicaId]) + allReplicas: Set[ReplicaId]) + //#setup + //#command-handler def commandHandler(setup: AuctionSetup, ctx: ActorContext[AuctionCommand], aaContext: ActiveActiveContext)( state: AuctionState, command: AuctionCommand): Effect[AuctionEvent, AuctionState] = { @@ -105,7 +111,7 @@ object AAAuctionExampleSpec { case Closing(_) | Closed => command match { case GetHighestBid(replyTo) => - replyTo ! state.highestBid + replyTo ! state.highestBid.copy(offer = state.highestCounterOffer) // TODO this is not as described Effect.none case IsClosed(replyTo) => replyTo ! (state.phase == Closed) @@ -143,14 +149,15 @@ object AAAuctionExampleSpec { } } } + //#command-handler private def shouldClose(auctionSetup: AuctionSetup, state: AuctionState): Boolean = { auctionSetup.responsibleForClosing && (state.phase match { case Closing(alreadyFinishedAtDc) => - val allDone = auctionSetup.allDcs.diff(alreadyFinishedAtDc).isEmpty + val allDone = auctionSetup.allReplicas.diff(alreadyFinishedAtDc).isEmpty if (!allDone) { println( - s"Not closing auction as not all DCs have reported finished. All DCs: ${auctionSetup.allDcs}. Reported finished ${alreadyFinishedAtDc}") + s"Not closing auction as not all DCs have reported finished. All DCs: ${auctionSetup.allReplicas}. Reported finished ${alreadyFinishedAtDc}") } allDone case _ => @@ -158,6 +165,7 @@ object AAAuctionExampleSpec { }) } + //#event-handler def eventHandler(ctx: ActorContext[AuctionCommand], aaCtx: ActiveActiveContext, setup: AuctionSetup)( state: AuctionState, event: AuctionEvent): AuctionState = { @@ -170,7 +178,9 @@ object AAAuctionExampleSpec { newState } + //#event-handler + //#event-triggers private def eventTriggers( setup: AuctionSetup, ctx: ActorContext[AuctionCommand], @@ -183,7 +193,7 @@ object AAAuctionExampleSpec { case Closing(alreadyFinishedAtDc) => ctx.log.infoN( "AuctionFinished at {}, already finished at [{}]", - finished.atDc, + finished.atReplica, alreadyFinishedAtDc.mkString(", ")) if (alreadyFinishedAtDc(aaCtx.replicaId)) { if (shouldClose(setup, newState)) ctx.self ! Close @@ -197,6 +207,7 @@ object AAAuctionExampleSpec { case _ => // no trigger for this event } } + //#event-triggers def initialState(setup: AuctionSetup) = AuctionState(phase = Running, highestBid = setup.initialBid, highestCounterOffer = setup.initialBid.offer) @@ -204,7 +215,7 @@ object AAAuctionExampleSpec { def behavior(replica: ReplicaId, setup: AuctionSetup): Behavior[AuctionCommand] = Behaviors.setup[AuctionCommand] { ctx => ActiveActiveEventSourcing - .withSharedJournal(setup.name, replica, setup.allDcs, PersistenceTestKitReadJournal.Identifier) { aaCtx => + .withSharedJournal(setup.name, replica, setup.allReplicas, PersistenceTestKitReadJournal.Identifier) { aaCtx => EventSourcedBehavior( aaCtx.persistenceId, initialState(setup), From 98309885663964d181c2cde0101681848f25346c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Tue, 21 Jul 2020 12:13:08 +0200 Subject: [PATCH 16/50] Active active sharding (#29364) --- ...ctiveActiveShardingDirectReplication.scala | 29 +- .../typed/ActiveActiveShardingExtension.scala | 68 +++++ .../typed/ActiveActiveShardingSettings.scala | 105 ++++++++ .../ActiveActiveShardingExtensionImpl.scala | 82 ++++++ .../typed/javadsl/ClusterSharding.scala | 18 +- .../typed/scaladsl/ClusterSharding.scala | 13 +- .../typed/ActiveActiveShardingTest.java | 255 ++++++++++++++++++ .../typed/ActiveActiveShardingSpec.scala | 146 ++++++++++ .../typed/persistence-active-active.md | 43 +++ 9 files changed, 754 insertions(+), 5 deletions(-) create mode 100644 akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala create mode 100644 akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala create mode 100644 akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala create mode 100644 akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java create mode 100644 akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala index 3700e26ac6..0ff09d5b6f 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala @@ -22,7 +22,10 @@ import scala.collection.JavaConverters._ * Akka Cluster. * * This actor should be started once on each node where Active Active entities will run (the same nodes that you start - * sharding on). + * sharding on). The entities should be set up with [[akka.persistence.typed.scaladsl.EventSourcedBehavior#withEventPublishing()]] + * (FIXME not supported in Java yet) + * If using [[ActiveActiveSharding]] the replication can be enabled through [[ActiveActiveShardingSettings#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 @@ -49,6 +52,14 @@ object ActiveActiveShardingDirectReplication { private final case class WrappedPublishedEvent(publishedEvent: PublishedEvent) extends Command + /** + * Java API: + * Factory for when the self replica id is unknown (or multiple) + * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system + */ + def create[T](replicaShardingProxies: java.util.Map[ReplicaId, ActorRef[T]]): Behavior[Command] = + apply(None, replicaShardingProxies.asScala.toMap) + /** * Java API: * @param selfReplica The replica id of the replica that runs on this node @@ -57,7 +68,14 @@ object ActiveActiveShardingDirectReplication { def create[T]( selfReplica: ReplicaId, replicaShardingProxies: java.util.Map[ReplicaId, ActorRef[T]]): Behavior[Command] = - apply(selfReplica, replicaShardingProxies.asScala.toMap) + apply(Some(selfReplica), replicaShardingProxies.asScala.toMap) + + /** + * Scala API: + * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system + */ + def apply[T](replicaShardingProxies: Map[ReplicaId, ActorRef[T]]): Behavior[Command] = + apply(None, replicaShardingProxies) /** * Scala API: @@ -65,6 +83,11 @@ object ActiveActiveShardingDirectReplication { * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system */ def apply[T](selfReplica: ReplicaId, replicaShardingProxies: Map[ReplicaId, ActorRef[T]]): Behavior[Command] = + apply(Some(selfReplica), replicaShardingProxies) + + private 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", @@ -81,7 +104,7 @@ object ActiveActiveShardingDirectReplication { replicaShardingProxies.foreach { case (replica, proxy) => val envelopedEvent = ShardingEnvelope(event.persistenceId.id, event) - if (replica != selfReplica) + if (!selfReplica.contains(replica)) proxy.asInstanceOf[ActorRef[ShardingEnvelope[PublishedEvent]]] ! envelopedEvent } Behaviors.same diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala new file mode 100644 index 0000000000..f359392e9a --- /dev/null +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala @@ -0,0 +1,68 @@ +/* + * 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.ActiveActiveShardingExtensionImpl +import akka.cluster.sharding.typed.scaladsl.EntityRef +import akka.persistence.typed.ReplicaId +import java.util.{ Map => JMap } + +/** + * Extension for running active active 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 ActiveActiveShardingExtension extends ExtensionId[ActiveActiveShardingExtension] { + + override def createExtension(system: ActorSystem[_]): ActiveActiveShardingExtension = + new ActiveActiveShardingExtensionImpl(system) + + def get(system: ActorSystem[_]): ActiveActiveShardingExtension = apply(system) + +} + +/** + * Not for user extension. + */ +@DoNotInherit +@ApiMayChange +trait ActiveActiveShardingExtension extends Extension { + + /** + * Init one instance sharding per replica in the given settings and return a [[ActiveActiveSharding]] representing those. + * + * @tparam M The type of messages the active active event sourced actor accepts + * @tparam E The type of envelope used for routing messages to actors, the same for all replicas + * + * Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ActiveActiveSharding]] + */ + def init[M, E](settings: ActiveActiveShardingSettings[M, E]): ActiveActiveSharding[M] +} + +/** + * Represents the sharding instances for the replicas of one active active entity type + * + * Not for user extension. + */ +@DoNotInherit +@ApiMayChange +trait ActiveActiveSharding[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, EntityRef[M]] + +} diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala new file mode 100644 index 0000000000..0f3b92cec4 --- /dev/null +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala @@ -0,0 +1,105 @@ +/* + * 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 scala.collection.JavaConverters._ +import java.util.{ Set => JSet } + +import akka.annotation.ApiMayChange +import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl + +@ApiMayChange +object ActiveActiveShardingSettings { + + /** + * Java API: + * + * @tparam M The type of messages the active active entity accepts + * @tparam E The type for envelopes used for sending `M`s over sharding + */ + def create[M, E]( + messageClass: Class[M], + allReplicaIds: JSet[ReplicaId], + settingsPerReplicaFactory: akka.japi.function.Function3[ + JEntityTypeKey[M], + ReplicaId, + JSet[ReplicaId], + ReplicaSettings[M, E]]): ActiveActiveShardingSettings[M, E] = { + implicit val classTag: ClassTag[M] = ClassTag(messageClass) + apply[M, E](allReplicaIds.asScala.toSet)((key, replica, _) => + settingsPerReplicaFactory(key.asInstanceOf[EntityTypeKeyImpl[M]], replica, allReplicaIds)) + } + + /** + * Scala API: + * + * @tparam M The type of messages the active active entity accepts + * @tparam E The type for envelopes used for sending `M`s over sharding + */ + def apply[M: ClassTag, E](allReplicaIds: Set[ReplicaId])( + settingsPerReplicaFactory: (EntityTypeKey[M], ReplicaId, Set[ReplicaId]) => ReplicaSettings[M, E]) + : ActiveActiveShardingSettings[M, E] = { + new ActiveActiveShardingSettings(allReplicaIds.map { replicaId => + val typeKey = EntityTypeKey[M](replicaId.id) + settingsPerReplicaFactory(typeKey, replicaId, allReplicaIds) + }.toVector, directReplication = false) + } +} + +/** + * @tparam M The type of messages the active active entity accepts + * @tparam E The type for envelopes used for sending `M`s over sharding + */ +@ApiMayChange +final class ActiveActiveShardingSettings[M, E] private ( + val replicas: immutable.Seq[ReplicaSettings[M, E]], + val directReplication: Boolean) { + + /** + * Start direct replication over sharding when active active sharding starts up, requires the entities + * to also have it enabled through [[akka.persistence.typed.scaladsl.EventSourcedBehavior#withEventPublishing()]] + * to work. + * + * FIXME no support for enabling that in Java because bin/source comp. + */ + def withDirectReplication(): ActiveActiveShardingSettings[M, E] = + new ActiveActiveShardingSettings(replicas, directReplication = true) + +} + +@ApiMayChange +object ReplicaSettings { + + /** + * 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.ActiveActiveEventSourcedBehavior]] 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, E](replicaId: ReplicaId, entity: JEntity[M, E]): ReplicaSettings[M, E] = + 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.ActiveActiveEventSourcing]] 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, E](replicaId: ReplicaId, entity: Entity[M, E]): ReplicaSettings[M, E] = + new ReplicaSettings(replicaId, entity) +} + +/** + * Settings for a specific replica id in active active sharding + */ +@ApiMayChange +final class ReplicaSettings[M, E] private (val replicaId: ReplicaId, val entity: Entity[M, E]) diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala new file mode 100644 index 0000000000..16e72853ae --- /dev/null +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala @@ -0,0 +1,82 @@ +/* + * 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.ActiveActiveShardingExtension +import akka.cluster.sharding.typed.ActiveActiveSharding +import akka.cluster.sharding.typed.ActiveActiveShardingSettings +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.PersistenceId +import akka.persistence.typed.ReplicaId +import org.slf4j.LoggerFactory +import akka.actor.typed.scaladsl.LoggerOps +import akka.cluster.sharding.typed.ActiveActiveShardingDirectReplication + +import scala.collection.JavaConverters._ + +/** + * INTERNAL API + */ +@InternalApi +private[akka] final class ActiveActiveShardingExtensionImpl(system: ActorSystem[_]) + extends ActiveActiveShardingExtension { + + private val counter = new AtomicLong(0) + + private val logger = LoggerFactory.getLogger(getClass) + + override def init[M, E](settings: ActiveActiveShardingSettings[M, E]): ActiveActiveSharding[M] = { + val sharding = ClusterSharding(system) + val initializedReplicas = settings.replicas.map { replicaSettings => + // start up a sharding instance per replica id + logger.infoN( + "Starting Active Active sharding for replica [{}] (ShardType: [{}])", + replicaSettings.replicaId.id, + replicaSettings.entity.typeKey.name) + val regionOrProxy = sharding.init(replicaSettings.entity) + (replicaSettings.replicaId, replicaSettings.entity.typeKey, regionOrProxy) + } + + if (settings.directReplication) { + logger.infoN("Starting Active Active Direct Replication") + val replicaToRegionOrProxy = initializedReplicas.map { + case (id, _, regionOrProxy) => id -> regionOrProxy + }.toMap + system.systemActorOf( + ActiveActiveShardingDirectReplication(replicaToRegionOrProxy), + s"activeActiveDirectReplication-${counter.incrementAndGet()}") + } + + val replicaToTypeKey = initializedReplicas.map { case (id, typeKey, _) => id -> typeKey }.toMap + new ActiveActiveShardingImpl(sharding, replicaToTypeKey) + } +} + +/** + * INTERNAL API + */ +@InternalApi +private[akka] final class ActiveActiveShardingImpl[M]( + sharding: ClusterSharding, + replicaTypeKeys: Map[ReplicaId, EntityTypeKey[M]]) + extends ActiveActiveSharding[M] { + + override def entityRefsFor(entityId: String): Map[ReplicaId, EntityRef[M]] = + replicaTypeKeys.map { + case (replicaId, typeKey) => + replicaId -> sharding.entityRefFor(typeKey, PersistenceId.ofUniqueId(entityId).id) + } + + override def getEntityRefsFor(entityId: String): JMap[ReplicaId, EntityRef[M]] = + entityRefsFor(entityId).asJava + +} 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..1a0ebf404c 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] @@ -337,6 +337,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) + } /** 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..31de6db557 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 @@ -348,7 +348,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 { diff --git a/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java b/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java new file mode 100644 index 0000000000..abb576a17a --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java @@ -0,0 +1,255 @@ +/* + * 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.scaladsl.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.javadsl.*; +import com.typesafe.config.ConfigFactory; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.scalatestplus.junit.JUnitSuite; +import scala.util.Random; + +import java.util.*; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; + +public class ActiveActiveShardingTest extends JUnitSuite { + + static class MyActiveActiveStringSet + extends ActiveActiveEventSourcedBehavior< + MyActiveActiveStringSet.Command, String, Set> { + 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( + String entityId, ReplicaId replicaId, Set allReplicas) { + return ActiveActiveEventSourcing.withSharedJournal( + entityId, + replicaId, + allReplicas, + PersistenceTestKitReadJournal.Identifier(), + MyActiveActiveStringSet::new); + } + + private MyActiveActiveStringSet(ActiveActiveContext activeActiveContext) { + super(activeActiveContext); + } + + @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 MyActiveActiveStringSet.Command message; + + public ForwardToRandom(String entityId, MyActiveActiveStringSet.Command message) { + this.entityId = entityId; + this.message = message; + } + } + + public static final class ForwardToAll implements Command { + public final String entityId; + public final MyActiveActiveStringSet.Command message; + + public ForwardToAll(String entityId, MyActiveActiveStringSet.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 ActiveActiveSharding aaSharding; + + private ProxyActor(ActorContext context) { + super(context); + + // #bootstrap + ActiveActiveShardingSettings< + MyActiveActiveStringSet.Command, ShardingEnvelope> + aaShardingSettings = + ActiveActiveShardingSettings.create( + MyActiveActiveStringSet.Command.class, + ALL_REPLICAS, + // factory for replica settings for a given replica + (entityTypeKey, replicaId, allReplicas) -> + ReplicaSettings.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 + MyActiveActiveStringSet.create( + entityContext.getEntityId(), replicaId, allReplicas)) + // 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()))); + + ActiveActiveShardingExtension extension = + ActiveActiveShardingExtension.get(getContext().getSystem()); + aaSharding = extension.init(aaShardingSettings); + // #bootstrap + } + + @Override + public Receive createReceive() { + return newReceiveBuilder() + .onMessage(ForwardToRandom.class, this::onForwardToRandom) + .onMessage(ForwardToAll.class, this::onForwardToAll) + .build(); + } + + private Behavior onForwardToRandom(ForwardToRandom forwardToRandom) { + Map> refs = + aaSharding.getEntityRefsFor(forwardToRandom.entityId); + int chosenIdx = new java.util.Random().nextInt(refs.size()); + new ArrayList<>(refs.values()).get(chosenIdx).tell(forwardToRandom.message); + return this; + } + + private Behavior onForwardToAll(ForwardToAll forwardToAll) { + // #all-entity-refs + Map> refs = + aaSharding.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 MyActiveActiveStringSet.Add("to-all"))); + proxy.tell(new ProxyActor.ForwardToRandom("id1", new MyActiveActiveStringSet.Add("to-random"))); + + testProbe.awaitAssert( + () -> { + TestProbe responseProbe = testKit.createTestProbe(); + proxy.tell( + new ProxyActor.ForwardToAll( + "id1", new MyActiveActiveStringSet.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/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala new file mode 100644 index 0000000000..74c7f3052a --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala @@ -0,0 +1,146 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package akka.cluster.sharding.typed + +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.cluster.MemberStatus +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.ActiveActiveEventSourcing +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 scala.util.Random + +object ActiveActiveShardingSpec { + def config = ConfigFactory.parseString(""" + akka.loglevel = DEBUG + akka.loggers = ["akka.testkit.SilenceAllTestEventListener"] + akka.actor.provider = "cluster" + # pretend we're a node in all dc:s + akka.cluster.roles = ["DC-A", "DC-B", "DC-C"] + akka.remote.classic.netty.tcp.port = 0 + akka.remote.artery.canonical.port = 0""").withFallback(PersistenceTestKitPlugin.config) +} + +class ActiveActiveShardingSpec + extends ScalaTestWithActorTestKit(ActiveActiveShardingSpec.config) + with AnyWordSpecLike + with LogCapturing { + + object MyActiveActiveStringSet { + 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(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] = + ActiveActiveEventSourcing.withSharedJournal( + entityId, + replicaId, + allReplicas, + PersistenceTestKitReadJournal.Identifier) { aaContext => + EventSourcedBehavior[Command, String, Set[String]]( + aaContext.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) + } + } + + object ProxyActor { + sealed trait Command + case class ForwardToRandom(entityId: String, msg: MyActiveActiveStringSet.Command) extends Command + case class ForwardToAll(entityId: String, msg: MyActiveActiveStringSet.Command) extends Command + + def apply(): Behavior[Command] = Behaviors.setup { context => + // #bootstrap + val aaShardingSettings = + ActiveActiveShardingSettings[ + MyActiveActiveStringSet.Command, + ShardingEnvelope[MyActiveActiveStringSet.Command]]( + // all replicas + Set(ReplicaId("DC-A"), ReplicaId("DC-B"), ReplicaId("DC-C"))) { (entityTypeKey, replicaId, allReplicaIds) => + // factory for replica settings for a given replica + ReplicaSettings( + replicaId, + // use the provided entity type key for sharding to get one sharding instance per replica + Entity(entityTypeKey) { entityContext => + // factory for the entity for a given entity in that replica + MyActiveActiveStringSet(entityContext.entityId, replicaId, allReplicaIds) + } + // 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)) + } + + val aaSharding = ActiveActiveShardingExtension(context.system).init(aaShardingSettings) + // #bootstrap + + Behaviors.receiveMessage { + case ForwardToAll(entityId, cmd) => + // #all-entity-refs + aaSharding.entityRefsFor(entityId).foreach { + case (_, ref) => ref ! cmd + } + // #all-entity-refs + Behaviors.same + case ForwardToRandom(entityId, cmd) => + val refs = aaSharding.entityRefsFor(entityId) + val chosenIdx = (new Random()).nextInt(refs.size) + refs.values.toIndexedSeq(chosenIdx) ! cmd; + Behaviors.same + } + } + } + + "Active active sharding" should { + + "form a one node cluster" in { + val node = Cluster(system) + node.manager ! Join(node.selfMember.address) + eventually { + node.selfMember.status should ===(MemberStatus.Up) + } + } + + "forward to replicas" in { + val proxy = spawn(ProxyActor()) + + proxy ! ProxyActor.ForwardToAll("id1", MyActiveActiveStringSet.Add("to-all")) + proxy ! ProxyActor.ForwardToRandom("id1", MyActiveActiveStringSet.Add("to-random")) + + eventually { + val probe = createTestProbe[MyActiveActiveStringSet.Texts]() + proxy ! ProxyActor.ForwardToAll("id1", MyActiveActiveStringSet.GetTexts(probe.ref)) + val responses: Seq[MyActiveActiveStringSet.Texts] = probe.receiveMessages(3) + val uniqueTexts = responses.flatMap(res => res.texts).toSet + uniqueTexts should ===(Set("to-all", "to-random")) + } + + } + + } + +} diff --git a/akka-docs/src/main/paradox/typed/persistence-active-active.md b/akka-docs/src/main/paradox/typed/persistence-active-active.md index 61a4bd88e0..0f4da28a9c 100644 --- a/akka-docs/src/main/paradox/typed/persistence-active-active.md +++ b/akka-docs/src/main/paradox/typed/persistence-active-active.md @@ -241,3 +241,46 @@ When comparing two version vectors `v1` and `v2`: * `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 Active Active entities + +To simplify what probably are the most common use cases for how you will want to distribute the active active actors there is a minimal API for running multiple instances of @ref[Akka Cluster Sharding](cluster-sharding.md), +each instance holding the entities for a single replica. + +The distribution of the replicas can be controlled either through cluster roles or using the @ref[multi datacenter](cluster-dc.md) support in Akka Cluster. + +The API consists of bootstrapping logic for starting the sharding instances through @apidoc[ActiveActiveShardingExtension] available from the +`akka-cluster-sharding-typed` module. + +Scala +: @@snip [ActiveActiveShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala) { #bootstrap } + +Java +: @@snip [ActiveActiveShardingTest.java](/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java) { #bootstrap } + +`init` returns an @apidoc[ActiveActiveSharding] instance which gives access to @apidoc[EntityRef]s for each of the replicas for arbitrary routing logic: + +Scala +: @@snip [ActiveActiveShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala) { #all-entity-refs } + +Java +: @@snip [ActiveActiveShardingTest.java](/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java) { #all-entity-refs } + +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)). + + +## Direct Replication of Events + +Normally an event has to be written in the journal and then picked up by the trailing read journal in the other replicas. +As an optimization the active active events can be published across the Akka cluster to the replicas. The read side +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. + +To enable this feature you first need to enable event publishing on the `EventSourcedBehavior` with `withEventPublishing` +(FIXME missing Java API) and then enable direct replication through `withDirectReplication()` on @apidoc[ActiveActiveShardingSettings] (if not using + active active sharding the replication can be run standalone by starting the @apidoc[ActiveActiveShardingDirectReplication] actor). + +The "event publishing" feature publishes each event to the local system event bus as a side effect after it has been written, +the `ActiveActiveShardingDirectReplication` actor subscribes to these events and forwards them to the replicas allowing them +to fast forward the stream of events for the origin replica. (With additional potential future support in journals for fast forwarding [#29311](https://github.com/akka/akka/issues/29311)). \ No newline at end of file From a563a9b69e9cd46061b841bfa18c9ab2de5e297a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 22 Jul 2020 09:22:38 +0200 Subject: [PATCH 17/50] Make sure to gracefully stop replication streams when aa actor stops #29406 --- .../typed/persistence-active-active.md | 4 +- .../typed/ActiveActiveCompileOnlyTest.java | 10 ++-- .../persistence/typed/AABlogExampleSpec.scala | 9 ++-- .../akka/persistence/typed/LwwTime.scala | 33 ------------ .../persistence/typed/internal/Running.scala | 51 +++++++++++-------- 5 files changed, 41 insertions(+), 66 deletions(-) delete mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/LwwTime.scala diff --git a/akka-docs/src/main/paradox/typed/persistence-active-active.md b/akka-docs/src/main/paradox/typed/persistence-active-active.md index 0f4da28a9c..bca97075c0 100644 --- a/akka-docs/src/main/paradox/typed/persistence-active-active.md +++ b/akka-docs/src/main/paradox/typed/persistence-active-active.md @@ -141,9 +141,9 @@ Sometimes it is enough to use timestamps to decide which update should win. Such ![images/lww.png](images/lww.png) -There is a small utility class @apidoc[akka.persistence.typed.LwwTime] that can be useful for implementing last writer wins semantics. +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[akka.persistence.typed.LwwTime] the greatest timestamp wins. The replica +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 data center sorted first in alphanumeric order wins. diff --git a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java index 51085a48b3..4020954f9f 100644 --- a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java @@ -13,15 +13,14 @@ public class ActiveActiveCompileOnlyTest { // dummy for docs example interface Command {} + interface Event {} + interface State {} static // #factory final class MyActiceActiveEventSourcedBehavior - extends ActiveActiveEventSourcedBehavior< - Command, - Event, - State> { + extends ActiveActiveEventSourcedBehavior { public MyActiceActiveEventSourcedBehavior(ActiveActiveContext activeActiveContext) { super(activeActiveContext); @@ -74,7 +73,8 @@ public class ActiveActiveCompileOnlyTest { allReplicasAndQueryPlugins.put(DCA, "journalForDCA"); allReplicasAndQueryPlugins.put(DCB, "journalForDCB"); - EventSourcedBehavior behavior = ActiveActiveEventSourcing.create( + EventSourcedBehavior behavior = + ActiveActiveEventSourcing.create( "entityId", DCA, allReplicasAndQueryPlugins, diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala index c3d7f4d365..f67fd19226 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala @@ -5,19 +5,18 @@ package docs.akka.persistence.typed import akka.Done -import akka.actor.testkit.typed.scaladsl.{LogCapturing, ScalaTestWithActorTestKit} +import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } import akka.actor.typed.ActorRef -import akka.actor.typed.scaladsl.{ActorContext, Behaviors} +import akka.actor.typed.scaladsl.{ ActorContext, Behaviors } import akka.persistence.testkit.PersistenceTestKitPlugin import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal -import akka.persistence.typed.LwwTime import akka.persistence.typed.ReplicaId import akka.persistence.typed.crdt.LwwTime import akka.persistence.typed.scaladsl._ import akka.serialization.jackson.CborSerializable -import org.scalatest.concurrent.{Eventually, ScalaFutures} +import org.scalatest.concurrent.{ Eventually, ScalaFutures } import org.scalatest.matchers.should.Matchers -import org.scalatest.time.{Millis, Span} +import org.scalatest.time.{ Millis, Span } import org.scalatest.wordspec.AnyWordSpecLike object AABlogExampleSpec { diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/LwwTime.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/LwwTime.scala deleted file mode 100644 index 1342d8000c..0000000000 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/LwwTime.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Copyright (C) 2020 Lightbend Inc. - */ - -package akka.persistence.typed - -/** - * Utility class for comparing timestamp and data center - * identifier when implementing last-writer wins. - */ -final case class LwwTime(timestamp: Long, originDc: 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.originDc.id.compareTo(originDc.id) > 0) true - else false - } -} 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 c0d83579c5..d93db654be 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 @@ -56,6 +56,8 @@ import akka.persistence.typed.scaladsl.Effect import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive 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 @@ -136,27 +138,34 @@ private[akka] object Running { val controlRef = new AtomicReference[ReplicationStreamControl]() - val source = RestartSource.withBackoff(2.seconds, 10.seconds, randomFactor = 0.2) { () => - replication - .eventsByPersistenceId(pid.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(_.eventMetadata.get.asInstanceOf[ReplicatedEventMetaData].originReplica == replicaId) - .viaMat(new FastForwardingFilter)(Keep.right) - .mapMaterializedValue(streamControl => controlRef.set(streamControl)) - .via(ActorFlow.ask[EventEnvelope, ReplicatedEventEnvelope[E], ReplicatedEventAck.type](ref) { - (eventEnvelope, replyTo) => - // Need to handle this not being available migration from non-active-active 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) - }) - } + val source = RestartSource + .withBackoff(2.seconds, 10.seconds, randomFactor = 0.2) { () => + replication + .eventsByPersistenceId(pid.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(_.eventMetadata.get.asInstanceOf[ReplicatedEventMetaData].originReplica == replicaId) + .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-active-active 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) From 165b39d1e0ec7987299e65161d93edcd4645088d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 22 Jul 2020 12:19:03 +0200 Subject: [PATCH 18/50] Incorrect assumption about event ordering in test fixed (#29411) --- .../persistence/typed/ActiveActiveSpec.scala | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala index 5ae2cf87a0..3c6ea4e25b 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala @@ -301,7 +301,7 @@ class ActiveActiveSpec r1 ! StoreMe("from r1 2", probe.ref) probe.expectMessage(Done) - // r2 + // 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)) @@ -310,13 +310,17 @@ class ActiveActiveSpec r2 ! StoreMe("from r2 2", probe.ref) probe.expectMessage(Done) - // r3 should only get the events 1, not R2s stored version of them + // 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)) - eventProbeR3.expectMessage(EventAndContext("from r1 1", ReplicaId("R1"), false, false)) - eventProbeR3.expectMessage(EventAndContext("from r1 2", ReplicaId("R1"), false, false)) - eventProbeR3.expectMessage(EventAndContext("from r2 1", ReplicaId("R2"), false, false)) - eventProbeR3.expectMessage(EventAndContext("from r2 2", ReplicaId("R2"), false, false)) + 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() } From 827dc14dbb4ce966516414bdda7f803a4e5b354f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 22 Jul 2020 12:26:36 +0200 Subject: [PATCH 19/50] EventPublishing API for Java (#29413) --- ...ctiveActiveShardingDirectReplication.scala | 2 +- .../typed/ActiveActiveShardingSettings.scala | 4 ++-- .../typed/persistence-active-active.md | 4 ++-- .../ActiveActiveEventSourcedBehavior.scala | 19 +++++++++++++++---- 4 files changed, 20 insertions(+), 9 deletions(-) diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala index 0ff09d5b6f..5e1ba61e80 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ * * This actor should be started once on each node where Active Active entities will run (the same nodes that you start * sharding on). The entities should be set up with [[akka.persistence.typed.scaladsl.EventSourcedBehavior#withEventPublishing()]] - * (FIXME not supported in Java yet) + * or [[akka.persistence.typed.javadsl.ActiveActiveEventSourcedBehavior#withEventPublishing()]] * If using [[ActiveActiveSharding]] the replication can be enabled through [[ActiveActiveShardingSettings#withDirectReplication()]] * instead of starting this actor manually. * diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala index 0f3b92cec4..09ae1fcb8c 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala @@ -67,9 +67,9 @@ final class ActiveActiveShardingSettings[M, E] private ( /** * Start direct replication over sharding when active active sharding starts up, requires the entities * to also have it enabled through [[akka.persistence.typed.scaladsl.EventSourcedBehavior#withEventPublishing()]] + * or [[akka.persistence.typed.javadsl.ActiveActiveEventSourcedBehavior#withEventPublishing()]] * to work. - * - * FIXME no support for enabling that in Java because bin/source comp. + */ def withDirectReplication(): ActiveActiveShardingSettings[M, E] = new ActiveActiveShardingSettings(replicas, directReplication = true) diff --git a/akka-docs/src/main/paradox/typed/persistence-active-active.md b/akka-docs/src/main/paradox/typed/persistence-active-active.md index bca97075c0..dcf79528b4 100644 --- a/akka-docs/src/main/paradox/typed/persistence-active-active.md +++ b/akka-docs/src/main/paradox/typed/persistence-active-active.md @@ -277,8 +277,8 @@ As an optimization the active active events can be published across the Akka clu 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. -To enable this feature you first need to enable event publishing on the `EventSourcedBehavior` with `withEventPublishing` -(FIXME missing Java API) and then enable direct replication through `withDirectReplication()` on @apidoc[ActiveActiveShardingSettings] (if not using +To enable this feature you first need to enable event publishing on the @scala[`EventSourcedBehavior`]@java[`ActiveActiveEventSourcedBehavior`] with `withEventPublishing` +and then enable direct replication through `withDirectReplication()` on @apidoc[ActiveActiveShardingSettings] (if not using active active sharding the replication can be run standalone by starting the @apidoc[ActiveActiveShardingDirectReplication] actor). The "event publishing" feature publishes each event to the local system event bus as a side effect after it has been written, diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala index d4ad096a74..9e315b04f1 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala @@ -9,11 +9,13 @@ 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 import akka.persistence.typed.internal.EffectImpl import akka.persistence.typed.scaladsl.ActiveActiveContextImpl +@ApiMayChange abstract class ActiveActiveEventSourcedBehavior[Command, Event, State]( activeActiveContext: ActiveActiveContext, onPersistFailure: Optional[BackoffSupervisorStrategy]) @@ -21,6 +23,11 @@ abstract class ActiveActiveEventSourcedBehavior[Command, Event, State]( def this(activeActiveContext: ActiveActiveContext) = this(activeActiveContext, Optional.empty()) + /** + * Override and return true to publish events to the system event stream as [[akka.persistence.typed.PublishedEvent]] after they have been persisted + */ + def withEventPublishing: Boolean = false + protected def getActiveActiveContext(): ActiveActiveContext = activeActiveContext /** @@ -59,9 +66,13 @@ abstract class ActiveActiveEventSourcedBehavior[Command, Event, State]( if (handler.isEmpty) behavior else behavior.receiveSignal(handler.handler) - if (onPersistFailure.isPresent) - behaviorWithSignalHandler.onPersistFailure(onPersistFailure.get) - else - behaviorWithSignalHandler + val behaviorWithOnPersistFailure = + if (onPersistFailure.isPresent) + behaviorWithSignalHandler.onPersistFailure(onPersistFailure.get) + else + behaviorWithSignalHandler + + if (withEventPublishing) behaviorWithOnPersistFailure.withEventPublishing() + else behaviorWithOnPersistFailure } } From 3d5fae809d0e94830cb6e7da888dc2f9c68740ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 22 Jul 2020 14:52:17 +0200 Subject: [PATCH 20/50] Don't log user event toString in active active #29393 (#29416) --- .../persistence/typed/internal/Running.scala | 27 ++++++++++++------- 1 file changed, 18 insertions(+), 9 deletions(-) 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 d93db654be..fcadd33a99 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 @@ -385,13 +385,14 @@ private[akka] object Running { val updatedVersion = event.originVersion.merge(state.version) activeActive.setContext(false, event.originReplica, isConcurrent) - setup.log.debugN( - "Processing event [{}] with version [{}]. Local version: {}. Updated version {}. Concurrent? {}", - event.event, - event.originVersion, - state.version, - updatedVersion, - isConcurrent) + 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) val newState: RunningState[S] = state.applyEvent(setup, event.event) val newState2: RunningState[S] = internalPersist( @@ -445,7 +446,11 @@ private[akka] object Running { ReplicatedEventMetaData(aa.replicaId, _currentSequenceNumber, updatedVersion, concurrent = false))) .copy(version = updatedVersion) - setup.log.debug("Event persisted [{}]. Version vector after: [{}]", eventToPersist, r.version) + if (setup.log.isTraceEnabled()) + setup.log.traceN( + "Event persisted [{}]. Version vector after: [{}]", + Logging.simpleName(event.getClass), + r.version) r case None => @@ -486,7 +491,11 @@ private[akka] object Running { val eventMetadata = metadataTemplate match { case Some(template) => val updatedVersion = currentState.version.updated(template.originReplica.id, _currentSequenceNumber) - setup.log.trace("Processing event [{}] with version vector [{}]", event, updatedVersion) + 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 From 7ec631d3c1ef919726e74fed50ccf130071497a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 22 Jul 2020 15:12:22 +0200 Subject: [PATCH 21/50] ActiveActiveSharding make it possible to use with custom envelope (#29417) --- .../typed/ActiveActiveShardingExtension.scala | 24 +++++++++++++++++-- .../ActiveActiveShardingExtensionImpl.scala | 20 +++++++++------- .../typed/javadsl/ClusterSharding.scala | 6 +++++ .../typed/scaladsl/ClusterSharding.scala | 6 +++++ .../typed/ActiveActiveShardingTest.java | 4 +++- 5 files changed, 49 insertions(+), 11 deletions(-) diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala index f359392e9a..163dabe3e6 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala @@ -14,6 +14,8 @@ import akka.cluster.sharding.typed.scaladsl.EntityRef import akka.persistence.typed.ReplicaId import java.util.{ Map => JMap } +import akka.actor.typed.ActorRef + /** * Extension for running active active 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. @@ -43,7 +45,7 @@ trait ActiveActiveShardingExtension extends Extension { * * Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ActiveActiveSharding]] */ - def init[M, E](settings: ActiveActiveShardingSettings[M, E]): ActiveActiveSharding[M] + def init[M, E](settings: ActiveActiveShardingSettings[M, E]): ActiveActiveSharding[M, E] } /** @@ -53,15 +55,33 @@ trait ActiveActiveShardingExtension extends Extension { */ @DoNotInherit @ApiMayChange -trait ActiveActiveSharding[M] { +trait ActiveActiveSharding[M, E] { + + /** + * Scala API: Returns the actor refs for the shard region or proxies of sharding for each replica for user defined + * routing/replica selection. + */ + def shardingRefs: Map[ReplicaId, ActorRef[E]] + + /** + * Java API: Returns the actor refs for the shard region or proxies of sharding for each replica for user defined + * routing/replica selection. + */ + def getShardingRefs: JMap[ReplicaId, ActorRef[E]] /** * Scala API: Returns the entity ref for each replica for user defined routing/replica selection + * + * 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 [[#shardingRefs]] */ def entityRefsFor(entityId: String): Map[ReplicaId, EntityRef[M]] /** * Java API: Returns the entity ref for each replica for user defined routing/replica selection + * + * 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 [[#getShardingRefs]] */ def getEntityRefsFor(entityId: String): JMap[ReplicaId, EntityRef[M]] diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala index 16e72853ae..2d8eea6155 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala @@ -7,6 +7,7 @@ package akka.cluster.sharding.typed.internal import java.util.concurrent.atomic.AtomicLong import java.util.{ Map => JMap } +import akka.actor.typed.ActorRef import akka.actor.typed.ActorSystem import akka.annotation.InternalApi import akka.cluster.sharding.typed.ActiveActiveShardingExtension @@ -34,7 +35,7 @@ private[akka] final class ActiveActiveShardingExtensionImpl(system: ActorSystem[ private val logger = LoggerFactory.getLogger(getClass) - override def init[M, E](settings: ActiveActiveShardingSettings[M, E]): ActiveActiveSharding[M] = { + override def init[M, E](settings: ActiveActiveShardingSettings[M, E]): ActiveActiveSharding[M, E] = { val sharding = ClusterSharding(system) val initializedReplicas = settings.replicas.map { replicaSettings => // start up a sharding instance per replica id @@ -45,19 +46,18 @@ private[akka] final class ActiveActiveShardingExtensionImpl(system: ActorSystem[ val regionOrProxy = sharding.init(replicaSettings.entity) (replicaSettings.replicaId, replicaSettings.entity.typeKey, regionOrProxy) } - + val replicaToRegionOrProxy = initializedReplicas.map { + case (id, _, regionOrProxy) => id -> regionOrProxy + }.toMap if (settings.directReplication) { logger.infoN("Starting Active Active Direct Replication") - val replicaToRegionOrProxy = initializedReplicas.map { - case (id, _, regionOrProxy) => id -> regionOrProxy - }.toMap system.systemActorOf( ActiveActiveShardingDirectReplication(replicaToRegionOrProxy), s"activeActiveDirectReplication-${counter.incrementAndGet()}") } val replicaToTypeKey = initializedReplicas.map { case (id, typeKey, _) => id -> typeKey }.toMap - new ActiveActiveShardingImpl(sharding, replicaToTypeKey) + new ActiveActiveShardingImpl(sharding, replicaToRegionOrProxy, replicaToTypeKey) } } @@ -65,10 +65,14 @@ private[akka] final class ActiveActiveShardingExtensionImpl(system: ActorSystem[ * INTERNAL API */ @InternalApi -private[akka] final class ActiveActiveShardingImpl[M]( +private[akka] final class ActiveActiveShardingImpl[M, E]( sharding: ClusterSharding, + shardingPerReplica: Map[ReplicaId, ActorRef[E]], replicaTypeKeys: Map[ReplicaId, EntityTypeKey[M]]) - extends ActiveActiveSharding[M] { + extends ActiveActiveSharding[M, E] { + + override def shardingRefs: Map[ReplicaId, ActorRef[E]] = shardingPerReplica + override def getShardingRefs: JMap[ReplicaId, ActorRef[E]] = shardingRefs.asJava override def entityRefsFor(entityId: String): Map[ReplicaId, EntityRef[M]] = replicaTypeKeys.map { 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 1a0ebf404c..2211719e48 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 @@ -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] 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 31de6db557..9e46aaabdb 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] diff --git a/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java b/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java index abb576a17a..490f4c2606 100644 --- a/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java +++ b/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java @@ -144,7 +144,9 @@ public class ActiveActiveShardingTest extends JUnitSuite { Arrays.asList( new ReplicaId("DC-A"), new ReplicaId("DC-B"), new ReplicaId("DC-C")))); - private final ActiveActiveSharding aaSharding; + private final ActiveActiveSharding< + MyActiveActiveStringSet.Command, ShardingEnvelope> + aaSharding; private ProxyActor(ActorContext context) { super(context); From f63ca66e56faf2a2ac45afa57f3131ee7915bc88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 22 Jul 2020 16:22:12 +0200 Subject: [PATCH 22/50] Active active docs mention plugin support (#29418) * Mention specific support for active active is needed in plugins * Align metadata name journal vs snapshot --- .../paradox/typed/persistence-active-active.md | 14 ++++++++++++-- .../typed/internal/ReplayingSnapshot.scala | 2 +- .../scala/akka/persistence/SnapshotProtocol.scala | 9 +++++---- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/akka-docs/src/main/paradox/typed/persistence-active-active.md b/akka-docs/src/main/paradox/typed/persistence-active-active.md index dcf79528b4..108cf7f04a 100644 --- a/akka-docs/src/main/paradox/typed/persistence-active-active.md +++ b/akka-docs/src/main/paradox/typed/persistence-active-active.md @@ -33,7 +33,9 @@ there is no longer the single writer principle as there is with a normal `EventS The state of an active-active `EventSourcedBehavior` is **eventually consistent**. Event replication may be delayed due to network partitions and outages and the event handler and those reading the state must be designed to handle this. -## Relaxing the single writer p`rinciple for availability +To be able to use active active the journal and snapshot store used is required to have specific support for the metadata that active active needs (see @ref[Journal Support](#journal-support)) + +## Relaxing the single writer principle for availability Taking the example of using active-active to run a replica per data center. @@ -283,4 +285,12 @@ and then enable direct replication through `withDirectReplication()` on @apidoc[ The "event publishing" feature publishes each event to the local system event bus as a side effect after it has been written, the `ActiveActiveShardingDirectReplication` actor subscribes to these events and forwards them to the replicas allowing them -to fast forward the stream of events for the origin replica. (With additional potential future support in journals for fast forwarding [#29311](https://github.com/akka/akka/issues/29311)). \ No newline at end of file +to fast forward the stream of events for the origin replica. (With additional potential future support in journals for fast forwarding [#29311](https://github.com/akka/akka/issues/29311)). + +## Journal Support + +For a journal plugin to support active active 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. + +For a snapshot plugin to support active active 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. \ No newline at end of file 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 524901b342..d4c6481dda 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 @@ -151,7 +151,7 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup case Some(SelectedSnapshot(metadata, snapshot)) => state = setup.snapshotAdapter.fromJournal(snapshot) setup.context.log.debug("Loaded snapshot with metadata {}", metadata) - metadata.meta match { + metadata.metadata match { case Some(rm: ReplicatedSnapshotMetaData) => (metadata.sequenceNr, rm.seenPerReplica, rm.version) case _ => (metadata.sequenceNr, Map.empty.withDefaultValue(0L), VersionVector.empty) } diff --git a/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala b/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala index 91d9b25755..c737aa21f8 100644 --- a/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala +++ b/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala @@ -11,14 +11,14 @@ import scala.runtime.AbstractFunction3 * @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 meta a journal can optionally support persisting metadata separate to the domain state + * @param metadata a journal can optionally support persisting metadata separate to the domain state, used for active active support */ @SerialVersionUID(1L) final class SnapshotMetadata( val persistenceId: String, val sequenceNr: Long, val timestamp: Long, - val meta: Option[Any]) + val metadata: Option[Any]) extends Product3[String, Long, Long] with Serializable { @@ -34,9 +34,10 @@ final class SnapshotMetadata( def copy( persistenceId: String = this.persistenceId, sequenceNr: Long = this.sequenceNr, - timestamp: Long = this.timestamp): SnapshotMetadata = SnapshotMetadata(persistenceId, sequenceNr, timestamp, meta) + timestamp: Long = this.timestamp): SnapshotMetadata = + SnapshotMetadata(persistenceId, sequenceNr, timestamp, metadata) - override def toString = s"SnapshotMetadata($persistenceId, $sequenceNr, $timestamp, $meta)" + override def toString = s"SnapshotMetadata($persistenceId, $sequenceNr, $timestamp, $metadata)" // Product 3 override def productPrefix = "SnapshotMetadata" From e4f5781d6580bc67be8b997fdc48685b9198467c Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Tue, 28 Jul 2020 13:45:25 +0100 Subject: [PATCH 23/50] Detect illegal access to context (#29431) --- .../typed/ActiveActiveIllegalAccessSpec.scala | 101 ++++++++++++++++++ .../typed/internal/ReplayingEvents.scala | 18 +++- .../persistence/typed/internal/Running.scala | 40 +++++-- .../scaladsl/ActiveActiveEventSourcing.scala | 35 +++++- .../typed/scaladsl/EventSourcedBehavior.scala | 9 ++ 5 files changed, 187 insertions(+), 16 deletions(-) create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveIllegalAccessSpec.scala diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveIllegalAccessSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveIllegalAccessSpec.scala new file mode 100644 index 0000000000..a8145639b0 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveIllegalAccessSpec.scala @@ -0,0 +1,101 @@ +/* + * 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.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import akka.serialization.jackson.CborSerializable +import org.scalatest.concurrent.Eventually +import org.scalatest.wordspec.AnyWordSpecLike + +object ActiveActiveIllegalAccessSpec { + + 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] = { + ActiveActiveEventSourcing.withSharedJournal( + entityId, + replica, + AllReplicas, + PersistenceTestKitReadJournal.Identifier)( + aaContext => + EventSourcedBehavior[Command, String, State]( + aaContext.persistenceId, + State(Nil), + (_, command) => + command match { + case AccessInCommandHandler(replyTo) => + val exception = try { + aaContext.origin + None + } catch { + case t: Throwable => + Some(t) + } + replyTo ! Thrown(exception) + Effect.none + case AccessInPersistCallback(replyTo) => + Effect.persist("cat").thenRun { _ => + val exception = try { + aaContext.concurrent + None + } catch { + case t: Throwable => + Some(t) + } + replyTo ! Thrown(exception) + } + }, + (state, event) => state.copy(all = event :: state.all))) + } + +} + +class ActiveActiveIllegalAccessSpec + extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) + with AnyWordSpecLike + with LogCapturing + with Eventually { + import ActiveActiveIllegalAccessSpec._ + "ActiveActive" should { + "detect illegal access to context in command handler" in { + val probe = createTestProbe[Thrown]() + val ref = spawn(ActiveActiveIllegalAccessSpec("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(ActiveActiveIllegalAccessSpec("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] { + ActiveActiveEventSourcing.withSharedJournal("id2", R1, AllReplicas, PersistenceTestKitReadJournal.Identifier) { + aaContext => + aaContext.origin + ??? + } + } + exception.getMessage should include("from the event handler") + } + } +} 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 96b90d565f..aa7dd1c1d1 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 @@ -23,6 +23,7 @@ import akka.persistence.typed.SingleEventSeq import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState import akka.persistence.typed.internal.ReplayingEvents.ReplayingState import akka.persistence.typed.internal.Running.WithSeqNrAccessible +import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive import akka.util.OptionVal import akka.util.PrettyDuration._ import akka.util.unused @@ -122,7 +123,7 @@ private[akka] final class ReplayingEvents[C, E, S]( eventForErrorReporting = OptionVal.Some(event) state = state.copy(seqNr = repr.sequenceNr) - val aaMetaAndSelfReplica: Option[(ReplicatedEventMetaData, ReplicaId)] = + val aaMetaAndSelfReplica: Option[(ReplicatedEventMetaData, ReplicaId, ActiveActive)] = setup.activeActive match { case Some(aa) => val meta = repr.metadata match { @@ -133,19 +134,30 @@ private[akka] final class ReplayingEvents[C, E, S]( } aa.setContext(recoveryRunning = true, meta.originReplica, meta.concurrent) - Some(meta -> aa.replicaId) + Some((meta, aa.replicaId, aa)) case None => None } + val newState = setup.eventHandler(state.state, event) + setup.activeActive match { + case Some(aa) => + aa.clearContext() + case None => + } + aaMetaAndSelfReplica match { - case Some((meta, selfReplica)) if meta.originReplica != selfReplica => + case Some((meta, selfReplica, aa)) 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)) + aa.clearContext() + case Some((_, _, aa)) => + aa.clearContext() + state = state.copy(state = newState, eventSeenInInterval = true) case _ => state = state.copy(state = newState, eventSeenInInterval = true) } 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 fcadd33a99..7ec2bf42d0 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 @@ -377,13 +377,19 @@ private[akka] object Running { this } + def withContext[A](aa: ActiveActive, withActiveActive: ActiveActive => Unit, f: () => A): A = { + withActiveActive(aa) + val result = f() + aa.clearContext() + result + } + private def handleExternalReplicatedEventPersist( activeActive: ActiveActive, event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { _currentSequenceNumber = state.seqNr + 1 val isConcurrent: Boolean = event.originVersion <> state.version val updatedVersion = event.originVersion.merge(state.version) - activeActive.setContext(false, event.originReplica, isConcurrent) if (setup.log.isDebugEnabled()) setup.log.debugN( @@ -394,7 +400,11 @@ private[akka] object Running { updatedVersion, isConcurrent) - val newState: RunningState[S] = state.applyEvent(setup, event.event) + val newState: RunningState[S] = withContext( + activeActive, + aa => aa.setContext(recoveryRunning = false, event.originReplica, concurrent = isConcurrent), + () => state.applyEvent(setup, event.event)) + val newState2: RunningState[S] = internalPersist( setup.context, null, @@ -423,13 +433,17 @@ private[akka] object Running { // also, ensure that there is an event handler for each single event _currentSequenceNumber = state.seqNr + 1 - setup.activeActive.foreach { aa => - // set concurrent to false, local events are never concurrent - aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false) + val newState: RunningState[S] = setup.activeActive match { + case Some(aa) => + // set concurrent to false, local events are never concurrent + withContext( + aa, + aa => aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false), + () => state.applyEvent(setup, event)) + case None => + state.applyEvent(setup, event) } - val newState: RunningState[S] = state.applyEvent(setup, event) - val eventToPersist = adaptEvent(event) val eventAdapterManifest = setup.eventAdapter.manifest(event) @@ -500,7 +514,17 @@ private[akka] object Running { Some(template.copy(originSequenceNr = _currentSequenceNumber, version = updatedVersion)) case None => None } - currentState = currentState.applyEvent(setup, event) + + currentState = setup.activeActive match { + case Some(aa) => + withContext( + aa, + aa => aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false), + () => currentState.applyEvent(setup, event)) + case None => + currentState.applyEvent(setup, event) + } + eventsToPersist = EventToPersist(adaptedEvent, evtManifest, eventMetadata) :: eventsToPersist } diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala index 1f7b3dee1f..eec1a91976 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala @@ -6,7 +6,8 @@ package akka.persistence.typed.scaladsl import akka.persistence.typed.PersistenceId import akka.persistence.typed.ReplicaId -import akka.util.WallClock +import akka.util.{ OptionVal, WallClock } + import scala.collection.JavaConverters._ // FIXME docs @@ -34,30 +35,54 @@ private[akka] class ActiveActiveContextImpl( extends ActiveActiveContext with akka.persistence.typed.javadsl.ActiveActiveContext { val allReplicas: Set[ReplicaId] = replicasAndQueryPlugins.keySet + + // these are not volatile as they are set on the same thread as they should be accessed var _origin: ReplicaId = null var _recoveryRunning: Boolean = false var _concurrent: Boolean = false + var _currentThread: OptionVal[Thread] = OptionVal.None - // FIXME check illegal access https://github.com/akka/akka/issues/29264 + private def checkAccess(functionName: String): Unit = { + val callerThread = Thread.currentThread() + def error() = + throw new UnsupportedOperationException( + s"Unsupported access to ActiveActiveContext 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 = _origin + override def origin: ReplicaId = { + checkAccess("origin") + _origin + } /** * 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 = _concurrent + override def concurrent: Boolean = { + checkAccess("concurrent") + _concurrent + } override def persistenceId: PersistenceId = PersistenceId.replicatedUniqueId(entityId, replicaId) override def currentTimeMillis(): Long = { WallClock.AlwaysIncreasingClock.currentTimeMillis() } - override def recoveryRunning: Boolean = _recoveryRunning + override def recoveryRunning: Boolean = { + checkAccess("recoveryRunning") + _recoveryRunning + } override def getAllReplicas: java.util.Set[ReplicaId] = allReplicas.asJava } 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 eba5e0a6a8..b40d3a327e 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 @@ -20,6 +20,7 @@ import akka.persistence.typed.ReplicaId import akka.persistence.typed.SnapshotAdapter import akka.persistence.typed.SnapshotSelectionCriteria import akka.persistence.typed.internal._ +import akka.util.OptionVal object EventSourcedBehavior { @@ -36,11 +37,19 @@ object EventSourcedBehavior { * Must only be called on the same thread that will execute the user code */ def setContext(recoveryRunning: Boolean, originReplica: ReplicaId, concurrent: Boolean): Unit = { + aaContext._currentThread = OptionVal.Some(Thread.currentThread()) aaContext._recoveryRunning = recoveryRunning aaContext._concurrent = concurrent aaContext._origin = originReplica } + def clearContext(): Unit = { + aaContext._currentThread = OptionVal.None + aaContext._recoveryRunning = false + aaContext._concurrent = false + aaContext._origin = null + } + } /** From b8a1584e1096334fda3cc15ba80e4f70a2d33061 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Tue, 28 Jul 2020 16:13:18 +0200 Subject: [PATCH 24/50] Active active journal support changes (#29421) * Add missing serializers (and rename classes because metadata is a word) * Add test instances for journal and snapshot support * Method to add metadata to existing snapshotmetadata * Better error message if replicated stream does not have metadata * Snapshot store tck support for metadata * Docs for tck coverage of AA metadata --- .../typed/persistence-active-active.md | 6 +- .../akka/persistence/CapabilityFlags.scala | 6 + .../persistence/journal/JournalSpec.scala | 4 +- .../snapshot/SnapshotStoreSpec.scala | 23 + .../{Crdts.java => ActiveActive.java} | 4274 ++++++++++++++--- .../{Crdts.proto => ActiveActive.proto} | 16 + .../src/main/resources/reference.conf | 16 +- .../internal/EventSourcedBehaviorImpl.scala | 30 +- .../typed/internal/ExternalInteractions.scala | 4 +- .../typed/internal/ReplayingEvents.scala | 4 +- .../typed/internal/ReplayingSnapshot.scala | 2 +- .../persistence/typed/internal/Running.scala | 21 +- ...zer.scala => ActiveActiveSerializer.scala} | 126 +- .../typed/ActiveActiveSerializationSpec.scala | 40 + .../akka/persistence/SnapshotProtocol.scala | 3 + project/OSGi.scala | 17 +- 16 files changed, 3970 insertions(+), 622 deletions(-) rename akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/{Crdts.java => ActiveActive.java} (55%) rename akka-persistence-typed/src/main/protobuf/{Crdts.proto => ActiveActive.proto} (71%) rename akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/{CrdtSerializer.scala => ActiveActiveSerializer.scala} (63%) create mode 100644 akka-persistence-typed/src/test/scala/akka/persistence/typed/ActiveActiveSerializationSpec.scala diff --git a/akka-docs/src/main/paradox/typed/persistence-active-active.md b/akka-docs/src/main/paradox/typed/persistence-active-active.md index 108cf7f04a..4d621b2b31 100644 --- a/akka-docs/src/main/paradox/typed/persistence-active-active.md +++ b/akka-docs/src/main/paradox/typed/persistence-active-active.md @@ -290,7 +290,9 @@ to fast forward the stream of events for the origin replica. (With additional po ## Journal Support For a journal plugin to support active active 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. + `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 active active 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. \ No newline at end of file +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. \ No newline at end of file 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 7434737402..fde55aa446 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala @@ -69,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 Active Active) 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 1d6eaed53b..3380126b73 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 @@ -323,7 +323,7 @@ abstract class JournalSpec(config: Config) AtomicWrite( PersistentRepr( payload = event, - sequenceNr = 1L, + sequenceNr = 6L, persistenceId = pid, sender = Actor.noSender, writerUuid = writerUuid).withMetadata(meta)) @@ -335,7 +335,7 @@ abstract class JournalSpec(config: Config) val WriterUuid = writerUuid probe.expectMsgPF() { case WriteMessageSuccess( - PersistentImpl(payload, 1L, Pid, _, _, Actor.noSender, WriterUuid, _, Some(`meta`)), + PersistentImpl(payload, 6L, Pid, _, _, Actor.noSender, WriterUuid, _, Some(`meta`)), _) => payload should be(event) } 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-typed/src/main/java/akka/persistence/typed/serialization/Crdts.java b/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ActiveActive.java similarity index 55% rename from akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/Crdts.java rename to akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ActiveActive.java index dad9330ab0..cd0ca60693 100644 --- a/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/Crdts.java +++ b/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ActiveActive.java @@ -3,12 +3,12 @@ */ // Generated by the protocol buffer compiler. DO NOT EDIT! -// source: Crdts.proto +// source: ActiveActive.proto package akka.persistence.typed.serialization; -public final class Crdts { - private Crdts() {} +public final class ActiveActive { + private ActiveActive() {} public static void registerAllExtensions( akka.protobufv3.internal.ExtensionRegistryLite registry) {} @@ -86,7 +86,9 @@ public final class Crdts { } public static final akka.protobufv3.internal.Descriptors.EnumDescriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.getDescriptor().getEnumTypes().get(0); + return akka.persistence.typed.serialization.ActiveActive.getDescriptor() + .getEnumTypes() + .get(0); } private static final ORSetDeltaOp[] VALUES = values(); @@ -199,16 +201,17 @@ public final class Crdts { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.internal_static_Counter_descriptor; + return akka.persistence.typed.serialization.ActiveActive.internal_static_Counter_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts.internal_static_Counter_fieldAccessorTable + return akka.persistence.typed.serialization.ActiveActive + .internal_static_Counter_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.Counter.class, - akka.persistence.typed.serialization.Crdts.Counter.Builder.class); + akka.persistence.typed.serialization.ActiveActive.Counter.class, + akka.persistence.typed.serialization.ActiveActive.Counter.Builder.class); } private int bitField0_; @@ -275,11 +278,11 @@ public final class Crdts { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.Crdts.Counter)) { + if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.Counter)) { return super.equals(obj); } - akka.persistence.typed.serialization.Crdts.Counter other = - (akka.persistence.typed.serialization.Crdts.Counter) obj; + akka.persistence.typed.serialization.ActiveActive.Counter other = + (akka.persistence.typed.serialization.ActiveActive.Counter) obj; if (hasValue() != other.hasValue()) return false; if (hasValue()) { @@ -305,72 +308,72 @@ public final class Crdts { return hash; } - public static akka.persistence.typed.serialization.Crdts.Counter parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.Counter parseFrom( java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.Counter parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.Counter parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.Counter parseFrom( akka.protobufv3.internal.ByteString data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.Counter parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.Counter parseFrom(byte[] data) + public static akka.persistence.typed.serialization.ActiveActive.Counter parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.Counter parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.Counter parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.Counter parseFrom( java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.Counter parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.Counter parseDelimitedFrom( + public static akka.persistence.typed.serialization.ActiveActive.Counter parseDelimitedFrom( java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.Counter parseDelimitedFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.Counter parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.Counter parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.Counter parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -387,7 +390,8 @@ public final class Crdts { return DEFAULT_INSTANCE.toBuilder(); } - public static Builder newBuilder(akka.persistence.typed.serialization.Crdts.Counter prototype) { + public static Builder newBuilder( + akka.persistence.typed.serialization.ActiveActive.Counter prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -407,21 +411,22 @@ public final class Crdts { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:Counter) - akka.persistence.typed.serialization.Crdts.CounterOrBuilder { + akka.persistence.typed.serialization.ActiveActive.CounterOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.internal_static_Counter_descriptor; + return akka.persistence.typed.serialization.ActiveActive.internal_static_Counter_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts.internal_static_Counter_fieldAccessorTable + return akka.persistence.typed.serialization.ActiveActive + .internal_static_Counter_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.Counter.class, - akka.persistence.typed.serialization.Crdts.Counter.Builder.class); + akka.persistence.typed.serialization.ActiveActive.Counter.class, + akka.persistence.typed.serialization.ActiveActive.Counter.Builder.class); } - // Construct using akka.persistence.typed.serialization.Crdts.Counter.newBuilder() + // Construct using akka.persistence.typed.serialization.ActiveActive.Counter.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -445,17 +450,17 @@ public final class Crdts { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.Crdts.internal_static_Counter_descriptor; + return akka.persistence.typed.serialization.ActiveActive.internal_static_Counter_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.Counter getDefaultInstanceForType() { - return akka.persistence.typed.serialization.Crdts.Counter.getDefaultInstance(); + public akka.persistence.typed.serialization.ActiveActive.Counter getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ActiveActive.Counter.getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.Counter build() { - akka.persistence.typed.serialization.Crdts.Counter result = buildPartial(); + public akka.persistence.typed.serialization.ActiveActive.Counter build() { + akka.persistence.typed.serialization.ActiveActive.Counter result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -463,9 +468,9 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.Counter buildPartial() { - akka.persistence.typed.serialization.Crdts.Counter result = - new akka.persistence.typed.serialization.Crdts.Counter(this); + public akka.persistence.typed.serialization.ActiveActive.Counter buildPartial() { + akka.persistence.typed.serialization.ActiveActive.Counter result = + new akka.persistence.typed.serialization.ActiveActive.Counter(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) != 0)) { @@ -514,16 +519,16 @@ public final class Crdts { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.Crdts.Counter) { - return mergeFrom((akka.persistence.typed.serialization.Crdts.Counter) other); + if (other instanceof akka.persistence.typed.serialization.ActiveActive.Counter) { + return mergeFrom((akka.persistence.typed.serialization.ActiveActive.Counter) other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.Counter other) { - if (other == akka.persistence.typed.serialization.Crdts.Counter.getDefaultInstance()) + public Builder mergeFrom(akka.persistence.typed.serialization.ActiveActive.Counter other) { + if (other == akka.persistence.typed.serialization.ActiveActive.Counter.getDefaultInstance()) return this; if (other.hasValue()) { setValue(other.getValue()); @@ -546,12 +551,12 @@ public final class Crdts { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.Crdts.Counter parsedMessage = null; + akka.persistence.typed.serialization.ActiveActive.Counter parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { parsedMessage = - (akka.persistence.typed.serialization.Crdts.Counter) e.getUnfinishedMessage(); + (akka.persistence.typed.serialization.ActiveActive.Counter) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { if (parsedMessage != null) { @@ -624,13 +629,13 @@ public final class Crdts { } // @@protoc_insertion_point(class_scope:Counter) - private static final akka.persistence.typed.serialization.Crdts.Counter DEFAULT_INSTANCE; + private static final akka.persistence.typed.serialization.ActiveActive.Counter DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.Counter(); + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.Counter(); } - public static akka.persistence.typed.serialization.Crdts.Counter getDefaultInstance() { + public static akka.persistence.typed.serialization.ActiveActive.Counter getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -656,7 +661,7 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.Counter getDefaultInstanceForType() { + public akka.persistence.typed.serialization.ActiveActive.Counter getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } @@ -752,17 +757,18 @@ public final class Crdts { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.internal_static_CounterUpdate_descriptor; + return akka.persistence.typed.serialization.ActiveActive + .internal_static_CounterUpdate_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_CounterUpdate_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.CounterUpdate.class, - akka.persistence.typed.serialization.Crdts.CounterUpdate.Builder.class); + akka.persistence.typed.serialization.ActiveActive.CounterUpdate.class, + akka.persistence.typed.serialization.ActiveActive.CounterUpdate.Builder.class); } private int bitField0_; @@ -829,11 +835,11 @@ public final class Crdts { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.Crdts.CounterUpdate)) { + if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.CounterUpdate)) { return super.equals(obj); } - akka.persistence.typed.serialization.Crdts.CounterUpdate other = - (akka.persistence.typed.serialization.Crdts.CounterUpdate) obj; + akka.persistence.typed.serialization.ActiveActive.CounterUpdate other = + (akka.persistence.typed.serialization.ActiveActive.CounterUpdate) obj; if (hasDelta() != other.hasDelta()) return false; if (hasDelta()) { @@ -859,72 +865,74 @@ public final class Crdts { return hash; } - public static akka.persistence.typed.serialization.Crdts.CounterUpdate parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate parseFrom( java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.CounterUpdate parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.CounterUpdate parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate parseFrom( akka.protobufv3.internal.ByteString data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.CounterUpdate parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.CounterUpdate parseFrom(byte[] data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate parseFrom( + byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.CounterUpdate parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.CounterUpdate parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate parseFrom( java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.CounterUpdate parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.CounterUpdate parseDelimitedFrom( - java.io.InputStream input) throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.CounterUpdate parseDelimitedFrom( - java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.CounterUpdate parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.CounterUpdate parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -942,7 +950,7 @@ public final class Crdts { } public static Builder newBuilder( - akka.persistence.typed.serialization.Crdts.CounterUpdate prototype) { + akka.persistence.typed.serialization.ActiveActive.CounterUpdate prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -962,22 +970,24 @@ public final class Crdts { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:CounterUpdate) - akka.persistence.typed.serialization.Crdts.CounterUpdateOrBuilder { + akka.persistence.typed.serialization.ActiveActive.CounterUpdateOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.internal_static_CounterUpdate_descriptor; + return akka.persistence.typed.serialization.ActiveActive + .internal_static_CounterUpdate_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_CounterUpdate_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.CounterUpdate.class, - akka.persistence.typed.serialization.Crdts.CounterUpdate.Builder.class); + akka.persistence.typed.serialization.ActiveActive.CounterUpdate.class, + akka.persistence.typed.serialization.ActiveActive.CounterUpdate.Builder.class); } - // Construct using akka.persistence.typed.serialization.Crdts.CounterUpdate.newBuilder() + // Construct using + // akka.persistence.typed.serialization.ActiveActive.CounterUpdate.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -1001,17 +1011,19 @@ public final class Crdts { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.Crdts.internal_static_CounterUpdate_descriptor; + return akka.persistence.typed.serialization.ActiveActive + .internal_static_CounterUpdate_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.CounterUpdate getDefaultInstanceForType() { - return akka.persistence.typed.serialization.Crdts.CounterUpdate.getDefaultInstance(); + public akka.persistence.typed.serialization.ActiveActive.CounterUpdate + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ActiveActive.CounterUpdate.getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.CounterUpdate build() { - akka.persistence.typed.serialization.Crdts.CounterUpdate result = buildPartial(); + public akka.persistence.typed.serialization.ActiveActive.CounterUpdate build() { + akka.persistence.typed.serialization.ActiveActive.CounterUpdate result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -1019,9 +1031,9 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.CounterUpdate buildPartial() { - akka.persistence.typed.serialization.Crdts.CounterUpdate result = - new akka.persistence.typed.serialization.Crdts.CounterUpdate(this); + public akka.persistence.typed.serialization.ActiveActive.CounterUpdate buildPartial() { + akka.persistence.typed.serialization.ActiveActive.CounterUpdate result = + new akka.persistence.typed.serialization.ActiveActive.CounterUpdate(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) != 0)) { @@ -1070,16 +1082,18 @@ public final class Crdts { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.Crdts.CounterUpdate) { - return mergeFrom((akka.persistence.typed.serialization.Crdts.CounterUpdate) other); + if (other instanceof akka.persistence.typed.serialization.ActiveActive.CounterUpdate) { + return mergeFrom((akka.persistence.typed.serialization.ActiveActive.CounterUpdate) other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.CounterUpdate other) { - if (other == akka.persistence.typed.serialization.Crdts.CounterUpdate.getDefaultInstance()) + public Builder mergeFrom( + akka.persistence.typed.serialization.ActiveActive.CounterUpdate other) { + if (other + == akka.persistence.typed.serialization.ActiveActive.CounterUpdate.getDefaultInstance()) return this; if (other.hasDelta()) { setDelta(other.getDelta()); @@ -1102,12 +1116,13 @@ public final class Crdts { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.Crdts.CounterUpdate parsedMessage = null; + akka.persistence.typed.serialization.ActiveActive.CounterUpdate parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { parsedMessage = - (akka.persistence.typed.serialization.Crdts.CounterUpdate) e.getUnfinishedMessage(); + (akka.persistence.typed.serialization.ActiveActive.CounterUpdate) + e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { if (parsedMessage != null) { @@ -1180,13 +1195,15 @@ public final class Crdts { } // @@protoc_insertion_point(class_scope:CounterUpdate) - private static final akka.persistence.typed.serialization.Crdts.CounterUpdate DEFAULT_INSTANCE; + private static final akka.persistence.typed.serialization.ActiveActive.CounterUpdate + DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.CounterUpdate(); + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.CounterUpdate(); } - public static akka.persistence.typed.serialization.Crdts.CounterUpdate getDefaultInstance() { + public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate + getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -1212,7 +1229,8 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.CounterUpdate getDefaultInstanceForType() { + public akka.persistence.typed.serialization.ActiveActive.CounterUpdate + getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } @@ -1252,21 +1270,23 @@ public final class Crdts { * * @return The vvector. */ - akka.persistence.typed.serialization.Crdts.VersionVector getVvector(); + akka.persistence.typed.serialization.ActiveActive.VersionVector getVvector(); /** required .VersionVector vvector = 2; */ - akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder getVvectorOrBuilder(); + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder getVvectorOrBuilder(); /** repeated .VersionVector dots = 3; */ - java.util.List getDotsList(); + java.util.List getDotsList(); /** repeated .VersionVector dots = 3; */ - akka.persistence.typed.serialization.Crdts.VersionVector getDots(int index); + akka.persistence.typed.serialization.ActiveActive.VersionVector getDots(int index); /** repeated .VersionVector dots = 3; */ int getDotsCount(); /** repeated .VersionVector dots = 3; */ - java.util.List + java.util.List< + ? extends akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> getDotsOrBuilderList(); /** repeated .VersionVector dots = 3; */ - akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder getDotsOrBuilder(int index); + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder getDotsOrBuilder( + int index); /** * repeated string stringElements = 4; @@ -1407,13 +1427,14 @@ public final class Crdts { } case 18: { - akka.persistence.typed.serialization.Crdts.VersionVector.Builder subBuilder = null; + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder subBuilder = + null; if (((bitField0_ & 0x00000002) != 0)) { subBuilder = vvector_.toBuilder(); } vvector_ = input.readMessage( - akka.persistence.typed.serialization.Crdts.VersionVector.PARSER, + akka.persistence.typed.serialization.ActiveActive.VersionVector.PARSER, extensionRegistry); if (subBuilder != null) { subBuilder.mergeFrom(vvector_); @@ -1427,12 +1448,12 @@ public final class Crdts { if (!((mutable_bitField0_ & 0x00000004) != 0)) { dots_ = new java.util.ArrayList< - akka.persistence.typed.serialization.Crdts.VersionVector>(); + akka.persistence.typed.serialization.ActiveActive.VersionVector>(); mutable_bitField0_ |= 0x00000004; } dots_.add( input.readMessage( - akka.persistence.typed.serialization.Crdts.VersionVector.PARSER, + akka.persistence.typed.serialization.ActiveActive.VersionVector.PARSER, extensionRegistry)); break; } @@ -1539,16 +1560,17 @@ public final class Crdts { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.internal_static_ORSet_descriptor; + return akka.persistence.typed.serialization.ActiveActive.internal_static_ORSet_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts.internal_static_ORSet_fieldAccessorTable + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ORSet_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.ORSet.class, - akka.persistence.typed.serialization.Crdts.ORSet.Builder.class); + akka.persistence.typed.serialization.ActiveActive.ORSet.class, + akka.persistence.typed.serialization.ActiveActive.ORSet.Builder.class); } private int bitField0_; @@ -1598,7 +1620,7 @@ public final class Crdts { } public static final int VVECTOR_FIELD_NUMBER = 2; - private akka.persistence.typed.serialization.Crdts.VersionVector vvector_; + private akka.persistence.typed.serialization.ActiveActive.VersionVector vvector_; /** * required .VersionVector vvector = 2; * @@ -1612,27 +1634,29 @@ public final class Crdts { * * @return The vvector. */ - public akka.persistence.typed.serialization.Crdts.VersionVector getVvector() { + public akka.persistence.typed.serialization.ActiveActive.VersionVector getVvector() { return vvector_ == null - ? akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() : vvector_; } /** required .VersionVector vvector = 2; */ - public akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder getVvectorOrBuilder() { + public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + getVvectorOrBuilder() { return vvector_ == null - ? akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() : vvector_; } public static final int DOTS_FIELD_NUMBER = 3; - private java.util.List dots_; + private java.util.List dots_; /** repeated .VersionVector dots = 3; */ - public java.util.List getDotsList() { + public java.util.List + getDotsList() { return dots_; } /** repeated .VersionVector dots = 3; */ public java.util.List< - ? extends akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + ? extends akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> getDotsOrBuilderList() { return dots_; } @@ -1641,12 +1665,12 @@ public final class Crdts { return dots_.size(); } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.Crdts.VersionVector getDots(int index) { + public akka.persistence.typed.serialization.ActiveActive.VersionVector getDots(int index) { return dots_.get(index); } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder getDotsOrBuilder( - int index) { + public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + getDotsOrBuilder(int index) { return dots_.get(index); } @@ -1909,11 +1933,11 @@ public final class Crdts { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.Crdts.ORSet)) { + if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.ORSet)) { return super.equals(obj); } - akka.persistence.typed.serialization.Crdts.ORSet other = - (akka.persistence.typed.serialization.Crdts.ORSet) obj; + akka.persistence.typed.serialization.ActiveActive.ORSet other = + (akka.persistence.typed.serialization.ActiveActive.ORSet) obj; if (hasOriginDc() != other.hasOriginDc()) return false; if (hasOriginDc()) { @@ -1972,72 +1996,72 @@ public final class Crdts { return hash; } - public static akka.persistence.typed.serialization.Crdts.ORSet parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.ORSet parseFrom( java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.ORSet parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSet parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.ORSet parseFrom( akka.protobufv3.internal.ByteString data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.ORSet parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSet parseFrom(byte[] data) + public static akka.persistence.typed.serialization.ActiveActive.ORSet parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.ORSet parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSet parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.ORSet parseFrom( java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.ORSet parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSet parseDelimitedFrom( + public static akka.persistence.typed.serialization.ActiveActive.ORSet parseDelimitedFrom( java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.ORSet parseDelimitedFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSet parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSet parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.ORSet parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2054,7 +2078,8 @@ public final class Crdts { return DEFAULT_INSTANCE.toBuilder(); } - public static Builder newBuilder(akka.persistence.typed.serialization.Crdts.ORSet prototype) { + public static Builder newBuilder( + akka.persistence.typed.serialization.ActiveActive.ORSet prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -2074,21 +2099,22 @@ public final class Crdts { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:ORSet) - akka.persistence.typed.serialization.Crdts.ORSetOrBuilder { + akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.internal_static_ORSet_descriptor; + return akka.persistence.typed.serialization.ActiveActive.internal_static_ORSet_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts.internal_static_ORSet_fieldAccessorTable + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ORSet_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.ORSet.class, - akka.persistence.typed.serialization.Crdts.ORSet.Builder.class); + akka.persistence.typed.serialization.ActiveActive.ORSet.class, + akka.persistence.typed.serialization.ActiveActive.ORSet.Builder.class); } - // Construct using akka.persistence.typed.serialization.Crdts.ORSet.newBuilder() + // Construct using akka.persistence.typed.serialization.ActiveActive.ORSet.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -2140,17 +2166,17 @@ public final class Crdts { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.Crdts.internal_static_ORSet_descriptor; + return akka.persistence.typed.serialization.ActiveActive.internal_static_ORSet_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSet getDefaultInstanceForType() { - return akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance(); + public akka.persistence.typed.serialization.ActiveActive.ORSet getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSet build() { - akka.persistence.typed.serialization.Crdts.ORSet result = buildPartial(); + public akka.persistence.typed.serialization.ActiveActive.ORSet build() { + akka.persistence.typed.serialization.ActiveActive.ORSet result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -2158,9 +2184,9 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSet buildPartial() { - akka.persistence.typed.serialization.Crdts.ORSet result = - new akka.persistence.typed.serialization.Crdts.ORSet(this); + public akka.persistence.typed.serialization.ActiveActive.ORSet buildPartial() { + akka.persistence.typed.serialization.ActiveActive.ORSet result = + new akka.persistence.typed.serialization.ActiveActive.ORSet(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) != 0)) { @@ -2250,16 +2276,16 @@ public final class Crdts { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.Crdts.ORSet) { - return mergeFrom((akka.persistence.typed.serialization.Crdts.ORSet) other); + if (other instanceof akka.persistence.typed.serialization.ActiveActive.ORSet) { + return mergeFrom((akka.persistence.typed.serialization.ActiveActive.ORSet) other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.ORSet other) { - if (other == akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance()) + public Builder mergeFrom(akka.persistence.typed.serialization.ActiveActive.ORSet other) { + if (other == akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance()) return this; if (other.hasOriginDc()) { bitField0_ |= 0x00000001; @@ -2387,12 +2413,12 @@ public final class Crdts { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.Crdts.ORSet parsedMessage = null; + akka.persistence.typed.serialization.ActiveActive.ORSet parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { parsedMessage = - (akka.persistence.typed.serialization.Crdts.ORSet) e.getUnfinishedMessage(); + (akka.persistence.typed.serialization.ActiveActive.ORSet) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { if (parsedMessage != null) { @@ -2489,11 +2515,11 @@ public final class Crdts { return this; } - private akka.persistence.typed.serialization.Crdts.VersionVector vvector_; + private akka.persistence.typed.serialization.ActiveActive.VersionVector vvector_; private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.VersionVector, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder, - akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> vvectorBuilder_; /** * required .VersionVector vvector = 2; @@ -2508,17 +2534,18 @@ public final class Crdts { * * @return The vvector. */ - public akka.persistence.typed.serialization.Crdts.VersionVector getVvector() { + public akka.persistence.typed.serialization.ActiveActive.VersionVector getVvector() { if (vvectorBuilder_ == null) { return vvector_ == null - ? akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() : vvector_; } else { return vvectorBuilder_.getMessage(); } } /** required .VersionVector vvector = 2; */ - public Builder setVvector(akka.persistence.typed.serialization.Crdts.VersionVector value) { + public Builder setVvector( + akka.persistence.typed.serialization.ActiveActive.VersionVector value) { if (vvectorBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2533,7 +2560,7 @@ public final class Crdts { } /** required .VersionVector vvector = 2; */ public Builder setVvector( - akka.persistence.typed.serialization.Crdts.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { if (vvectorBuilder_ == null) { vvector_ = builderForValue.build(); onChanged(); @@ -2544,15 +2571,16 @@ public final class Crdts { return this; } /** required .VersionVector vvector = 2; */ - public Builder mergeVvector(akka.persistence.typed.serialization.Crdts.VersionVector value) { + public Builder mergeVvector( + akka.persistence.typed.serialization.ActiveActive.VersionVector value) { if (vvectorBuilder_ == null) { if (((bitField0_ & 0x00000002) != 0) && vvector_ != null && vvector_ - != akka.persistence.typed.serialization.Crdts.VersionVector + != akka.persistence.typed.serialization.ActiveActive.VersionVector .getDefaultInstance()) { vvector_ = - akka.persistence.typed.serialization.Crdts.VersionVector.newBuilder(vvector_) + akka.persistence.typed.serialization.ActiveActive.VersionVector.newBuilder(vvector_) .mergeFrom(value) .buildPartial(); } else { @@ -2577,60 +2605,61 @@ public final class Crdts { return this; } /** required .VersionVector vvector = 2; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.Builder getVvectorBuilder() { + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder + getVvectorBuilder() { bitField0_ |= 0x00000002; onChanged(); return getVvectorFieldBuilder().getBuilder(); } /** required .VersionVector vvector = 2; */ - public akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder + public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder getVvectorOrBuilder() { if (vvectorBuilder_ != null) { return vvectorBuilder_.getMessageOrBuilder(); } else { return vvector_ == null - ? akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() : vvector_; } } /** required .VersionVector vvector = 2; */ private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.VersionVector, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder, - akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> getVvectorFieldBuilder() { if (vvectorBuilder_ == null) { vvectorBuilder_ = new akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.VersionVector, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder, - akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder>( + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder>( getVvector(), getParentForChildren(), isClean()); vvector_ = null; } return vvectorBuilder_; } - private java.util.List dots_ = - java.util.Collections.emptyList(); + private java.util.List + dots_ = java.util.Collections.emptyList(); private void ensureDotsIsMutable() { if (!((bitField0_ & 0x00000004) != 0)) { dots_ = - new java.util.ArrayList( - dots_); + new java.util.ArrayList< + akka.persistence.typed.serialization.ActiveActive.VersionVector>(dots_); bitField0_ |= 0x00000004; } } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.VersionVector, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder, - akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> dotsBuilder_; /** repeated .VersionVector dots = 3; */ - public java.util.List + public java.util.List getDotsList() { if (dotsBuilder_ == null) { return java.util.Collections.unmodifiableList(dots_); @@ -2647,7 +2676,7 @@ public final class Crdts { } } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.Crdts.VersionVector getDots(int index) { + public akka.persistence.typed.serialization.ActiveActive.VersionVector getDots(int index) { if (dotsBuilder_ == null) { return dots_.get(index); } else { @@ -2656,7 +2685,7 @@ public final class Crdts { } /** repeated .VersionVector dots = 3; */ public Builder setDots( - int index, akka.persistence.typed.serialization.Crdts.VersionVector value) { + int index, akka.persistence.typed.serialization.ActiveActive.VersionVector value) { if (dotsBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2672,7 +2701,7 @@ public final class Crdts { /** repeated .VersionVector dots = 3; */ public Builder setDots( int index, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { if (dotsBuilder_ == null) { ensureDotsIsMutable(); dots_.set(index, builderForValue.build()); @@ -2683,7 +2712,8 @@ public final class Crdts { return this; } /** repeated .VersionVector dots = 3; */ - public Builder addDots(akka.persistence.typed.serialization.Crdts.VersionVector value) { + public Builder addDots( + akka.persistence.typed.serialization.ActiveActive.VersionVector value) { if (dotsBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2698,7 +2728,7 @@ public final class Crdts { } /** repeated .VersionVector dots = 3; */ public Builder addDots( - int index, akka.persistence.typed.serialization.Crdts.VersionVector value) { + int index, akka.persistence.typed.serialization.ActiveActive.VersionVector value) { if (dotsBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2713,7 +2743,7 @@ public final class Crdts { } /** repeated .VersionVector dots = 3; */ public Builder addDots( - akka.persistence.typed.serialization.Crdts.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { if (dotsBuilder_ == null) { ensureDotsIsMutable(); dots_.add(builderForValue.build()); @@ -2726,7 +2756,7 @@ public final class Crdts { /** repeated .VersionVector dots = 3; */ public Builder addDots( int index, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { if (dotsBuilder_ == null) { ensureDotsIsMutable(); dots_.add(index, builderForValue.build()); @@ -2738,7 +2768,8 @@ public final class Crdts { } /** repeated .VersionVector dots = 3; */ public Builder addAllDots( - java.lang.Iterable + java.lang.Iterable< + ? extends akka.persistence.typed.serialization.ActiveActive.VersionVector> values) { if (dotsBuilder_ == null) { ensureDotsIsMutable(); @@ -2772,13 +2803,13 @@ public final class Crdts { return this; } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.Builder getDotsBuilder( + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder getDotsBuilder( int index) { return getDotsFieldBuilder().getBuilder(index); } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder getDotsOrBuilder( - int index) { + public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + getDotsOrBuilder(int index) { if (dotsBuilder_ == null) { return dots_.get(index); } else { @@ -2787,7 +2818,7 @@ public final class Crdts { } /** repeated .VersionVector dots = 3; */ public java.util.List< - ? extends akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + ? extends akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> getDotsOrBuilderList() { if (dotsBuilder_ != null) { return dotsBuilder_.getMessageOrBuilderList(); @@ -2796,36 +2827,39 @@ public final class Crdts { } } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.Builder addDotsBuilder() { + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder + addDotsBuilder() { return getDotsFieldBuilder() .addBuilder( - akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance()); + akka.persistence.typed.serialization.ActiveActive.VersionVector + .getDefaultInstance()); } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.Builder addDotsBuilder( + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder addDotsBuilder( int index) { return getDotsFieldBuilder() .addBuilder( index, - akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance()); + akka.persistence.typed.serialization.ActiveActive.VersionVector + .getDefaultInstance()); } /** repeated .VersionVector dots = 3; */ - public java.util.List + public java.util.List getDotsBuilderList() { return getDotsFieldBuilder().getBuilderList(); } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.VersionVector, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder, - akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder> + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> getDotsFieldBuilder() { if (dotsBuilder_ == null) { dotsBuilder_ = new akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.VersionVector, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder, - akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder>( + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder>( dots_, ((bitField0_ & 0x00000004) != 0), getParentForChildren(), isClean()); dots_ = null; } @@ -3339,13 +3373,13 @@ public final class Crdts { } // @@protoc_insertion_point(class_scope:ORSet) - private static final akka.persistence.typed.serialization.Crdts.ORSet DEFAULT_INSTANCE; + private static final akka.persistence.typed.serialization.ActiveActive.ORSet DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.ORSet(); + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.ORSet(); } - public static akka.persistence.typed.serialization.Crdts.ORSet getDefaultInstance() { + public static akka.persistence.typed.serialization.ActiveActive.ORSet getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -3371,7 +3405,7 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSet getDefaultInstanceForType() { + public akka.persistence.typed.serialization.ActiveActive.ORSet getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } @@ -3382,19 +3416,20 @@ public final class Crdts { akka.protobufv3.internal.MessageOrBuilder { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - java.util.List + java.util.List getEntriesList(); /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry getEntries(int index); + akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup.EntryOrBuilder> + ? extends + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> getEntriesOrBuilderList(); /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder getEntriesOrBuilder( - int index); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder + getEntriesOrBuilder(int index); } /** Protobuf type {@code ORSetDeltaGroup} */ public static final class ORSetDeltaGroup extends akka.protobufv3.internal.GeneratedMessageV3 @@ -3447,12 +3482,14 @@ public final class Crdts { if (!((mutable_bitField0_ & 0x00000001) != 0)) { entries_ = new java.util.ArrayList< - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry>(); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + .Entry>(); mutable_bitField0_ |= 0x00000001; } entries_.add( input.readMessage( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.PARSER, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + .PARSER, extensionRegistry)); break; } @@ -3480,17 +3517,18 @@ public final class Crdts { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.internal_static_ORSetDeltaGroup_descriptor; + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ORSetDeltaGroup_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_ORSetDeltaGroup_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.class, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Builder.class); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.class, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Builder.class); } public interface EntryOrBuilder @@ -3509,7 +3547,7 @@ public final class Crdts { * * @return The operation. */ - akka.persistence.typed.serialization.Crdts.ORSetDeltaOp getOperation(); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp getOperation(); /** * required .ORSet underlying = 2; @@ -3522,9 +3560,9 @@ public final class Crdts { * * @return The underlying. */ - akka.persistence.typed.serialization.Crdts.ORSet getUnderlying(); + akka.persistence.typed.serialization.ActiveActive.ORSet getUnderlying(); /** required .ORSet underlying = 2; */ - akka.persistence.typed.serialization.Crdts.ORSetOrBuilder getUnderlyingOrBuilder(); + akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder getUnderlyingOrBuilder(); } /** Protobuf type {@code ORSetDeltaGroup.Entry} */ public static final class Entry extends akka.protobufv3.internal.GeneratedMessageV3 @@ -3576,8 +3614,9 @@ public final class Crdts { { int rawValue = input.readEnum(); @SuppressWarnings("deprecation") - akka.persistence.typed.serialization.Crdts.ORSetDeltaOp value = - akka.persistence.typed.serialization.Crdts.ORSetDeltaOp.valueOf(rawValue); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp value = + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.valueOf( + rawValue); if (value == null) { unknownFields.mergeVarintField(1, rawValue); } else { @@ -3588,13 +3627,13 @@ public final class Crdts { } case 18: { - akka.persistence.typed.serialization.Crdts.ORSet.Builder subBuilder = null; + akka.persistence.typed.serialization.ActiveActive.ORSet.Builder subBuilder = null; if (((bitField0_ & 0x00000002) != 0)) { subBuilder = underlying_.toBuilder(); } underlying_ = input.readMessage( - akka.persistence.typed.serialization.Crdts.ORSet.PARSER, + akka.persistence.typed.serialization.ActiveActive.ORSet.PARSER, extensionRegistry); if (subBuilder != null) { subBuilder.mergeFrom(underlying_); @@ -3624,18 +3663,19 @@ public final class Crdts { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_ORSetDeltaGroup_Entry_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.class, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder.class); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.class, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder + .class); } private int bitField0_; @@ -3654,17 +3694,17 @@ public final class Crdts { * * @return The operation. */ - public akka.persistence.typed.serialization.Crdts.ORSetDeltaOp getOperation() { + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp getOperation() { @SuppressWarnings("deprecation") - akka.persistence.typed.serialization.Crdts.ORSetDeltaOp result = - akka.persistence.typed.serialization.Crdts.ORSetDeltaOp.valueOf(operation_); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp result = + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.valueOf(operation_); return result == null - ? akka.persistence.typed.serialization.Crdts.ORSetDeltaOp.Add + ? akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.Add : result; } public static final int UNDERLYING_FIELD_NUMBER = 2; - private akka.persistence.typed.serialization.Crdts.ORSet underlying_; + private akka.persistence.typed.serialization.ActiveActive.ORSet underlying_; /** * required .ORSet underlying = 2; * @@ -3678,15 +3718,16 @@ public final class Crdts { * * @return The underlying. */ - public akka.persistence.typed.serialization.Crdts.ORSet getUnderlying() { + public akka.persistence.typed.serialization.ActiveActive.ORSet getUnderlying() { return underlying_ == null - ? akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance() + ? akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance() : underlying_; } /** required .ORSet underlying = 2; */ - public akka.persistence.typed.serialization.Crdts.ORSetOrBuilder getUnderlyingOrBuilder() { + public akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder + getUnderlyingOrBuilder() { return underlying_ == null - ? akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance() + ? akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance() : underlying_; } @@ -3748,11 +3789,12 @@ public final class Crdts { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry)) { + if (!(obj + instanceof akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry)) { return super.equals(obj); } - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry other = - (akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry) obj; + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry other = + (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry) obj; if (hasOperation() != other.hasOperation()) return false; if (hasOperation()) { @@ -3786,62 +3828,66 @@ public final class Crdts { return hash; } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry parseFrom( - java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry parseFrom( - java.nio.ByteBuffer data, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup.Entry parseFrom( - akka.protobufv3.internal.ByteString data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry parseFrom( - akka.protobufv3.internal.ByteString data, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup.Entry parseFrom( - byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry parseFrom( - byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup.Entry parseFrom( - java.io.InputStream input) throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup.Entry parseFrom( - java.io.InputStream input, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup.Entry + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup.Entry + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -3850,15 +3896,16 @@ public final class Crdts { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry parseFrom( - akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup.Entry parseFrom( - akka.protobufv3.internal.CodedInputStream input, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.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); } @@ -3873,7 +3920,7 @@ public final class Crdts { } public static Builder newBuilder( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry prototype) { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -3893,24 +3940,25 @@ public final class Crdts { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:ORSetDeltaGroup.Entry) - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_ORSetDeltaGroup_Entry_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.class, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder.class); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.class, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder + .class); } // Construct using - // akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.newBuilder() + // akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -3942,20 +3990,21 @@ public final class Crdts { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_ORSetDeltaGroup_Entry_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry getDefaultInstanceForType() { - return akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + return akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry build() { - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry result = buildPartial(); + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry build() { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry result = + buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -3963,9 +4012,10 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry buildPartial() { - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry result = - new akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry(this); + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + buildPartial() { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry result = + new akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) != 0)) { @@ -4022,9 +4072,10 @@ public final class Crdts { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry) { + if (other + instanceof akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry) { return mergeFrom( - (akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry) other); + (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry) other); } else { super.mergeFrom(other); return this; @@ -4032,9 +4083,9 @@ public final class Crdts { } public Builder mergeFrom( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry other) { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry other) { if (other - == akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + == akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry .getDefaultInstance()) return this; if (other.hasOperation()) { setOperation(other.getOperation()); @@ -4066,12 +4117,13 @@ public final class Crdts { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry parsedMessage = null; + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry parsedMessage = + null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { parsedMessage = - (akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry) + (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -4098,12 +4150,12 @@ public final class Crdts { * * @return The operation. */ - public akka.persistence.typed.serialization.Crdts.ORSetDeltaOp getOperation() { + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp getOperation() { @SuppressWarnings("deprecation") - akka.persistence.typed.serialization.Crdts.ORSetDeltaOp result = - akka.persistence.typed.serialization.Crdts.ORSetDeltaOp.valueOf(operation_); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp result = + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.valueOf(operation_); return result == null - ? akka.persistence.typed.serialization.Crdts.ORSetDeltaOp.Add + ? akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.Add : result; } /** @@ -4112,7 +4164,8 @@ public final class Crdts { * @param value The operation to set. * @return This builder for chaining. */ - public Builder setOperation(akka.persistence.typed.serialization.Crdts.ORSetDeltaOp value) { + public Builder setOperation( + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp value) { if (value == null) { throw new NullPointerException(); } @@ -4133,11 +4186,11 @@ public final class Crdts { return this; } - private akka.persistence.typed.serialization.Crdts.ORSet underlying_; + private akka.persistence.typed.serialization.ActiveActive.ORSet underlying_; private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.ORSet, - akka.persistence.typed.serialization.Crdts.ORSet.Builder, - akka.persistence.typed.serialization.Crdts.ORSetOrBuilder> + akka.persistence.typed.serialization.ActiveActive.ORSet, + akka.persistence.typed.serialization.ActiveActive.ORSet.Builder, + akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder> underlyingBuilder_; /** * required .ORSet underlying = 2; @@ -4152,17 +4205,18 @@ public final class Crdts { * * @return The underlying. */ - public akka.persistence.typed.serialization.Crdts.ORSet getUnderlying() { + public akka.persistence.typed.serialization.ActiveActive.ORSet getUnderlying() { if (underlyingBuilder_ == null) { return underlying_ == null - ? akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance() + ? akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance() : underlying_; } else { return underlyingBuilder_.getMessage(); } } /** required .ORSet underlying = 2; */ - public Builder setUnderlying(akka.persistence.typed.serialization.Crdts.ORSet value) { + public Builder setUnderlying( + akka.persistence.typed.serialization.ActiveActive.ORSet value) { if (underlyingBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -4177,7 +4231,7 @@ public final class Crdts { } /** required .ORSet underlying = 2; */ public Builder setUnderlying( - akka.persistence.typed.serialization.Crdts.ORSet.Builder builderForValue) { + akka.persistence.typed.serialization.ActiveActive.ORSet.Builder builderForValue) { if (underlyingBuilder_ == null) { underlying_ = builderForValue.build(); onChanged(); @@ -4188,14 +4242,16 @@ public final class Crdts { return this; } /** required .ORSet underlying = 2; */ - public Builder mergeUnderlying(akka.persistence.typed.serialization.Crdts.ORSet value) { + public Builder mergeUnderlying( + akka.persistence.typed.serialization.ActiveActive.ORSet value) { if (underlyingBuilder_ == null) { if (((bitField0_ & 0x00000002) != 0) && underlying_ != null && underlying_ - != akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance()) { + != akka.persistence.typed.serialization.ActiveActive.ORSet + .getDefaultInstance()) { underlying_ = - akka.persistence.typed.serialization.Crdts.ORSet.newBuilder(underlying_) + akka.persistence.typed.serialization.ActiveActive.ORSet.newBuilder(underlying_) .mergeFrom(value) .buildPartial(); } else { @@ -4220,33 +4276,35 @@ public final class Crdts { return this; } /** required .ORSet underlying = 2; */ - public akka.persistence.typed.serialization.Crdts.ORSet.Builder getUnderlyingBuilder() { + public akka.persistence.typed.serialization.ActiveActive.ORSet.Builder + getUnderlyingBuilder() { bitField0_ |= 0x00000002; onChanged(); return getUnderlyingFieldBuilder().getBuilder(); } /** required .ORSet underlying = 2; */ - public akka.persistence.typed.serialization.Crdts.ORSetOrBuilder getUnderlyingOrBuilder() { + public akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder + getUnderlyingOrBuilder() { if (underlyingBuilder_ != null) { return underlyingBuilder_.getMessageOrBuilder(); } else { return underlying_ == null - ? akka.persistence.typed.serialization.Crdts.ORSet.getDefaultInstance() + ? akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance() : underlying_; } } /** required .ORSet underlying = 2; */ private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.ORSet, - akka.persistence.typed.serialization.Crdts.ORSet.Builder, - akka.persistence.typed.serialization.Crdts.ORSetOrBuilder> + akka.persistence.typed.serialization.ActiveActive.ORSet, + akka.persistence.typed.serialization.ActiveActive.ORSet.Builder, + akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder> getUnderlyingFieldBuilder() { if (underlyingBuilder_ == null) { underlyingBuilder_ = new akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.ORSet, - akka.persistence.typed.serialization.Crdts.ORSet.Builder, - akka.persistence.typed.serialization.Crdts.ORSetOrBuilder>( + akka.persistence.typed.serialization.ActiveActive.ORSet, + akka.persistence.typed.serialization.ActiveActive.ORSet.Builder, + akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder>( getUnderlying(), getParentForChildren(), isClean()); underlying_ = null; } @@ -4269,14 +4327,15 @@ public final class Crdts { } // @@protoc_insertion_point(class_scope:ORSetDeltaGroup.Entry) - private static final akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + private static final akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry(); + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry(); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -4303,23 +4362,24 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } public static final int ENTRIES_FIELD_NUMBER = 1; - private java.util.List + private java.util.List entries_; /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public java.util.List + public java.util.List getEntriesList() { return entries_; } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public java.util.List< - ? extends akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder> + ? extends + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> getEntriesOrBuilderList() { return entries_; } @@ -4328,11 +4388,12 @@ public final class Crdts { return entries_.size(); } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry getEntries(int index) { + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry getEntries( + int index) { return entries_.get(index); } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder getEntriesOrBuilder(int index) { return entries_.get(index); } @@ -4383,11 +4444,11 @@ public final class Crdts { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup)) { + if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup)) { return super.equals(obj); } - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup other = - (akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup) obj; + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup other = + (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup) obj; if (!getEntriesList().equals(other.getEntriesList())) return false; if (!unknownFields.equals(other.unknownFields)) return false; @@ -4410,72 +4471,74 @@ public final class Crdts { return hash; } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup parseFrom( java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup parseFrom( akka.protobufv3.internal.ByteString data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup parseFrom(byte[] data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup parseFrom( + byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup parseFrom( java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup parseDelimitedFrom( - java.io.InputStream input) throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup parseDelimitedFrom( - java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.ORSetDeltaGroup parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -4493,7 +4556,7 @@ public final class Crdts { } public static Builder newBuilder( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup prototype) { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -4513,23 +4576,24 @@ public final class Crdts { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:ORSetDeltaGroup) - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroupOrBuilder { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroupOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_ORSetDeltaGroup_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_ORSetDeltaGroup_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.class, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Builder.class); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.class, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Builder.class); } - // Construct using akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.newBuilder() + // Construct using + // akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -4559,19 +4623,20 @@ public final class Crdts { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_ORSetDeltaGroup_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup getDefaultInstanceForType() { - return akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.getDefaultInstance(); + return akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup build() { - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup result = buildPartial(); + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup build() { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -4579,9 +4644,9 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup buildPartial() { - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup result = - new akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup(this); + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup buildPartial() { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup result = + new akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup(this); int from_bitField0_ = bitField0_; if (entriesBuilder_ == null) { if (((bitField0_ & 0x00000001) != 0)) { @@ -4633,18 +4698,20 @@ public final class Crdts { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup) { - return mergeFrom((akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup) other); + if (other instanceof akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup) { + return mergeFrom( + (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup) other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup other) { + public Builder mergeFrom( + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup other) { if (other - == akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.getDefaultInstance()) - return this; + == akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + .getDefaultInstance()) return this; if (entriesBuilder_ == null) { if (!other.entries_.isEmpty()) { if (entries_.isEmpty()) { @@ -4692,12 +4759,13 @@ public final class Crdts { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup parsedMessage = null; + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { parsedMessage = - (akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup) e.getUnfinishedMessage(); + (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup) + e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { if (parsedMessage != null) { @@ -4709,26 +4777,28 @@ public final class Crdts { private int bitField0_; - private java.util.List + private java.util.List< + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry> entries_ = java.util.Collections.emptyList(); private void ensureEntriesIsMutable() { if (!((bitField0_ & 0x00000001) != 0)) { entries_ = new java.util.ArrayList< - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry>(entries_); + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry>( + entries_); bitField0_ |= 0x00000001; } } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder> + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> entriesBuilder_; /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public java.util.List + public java.util.List getEntriesList() { if (entriesBuilder_ == null) { return java.util.Collections.unmodifiableList(entries_); @@ -4745,7 +4815,7 @@ public final class Crdts { } } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry getEntries( + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry getEntries( int index) { if (entriesBuilder_ == null) { return entries_.get(index); @@ -4755,7 +4825,8 @@ public final class Crdts { } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder setEntries( - int index, akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry value) { + int index, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -4771,7 +4842,7 @@ public final class Crdts { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder setEntries( int index, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -4784,7 +4855,7 @@ public final class Crdts { } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addEntries( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry value) { + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -4799,7 +4870,8 @@ public final class Crdts { } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addEntries( - int index, akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry value) { + int index, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -4814,7 +4886,7 @@ public final class Crdts { } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addEntries( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -4828,7 +4900,7 @@ public final class Crdts { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addEntries( int index, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -4842,7 +4914,7 @@ public final class Crdts { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addAllEntries( java.lang.Iterable< - ? extends akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry> + ? extends akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry> values) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -4876,12 +4948,12 @@ public final class Crdts { return this; } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder getEntriesBuilder(int index) { return getEntriesFieldBuilder().getBuilder(index); } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder getEntriesOrBuilder(int index) { if (entriesBuilder_ == null) { return entries_.get(index); @@ -4891,7 +4963,8 @@ public final class Crdts { } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public java.util.List< - ? extends akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder> + ? extends + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> getEntriesOrBuilderList() { if (entriesBuilder_ != null) { return entriesBuilder_.getMessageOrBuilderList(); @@ -4900,40 +4973,40 @@ public final class Crdts { } } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder addEntriesBuilder() { return getEntriesFieldBuilder() .addBuilder( - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry .getDefaultInstance()); } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder addEntriesBuilder(int index) { return getEntriesFieldBuilder() .addBuilder( index, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry .getDefaultInstance()); } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public java.util.List< - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder> + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder> getEntriesBuilderList() { return getEntriesFieldBuilder().getBuilderList(); } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder> + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> getEntriesFieldBuilder() { if (entriesBuilder_ == null) { entriesBuilder_ = new akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.Entry.Builder, - akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup.EntryOrBuilder>( + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder, + akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder>( entries_, ((bitField0_ & 0x00000001) != 0), getParentForChildren(), isClean()); entries_ = null; } @@ -4956,14 +5029,15 @@ public final class Crdts { } // @@protoc_insertion_point(class_scope:ORSetDeltaGroup) - private static final akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup + private static final akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup(); + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup(); } - public static akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup getDefaultInstance() { + public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -4989,7 +5063,8 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.ORSetDeltaGroup getDefaultInstanceForType() { + public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } @@ -5000,18 +5075,20 @@ public final class Crdts { akka.protobufv3.internal.MessageOrBuilder { /** repeated .VersionVector.Entry entries = 1; */ - java.util.List getEntriesList(); + java.util.List + getEntriesList(); /** repeated .VersionVector.Entry entries = 1; */ - akka.persistence.typed.serialization.Crdts.VersionVector.Entry getEntries(int index); + akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector.EntryOrBuilder> + ? extends + akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder> getEntriesOrBuilderList(); /** repeated .VersionVector.Entry entries = 1; */ - akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder getEntriesOrBuilder( - int index); + akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder + getEntriesOrBuilder(int index); } /** Protobuf type {@code VersionVector} */ public static final class VersionVector extends akka.protobufv3.internal.GeneratedMessageV3 @@ -5064,12 +5141,13 @@ public final class Crdts { if (!((mutable_bitField0_ & 0x00000001) != 0)) { entries_ = new java.util.ArrayList< - akka.persistence.typed.serialization.Crdts.VersionVector.Entry>(); + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry>(); mutable_bitField0_ |= 0x00000001; } entries_.add( input.readMessage( - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.PARSER, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + .PARSER, extensionRegistry)); break; } @@ -5097,17 +5175,18 @@ public final class Crdts { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.internal_static_VersionVector_descriptor; + return akka.persistence.typed.serialization.ActiveActive + .internal_static_VersionVector_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_VersionVector_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.VersionVector.class, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder.class); + akka.persistence.typed.serialization.ActiveActive.VersionVector.class, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder.class); } public interface EntryOrBuilder @@ -5227,18 +5306,19 @@ public final class Crdts { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_VersionVector_Entry_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_VersionVector_Entry_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.class, - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder.class); + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.class, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + .class); } private int bitField0_; @@ -5360,11 +5440,12 @@ public final class Crdts { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.Crdts.VersionVector.Entry)) { + if (!(obj + instanceof akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry)) { return super.equals(obj); } - akka.persistence.typed.serialization.Crdts.VersionVector.Entry other = - (akka.persistence.typed.serialization.Crdts.VersionVector.Entry) obj; + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry other = + (akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry) obj; if (hasKey() != other.hasKey()) return false; if (hasKey()) { @@ -5398,48 +5479,48 @@ public final class Crdts { return hash; } - public static akka.persistence.typed.serialization.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parseFrom( java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parseFrom( akka.protobufv3.internal.ByteString data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parseFrom( byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parseFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -5447,13 +5528,13 @@ public final class Crdts { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.Crdts.VersionVector.Entry + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector.Entry + public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -5462,12 +5543,12 @@ public final class Crdts { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector.Entry parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -5485,7 +5566,7 @@ public final class Crdts { } public static Builder newBuilder( - akka.persistence.typed.serialization.Crdts.VersionVector.Entry prototype) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -5505,24 +5586,25 @@ public final class Crdts { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:VersionVector.Entry) - akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder { + akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_VersionVector_Entry_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_VersionVector_Entry_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.class, - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder.class); + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.class, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + .class); } // Construct using - // akka.persistence.typed.serialization.Crdts.VersionVector.Entry.newBuilder() + // akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -5548,20 +5630,21 @@ public final class Crdts { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_VersionVector_Entry_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.VersionVector.Entry + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry getDefaultInstanceForType() { - return akka.persistence.typed.serialization.Crdts.VersionVector.Entry + return akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.VersionVector.Entry build() { - akka.persistence.typed.serialization.Crdts.VersionVector.Entry result = buildPartial(); + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry build() { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry result = + buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -5569,9 +5652,10 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.VersionVector.Entry buildPartial() { - akka.persistence.typed.serialization.Crdts.VersionVector.Entry result = - new akka.persistence.typed.serialization.Crdts.VersionVector.Entry(this); + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + buildPartial() { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry result = + new akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) != 0)) { @@ -5624,9 +5708,10 @@ public final class Crdts { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.Crdts.VersionVector.Entry) { + if (other + instanceof akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry) { return mergeFrom( - (akka.persistence.typed.serialization.Crdts.VersionVector.Entry) other); + (akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry) other); } else { super.mergeFrom(other); return this; @@ -5634,9 +5719,9 @@ public final class Crdts { } public Builder mergeFrom( - akka.persistence.typed.serialization.Crdts.VersionVector.Entry other) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry other) { if (other - == akka.persistence.typed.serialization.Crdts.VersionVector.Entry + == akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry .getDefaultInstance()) return this; if (other.hasKey()) { bitField0_ |= 0x00000001; @@ -5667,12 +5752,13 @@ public final class Crdts { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.Crdts.VersionVector.Entry parsedMessage = null; + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parsedMessage = + null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { parsedMessage = - (akka.persistence.typed.serialization.Crdts.VersionVector.Entry) + (akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -5827,14 +5913,15 @@ public final class Crdts { } // @@protoc_insertion_point(class_scope:VersionVector.Entry) - private static final akka.persistence.typed.serialization.Crdts.VersionVector.Entry + private static final akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.VersionVector.Entry(); + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry(); } - public static akka.persistence.typed.serialization.Crdts.VersionVector.Entry + public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -5861,22 +5948,24 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.VersionVector.Entry + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } public static final int ENTRIES_FIELD_NUMBER = 1; - private java.util.List entries_; + private java.util.List + entries_; /** repeated .VersionVector.Entry entries = 1; */ - public java.util.List + public java.util.List getEntriesList() { return entries_; } /** repeated .VersionVector.Entry entries = 1; */ public java.util.List< - ? extends akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder> + ? extends + akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder> getEntriesOrBuilderList() { return entries_; } @@ -5885,11 +5974,12 @@ public final class Crdts { return entries_.size(); } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.Entry getEntries(int index) { + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry getEntries( + int index) { return entries_.get(index); } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder + public akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder getEntriesOrBuilder(int index) { return entries_.get(index); } @@ -5940,11 +6030,11 @@ public final class Crdts { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.Crdts.VersionVector)) { + if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.VersionVector)) { return super.equals(obj); } - akka.persistence.typed.serialization.Crdts.VersionVector other = - (akka.persistence.typed.serialization.Crdts.VersionVector) obj; + akka.persistence.typed.serialization.ActiveActive.VersionVector other = + (akka.persistence.typed.serialization.ActiveActive.VersionVector) obj; if (!getEntriesList().equals(other.getEntriesList())) return false; if (!unknownFields.equals(other.unknownFields)) return false; @@ -5967,72 +6057,74 @@ public final class Crdts { return hash; } - public static akka.persistence.typed.serialization.Crdts.VersionVector parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.VersionVector parseFrom( java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.VersionVector parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.VersionVector parseFrom( akka.protobufv3.internal.ByteString data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.VersionVector parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector parseFrom(byte[] data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + public static akka.persistence.typed.serialization.ActiveActive.VersionVector parseFrom( + byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { return PARSER.parseFrom(data); } - public static akka.persistence.typed.serialization.Crdts.VersionVector parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.VersionVector parseFrom( java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.VersionVector parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector parseDelimitedFrom( - java.io.InputStream input) throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.VersionVector + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( PARSER, input); } - public static akka.persistence.typed.serialization.Crdts.VersionVector parseDelimitedFrom( - java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.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.Crdts.VersionVector parseFrom( + public static akka.persistence.typed.serialization.ActiveActive.VersionVector parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -6050,7 +6142,7 @@ public final class Crdts { } public static Builder newBuilder( - akka.persistence.typed.serialization.Crdts.VersionVector prototype) { + akka.persistence.typed.serialization.ActiveActive.VersionVector prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -6070,22 +6162,24 @@ public final class Crdts { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:VersionVector) - akka.persistence.typed.serialization.Crdts.VersionVectorOrBuilder { + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.Crdts.internal_static_VersionVector_descriptor; + return akka.persistence.typed.serialization.ActiveActive + .internal_static_VersionVector_descriptor; } @java.lang.Override protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.persistence.typed.serialization.Crdts + return akka.persistence.typed.serialization.ActiveActive .internal_static_VersionVector_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.Crdts.VersionVector.class, - akka.persistence.typed.serialization.Crdts.VersionVector.Builder.class); + akka.persistence.typed.serialization.ActiveActive.VersionVector.class, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder.class); } - // Construct using akka.persistence.typed.serialization.Crdts.VersionVector.newBuilder() + // Construct using + // akka.persistence.typed.serialization.ActiveActive.VersionVector.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -6115,17 +6209,19 @@ public final class Crdts { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.Crdts.internal_static_VersionVector_descriptor; + return akka.persistence.typed.serialization.ActiveActive + .internal_static_VersionVector_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.VersionVector getDefaultInstanceForType() { - return akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance(); + public akka.persistence.typed.serialization.ActiveActive.VersionVector + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.VersionVector build() { - akka.persistence.typed.serialization.Crdts.VersionVector result = buildPartial(); + public akka.persistence.typed.serialization.ActiveActive.VersionVector build() { + akka.persistence.typed.serialization.ActiveActive.VersionVector result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -6133,9 +6229,9 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.VersionVector buildPartial() { - akka.persistence.typed.serialization.Crdts.VersionVector result = - new akka.persistence.typed.serialization.Crdts.VersionVector(this); + public akka.persistence.typed.serialization.ActiveActive.VersionVector buildPartial() { + akka.persistence.typed.serialization.ActiveActive.VersionVector result = + new akka.persistence.typed.serialization.ActiveActive.VersionVector(this); int from_bitField0_ = bitField0_; if (entriesBuilder_ == null) { if (((bitField0_ & 0x00000001) != 0)) { @@ -6187,16 +6283,18 @@ public final class Crdts { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.Crdts.VersionVector) { - return mergeFrom((akka.persistence.typed.serialization.Crdts.VersionVector) other); + if (other instanceof akka.persistence.typed.serialization.ActiveActive.VersionVector) { + return mergeFrom((akka.persistence.typed.serialization.ActiveActive.VersionVector) other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(akka.persistence.typed.serialization.Crdts.VersionVector other) { - if (other == akka.persistence.typed.serialization.Crdts.VersionVector.getDefaultInstance()) + public Builder mergeFrom( + akka.persistence.typed.serialization.ActiveActive.VersionVector other) { + if (other + == akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance()) return this; if (entriesBuilder_ == null) { if (!other.entries_.isEmpty()) { @@ -6245,12 +6343,13 @@ public final class Crdts { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.Crdts.VersionVector parsedMessage = null; + akka.persistence.typed.serialization.ActiveActive.VersionVector parsedMessage = null; try { parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { parsedMessage = - (akka.persistence.typed.serialization.Crdts.VersionVector) e.getUnfinishedMessage(); + (akka.persistence.typed.serialization.ActiveActive.VersionVector) + e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { if (parsedMessage != null) { @@ -6262,26 +6361,26 @@ public final class Crdts { private int bitField0_; - private java.util.List + private java.util.List entries_ = java.util.Collections.emptyList(); private void ensureEntriesIsMutable() { if (!((bitField0_ & 0x00000001) != 0)) { entries_ = new java.util.ArrayList< - akka.persistence.typed.serialization.Crdts.VersionVector.Entry>(entries_); + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry>(entries_); bitField0_ |= 0x00000001; } } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.VersionVector.Entry, - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder, - akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder> + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder> entriesBuilder_; /** repeated .VersionVector.Entry entries = 1; */ - public java.util.List + public java.util.List getEntriesList() { if (entriesBuilder_ == null) { return java.util.Collections.unmodifiableList(entries_); @@ -6298,7 +6397,8 @@ public final class Crdts { } } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.Entry getEntries(int index) { + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry getEntries( + int index) { if (entriesBuilder_ == null) { return entries_.get(index); } else { @@ -6307,7 +6407,7 @@ public final class Crdts { } /** repeated .VersionVector.Entry entries = 1; */ public Builder setEntries( - int index, akka.persistence.typed.serialization.Crdts.VersionVector.Entry value) { + int index, akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -6323,7 +6423,8 @@ public final class Crdts { /** repeated .VersionVector.Entry entries = 1; */ public Builder setEntries( int index, - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder builderForValue) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); entries_.set(index, builderForValue.build()); @@ -6335,7 +6436,7 @@ public final class Crdts { } /** repeated .VersionVector.Entry entries = 1; */ public Builder addEntries( - akka.persistence.typed.serialization.Crdts.VersionVector.Entry value) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -6350,7 +6451,7 @@ public final class Crdts { } /** repeated .VersionVector.Entry entries = 1; */ public Builder addEntries( - int index, akka.persistence.typed.serialization.Crdts.VersionVector.Entry value) { + int index, akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -6365,7 +6466,8 @@ public final class Crdts { } /** repeated .VersionVector.Entry entries = 1; */ public Builder addEntries( - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder builderForValue) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); entries_.add(builderForValue.build()); @@ -6378,7 +6480,8 @@ public final class Crdts { /** repeated .VersionVector.Entry entries = 1; */ public Builder addEntries( int index, - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder builderForValue) { + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); entries_.add(index, builderForValue.build()); @@ -6391,7 +6494,7 @@ public final class Crdts { /** repeated .VersionVector.Entry entries = 1; */ public Builder addAllEntries( java.lang.Iterable< - ? extends akka.persistence.typed.serialization.Crdts.VersionVector.Entry> + ? extends akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry> values) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -6425,12 +6528,12 @@ public final class Crdts { return this; } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder getEntriesBuilder(int index) { return getEntriesFieldBuilder().getBuilder(index); } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder + public akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder getEntriesOrBuilder(int index) { if (entriesBuilder_ == null) { return entries_.get(index); @@ -6440,7 +6543,8 @@ public final class Crdts { } /** repeated .VersionVector.Entry entries = 1; */ public java.util.List< - ? extends akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder> + ? extends + akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder> getEntriesOrBuilderList() { if (entriesBuilder_ != null) { return entriesBuilder_.getMessageOrBuilderList(); @@ -6449,39 +6553,40 @@ public final class Crdts { } } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder addEntriesBuilder() { return getEntriesFieldBuilder() .addBuilder( - akka.persistence.typed.serialization.Crdts.VersionVector.Entry + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry .getDefaultInstance()); } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder + public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder addEntriesBuilder(int index) { return getEntriesFieldBuilder() .addBuilder( index, - akka.persistence.typed.serialization.Crdts.VersionVector.Entry + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry .getDefaultInstance()); } /** repeated .VersionVector.Entry entries = 1; */ - public java.util.List + public java.util.List< + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder> getEntriesBuilderList() { return getEntriesFieldBuilder().getBuilderList(); } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.VersionVector.Entry, - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder, - akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder> + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder> getEntriesFieldBuilder() { if (entriesBuilder_ == null) { entriesBuilder_ = new akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.Crdts.VersionVector.Entry, - akka.persistence.typed.serialization.Crdts.VersionVector.Entry.Builder, - akka.persistence.typed.serialization.Crdts.VersionVector.EntryOrBuilder>( + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder>( entries_, ((bitField0_ & 0x00000001) != 0), getParentForChildren(), isClean()); entries_ = null; } @@ -6504,13 +6609,15 @@ public final class Crdts { } // @@protoc_insertion_point(class_scope:VersionVector) - private static final akka.persistence.typed.serialization.Crdts.VersionVector DEFAULT_INSTANCE; + private static final akka.persistence.typed.serialization.ActiveActive.VersionVector + DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.Crdts.VersionVector(); + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.VersionVector(); } - public static akka.persistence.typed.serialization.Crdts.VersionVector getDefaultInstance() { + public static akka.persistence.typed.serialization.ActiveActive.VersionVector + getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -6536,7 +6643,3023 @@ public final class Crdts { } @java.lang.Override - public akka.persistence.typed.serialization.Crdts.VersionVector getDefaultInstanceForType() { + public akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.VersionVector getVersionVector(); + /** required .VersionVector versionVector = 3; */ + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.VersionVector.Builder subBuilder = + null; + if (((bitField0_ & 0x00000004) != 0)) { + subBuilder = versionVector_.toBuilder(); + } + versionVector_ = + input.readMessage( + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive + .internal_static_ReplicatedEventMetadata_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ReplicatedEventMetadata_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata.class, + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.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.ActiveActive.VersionVector getVersionVector() { + return versionVector_ == null + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + : versionVector_; + } + /** required .VersionVector versionVector = 3; */ + public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + getVersionVectorOrBuilder() { + return versionVector_ == null + ? akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedEventMetadata)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata other = + (akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedEventMetadata + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedEventMetadata + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedEventMetadata + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedEventMetadata + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedEventMetadata + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.ReplicatedEventMetadataOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ReplicatedEventMetadata_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ReplicatedEventMetadata_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata.class, + akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata.Builder + .class); + } + + // Construct using + // akka.persistence.typed.serialization.ActiveActive.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.ActiveActive + .internal_static_ReplicatedEventMetadata_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata build() { + akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + buildPartial() { + akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata result = + new akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedEventMetadata) { + return mergeFrom( + (akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata other) { + if (other + == akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedEventMetadata parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.VersionVector versionVector_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.VersionVector getVersionVector() { + if (versionVectorBuilder_ == null) { + return versionVector_ == null + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + : versionVector_; + } else { + return versionVectorBuilder_.getMessage(); + } + } + /** required .VersionVector versionVector = 3; */ + public Builder setVersionVector( + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.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.ActiveActive.VersionVector value) { + if (versionVectorBuilder_ == null) { + if (((bitField0_ & 0x00000004) != 0) + && versionVector_ != null + && versionVector_ + != akka.persistence.typed.serialization.ActiveActive.VersionVector + .getDefaultInstance()) { + versionVector_ = + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.VersionVector.Builder + getVersionVectorBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getVersionVectorFieldBuilder().getBuilder(); + } + /** required .VersionVector versionVector = 3; */ + public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + getVersionVectorOrBuilder() { + if (versionVectorBuilder_ != null) { + return versionVectorBuilder_.getMessageOrBuilder(); + } else { + return versionVector_ == null + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + : versionVector_; + } + } + /** required .VersionVector versionVector = 3; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + getVersionVectorFieldBuilder() { + if (versionVectorBuilder_ == null) { + versionVectorBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedEventMetadata + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata(); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.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.ActiveActive.VersionVector getVersion(); + /** required .VersionVector version = 1; */ + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder getVersionOrBuilder(); + + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + java.util.List< + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen> + getSeenPerReplicaList(); + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + .SeenOrBuilder> + getSeenPerReplicaOrBuilderList(); + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.VersionVector.Builder subBuilder = + null; + if (((bitField0_ & 0x00000001) != 0)) { + subBuilder = version_.toBuilder(); + } + version_ = + input.readMessage( + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive + .ReplicatedSnapshotMetadata.Seen>(); + mutable_bitField0_ |= 0x00000002; + } + seenPerReplica_.add( + input.readMessage( + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.class, + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_Seen_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_Seen_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .class, + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata.Seen)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen other = + (akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + .Seen + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + .Seen + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + .Seen + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + .SeenOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_Seen_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_Seen_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .class, + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .Builder.class); + } + + // Construct using + // akka.persistence.typed.serialization.ActiveActive.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.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_Seen_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + build() { + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + buildPartial() { + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen result = + new akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata.Seen) { + return mergeFrom( + (akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen) + other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + other) { + if (other + == akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata.Seen + parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ActiveActive.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.ActiveActive + .ReplicatedSnapshotMetadata.Seen + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen(); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata.Seen + getDefaultInstanceForType() { + return DEFAULT_INSTANCE; + } + } + + private int bitField0_; + public static final int VERSION_FIELD_NUMBER = 1; + private akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.VersionVector getVersion() { + return version_ == null + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + : version_; + } + /** required .VersionVector version = 1; */ + public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + getVersionOrBuilder() { + return version_ == null + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + : version_; + } + + public static final int SEENPERREPLICA_FIELD_NUMBER = 2; + private java.util.List< + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen> + seenPerReplica_; + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public java.util.List< + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen> + getSeenPerReplicaList() { + return seenPerReplica_; + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public java.util.List< + ? extends + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata.Seen + getSeenPerReplica(int index) { + return seenPerReplica_.get(index); + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata)) { + return super.equals(obj); + } + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata other = + (akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + parseFrom(java.nio.ByteBuffer data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + parseFrom(akka.protobufv3.internal.ByteString data) + throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + parseFrom(byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + parseFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseWithIOException(PARSER, input); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { + return akka.protobufv3.internal.GeneratedMessageV3.parseDelimitedWithIOException( + PARSER, input); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadataOrBuilder { + public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_descriptor; + } + + @java.lang.Override + protected akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.persistence.typed.serialization.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_fieldAccessorTable + .ensureFieldAccessorsInitialized( + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.class, + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Builder + .class); + } + + // Construct using + // akka.persistence.typed.serialization.ActiveActive.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.ActiveActive + .internal_static_ReplicatedSnapshotMetadata_descriptor; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + .getDefaultInstance(); + } + + @java.lang.Override + public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata build() { + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata result = + buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + @java.lang.Override + public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + buildPartial() { + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata result = + new akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata) { + return mergeFrom( + (akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata) other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom( + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata other) { + if (other + == akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata parsedMessage = + null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (akka.protobufv3.internal.InvalidProtocolBufferException e) { + parsedMessage = + (akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata) + e.getUnfinishedMessage(); + throw e.unwrapIOException(); + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + private int bitField0_; + + private akka.persistence.typed.serialization.ActiveActive.VersionVector version_; + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.VersionVector getVersion() { + if (versionBuilder_ == null) { + return version_ == null + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + : version_; + } else { + return versionBuilder_.getMessage(); + } + } + /** required .VersionVector version = 1; */ + public Builder setVersion( + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.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.ActiveActive.VersionVector value) { + if (versionBuilder_ == null) { + if (((bitField0_ & 0x00000001) != 0) + && version_ != null + && version_ + != akka.persistence.typed.serialization.ActiveActive.VersionVector + .getDefaultInstance()) { + version_ = + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.VersionVector.Builder + getVersionBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getVersionFieldBuilder().getBuilder(); + } + /** required .VersionVector version = 1; */ + public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + getVersionOrBuilder() { + if (versionBuilder_ != null) { + return versionBuilder_.getMessageOrBuilder(); + } else { + return version_ == null + ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + : version_; + } + } + /** required .VersionVector version = 1; */ + private akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + getVersionFieldBuilder() { + if (versionBuilder_ == null) { + versionBuilder_ = + new akka.protobufv3.internal.SingleFieldBuilderV3< + akka.persistence.typed.serialization.ActiveActive.VersionVector, + akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, + akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder>( + getVersion(), getParentForChildren(), isClean()); + version_ = null; + } + return versionBuilder_; + } + + private java.util.List< + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen> + seenPerReplica_ = java.util.Collections.emptyList(); + + private void ensureSeenPerReplicaIsMutable() { + if (!((bitField0_ & 0x00000002) != 0)) { + seenPerReplica_ = + new java.util.ArrayList< + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + .Seen>(seenPerReplica_); + bitField0_ |= 0x00000002; + } + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen, + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .Builder, + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + .SeenOrBuilder> + seenPerReplicaBuilder_; + + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public java.util.List< + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata.Seen + .Builder + getSeenPerReplicaBuilder(int index) { + return getSeenPerReplicaFieldBuilder().getBuilder(index); + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata.Seen + .Builder + addSeenPerReplicaBuilder() { + return getSeenPerReplicaFieldBuilder() + .addBuilder( + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .getDefaultInstance()); + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .Builder + addSeenPerReplicaBuilder(int index) { + return getSeenPerReplicaFieldBuilder() + .addBuilder( + index, + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .getDefaultInstance()); + } + /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ + public java.util.List< + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .Builder> + getSeenPerReplicaBuilderList() { + return getSeenPerReplicaFieldBuilder().getBuilderList(); + } + + private akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen, + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .Builder, + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + .SeenOrBuilder> + getSeenPerReplicaFieldBuilder() { + if (seenPerReplicaBuilder_ == null) { + seenPerReplicaBuilder_ = + new akka.protobufv3.internal.RepeatedFieldBuilderV3< + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen, + akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + .Builder, + akka.persistence.typed.serialization.ActiveActive.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.ActiveActive + .ReplicatedSnapshotMetadata + DEFAULT_INSTANCE; + + static { + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata(); + } + + public static akka.persistence.typed.serialization.ActiveActive.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.ActiveActive.ReplicatedSnapshotMetadata + getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } @@ -6569,6 +9692,18 @@ public final class Crdts { 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; public static akka.protobufv3.internal.Descriptors.FileDescriptor getDescriptor() { return descriptor; @@ -6578,22 +9713,30 @@ public final class Crdts { static { java.lang.String[] descriptorData = { - "\n\013Crdts.proto\032\026ContainerFormats.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\005ORSet\022\020\n\010originDc\030\001 " - + "\002(\t\022\037\n\007vvector\030\002 \002(\0132\016.VersionVector\022\034\n\004" - + "dots\030\003 \003(\0132\016.VersionVector\022\026\n\016stringElem" - + "ents\030\004 \003(\t\022\027\n\013intElements\030\005 \003(\021B\002\020\001\022\030\n\014l" - + "ongElements\030\006 \003(\022B\002\020\001\022\037\n\rotherElements\030\007" - + " \003(\0132\010.Payload\"\201\001\n\017ORSetDeltaGroup\022\'\n\007en" - + "tries\030\001 \003(\0132\026.ORSetDeltaGroup.Entry\032E\n\005E" - + "ntry\022 \n\toperation\030\001 \002(\0162\r.ORSetDeltaOp\022\032" - + "\n\nunderlying\030\002 \002(\0132\006.ORSet\"]\n\rVersionVec" - + "tor\022%\n\007entries\030\001 \003(\0132\024.VersionVector.Ent" - + "ry\032%\n\005Entry\022\013\n\003key\030\001 \002(\t\022\017\n\007version\030\002 \002(" - + "\003*-\n\014ORSetDeltaOp\022\007\n\003Add\020\000\022\n\n\006Remove\020\001\022\010" - + "\n\004Full\020\002B(\n$akka.persistence.typed.seria" - + "lizationH\001" + "\n\022ActiveActive.proto\032\026ContainerFormats.p" + + "roto\"\030\n\007Counter\022\r\n\005value\030\001 \002(\014\"\036\n\rCounte" + + "rUpdate\022\r\n\005delta\030\001 \002(\014\"\304\001\n\005ORSet\022\020\n\010orig" + + "inDc\030\001 \002(\t\022\037\n\007vvector\030\002 \002(\0132\016.VersionVec" + + "tor\022\034\n\004dots\030\003 \003(\0132\016.VersionVector\022\026\n\016str" + + "ingElements\030\004 \003(\t\022\027\n\013intElements\030\005 \003(\021B\002" + + "\020\001\022\030\n\014longElements\030\006 \003(\022B\002\020\001\022\037\n\rotherEle" + + "ments\030\007 \003(\0132\010.Payload\"\201\001\n\017ORSetDeltaGrou" + + "p\022\'\n\007entries\030\001 \003(\0132\026.ORSetDeltaGroup.Ent" + + "ry\032E\n\005Entry\022 \n\toperation\030\001 \002(\0162\r.ORSetDe" + + "ltaOp\022\032\n\nunderlying\030\002 \002(\0132\006.ORSet\"]\n\rVer" + + "sionVector\022%\n\007entries\030\001 \003(\0132\024.VersionVec" + + "tor.Entry\032%\n\005Entry\022\013\n\003key\030\001 \002(\t\022\017\n\007versi" + + "on\030\002 \002(\003\"\205\001\n\027ReplicatedEventMetadata\022\025\n\r" + + "originReplica\030\001 \002(\t\022\030\n\020originSequenceNr\030" + + "\002 \002(\003\022%\n\rversionVector\030\003 \002(\0132\016.VersionVe" + + "ctor\022\022\n\nconcurrent\030\004 \002(\010\"\246\001\n\032ReplicatedS" + + "napshotMetadata\022\037\n\007version\030\001 \002(\0132\016.Versi" + + "onVector\0228\n\016seenPerReplica\030\002 \003(\0132 .Repli" + + "catedSnapshotMetadata.Seen\032-\n\004Seen\022\021\n\tre" + + "plicaId\030\001 \002(\t\022\022\n\nsequenceNr\030\002 \002(\003*-\n\014ORS" + + "etDeltaOp\022\007\n\003Add\020\000\022\n\n\006Remove\020\001\022\010\n\004Full\020\002" + + "B(\n$akka.persistence.typed.serialization" + + "H\001" }; descriptor = akka.protobufv3.internal.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom( @@ -6658,6 +9801,29 @@ public final class Crdts { 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", + }); akka.remote.ContainerFormats.getDescriptor(); } diff --git a/akka-persistence-typed/src/main/protobuf/Crdts.proto b/akka-persistence-typed/src/main/protobuf/ActiveActive.proto similarity index 71% rename from akka-persistence-typed/src/main/protobuf/Crdts.proto rename to akka-persistence-typed/src/main/protobuf/ActiveActive.proto index cb95464f1c..ab6f433448 100644 --- a/akka-persistence-typed/src/main/protobuf/Crdts.proto +++ b/akka-persistence-typed/src/main/protobuf/ActiveActive.proto @@ -48,3 +48,19 @@ message VersionVector { } 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; +} \ 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 1d3b90a8ec..3e1c20d1b3 100644 --- a/akka-persistence-typed/src/main/resources/reference.conf +++ b/akka-persistence-typed/src/main/resources/reference.conf @@ -1,15 +1,17 @@ akka.actor { - serialization-identifiers."akka.persistence.typed.serialization.CrdtSerializer" = 40 + serialization-identifiers."akka.persistence.typed.serialization.ActiveActiveSerializer" = 40 - serializers.replicated-crdts = "akka.persistence.typed.serialization.CrdtSerializer" + serializers.active-active = "akka.persistence.typed.serialization.ActiveActiveSerializer" serialization-bindings { - "akka.persistence.typed.crdt.Counter" = replicated-crdts - "akka.persistence.typed.crdt.Counter$Updated" = replicated-crdts - "akka.persistence.typed.internal.VersionVector" = replicated-crdts - "akka.persistence.typed.crdt.ORSet" = replicated-crdts - "akka.persistence.typed.crdt.ORSet$DeltaOp" = replicated-crdts + "akka.persistence.typed.internal.VersionVector" = active-active + "akka.persistence.typed.crdt.Counter" = active-active + "akka.persistence.typed.crdt.Counter$Updated" = active-active + "akka.persistence.typed.crdt.ORSet" = active-active + "akka.persistence.typed.crdt.ORSet$DeltaOp" = active-active + "akka.persistence.typed.internal.ReplicatedEventMetadata" = active-active + "akka.persistence.typed.internal.ReplicatedSnapshotMetadata" = active-active } } 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 27e47222e4..521ba48cfb 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 @@ -271,7 +271,16 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( } -// FIXME serializer +object ReplicatedEventMetadata { + + /** + * For a journal supporting active active 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 @@ -279,15 +288,28 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( * at each location as they are received at different times */ @InternalApi -private[akka] final case class ReplicatedEventMetaData( +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 -// FIXME serializer +object ReplicatedSnapshotMetadata { + + /** + * For a snapshot store supporting active active 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]) +private[akka] final case class ReplicatedSnapshotMetadata(version: VersionVector, seenPerReplica: Map[ReplicaId, Long]) /** * An event replicated from a different replica. 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 510099c1ed..04ae7984b2 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 @@ -29,7 +29,7 @@ private[akka] object JournalInteractions { final case class EventToPersist( adaptedEvent: EventOrTaggedOrReplicated, manifest: String, - metadata: Option[ReplicatedEventMetaData]) + metadata: Option[ReplicatedEventMetadata]) } @@ -193,7 +193,7 @@ private[akka] trait SnapshotInteractions[C, E, S] { else { val meta = setup.activeActive match { case Some(_) => - val m = ReplicatedSnapshotMetaData(state.version, state.seenPerReplica) + val m = ReplicatedSnapshotMetadata(state.version, state.seenPerReplica) Some(m) case None => None } 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 aa7dd1c1d1..e33300a277 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 @@ -123,11 +123,11 @@ private[akka] final class ReplayingEvents[C, E, S]( eventForErrorReporting = OptionVal.Some(event) state = state.copy(seqNr = repr.sequenceNr) - val aaMetaAndSelfReplica: Option[(ReplicatedEventMetaData, ReplicaId, ActiveActive)] = + val aaMetaAndSelfReplica: Option[(ReplicatedEventMetadata, ReplicaId, ActiveActive)] = setup.activeActive match { case Some(aa) => val meta = repr.metadata match { - case Some(m) => m.asInstanceOf[ReplicatedEventMetaData] + case Some(m) => m.asInstanceOf[ReplicatedEventMetadata] case None => throw new IllegalStateException( s"Active active enabled but existing event has no metadata. Migration isn't supported yet.") 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 d4c6481dda..1ead8a0af6 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 @@ -152,7 +152,7 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup state = setup.snapshotAdapter.fromJournal(snapshot) setup.context.log.debug("Loaded snapshot with metadata {}", metadata) metadata.metadata match { - case Some(rm: ReplicatedSnapshotMetaData) => (metadata.sequenceNr, rm.seenPerReplica, rm.version) + case Some(rm: ReplicatedSnapshotMetadata) => (metadata.sequenceNr, rm.seenPerReplica, rm.version) case _ => (metadata.sequenceNr, Map.empty.withDefaultValue(0L), VersionVector.empty) } case None => (0L, Map.empty.withDefaultValue(0L), VersionVector.empty) 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 7ec2bf42d0..4eeec5bdc1 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 @@ -144,7 +144,16 @@ private[akka] object Running { .eventsByPersistenceId(pid.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(_.eventMetadata.get.asInstanceOf[ReplicatedEventMetaData].originReplica == replicaId) + .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 active active?") + }) .viaMat(new FastForwardingFilter)(Keep.right) .mapMaterializedValue(streamControl => controlRef.set(streamControl)) } @@ -152,7 +161,7 @@ private[akka] object Running { .via(ActorFlow .ask[EventEnvelope, ReplicatedEventEnvelope[E], ReplicatedEventAck.type](ref) { (eventEnvelope, replyTo) => // Need to handle this not being available migration from non-active-active is supported - val meta = eventEnvelope.eventMetadata.get.asInstanceOf[ReplicatedEventMetaData] + val meta = eventEnvelope.eventMetadata.get.asInstanceOf[ReplicatedEventMetadata] val re = ReplicatedEvent[E]( eventEnvelope.event.asInstanceOf[E], @@ -412,7 +421,7 @@ private[akka] object Running { event.event, "", OptionVal.Some( - ReplicatedEventMetaData(event.originReplica, event.originSequenceNr, updatedVersion, isConcurrent))) + 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) @@ -457,7 +466,7 @@ private[akka] object Running { eventToPersist, eventAdapterManifest, OptionVal.Some( - ReplicatedEventMetaData(aa.replicaId, _currentSequenceNumber, updatedVersion, concurrent = false))) + ReplicatedEventMetadata(aa.replicaId, _currentSequenceNumber, updatedVersion, concurrent = false))) .copy(version = updatedVersion) if (setup.log.isTraceEnabled()) @@ -485,10 +494,10 @@ private[akka] object Running { // also, ensure that there is an event handler for each single event _currentSequenceNumber = state.seqNr - val metadataTemplate: Option[ReplicatedEventMetaData] = setup.activeActive match { + val metadataTemplate: Option[ReplicatedEventMetadata] = setup.activeActive match { case Some(aa) => aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false) // local events are never concurrent - Some(ReplicatedEventMetaData(aa.replicaId, 0L, state.version, concurrent = false)) // we replace it with actual seqnr later + Some(ReplicatedEventMetadata(aa.replicaId, 0L, state.version, concurrent = false)) // we replace it with actual seqnr later case None => None } diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/CrdtSerializer.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala similarity index 63% rename from akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/CrdtSerializer.scala rename to akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala index ab193b6f85..28aa020c4e 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/CrdtSerializer.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala @@ -10,7 +10,10 @@ import java.{ lang => jl } import akka.actor.ExtendedActorSystem import akka.annotation.InternalApi +import akka.persistence.typed.ReplicaId import akka.persistence.typed.crdt.{ Counter, ORSet } +import akka.persistence.typed.internal.ReplicatedEventMetadata +import akka.persistence.typed.internal.ReplicatedSnapshotMetadata import akka.persistence.typed.internal.VersionVector import akka.protobufv3.internal.ByteString import akka.remote.ContainerFormats.Payload @@ -21,7 +24,7 @@ import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.immutable.TreeMap -object CrdtSerializer { +object ActiveActiveSerializer { object Comparator extends Comparator[Payload] { override def compare(a: Payload, b: Payload): Int = { val aByteString = a.getEnclosedMessage @@ -50,7 +53,7 @@ object CrdtSerializer { /** * INTERNAL API */ -@InternalApi private[akka] final class CrdtSerializer(val system: ExtendedActorSystem) +@InternalApi private[akka] final class ActiveActiveSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer { @@ -67,6 +70,9 @@ object CrdtSerializer { private val VersionVectorManifest = "DA" + private val ReplicatedEventMetadataManifest = "RE" + private val ReplicatedSnapshotMetadataManifest = "RS" + def manifest(o: AnyRef) = o match { case _: ORSet[_] => ORSetManifest case _: ORSet.AddDeltaOp[_] => ORSetAddManifest @@ -78,11 +84,19 @@ object CrdtSerializer { case _: Counter.Updated => CrdtCounterUpdatedManifest case _: VersionVector => VersionVectorManifest + + case _: ReplicatedEventMetadata => ReplicatedEventMetadataManifest + case _: ReplicatedSnapshotMetadata => ReplicatedSnapshotMetadataManifest 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 => replicatedEventMetadataToProtoByteArray(m) + 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 @@ -91,12 +105,18 @@ object CrdtSerializer { case m: Counter => counterToProtoByteArray(m) case m: Counter.Updated => counterUpdatedToProtoBufByteArray(m) - case m: VersionVector => versionVectorToProto(m).toByteArray + 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) @@ -106,29 +126,29 @@ object CrdtSerializer { case CrdtCounterManifest => counterFromBinary(bytes) case CrdtCounterUpdatedManifest => counterUpdatedFromBinary(bytes) - case VersionVectorManifest => versionVectorFromBinary(bytes) case _ => throw new NotSerializableException( s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]") } def counterFromBinary(bytes: Array[Byte]): Counter = - Counter(BigInt(Crdts.Counter.parseFrom(bytes).getValue.toByteArray)) + Counter(BigInt(ActiveActive.Counter.parseFrom(bytes).getValue.toByteArray)) def counterUpdatedFromBinary(bytes: Array[Byte]): Counter.Updated = - Counter.Updated(BigInt(Crdts.CounterUpdate.parseFrom(bytes).getDelta.toByteArray)) + Counter.Updated(BigInt(ActiveActive.CounterUpdate.parseFrom(bytes).getDelta.toByteArray)) def counterToProtoByteArray(counter: Counter): Array[Byte] = - Crdts.Counter.newBuilder().setValue(ByteString.copyFrom(counter.value.toByteArray)).build().toByteArray + ActiveActive.Counter.newBuilder().setValue(ByteString.copyFrom(counter.value.toByteArray)).build().toByteArray def counterUpdatedToProtoBufByteArray(updated: Counter.Updated): Array[Byte] = - Crdts.CounterUpdate.newBuilder().setDelta(ByteString.copyFrom(updated.delta.toByteArray)).build().toByteArray + ActiveActive.CounterUpdate.newBuilder().setDelta(ByteString.copyFrom(updated.delta.toByteArray)).build().toByteArray - def orsetToProto(orset: ORSet[_]): Crdts.ORSet = + def orsetToProto(orset: ORSet[_]): ActiveActive.ORSet = orsetToProtoImpl(orset.asInstanceOf[ORSet[Any]]) - private def orsetToProtoImpl(orset: ORSet[Any]): Crdts.ORSet = { - val b = Crdts.ORSet.newBuilder().setOriginDc(orset.originReplica).setVvector(versionVectorToProto(orset.vvector)) + private def orsetToProtoImpl(orset: ORSet[Any]): ActiveActive.ORSet = { + val b = + ActiveActive.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] @@ -174,7 +194,7 @@ object CrdtSerializer { addDots(longElements) } if (!otherElements.isEmpty) { - Collections.sort(otherElements, CrdtSerializer.Comparator) + Collections.sort(otherElements, ActiveActiveSerializer.Comparator) b.addAllOtherElements(otherElements) addDots(otherElements) } @@ -182,31 +202,55 @@ object CrdtSerializer { b.build() } + def replicatedEventMetadataToProtoByteArray(rem: ReplicatedEventMetadata): Array[Byte] = { + ActiveActive.ReplicatedEventMetadata + .newBuilder() + .setOriginSequenceNr(rem.originSequenceNr) + .setConcurrent(rem.concurrent) + .setOriginReplica(rem.originReplica.id) + .setVersionVector(versionVectorToProto(rem.version)) + .build() + .toByteArray + } + + def replicatedSnapshotMetadataToByteArray(rsm: ReplicatedSnapshotMetadata): Array[Byte] = { + ActiveActive.ReplicatedSnapshotMetadata + .newBuilder() + .setVersion(versionVectorToProto(rsm.version)) + .addAllSeenPerReplica(rsm.seenPerReplica.map(seenToProto).asJava) + .build() + .toByteArray + } + + def seenToProto(t: (ReplicaId, Long)): ActiveActive.ReplicatedSnapshotMetadata.Seen = { + ActiveActive.ReplicatedSnapshotMetadata.Seen.newBuilder().setReplicaId(t._1.id).setSequenceNr(t._2).build() + } + def orsetFromBinary(bytes: Array[Byte]): ORSet[Any] = - orsetFromProto(Crdts.ORSet.parseFrom(bytes)) + orsetFromProto(ActiveActive.ORSet.parseFrom(bytes)) private def orsetAddFromBinary(bytes: Array[Byte]): ORSet.AddDeltaOp[Any] = - new ORSet.AddDeltaOp(orsetFromProto(Crdts.ORSet.parseFrom(bytes))) + new ORSet.AddDeltaOp(orsetFromProto(ActiveActive.ORSet.parseFrom(bytes))) private def orsetRemoveFromBinary(bytes: Array[Byte]): ORSet.RemoveDeltaOp[Any] = - new ORSet.RemoveDeltaOp(orsetFromProto(Crdts.ORSet.parseFrom(bytes))) + new ORSet.RemoveDeltaOp(orsetFromProto(ActiveActive.ORSet.parseFrom(bytes))) private def orsetFullFromBinary(bytes: Array[Byte]): ORSet.FullStateDeltaOp[Any] = - new ORSet.FullStateDeltaOp(orsetFromProto(Crdts.ORSet.parseFrom(bytes))) + new ORSet.FullStateDeltaOp(orsetFromProto(ActiveActive.ORSet.parseFrom(bytes))) - private def orsetDeltaGroupToProto(deltaGroup: ORSet.DeltaGroup[_]): Crdts.ORSetDeltaGroup = { - def createEntry(opType: Crdts.ORSetDeltaOp, u: ORSet[_]) = { - Crdts.ORSetDeltaGroup.Entry.newBuilder().setOperation(opType).setUnderlying(orsetToProto(u)) + private def orsetDeltaGroupToProto(deltaGroup: ORSet.DeltaGroup[_]): ActiveActive.ORSetDeltaGroup = { + def createEntry(opType: ActiveActive.ORSetDeltaOp, u: ORSet[_]) = { + ActiveActive.ORSetDeltaGroup.Entry.newBuilder().setOperation(opType).setUnderlying(orsetToProto(u)) } - val b = Crdts.ORSetDeltaGroup.newBuilder() + val b = ActiveActive.ORSetDeltaGroup.newBuilder() deltaGroup.ops.foreach { case ORSet.AddDeltaOp(u) => - b.addEntries(createEntry(Crdts.ORSetDeltaOp.Add, u)) + b.addEntries(createEntry(ActiveActive.ORSetDeltaOp.Add, u)) case ORSet.RemoveDeltaOp(u) => - b.addEntries(createEntry(Crdts.ORSetDeltaOp.Remove, u)) + b.addEntries(createEntry(ActiveActive.ORSetDeltaOp.Remove, u)) case ORSet.FullStateDeltaOp(u) => - b.addEntries(createEntry(Crdts.ORSetDeltaOp.Full, u)) + b.addEntries(createEntry(ActiveActive.ORSetDeltaOp.Full, u)) case ORSet.DeltaGroup(_) => throw new IllegalArgumentException("ORSet.DeltaGroup should not be nested") } @@ -214,14 +258,14 @@ object CrdtSerializer { } private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = { - val deltaGroup = Crdts.ORSetDeltaGroup.parseFrom(bytes) + val deltaGroup = ActiveActive.ORSetDeltaGroup.parseFrom(bytes) val ops: Vector[ORSet.DeltaOp] = deltaGroup.getEntriesList.asScala.map { entry => - if (entry.getOperation == Crdts.ORSetDeltaOp.Add) + if (entry.getOperation == ActiveActive.ORSetDeltaOp.Add) ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)) - else if (entry.getOperation == Crdts.ORSetDeltaOp.Remove) + else if (entry.getOperation == ActiveActive.ORSetDeltaOp.Remove) ORSet.RemoveDeltaOp(orsetFromProto(entry.getUnderlying)) - else if (entry.getOperation == Crdts.ORSetDeltaOp.Full) + else if (entry.getOperation == ActiveActive.ORSetDeltaOp.Full) ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying)) else throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}") @@ -229,7 +273,7 @@ object CrdtSerializer { ORSet.DeltaGroup(ops) } - def orsetFromProto(orset: Crdts.ORSet): ORSet[Any] = { + def orsetFromProto(orset: ActiveActive.ORSet): ORSet[Any] = { val elements: Iterator[Any] = (orset.getStringElementsList.iterator.asScala ++ orset.getIntElementsList.iterator.asScala ++ @@ -242,18 +286,18 @@ object CrdtSerializer { new ORSet(orset.getOriginDc, elementsMap, vvector = versionVectorFromProto(orset.getVvector)) } - def versionVectorToProto(versionVector: VersionVector): Crdts.VersionVector = { - val b = Crdts.VersionVector.newBuilder() + def versionVectorToProto(versionVector: VersionVector): ActiveActive.VersionVector = { + val b = ActiveActive.VersionVector.newBuilder() versionVector.versionsIterator.foreach { - case (key, value) => b.addEntries(Crdts.VersionVector.Entry.newBuilder().setKey(key).setVersion(value)) + case (key, value) => b.addEntries(ActiveActive.VersionVector.Entry.newBuilder().setKey(key).setVersion(value)) } b.build() } def versionVectorFromBinary(bytes: Array[Byte]): VersionVector = - versionVectorFromProto(Crdts.VersionVector.parseFrom(bytes)) + versionVectorFromProto(ActiveActive.VersionVector.parseFrom(bytes)) - def versionVectorFromProto(versionVector: Crdts.VersionVector): VersionVector = { + def versionVectorFromProto(versionVector: ActiveActive.VersionVector): VersionVector = { val entries = versionVector.getEntriesList if (entries.isEmpty) VersionVector.empty @@ -266,4 +310,20 @@ object CrdtSerializer { } } + def replicatedEventMetadataFromBinary(bytes: Array[Byte]): ReplicatedEventMetadata = { + val parsed = ActiveActive.ReplicatedEventMetadata.parseFrom(bytes) + ReplicatedEventMetadata( + ReplicaId(parsed.getOriginReplica), + parsed.getOriginSequenceNr, + versionVectorFromProto(parsed.getVersionVector), + parsed.getConcurrent) + } + + def replicatedSnapshotMetadataFromBinary(bytes: Array[Byte]): ReplicatedSnapshotMetadata = { + val parsed = ActiveActive.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/scala/akka/persistence/typed/ActiveActiveSerializationSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ActiveActiveSerializationSpec.scala new file mode 100644 index 0000000000..84b4fb2991 --- /dev/null +++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ActiveActiveSerializationSpec.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 ActiveActiveSerializationSpec + extends ScalaTestWithActorTestKit(ClusterSingletonPersistenceSpec.config) + with AnyWordSpecLike + with LogCapturing { + + "The ActiveActive 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/src/main/scala/akka/persistence/SnapshotProtocol.scala b/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala index c737aa21f8..75b9c38480 100644 --- a/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala +++ b/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala @@ -30,6 +30,9 @@ final class SnapshotMetadata( 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, 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.*")) From 0b11ae362ceb0c83ecfa9efd1aa7ef47ec81e83e Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Wed, 29 Jul 2020 07:31:17 +0100 Subject: [PATCH 25/50] PersistentRepr serializer support for metadata (#29434) And fixing the warnings for the 2.13 build, not sure why PR validation passes for these The write side of the JDBC plugin works now as it persists the PR --- ...ctiveActiveShardingDirectReplication.scala | 2 +- .../typed/ActiveActiveShardingSettings.scala | 2 +- .../ActiveActiveShardingExtensionImpl.scala | 2 +- .../persistence/journal/JournalSpec.scala | 8 + .../akka/persistence/typed/crdt/ORSet.scala | 6 +- .../typed/internal/ReplayingSnapshot.scala | 8 +- .../javadsl/ActiveActiveEventSourcing.scala | 2 +- .../scaladsl/ActiveActiveEventSourcing.scala | 2 +- .../ActiveActiveSerializer.scala | 2 +- .../serialization/MessageFormats.java | 253 ++++++++++++++++-- .../src/main/protobuf/MessageFormats.proto | 1 + .../scala/akka/persistence/Persistent.scala | 2 +- .../serialization/MessageSerializer.scala | 11 +- .../serialization/MessageSerializerSpec.scala | 22 ++ 14 files changed, 289 insertions(+), 34 deletions(-) create mode 100644 akka-persistence/src/test/scala/akka/persistence/serialization/MessageSerializerSpec.scala diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala index 5e1ba61e80..7073984f3e 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala @@ -15,7 +15,7 @@ import akka.annotation.InternalApi import akka.persistence.typed.PublishedEvent import akka.persistence.typed.ReplicaId -import scala.collection.JavaConverters._ +import akka.util.ccompat.JavaConverters._ /** * Used when sharding Active Active entities in multiple instances of sharding, for example one per DC in a Multi DC diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala index 09ae1fcb8c..d74d88aaf3 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala @@ -11,7 +11,7 @@ import akka.persistence.typed.ReplicaId import scala.collection.immutable import scala.reflect.ClassTag -import scala.collection.JavaConverters._ +import akka.util.ccompat.JavaConverters._ import java.util.{ Set => JSet } import akka.annotation.ApiMayChange diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala index 2d8eea6155..9c57b9f960 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala @@ -22,7 +22,7 @@ import org.slf4j.LoggerFactory import akka.actor.typed.scaladsl.LoggerOps import akka.cluster.sharding.typed.ActiveActiveShardingDirectReplication -import scala.collection.JavaConverters._ +import akka.util.ccompat.JavaConverters._ /** * INTERNAL API 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 3380126b73..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 @@ -339,6 +339,14 @@ abstract class JournalSpec(config: Config) _) => 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-typed/src/main/scala/akka/persistence/typed/crdt/ORSet.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/crdt/ORSet.scala index 099e9ed358..f881e1d433 100644 --- 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 @@ -295,7 +295,7 @@ final class ORSet[A] private[akka] ( * Java API */ def getElements(): java.util.Set[A] = { - import scala.collection.JavaConverters._ + import akka.util.ccompat.JavaConverters._ elements.asJava } @@ -324,7 +324,7 @@ final class ORSet[A] private[akka] ( * `elems` must not be empty. */ def addAll(elems: java.util.Set[A]): ORSet.DeltaOp = { - import scala.collection.JavaConverters._ + import akka.util.ccompat.JavaConverters._ addAll(elems.asScala.toSet) } @@ -366,7 +366,7 @@ final class ORSet[A] private[akka] ( * `elems` must not be empty. */ def removeAll(elems: java.util.Set[A]): ORSet.DeltaOp = { - import scala.collection.JavaConverters._ + import akka.util.ccompat.JavaConverters._ removeAll(elems.asScala.toSet) } 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 1ead8a0af6..5b8d538fab 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 @@ -147,15 +147,15 @@ private[akka] class ReplayingSnapshot[C, E, S](override val setup: BehaviorSetup case LoadSnapshotResult(sso, toSnr) => var state: S = setup.emptyState - val (seqNr: Long, seenPerReplica: Map[ReplicaId, Long], version: VersionVector) = sso match { + val (seqNr: Long, seenPerReplica, version) = sso match { case Some(SelectedSnapshot(metadata, snapshot)) => state = setup.snapshotAdapter.fromJournal(snapshot) - setup.context.log.debug("Loaded snapshot with metadata {}", metadata) + 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.withDefaultValue(0L), VersionVector.empty) + case _ => (metadata.sequenceNr, Map.empty[ReplicaId, Long].withDefaultValue(0L), VersionVector.empty) } - case None => (0L, Map.empty.withDefaultValue(0L), VersionVector.empty) + case None => (0L, Map.empty[ReplicaId, Long].withDefaultValue(0L), VersionVector.empty) } setup.context.log.debugN("Snapshot recovered from {} {} {}", seqNr, seenPerReplica, version) diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala index 67761d410a..d59b812bc6 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala @@ -13,7 +13,7 @@ import akka.persistence.typed.PersistenceId import akka.persistence.typed.ReplicaId import akka.persistence.typed.scaladsl.ActiveActiveContextImpl -import scala.collection.JavaConverters._ +import akka.util.ccompat.JavaConverters._ /** * Provides access to Active Active specific state diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala index eec1a91976..7570fa3b31 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala @@ -8,7 +8,7 @@ import akka.persistence.typed.PersistenceId import akka.persistence.typed.ReplicaId import akka.util.{ OptionVal, WallClock } -import scala.collection.JavaConverters._ +import akka.util.ccompat.JavaConverters._ // FIXME docs trait ActiveActiveContext { diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala index 28aa020c4e..51633d6eff 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala @@ -21,7 +21,7 @@ import akka.remote.serialization.WrappedPayloadSupport import akka.serialization.{ BaseSerializer, SerializerWithStringManifest } import scala.annotation.tailrec -import scala.collection.JavaConverters._ +import akka.util.ccompat.JavaConverters._ import scala.collection.immutable.TreeMap object ActiveActiveSerializer { 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/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 b7285058f9..159b3ae5d3 100644 --- a/akka-persistence/src/main/scala/akka/persistence/Persistent.scala +++ b/akka-persistence/src/main/scala/akka/persistence/Persistent.scala @@ -243,6 +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/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/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") + } + } + +} From b03412d5b2c49acaa4c10074bb4365ffb6640061 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 29 Jul 2020 12:12:48 +0200 Subject: [PATCH 26/50] Rename active active to replicated event sourcing (#29435) * Move active active internals into the internal package * Renaming active active to Replicated Event Sourcing * Rename of Active Active to Replicated Event Sourcing * Revert changes to testkit * Java test formatting... * Gave up on getting apidoc to link to EventSourcedBehavior and made code snippets of it Co-authored-by: Christopher Batey --- ...cala => ReplicatedShardingExtension.scala} | 26 +- ...scala => ReplicatedShardingSettings.scala} | 32 +- ....scala => ShardingDirectReplication.scala} | 10 +- ... => ReplicatedShardingExtensionImpl.scala} | 27 +- ...gTest.java => ReplicatedShardingTest.java} | 56 +- ...icatedShardingDirectReplicationSpec.scala} | 11 +- ...pec.scala => ReplicatedShardingSpec.scala} | 36 +- .../src/main/paradox/project/examples.md | 6 +- .../src/main/paradox/typed/cluster-dc.md | 4 +- .../main/paradox/typed/index-persistence.md | 4 +- ...d => replicated-eventsourcing-examples.md} | 22 +- ...-active.md => replicated-eventsourcing.md} | 80 +- .../akka/persistence/CapabilityFlags.scala | 2 +- ....java => ReplicatedEventSourcingTest.java} | 17 +- ...plicatedEventSourcingCompileOnlyTest.java} | 18 +- ...cala => MultiJournalReplicationSpec.scala} | 14 +- ...la => ReplicatedEventPublishingSpec.scala} | 76 +- ...cala => ReplicatedEventSourcingSpec.scala} | 14 +- ...seSpec.scala => ReplicationBaseSpec.scala} | 4 +- ...ala => ReplicationIllegalAccessSpec.scala} | 22 +- ...ec.scala => ReplicationSnapshotSpec.scala} | 16 +- .../persistence/typed/crdt/CounterSpec.scala | 14 +- .../akka/persistence/typed/crdt/LwwSpec.scala | 14 +- .../persistence/typed/crdt/ORSetSpec.scala | 14 +- ...ala => ReplicatedAuctionExampleSpec.scala} | 16 +- ....scala => ReplicatedBlogExampleSpec.scala} | 16 +- ...licatedEventSourcingCompileOnlySpec.scala} | 8 +- ...tive.java => ReplicatedEventSourcing.java} | 2164 ++++++++++------- ... 29217-replicated-event-sourcing.excludes} | 2 +- ...ve.proto => ReplicatedEventSourcing.proto} | 0 .../src/main/resources/reference.conf | 4 +- .../persistence/typed/PublishedEvent.scala | 4 +- .../akka/persistence/typed/ReplicaId.scala | 2 +- .../typed/internal/BehaviorSetup.scala | 5 +- .../internal/EventSourcedBehaviorImpl.scala | 15 +- .../typed/internal/ExternalInteractions.scala | 2 +- .../typed/internal/ReplayingEvents.scala | 9 +- .../typed/internal/ReplicationSetup.scala | 111 + .../persistence/typed/internal/Running.scala | 66 +- .../typed/javadsl/EventSourcedBehavior.scala | 2 +- ...a => ReplicatedEventSourcedBehavior.scala} | 17 +- ...ng.scala => ReplicatedEventSourcing.scala} | 63 +- .../typed/scaladsl/EventSourcedBehavior.scala | 44 +- ...ng.scala => ReplicatedEventSourcing.scala} | 119 +- ...> ReplicatedEventSourcingSerializer.scala} | 94 +- ...catedEventSourcingSerializationSpec.scala} | 4 +- .../EventSourcedBehaviorWatchSpec.scala | 2 +- .../akka/persistence/SnapshotProtocol.scala | 2 +- 48 files changed, 1871 insertions(+), 1439 deletions(-) rename akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/{ActiveActiveShardingExtension.scala => ReplicatedShardingExtension.scala} (68%) rename akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/{ActiveActiveShardingSettings.scala => ReplicatedShardingSettings.scala} (74%) rename akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/{ActiveActiveShardingDirectReplication.scala => ShardingDirectReplication.scala} (89%) rename akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/{ActiveActiveShardingExtensionImpl.scala => ReplicatedShardingExtensionImpl.scala} (70%) rename akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/{ActiveActiveShardingTest.java => ReplicatedShardingTest.java} (79%) rename akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/{ActiveActiveShardingDirectReplicationSpec.scala => ReplicatedShardingDirectReplicationSpec.scala} (85%) rename akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/{ActiveActiveShardingSpec.scala => ReplicatedShardingSpec.scala} (76%) rename akka-docs/src/main/paradox/typed/{persistence-active-active-examples.md => replicated-eventsourcing-examples.md} (82%) rename akka-docs/src/main/paradox/typed/{persistence-active-active.md => replicated-eventsourcing.md} (69%) rename akka-persistence-typed-tests/src/test/java/akka/persistence/typed/{ActiveActiveTest.java => ReplicatedEventSourcingTest.java} (91%) rename akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/{ActiveActiveCompileOnlyTest.java => ReplicatedEventSourcingCompileOnlyTest.java} (74%) rename akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/{MultiJournalActiveActiveSpec.scala => MultiJournalReplicationSpec.scala} (91%) rename akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/{ActiveActiveEventPublishingSpec.scala => ReplicatedEventPublishingSpec.scala} (75%) rename akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/{ActiveActiveSpec.scala => ReplicatedEventSourcingSpec.scala} (97%) rename akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/{ActiveActiveBaseSpec.scala => ReplicationBaseSpec.scala} (92%) rename akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/{ActiveActiveIllegalAccessSpec.scala => ReplicationIllegalAccessSpec.scala} (82%) rename akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/{ActiveActiveSnapshotSpec.scala => ReplicationSnapshotSpec.scala} (92%) rename akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/{AAAuctionExampleSpec.scala => ReplicatedAuctionExampleSpec.scala} (95%) rename akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/{AABlogExampleSpec.scala => ReplicatedBlogExampleSpec.scala} (92%) rename akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/{ActiveActiveCompileOnlySpec.scala => ReplicatedEventSourcingCompileOnlySpec.scala} (64%) rename akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/{ActiveActive.java => ReplicatedEventSourcing.java} (77%) rename akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/{29217-active-active-event-sourcing.excludes => 29217-replicated-event-sourcing.excludes} (83%) rename akka-persistence-typed/src/main/protobuf/{ActiveActive.proto => ReplicatedEventSourcing.proto} (100%) create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplicationSetup.scala rename akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/{ActiveActiveEventSourcedBehavior.scala => ReplicatedEventSourcedBehavior.scala} (79%) rename akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/{ActiveActiveEventSourcing.scala => ReplicatedEventSourcing.scala} (71%) rename akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/{ActiveActiveEventSourcing.scala => ReplicatedEventSourcing.scala} (54%) rename akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/{ActiveActiveSerializer.scala => ReplicatedEventSourcingSerializer.scala} (75%) rename akka-persistence-typed/src/test/scala/akka/persistence/typed/{ActiveActiveSerializationSpec.scala => ReplicatedEventSourcingSerializationSpec.scala} (91%) diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingExtension.scala similarity index 68% rename from akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala rename to akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingExtension.scala index 163dabe3e6..d650106857 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingExtension.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingExtension.scala @@ -9,7 +9,7 @@ import akka.actor.typed.Extension import akka.actor.typed.ExtensionId import akka.annotation.ApiMayChange import akka.annotation.DoNotInherit -import akka.cluster.sharding.typed.internal.ActiveActiveShardingExtensionImpl +import akka.cluster.sharding.typed.internal.ReplicatedShardingExtensionImpl import akka.cluster.sharding.typed.scaladsl.EntityRef import akka.persistence.typed.ReplicaId import java.util.{ Map => JMap } @@ -17,16 +17,16 @@ import java.util.{ Map => JMap } import akka.actor.typed.ActorRef /** - * Extension for running active active in sharding by starting one separate instance of sharding per replica. + * 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 ActiveActiveShardingExtension extends ExtensionId[ActiveActiveShardingExtension] { +object ReplicatedShardingExtension extends ExtensionId[ReplicatedShardingExtension] { - override def createExtension(system: ActorSystem[_]): ActiveActiveShardingExtension = - new ActiveActiveShardingExtensionImpl(system) + override def createExtension(system: ActorSystem[_]): ReplicatedShardingExtension = + new ReplicatedShardingExtensionImpl(system) - def get(system: ActorSystem[_]): ActiveActiveShardingExtension = apply(system) + def get(system: ActorSystem[_]): ReplicatedShardingExtension = apply(system) } @@ -35,27 +35,27 @@ object ActiveActiveShardingExtension extends ExtensionId[ActiveActiveShardingExt */ @DoNotInherit @ApiMayChange -trait ActiveActiveShardingExtension extends Extension { +trait ReplicatedShardingExtension extends Extension { /** - * Init one instance sharding per replica in the given settings and return a [[ActiveActiveSharding]] representing those. + * Init one instance sharding per replica in the given settings and return a [[ReplicatedSharding]] representing those. * - * @tparam M The type of messages the active active event sourced actor accepts + * @tparam M The type of messages the replicated event sourced actor accepts * @tparam E The type of envelope used for routing messages to actors, the same for all replicas * - * Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ActiveActiveSharding]] + * Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ReplicatedSharding]] */ - def init[M, E](settings: ActiveActiveShardingSettings[M, E]): ActiveActiveSharding[M, E] + def init[M, E](settings: ReplicatedShardingSettings[M, E]): ReplicatedSharding[M, E] } /** - * Represents the sharding instances for the replicas of one active active entity type + * Represents the sharding instances for the replicas of one replicated event sourcing entity type * * Not for user extension. */ @DoNotInherit @ApiMayChange -trait ActiveActiveSharding[M, E] { +trait ReplicatedSharding[M, E] { /** * Scala API: Returns the actor refs for the shard region or proxies of sharding for each replica for user defined diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala similarity index 74% rename from akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala rename to akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala index d74d88aaf3..799e243960 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingSettings.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala @@ -18,12 +18,12 @@ import akka.annotation.ApiMayChange import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl @ApiMayChange -object ActiveActiveShardingSettings { +object ReplicatedShardingSettings { /** * Java API: * - * @tparam M The type of messages the active active entity accepts + * @tparam M The type of messages the replicated entity accepts * @tparam E The type for envelopes used for sending `M`s over sharding */ def create[M, E]( @@ -33,7 +33,7 @@ object ActiveActiveShardingSettings { JEntityTypeKey[M], ReplicaId, JSet[ReplicaId], - ReplicaSettings[M, E]]): ActiveActiveShardingSettings[M, E] = { + ReplicaSettings[M, E]]): ReplicatedShardingSettings[M, E] = { implicit val classTag: ClassTag[M] = ClassTag(messageClass) apply[M, E](allReplicaIds.asScala.toSet)((key, replica, _) => settingsPerReplicaFactory(key.asInstanceOf[EntityTypeKeyImpl[M]], replica, allReplicaIds)) @@ -42,13 +42,13 @@ object ActiveActiveShardingSettings { /** * Scala API: * - * @tparam M The type of messages the active active entity accepts + * @tparam M The type of messages the replicated entity accepts * @tparam E The type for envelopes used for sending `M`s over sharding */ def apply[M: ClassTag, E](allReplicaIds: Set[ReplicaId])( settingsPerReplicaFactory: (EntityTypeKey[M], ReplicaId, Set[ReplicaId]) => ReplicaSettings[M, E]) - : ActiveActiveShardingSettings[M, E] = { - new ActiveActiveShardingSettings(allReplicaIds.map { replicaId => + : ReplicatedShardingSettings[M, E] = { + new ReplicatedShardingSettings(allReplicaIds.map { replicaId => val typeKey = EntityTypeKey[M](replicaId.id) settingsPerReplicaFactory(typeKey, replicaId, allReplicaIds) }.toVector, directReplication = false) @@ -56,23 +56,23 @@ object ActiveActiveShardingSettings { } /** - * @tparam M The type of messages the active active entity accepts + * @tparam M The type of messages the replicated entity accepts * @tparam E The type for envelopes used for sending `M`s over sharding */ @ApiMayChange -final class ActiveActiveShardingSettings[M, E] private ( +final class ReplicatedShardingSettings[M, E] private ( val replicas: immutable.Seq[ReplicaSettings[M, E]], val directReplication: Boolean) { /** - * Start direct replication over sharding when active active sharding starts up, requires the entities + * 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.ActiveActiveEventSourcedBehavior#withEventPublishing()]] + * or [[akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior#withEventPublishing()]] * to work. - + * */ - def withDirectReplication(): ActiveActiveShardingSettings[M, E] = - new ActiveActiveShardingSettings(replicas, directReplication = true) + def withDirectReplication(): ReplicatedShardingSettings[M, E] = + new ReplicatedShardingSettings(replicas, directReplication = true) } @@ -81,7 +81,7 @@ object ReplicaSettings { /** * 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.ActiveActiveEventSourcedBehavior]] or an arbitrary non persistent + * 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. */ @@ -90,7 +90,7 @@ object ReplicaSettings { /** * 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.ActiveActiveEventSourcing]] or an arbitrary non persistent + * 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. */ @@ -99,7 +99,7 @@ object ReplicaSettings { } /** - * Settings for a specific replica id in active active sharding + * Settings for a specific replica id in replicated sharding */ @ApiMayChange final class ReplicaSettings[M, E] private (val replicaId: ReplicaId, val entity: Entity[M, E]) diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingDirectReplication.scala similarity index 89% rename from akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala rename to akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingDirectReplication.scala index 7073984f3e..f667348a21 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplication.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ShardingDirectReplication.scala @@ -18,13 +18,13 @@ import akka.persistence.typed.ReplicaId import akka.util.ccompat.JavaConverters._ /** - * Used when sharding Active Active entities in multiple instances of sharding, for example one per DC in a Multi DC + * 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 Active Active entities will run (the same nodes that you start + * 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.ActiveActiveEventSourcedBehavior#withEventPublishing()]] - * If using [[ActiveActiveSharding]] the replication can be enabled through [[ActiveActiveShardingSettings#withDirectReplication()]] + * or [[akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior#withEventPublishing()]] + * If using [[ReplicatedSharding]] the replication can be enabled through [[ReplicatedShardingSettings#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 @@ -36,7 +36,7 @@ import akka.util.ccompat.JavaConverters._ * by default and with a custom extractor since the envelopes are handled internally. */ @ApiMayChange -object ActiveActiveShardingDirectReplication { +object ShardingDirectReplication { /** * Not for user extension diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ReplicatedShardingExtensionImpl.scala similarity index 70% rename from akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala rename to akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ReplicatedShardingExtensionImpl.scala index 9c57b9f960..1447c9e559 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ActiveActiveShardingExtensionImpl.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/internal/ReplicatedShardingExtensionImpl.scala @@ -10,9 +10,9 @@ import java.util.{ Map => JMap } import akka.actor.typed.ActorRef import akka.actor.typed.ActorSystem import akka.annotation.InternalApi -import akka.cluster.sharding.typed.ActiveActiveShardingExtension -import akka.cluster.sharding.typed.ActiveActiveSharding -import akka.cluster.sharding.typed.ActiveActiveShardingSettings +import akka.cluster.sharding.typed.ReplicatedShardingExtension +import akka.cluster.sharding.typed.ReplicatedSharding +import akka.cluster.sharding.typed.ReplicatedShardingSettings import akka.cluster.sharding.typed.scaladsl.ClusterSharding import akka.cluster.sharding.typed.scaladsl.EntityRef import akka.cluster.sharding.typed.scaladsl.EntityTypeKey @@ -20,7 +20,7 @@ import akka.persistence.typed.PersistenceId import akka.persistence.typed.ReplicaId import org.slf4j.LoggerFactory import akka.actor.typed.scaladsl.LoggerOps -import akka.cluster.sharding.typed.ActiveActiveShardingDirectReplication +import akka.cluster.sharding.typed.ShardingDirectReplication import akka.util.ccompat.JavaConverters._ @@ -28,19 +28,18 @@ import akka.util.ccompat.JavaConverters._ * INTERNAL API */ @InternalApi -private[akka] final class ActiveActiveShardingExtensionImpl(system: ActorSystem[_]) - extends ActiveActiveShardingExtension { +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, E](settings: ActiveActiveShardingSettings[M, E]): ActiveActiveSharding[M, E] = { + override def init[M, E](settings: ReplicatedShardingSettings[M, E]): ReplicatedSharding[M, E] = { val sharding = ClusterSharding(system) val initializedReplicas = settings.replicas.map { replicaSettings => // start up a sharding instance per replica id logger.infoN( - "Starting Active Active sharding for replica [{}] (ShardType: [{}])", + "Starting Replicated Event Sourcing sharding for replica [{}] (ShardType: [{}])", replicaSettings.replicaId.id, replicaSettings.entity.typeKey.name) val regionOrProxy = sharding.init(replicaSettings.entity) @@ -50,14 +49,14 @@ private[akka] final class ActiveActiveShardingExtensionImpl(system: ActorSystem[ case (id, _, regionOrProxy) => id -> regionOrProxy }.toMap if (settings.directReplication) { - logger.infoN("Starting Active Active Direct Replication") + logger.infoN("Starting Replicated Event Sourcing Direct Replication") system.systemActorOf( - ActiveActiveShardingDirectReplication(replicaToRegionOrProxy), - s"activeActiveDirectReplication-${counter.incrementAndGet()}") + ShardingDirectReplication(replicaToRegionOrProxy), + s"directReplication-${counter.incrementAndGet()}") } val replicaToTypeKey = initializedReplicas.map { case (id, typeKey, _) => id -> typeKey }.toMap - new ActiveActiveShardingImpl(sharding, replicaToRegionOrProxy, replicaToTypeKey) + new ReplicatedShardingImpl(sharding, replicaToRegionOrProxy, replicaToTypeKey) } } @@ -65,11 +64,11 @@ private[akka] final class ActiveActiveShardingExtensionImpl(system: ActorSystem[ * INTERNAL API */ @InternalApi -private[akka] final class ActiveActiveShardingImpl[M, E]( +private[akka] final class ReplicatedShardingImpl[M, E]( sharding: ClusterSharding, shardingPerReplica: Map[ReplicaId, ActorRef[E]], replicaTypeKeys: Map[ReplicaId, EntityTypeKey[M]]) - extends ActiveActiveSharding[M, E] { + extends ReplicatedSharding[M, E] { override def shardingRefs: Map[ReplicaId, ActorRef[E]] = shardingPerReplica override def getShardingRefs: JMap[ReplicaId, ActorRef[E]] = shardingRefs.asJava diff --git a/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java b/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ReplicatedShardingTest.java similarity index 79% rename from akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java rename to akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ReplicatedShardingTest.java index 490f4c2606..d9cd4dcab9 100644 --- a/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java +++ b/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ReplicatedShardingTest.java @@ -27,18 +27,16 @@ import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.scalatestplus.junit.JUnitSuite; -import scala.util.Random; import java.util.*; import java.util.stream.Collectors; import static org.junit.Assert.assertEquals; -public class ActiveActiveShardingTest extends JUnitSuite { +public class ReplicatedShardingTest extends JUnitSuite { - static class MyActiveActiveStringSet - extends ActiveActiveEventSourcedBehavior< - MyActiveActiveStringSet.Command, String, Set> { + static class MyReplicatedStringSet + extends ReplicatedEventSourcedBehavior> { interface Command {} static class Add implements Command { @@ -67,16 +65,16 @@ public class ActiveActiveShardingTest extends JUnitSuite { static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ActiveActiveEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.withSharedJournal( entityId, replicaId, allReplicas, PersistenceTestKitReadJournal.Identifier(), - MyActiveActiveStringSet::new); + MyReplicatedStringSet::new); } - private MyActiveActiveStringSet(ActiveActiveContext activeActiveContext) { - super(activeActiveContext); + private MyReplicatedStringSet(ReplicationContext replicationContext) { + super(replicationContext); } @Override @@ -116,9 +114,9 @@ public class ActiveActiveShardingTest extends JUnitSuite { public static final class ForwardToRandom implements Command { public final String entityId; - public final MyActiveActiveStringSet.Command message; + public final MyReplicatedStringSet.Command message; - public ForwardToRandom(String entityId, MyActiveActiveStringSet.Command message) { + public ForwardToRandom(String entityId, MyReplicatedStringSet.Command message) { this.entityId = entityId; this.message = message; } @@ -126,9 +124,9 @@ public class ActiveActiveShardingTest extends JUnitSuite { public static final class ForwardToAll implements Command { public final String entityId; - public final MyActiveActiveStringSet.Command message; + public final MyReplicatedStringSet.Command message; - public ForwardToAll(String entityId, MyActiveActiveStringSet.Command message) { + public ForwardToAll(String entityId, MyReplicatedStringSet.Command message) { this.entityId = entityId; this.message = message; } @@ -144,19 +142,19 @@ public class ActiveActiveShardingTest extends JUnitSuite { Arrays.asList( new ReplicaId("DC-A"), new ReplicaId("DC-B"), new ReplicaId("DC-C")))); - private final ActiveActiveSharding< - MyActiveActiveStringSet.Command, ShardingEnvelope> + private final ReplicatedSharding< + MyReplicatedStringSet.Command, ShardingEnvelope> aaSharding; private ProxyActor(ActorContext context) { super(context); // #bootstrap - ActiveActiveShardingSettings< - MyActiveActiveStringSet.Command, ShardingEnvelope> + ReplicatedShardingSettings< + MyReplicatedStringSet.Command, ShardingEnvelope> aaShardingSettings = - ActiveActiveShardingSettings.create( - MyActiveActiveStringSet.Command.class, + ReplicatedShardingSettings.create( + MyReplicatedStringSet.Command.class, ALL_REPLICAS, // factory for replica settings for a given replica (entityTypeKey, replicaId, allReplicas) -> @@ -168,7 +166,7 @@ public class ActiveActiveShardingTest extends JUnitSuite { entityTypeKey, entityContext -> // factory for the entity for a given entity in that replica - MyActiveActiveStringSet.create( + MyReplicatedStringSet.create( entityContext.getEntityId(), replicaId, allReplicas)) // potentially use replica id as role or dc in Akka multi dc for the // sharding instance @@ -176,8 +174,8 @@ public class ActiveActiveShardingTest extends JUnitSuite { // .withDataCenter(replicaId.id())) .withRole(replicaId.id()))); - ActiveActiveShardingExtension extension = - ActiveActiveShardingExtension.get(getContext().getSystem()); + ReplicatedShardingExtension extension = + ReplicatedShardingExtension.get(getContext().getSystem()); aaSharding = extension.init(aaShardingSettings); // #bootstrap } @@ -191,7 +189,7 @@ public class ActiveActiveShardingTest extends JUnitSuite { } private Behavior onForwardToRandom(ForwardToRandom forwardToRandom) { - Map> refs = + Map> refs = aaSharding.getEntityRefsFor(forwardToRandom.entityId); int chosenIdx = new java.util.Random().nextInt(refs.size()); new ArrayList<>(refs.values()).get(chosenIdx).tell(forwardToRandom.message); @@ -200,7 +198,7 @@ public class ActiveActiveShardingTest extends JUnitSuite { private Behavior onForwardToAll(ForwardToAll forwardToAll) { // #all-entity-refs - Map> refs = + Map> refs = aaSharding.getEntityRefsFor(forwardToAll.entityId); refs.forEach((replicaId, ref) -> ref.tell(forwardToAll.message)); // #all-entity-refs @@ -238,16 +236,16 @@ public class ActiveActiveShardingTest extends JUnitSuite { // forward messages to replicas ActorRef proxy = testKit.spawn(ProxyActor.create()); - proxy.tell(new ProxyActor.ForwardToAll("id1", new MyActiveActiveStringSet.Add("to-all"))); - proxy.tell(new ProxyActor.ForwardToRandom("id1", new MyActiveActiveStringSet.Add("to-random"))); + 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(); + TestProbe responseProbe = testKit.createTestProbe(); proxy.tell( new ProxyActor.ForwardToAll( - "id1", new MyActiveActiveStringSet.GetTexts(responseProbe.ref()))); - List responses = responseProbe.receiveSeveralMessages(3); + "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); diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingDirectReplicationSpec.scala similarity index 85% rename from akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala rename to akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingDirectReplicationSpec.scala index bb85730a88..1cd99bb3a2 100644 --- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingDirectReplicationSpec.scala +++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingDirectReplicationSpec.scala @@ -15,12 +15,9 @@ import akka.persistence.typed.PublishedEvent import akka.persistence.typed.internal.{ PublishedEventImpl, ReplicatedPublishedEventMetaData, VersionVector } import akka.persistence.typed.ReplicaId -class ActiveActiveShardingDirectReplicationSpec - extends ScalaTestWithActorTestKit - with AnyWordSpecLike - with LogCapturing { +class ReplicatedShardingDirectReplicationSpec extends ScalaTestWithActorTestKit with AnyWordSpecLike with LogCapturing { - "Active active sharding replication" must { + "Replicated sharding direct replication" must { "replicate published events to all sharding proxies" in { val replicaAProbe = createTestProbe[ShardingEnvelope[PublishedEvent]]() @@ -28,7 +25,7 @@ class ActiveActiveShardingDirectReplicationSpec val replicaCProbe = createTestProbe[ShardingEnvelope[PublishedEvent]]() val replicationActor = spawn( - ActiveActiveShardingDirectReplication( + ShardingDirectReplication( typed.ReplicaId("ReplicaA"), replicaShardingProxies = Map( ReplicaId("ReplicaA") -> replicaAProbe.ref, @@ -36,7 +33,7 @@ class ActiveActiveShardingDirectReplicationSpec ReplicaId("ReplicaC") -> replicaCProbe.ref))) val upProbe = createTestProbe[Done]() - replicationActor ! ActiveActiveShardingDirectReplication.VerifyStarted(upProbe.ref) + replicationActor ! ShardingDirectReplication.VerifyStarted(upProbe.ref) upProbe.receiveMessage() // not bullet proof wrt to subscription being complete but good enough val event = PublishedEventImpl( diff --git a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala similarity index 76% rename from akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala rename to akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala index 74c7f3052a..d87a13673f 100644 --- a/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala +++ b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala @@ -16,7 +16,7 @@ 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.ActiveActiveEventSourcing +import akka.persistence.typed.scaladsl.ReplicatedEventSourcing import akka.persistence.typed.scaladsl.Effect import akka.persistence.typed.scaladsl.EventSourcedBehavior import akka.serialization.jackson.CborSerializable @@ -25,7 +25,7 @@ import org.scalatest.wordspec.AnyWordSpecLike import scala.util.Random -object ActiveActiveShardingSpec { +object ReplicatedShardingSpec { def config = ConfigFactory.parseString(""" akka.loglevel = DEBUG akka.loggers = ["akka.testkit.SilenceAllTestEventListener"] @@ -36,19 +36,19 @@ object ActiveActiveShardingSpec { akka.remote.artery.canonical.port = 0""").withFallback(PersistenceTestKitPlugin.config) } -class ActiveActiveShardingSpec - extends ScalaTestWithActorTestKit(ActiveActiveShardingSpec.config) +class ReplicatedShardingSpec + extends ScalaTestWithActorTestKit(ReplicatedShardingSpec.config) with AnyWordSpecLike with LogCapturing { - object MyActiveActiveStringSet { + 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(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] = - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( entityId, replicaId, allReplicas, @@ -70,15 +70,13 @@ class ActiveActiveShardingSpec object ProxyActor { sealed trait Command - case class ForwardToRandom(entityId: String, msg: MyActiveActiveStringSet.Command) extends Command - case class ForwardToAll(entityId: String, msg: MyActiveActiveStringSet.Command) extends Command + case class ForwardToRandom(entityId: String, msg: MyReplicatedStringSet.Command) extends Command + case class ForwardToAll(entityId: String, msg: MyReplicatedStringSet.Command) extends Command def apply(): Behavior[Command] = Behaviors.setup { context => // #bootstrap val aaShardingSettings = - ActiveActiveShardingSettings[ - MyActiveActiveStringSet.Command, - ShardingEnvelope[MyActiveActiveStringSet.Command]]( + ReplicatedShardingSettings[MyReplicatedStringSet.Command, ShardingEnvelope[MyReplicatedStringSet.Command]]( // all replicas Set(ReplicaId("DC-A"), ReplicaId("DC-B"), ReplicaId("DC-C"))) { (entityTypeKey, replicaId, allReplicaIds) => // factory for replica settings for a given replica @@ -87,7 +85,7 @@ class ActiveActiveShardingSpec // use the provided entity type key for sharding to get one sharding instance per replica Entity(entityTypeKey) { entityContext => // factory for the entity for a given entity in that replica - MyActiveActiveStringSet(entityContext.entityId, replicaId, allReplicaIds) + MyReplicatedStringSet(entityContext.entityId, replicaId, allReplicaIds) } // potentially use replica id as role or dc in Akka multi dc for the sharding instance // to control where replicas will live @@ -95,7 +93,7 @@ class ActiveActiveShardingSpec .withRole(replicaId.id)) } - val aaSharding = ActiveActiveShardingExtension(context.system).init(aaShardingSettings) + val aaSharding = ReplicatedShardingExtension(context.system).init(aaShardingSettings) // #bootstrap Behaviors.receiveMessage { @@ -115,7 +113,7 @@ class ActiveActiveShardingSpec } } - "Active active sharding" should { + "Replicated sharding" should { "form a one node cluster" in { val node = Cluster(system) @@ -128,13 +126,13 @@ class ActiveActiveShardingSpec "forward to replicas" in { val proxy = spawn(ProxyActor()) - proxy ! ProxyActor.ForwardToAll("id1", MyActiveActiveStringSet.Add("to-all")) - proxy ! ProxyActor.ForwardToRandom("id1", MyActiveActiveStringSet.Add("to-random")) + proxy ! ProxyActor.ForwardToAll("id1", MyReplicatedStringSet.Add("to-all")) + proxy ! ProxyActor.ForwardToRandom("id1", MyReplicatedStringSet.Add("to-random")) eventually { - val probe = createTestProbe[MyActiveActiveStringSet.Texts]() - proxy ! ProxyActor.ForwardToAll("id1", MyActiveActiveStringSet.GetTexts(probe.ref)) - val responses: Seq[MyActiveActiveStringSet.Texts] = probe.receiveMessages(3) + val probe = createTestProbe[MyReplicatedStringSet.Texts]() + proxy ! ProxyActor.ForwardToAll("id1", MyReplicatedStringSet.GetTexts(probe.ref)) + val responses: Seq[MyReplicatedStringSet.Texts] = probe.receiveMessages(3) val uniqueTexts = responses.flatMap(res => res.texts).toSet uniqueTexts should ===(Set("to-all", "to-random")) } diff --git a/akka-docs/src/main/paradox/project/examples.md b/akka-docs/src/main/paradox/project/examples.md index af9e879ffe..d790e8590e 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 superseeded by @ref[Replicated Event Sourcing](../typed/replicated-eventsourcing.md) ## Cluster with Docker diff --git a/akka-docs/src/main/paradox/typed/cluster-dc.md b/akka-docs/src/main/paradox/typed/cluster-dc.md index 8d512cb7af..ca0003bb71 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[Replciated 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/index-persistence.md b/akka-docs/src/main/paradox/typed/index-persistence.md index a3da54c018..2a7fd09aa5 100644 --- a/akka-docs/src/main/paradox/typed/index-persistence.md +++ b/akka-docs/src/main/paradox/typed/index-persistence.md @@ -9,7 +9,7 @@ project.description: Event Sourcing with Akka Persistence enables actors to pers @@@ index * [persistence](persistence.md) -* [active-active](persistence-active-active.md) +* [active-active](replicated-eventsourcing.md) * [cqrs](cqrs.md) * [persistence-style](persistence-style.md) * [persistence-snapshot](persistence-snapshot.md) @@ -20,6 +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) -* [active-active-examples](persistence-active-active-examples.md) +* [active-active-examples](replicated-eventsourcing-examples.md) @@@ diff --git a/akka-docs/src/main/paradox/typed/persistence-active-active-examples.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md similarity index 82% rename from akka-docs/src/main/paradox/typed/persistence-active-active-examples.md rename to akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md index 782557c848..d7b49e92eb 100644 --- a/akka-docs/src/main/paradox/typed/persistence-active-active-examples.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md @@ -1,6 +1,6 @@ -# Active-Active Examples +# Replicated Event Sourcing Examples -The following are more realistic examples of building systems with active-active event sourcing. +The following are more realistic examples of building systems with Replicated Event Sourcing. ## Auction @@ -16,21 +16,21 @@ We are building a small auction service. It has the following operations: 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/AAAuctionExampleSpec.scala) { #commands } +: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #commands } The events: Scala -: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala) { #events } +: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #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/AAAuctionExampleSpec.scala) { #command-handler } +: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #command-handler } -There is nothing specific to active-active about the command handler. It is the same as a command handler for a standard `EventSourcedBehavior`. +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 @@ -41,13 +41,13 @@ The initial state is taken from a `AuctionSetup` instance. The minimum bid is mo an `initialBid`. Scala -: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala) { #setup } +: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #setup } The auction moves through the following phases: Scala -: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala) { #phase } +: @@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. @@ -56,7 +56,7 @@ Let's have a look at our state class, `AuctionState` which also represents the C Scala -: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala) { #state } +: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #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. @@ -95,9 +95,9 @@ 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/AAAuctionExampleSpec.scala) { #event-triggers } +: @@snip [AuctionExample](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala) { #event-triggers } -The event trigger uses the `ActiveActiveContext` to decide when to trigger the Finish of the action. +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/persistence-active-active.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md similarity index 69% rename from akka-docs/src/main/paradox/typed/persistence-active-active.md rename to akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index 4d621b2b31..df09cd5049 100644 --- a/akka-docs/src/main/paradox/typed/persistence-active-active.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -1,4 +1,4 @@ -# Active-Active Event Sourcing +# Replicated Event Sourcing @@@ warning @@ -13,7 +13,7 @@ warning or deprecation period. It is also not recommended to use this module in This restriction means that in the event of network partitions, and for a short time during rolling re-deploys, `EventSourcedBehaviors`s are unavailable. -Active-active event sourcing enables running multiple replicas of each entity. +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: @@ -27,27 +27,27 @@ The motivations are: * 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 active-active is enabled +However, the event handler must be able to **handle concurrent events** as when replication is enabled there is no longer the single writer principle as there is with a normal `EventSourcedBehavior`. -The state of an active-active `EventSourcedBehavior` is **eventually consistent**. Event replication may be delayed +The state of a replicated `EventSourcedBehavior` is **eventually consistent**. Event replication may be delayed due to network partitions and outages and the event handler and those reading the state must be designed to handle this. -To be able to use active active the journal and snapshot store used is required to have specific support for the metadata that active active needs (see @ref[Journal Support](#journal-support)) +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 active-active to run a replica per data center. +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. +When there is no network partitions and no concurrent writes the events stored by a `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. +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 and the events can just 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. +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. @@ -58,44 +58,44 @@ To assist in implementing the event handler active-active detects these conflict @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 active-active -replication @java[let it extend `ActiveActiveEventSourcedBehavior` instead of `EventSourcedBehavior` and] use the factory methods on @apidoc[ActiveActiveEventSourcing]. +replication @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 [ActiveActiveCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala) { #replicas } +: @@snip [ReplicatedEventSourcingCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala) { #replicas } Java -: @@snip [ActiveActiveCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java) { #replicas } +: @@snip [ReplicatedEventSourcingCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java) { #replicas } Then to enable replication create the event sourced behavior with the factory method: Scala -: @@snip [ActiveActiveCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala) { #factory } +: @@snip [ReplicatedEventSourcingCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala) { #factory } Java -: @@snip [ActiveActiveCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java) { #factory } +: @@snip [ReplicatedEventSourcingCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java) { #factory } The factory takes in: * EntityID: this will be used as part of the underlying persistenceId * Replica: Which replica this instance is * All Replicas and the query plugin used to read their events -* A factory function to create an instance of the @scala[`EventSourcedBehavior`]@java[`ActiveActiveEventSourcedBehavior`] +* 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 [ActiveActiveCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala) { #factory-shared} +: @@snip [ReplicatedEventSourcingCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala) { #factory-shared} Java -: @@snip [ActiveActiveCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java) { #factory-shared } +: @@snip [ReplicatedEventSourcingCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java) { #factory-shared } @@@ div { .group-scala } -The function passed to both factory methods return an `EventSourcedBehavior` and provide access to the @apidoc[ActiveActiveContext] that has the following methods: +The function passed to both factory methods return an `EventSourcedBehavior` and provide access to the @apidoc[ReplicationContext] that has the following methods: * entityId * replicaId @@ -108,8 +108,8 @@ As well as methods that **can only be** used in the event handler. The values th @@@ div { .group-java } -The function passed to both factory methods is invoked with a special @apidoc[ActiveActiveContext] that needs to be passed to the -concrete `ActiveActiveEventSourcedBehavior` and on to the super constructor. +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: @@ -118,7 +118,7 @@ The context gives access to: * allReplicas * persistenceId -As well as methods that **can only be** used in the event handler, accessed through `getActiveActiveContext`. The values these methods return relate to the event that is being processed. +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. @@@ @@ -190,9 +190,9 @@ There is no built in support for knowing an event has been replicated to all rep 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[ActiveActiveContext] contains the current replica, the origin replica for the event processes, and if a recovery is running. These can be used to +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](./persistence-active-active-examples.md#auction) uses these techniques. +designated to do it. The @ref[Auction example](./replicated-eventsourcing-examples.md#auction) uses these techniques. ## How it works @@ -203,7 +203,7 @@ You don’t have to read this section to be able to use the feature, but to use Causal delivery order means that events persisted in one data center are read in the same order in other data centers. The order of concurrent events is undefined, which should be no problem when using [CRDT's](#conflict-free-replicated-data-types) -and otherwise will be detected via the `ActiveActiveContext` concurrent method. +and otherwise will be detected via the `ReplicationContext` concurrent method. For example: @@ -231,7 +231,7 @@ A third data center may also see the events as either "e1, e3, e2" or as "e1, e2 ### Concurrent updates -Active-active automatically tracks causality between events from different replias using [version vectors](https://en.wikipedia.org/wiki/Version_vector). +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) @@ -245,29 +245,29 @@ When comparing two version vectors `v1` and `v2`: * `v1`is CONCURRENT with `v2` otherwise -## Sharded Active Active entities +## Sharded Replicated Event Sourced entities -To simplify what probably are the most common use cases for how you will want to distribute the active active actors there is a minimal API for running multiple instances of @ref[Akka Cluster Sharding](cluster-sharding.md), +To simplify what probably are the most common use cases for how you will want to distribute the replicated actors there is a minimal API for running multiple instances of @ref[Akka Cluster Sharding](cluster-sharding.md), each instance holding the entities for a single replica. The distribution of the replicas can be controlled either through cluster roles or using the @ref[multi datacenter](cluster-dc.md) support in Akka Cluster. -The API consists of bootstrapping logic for starting the sharding instances through @apidoc[ActiveActiveShardingExtension] available from the +The API consists of bootstrapping logic for starting the sharding instances through @apidoc[ReplicatedShardingExtension] available from the `akka-cluster-sharding-typed` module. Scala -: @@snip [ActiveActiveShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala) { #bootstrap } +: @@snip [ReplicatedShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala) { #bootstrap } Java -: @@snip [ActiveActiveShardingTest.java](/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java) { #bootstrap } +: @@snip [ReplicatedShardingTest.java](/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ReplicatedShardingTest.java) { #bootstrap } -`init` returns an @apidoc[ActiveActiveSharding] instance which gives access to @apidoc[EntityRef]s for each of the replicas for arbitrary routing logic: +`init` returns an @apidoc[ReplicatedSharding] instance which gives access to @apidoc[EntityRef]s for each of the replicas for arbitrary routing logic: Scala -: @@snip [ActiveActiveShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ActiveActiveShardingSpec.scala) { #all-entity-refs } +: @@snip [ReplicatedShardingSpec.scala](/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala) { #all-entity-refs } Java -: @@snip [ActiveActiveShardingTest.java](/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ActiveActiveShardingTest.java) { #all-entity-refs } +: @@snip [ReplicatedShardingTest.java](/akka-cluster-sharding-typed/src/test/java/akka/cluster/sharding/typed/ReplicatedShardingTest.java) { #all-entity-refs } 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)). @@ -275,24 +275,24 @@ More advanced routing among the replicas is currently left as an exercise for th ## Direct Replication of Events Normally an event has to be written in the journal and then picked up by the trailing read journal in the other replicas. -As an optimization the active active events can be published across the Akka cluster to the replicas. The read side +As an optimization the replicated events can be published across the Akka cluster to the replicas. The read side 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. -To enable this feature you first need to enable event publishing on the @scala[`EventSourcedBehavior`]@java[`ActiveActiveEventSourcedBehavior`] with `withEventPublishing` -and then enable direct replication through `withDirectReplication()` on @apidoc[ActiveActiveShardingSettings] (if not using - active active sharding the replication can be run standalone by starting the @apidoc[ActiveActiveShardingDirectReplication] actor). +To enable this feature you first need to enable event publishing on the @scala[`EventSourcedBehavior`]@java[`ReplicatedEventSourcedBehavior`] with `withEventPublishing` +and then enable direct replication through `withDirectReplication()` on @apidoc[ReplicatedShardingSettings] (if not using + replicated sharding the replication can be run standalone by starting the @apidoc[ShardingDirectReplication] actor). The "event publishing" feature publishes each event to the local system event bus as a side effect after it has been written, -the `ActiveActiveShardingDirectReplication` actor subscribes to these events and forwards them to the replicas allowing them +the @apidoc[ShardingDirectReplication] actor subscribes to these events and forwards them to the replicas allowing them to fast forward the stream of events for the origin replica. (With additional potential future support in journals for fast forwarding [#29311](https://github.com/akka/akka/issues/29311)). ## Journal Support -For a journal plugin to support active active it needs to store and read metadata for each event if it is defined in the @apiref[PersistentRepr] +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 active active it needs to store and read metadata for the snapshot if it is defined in the @apiref[akka.persistence.SnapshotMetadata] `metadata` field. +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. \ No newline at end of file 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 fde55aa446..26b4fbc700 100644 --- a/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala +++ b/akka-persistence-tck/src/main/scala/akka/persistence/CapabilityFlags.scala @@ -72,7 +72,7 @@ trait SnapshotStoreCapabilityFlags extends CapabilityFlags { /** * When `true` enables tests which check if the snapshot store properly stores and - * loads metadata (needed for Active Active) along with the snapshots + * loads metadata (needed for replication) along with the snapshots */ protected def supportsMetadata: CapabilityFlag } diff --git a/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ActiveActiveTest.java b/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ReplicatedEventSourcingTest.java similarity index 91% rename from akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ActiveActiveTest.java rename to akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ReplicatedEventSourcingTest.java index 38299e1530..a22d3bbb85 100644 --- a/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ActiveActiveTest.java +++ b/akka-persistence-typed-tests/src/test/java/akka/persistence/typed/ReplicatedEventSourcingTest.java @@ -11,7 +11,6 @@ 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.javadsl.PersistenceTestKit; import akka.persistence.testkit.query.javadsl.PersistenceTestKitReadJournal; import akka.persistence.typed.javadsl.*; import com.typesafe.config.ConfigFactory; @@ -25,10 +24,10 @@ import java.util.*; import static akka.Done.done; import static org.junit.Assert.assertEquals; -public class ActiveActiveTest extends JUnitSuite { +public class ReplicatedEventSourcingTest extends JUnitSuite { static final class TestBehavior - extends ActiveActiveEventSourcedBehavior> { + extends ReplicatedEventSourcedBehavior> { interface Command {} static final class GetState implements Command { @@ -81,7 +80,7 @@ public class ActiveActiveTest extends JUnitSuite { public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ActiveActiveEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.withSharedJournal( entityId, replicaId, allReplicas, @@ -89,8 +88,8 @@ public class ActiveActiveTest extends JUnitSuite { TestBehavior::new); } - private TestBehavior(ActiveActiveContext activeActiveContext) { - super(activeActiveContext); + private TestBehavior(ReplicationContext replicationContext) { + super(replicationContext); } @Override @@ -124,7 +123,7 @@ public class ActiveActiveTest extends JUnitSuite { (GetReplica cmd) -> Effect() .none() - .thenRun(() -> cmd.replyTo.tell(getActiveActiveContext().replicaId()))) + .thenRun(() -> cmd.replyTo.tell(getReplicationContext().replicaId()))) .onCommand(Stop.class, __ -> Effect().stop()) .build(); } @@ -153,9 +152,9 @@ public class ActiveActiveTest extends JUnitSuite { @Rule public final LogCapturing logCapturing = new LogCapturing(); - // minimal test, full coverage over in ActiveActiveSpec + // minimal test, full coverage over in ReplicatedEventSourcingSpec @Test - public void activeActiveReplicationTest() { + public void replicatedEventSourcingReplicationTest() { ReplicaId dcA = new ReplicaId("DC-A"); ReplicaId dcB = new ReplicaId("DC-B"); ReplicaId dcC = new ReplicaId("DC-C"); diff --git a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java similarity index 74% rename from akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java rename to akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java index 4020954f9f..d9928f2bd6 100644 --- a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ActiveActiveCompileOnlyTest.java +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java @@ -9,7 +9,7 @@ import akka.persistence.typed.javadsl.*; import java.util.*; -public class ActiveActiveCompileOnlyTest { +public class ReplicatedEventSourcingCompileOnlyTest { // dummy for docs example interface Command {} @@ -19,11 +19,11 @@ public class ActiveActiveCompileOnlyTest { interface State {} static // #factory - final class MyActiceActiveEventSourcedBehavior - extends ActiveActiveEventSourcedBehavior { + final class MyReplicatedEventSourcedBehavior + extends ReplicatedEventSourcedBehavior { - public MyActiceActiveEventSourcedBehavior(ActiveActiveContext activeActiveContext) { - super(activeActiveContext); + public MyReplicatedEventSourcedBehavior(ReplicationContext replicationContext) { + super(replicationContext); } // ... implementation of abstract methods ... // #factory @@ -58,12 +58,12 @@ public class ActiveActiveCompileOnlyTest { String queryPluginId = ""; // #factory-shared - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( "entityId", DCA, allReplicas, queryPluginId, - context -> new MyActiceActiveEventSourcedBehavior(context)); + context -> new MyReplicatedEventSourcedBehavior(context)); // #factory-shared // #factory @@ -74,11 +74,11 @@ public class ActiveActiveCompileOnlyTest { allReplicasAndQueryPlugins.put(DCB, "journalForDCB"); EventSourcedBehavior behavior = - ActiveActiveEventSourcing.create( + ReplicatedEventSourcing.create( "entityId", DCA, allReplicasAndQueryPlugins, - context -> new MyActiceActiveEventSourcedBehavior(context)); + context -> new MyReplicatedEventSourcedBehavior(context)); // #factory } } diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalReplicationSpec.scala similarity index 91% rename from akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalActiveActiveSpec.scala rename to akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalReplicationSpec.scala index d4e0803997..8de42eacaf 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalActiveActiveSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/MultiJournalReplicationSpec.scala @@ -14,7 +14,7 @@ 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.ActiveActiveEventSourcing +import akka.persistence.typed.scaladsl.ReplicatedEventSourcing import akka.persistence.typed.scaladsl.Effect import akka.persistence.typed.scaladsl.EventSourcedBehavior import akka.stream.scaladsl.Sink @@ -23,7 +23,7 @@ import com.typesafe.config.ConfigFactory import org.scalatest.concurrent.Eventually import org.scalatest.wordspec.AnyWordSpecLike -object MultiJournalActiveActiveSpec { +object MultiJournalReplicationSpec { object Actor { sealed trait Command @@ -32,7 +32,7 @@ object MultiJournalActiveActiveSpec { private val writeJournalPerReplica = Map("R1" -> "journal1.journal", "R2" -> "journal2.journal") def apply(entityId: String, replicaId: String): Behavior[Command] = { - ActiveActiveEventSourcing( + ReplicatedEventSourcing( entityId, ReplicaId(replicaId), Map(ReplicaId("R1") -> "journal1.query", ReplicaId("R2") -> "journal2.query"))( @@ -65,15 +65,15 @@ object MultiJournalActiveActiveSpec { } -class MultiJournalActiveActiveSpec - extends ScalaTestWithActorTestKit(MultiJournalActiveActiveSpec.separateJournalsConfig) +class MultiJournalReplicationSpec + extends ScalaTestWithActorTestKit(MultiJournalReplicationSpec.separateJournalsConfig) with AnyWordSpecLike with LogCapturing with Eventually { - import MultiJournalActiveActiveSpec._ + import MultiJournalReplicationSpec._ val ids = new AtomicInteger(0) def nextEntityId = s"e-${ids.getAndIncrement()}" - "ActiveActiveEventSourcing" should { + "ReplicatedEventSourcing" should { "support one journal per replica" in { val r1 = spawn(Actor("id1", "R1")) diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventPublishingSpec.scala similarity index 75% rename from akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala rename to akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventPublishingSpec.scala index 56c8042ac8..90c498efb8 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveEventPublishingSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventPublishingSpec.scala @@ -13,14 +13,14 @@ 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.ActiveActiveEventSourcing +import akka.persistence.typed.scaladsl.ReplicatedEventSourcing import akka.persistence.typed.scaladsl.Effect import akka.persistence.typed.scaladsl.EventSourcedBehavior import org.scalatest.wordspec.AnyWordSpecLike -object ActiveActiveEventPublishingSpec { +object ReplicatedEventPublishingSpec { - object MyActiveActive { + object MyReplicatedBehavior { trait Command case class Add(text: String, replyTo: ActorRef[Done]) extends Command case class Get(replyTo: ActorRef[Set[String]]) extends Command @@ -28,7 +28,7 @@ object ActiveActiveEventPublishingSpec { def apply(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] = Behaviors.setup { ctx => - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( entityId, replicaId, allReplicas, @@ -56,7 +56,7 @@ object ActiveActiveEventPublishingSpec { } } -class ActiveActiveEventPublishingSpec +class ReplicatedEventPublishingSpec extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) with AnyWordSpecLike with LogCapturing { @@ -71,14 +71,14 @@ class ActiveActiveEventPublishingSpec s"myId$idCounter" } - import ActiveActiveEventPublishingSpec._ + import ReplicatedEventPublishingSpec._ - "An active active actor" must { + "An Replicated Event Sourced actor" must { "move forward when a published event from a replica is received" in { val id = nextEntityId() - val actor = spawn(MyActiveActive(id, DCA, Set(DCA, DCB))) + val actor = spawn(MyReplicatedBehavior(id, DCA, Set(DCA, DCB))) val probe = createTestProbe[Any]() - actor ! MyActiveActive.Add("one", probe.ref) + actor ! MyReplicatedBehavior.Add("one", probe.ref) probe.expectMessage(Done) // simulate a published event from another replica @@ -88,18 +88,18 @@ class ActiveActiveEventPublishingSpec "two", System.currentTimeMillis(), Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) - actor ! MyActiveActive.Add("three", probe.ref) + actor ! MyReplicatedBehavior.Add("three", probe.ref) probe.expectMessage(Done) - actor ! MyActiveActive.Get(probe.ref) + 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(MyActiveActive(id, DCA, Set(DCA, DCB))) + val actor = spawn(MyReplicatedBehavior(id, DCA, Set(DCA, DCB))) val probe = createTestProbe[Any]() - actor ! MyActiveActive.Add("one", probe.ref) + actor ! MyReplicatedBehavior.Add("one", probe.ref) probe.expectMessage(Done) // simulate a published event from another replica @@ -109,18 +109,18 @@ class ActiveActiveEventPublishingSpec "two", System.currentTimeMillis(), Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) - actor ! MyActiveActive.Add("three", probe.ref) + actor ! MyReplicatedBehavior.Add("three", probe.ref) probe.expectMessage(Done) - actor ! MyActiveActive.Get(probe.ref) + 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(MyActiveActive(id, DCA, Set(DCA, DCB))) + val actor = spawn(MyReplicatedBehavior(id, DCA, Set(DCA, DCB))) val probe = createTestProbe[Any]() - actor ! MyActiveActive.Add("one", probe.ref) + actor ! MyReplicatedBehavior.Add("one", probe.ref) probe.expectMessage(Done) // simulate a published event from another replica @@ -130,18 +130,18 @@ class ActiveActiveEventPublishingSpec "two", System.currentTimeMillis(), Some(new ReplicatedPublishedEventMetaData(DCC, VersionVector.empty))) - actor ! MyActiveActive.Add("three", probe.ref) + actor ! MyReplicatedBehavior.Add("three", probe.ref) probe.expectMessage(Done) - actor ! MyActiveActive.Get(probe.ref) + 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(MyActiveActive(id, DCA, Set(DCA, DCB))) + val actor = spawn(MyReplicatedBehavior(id, DCA, Set(DCA, DCB))) val probe = createTestProbe[Any]() - actor ! MyActiveActive.Add("one", probe.ref) + actor ! MyReplicatedBehavior.Add("one", probe.ref) probe.expectMessage(Done) // simulate a published event from another replica @@ -159,27 +159,27 @@ class ActiveActiveEventPublishingSpec System.currentTimeMillis(), Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) - actor ! MyActiveActive.Add("three", probe.ref) + actor ! MyReplicatedBehavior.Add("three", probe.ref) probe.expectMessage(Done) - actor ! MyActiveActive.Get(probe.ref) + 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 activeActiveBehavior = MyActiveActive(id, DCA, Set(DCA, DCB)) - val incarnation1 = spawn(activeActiveBehavior) - incarnation1 ! MyActiveActive.Add("one", probe.ref) + val replicatedBehavior = MyReplicatedBehavior(id, DCA, Set(DCA, DCB)) + val incarnation1 = spawn(replicatedBehavior) + incarnation1 ! MyReplicatedBehavior.Add("one", probe.ref) probe.expectMessage(Done) - incarnation1 ! MyActiveActive.Stop + incarnation1 ! MyReplicatedBehavior.Stop probe.expectTerminated(incarnation1) - val incarnation2 = spawn(activeActiveBehavior) + val incarnation2 = spawn(replicatedBehavior) - incarnation2 ! MyActiveActive.Get(probe.ref) + incarnation2 ! MyReplicatedBehavior.Get(probe.ref) probe.expectMessage(Set("one")) // replay completed @@ -191,19 +191,19 @@ class ActiveActiveEventPublishingSpec System.currentTimeMillis(), Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) - incarnation2 ! MyActiveActive.Add("three", probe.ref) + incarnation2 ! MyReplicatedBehavior.Add("three", probe.ref) probe.expectMessage(Done) - incarnation2 ! MyActiveActive.Get(probe.ref) + 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 activeActiveBehaviorA = MyActiveActive(id, DCA, Set(DCA, DCB)) - val incarnationA1 = spawn(activeActiveBehaviorA) - incarnationA1 ! MyActiveActive.Add("one", probe.ref) + 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 @@ -214,10 +214,10 @@ class ActiveActiveEventPublishingSpec System.currentTimeMillis(), Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) - incarnationA1 ! MyActiveActive.Stop + incarnationA1 ! MyReplicatedBehavior.Stop probe.expectTerminated(incarnationA1) - val incarnationA2 = spawn(activeActiveBehaviorA) + val incarnationA2 = spawn(replicatedBehaviorA) // simulate a published event from another replica incarnationA2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( @@ -227,10 +227,10 @@ class ActiveActiveEventPublishingSpec System.currentTimeMillis(), Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) - incarnationA2 ! MyActiveActive.Add("four", probe.ref) + incarnationA2 ! MyReplicatedBehavior.Add("four", probe.ref) probe.expectMessage(Done) - incarnationA2 ! MyActiveActive.Get(probe.ref) + 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/ActiveActiveSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSpec.scala similarity index 97% rename from akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala rename to akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSpec.scala index 3c6ea4e25b..d50c426a7a 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSpec.scala @@ -13,12 +13,12 @@ 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.scaladsl.{ ActiveActiveContext, ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +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 ActiveActiveSpec { +object ReplicatedEventSourcingSpec { val AllReplicas = Set(ReplicaId("R1"), ReplicaId("R2"), ReplicaId("R3")) @@ -35,7 +35,7 @@ object ActiveActiveSpec { testBehavior(entityId, replicaId, Some(probe)) def eventSourcedBehavior( - aaContext: ActiveActiveContext, + aaContext: ReplicationContext, probe: Option[ActorRef[EventAndContext]]): EventSourcedBehavior[Command, String, State] = { EventSourcedBehavior[Command, String, State]( aaContext.persistenceId, @@ -65,7 +65,7 @@ object ActiveActiveSpec { entityId: String, replicaId: String, probe: Option[ActorRef[EventAndContext]] = None): Behavior[Command] = - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( entityId, ReplicaId(replicaId), AllReplicas, @@ -75,15 +75,15 @@ object ActiveActiveSpec { case class EventAndContext(event: Any, origin: ReplicaId, recoveryRunning: Boolean, concurrent: Boolean) -class ActiveActiveSpec +class ReplicatedEventSourcingSpec extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) with AnyWordSpecLike with LogCapturing with Eventually { - import ActiveActiveSpec._ + import ReplicatedEventSourcingSpec._ val ids = new AtomicInteger(0) def nextEntityId = s"e-${ids.getAndIncrement()}" - "ActiveActiveEventSourcing" should { + "ReplicatedEventSourcing" should { "replicate events between entities" in { val entityId = nextEntityId val probe = createTestProbe[Done]() diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveBaseSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationBaseSpec.scala similarity index 92% rename from akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveBaseSpec.scala rename to akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationBaseSpec.scala index cfa8a820dc..ec8a9487a7 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveBaseSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationBaseSpec.scala @@ -11,13 +11,13 @@ import akka.persistence.testkit.{ PersistenceTestKitPlugin, PersistenceTestKitSn import org.scalatest.concurrent.Eventually import org.scalatest.wordspec.AnyWordSpecLike -object ActiveActiveBaseSpec { +object ReplicationBaseSpec { val R1 = ReplicaId("R1") val R2 = ReplicaId("R2") val AllReplicas = Set(R1, R2) } -abstract class ActiveActiveBaseSpec +abstract class ReplicationBaseSpec extends ScalaTestWithActorTestKit( PersistenceTestKitPlugin.config.withFallback(PersistenceTestKitSnapshotPlugin.config)) with AnyWordSpecLike diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveIllegalAccessSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationIllegalAccessSpec.scala similarity index 82% rename from akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveIllegalAccessSpec.scala rename to akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationIllegalAccessSpec.scala index a8145639b0..8415b0dcd9 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveIllegalAccessSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationIllegalAccessSpec.scala @@ -8,12 +8,12 @@ import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestK import akka.actor.typed.{ ActorRef, Behavior } import akka.persistence.testkit.PersistenceTestKitPlugin import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal -import akka.persistence.typed.scaladsl.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing } import akka.serialization.jackson.CborSerializable import org.scalatest.concurrent.Eventually import org.scalatest.wordspec.AnyWordSpecLike -object ActiveActiveIllegalAccessSpec { +object ReplicationIllegalAccessSpec { val R1 = ReplicaId("R1") val R2 = ReplicaId("R1") @@ -28,11 +28,7 @@ object ActiveActiveIllegalAccessSpec { case class State(all: List[String]) extends CborSerializable def apply(entityId: String, replica: ReplicaId): Behavior[Command] = { - ActiveActiveEventSourcing.withSharedJournal( - entityId, - replica, - AllReplicas, - PersistenceTestKitReadJournal.Identifier)( + ReplicatedEventSourcing.withSharedJournal(entityId, replica, AllReplicas, PersistenceTestKitReadJournal.Identifier)( aaContext => EventSourcedBehavior[Command, String, State]( aaContext.persistenceId, @@ -66,30 +62,30 @@ object ActiveActiveIllegalAccessSpec { } -class ActiveActiveIllegalAccessSpec +class ReplicationIllegalAccessSpec extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) with AnyWordSpecLike with LogCapturing with Eventually { - import ActiveActiveIllegalAccessSpec._ - "ActiveActive" should { + import ReplicationIllegalAccessSpec._ + "ReplicatedEventSourcing" should { "detect illegal access to context in command handler" in { val probe = createTestProbe[Thrown]() - val ref = spawn(ActiveActiveIllegalAccessSpec("id1", R1)) + 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(ActiveActiveIllegalAccessSpec("id1", R1)) + 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] { - ActiveActiveEventSourcing.withSharedJournal("id2", R1, AllReplicas, PersistenceTestKitReadJournal.Identifier) { + ReplicatedEventSourcing.withSharedJournal("id2", R1, AllReplicas, PersistenceTestKitReadJournal.Identifier) { aaContext => aaContext.origin ??? diff --git a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSnapshotSpec.scala b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationSnapshotSpec.scala similarity index 92% rename from akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSnapshotSpec.scala rename to akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationSnapshotSpec.scala index 8eeda55c15..f71b12afbb 100644 --- a/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ActiveActiveSnapshotSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicationSnapshotSpec.scala @@ -12,13 +12,13 @@ import akka.persistence.testkit.{ PersistenceTestKitPlugin, PersistenceTestKitSn 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.ActiveActiveEventSourcing +import akka.persistence.typed.scaladsl.ReplicatedEventSourcing import org.scalatest.concurrent.Eventually import org.scalatest.wordspec.AnyWordSpecLike -object ActiveActiveSnapshotSpec { +object ReplicationSnapshotSpec { - import ActiveActiveSpec._ + import ReplicatedEventSourcingSpec._ def behaviorWithSnapshotting(entityId: String, replicaId: ReplicaId): Behavior[Command] = behaviorWithSnapshotting(entityId, replicaId, None) @@ -33,7 +33,7 @@ object ActiveActiveSnapshotSpec { entityId: String, replicaId: ReplicaId, probe: Option[ActorRef[EventAndContext]]): Behavior[Command] = { - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( entityId, replicaId, AllReplicas, @@ -43,14 +43,14 @@ object ActiveActiveSnapshotSpec { } } -class ActiveActiveSnapshotSpec +class ReplicationSnapshotSpec extends ScalaTestWithActorTestKit( PersistenceTestKitPlugin.config.withFallback(PersistenceTestKitSnapshotPlugin.config)) with AnyWordSpecLike with LogCapturing with Eventually { - import ActiveActiveSpec._ - import ActiveActiveSnapshotSpec._ + import ReplicatedEventSourcingSpec._ + import ReplicationSnapshotSpec._ val ids = new AtomicInteger(0) def nextEntityId = s"e-${ids.getAndIncrement()}" @@ -61,7 +61,7 @@ class ActiveActiveSnapshotSpec val R1 = ReplicaId("R1") val R2 = ReplicaId("R2") - "ActiveActive" should { + "ReplicatedEventSourcing" should { "recover state from snapshots" in { val entityId = nextEntityId val persistenceIdR1 = s"$entityId|R1" 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 index 6393784eed..3f95c87755 100644 --- 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 @@ -8,8 +8,8 @@ import akka.actor.typed.ActorRef import akka.actor.typed.scaladsl.Behaviors import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal import akka.persistence.typed.crdt.CounterSpec.PlainCounter.{ Decrement, Get, Increment } -import akka.persistence.typed.scaladsl.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } -import akka.persistence.typed.{ ActiveActiveBaseSpec, ReplicaId } +import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing } +import akka.persistence.typed.{ ReplicaId, ReplicationBaseSpec } object CounterSpec { @@ -20,7 +20,7 @@ object CounterSpec { case object Decrement extends Command } - import ActiveActiveBaseSpec._ + import ReplicationBaseSpec._ def apply( entityId: String, @@ -28,7 +28,7 @@ object CounterSpec { snapshotEvery: Long = 100, eventProbe: Option[ActorRef[Counter.Updated]] = None) = Behaviors.setup[PlainCounter.Command] { context => - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( entityId, replicaId, AllReplicas, @@ -58,12 +58,12 @@ object CounterSpec { } } -class CounterSpec extends ActiveActiveBaseSpec { +class CounterSpec extends ReplicationBaseSpec { import CounterSpec._ - import ActiveActiveBaseSpec._ + import ReplicationBaseSpec._ - "Active active entity using CRDT counter" should { + "Replicated entity using CRDT counter" should { "replicate" in { val id = nextEntityId val r1 = spawn(apply(id, R1)) 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 index 8c0718b39a..48b300b5cf 100644 --- 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 @@ -6,13 +6,13 @@ package akka.persistence.typed.crdt import akka.actor.typed.{ ActorRef, Behavior } import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal -import akka.persistence.typed.scaladsl.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } -import akka.persistence.typed.{ ActiveActiveBaseSpec, ReplicaId } +import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing } +import akka.persistence.typed.{ ReplicaId, ReplicationBaseSpec } import akka.serialization.jackson.CborSerializable object LwwSpec { - import ActiveActiveBaseSpec._ + import ReplicationBaseSpec._ sealed trait Command final case class Update(item: String, timestamp: Long, error: ActorRef[String]) extends Command @@ -26,7 +26,7 @@ object LwwSpec { object LwwRegistry { def apply(entityId: String, replica: ReplicaId): Behavior[Command] = { - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( entityId, replica, AllReplicas, @@ -59,9 +59,9 @@ object LwwSpec { } } -class LwwSpec extends ActiveActiveBaseSpec { +class LwwSpec extends ReplicationBaseSpec { import LwwSpec._ - import ActiveActiveBaseSpec._ + import ReplicationBaseSpec._ class Setup { val entityId = nextEntityId @@ -73,7 +73,7 @@ class LwwSpec extends ActiveActiveBaseSpec { val r2GetProbe = createTestProbe[Registry]() } - "Lww Active Active Event Sourced Behavior" should { + "Lww Replicated Event Sourced Behavior" should { "replicate a single event" in new Setup { r1 ! Update("a1", 1L, r1Probe.ref) eventually { 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 index ce65f0b9da..e17afe3090 100644 --- 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 @@ -6,17 +6,17 @@ package akka.persistence.typed.crdt import akka.actor.typed.{ ActorRef, Behavior } import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal -import akka.persistence.typed.scaladsl.{ ActiveActiveEventSourcing, Effect, EventSourcedBehavior } -import akka.persistence.typed.{ ActiveActiveBaseSpec, ReplicaId } +import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing } +import akka.persistence.typed.{ ReplicaId, ReplicationBaseSpec } import ORSetSpec.ORSetEntity._ -import akka.persistence.typed.ActiveActiveBaseSpec.{ R1, R2 } +import akka.persistence.typed.ReplicationBaseSpec.{ R1, R2 } import akka.persistence.typed.crdt.ORSetSpec.ORSetEntity import scala.util.Random object ORSetSpec { - import ActiveActiveBaseSpec._ + import ReplicationBaseSpec._ object ORSetEntity { sealed trait Command @@ -27,7 +27,7 @@ object ORSetSpec { def apply(entityId: String, replica: ReplicaId): Behavior[ORSetEntity.Command] = { - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( entityId, replica, AllReplicas, @@ -54,7 +54,7 @@ object ORSetSpec { } -class ORSetSpec extends ActiveActiveBaseSpec { +class ORSetSpec extends ReplicationBaseSpec { class Setup { val entityId = nextEntityId @@ -78,7 +78,7 @@ class ORSetSpec extends ActiveActiveBaseSpec { Thread.sleep(Random.nextInt(200).toLong) } - "ORSet Active Active Entity" should { + "ORSet Replicated Entity" should { "support concurrent updates" in new Setup { r1 ! Add("a1") diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala similarity index 95% rename from akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala rename to akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala index b0b553bc42..e0c4d408e2 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AAAuctionExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala @@ -12,13 +12,13 @@ import akka.actor.typed.{ ActorRef, Behavior } import akka.persistence.testkit.PersistenceTestKitPlugin import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal import akka.persistence.typed.ReplicaId -import akka.persistence.typed.scaladsl.{ ActiveActiveContext, ActiveActiveEventSourcing, Effect, EventSourcedBehavior } +import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing, ReplicationContext } import akka.serialization.jackson.CborSerializable import org.scalatest.concurrent.{ Eventually, ScalaFutures } import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike -object AAAuctionExampleSpec { +object ReplicatedAuctionExampleSpec { type MoneyAmount = Int @@ -104,7 +104,7 @@ object AAAuctionExampleSpec { //#setup //#command-handler - def commandHandler(setup: AuctionSetup, ctx: ActorContext[AuctionCommand], aaContext: ActiveActiveContext)( + def commandHandler(setup: AuctionSetup, ctx: ActorContext[AuctionCommand], aaContext: ReplicationContext)( state: AuctionState, command: AuctionCommand): Effect[AuctionEvent, AuctionState] = { state.phase match { @@ -166,7 +166,7 @@ object AAAuctionExampleSpec { } //#event-handler - def eventHandler(ctx: ActorContext[AuctionCommand], aaCtx: ActiveActiveContext, setup: AuctionSetup)( + def eventHandler(ctx: ActorContext[AuctionCommand], aaCtx: ReplicationContext, setup: AuctionSetup)( state: AuctionState, event: AuctionEvent): AuctionState = { @@ -184,7 +184,7 @@ object AAAuctionExampleSpec { private def eventTriggers( setup: AuctionSetup, ctx: ActorContext[AuctionCommand], - aaCtx: ActiveActiveContext, + aaCtx: ReplicationContext, event: AuctionEvent, newState: AuctionState) = { event match { @@ -214,7 +214,7 @@ object AAAuctionExampleSpec { def behavior(replica: ReplicaId, setup: AuctionSetup): Behavior[AuctionCommand] = Behaviors.setup[AuctionCommand] { ctx => - ActiveActiveEventSourcing + ReplicatedEventSourcing .withSharedJournal(setup.name, replica, setup.allReplicas, PersistenceTestKitReadJournal.Identifier) { aaCtx => EventSourcedBehavior( aaCtx.persistenceId, @@ -225,14 +225,14 @@ object AAAuctionExampleSpec { } } -class AAAuctionExampleSpec +class ReplicatedAuctionExampleSpec extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) with AnyWordSpecLike with Matchers with LogCapturing with ScalaFutures with Eventually { - import AAAuctionExampleSpec._ + import ReplicatedAuctionExampleSpec._ "Auction example" should { diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala similarity index 92% rename from akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala rename to akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala index f67fd19226..2450592182 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/AABlogExampleSpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala @@ -19,7 +19,7 @@ import org.scalatest.matchers.should.Matchers import org.scalatest.time.{ Millis, Span } import org.scalatest.wordspec.AnyWordSpecLike -object AABlogExampleSpec { +object ReplicatedBlogExampleSpec { final case class BlogState(content: Option[PostContent], contentTimestamp: LwwTime, published: Boolean) { def withContent(newContent: PostContent, timestamp: LwwTime): BlogState = @@ -44,18 +44,18 @@ object AABlogExampleSpec { final case class BodyChanged(postId: String, newContent: PostContent, timestamp: LwwTime) extends BlogEvent } -class AABlogExampleSpec +class ReplicatedBlogExampleSpec extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) with AnyWordSpecLike with Matchers with LogCapturing with ScalaFutures with Eventually { - import AABlogExampleSpec._ + import ReplicatedBlogExampleSpec._ implicit val config: PatienceConfig = PatienceConfig(timeout = Span(timeout.duration.toMillis, Millis)) - def behavior(aa: ActiveActiveContext, ctx: ActorContext[BlogCommand]) = + def behavior(aa: ReplicationContext, ctx: ActorContext[BlogCommand]) = EventSourcedBehavior[BlogCommand, BlogEvent, BlogState]( aa.persistenceId, emptyState, @@ -114,11 +114,11 @@ class AABlogExampleSpec val refDcA: ActorRef[BlogCommand] = spawn( Behaviors.setup[BlogCommand] { ctx => - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( "cat", ReplicaId("DC-A"), Set(ReplicaId("DC-A"), ReplicaId("DC-B")), - PersistenceTestKitReadJournal.Identifier) { (aa: ActiveActiveContext) => + PersistenceTestKitReadJournal.Identifier) { (aa: ReplicationContext) => behavior(aa, ctx) } }, @@ -127,11 +127,11 @@ class AABlogExampleSpec val refDcB: ActorRef[BlogCommand] = spawn( Behaviors.setup[BlogCommand] { ctx => - ActiveActiveEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal( "cat", ReplicaId("DC-B"), Set(ReplicaId("DC-A"), ReplicaId("DC-B")), - PersistenceTestKitReadJournal.Identifier) { (aa: ActiveActiveContext) => + PersistenceTestKitReadJournal.Identifier) { (aa: ReplicationContext) => behavior(aa, ctx) } }, diff --git a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala similarity index 64% rename from akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala rename to akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala index dc0668ff6c..6d98e47bb9 100644 --- a/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ActiveActiveCompileOnlySpec.scala +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala @@ -5,11 +5,11 @@ package docs.akka.persistence.typed import akka.persistence.typed.ReplicaId -import akka.persistence.typed.scaladsl.{ActiveActiveEventSourcing, EventSourcedBehavior} +import akka.persistence.typed.scaladsl.{ EventSourcedBehavior, ReplicatedEventSourcing } import com.github.ghik.silencer.silent @silent("never used") -object ActiveActiveCompileOnlySpec { +object ReplicatedEventSourcingCompileOnlySpec { //#replicas val DCA = ReplicaId("DC-A") @@ -24,13 +24,13 @@ object ActiveActiveCompileOnlySpec { trait Event //#factory-shared - ActiveActiveEventSourcing.withSharedJournal("entityId", DCA, AllReplicas, queryPluginId) { context => + ReplicatedEventSourcing.withSharedJournal("entityId", DCA, AllReplicas, queryPluginId) { context => EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) } //#factory-shared //#factory - ActiveActiveEventSourcing("entityId", DCA, Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { context => + ReplicatedEventSourcing("entityId", DCA, Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { context => EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) } //#factory diff --git a/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ActiveActive.java b/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ReplicatedEventSourcing.java similarity index 77% rename from akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ActiveActive.java rename to akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ReplicatedEventSourcing.java index cd0ca60693..2bc39615d6 100644 --- a/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ActiveActive.java +++ b/akka-persistence-typed/src/main/java/akka/persistence/typed/serialization/ReplicatedEventSourcing.java @@ -3,12 +3,12 @@ */ // Generated by the protocol buffer compiler. DO NOT EDIT! -// source: ActiveActive.proto +// source: ReplicatedEventSourcing.proto package akka.persistence.typed.serialization; -public final class ActiveActive { - private ActiveActive() {} +public final class ReplicatedEventSourcing { + private ReplicatedEventSourcing() {} public static void registerAllExtensions( akka.protobufv3.internal.ExtensionRegistryLite registry) {} @@ -86,7 +86,7 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.EnumDescriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive.getDescriptor() + return akka.persistence.typed.serialization.ReplicatedEventSourcing.getDescriptor() .getEnumTypes() .get(0); } @@ -201,17 +201,18 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive.internal_static_Counter_descriptor; + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_Counter_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.Counter.class, - akka.persistence.typed.serialization.ActiveActive.Counter.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.Builder.class); } private int bitField0_; @@ -278,11 +279,11 @@ public final class ActiveActive { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.Counter)) { + if (!(obj instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.Counter other = - (akka.persistence.typed.serialization.ActiveActive.Counter) obj; + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter) obj; if (hasValue() != other.hasValue()) return false; if (hasValue()) { @@ -308,72 +309,74 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.Counter parseFrom( + 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.ActiveActive.Counter parseFrom( + 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.ActiveActive.Counter parseFrom( + 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.ActiveActive.Counter parseFrom( + 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.ActiveActive.Counter parseFrom(byte[] data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.Counter parseFrom( + 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.ActiveActive.Counter parseFrom( + 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.ActiveActive.Counter parseFrom( + 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.ActiveActive.Counter parseDelimitedFrom( - java.io.InputStream input) throws java.io.IOException { + 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.ActiveActive.Counter parseDelimitedFrom( - java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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.ActiveActive.Counter parseFrom( + 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.ActiveActive.Counter parseFrom( + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -391,7 +394,7 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.Counter prototype) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -411,22 +414,24 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:Counter) - akka.persistence.typed.serialization.ActiveActive.CounterOrBuilder { + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive.internal_static_Counter_descriptor; + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_Counter_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.Counter.class, - akka.persistence.typed.serialization.ActiveActive.Counter.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.Builder.class); } - // Construct using akka.persistence.typed.serialization.ActiveActive.Counter.newBuilder() + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -450,17 +455,21 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive.internal_static_Counter_descriptor; + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_Counter_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.Counter getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.Counter.getDefaultInstance(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.Counter build() { - akka.persistence.typed.serialization.ActiveActive.Counter result = buildPartial(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter result = + buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -468,9 +477,9 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.Counter buildPartial() { - akka.persistence.typed.serialization.ActiveActive.Counter result = - new akka.persistence.typed.serialization.ActiveActive.Counter(this); + 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)) { @@ -519,17 +528,20 @@ public final class ActiveActive { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.ActiveActive.Counter) { - return mergeFrom((akka.persistence.typed.serialization.ActiveActive.Counter) 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.ActiveActive.Counter other) { - if (other == akka.persistence.typed.serialization.ActiveActive.Counter.getDefaultInstance()) - 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()); } @@ -551,12 +563,13 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.Counter parsedMessage = null; + 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.ActiveActive.Counter) e.getUnfinishedMessage(); + (akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter) + e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { if (parsedMessage != null) { @@ -629,13 +642,15 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:Counter) - private static final akka.persistence.typed.serialization.ActiveActive.Counter DEFAULT_INSTANCE; + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.Counter(); + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter(); } - public static akka.persistence.typed.serialization.ActiveActive.Counter getDefaultInstance() { + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -661,7 +676,8 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.Counter getDefaultInstanceForType() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.Counter + getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } @@ -757,18 +773,19 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_CounterUpdate_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.CounterUpdate.class, - akka.persistence.typed.serialization.ActiveActive.CounterUpdate.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.Builder + .class); } private int bitField0_; @@ -835,11 +852,12 @@ public final class ActiveActive { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.CounterUpdate)) { + if (!(obj + instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.CounterUpdate other = - (akka.persistence.typed.serialization.ActiveActive.CounterUpdate) obj; + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate) obj; if (hasDelta() != other.hasDelta()) return false; if (hasDelta()) { @@ -865,60 +883,66 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate parseFrom( - java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.CounterUpdate parseFrom( - java.nio.ByteBuffer data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.CounterUpdate parseFrom( - akka.protobufv3.internal.ByteString data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.CounterUpdate parseFrom( - akka.protobufv3.internal.ByteString data, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.CounterUpdate parseFrom( - byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.CounterUpdate parseFrom( - byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.CounterUpdate parseFrom( - java.io.InputStream input) throws java.io.IOException { + 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.ActiveActive.CounterUpdate parseFrom( - java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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.ActiveActive.CounterUpdate + 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.ActiveActive.CounterUpdate + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -927,15 +951,16 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate parseFrom( - akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + 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.ActiveActive.CounterUpdate parseFrom( - akka.protobufv3.internal.CodedInputStream input, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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); } @@ -950,7 +975,7 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.CounterUpdate prototype) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -970,24 +995,25 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:CounterUpdate) - akka.persistence.typed.serialization.ActiveActive.CounterUpdateOrBuilder { + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdateOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_CounterUpdate_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.CounterUpdate.class, - akka.persistence.typed.serialization.ActiveActive.CounterUpdate.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.Builder + .class); } // Construct using - // akka.persistence.typed.serialization.ActiveActive.CounterUpdate.newBuilder() + // akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -1011,19 +1037,21 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_CounterUpdate_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.CounterUpdate + public akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.CounterUpdate.getDefaultInstance(); + return akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.CounterUpdate build() { - akka.persistence.typed.serialization.ActiveActive.CounterUpdate result = buildPartial(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate result = + buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -1031,9 +1059,10 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.CounterUpdate buildPartial() { - akka.persistence.typed.serialization.ActiveActive.CounterUpdate result = - new akka.persistence.typed.serialization.ActiveActive.CounterUpdate(this); + 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)) { @@ -1082,8 +1111,10 @@ public final class ActiveActive { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.ActiveActive.CounterUpdate) { - return mergeFrom((akka.persistence.typed.serialization.ActiveActive.CounterUpdate) 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; @@ -1091,10 +1122,10 @@ public final class ActiveActive { } public Builder mergeFrom( - akka.persistence.typed.serialization.ActiveActive.CounterUpdate other) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate other) { if (other - == akka.persistence.typed.serialization.ActiveActive.CounterUpdate.getDefaultInstance()) - return this; + == akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate + .getDefaultInstance()) return this; if (other.hasDelta()) { setDelta(other.getDelta()); } @@ -1116,12 +1147,13 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.CounterUpdate parsedMessage = null; + 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.ActiveActive.CounterUpdate) + (akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -1195,14 +1227,15 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:CounterUpdate) - private static final akka.persistence.typed.serialization.ActiveActive.CounterUpdate + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.CounterUpdate(); + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate(); } - public static akka.persistence.typed.serialization.ActiveActive.CounterUpdate + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -1229,7 +1262,7 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.CounterUpdate + public akka.persistence.typed.serialization.ReplicatedEventSourcing.CounterUpdate getDefaultInstanceForType() { return DEFAULT_INSTANCE; } @@ -1270,23 +1303,26 @@ public final class ActiveActive { * * @return The vvector. */ - akka.persistence.typed.serialization.ActiveActive.VersionVector getVvector(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVvector(); /** required .VersionVector vvector = 2; */ - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder getVvectorOrBuilder(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVvectorOrBuilder(); /** repeated .VersionVector dots = 3; */ - java.util.List getDotsList(); + java.util.List + getDotsList(); /** repeated .VersionVector dots = 3; */ - akka.persistence.typed.serialization.ActiveActive.VersionVector getDots(int index); + 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.ActiveActive.VersionVectorOrBuilder> + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> getDotsOrBuilderList(); /** repeated .VersionVector dots = 3; */ - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder getDotsOrBuilder( - int index); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getDotsOrBuilder(int index); /** * repeated string stringElements = 4; @@ -1427,14 +1463,15 @@ public final class ActiveActive { } case 18: { - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder subBuilder = - null; + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + subBuilder = null; if (((bitField0_ & 0x00000002) != 0)) { subBuilder = vvector_.toBuilder(); } vvector_ = input.readMessage( - akka.persistence.typed.serialization.ActiveActive.VersionVector.PARSER, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .PARSER, extensionRegistry); if (subBuilder != null) { subBuilder.mergeFrom(vvector_); @@ -1448,12 +1485,14 @@ public final class ActiveActive { if (!((mutable_bitField0_ & 0x00000004) != 0)) { dots_ = new java.util.ArrayList< - akka.persistence.typed.serialization.ActiveActive.VersionVector>(); + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVector>(); mutable_bitField0_ |= 0x00000004; } dots_.add( input.readMessage( - akka.persistence.typed.serialization.ActiveActive.VersionVector.PARSER, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .PARSER, extensionRegistry)); break; } @@ -1560,17 +1599,18 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive.internal_static_ORSet_descriptor; + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ORSet_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ORSet.class, - akka.persistence.typed.serialization.ActiveActive.ORSet.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder.class); } private int bitField0_; @@ -1620,7 +1660,7 @@ public final class ActiveActive { } public static final int VVECTOR_FIELD_NUMBER = 2; - private akka.persistence.typed.serialization.ActiveActive.VersionVector vvector_; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector vvector_; /** * required .VersionVector vvector = 2; * @@ -1634,29 +1674,35 @@ public final class ActiveActive { * * @return The vvector. */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector getVvector() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVvector() { return vvector_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : vvector_; } /** required .VersionVector vvector = 2; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder getVvectorOrBuilder() { return vvector_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : vvector_; } public static final int DOTS_FIELD_NUMBER = 3; - private java.util.List dots_; + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector> + dots_; /** repeated .VersionVector dots = 3; */ - public java.util.List + 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.ActiveActive.VersionVectorOrBuilder> + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> getDotsOrBuilderList() { return dots_; } @@ -1665,11 +1711,12 @@ public final class ActiveActive { return dots_.size(); } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector getDots(int index) { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getDots( + int index) { return dots_.get(index); } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder getDotsOrBuilder(int index) { return dots_.get(index); } @@ -1933,11 +1980,11 @@ public final class ActiveActive { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.ORSet)) { + if (!(obj instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.ORSet other = - (akka.persistence.typed.serialization.ActiveActive.ORSet) obj; + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet) obj; if (hasOriginDc() != other.hasOriginDc()) return false; if (hasOriginDc()) { @@ -1996,72 +2043,74 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.ORSet parseFrom( + 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.ActiveActive.ORSet parseFrom( + 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.ActiveActive.ORSet parseFrom( + 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.ActiveActive.ORSet parseFrom( + 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.ActiveActive.ORSet parseFrom(byte[] data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.ORSet parseFrom( + 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.ActiveActive.ORSet parseFrom( + 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.ActiveActive.ORSet parseFrom( + 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.ActiveActive.ORSet parseDelimitedFrom( - java.io.InputStream input) throws java.io.IOException { + 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.ActiveActive.ORSet parseDelimitedFrom( - java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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.ActiveActive.ORSet parseFrom( + 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.ActiveActive.ORSet parseFrom( + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -2079,7 +2128,7 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.ORSet prototype) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -2099,22 +2148,24 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:ORSet) - akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive.internal_static_ORSet_descriptor; + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ORSet_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ORSet.class, - akka.persistence.typed.serialization.ActiveActive.ORSet.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder.class); } - // Construct using akka.persistence.typed.serialization.ActiveActive.ORSet.newBuilder() + // Construct using + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -2166,17 +2217,20 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive.internal_static_ORSet_descriptor; + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .internal_static_ORSet_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSet getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + getDefaultInstanceForType() { + return akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSet build() { - akka.persistence.typed.serialization.ActiveActive.ORSet result = buildPartial(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -2184,9 +2238,9 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSet buildPartial() { - akka.persistence.typed.serialization.ActiveActive.ORSet result = - new akka.persistence.typed.serialization.ActiveActive.ORSet(this); + 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)) { @@ -2276,17 +2330,20 @@ public final class ActiveActive { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.ActiveActive.ORSet) { - return mergeFrom((akka.persistence.typed.serialization.ActiveActive.ORSet) 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.ActiveActive.ORSet other) { - if (other == akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance()) - 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_; @@ -2413,12 +2470,13 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.ORSet parsedMessage = null; + 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.ActiveActive.ORSet) e.getUnfinishedMessage(); + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet) + e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { if (parsedMessage != null) { @@ -2515,11 +2573,11 @@ public final class ActiveActive { return this; } - private akka.persistence.typed.serialization.ActiveActive.VersionVector vvector_; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector vvector_; private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> vvectorBuilder_; /** * required .VersionVector vvector = 2; @@ -2534,10 +2592,12 @@ public final class ActiveActive { * * @return The vvector. */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector getVvector() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVvector() { if (vvectorBuilder_ == null) { return vvector_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : vvector_; } else { return vvectorBuilder_.getMessage(); @@ -2545,7 +2605,7 @@ public final class ActiveActive { } /** required .VersionVector vvector = 2; */ public Builder setVvector( - akka.persistence.typed.serialization.ActiveActive.VersionVector value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { if (vvectorBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2560,7 +2620,8 @@ public final class ActiveActive { } /** required .VersionVector vvector = 2; */ public Builder setVvector( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { if (vvectorBuilder_ == null) { vvector_ = builderForValue.build(); onChanged(); @@ -2572,15 +2633,16 @@ public final class ActiveActive { } /** required .VersionVector vvector = 2; */ public Builder mergeVvector( - akka.persistence.typed.serialization.ActiveActive.VersionVector value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { if (vvectorBuilder_ == null) { if (((bitField0_ & 0x00000002) != 0) && vvector_ != null && vvector_ - != akka.persistence.typed.serialization.ActiveActive.VersionVector + != akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector .getDefaultInstance()) { vvector_ = - akka.persistence.typed.serialization.ActiveActive.VersionVector.newBuilder(vvector_) + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .newBuilder(vvector_) .mergeFrom(value) .buildPartial(); } else { @@ -2605,61 +2667,66 @@ public final class ActiveActive { return this; } /** required .VersionVector vvector = 2; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder getVvectorBuilder() { bitField0_ |= 0x00000002; onChanged(); return getVvectorFieldBuilder().getBuilder(); } /** required .VersionVector vvector = 2; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder getVvectorOrBuilder() { if (vvectorBuilder_ != null) { return vvectorBuilder_.getMessageOrBuilder(); } else { return vvector_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : vvector_; } } /** required .VersionVector vvector = 2; */ private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + 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.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder>( - getVvector(), getParentForChildren(), isClean()); + 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 + 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.ActiveActive.VersionVector>(dots_); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector>( + dots_); bitField0_ |= 0x00000004; } } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + 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 + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector> getDotsList() { if (dotsBuilder_ == null) { return java.util.Collections.unmodifiableList(dots_); @@ -2676,7 +2743,8 @@ public final class ActiveActive { } } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector getDots(int index) { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getDots( + int index) { if (dotsBuilder_ == null) { return dots_.get(index); } else { @@ -2685,7 +2753,8 @@ public final class ActiveActive { } /** repeated .VersionVector dots = 3; */ public Builder setDots( - int index, akka.persistence.typed.serialization.ActiveActive.VersionVector value) { + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { if (dotsBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2701,7 +2770,8 @@ public final class ActiveActive { /** repeated .VersionVector dots = 3; */ public Builder setDots( int index, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { if (dotsBuilder_ == null) { ensureDotsIsMutable(); dots_.set(index, builderForValue.build()); @@ -2713,7 +2783,7 @@ public final class ActiveActive { } /** repeated .VersionVector dots = 3; */ public Builder addDots( - akka.persistence.typed.serialization.ActiveActive.VersionVector value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { if (dotsBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2728,7 +2798,8 @@ public final class ActiveActive { } /** repeated .VersionVector dots = 3; */ public Builder addDots( - int index, akka.persistence.typed.serialization.ActiveActive.VersionVector value) { + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { if (dotsBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -2743,7 +2814,8 @@ public final class ActiveActive { } /** repeated .VersionVector dots = 3; */ public Builder addDots( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { if (dotsBuilder_ == null) { ensureDotsIsMutable(); dots_.add(builderForValue.build()); @@ -2756,7 +2828,8 @@ public final class ActiveActive { /** repeated .VersionVector dots = 3; */ public Builder addDots( int index, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { if (dotsBuilder_ == null) { ensureDotsIsMutable(); dots_.add(index, builderForValue.build()); @@ -2769,7 +2842,8 @@ public final class ActiveActive { /** repeated .VersionVector dots = 3; */ public Builder addAllDots( java.lang.Iterable< - ? extends akka.persistence.typed.serialization.ActiveActive.VersionVector> + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector> values) { if (dotsBuilder_ == null) { ensureDotsIsMutable(); @@ -2803,12 +2877,12 @@ public final class ActiveActive { return this; } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder getDotsBuilder( - int index) { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + getDotsBuilder(int index) { return getDotsFieldBuilder().getBuilder(index); } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder getDotsOrBuilder(int index) { if (dotsBuilder_ == null) { return dots_.get(index); @@ -2818,7 +2892,9 @@ public final class ActiveActive { } /** repeated .VersionVector dots = 3; */ public java.util.List< - ? extends akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVectorOrBuilder> getDotsOrBuilderList() { if (dotsBuilder_ != null) { return dotsBuilder_.getMessageOrBuilderList(); @@ -2827,39 +2903,42 @@ public final class ActiveActive { } } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder addDotsBuilder() { return getDotsFieldBuilder() .addBuilder( - akka.persistence.typed.serialization.ActiveActive.VersionVector + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector .getDefaultInstance()); } /** repeated .VersionVector dots = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder addDotsBuilder( - int index) { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + addDotsBuilder(int index) { return getDotsFieldBuilder() .addBuilder( index, - akka.persistence.typed.serialization.ActiveActive.VersionVector + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector .getDefaultInstance()); } /** repeated .VersionVector dots = 3; */ - public java.util.List + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder> getDotsBuilderList() { return getDotsFieldBuilder().getBuilderList(); } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + 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.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder>( + 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; } @@ -3373,13 +3452,15 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:ORSet) - private static final akka.persistence.typed.serialization.ActiveActive.ORSet DEFAULT_INSTANCE; + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.ORSet(); + DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet(); } - public static akka.persistence.typed.serialization.ActiveActive.ORSet getDefaultInstance() { + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -3405,7 +3486,8 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSet getDefaultInstanceForType() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } @@ -3416,19 +3498,22 @@ public final class ActiveActive { akka.protobufv3.internal.MessageOrBuilder { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - java.util.List + java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry> getEntriesList(); /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry getEntries(int index); + 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.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder> getEntriesOrBuilderList(); /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.EntryOrBuilder getEntriesOrBuilder(int index); } /** Protobuf type {@code ORSetDeltaGroup} */ @@ -3482,14 +3567,14 @@ public final class ActiveActive { if (!((mutable_bitField0_ & 0x00000001) != 0)) { entries_ = new java.util.ArrayList< - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup - .Entry>(); + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ORSetDeltaGroup.Entry>(); mutable_bitField0_ |= 0x00000001; } entries_.add( input.readMessage( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry - .PARSER, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry.PARSER, extensionRegistry)); break; } @@ -3517,18 +3602,19 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ORSetDeltaGroup_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.class, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Builder + .class); } public interface EntryOrBuilder @@ -3547,7 +3633,7 @@ public final class ActiveActive { * * @return The operation. */ - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp getOperation(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp getOperation(); /** * required .ORSet underlying = 2; @@ -3560,9 +3646,10 @@ public final class ActiveActive { * * @return The underlying. */ - akka.persistence.typed.serialization.ActiveActive.ORSet getUnderlying(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet getUnderlying(); /** required .ORSet underlying = 2; */ - akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder getUnderlyingOrBuilder(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder + getUnderlyingOrBuilder(); } /** Protobuf type {@code ORSetDeltaGroup.Entry} */ public static final class Entry extends akka.protobufv3.internal.GeneratedMessageV3 @@ -3614,9 +3701,9 @@ public final class ActiveActive { { int rawValue = input.readEnum(); @SuppressWarnings("deprecation") - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp value = - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.valueOf( - rawValue); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp value = + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp + .valueOf(rawValue); if (value == null) { unknownFields.mergeVarintField(1, rawValue); } else { @@ -3627,13 +3714,14 @@ public final class ActiveActive { } case 18: { - akka.persistence.typed.serialization.ActiveActive.ORSet.Builder subBuilder = null; + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder + subBuilder = null; if (((bitField0_ & 0x00000002) != 0)) { subBuilder = underlying_.toBuilder(); } underlying_ = input.readMessage( - akka.persistence.typed.serialization.ActiveActive.ORSet.PARSER, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.PARSER, extensionRegistry); if (subBuilder != null) { subBuilder.mergeFrom(underlying_); @@ -3663,19 +3751,20 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.class, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder - .class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder.class); } private int bitField0_; @@ -3694,17 +3783,19 @@ public final class ActiveActive { * * @return The operation. */ - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp getOperation() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp + getOperation() { @SuppressWarnings("deprecation") - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp result = - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.valueOf(operation_); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp result = + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp.valueOf( + operation_); return result == null - ? akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.Add + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp.Add : result; } public static final int UNDERLYING_FIELD_NUMBER = 2; - private akka.persistence.typed.serialization.ActiveActive.ORSet underlying_; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet underlying_; /** * required .ORSet underlying = 2; * @@ -3718,16 +3809,18 @@ public final class ActiveActive { * * @return The underlying. */ - public akka.persistence.typed.serialization.ActiveActive.ORSet getUnderlying() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet getUnderlying() { return underlying_ == null - ? akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance() : underlying_; } /** required .ORSet underlying = 2; */ - public akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder getUnderlyingOrBuilder() { return underlying_ == null - ? akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance() : underlying_; } @@ -3790,11 +3883,13 @@ public final class ActiveActive { return true; } if (!(obj - instanceof akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry)) { + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry other = - (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry) 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()) { @@ -3828,13 +3923,15 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + 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.ActiveActive.ORSetDeltaGroup.Entry + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry parseFrom( java.nio.ByteBuffer data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -3842,13 +3939,15 @@ public final class ActiveActive { return PARSER.parseFrom(data, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + 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.ActiveActive.ORSetDeltaGroup.Entry + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry parseFrom( akka.protobufv3.internal.ByteString data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -3856,23 +3955,27 @@ public final class ActiveActive { return PARSER.parseFrom(data, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + 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.ActiveActive.ORSetDeltaGroup.Entry + 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.ActiveActive.ORSetDeltaGroup.Entry + 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.ActiveActive.ORSetDeltaGroup.Entry + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry parseFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -3881,13 +3984,15 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + 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.ActiveActive.ORSetDeltaGroup.Entry + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -3896,12 +4001,14 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + 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.ActiveActive.ORSetDeltaGroup.Entry + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -3920,7 +4027,8 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry prototype) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -3940,25 +4048,27 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:ORSetDeltaGroup.Entry) - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ORSetDeltaGroup_Entry_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.class, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder - .class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder.class); } // Construct using - // akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.newBuilder() + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -3990,21 +4100,22 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ORSetDeltaGroup_Entry_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + return akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry build() { - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry result = - buildPartial(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -4012,10 +4123,12 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry buildPartial() { - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry result = - new akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry(this); + 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)) { @@ -4073,9 +4186,11 @@ public final class ActiveActive { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { if (other - instanceof akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry) { + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry) { return mergeFrom( - (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry) other); + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry) + other); } else { super.mergeFrom(other); return this; @@ -4083,9 +4198,10 @@ public final class ActiveActive { } public Builder mergeFrom( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry other) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + other) { if (other - == akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + == akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry .getDefaultInstance()) return this; if (other.hasOperation()) { setOperation(other.getOperation()); @@ -4117,13 +4233,13 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry parsedMessage = - null; + 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.ActiveActive.ORSetDeltaGroup.Entry) + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -4150,12 +4266,14 @@ public final class ActiveActive { * * @return The operation. */ - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp getOperation() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp + getOperation() { @SuppressWarnings("deprecation") - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp result = - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.valueOf(operation_); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp result = + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp.valueOf( + operation_); return result == null - ? akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp.Add + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp.Add : result; } /** @@ -4165,7 +4283,7 @@ public final class ActiveActive { * @return This builder for chaining. */ public Builder setOperation( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaOp value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaOp value) { if (value == null) { throw new NullPointerException(); } @@ -4186,11 +4304,11 @@ public final class ActiveActive { return this; } - private akka.persistence.typed.serialization.ActiveActive.ORSet underlying_; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet underlying_; private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.ORSet, - akka.persistence.typed.serialization.ActiveActive.ORSet.Builder, - akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder> underlyingBuilder_; /** * required .ORSet underlying = 2; @@ -4205,10 +4323,11 @@ public final class ActiveActive { * * @return The underlying. */ - public akka.persistence.typed.serialization.ActiveActive.ORSet getUnderlying() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet getUnderlying() { if (underlyingBuilder_ == null) { return underlying_ == null - ? akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance() : underlying_; } else { return underlyingBuilder_.getMessage(); @@ -4216,7 +4335,7 @@ public final class ActiveActive { } /** required .ORSet underlying = 2; */ public Builder setUnderlying( - akka.persistence.typed.serialization.ActiveActive.ORSet value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet value) { if (underlyingBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -4231,7 +4350,8 @@ public final class ActiveActive { } /** required .ORSet underlying = 2; */ public Builder setUnderlying( - akka.persistence.typed.serialization.ActiveActive.ORSet.Builder builderForValue) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder + builderForValue) { if (underlyingBuilder_ == null) { underlying_ = builderForValue.build(); onChanged(); @@ -4243,15 +4363,16 @@ public final class ActiveActive { } /** required .ORSet underlying = 2; */ public Builder mergeUnderlying( - akka.persistence.typed.serialization.ActiveActive.ORSet value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet value) { if (underlyingBuilder_ == null) { if (((bitField0_ & 0x00000002) != 0) && underlying_ != null && underlying_ - != akka.persistence.typed.serialization.ActiveActive.ORSet + != akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet .getDefaultInstance()) { underlying_ = - akka.persistence.typed.serialization.ActiveActive.ORSet.newBuilder(underlying_) + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.newBuilder( + underlying_) .mergeFrom(value) .buildPartial(); } else { @@ -4276,35 +4397,36 @@ public final class ActiveActive { return this; } /** required .ORSet underlying = 2; */ - public akka.persistence.typed.serialization.ActiveActive.ORSet.Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder getUnderlyingBuilder() { bitField0_ |= 0x00000002; onChanged(); return getUnderlyingFieldBuilder().getBuilder(); } /** required .ORSet underlying = 2; */ - public akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder getUnderlyingOrBuilder() { if (underlyingBuilder_ != null) { return underlyingBuilder_.getMessageOrBuilder(); } else { return underlying_ == null - ? akka.persistence.typed.serialization.ActiveActive.ORSet.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet + .getDefaultInstance() : underlying_; } } /** required .ORSet underlying = 2; */ private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.ORSet, - akka.persistence.typed.serialization.ActiveActive.ORSet.Builder, - akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder> + 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.ActiveActive.ORSet, - akka.persistence.typed.serialization.ActiveActive.ORSet.Builder, - akka.persistence.typed.serialization.ActiveActive.ORSetOrBuilder>( + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSet.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetOrBuilder>( getUnderlying(), getParentForChildren(), isClean()); underlying_ = null; } @@ -4327,15 +4449,18 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:ORSetDeltaGroup.Entry) - private static final akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .ORSetDeltaGroup.Entry DEFAULT_INSTANCE; static { DEFAULT_INSTANCE = - new akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry(); + new akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry(); } - public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -4362,24 +4487,27 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } public static final int ENTRIES_FIELD_NUMBER = 1; - private java.util.List + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry> entries_; /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public java.util.List + 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.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder> getEntriesOrBuilderList() { return entries_; } @@ -4388,12 +4516,13 @@ public final class ActiveActive { return entries_.size(); } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry getEntries( - int index) { + 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.ActiveActive.ORSetDeltaGroup.EntryOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder getEntriesOrBuilder(int index) { return entries_.get(index); } @@ -4444,11 +4573,13 @@ public final class ActiveActive { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup)) { + if (!(obj + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup other = - (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup) 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; @@ -4471,60 +4602,66 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup parseFrom( - java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.ORSetDeltaGroup parseFrom( - java.nio.ByteBuffer data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.ORSetDeltaGroup parseFrom( - akka.protobufv3.internal.ByteString data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.ORSetDeltaGroup parseFrom( - akka.protobufv3.internal.ByteString data, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.ORSetDeltaGroup parseFrom( - byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.ORSetDeltaGroup parseFrom( - byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.ORSetDeltaGroup parseFrom( - java.io.InputStream input) throws java.io.IOException { + 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.ActiveActive.ORSetDeltaGroup parseFrom( - java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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.ActiveActive.ORSetDeltaGroup + 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.ActiveActive.ORSetDeltaGroup + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -4533,15 +4670,16 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup parseFrom( - akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + 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.ActiveActive.ORSetDeltaGroup parseFrom( - akka.protobufv3.internal.CodedInputStream input, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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); } @@ -4556,7 +4694,7 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup prototype) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -4576,24 +4714,25 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:ORSetDeltaGroup) - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroupOrBuilder { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroupOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ORSetDeltaGroup_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.class, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Builder + .class); } // Construct using - // akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.newBuilder() + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -4623,20 +4762,21 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ORSetDeltaGroup_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + return akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup build() { - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup result = buildPartial(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup result = + buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -4644,9 +4784,10 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup buildPartial() { - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup result = - new akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup(this); + 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)) { @@ -4698,9 +4839,11 @@ public final class ActiveActive { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup) { + if (other + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup) { return mergeFrom( - (akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup) other); + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup) other); } else { super.mergeFrom(other); return this; @@ -4708,9 +4851,9 @@ public final class ActiveActive { } public Builder mergeFrom( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup other) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup other) { if (other - == akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + == akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup .getDefaultInstance()) return this; if (entriesBuilder_ == null) { if (!other.entries_.isEmpty()) { @@ -4759,12 +4902,13 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup parsedMessage = null; + 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.ActiveActive.ORSetDeltaGroup) + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -4778,27 +4922,30 @@ public final class ActiveActive { private int bitField0_; private java.util.List< - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry> + 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.ActiveActive.ORSetDeltaGroup.Entry>( - entries_); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry>(entries_); bitField0_ |= 0x00000001; } } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> + 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 + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry> getEntriesList() { if (entriesBuilder_ == null) { return java.util.Collections.unmodifiableList(entries_); @@ -4815,8 +4962,8 @@ public final class ActiveActive { } } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry getEntries( - int index) { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + getEntries(int index) { if (entriesBuilder_ == null) { return entries_.get(index); } else { @@ -4826,7 +4973,8 @@ public final class ActiveActive { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder setEntries( int index, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -4842,7 +4990,7 @@ public final class ActiveActive { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder setEntries( int index, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry.Builder builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -4855,7 +5003,8 @@ public final class ActiveActive { } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addEntries( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -4871,7 +5020,8 @@ public final class ActiveActive { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addEntries( int index, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -4886,7 +5036,7 @@ public final class ActiveActive { } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addEntries( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry.Builder builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -4900,7 +5050,7 @@ public final class ActiveActive { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addEntries( int index, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry.Builder builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -4914,7 +5064,9 @@ public final class ActiveActive { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public Builder addAllEntries( java.lang.Iterable< - ? extends akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry> + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .Entry> values) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -4948,12 +5100,14 @@ public final class ActiveActive { return this; } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder + 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.ActiveActive.ORSetDeltaGroup.EntryOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder getEntriesOrBuilder(int index) { if (entriesBuilder_ == null) { return entries_.get(index); @@ -4964,7 +5118,8 @@ public final class ActiveActive { /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public java.util.List< ? extends - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup + .EntryOrBuilder> getEntriesOrBuilderList() { if (entriesBuilder_ != null) { return entriesBuilder_.getMessageOrBuilderList(); @@ -4973,40 +5128,48 @@ public final class ActiveActive { } } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder addEntriesBuilder() { return getEntriesFieldBuilder() .addBuilder( - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry .getDefaultInstance()); } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder addEntriesBuilder(int index) { return getEntriesFieldBuilder() .addBuilder( index, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry .getDefaultInstance()); } /** repeated .ORSetDeltaGroup.Entry entries = 1; */ public java.util.List< - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup.Entry + .Builder> getEntriesBuilderList() { return getEntriesFieldBuilder().getBuilderList(); } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder> + 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.ActiveActive.ORSetDeltaGroup.Entry, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.Entry.Builder, - akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup.EntryOrBuilder>( + 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; } @@ -5029,14 +5192,16 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:ORSetDeltaGroup) - private static final akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .ORSetDeltaGroup DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup(); + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup(); } - public static akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -5063,7 +5228,7 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ORSetDeltaGroup + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ORSetDeltaGroup getDefaultInstanceForType() { return DEFAULT_INSTANCE; } @@ -5075,19 +5240,21 @@ public final class ActiveActive { akka.protobufv3.internal.MessageOrBuilder { /** repeated .VersionVector.Entry entries = 1; */ - java.util.List + java.util.List getEntriesList(); /** repeated .VersionVector.Entry entries = 1; */ - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry getEntries(int index); + 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.ActiveActive.VersionVector.EntryOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder> getEntriesOrBuilderList(); /** repeated .VersionVector.Entry entries = 1; */ - akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.EntryOrBuilder getEntriesOrBuilder(int index); } /** Protobuf type {@code VersionVector} */ @@ -5141,13 +5308,14 @@ public final class ActiveActive { if (!((mutable_bitField0_ & 0x00000001) != 0)) { entries_ = new java.util.ArrayList< - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry>(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Entry>(); mutable_bitField0_ |= 0x00000001; } entries_.add( input.readMessage( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry - .PARSER, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Entry.PARSER, extensionRegistry)); break; } @@ -5175,18 +5343,19 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_VersionVector_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.VersionVector.class, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + .class); } public interface EntryOrBuilder @@ -5306,19 +5475,20 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_VersionVector_Entry_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.class, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder - .class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder.class); } private int bitField0_; @@ -5441,11 +5611,12 @@ public final class ActiveActive { return true; } if (!(obj - instanceof akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry)) { + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry other = - (akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry) 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()) { @@ -5479,62 +5650,66 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parseFrom( - java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector.Entry parseFrom( - java.nio.ByteBuffer data, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector.Entry parseFrom( - akka.protobufv3.internal.ByteString data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector.Entry parseFrom( - akka.protobufv3.internal.ByteString data, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector.Entry parseFrom( - byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector.Entry parseFrom( - byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector.Entry parseFrom( - java.io.InputStream input) throws java.io.IOException { + 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.ActiveActive.VersionVector.Entry parseFrom( - java.io.InputStream input, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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.ActiveActive.VersionVector.Entry + 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.ActiveActive.VersionVector.Entry + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -5543,15 +5718,16 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parseFrom( - akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + 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.ActiveActive.VersionVector.Entry parseFrom( - akka.protobufv3.internal.CodedInputStream input, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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); } @@ -5566,7 +5742,8 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry prototype) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -5586,25 +5763,27 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:VersionVector.Entry) - akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_VersionVector_Entry_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.class, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder - .class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder.class); } // Construct using - // akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.newBuilder() + // akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -5630,20 +5809,21 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_VersionVector_Entry_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + return akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry build() { - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry result = + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); @@ -5652,10 +5832,11 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry buildPartial() { - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry result = - new akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry(this); + 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)) { @@ -5709,9 +5890,11 @@ public final class ActiveActive { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { if (other - instanceof akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry) { + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry) { return mergeFrom( - (akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry) other); + (akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry) + other); } else { super.mergeFrom(other); return this; @@ -5719,9 +5902,10 @@ public final class ActiveActive { } public Builder mergeFrom( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry other) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + other) { if (other - == akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + == akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry .getDefaultInstance()) return this; if (other.hasKey()) { bitField0_ |= 0x00000001; @@ -5752,13 +5936,13 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry parsedMessage = - null; + 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.ActiveActive.VersionVector.Entry) + (akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -5913,15 +6097,16 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:VersionVector.Entry) - private static final akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVector.Entry DEFAULT_INSTANCE; static { DEFAULT_INSTANCE = - new akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry(); + new akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry(); } - public static akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -5948,24 +6133,27 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry getDefaultInstanceForType() { return DEFAULT_INSTANCE; } } public static final int ENTRIES_FIELD_NUMBER = 1; - private java.util.List + private java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry> entries_; /** repeated .VersionVector.Entry entries = 1; */ - public java.util.List + 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.ActiveActive.VersionVector.EntryOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder> getEntriesOrBuilderList() { return entries_; } @@ -5974,12 +6162,12 @@ public final class ActiveActive { return entries_.size(); } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry getEntries( - int index) { + 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.ActiveActive.VersionVector.EntryOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.EntryOrBuilder getEntriesOrBuilder(int index) { return entries_.get(index); } @@ -6030,11 +6218,12 @@ public final class ActiveActive { if (obj == this) { return true; } - if (!(obj instanceof akka.persistence.typed.serialization.ActiveActive.VersionVector)) { + if (!(obj + instanceof akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.VersionVector other = - (akka.persistence.typed.serialization.ActiveActive.VersionVector) 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; @@ -6057,60 +6246,66 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.VersionVector parseFrom( - java.nio.ByteBuffer data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector parseFrom( - java.nio.ByteBuffer data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector parseFrom( - akka.protobufv3.internal.ByteString data) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector parseFrom( - akka.protobufv3.internal.ByteString data, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector parseFrom( - byte[] data) throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector parseFrom( - byte[] data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws akka.protobufv3.internal.InvalidProtocolBufferException { + 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.ActiveActive.VersionVector parseFrom( - java.io.InputStream input) throws java.io.IOException { + 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.ActiveActive.VersionVector parseFrom( - java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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.ActiveActive.VersionVector + 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.ActiveActive.VersionVector + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -6119,15 +6314,16 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.VersionVector parseFrom( - akka.protobufv3.internal.CodedInputStream input) throws java.io.IOException { + 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.ActiveActive.VersionVector parseFrom( - akka.protobufv3.internal.CodedInputStream input, - akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { + 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); } @@ -6142,7 +6338,7 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.VersionVector prototype) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -6162,24 +6358,25 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:VersionVector) - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_VersionVector_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.VersionVector.class, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + .class); } // Construct using - // akka.persistence.typed.serialization.ActiveActive.VersionVector.newBuilder() + // akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -6209,19 +6406,21 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_VersionVector_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.VersionVector + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance(); + return akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.VersionVector build() { - akka.persistence.typed.serialization.ActiveActive.VersionVector result = buildPartial(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector result = + buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -6229,9 +6428,10 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.VersionVector buildPartial() { - akka.persistence.typed.serialization.ActiveActive.VersionVector result = - new akka.persistence.typed.serialization.ActiveActive.VersionVector(this); + 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)) { @@ -6283,8 +6483,10 @@ public final class ActiveActive { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { - if (other instanceof akka.persistence.typed.serialization.ActiveActive.VersionVector) { - return mergeFrom((akka.persistence.typed.serialization.ActiveActive.VersionVector) 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; @@ -6292,10 +6494,10 @@ public final class ActiveActive { } public Builder mergeFrom( - akka.persistence.typed.serialization.ActiveActive.VersionVector other) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector other) { if (other - == akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance()) - return this; + == akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance()) return this; if (entriesBuilder_ == null) { if (!other.entries_.isEmpty()) { if (entries_.isEmpty()) { @@ -6343,12 +6545,13 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.VersionVector parsedMessage = null; + 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.ActiveActive.VersionVector) + (akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -6361,26 +6564,31 @@ public final class ActiveActive { private int bitField0_; - private java.util.List + 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.ActiveActive.VersionVector.Entry>(entries_); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry>( + entries_); bitField0_ |= 0x00000001; } } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder> + 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 + public java.util.List< + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry> getEntriesList() { if (entriesBuilder_ == null) { return java.util.Collections.unmodifiableList(entries_); @@ -6397,8 +6605,8 @@ public final class ActiveActive { } } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry getEntries( - int index) { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + getEntries(int index) { if (entriesBuilder_ == null) { return entries_.get(index); } else { @@ -6407,7 +6615,8 @@ public final class ActiveActive { } /** repeated .VersionVector.Entry entries = 1; */ public Builder setEntries( - int index, akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry value) { + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -6423,7 +6632,7 @@ public final class ActiveActive { /** repeated .VersionVector.Entry entries = 1; */ public Builder setEntries( int index, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry.Builder builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -6436,7 +6645,7 @@ public final class ActiveActive { } /** repeated .VersionVector.Entry entries = 1; */ public Builder addEntries( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -6451,7 +6660,8 @@ public final class ActiveActive { } /** repeated .VersionVector.Entry entries = 1; */ public Builder addEntries( - int index, akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry value) { + int index, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry value) { if (entriesBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -6466,7 +6676,7 @@ public final class ActiveActive { } /** repeated .VersionVector.Entry entries = 1; */ public Builder addEntries( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry.Builder builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -6480,7 +6690,7 @@ public final class ActiveActive { /** repeated .VersionVector.Entry entries = 1; */ public Builder addEntries( int index, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry.Builder builderForValue) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -6494,7 +6704,9 @@ public final class ActiveActive { /** repeated .VersionVector.Entry entries = 1; */ public Builder addAllEntries( java.lang.Iterable< - ? extends akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry> + ? extends + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Entry> values) { if (entriesBuilder_ == null) { ensureEntriesIsMutable(); @@ -6528,12 +6740,14 @@ public final class ActiveActive { return this; } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + 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.ActiveActive.VersionVector.EntryOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder getEntriesOrBuilder(int index) { if (entriesBuilder_ == null) { return entries_.get(index); @@ -6544,7 +6758,8 @@ public final class ActiveActive { /** repeated .VersionVector.Entry entries = 1; */ public java.util.List< ? extends - akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .EntryOrBuilder> getEntriesOrBuilderList() { if (entriesBuilder_ != null) { return entriesBuilder_.getMessageOrBuilderList(); @@ -6553,40 +6768,47 @@ public final class ActiveActive { } } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder addEntriesBuilder() { return getEntriesFieldBuilder() .addBuilder( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry .getDefaultInstance()); } /** repeated .VersionVector.Entry entries = 1; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder addEntriesBuilder(int index) { return getEntriesFieldBuilder() .addBuilder( index, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry .getDefaultInstance()); } /** repeated .VersionVector.Entry entries = 1; */ public java.util.List< - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Entry + .Builder> getEntriesBuilderList() { return getEntriesFieldBuilder().getBuilderList(); } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder> + 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.ActiveActive.VersionVector.Entry, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Entry.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVector.EntryOrBuilder>( + 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; } @@ -6609,14 +6831,15 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:VersionVector) - private static final akka.persistence.typed.serialization.ActiveActive.VersionVector + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector DEFAULT_INSTANCE; static { - DEFAULT_INSTANCE = new akka.persistence.typed.serialization.ActiveActive.VersionVector(); + DEFAULT_INSTANCE = + new akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector(); } - public static akka.persistence.typed.serialization.ActiveActive.VersionVector + public static akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -6643,7 +6866,7 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.VersionVector + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getDefaultInstanceForType() { return DEFAULT_INSTANCE; } @@ -6697,9 +6920,9 @@ public final class ActiveActive { * * @return The versionVector. */ - akka.persistence.typed.serialization.ActiveActive.VersionVector getVersionVector(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVersionVector(); /** required .VersionVector versionVector = 3; */ - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder getVersionVectorOrBuilder(); /** @@ -6778,14 +7001,15 @@ public final class ActiveActive { } case 26: { - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder subBuilder = - null; + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + subBuilder = null; if (((bitField0_ & 0x00000004) != 0)) { subBuilder = versionVector_.toBuilder(); } versionVector_ = input.readMessage( - akka.persistence.typed.serialization.ActiveActive.VersionVector.PARSER, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .PARSER, extensionRegistry); if (subBuilder != null) { subBuilder.mergeFrom(versionVector_); @@ -6821,19 +7045,20 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ReplicatedEventMetadata_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata.class, - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata.Builder - .class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .Builder.class); } private int bitField0_; @@ -6902,7 +7127,8 @@ public final class ActiveActive { } public static final int VERSIONVECTOR_FIELD_NUMBER = 3; - private akka.persistence.typed.serialization.ActiveActive.VersionVector versionVector_; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + versionVector_; /** * required .VersionVector versionVector = 3; * @@ -6916,16 +7142,19 @@ public final class ActiveActive { * * @return The versionVector. */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector getVersionVector() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVersionVector() { return versionVector_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : versionVector_; } /** required .VersionVector versionVector = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder getVersionVectorOrBuilder() { return versionVector_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : versionVector_; } @@ -7028,11 +7257,13 @@ public final class ActiveActive { return true; } if (!(obj - instanceof akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata)) { + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata other = - (akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata) obj; + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata) + obj; if (hasOriginReplica() != other.hasOriginReplica()) return false; if (hasOriginReplica()) { @@ -7082,13 +7313,15 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + 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.ActiveActive.ReplicatedEventMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata parseFrom( java.nio.ByteBuffer data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -7096,13 +7329,15 @@ public final class ActiveActive { return PARSER.parseFrom(data, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + 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.ActiveActive.ReplicatedEventMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata parseFrom( akka.protobufv3.internal.ByteString data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -7110,23 +7345,27 @@ public final class ActiveActive { return PARSER.parseFrom(data, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + 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.ActiveActive.ReplicatedEventMetadata + 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.ActiveActive.ReplicatedEventMetadata + 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.ActiveActive.ReplicatedEventMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata parseFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -7135,13 +7374,15 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + 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.ActiveActive.ReplicatedEventMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -7150,12 +7391,14 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + 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.ActiveActive.ReplicatedEventMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -7174,7 +7417,8 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata prototype) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -7194,25 +7438,27 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:ReplicatedEventMetadata) - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadataOrBuilder { + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadataOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ReplicatedEventMetadata_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata.class, - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata.Builder - .class); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .class, + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + .Builder.class); } // Construct using - // akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata.newBuilder() + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -7248,21 +7494,22 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ReplicatedEventMetadata_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + return akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata .getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata build() { - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata result = - buildPartial(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -7270,10 +7517,12 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata buildPartial() { - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata result = - new akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata(this); + 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)) { @@ -7339,9 +7588,11 @@ public final class ActiveActive { @java.lang.Override public Builder mergeFrom(akka.protobufv3.internal.Message other) { if (other - instanceof akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata) { + instanceof + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata) { return mergeFrom( - (akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata) other); + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata) + other); } else { super.mergeFrom(other); return this; @@ -7349,9 +7600,10 @@ public final class ActiveActive { } public Builder mergeFrom( - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata other) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata + other) { if (other - == akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + == akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata .getDefaultInstance()) return this; if (other.hasOriginReplica()) { bitField0_ |= 0x00000001; @@ -7397,13 +7649,13 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata parsedMessage = - null; + 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.ActiveActive.ReplicatedEventMetadata) + (akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -7542,11 +7794,12 @@ public final class ActiveActive { return this; } - private akka.persistence.typed.serialization.ActiveActive.VersionVector versionVector_; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + versionVector_; private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> versionVectorBuilder_; /** * required .VersionVector versionVector = 3; @@ -7561,10 +7814,12 @@ public final class ActiveActive { * * @return The versionVector. */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector getVersionVector() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVersionVector() { if (versionVectorBuilder_ == null) { return versionVector_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : versionVector_; } else { return versionVectorBuilder_.getMessage(); @@ -7572,7 +7827,7 @@ public final class ActiveActive { } /** required .VersionVector versionVector = 3; */ public Builder setVersionVector( - akka.persistence.typed.serialization.ActiveActive.VersionVector value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { if (versionVectorBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -7587,7 +7842,8 @@ public final class ActiveActive { } /** required .VersionVector versionVector = 3; */ public Builder setVersionVector( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { if (versionVectorBuilder_ == null) { versionVector_ = builderForValue.build(); onChanged(); @@ -7599,16 +7855,16 @@ public final class ActiveActive { } /** required .VersionVector versionVector = 3; */ public Builder mergeVersionVector( - akka.persistence.typed.serialization.ActiveActive.VersionVector value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { if (versionVectorBuilder_ == null) { if (((bitField0_ & 0x00000004) != 0) && versionVector_ != null && versionVector_ - != akka.persistence.typed.serialization.ActiveActive.VersionVector + != akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector .getDefaultInstance()) { versionVector_ = - akka.persistence.typed.serialization.ActiveActive.VersionVector.newBuilder( - versionVector_) + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .newBuilder(versionVector_) .mergeFrom(value) .buildPartial(); } else { @@ -7633,35 +7889,38 @@ public final class ActiveActive { return this; } /** required .VersionVector versionVector = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder getVersionVectorBuilder() { bitField0_ |= 0x00000004; onChanged(); return getVersionVectorFieldBuilder().getBuilder(); } /** required .VersionVector versionVector = 3; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder getVersionVectorOrBuilder() { if (versionVectorBuilder_ != null) { return versionVectorBuilder_.getMessageOrBuilder(); } else { return versionVector_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : versionVector_; } } /** required .VersionVector versionVector = 3; */ private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + 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.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder>( + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .VersionVectorOrBuilder>( getVersionVector(), getParentForChildren(), isClean()); versionVector_ = null; } @@ -7725,15 +7984,18 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:ReplicatedEventMetadata) - private static final akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata DEFAULT_INSTANCE; static { DEFAULT_INSTANCE = - new akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata(); + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata(); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedEventMetadata getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -7760,7 +8022,7 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedEventMetadata + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedEventMetadata getDefaultInstanceForType() { return DEFAULT_INSTANCE; } @@ -7782,27 +8044,30 @@ public final class ActiveActive { * * @return The version. */ - akka.persistence.typed.serialization.ActiveActive.VersionVector getVersion(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVersion(); /** required .VersionVector version = 1; */ - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder getVersionOrBuilder(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder + getVersionOrBuilder(); /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ java.util.List< - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen> + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen> getSeenPerReplicaList(); /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + 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.ActiveActive.ReplicatedSnapshotMetadata - .SeenOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.SeenOrBuilder> getSeenPerReplicaOrBuilderList(); /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.SeenOrBuilder + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .SeenOrBuilder getSeenPerReplicaOrBuilder(int index); } /** Protobuf type {@code ReplicatedSnapshotMetadata} */ @@ -7855,14 +8120,15 @@ public final class ActiveActive { break; case 10: { - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder subBuilder = - null; + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + subBuilder = null; if (((bitField0_ & 0x00000001) != 0)) { subBuilder = version_.toBuilder(); } version_ = input.readMessage( - akka.persistence.typed.serialization.ActiveActive.VersionVector.PARSER, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .PARSER, extensionRegistry); if (subBuilder != null) { subBuilder.mergeFrom(version_); @@ -7876,14 +8142,14 @@ public final class ActiveActive { if (!((mutable_bitField0_ & 0x00000002) != 0)) { seenPerReplica_ = new java.util.ArrayList< - akka.persistence.typed.serialization.ActiveActive + akka.persistence.typed.serialization.ReplicatedEventSourcing .ReplicatedSnapshotMetadata.Seen>(); mutable_bitField0_ |= 0x00000002; } seenPerReplica_.add( input.readMessage( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .Seen.PARSER, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.PARSER, extensionRegistry)); break; } @@ -7911,19 +8177,20 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ReplicatedSnapshotMetadata_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.class, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Builder - .class); + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Builder.class); } public interface SeenOrBuilder @@ -8043,20 +8310,20 @@ public final class ActiveActive { } public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ReplicatedSnapshotMetadata_Seen_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .class, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.Builder.class); } private int bitField0_; @@ -8180,11 +8447,15 @@ public final class ActiveActive { } if (!(obj instanceof - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen)) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen other = - (akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen) 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()) { @@ -8218,15 +8489,15 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .Seen + 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.ActiveActive.ReplicatedSnapshotMetadata - .Seen + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen parseFrom( java.nio.ByteBuffer data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8234,15 +8505,15 @@ public final class ActiveActive { return PARSER.parseFrom(data, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .Seen + 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.ActiveActive.ReplicatedSnapshotMetadata - .Seen + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen parseFrom( akka.protobufv3.internal.ByteString data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8250,27 +8521,27 @@ public final class ActiveActive { return PARSER.parseFrom(data, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .Seen + 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.ActiveActive.ReplicatedSnapshotMetadata - .Seen + 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.ActiveActive.ReplicatedSnapshotMetadata - .Seen + 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.ActiveActive.ReplicatedSnapshotMetadata - .Seen + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen parseFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8279,15 +8550,15 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .Seen + 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.ActiveActive.ReplicatedSnapshotMetadata - .Seen + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8296,14 +8567,14 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .Seen + 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.ActiveActive.ReplicatedSnapshotMetadata - .Seen + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8322,7 +8593,8 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -8343,27 +8615,27 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:ReplicatedSnapshotMetadata.Seen) - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata .SeenOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ReplicatedSnapshotMetadata_Seen_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .class, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .Builder.class); + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.Builder.class); } // Construct using - // akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen.newBuilder() + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata.Seen.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -8389,22 +8661,25 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ReplicatedSnapshotMetadata_Seen_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .getDefaultInstance(); + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen build() { - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen result = - buildPartial(); + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -8412,11 +8687,14 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + public akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen buildPartial() { - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen result = - new akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen( - this); + 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)) { @@ -8471,9 +8749,11 @@ public final class ActiveActive { public Builder mergeFrom(akka.protobufv3.internal.Message other) { if (other instanceof - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen) { + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen) { return mergeFrom( - (akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen) + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen) other); } else { super.mergeFrom(other); @@ -8482,11 +8762,12 @@ public final class ActiveActive { } public Builder mergeFrom( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen other) { if (other - == akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .getDefaultInstance()) return this; + == akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.getDefaultInstance()) return this; if (other.hasReplicaId()) { bitField0_ |= 0x00000001; replicaId_ = other.replicaId_; @@ -8516,13 +8797,15 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + 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.ActiveActive.ReplicatedSnapshotMetadata.Seen) + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -8677,17 +8960,18 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:ReplicatedSnapshotMetadata.Seen) - private static final akka.persistence.typed.serialization.ActiveActive + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing .ReplicatedSnapshotMetadata.Seen DEFAULT_INSTANCE; static { DEFAULT_INSTANCE = - new akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen(); + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen(); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .Seen + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -8714,7 +8998,8 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen getDefaultInstanceForType() { return DEFAULT_INSTANCE; } @@ -8722,7 +9007,7 @@ public final class ActiveActive { private int bitField0_; public static final int VERSION_FIELD_NUMBER = 1; - private akka.persistence.typed.serialization.ActiveActive.VersionVector version_; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector version_; /** * required .VersionVector version = 1; * @@ -8736,34 +9021,38 @@ public final class ActiveActive { * * @return The version. */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector getVersion() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector getVersion() { return version_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : version_; } /** required .VersionVector version = 1; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder getVersionOrBuilder() { return version_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : version_; } public static final int SEENPERREPLICA_FIELD_NUMBER = 2; private java.util.List< - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen> + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen> seenPerReplica_; /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public java.util.List< - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen> + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen> getSeenPerReplicaList() { return seenPerReplica_; } /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public java.util.List< ? extends - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .SeenOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.SeenOrBuilder> getSeenPerReplicaOrBuilderList() { return seenPerReplica_; } @@ -8772,12 +9061,13 @@ public final class ActiveActive { return seenPerReplica_.size(); } /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + 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.ActiveActive.ReplicatedSnapshotMetadata + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata .SeenOrBuilder getSeenPerReplicaOrBuilder(int index) { return seenPerReplica_.get(index); @@ -8847,11 +9137,15 @@ public final class ActiveActive { } if (!(obj instanceof - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata)) { + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata)) { return super.equals(obj); } - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata other = - (akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata) obj; + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + other = + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata) + obj; if (hasVersion() != other.hasVersion()) return false; if (hasVersion()) { @@ -8882,13 +9176,15 @@ public final class ActiveActive { return hash; } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + 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.ActiveActive.ReplicatedSnapshotMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata parseFrom( java.nio.ByteBuffer data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8896,13 +9192,15 @@ public final class ActiveActive { return PARSER.parseFrom(data, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + 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.ActiveActive.ReplicatedSnapshotMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata parseFrom( akka.protobufv3.internal.ByteString data, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8910,23 +9208,27 @@ public final class ActiveActive { return PARSER.parseFrom(data, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + 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.ActiveActive.ReplicatedSnapshotMetadata + 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.ActiveActive.ReplicatedSnapshotMetadata + 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.ActiveActive.ReplicatedSnapshotMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata parseFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8935,13 +9237,15 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + 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.ActiveActive.ReplicatedSnapshotMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata parseDelimitedFrom( java.io.InputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8950,12 +9254,14 @@ public final class ActiveActive { PARSER, input, extensionRegistry); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + 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.ActiveActive.ReplicatedSnapshotMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata parseFrom( akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) @@ -8974,7 +9280,8 @@ public final class ActiveActive { } public static Builder newBuilder( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata prototype) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + prototype) { return DEFAULT_INSTANCE.toBuilder().mergeFrom(prototype); } @@ -8994,25 +9301,27 @@ public final class ActiveActive { extends akka.protobufv3.internal.GeneratedMessageV3.Builder implements // @@protoc_insertion_point(builder_implements:ReplicatedSnapshotMetadata) - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadataOrBuilder { + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadataOrBuilder { public static final akka.protobufv3.internal.Descriptors.Descriptor getDescriptor() { - return akka.persistence.typed.serialization.ActiveActive + 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.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ReplicatedSnapshotMetadata_fieldAccessorTable .ensureFieldAccessorsInitialized( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.class, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Builder - .class); + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.class, + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Builder.class); } // Construct using - // akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.newBuilder() + // akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -9049,21 +9358,22 @@ public final class ActiveActive { @java.lang.Override public akka.protobufv3.internal.Descriptors.Descriptor getDescriptorForType() { - return akka.persistence.typed.serialization.ActiveActive + return akka.persistence.typed.serialization.ReplicatedEventSourcing .internal_static_ReplicatedSnapshotMetadata_descriptor; } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata getDefaultInstanceForType() { - return akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .getDefaultInstance(); + return akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.getDefaultInstance(); } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata build() { - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata result = - buildPartial(); + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + build() { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } @@ -9071,10 +9381,12 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata buildPartial() { - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata result = - new akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata(this); + 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)) { @@ -9138,9 +9450,12 @@ public final class ActiveActive { public Builder mergeFrom(akka.protobufv3.internal.Message other) { if (other instanceof - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata) { + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata) { return mergeFrom( - (akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata) other); + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata) + other); } else { super.mergeFrom(other); return this; @@ -9148,10 +9463,11 @@ public final class ActiveActive { } public Builder mergeFrom( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata other) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + other) { if (other - == akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .getDefaultInstance()) return this; + == akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.getDefaultInstance()) return this; if (other.hasVersion()) { mergeVersion(other.getVersion()); } @@ -9208,13 +9524,14 @@ public final class ActiveActive { akka.protobufv3.internal.CodedInputStream input, akka.protobufv3.internal.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata parsedMessage = - null; + 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.ActiveActive.ReplicatedSnapshotMetadata) + (akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata) e.getUnfinishedMessage(); throw e.unwrapIOException(); } finally { @@ -9227,11 +9544,11 @@ public final class ActiveActive { private int bitField0_; - private akka.persistence.typed.serialization.ActiveActive.VersionVector version_; + private akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector version_; private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder, + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder> versionBuilder_; /** * required .VersionVector version = 1; @@ -9246,10 +9563,12 @@ public final class ActiveActive { * * @return The version. */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector getVersion() { + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + getVersion() { if (versionBuilder_ == null) { return version_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : version_; } else { return versionBuilder_.getMessage(); @@ -9257,7 +9576,7 @@ public final class ActiveActive { } /** required .VersionVector version = 1; */ public Builder setVersion( - akka.persistence.typed.serialization.ActiveActive.VersionVector value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { if (versionBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -9272,7 +9591,8 @@ public final class ActiveActive { } /** required .VersionVector version = 1; */ public Builder setVersion( - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder builderForValue) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder + builderForValue) { if (versionBuilder_ == null) { version_ = builderForValue.build(); onChanged(); @@ -9284,15 +9604,16 @@ public final class ActiveActive { } /** required .VersionVector version = 1; */ public Builder mergeVersion( - akka.persistence.typed.serialization.ActiveActive.VersionVector value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector value) { if (versionBuilder_ == null) { if (((bitField0_ & 0x00000001) != 0) && version_ != null && version_ - != akka.persistence.typed.serialization.ActiveActive.VersionVector + != akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector .getDefaultInstance()) { version_ = - akka.persistence.typed.serialization.ActiveActive.VersionVector.newBuilder(version_) + akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .newBuilder(version_) .mergeFrom(value) .buildPartial(); } else { @@ -9317,66 +9638,71 @@ public final class ActiveActive { return this; } /** required .VersionVector version = 1; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector.Builder getVersionBuilder() { bitField0_ |= 0x00000001; onChanged(); return getVersionFieldBuilder().getBuilder(); } /** required .VersionVector version = 1; */ - public akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVectorOrBuilder getVersionOrBuilder() { if (versionBuilder_ != null) { return versionBuilder_.getMessageOrBuilder(); } else { return version_ == null - ? akka.persistence.typed.serialization.ActiveActive.VersionVector.getDefaultInstance() + ? akka.persistence.typed.serialization.ReplicatedEventSourcing.VersionVector + .getDefaultInstance() : version_; } } /** required .VersionVector version = 1; */ private akka.protobufv3.internal.SingleFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder> + 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.ActiveActive.VersionVector, - akka.persistence.typed.serialization.ActiveActive.VersionVector.Builder, - akka.persistence.typed.serialization.ActiveActive.VersionVectorOrBuilder>( - getVersion(), getParentForChildren(), isClean()); + 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.ActiveActive.ReplicatedSnapshotMetadata.Seen> + 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.ActiveActive.ReplicatedSnapshotMetadata - .Seen>(seenPerReplica_); + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen>(seenPerReplica_); bitField0_ |= 0x00000002; } } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .Builder, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .SeenOrBuilder> + 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.ActiveActive.ReplicatedSnapshotMetadata.Seen> + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen> getSeenPerReplicaList() { if (seenPerReplicaBuilder_ == null) { return java.util.Collections.unmodifiableList(seenPerReplica_); @@ -9393,7 +9719,8 @@ public final class ActiveActive { } } /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen getSeenPerReplica(int index) { if (seenPerReplicaBuilder_ == null) { return seenPerReplica_.get(index); @@ -9404,7 +9731,9 @@ public final class ActiveActive { /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public Builder setSeenPerReplica( int index, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + value) { if (seenPerReplicaBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -9420,7 +9749,8 @@ public final class ActiveActive { /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public Builder setSeenPerReplica( int index, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen.Builder + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder builderForValue) { if (seenPerReplicaBuilder_ == null) { ensureSeenPerReplicaIsMutable(); @@ -9433,7 +9763,9 @@ public final class ActiveActive { } /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public Builder addSeenPerReplica( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + value) { if (seenPerReplicaBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -9449,7 +9781,9 @@ public final class ActiveActive { /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public Builder addSeenPerReplica( int index, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen value) { + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen + value) { if (seenPerReplicaBuilder_ == null) { if (value == null) { throw new NullPointerException(); @@ -9464,7 +9798,8 @@ public final class ActiveActive { } /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public Builder addSeenPerReplica( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen.Builder + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder builderForValue) { if (seenPerReplicaBuilder_ == null) { ensureSeenPerReplicaIsMutable(); @@ -9478,7 +9813,8 @@ public final class ActiveActive { /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public Builder addSeenPerReplica( int index, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen.Builder + akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder builderForValue) { if (seenPerReplicaBuilder_ == null) { ensureSeenPerReplicaIsMutable(); @@ -9493,8 +9829,8 @@ public final class ActiveActive { public Builder addAllSeenPerReplica( java.lang.Iterable< ? extends - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .Seen> + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen> values) { if (seenPerReplicaBuilder_ == null) { ensureSeenPerReplicaIsMutable(); @@ -9528,13 +9864,13 @@ public final class ActiveActive { return this; } /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .Builder + 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.ActiveActive.ReplicatedSnapshotMetadata + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata .SeenOrBuilder getSeenPerReplicaOrBuilder(int index) { if (seenPerReplicaBuilder_ == null) { @@ -9546,8 +9882,8 @@ public final class ActiveActive { /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public java.util.List< ? extends - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .SeenOrBuilder> + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.SeenOrBuilder> getSeenPerReplicaOrBuilderList() { if (seenPerReplicaBuilder_ != null) { return seenPerReplicaBuilder_.getMessageOrBuilderList(); @@ -9556,47 +9892,49 @@ public final class ActiveActive { } } /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder addSeenPerReplicaBuilder() { return getSeenPerReplicaFieldBuilder() .addBuilder( - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .getDefaultInstance()); + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.getDefaultInstance()); } /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .Builder + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata + .Seen.Builder addSeenPerReplicaBuilder(int index) { return getSeenPerReplicaFieldBuilder() .addBuilder( index, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .getDefaultInstance()); + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.getDefaultInstance()); } /** repeated .ReplicatedSnapshotMetadata.Seen seenPerReplica = 2; */ public java.util.List< - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .Builder> + akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata.Seen.Builder> getSeenPerReplicaBuilderList() { return getSeenPerReplicaFieldBuilder().getBuilderList(); } private akka.protobufv3.internal.RepeatedFieldBuilderV3< - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .Builder, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .SeenOrBuilder> + 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.ActiveActive.ReplicatedSnapshotMetadata.Seen, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata.Seen - .Builder, - akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata - .SeenOrBuilder>( + 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(), @@ -9622,16 +9960,18 @@ public final class ActiveActive { } // @@protoc_insertion_point(class_scope:ReplicatedSnapshotMetadata) - private static final akka.persistence.typed.serialization.ActiveActive + private static final akka.persistence.typed.serialization.ReplicatedEventSourcing .ReplicatedSnapshotMetadata DEFAULT_INSTANCE; static { DEFAULT_INSTANCE = - new akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata(); + new akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata(); } - public static akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + public static akka.persistence.typed.serialization.ReplicatedEventSourcing + .ReplicatedSnapshotMetadata getDefaultInstance() { return DEFAULT_INSTANCE; } @@ -9658,7 +9998,7 @@ public final class ActiveActive { } @java.lang.Override - public akka.persistence.typed.serialization.ActiveActive.ReplicatedSnapshotMetadata + public akka.persistence.typed.serialization.ReplicatedEventSourcing.ReplicatedSnapshotMetadata getDefaultInstanceForType() { return DEFAULT_INSTANCE; } @@ -9713,30 +10053,30 @@ public final class ActiveActive { static { java.lang.String[] descriptorData = { - "\n\022ActiveActive.proto\032\026ContainerFormats.p" - + "roto\"\030\n\007Counter\022\r\n\005value\030\001 \002(\014\"\036\n\rCounte" - + "rUpdate\022\r\n\005delta\030\001 \002(\014\"\304\001\n\005ORSet\022\020\n\010orig" - + "inDc\030\001 \002(\t\022\037\n\007vvector\030\002 \002(\0132\016.VersionVec" - + "tor\022\034\n\004dots\030\003 \003(\0132\016.VersionVector\022\026\n\016str" - + "ingElements\030\004 \003(\t\022\027\n\013intElements\030\005 \003(\021B\002" - + "\020\001\022\030\n\014longElements\030\006 \003(\022B\002\020\001\022\037\n\rotherEle" - + "ments\030\007 \003(\0132\010.Payload\"\201\001\n\017ORSetDeltaGrou" - + "p\022\'\n\007entries\030\001 \003(\0132\026.ORSetDeltaGroup.Ent" - + "ry\032E\n\005Entry\022 \n\toperation\030\001 \002(\0162\r.ORSetDe" - + "ltaOp\022\032\n\nunderlying\030\002 \002(\0132\006.ORSet\"]\n\rVer" - + "sionVector\022%\n\007entries\030\001 \003(\0132\024.VersionVec" - + "tor.Entry\032%\n\005Entry\022\013\n\003key\030\001 \002(\t\022\017\n\007versi" - + "on\030\002 \002(\003\"\205\001\n\027ReplicatedEventMetadata\022\025\n\r" - + "originReplica\030\001 \002(\t\022\030\n\020originSequenceNr\030" - + "\002 \002(\003\022%\n\rversionVector\030\003 \002(\0132\016.VersionVe" - + "ctor\022\022\n\nconcurrent\030\004 \002(\010\"\246\001\n\032ReplicatedS" - + "napshotMetadata\022\037\n\007version\030\001 \002(\0132\016.Versi" - + "onVector\0228\n\016seenPerReplica\030\002 \003(\0132 .Repli" - + "catedSnapshotMetadata.Seen\032-\n\004Seen\022\021\n\tre" - + "plicaId\030\001 \002(\t\022\022\n\nsequenceNr\030\002 \002(\003*-\n\014ORS" - + "etDeltaOp\022\007\n\003Add\020\000\022\n\n\006Remove\020\001\022\010\n\004Full\020\002" - + "B(\n$akka.persistence.typed.serialization" - + "H\001" + "\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\014ORSetDeltaOp\022\007\n\003Add\020\000\022\n\n\006Remove\020" + + "\001\022\010\n\004Full\020\002B(\n$akka.persistence.typed.se" + + "rializationH\001" }; descriptor = akka.protobufv3.internal.Descriptors.FileDescriptor.internalBuildGeneratedFileFrom( diff --git a/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes b/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-replicated-event-sourcing.excludes similarity index 83% rename from akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes rename to akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-replicated-event-sourcing.excludes index 5a6490fdc2..c0cfd45bdd 100644 --- a/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes +++ b/akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-replicated-event-sourcing.excludes @@ -1,4 +1,4 @@ # Changes to internal/private/do not extend -ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.typed.scaladsl.EventSourcedBehavior.withActiveActive") +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/ActiveActive.proto b/akka-persistence-typed/src/main/protobuf/ReplicatedEventSourcing.proto similarity index 100% rename from akka-persistence-typed/src/main/protobuf/ActiveActive.proto rename to akka-persistence-typed/src/main/protobuf/ReplicatedEventSourcing.proto diff --git a/akka-persistence-typed/src/main/resources/reference.conf b/akka-persistence-typed/src/main/resources/reference.conf index 3e1c20d1b3..fb72910aca 100644 --- a/akka-persistence-typed/src/main/resources/reference.conf +++ b/akka-persistence-typed/src/main/resources/reference.conf @@ -1,8 +1,8 @@ akka.actor { - serialization-identifiers."akka.persistence.typed.serialization.ActiveActiveSerializer" = 40 + serialization-identifiers."akka.persistence.typed.serialization.ReplicatedEventSourcingSerializer" = 40 - serializers.active-active = "akka.persistence.typed.serialization.ActiveActiveSerializer" + serializers.active-active = "akka.persistence.typed.serialization.ReplicatedEventSourcingSerializer" serialization-bindings { "akka.persistence.typed.internal.VersionVector" = active-active 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 index 1311ea6574..1bdd6889b1 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/PublishedEvent.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/PublishedEvent.scala @@ -17,10 +17,10 @@ import akka.persistence.typed.internal.ReplicatedPublishedEventMetaData @DoNotInherit trait PublishedEvent { - /** Scala API: When emitted from an Active Active actor this will contain the replica id */ + /** 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 Active Active actor this will contain the replica id */ + /** Java API: When emitted from an Replicated Event Sourcing actor this will contain the replica id */ def getReplicatedMetaData: Optional[ReplicatedPublishedEventMetaData] def persistenceId: PersistenceId 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 index b01082a980..a842bf1f35 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicaId.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicaId.scala @@ -5,6 +5,6 @@ package akka.persistence.typed /** - * Identifies a replica in Active Active eventsourcing, could be a datacenter name or a logical identifier. + * 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/internal/BehaviorSetup.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/BehaviorSetup.scala index 4528b602c4..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 @@ -13,7 +13,6 @@ import akka.actor.typed.scaladsl.ActorContext import akka.annotation.InternalApi import akka.persistence._ import akka.persistence.typed.ReplicaId -import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive import akka.persistence.typed.{ EventAdapter, PersistenceId, SnapshotAdapter } import akka.persistence.typed.scaladsl.{ EventSourcedBehavior, RetentionCriteria } import akka.util.OptionVal @@ -49,7 +48,7 @@ private[akka] final class BehaviorSetup[C, E, S]( var holdingRecoveryPermit: Boolean, val settings: EventSourcedSettings, val stashState: StashState, - val activeActive: Option[ActiveActive], + val replication: Option[ReplicationSetup], val publishEvents: Boolean) { import BehaviorSetup._ @@ -62,7 +61,7 @@ private[akka] final class BehaviorSetup[C, E, S]( val journal: ClassicActorRef = persistence.journalFor(settings.journalPluginId) val snapshotStore: ClassicActorRef = persistence.snapshotStoreFor(settings.snapshotPluginId) - val replicaId: Option[ReplicaId] = activeActive.map(_.replicaId) + val replicaId: Option[ReplicaId] = replication.map(_.replicaId) def selfClassic: ClassicActorRef = context.self.toClassic 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 521ba48cfb..299cfab3d5 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 @@ -40,7 +40,6 @@ import akka.persistence.typed.SnapshotAdapter import akka.persistence.typed.SnapshotCompleted import akka.persistence.typed.SnapshotFailed import akka.persistence.typed.SnapshotSelectionCriteria -import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive import akka.persistence.typed.scaladsl._ import akka.persistence.typed.scaladsl.{ Recovery => TypedRecovery } import akka.persistence.typed.scaladsl.RetentionCriteria @@ -93,7 +92,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( retention: RetentionCriteria = RetentionCriteria.disabled, supervisionStrategy: SupervisorStrategy = SupervisorStrategy.stop, override val signalHandler: PartialFunction[(State, Signal), Unit] = PartialFunction.empty, - activeActive: Option[ActiveActive] = None, + replication: Option[ReplicationSetup] = None, publishEvents: Boolean = false) extends EventSourcedBehavior[Command, Event, State] { @@ -158,7 +157,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( holdingRecoveryPermit = false, settings = settings, stashState = stashState, - activeActive = activeActive, + replication = replication, publishEvents = publishEvents) // needs to accept Any since we also can get messages from the journal @@ -251,9 +250,9 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( copy(publishEvents = true) } - override private[akka] def withActiveActive( - context: ActiveActiveContextImpl): EventSourcedBehavior[Command, Event, State] = { - copy(activeActive = Some(ActiveActive(context.replicaId, context.replicasAndQueryPlugins, context))) + override private[akka] def withReplication( + context: ReplicationContextImpl): EventSourcedBehavior[Command, Event, State] = { + copy(replication = Some(ReplicationSetup(context.replicaId, context.replicasAndQueryPlugins, context))) } } @@ -274,7 +273,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( object ReplicatedEventMetadata { /** - * For a journal supporting active active needing to add test coverage, use this instance as metadata and defer + * 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 @@ -297,7 +296,7 @@ private[akka] final case class ReplicatedEventMetadata( object ReplicatedSnapshotMetadata { /** - * For a snapshot store supporting active active needing to add test coverage, use this instance as metadata and defer + * 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 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 04ae7984b2..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 @@ -191,7 +191,7 @@ private[akka] trait SnapshotInteractions[C, E, S] { if (state.state == null) throw new IllegalStateException("A snapshot must not be a null state.") else { - val meta = setup.activeActive match { + val meta = setup.replication match { case Some(_) => val m = ReplicatedSnapshotMetadata(state.version, state.seenPerReplica) Some(m) 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 e33300a277..992b2a1ca4 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 @@ -23,7 +23,6 @@ import akka.persistence.typed.SingleEventSeq import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState import akka.persistence.typed.internal.ReplayingEvents.ReplayingState import akka.persistence.typed.internal.Running.WithSeqNrAccessible -import akka.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive import akka.util.OptionVal import akka.util.PrettyDuration._ import akka.util.unused @@ -123,14 +122,14 @@ private[akka] final class ReplayingEvents[C, E, S]( eventForErrorReporting = OptionVal.Some(event) state = state.copy(seqNr = repr.sequenceNr) - val aaMetaAndSelfReplica: Option[(ReplicatedEventMetadata, ReplicaId, ActiveActive)] = - setup.activeActive match { + val aaMetaAndSelfReplica: Option[(ReplicatedEventMetadata, ReplicaId, ReplicationSetup)] = + setup.replication match { case Some(aa) => val meta = repr.metadata match { case Some(m) => m.asInstanceOf[ReplicatedEventMetadata] case None => throw new IllegalStateException( - s"Active active enabled but existing event has no metadata. Migration isn't supported yet.") + s"Replicated Event Sourcing enabled but existing event has no metadata. Migration isn't supported yet.") } aa.setContext(recoveryRunning = true, meta.originReplica, meta.concurrent) @@ -140,7 +139,7 @@ private[akka] final class ReplayingEvents[C, E, S]( val newState = setup.eventHandler(state.state, event) - setup.activeActive match { + setup.replication match { case Some(aa) => aa.clearContext() case None => 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..755201eb63 --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplicationSetup.scala @@ -0,0 +1,111 @@ +/* + * 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.util.OptionVal +import akka.util.WallClock +import akka.util.ccompat.JavaConverters._ + +/** + * INTERNAL API + */ +// FIXME, parts of this can be set during initialisation +// Other fields will be set before executing the event handler as they change per event +// https://github.com/akka/akka/issues/29258 +@InternalApi +private[akka] final class ReplicationContextImpl( + val entityId: String, + val replicaId: ReplicaId, + 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 = PersistenceId.replicatedUniqueId(entityId, replicaId) + + 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], + aaContext: 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 = { + aaContext._currentThread = OptionVal.Some(Thread.currentThread()) + aaContext._recoveryRunning = recoveryRunning + aaContext._concurrent = concurrent + aaContext._origin = OptionVal.Some(originReplica) + } + + def clearContext(): Unit = { + aaContext._currentThread = OptionVal.None + aaContext._recoveryRunning = false + aaContext._concurrent = false + aaContext._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 4eeec5bdc1..5815e80653 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 @@ -53,7 +53,6 @@ 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.persistence.typed.scaladsl.EventSourcedBehavior.ActiveActive import akka.stream.scaladsl.Keep import akka.stream.SystemMaterializer import akka.stream.WatchedActorTerminatedException @@ -111,7 +110,7 @@ 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)) - val initialState = setup.activeActive match { + val initialState = setup.replication match { case Some(aa) => startReplicationStream(setup, state, aa) case None => state } @@ -121,17 +120,17 @@ private[akka] object Running { def startReplicationStream[C, E, S]( setup: BehaviorSetup[C, E, S], state: RunningState[S], - aa: ActiveActive): RunningState[S] = { + replicationSetup: ReplicationSetup): RunningState[S] = { import scala.concurrent.duration._ val system = setup.context.system val ref = setup.context.self val query = PersistenceQuery(system) - aa.allReplicas.foldLeft(state) { (state, replicaId) => - if (replicaId != aa.replicaId) { + replicationSetup.allReplicas.foldLeft(state) { (state, replicaId) => + if (replicaId != replicationSetup.replicaId) { val seqNr = state.seenPerReplica(replicaId) - val pid = PersistenceId.replicatedUniqueId(aa.aaContext.entityId, replicaId) - val queryPluginId = aa.allReplicasAndQueryPlugins(replicaId) + val pid = PersistenceId.replicatedUniqueId(replicationSetup.aaContext.entityId, replicaId) + val queryPluginId = replicationSetup.allReplicasAndQueryPlugins(replicaId) val replication = query.readJournalFor[EventsByPersistenceIdQuery](queryPluginId) implicit val timeout = Timeout(30.seconds) @@ -152,7 +151,7 @@ private[akka] object Running { 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 active active?") + "used that does not support Replicated Event Sourcing?") }) .viaMat(new FastForwardingFilter)(Keep.right) .mapMaterializedValue(streamControl => controlRef.set(streamControl)) @@ -240,7 +239,7 @@ private[akka] object Running { def onMessage(msg: InternalProtocol): Behavior[InternalProtocol] = msg match { case IncomingCommand(c: C @unchecked) => onCommand(state, c) - case re: ReplicatedEventEnvelope[E @unchecked] => onReplicatedEvent(state, re, setup.activeActive.get) + 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) @@ -267,19 +266,19 @@ private[akka] object Running { def onReplicatedEvent( state: Running.RunningState[S], envelope: ReplicatedEventEnvelope[E], - activeActive: ActiveActive): Behavior[InternalProtocol] = { + replication: ReplicationSetup): Behavior[InternalProtocol] = { setup.log.infoN( "Replica {} received replicated event. Replica seqs nrs: {}. Envelope {}", - setup.activeActive, + setup.replication, state.seenPerReplica, envelope) envelope.ack ! ReplicatedEventAck - if (envelope.event.originReplica != activeActive.replicaId && !alreadySeen(envelope.event)) { + 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(activeActive, envelope.event) + handleExternalReplicatedEventPersist(replication, envelope.event) } else { setup.log.debug( "Filtering event [{}] from [{}] as it was already seen", @@ -290,19 +289,20 @@ private[akka] object Running { } def onPublishedEvent(state: Running.RunningState[S], event: PublishedEventImpl): Behavior[InternalProtocol] = { - val newBehavior: Behavior[InternalProtocol] = setup.activeActive match { + val newBehavior: Behavior[InternalProtocol] = setup.replication match { case None => - setup.log - .warn("Received published event for [{}] but not an active active actor, dropping", event.persistenceId) + setup.log.warn( + "Received published event for [{}] but not an Replicated Event Sourcing actor, dropping", + event.persistenceId) this - case Some(activeActive) => + 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, activeActive, replicatedEventMetaData, event) + onPublishedEvent(state, replication, replicatedEventMetaData, event) } } tryUnstashOne(newBehavior) @@ -310,7 +310,7 @@ private[akka] object Running { private def onPublishedEvent( state: Running.RunningState[S], - activeActive: ActiveActive, + replication: ReplicationSetup, replicatedMetadata: ReplicatedPublishedEventMetaData, event: PublishedEventImpl): Behavior[InternalProtocol] = { val log = setup.log @@ -320,18 +320,18 @@ private[akka] object Running { if (!setup.persistenceId.id.startsWith(idPrefix)) { log.warn("Ignoring published replicated event for the wrong actor [{}]", event.persistenceId) this - } else if (originReplicaId == activeActive.replicaId) { + } 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 (!activeActive.allReplicas.contains(originReplicaId)) { + } else if (!replication.allReplicas.contains(originReplicaId)) { log.warnN( - "Received published replicated event from replica [{}], which is unknown. Active active must be set up with a list of all replicas (known are [{}]).", + "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, - activeActive.allReplicas.mkString(", ")) + replication.allReplicas.mkString(", ")) this } else { val expectedSequenceNumber = state.seenPerReplica(originReplicaId) + 1 @@ -369,7 +369,7 @@ private[akka] object Running { state.replicationControl.get(originReplicaId).foreach(_.fastForward(event.sequenceNumber)) handleExternalReplicatedEventPersist( - activeActive, + replication, ReplicatedEvent( event.event.asInstanceOf[E], originReplicaId, @@ -386,15 +386,15 @@ private[akka] object Running { this } - def withContext[A](aa: ActiveActive, withActiveActive: ActiveActive => Unit, f: () => A): A = { - withActiveActive(aa) + def withContext[A](aa: ReplicationSetup, withReplication: ReplicationSetup => Unit, f: () => A): A = { + withReplication(aa) val result = f() aa.clearContext() result } private def handleExternalReplicatedEventPersist( - activeActive: ActiveActive, + replication: ReplicationSetup, event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { _currentSequenceNumber = state.seqNr + 1 val isConcurrent: Boolean = event.originVersion <> state.version @@ -410,7 +410,7 @@ private[akka] object Running { isConcurrent) val newState: RunningState[S] = withContext( - activeActive, + replication, aa => aa.setContext(recoveryRunning = false, event.originReplica, concurrent = isConcurrent), () => state.applyEvent(setup, event.event)) @@ -442,7 +442,7 @@ private[akka] object Running { // also, ensure that there is an event handler for each single event _currentSequenceNumber = state.seqNr + 1 - val newState: RunningState[S] = setup.activeActive match { + val newState: RunningState[S] = setup.replication match { case Some(aa) => // set concurrent to false, local events are never concurrent withContext( @@ -456,7 +456,7 @@ private[akka] object Running { val eventToPersist = adaptEvent(event) val eventAdapterManifest = setup.eventAdapter.manifest(event) - val newState2 = setup.activeActive match { + val newState2 = setup.replication match { case Some(aa) => val updatedVersion = newState.version.updated(aa.replicaId.id, _currentSequenceNumber) val r = internalPersist( @@ -494,7 +494,7 @@ private[akka] object Running { // also, ensure that there is an event handler for each single event _currentSequenceNumber = state.seqNr - val metadataTemplate: Option[ReplicatedEventMetadata] = setup.activeActive match { + val metadataTemplate: Option[ReplicatedEventMetadata] = setup.replication match { case Some(aa) => aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false) // local events are never concurrent Some(ReplicatedEventMetadata(aa.replicaId, 0L, state.version, concurrent = false)) // we replace it with actual seqnr later @@ -524,7 +524,7 @@ private[akka] object Running { case None => None } - currentState = setup.activeActive match { + currentState = setup.replication match { case Some(aa) => withContext( aa, @@ -679,7 +679,7 @@ private[akka] object Running { onWriteSuccess(setup.context, p) if (setup.publishEvents) { - val meta = setup.activeActive.map(aa => new ReplicatedPublishedEventMetaData(aa.replicaId, state.version)) + val meta = setup.replication.map(aa => new ReplicatedPublishedEventMetaData(aa.replicaId, state.version)) context.system.eventStream ! EventStream.Publish( PublishedEventImpl(setup.persistenceId, p.sequenceNr, p.payload, p.timestamp, meta)) } 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 00c6008121..fc0a1cf57c 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 @@ -179,7 +179,7 @@ abstract class EventSourcedBehavior[Command, Event, State] private[akka] ( * INTERNAL API: DeferredBehavior init, not for user extension */ @InternalApi override def apply(context: typed.TypedActorContext[Command]): Behavior[Command] = { - // Note: duplicated in ActiveActiveEventSourcedBehavior to not break source compatibility + // Note: duplicated in ReplicatedEventSourcedBehavior to not break source compatibility 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/ActiveActiveEventSourcedBehavior.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcedBehavior.scala similarity index 79% rename from akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala rename to akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcedBehavior.scala index 9e315b04f1..7346615e8f 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcedBehavior.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcedBehavior.scala @@ -12,23 +12,26 @@ import akka.actor.typed.TypedActorContext import akka.annotation.ApiMayChange import akka.annotation.InternalApi import akka.persistence.typed.internal +import akka.persistence.typed.internal.ReplicationContextImpl import akka.persistence.typed.internal.EffectImpl -import akka.persistence.typed.scaladsl.ActiveActiveContextImpl +/** + * Base class for replicated event sourced behaviors. + */ @ApiMayChange -abstract class ActiveActiveEventSourcedBehavior[Command, Event, State]( - activeActiveContext: ActiveActiveContext, +abstract class ReplicatedEventSourcedBehavior[Command, Event, State]( + replicationContext: ReplicationContext, onPersistFailure: Optional[BackoffSupervisorStrategy]) - extends EventSourcedBehavior[Command, Event, State](activeActiveContext.persistenceId, onPersistFailure) { + extends EventSourcedBehavior[Command, Event, State](replicationContext.persistenceId, onPersistFailure) { - def this(activeActiveContext: ActiveActiveContext) = this(activeActiveContext, Optional.empty()) + def this(replicationContext: ReplicationContext) = this(replicationContext, Optional.empty()) /** * Override and return true to publish events to the system event stream as [[akka.persistence.typed.PublishedEvent]] after they have been persisted */ def withEventPublishing: Boolean = false - protected def getActiveActiveContext(): ActiveActiveContext = activeActiveContext + protected def getReplicationContext(): ReplicationContext = replicationContext /** * INTERNAL API: DeferredBehavior init, not for user extension @@ -59,7 +62,7 @@ abstract class ActiveActiveEventSourcedBehavior[Command, Event, State]( .withSnapshotPluginId(snapshotPluginId) .withRecovery(recovery.asScala) // context not user extendable so there should never be any other impls - .withActiveActive(activeActiveContext.asInstanceOf[ActiveActiveContextImpl]) + .withReplication(replicationContext.asInstanceOf[ReplicationContextImpl]) val handler = signalHandler() val behaviorWithSignalHandler = diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcing.scala similarity index 71% rename from akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala rename to akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcing.scala index d59b812bc6..f5ed8f29be 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/javadsl/ReplicatedEventSourcing.scala @@ -11,36 +11,63 @@ import java.util.{ Map => JMap } import akka.annotation.DoNotInherit import akka.persistence.typed.PersistenceId import akka.persistence.typed.ReplicaId -import akka.persistence.typed.scaladsl.ActiveActiveContextImpl +import akka.persistence.typed.internal.ReplicationContextImpl import akka.util.ccompat.JavaConverters._ /** - * Provides access to Active Active specific state + * Provides access to replication specific state * * Not for user extension */ @DoNotInherit -trait ActiveActiveContext { - def origin: ReplicaId - def concurrent: Boolean +trait ReplicationContext { + + /** + * @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 - def recoveryRunning: Boolean + + /** + * @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 } -/** - * Factory to create an instance of an ActiveActiveEventSourcedBehavior - */ -@FunctionalInterface -trait ActiveActiveBehaviorFactory[Command, Event, State] { - def apply(aaContext: ActiveActiveContext): ActiveActiveEventSourcedBehavior[Command, Event, State] -} - -object ActiveActiveEventSourcing { +object ReplicatedEventSourcing { /** * Initialize a replicated event sourced behavior where all entity replicas are stored in the same journal. @@ -63,7 +90,7 @@ object ActiveActiveEventSourcing { replicaId: ReplicaId, allReplicaIds: JSet[ReplicaId], queryPluginId: String, - behaviorFactory: JFunction[ActiveActiveContext, EventSourcedBehavior[Command, Event, State]]) + behaviorFactory: JFunction[ReplicationContext, EventSourcedBehavior[Command, Event, State]]) : EventSourcedBehavior[Command, Event, State] = create(entityId, replicaId, allReplicaIds.asScala.map(id => id -> queryPluginId).toMap.asJava, behaviorFactory) @@ -87,9 +114,9 @@ object ActiveActiveEventSourcing { entityId: String, replicaId: ReplicaId, allReplicasAndQueryPlugins: JMap[ReplicaId, String], - eventSourcedBehaviorFactory: JFunction[ActiveActiveContext, EventSourcedBehavior[Command, Event, State]]) + eventSourcedBehaviorFactory: JFunction[ReplicationContext, EventSourcedBehavior[Command, Event, State]]) : EventSourcedBehavior[Command, Event, State] = { - val context = new ActiveActiveContextImpl(entityId, replicaId, allReplicasAndQueryPlugins.asScala.toMap) + val context = new ReplicationContextImpl(entityId, replicaId, 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 b40d3a327e..0c4c114ed8 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,7 +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,45 +12,18 @@ 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, InternalApi } +import akka.annotation.DoNotInherit +import akka.annotation.InternalApi import akka.persistence.typed.EventAdapter import akka.persistence.typed.PersistenceId -import akka.persistence.typed.ReplicaId import akka.persistence.typed.SnapshotAdapter import akka.persistence.typed.SnapshotSelectionCriteria import akka.persistence.typed.internal._ -import akka.util.OptionVal + +import scala.annotation.tailrec object EventSourcedBehavior { - // FIXME move to internal - @InternalApi - private[akka] final case class ActiveActive( - replicaId: ReplicaId, - allReplicasAndQueryPlugins: Map[ReplicaId, String], - aaContext: ActiveActiveContextImpl) { - - 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 = { - aaContext._currentThread = OptionVal.Some(Thread.currentThread()) - aaContext._recoveryRunning = recoveryRunning - aaContext._concurrent = concurrent - aaContext._origin = originReplica - } - - def clearContext(): Unit = { - aaContext._currentThread = OptionVal.None - aaContext._recoveryRunning = false - aaContext._concurrent = false - aaContext._origin = null - } - - } - /** * Type alias for the command handler function that defines how to act on commands. * @@ -175,8 +147,6 @@ object EventSourcedBehavior { */ def withJournalPluginId(id: String): EventSourcedBehavior[Command, Event, State] - private[akka] def withActiveActive(context: ActiveActiveContextImpl): EventSourcedBehavior[Command, Event, State] - /** * Change the snapshot store plugin id that this actor should use. */ @@ -253,4 +223,10 @@ object EventSourcedBehavior { */ @ApiMayChange def withEventPublishing(): 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/ActiveActiveEventSourcing.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ReplicatedEventSourcing.scala similarity index 54% rename from akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala rename to akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ReplicatedEventSourcing.scala index 7570fa3b31..6cf395f51a 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ActiveActiveEventSourcing.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/scaladsl/ReplicatedEventSourcing.scala @@ -4,90 +4,65 @@ package akka.persistence.typed.scaladsl +import akka.annotation.DoNotInherit import akka.persistence.typed.PersistenceId import akka.persistence.typed.ReplicaId -import akka.util.{ OptionVal, WallClock } +import akka.persistence.typed.internal.ReplicationContextImpl -import akka.util.ccompat.JavaConverters._ - -// FIXME docs -trait ActiveActiveContext { +/** + * Provides access to replication specific state + * + * Not for user extension + */ +@DoNotInherit +trait ReplicationContext { + /** + * @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 + + /** + * @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 + /** + * 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 } -// FIXME, parts of this can be set during initialisation -// Other fields will be set before executing the event handler as they change per event -// https://github.com/akka/akka/issues/29258 -private[akka] class ActiveActiveContextImpl( - val entityId: String, - val replicaId: ReplicaId, - val replicasAndQueryPlugins: Map[ReplicaId, String]) - extends ActiveActiveContext - with akka.persistence.typed.javadsl.ActiveActiveContext { - val allReplicas: Set[ReplicaId] = replicasAndQueryPlugins.keySet - - // these are not volatile as they are set on the same thread as they should be accessed - var _origin: ReplicaId = null - var _recoveryRunning: Boolean = false - var _concurrent: Boolean = false - var _currentThread: OptionVal[Thread] = OptionVal.None - - private def checkAccess(functionName: String): Unit = { - val callerThread = Thread.currentThread() - def error() = - throw new UnsupportedOperationException( - s"Unsupported access to ActiveActiveContext 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 - } - - /** - * 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 = PersistenceId.replicatedUniqueId(entityId, replicaId) - - override def currentTimeMillis(): Long = { - WallClock.AlwaysIncreasingClock.currentTimeMillis() - } - override def recoveryRunning: Boolean = { - checkAccess("recoveryRunning") - _recoveryRunning - } - - override def getAllReplicas: java.util.Set[ReplicaId] = allReplicas.asJava -} - -object ActiveActiveEventSourcing { +object ReplicatedEventSourcing { /** * Initialize a replicated event sourced behavior where all entity replicas are stored in the same journal. @@ -110,7 +85,7 @@ object ActiveActiveEventSourcing { replicaId: ReplicaId, allReplicaIds: Set[ReplicaId], queryPluginId: String)( - eventSourcedBehaviorFactory: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) + eventSourcedBehaviorFactory: ReplicationContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = apply(entityId, replicaId, allReplicaIds.map(id => id -> queryPluginId).toMap)(eventSourcedBehaviorFactory) @@ -134,10 +109,10 @@ object ActiveActiveEventSourcing { entityId: String, replicaId: ReplicaId, allReplicasAndQueryPlugins: Map[ReplicaId, String])( - eventSourcedBehaviorFactory: ActiveActiveContext => EventSourcedBehavior[Command, Event, State]) + eventSourcedBehaviorFactory: ReplicationContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = { - val context = new ActiveActiveContextImpl(entityId, replicaId, allReplicasAndQueryPlugins) - eventSourcedBehaviorFactory(context).withActiveActive(context) + val context = new ReplicationContextImpl(entityId, replicaId, allReplicasAndQueryPlugins) + eventSourcedBehaviorFactory(context).withReplication(context) } } diff --git a/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializer.scala similarity index 75% rename from akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala rename to akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializer.scala index 51633d6eff..5c4b4dc25d 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ActiveActiveSerializer.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializer.scala @@ -22,9 +22,13 @@ import akka.serialization.{ BaseSerializer, SerializerWithStringManifest } import scala.annotation.tailrec import akka.util.ccompat.JavaConverters._ + import scala.collection.immutable.TreeMap -object ActiveActiveSerializer { +/** + * INTERNAL API + */ +@InternalApi private[akka] object ReplicatedEventSourcingSerializer { object Comparator extends Comparator[Payload] { override def compare(a: Payload, b: Payload): Int = { val aByteString = a.getEnclosedMessage @@ -53,7 +57,7 @@ object ActiveActiveSerializer { /** * INTERNAL API */ -@InternalApi private[akka] final class ActiveActiveSerializer(val system: ExtendedActorSystem) +@InternalApi private[akka] final class ReplicatedEventSourcingSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer { @@ -132,23 +136,34 @@ object ActiveActiveSerializer { } def counterFromBinary(bytes: Array[Byte]): Counter = - Counter(BigInt(ActiveActive.Counter.parseFrom(bytes).getValue.toByteArray)) + Counter(BigInt(ReplicatedEventSourcing.Counter.parseFrom(bytes).getValue.toByteArray)) def counterUpdatedFromBinary(bytes: Array[Byte]): Counter.Updated = - Counter.Updated(BigInt(ActiveActive.CounterUpdate.parseFrom(bytes).getDelta.toByteArray)) + Counter.Updated(BigInt(ReplicatedEventSourcing.CounterUpdate.parseFrom(bytes).getDelta.toByteArray)) def counterToProtoByteArray(counter: Counter): Array[Byte] = - ActiveActive.Counter.newBuilder().setValue(ByteString.copyFrom(counter.value.toByteArray)).build().toByteArray + ReplicatedEventSourcing.Counter + .newBuilder() + .setValue(ByteString.copyFrom(counter.value.toByteArray)) + .build() + .toByteArray def counterUpdatedToProtoBufByteArray(updated: Counter.Updated): Array[Byte] = - ActiveActive.CounterUpdate.newBuilder().setDelta(ByteString.copyFrom(updated.delta.toByteArray)).build().toByteArray + ReplicatedEventSourcing.CounterUpdate + .newBuilder() + .setDelta(ByteString.copyFrom(updated.delta.toByteArray)) + .build() + .toByteArray - def orsetToProto(orset: ORSet[_]): ActiveActive.ORSet = + def orsetToProto(orset: ORSet[_]): ReplicatedEventSourcing.ORSet = orsetToProtoImpl(orset.asInstanceOf[ORSet[Any]]) - private def orsetToProtoImpl(orset: ORSet[Any]): ActiveActive.ORSet = { + private def orsetToProtoImpl(orset: ORSet[Any]): ReplicatedEventSourcing.ORSet = { val b = - ActiveActive.ORSet.newBuilder().setOriginDc(orset.originReplica).setVvector(versionVectorToProto(orset.vvector)) + 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] @@ -194,7 +209,7 @@ object ActiveActiveSerializer { addDots(longElements) } if (!otherElements.isEmpty) { - Collections.sort(otherElements, ActiveActiveSerializer.Comparator) + Collections.sort(otherElements, ReplicatedEventSourcingSerializer.Comparator) b.addAllOtherElements(otherElements) addDots(otherElements) } @@ -203,7 +218,7 @@ object ActiveActiveSerializer { } def replicatedEventMetadataToProtoByteArray(rem: ReplicatedEventMetadata): Array[Byte] = { - ActiveActive.ReplicatedEventMetadata + ReplicatedEventSourcing.ReplicatedEventMetadata .newBuilder() .setOriginSequenceNr(rem.originSequenceNr) .setConcurrent(rem.concurrent) @@ -214,7 +229,7 @@ object ActiveActiveSerializer { } def replicatedSnapshotMetadataToByteArray(rsm: ReplicatedSnapshotMetadata): Array[Byte] = { - ActiveActive.ReplicatedSnapshotMetadata + ReplicatedEventSourcing.ReplicatedSnapshotMetadata .newBuilder() .setVersion(versionVectorToProto(rsm.version)) .addAllSeenPerReplica(rsm.seenPerReplica.map(seenToProto).asJava) @@ -222,35 +237,39 @@ object ActiveActiveSerializer { .toByteArray } - def seenToProto(t: (ReplicaId, Long)): ActiveActive.ReplicatedSnapshotMetadata.Seen = { - ActiveActive.ReplicatedSnapshotMetadata.Seen.newBuilder().setReplicaId(t._1.id).setSequenceNr(t._2).build() + 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(ActiveActive.ORSet.parseFrom(bytes)) + orsetFromProto(ReplicatedEventSourcing.ORSet.parseFrom(bytes)) private def orsetAddFromBinary(bytes: Array[Byte]): ORSet.AddDeltaOp[Any] = - new ORSet.AddDeltaOp(orsetFromProto(ActiveActive.ORSet.parseFrom(bytes))) + new ORSet.AddDeltaOp(orsetFromProto(ReplicatedEventSourcing.ORSet.parseFrom(bytes))) private def orsetRemoveFromBinary(bytes: Array[Byte]): ORSet.RemoveDeltaOp[Any] = - new ORSet.RemoveDeltaOp(orsetFromProto(ActiveActive.ORSet.parseFrom(bytes))) + new ORSet.RemoveDeltaOp(orsetFromProto(ReplicatedEventSourcing.ORSet.parseFrom(bytes))) private def orsetFullFromBinary(bytes: Array[Byte]): ORSet.FullStateDeltaOp[Any] = - new ORSet.FullStateDeltaOp(orsetFromProto(ActiveActive.ORSet.parseFrom(bytes))) + new ORSet.FullStateDeltaOp(orsetFromProto(ReplicatedEventSourcing.ORSet.parseFrom(bytes))) - private def orsetDeltaGroupToProto(deltaGroup: ORSet.DeltaGroup[_]): ActiveActive.ORSetDeltaGroup = { - def createEntry(opType: ActiveActive.ORSetDeltaOp, u: ORSet[_]) = { - ActiveActive.ORSetDeltaGroup.Entry.newBuilder().setOperation(opType).setUnderlying(orsetToProto(u)) + 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 = ActiveActive.ORSetDeltaGroup.newBuilder() + val b = ReplicatedEventSourcing.ORSetDeltaGroup.newBuilder() deltaGroup.ops.foreach { case ORSet.AddDeltaOp(u) => - b.addEntries(createEntry(ActiveActive.ORSetDeltaOp.Add, u)) + b.addEntries(createEntry(ReplicatedEventSourcing.ORSetDeltaOp.Add, u)) case ORSet.RemoveDeltaOp(u) => - b.addEntries(createEntry(ActiveActive.ORSetDeltaOp.Remove, u)) + b.addEntries(createEntry(ReplicatedEventSourcing.ORSetDeltaOp.Remove, u)) case ORSet.FullStateDeltaOp(u) => - b.addEntries(createEntry(ActiveActive.ORSetDeltaOp.Full, u)) + b.addEntries(createEntry(ReplicatedEventSourcing.ORSetDeltaOp.Full, u)) case ORSet.DeltaGroup(_) => throw new IllegalArgumentException("ORSet.DeltaGroup should not be nested") } @@ -258,14 +277,14 @@ object ActiveActiveSerializer { } private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = { - val deltaGroup = ActiveActive.ORSetDeltaGroup.parseFrom(bytes) + val deltaGroup = ReplicatedEventSourcing.ORSetDeltaGroup.parseFrom(bytes) val ops: Vector[ORSet.DeltaOp] = deltaGroup.getEntriesList.asScala.map { entry => - if (entry.getOperation == ActiveActive.ORSetDeltaOp.Add) + if (entry.getOperation == ReplicatedEventSourcing.ORSetDeltaOp.Add) ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)) - else if (entry.getOperation == ActiveActive.ORSetDeltaOp.Remove) + else if (entry.getOperation == ReplicatedEventSourcing.ORSetDeltaOp.Remove) ORSet.RemoveDeltaOp(orsetFromProto(entry.getUnderlying)) - else if (entry.getOperation == ActiveActive.ORSetDeltaOp.Full) + else if (entry.getOperation == ReplicatedEventSourcing.ORSetDeltaOp.Full) ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying)) else throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}") @@ -273,7 +292,7 @@ object ActiveActiveSerializer { ORSet.DeltaGroup(ops) } - def orsetFromProto(orset: ActiveActive.ORSet): ORSet[Any] = { + def orsetFromProto(orset: ReplicatedEventSourcing.ORSet): ORSet[Any] = { val elements: Iterator[Any] = (orset.getStringElementsList.iterator.asScala ++ orset.getIntElementsList.iterator.asScala ++ @@ -286,18 +305,19 @@ object ActiveActiveSerializer { new ORSet(orset.getOriginDc, elementsMap, vvector = versionVectorFromProto(orset.getVvector)) } - def versionVectorToProto(versionVector: VersionVector): ActiveActive.VersionVector = { - val b = ActiveActive.VersionVector.newBuilder() + def versionVectorToProto(versionVector: VersionVector): ReplicatedEventSourcing.VersionVector = { + val b = ReplicatedEventSourcing.VersionVector.newBuilder() versionVector.versionsIterator.foreach { - case (key, value) => b.addEntries(ActiveActive.VersionVector.Entry.newBuilder().setKey(key).setVersion(value)) + case (key, value) => + b.addEntries(ReplicatedEventSourcing.VersionVector.Entry.newBuilder().setKey(key).setVersion(value)) } b.build() } def versionVectorFromBinary(bytes: Array[Byte]): VersionVector = - versionVectorFromProto(ActiveActive.VersionVector.parseFrom(bytes)) + versionVectorFromProto(ReplicatedEventSourcing.VersionVector.parseFrom(bytes)) - def versionVectorFromProto(versionVector: ActiveActive.VersionVector): VersionVector = { + def versionVectorFromProto(versionVector: ReplicatedEventSourcing.VersionVector): VersionVector = { val entries = versionVector.getEntriesList if (entries.isEmpty) VersionVector.empty @@ -311,7 +331,7 @@ object ActiveActiveSerializer { } def replicatedEventMetadataFromBinary(bytes: Array[Byte]): ReplicatedEventMetadata = { - val parsed = ActiveActive.ReplicatedEventMetadata.parseFrom(bytes) + val parsed = ReplicatedEventSourcing.ReplicatedEventMetadata.parseFrom(bytes) ReplicatedEventMetadata( ReplicaId(parsed.getOriginReplica), parsed.getOriginSequenceNr, @@ -320,7 +340,7 @@ object ActiveActiveSerializer { } def replicatedSnapshotMetadataFromBinary(bytes: Array[Byte]): ReplicatedSnapshotMetadata = { - val parsed = ActiveActive.ReplicatedSnapshotMetadata.parseFrom(bytes) + val parsed = 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/scala/akka/persistence/typed/ActiveActiveSerializationSpec.scala b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSerializationSpec.scala similarity index 91% rename from akka-persistence-typed/src/test/scala/akka/persistence/typed/ActiveActiveSerializationSpec.scala rename to akka-persistence-typed/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSerializationSpec.scala index 84b4fb2991..05c6e3fa43 100644 --- a/akka-persistence-typed/src/test/scala/akka/persistence/typed/ActiveActiveSerializationSpec.scala +++ b/akka-persistence-typed/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingSerializationSpec.scala @@ -13,12 +13,12 @@ import akka.persistence.typed.internal.ReplicatedSnapshotMetadata import akka.persistence.typed.internal.VersionVector import org.scalatest.wordspec.AnyWordSpecLike -class ActiveActiveSerializationSpec +class ReplicatedEventSourcingSerializationSpec extends ScalaTestWithActorTestKit(ClusterSingletonPersistenceSpec.config) with AnyWordSpecLike with LogCapturing { - "The ActiveActive components that needs to be serializable" must { + "The Replicated Event Sourcing components that needs to be serializable" must { "be serializable" in { serializationTestKit.verifySerialization( 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 b3175e5777..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 @@ -67,7 +67,7 @@ class EventSourcedBehaviorWatchSpec holdingRecoveryPermit = false, settings = settings, stashState = new StashState(context.asInstanceOf[ActorContext[InternalProtocol]], settings), - activeActive = None, + replication = None, publishEvents = false) "A typed persistent parent actor watching a child" must { diff --git a/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala b/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala index 75b9c38480..d1d8f151ef 100644 --- a/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala +++ b/akka-persistence/src/main/scala/akka/persistence/SnapshotProtocol.scala @@ -11,7 +11,7 @@ import scala.runtime.AbstractFunction3 * @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 active active support + * @param metadata a journal can optionally support persisting metadata separate to the domain state, used for Replicated Event Sourcing support */ @SerialVersionUID(1L) final class SnapshotMetadata( From bf5ec198a4019b42f0289dc8ee4c56591f6f367c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Wed, 29 Jul 2020 15:59:55 +0200 Subject: [PATCH 27/50] Test case for three replicated event sourced entities added (#29438) --- .../typed/ReplicatedEventSourcingSpec.scala | 53 ++++++++++++++++++- 1 file changed, 52 insertions(+), 1 deletion(-) 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 index d50c426a7a..0bbbfb1a39 100644 --- 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 @@ -84,7 +84,7 @@ class ReplicatedEventSourcingSpec val ids = new AtomicInteger(0) def nextEntityId = s"e-${ids.getAndIncrement()}" "ReplicatedEventSourcing" should { - "replicate events between entities" in { + "replicate events between two entities" in { val entityId = nextEntityId val probe = createTestProbe[Done]() val r1 = spawn(testBehavior(entityId, "R1")) @@ -346,5 +346,56 @@ class ReplicatedEventSourcingSpec 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) + } + } + + } } } From 4b27bc34a882b10fe5beb52fdf174536c5893923 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Thu, 30 Jul 2020 10:43:16 +0200 Subject: [PATCH 28/50] Document hot standby for replicated event sourcing (#29439) --- .../src/main/paradox/typed/replicated-eventsourcing.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index df09cd5049..f1073f7a2b 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -287,6 +287,12 @@ The "event publishing" feature publishes each event to the local system event bu the @apidoc[ShardingDirectReplication] actor subscribes to these events and forwards them to the replicas allowing them to fast forward the stream of events for the origin replica. (With additional potential future support in journals for fast forwarding [#29311](https://github.com/akka/akka/issues/29311)). +## Hot Standby + +If all writes occur to one replica 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. + ## 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] @@ -295,4 +301,4 @@ For a journal plugin to support replication it needs to store and read metadata 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. \ No newline at end of file +The @apidoc[SnapshotStoreSpec] in the Persistence TCK provides a capability flag `supportsMetadata` to toggle verification that metadata is handled correctly. From 742352caba180081450aa22169cd4905b7823621 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Thu, 30 Jul 2020 10:03:46 +0100 Subject: [PATCH 29/50] Restart replication stream from correct seen seqNr (#29436) --- .../typed/ReplicatedEventSourcingSpec.scala | 24 ++++++++ .../internal/EventSourcedBehaviorImpl.scala | 5 ++ .../typed/internal/ReplayingEvents.scala | 3 +- .../typed/internal/ReplayingSnapshot.scala | 3 +- .../persistence/typed/internal/Running.scala | 58 +++++++++++++------ 5 files changed, 72 insertions(+), 21 deletions(-) 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 index 0bbbfb1a39..4a4c19766b 100644 --- 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 @@ -13,6 +13,7 @@ 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 @@ -397,5 +398,28 @@ class ReplicatedEventSourcingSpec } } + + "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/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/EventSourcedBehaviorImpl.scala index 299cfab3d5..56daa2493d 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 @@ -73,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 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 992b2a1ca4..037a056041 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 @@ -20,7 +20,7 @@ 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 @@ -97,6 +97,7 @@ private[akka] final class ReplayingEvents[C, E, S]( 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 } } 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 5b8d538fab..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 @@ -12,7 +12,7 @@ import akka.persistence._ import akka.persistence.SnapshotProtocol.LoadSnapshotFailed import akka.persistence.SnapshotProtocol.LoadSnapshotResult import akka.persistence.typed.{ RecoveryFailed, ReplicaId } -import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState +import akka.persistence.typed.internal.EventSourcedBehaviorImpl.{ GetSeenSequenceNr, GetState } import akka.util.unused import akka.actor.typed.scaladsl.LoggerOps @@ -71,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 { 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 5815e80653..0bfff4afed 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 @@ -48,7 +48,7 @@ import akka.persistence.typed.{ SnapshotMetadata, SnapshotSelectionCriteria } -import akka.persistence.typed.internal.EventSourcedBehaviorImpl.GetState +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 @@ -128,33 +128,39 @@ private[akka] object Running { val query = PersistenceQuery(system) replicationSetup.allReplicas.foldLeft(state) { (state, replicaId) => if (replicaId != replicationSetup.replicaId) { - val seqNr = state.seenPerReplica(replicaId) val pid = PersistenceId.replicatedUniqueId(replicationSetup.aaContext.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) { () => - replication - .eventsByPersistenceId(pid.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)) + Source.futureSource { + setup.context.self.ask[Long](replyTo => GetSeenSequenceNr(replicaId, replyTo)).map { seqNr => + replication + .eventsByPersistenceId(pid.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 @@ -177,7 +183,7 @@ private[akka] object Running { source.runWith(Sink.ignore)(SystemMaterializer(system).materializer) - // FIXME support from journal to fast forward https://github.com/akka/akka/issues/29311 + // TODO support from journal to fast forward https://github.com/akka/akka/issues/29311 state.copy( replicationControl = state.replicationControl.updated(replicaId, new ReplicationStreamControl { @@ -244,6 +250,7 @@ private[akka] object Running { 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 } @@ -386,6 +393,11 @@ private[akka] object Running { this } + def onGetSeenSequenceNr(get: GetSeenSequenceNr): Behavior[InternalProtocol] = { + get.replyTo ! state.seenPerReplica(get.replica) + this + } + def withContext[A](aa: ReplicationSetup, withReplication: ReplicationSetup => Unit, f: () => A): A = { withReplication(aa) val result = f() @@ -632,6 +644,7 @@ private[akka] object Running { 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 @@ -648,6 +661,11 @@ 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 @@ -826,6 +844,8 @@ private[akka] object Running { } case get: GetState[S @unchecked] => stashInternal(get) + case get: GetSeenSequenceNr => + stashInternal(get) case _ => Behaviors.unhandled } From c945fbd7a1b492e514f756585c5241c4aecace87 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Thu, 30 Jul 2020 17:03:46 +0100 Subject: [PATCH 30/50] Java auction example (#29443) --- .../replicated-eventsourcing-examples.md | 22 +- .../typed/ReplicatedAuctionExampleTest.java | 400 ++++++++++++++++++ .../typed/ReplicatedAuctionExampleSpec.scala | 6 +- build.sbt | 1 + 4 files changed, 425 insertions(+), 4 deletions(-) create mode 100644 akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md index d7b49e92eb..e74119b2f2 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md @@ -18,11 +18,17 @@ 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: @@ -30,6 +36,9 @@ 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 @@ -43,21 +52,29 @@ 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. @@ -97,6 +114,9 @@ In the event handler above, when recovery is not running, it calls `eventTrigger 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 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..70d38cf566 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedAuctionExampleTest.java @@ -0,0 +1,400 @@ +/* + * 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.persistence.testkit.PersistenceTestKitPlugin; +import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal; +import akka.persistence.typed.ReplicaId; +import akka.persistence.typed.javadsl.*; +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.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.ReplicatedAuctionExample.*; +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() { + AuctionSetup setupA = + new AuctionSetup( + "old-skis", + new Bid("chbatey", 12, Instant.now(), R1), + Instant.now().plusSeconds(10), + true); + + AuctionSetup setupB = + new AuctionSetup( + "old-skis", + new Bid("chbatey", 12, Instant.now(), R1), + Instant.now().plusSeconds(10), + false); + + ActorRef replicaA = testKit.spawn(create(setupA, R1)); + ActorRef replicaB = testKit.spawn(create(setupA, R2)); + + 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); + }); + } +} + +class ReplicatedAuctionExample + extends ReplicatedEventSourcedBehavior { + + 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 AuctionSetup setup; + + public static Behavior create(AuctionSetup setup, ReplicaId replica) { + return Behaviors.setup( + ctx -> + ReplicatedEventSourcing.withSharedJournal( + setup.name, + replica, + ALL_REPLICAS, + PersistenceTestKitReadJournal.Identifier(), + replicationCtx -> new ReplicatedAuctionExample(replicationCtx, ctx, setup))); + } + + public ReplicatedAuctionExample( + ReplicationContext replicationContext, ActorContext context, AuctionSetup setup) { + super(replicationContext); + this.context = context; + this.setup = setup; + } + + //#setup + static class AuctionSetup { + final String name; + final Bid initialBid; // the initial bid is the minimum price bidden at start time by the owner + final Instant closingAt; + final boolean responsibleForClosing; + + public AuctionSetup( + String name, Bid initialBid, Instant closingAt, boolean responsibleForClosing) { + this.name = name; + this.initialBid = initialBid; + this.closingAt = closingAt; + this.responsibleForClosing = responsibleForClosing; + } + } + //#setup + + public static final class Bid implements CborSerializable { + 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; + } + } + + //#commands + 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 + + @Override + public AuctionState emptyState() { + return new AuctionState(true, setup.initialBid, setup.initialBid.offer, Collections.emptySet()); + } + + //#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 + + @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-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); + } + } + //#event-triggers + + private boolean shouldClose(AuctionState state) { + return setup.responsibleForClosing + && !state.isClosed() + && getReplicationContext().getAllReplicas().stream() + .map(ReplicaId::id) + .collect(Collectors.toSet()) + .equals(state.finishedAtDc); + } +} 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 index e0c4d408e2..f5aaa3f9b2 100644 --- 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 @@ -22,7 +22,7 @@ object ReplicatedAuctionExampleSpec { type MoneyAmount = Int - case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originDc: ReplicaId) + case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originReplica: ReplicaId) //#commands sealed trait AuctionCommand @@ -89,8 +89,8 @@ object ReplicatedAuctionExampleSpec { // 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.originDc.id - .compareTo(second.originDc.id) < 0) + (first.offer == second.offer && first.timestamp.equals(second.timestamp) && first.originReplica.id + .compareTo(second.originReplica.id) < 0) } //#state diff --git a/build.sbt b/build.sbt index 87c6ce871a..0794926692 100644 --- a/build.sbt +++ b/build.sbt @@ -318,6 +318,7 @@ 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) From 9fb76bbea41d29d8f49e32ffed1ce343cd0ff127 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Johan=20Andr=C3=A9n?= Date: Fri, 31 Jul 2020 14:07:37 +0200 Subject: [PATCH 31/50] Tagging for replicated event sourcing (#29442) * Tagging for replicated event sourcing * Docs improvements * Support for currentEventsByTag in persistence testkit and test coverage for tags in replicated --- .../paradox/typed/replicated-eventsourcing.md | 42 +++- .../persistence/testkit/EventStorage.scala | 24 ++- .../testkit/PersistenceTestKitPlugin.scala | 20 +- .../internal/SerializedEventStorageImpl.scala | 13 +- .../testkit/internal/TestKitStorage.scala | 5 + .../internal/EventsByPersistenceIdStage.scala | 14 +- .../PersistenceTestKitReadJournal.scala | 15 +- .../PersistenceTestKitReadJournal.scala | 37 +++- .../testkit/scaladsl/PersistenceTestKit.scala | 6 +- .../query/CurrentEventsByTagSpec.scala | 68 ++++++ .../typed/MyReplicatedBehavior.java | 69 +++++++ ...eplicatedEventSourcingCompileOnlyTest.java | 84 -------- .../typed/ReplicatedStringSet.java | 86 ++++++++ .../ReplicatedEventSourcingTaggingSpec.scala | 114 ++++++++++ .../typed/internal/ReplayingEvents.scala | 10 +- .../persistence/typed/internal/Running.scala | 194 ++++++++---------- 16 files changed, 585 insertions(+), 216 deletions(-) create mode 100644 akka-persistence-testkit/src/test/scala/akka/persistence/testkit/query/CurrentEventsByTagSpec.scala create mode 100644 akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java delete mode 100644 akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java create mode 100644 akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedStringSet.java create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingTaggingSpec.scala diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index f1073f7a2b..b0f555141b 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -66,7 +66,7 @@ Scala : @@snip [ReplicatedEventSourcingCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala) { #replicas } Java -: @@snip [ReplicatedEventSourcingCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java) { #replicas } +: @@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: @@ -75,7 +75,7 @@ Scala : @@snip [ReplicatedEventSourcingCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala) { #factory } Java -: @@snip [ReplicatedEventSourcingCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java) { #factory } +: @@snip [MyReplicatedBehavior.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java) { #factory } The factory takes in: @@ -90,7 +90,7 @@ Scala : @@snip [ReplicatedEventSourcingCompileOnlySpec.scala](/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlySpec.scala) { #factory-shared} Java -: @@snip [ReplicatedEventSourcingCompileOnlyTest.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java) { #factory-shared } +: @@snip [MyReplicatedBehavior.java](/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java) { #factory-shared } @@@ div { .group-scala } @@ -271,6 +271,42 @@ Java 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 the 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 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 14f8c90a46..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 @@ -10,6 +10,7 @@ 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._ @@ -46,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 = @@ -81,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) 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 8738d8c206..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 @@ -12,7 +12,8 @@ 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 @@ -35,8 +36,7 @@ class PersistenceTestKitPlugin(@unused cfg: Config, cfgPath: String) extends Asy Future.fromTry(Try(messages.map(aw => { val data = aw.payload.map(pl => pl.payload match { - case Tagged(p, _) => pl.withPayload(p).withTimestamp(System.currentTimeMillis()) - case _ => pl.withTimestamp(System.currentTimeMillis()) + case _ => pl.withTimestamp(System.currentTimeMillis()) }) val result: Try[Unit] = storage.tryAdd(data) @@ -54,7 +54,19 @@ class PersistenceTestKitPlugin(@unused cfg: Config, cfgPath: String) extends Asy 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 { 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 895874a423..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 @@ -7,6 +7,7 @@ package akka.persistence.testkit.internal 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.persistence.testkit.internal.SerializedEventStorageImpl.Serialized import akka.serialization.{ Serialization, SerializationExtension, Serializers } @@ -20,6 +21,7 @@ private[testkit] object SerializedEventStorageImpl { payloadSerManifest: String, writerUuid: String, payload: Array[Byte], + tags: Set[String], metadata: Option[Any]) } @@ -38,7 +40,10 @@ private[testkit] class SerializedEventStorageImpl(system: ActorSystem) extends E */ override def toInternal(pr: PersistentRepr): Serialized = Serialization.withTransportInformation(system.asInstanceOf[ExtendedActorSystem]) { () => - val payload = pr.payload.asInstanceOf[AnyRef] + 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( @@ -48,6 +53,7 @@ private[testkit] class SerializedEventStorageImpl(system: ActorSystem) extends E manifest, pr.writerUuid, s.toBinary(payload), + tags, pr.metadata) } @@ -56,7 +62,10 @@ private[testkit] class SerializedEventStorageImpl(system: ActorSystem) extends E */ override def toRepr(internal: Serialized): PersistentRepr = { val event = serialization.deserialize(internal.payload, internal.payloadSerId, internal.payloadSerManifest).get - val pr = PersistentRepr(event, internal.sequenceNr, internal.persistenceId, writerUuid = internal.writerUuid) + 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/TestKitStorage.scala b/akka-persistence-testkit/src/main/scala/akka/persistence/testkit/internal/TestKitStorage.scala index 6f85dd7d83..47c6ddc7fb 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,6 +8,7 @@ import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicReference import scala.collection.immutable +import scala.collection.JavaConverters._ import akka.annotation.InternalApi import akka.persistence.testkit.ProcessingPolicy @@ -86,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() 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 index 1f2bc623d0..5d82e6bb1e 100644 --- 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 @@ -5,6 +5,7 @@ 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 } @@ -49,15 +50,10 @@ final private[akka] class EventsByPersistenceIdStage( 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, - pr.timestamp, - pr.metadata)) + 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 { 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 index 9114994f10..2b96285343 100644 --- 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 @@ -5,7 +5,13 @@ package akka.persistence.testkit.query.javadsl import akka.NotUsed import akka.persistence.query.EventEnvelope -import akka.persistence.query.javadsl.{ CurrentEventsByPersistenceIdQuery, EventsByPersistenceIdQuery, ReadJournal } +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 @@ -16,7 +22,8 @@ object PersistenceTestKitReadJournal { final class PersistenceTestKitReadJournal(delegate: scaladsl.PersistenceTestKitReadJournal) extends ReadJournal with EventsByPersistenceIdQuery - with CurrentEventsByPersistenceIdQuery { + with CurrentEventsByPersistenceIdQuery + with CurrentEventsByTagQuery { override def eventsByPersistenceId( persistenceId: String, @@ -29,4 +36,8 @@ final class PersistenceTestKitReadJournal(delegate: scaladsl.PersistenceTestKitR 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 index 1df3f34bde..491032f04f 100644 --- 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 @@ -5,6 +5,10 @@ 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 @@ -22,7 +26,8 @@ object PersistenceTestKitReadJournal { final class PersistenceTestKitReadJournal(system: ExtendedActorSystem, @unused config: Config, configPath: String) extends ReadJournal with EventsByPersistenceIdQuery - with CurrentEventsByPersistenceIdQuery { + with CurrentEventsByPersistenceIdQuery + with CurrentEventsByTagQuery { private val log = LoggerFactory.getLogger(getClass) @@ -33,6 +38,11 @@ final class PersistenceTestKitReadJournal(system: ExtendedActorSystem, @unused c 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, @@ -45,7 +55,30 @@ final class PersistenceTestKitReadJournal(system: ExtendedActorSystem, @unused c fromSequenceNr: Long, toSequenceNr: Long): Source[EventEnvelope, NotUsed] = { Source(storage.tryRead(persistenceId, fromSequenceNr, toSequenceNr, Long.MaxValue)).map { pr => - EventEnvelope(Sequence(pr.sequenceNr), persistenceId, pr.sequenceNr, pr.payload, pr.timestamp, pr.metadata) + 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 2fa2033421..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 @@ -18,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 @@ -493,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-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..3ada97caf7 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/MyReplicatedBehavior.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package jdocs.akka.persistence.typed; + +import akka.actor.typed.Behavior; +import akka.persistence.typed.ReplicaId; +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.withSharedJournal( + 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.create( + 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/ReplicatedEventSourcingCompileOnlyTest.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java deleted file mode 100644 index d9928f2bd6..0000000000 --- a/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedEventSourcingCompileOnlyTest.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Copyright (C) 2020 Lightbend Inc. - */ - -package jdocs.akka.persistence.typed; - -import akka.persistence.typed.ReplicaId; -import akka.persistence.typed.javadsl.*; - -import java.util.*; - -public class ReplicatedEventSourcingCompileOnlyTest { - - // dummy for docs example - interface Command {} - - interface Event {} - - interface State {} - - static // #factory - final class MyReplicatedEventSourcedBehavior - extends ReplicatedEventSourcedBehavior { - - public MyReplicatedEventSourcedBehavior(ReplicationContext replicationContext) { - super(replicationContext); - } - // ... implementation of abstract methods ... - // #factory - - @Override - public State emptyState() { - return null; - } - - @Override - public CommandHandler commandHandler() { - return null; - } - - @Override - public EventHandler eventHandler() { - return null; - } - // #factory - } - - // #factory - - { - // #replicas - ReplicaId DCA = new ReplicaId("DC-A"); - ReplicaId DCB = new ReplicaId("DC-B"); - Set allReplicas = - Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DCA, DCB))); - // #replicas - - String queryPluginId = ""; - - // #factory-shared - ReplicatedEventSourcing.withSharedJournal( - "entityId", - DCA, - allReplicas, - queryPluginId, - context -> new MyReplicatedEventSourcedBehavior(context)); - // #factory-shared - - // #factory - - // bootstrap logic - Map allReplicasAndQueryPlugins = new HashMap<>(); - allReplicasAndQueryPlugins.put(DCA, "journalForDCA"); - allReplicasAndQueryPlugins.put(DCB, "journalForDCB"); - - EventSourcedBehavior behavior = - ReplicatedEventSourcing.create( - "entityId", - DCA, - allReplicasAndQueryPlugins, - context -> new MyReplicatedEventSourcedBehavior(context)); - // #factory - } -} 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..0dc9e078a0 --- /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.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.withSharedJournal( + 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/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..033a8dee14 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/ReplicatedEventSourcingTaggingSpec.scala @@ -0,0 +1,114 @@ +/* + * 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.withSharedJournal(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/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/ReplayingEvents.scala index 037a056041..1bac5c2683 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 @@ -125,7 +125,7 @@ private[akka] final class ReplayingEvents[C, E, S]( val aaMetaAndSelfReplica: Option[(ReplicatedEventMetadata, ReplicaId, ReplicationSetup)] = setup.replication match { - case Some(aa) => + case Some(replication) => val meta = repr.metadata match { case Some(m) => m.asInstanceOf[ReplicatedEventMetadata] case None => @@ -133,16 +133,16 @@ private[akka] final class ReplayingEvents[C, E, S]( s"Replicated Event Sourcing enabled but existing event has no metadata. Migration isn't supported yet.") } - aa.setContext(recoveryRunning = true, meta.originReplica, meta.concurrent) - Some((meta, aa.replicaId, aa)) + 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(aa) => - aa.clearContext() + case Some(replication) => + replication.clearContext() case 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 0bfff4afed..e0fb66ffe8 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 @@ -398,13 +398,6 @@ private[akka] object Running { this } - def withContext[A](aa: ReplicationSetup, withReplication: ReplicationSetup => Unit, f: () => A): A = { - withReplication(aa) - val result = f() - aa.clearContext() - result - } - private def handleExternalReplicatedEventPersist( replication: ReplicationSetup, event: ReplicatedEvent[E]): Behavior[InternalProtocol] = { @@ -421,17 +414,20 @@ private[akka] object Running { updatedVersion, isConcurrent) - val newState: RunningState[S] = withContext( - replication, - aa => aa.setContext(recoveryRunning = false, event.originReplica, concurrent = isConcurrent), - () => state.applyEvent(setup, event.event)) + 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, - newState, - event.event, - "", + stateAfterApply, + eventToPersist, + eventAdapterManifest, OptionVal.Some( ReplicatedEventMetadata(event.originReplica, event.originSequenceNr, updatedVersion, isConcurrent))) val shouldSnapshotAfterPersist = setup.shouldSnapshot(newState2.state, event.event, newState2.seqNr) @@ -449,51 +445,48 @@ private[akka] object Running { event: E, cmd: Any, sideEffects: immutable.Seq[SideEffect[S]]): (Behavior[InternalProtocol], Boolean) = { - // 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 + 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 - val newState: RunningState[S] = setup.replication match { - case Some(aa) => - // set concurrent to false, local events are never concurrent - withContext( - aa, - aa => aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false), - () => state.applyEvent(setup, event)) - case None => - state.applyEvent(setup, event) + 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(aa) => + val updatedVersion = stateAfterApply.version.updated(aa.replicaId.id, _currentSequenceNumber) + val r = internalPersist( + setup.context, + cmd, + stateAfterApply, + eventToPersist, + eventAdapterManifest, + OptionVal.Some( + ReplicatedEventMetadata(aa.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, sideEffects), false) + + } finally { + setup.replication.foreach(_.clearContext()) } - - val eventToPersist = adaptEvent(event) - val eventAdapterManifest = setup.eventAdapter.manifest(event) - - val newState2 = setup.replication match { - case Some(aa) => - val updatedVersion = newState.version.updated(aa.replicaId.id, _currentSequenceNumber) - val r = internalPersist( - setup.context, - cmd, - newState, - eventToPersist, - eventAdapterManifest, - OptionVal.Some( - ReplicatedEventMetadata(aa.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, newState, eventToPersist, eventAdapterManifest, OptionVal.None) - } - - val shouldSnapshotAfterPersist = setup.shouldSnapshot(newState2.state, event, newState2.seqNr) - (persistingEvents(newState2, state, numberOfEvents = 1, shouldSnapshotAfterPersist, sideEffects), false) } private def handleEventPersistAll( @@ -501,64 +494,59 @@ private[akka] object Running { cmd: Any, sideEffects: immutable.Seq[SideEffect[S]]): (Behavior[InternalProtocol], Boolean) = { 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 + 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(aa) => - aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false) // local events are never concurrent - Some(ReplicatedEventMetadata(aa.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)) + val metadataTemplate: Option[ReplicatedEventMetadata] = setup.replication match { + case Some(aa) => + aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false) // local events are never concurrent + Some(ReplicatedEventMetadata(aa.replicaId, 0L, state.version, concurrent = false)) // we replace it with actual seqnr later case None => None } - currentState = setup.replication match { - case Some(aa) => - withContext( - aa, - aa => aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false), - () => currentState.applyEvent(setup, event)) - case None => - currentState.applyEvent(setup, event) + 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 } - eventsToPersist = EventToPersist(adaptedEvent, evtManifest, eventMetadata) :: eventsToPersist + val newState2 = + internalPersistAll(setup.context, cmd, currentState, eventsToPersist.reverse) + + (persistingEvents(newState2, state, events.size, shouldSnapshotAfterPersist, sideEffects), false) + } finally { + setup.replication.foreach(_.clearContext()) } - - val newState2 = - internalPersistAll(setup.context, cmd, currentState, eventsToPersist.reverse) - - (persistingEvents(newState2, state, events.size, shouldSnapshotAfterPersist, sideEffects), false) } else { // run side-effects even when no events are emitted (applySideEffects(sideEffects, state), true) } } - @tailrec def applyEffects( msg: Any, state: RunningState[S], From cd821fe3f8d428b2b36c74b82bedbe1ff2ba53a0 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 4 Aug 2020 08:12:45 +0200 Subject: [PATCH 32/50] Minor adjustments to replicated es docs (#29450) * Minor adjustments to replicated es docs * and a few aa leftovers * link to more examples --- .../typed/ReplicatedShardingTest.java | 17 +++-- .../typed/ReplicatedShardingSpec.scala | 18 ++--- .../main/paradox/typed/index-persistence.md | 4 +- .../paradox/typed/replicated-eventsourcing.md | 71 ++++++++++--------- .../typed/MultiJournalReplicationSpec.scala | 4 +- .../typed/ReplicatedEventPublishingSpec.scala | 4 +- .../typed/ReplicatedEventSourcingSpec.scala | 15 ++-- .../typed/ReplicationIllegalAccessSpec.scala | 12 ++-- .../typed/ReplicationSnapshotSpec.scala | 4 +- .../akka/persistence/typed/crdt/LwwSpec.scala | 8 +-- .../persistence/typed/crdt/ORSetSpec.scala | 4 +- .../typed/ReplicatedAuctionExampleSpec.scala | 37 +++++----- .../typed/ReplicatedBlogExampleSpec.scala | 24 +++---- .../typed/internal/ReplayingEvents.scala | 12 ++-- .../typed/internal/ReplicationSetup.scala | 18 ++--- .../persistence/typed/internal/Running.scala | 26 ++++--- 16 files changed, 151 insertions(+), 127 deletions(-) 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 index d9cd4dcab9..eeab3cfb16 100644 --- 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 @@ -29,6 +29,7 @@ import org.junit.Test; import org.scalatestplus.junit.JUnitSuite; import java.util.*; +import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import static org.junit.Assert.assertEquals; @@ -144,7 +145,7 @@ public class ReplicatedShardingTest extends JUnitSuite { private final ReplicatedSharding< MyReplicatedStringSet.Command, ShardingEnvelope> - aaSharding; + replicatedSharding; private ProxyActor(ActorContext context) { super(context); @@ -152,7 +153,7 @@ public class ReplicatedShardingTest extends JUnitSuite { // #bootstrap ReplicatedShardingSettings< MyReplicatedStringSet.Command, ShardingEnvelope> - aaShardingSettings = + replicatedShardingSettings = ReplicatedShardingSettings.create( MyReplicatedStringSet.Command.class, ALL_REPLICAS, @@ -176,8 +177,12 @@ public class ReplicatedShardingTest extends JUnitSuite { ReplicatedShardingExtension extension = ReplicatedShardingExtension.get(getContext().getSystem()); - aaSharding = extension.init(aaShardingSettings); + ReplicatedSharding< + MyReplicatedStringSet.Command, ShardingEnvelope> + replicatedSharding = extension.init(replicatedShardingSettings); // #bootstrap + + this.replicatedSharding = replicatedSharding; } @Override @@ -190,8 +195,8 @@ public class ReplicatedShardingTest extends JUnitSuite { private Behavior onForwardToRandom(ForwardToRandom forwardToRandom) { Map> refs = - aaSharding.getEntityRefsFor(forwardToRandom.entityId); - int chosenIdx = new java.util.Random().nextInt(refs.size()); + replicatedSharding.getEntityRefsFor(forwardToRandom.entityId); + int chosenIdx = ThreadLocalRandom.current().nextInt(refs.size()); new ArrayList<>(refs.values()).get(chosenIdx).tell(forwardToRandom.message); return this; } @@ -199,7 +204,7 @@ public class ReplicatedShardingTest extends JUnitSuite { private Behavior onForwardToAll(ForwardToAll forwardToAll) { // #all-entity-refs Map> refs = - aaSharding.getEntityRefsFor(forwardToAll.entityId); + replicatedSharding.getEntityRefsFor(forwardToAll.entityId); refs.forEach((replicaId, ref) -> ref.tell(forwardToAll.message)); // #all-entity-refs return this; 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 index d87a13673f..90d4d5fa22 100644 --- 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 @@ -4,6 +4,8 @@ package akka.cluster.sharding.typed +import java.util.concurrent.ThreadLocalRandom + import akka.actor.testkit.typed.scaladsl.LogCapturing import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.typed.ActorRef @@ -23,8 +25,6 @@ import akka.serialization.jackson.CborSerializable import com.typesafe.config.ConfigFactory import org.scalatest.wordspec.AnyWordSpecLike -import scala.util.Random - object ReplicatedShardingSpec { def config = ConfigFactory.parseString(""" akka.loglevel = DEBUG @@ -52,9 +52,9 @@ class ReplicatedShardingSpec entityId, replicaId, allReplicas, - PersistenceTestKitReadJournal.Identifier) { aaContext => + PersistenceTestKitReadJournal.Identifier) { replicationContext => EventSourcedBehavior[Command, String, Set[String]]( - aaContext.persistenceId, + replicationContext.persistenceId, Set.empty[String], (state, command) => command match { @@ -75,7 +75,7 @@ class ReplicatedShardingSpec def apply(): Behavior[Command] = Behaviors.setup { context => // #bootstrap - val aaShardingSettings = + val replicatedShardingSettings = ReplicatedShardingSettings[MyReplicatedStringSet.Command, ShardingEnvelope[MyReplicatedStringSet.Command]]( // all replicas Set(ReplicaId("DC-A"), ReplicaId("DC-B"), ReplicaId("DC-C"))) { (entityTypeKey, replicaId, allReplicaIds) => @@ -93,20 +93,20 @@ class ReplicatedShardingSpec .withRole(replicaId.id)) } - val aaSharding = ReplicatedShardingExtension(context.system).init(aaShardingSettings) + val replicatedSharding = ReplicatedShardingExtension(context.system).init(replicatedShardingSettings) // #bootstrap Behaviors.receiveMessage { case ForwardToAll(entityId, cmd) => // #all-entity-refs - aaSharding.entityRefsFor(entityId).foreach { + replicatedSharding.entityRefsFor(entityId).foreach { case (_, ref) => ref ! cmd } // #all-entity-refs Behaviors.same case ForwardToRandom(entityId, cmd) => - val refs = aaSharding.entityRefsFor(entityId) - val chosenIdx = (new Random()).nextInt(refs.size) + val refs = replicatedSharding.entityRefsFor(entityId) + val chosenIdx = ThreadLocalRandom.current().nextInt(refs.size) refs.values.toIndexedSeq(chosenIdx) ! cmd; Behaviors.same } diff --git a/akka-docs/src/main/paradox/typed/index-persistence.md b/akka-docs/src/main/paradox/typed/index-persistence.md index 2a7fd09aa5..739a8d2349 100644 --- a/akka-docs/src/main/paradox/typed/index-persistence.md +++ b/akka-docs/src/main/paradox/typed/index-persistence.md @@ -9,7 +9,7 @@ project.description: Event Sourcing with Akka Persistence enables actors to pers @@@ index * [persistence](persistence.md) -* [active-active](replicated-eventsourcing.md) +* [replicated-eventsourcing](replicated-eventsourcing.md) * [cqrs](cqrs.md) * [persistence-style](persistence-style.md) * [persistence-snapshot](persistence-snapshot.md) @@ -20,6 +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) -* [active-active-examples](replicated-eventsourcing-examples.md) +* [replicated-eventsourcing-examples](replicated-eventsourcing-examples.md) @@@ diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index b0f555141b..66e8932173 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -10,8 +10,8 @@ warning or deprecation period. It is also not recommended to use this module in @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, `EventSourcedBehaviors`s are unavailable. - +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. @@ -31,15 +31,16 @@ However, the event handler must be able to **handle concurrent events** as when there is no longer the single writer principle as there 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 and the event handler and those reading the state must be designed to handle this. +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)) +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 a `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. +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) @@ -51,14 +52,14 @@ The event handler logic for applying events to the state of the entity must be a For example, sometimes it's enough to use application specific timestamps to decide which update should win. -To assist in implementing the event handler active-active detects these conflicts. +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 active-active -replication @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`]. +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: @@ -79,9 +80,9 @@ Java The factory takes in: -* EntityID: this will be used as part of the underlying persistenceId -* Replica: Which replica this instance is -* All Replicas and the query plugin used to read their events +* `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. @@ -97,10 +98,10 @@ Java 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**. +* `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. @@ -113,19 +114,19 @@ concrete `ReplicatedEventSourcedBehavior` and on to the super constructor. The context gives access to: -* entityId -* replicaId -* allReplicas -* persistenceId +* `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 +* `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. @@ -146,11 +147,11 @@ Sometimes it is enough to use timestamps to decide which update should win. Such 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 data center sorted first in +identifier is used if the two timestamps are equal, and then the one from the `replicaId` sorted first in alphanumeric order wins. 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 data centers will be different and +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 @@ -201,7 +202,7 @@ You don’t have to read this section to be able to use the feature, but to use ### Causal deliver order -Causal delivery order means that events persisted in one data center are read in the same order in other data centers. The order of concurrent events is undefined, which should be no problem +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 [CRDT's](#conflict-free-replicated-data-types) and otherwise will be detected via the `ReplicationContext` concurrent method. @@ -213,7 +214,7 @@ 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 data center DC-3 these events will be read in the same order e1, e2, 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: @@ -227,7 +228,7 @@ 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 data center may also see the events as either "e1, e3, e2" or as "e1, e2, e3". +A third replica may also see the events as either "e1, e3, e2" or as "e1, e2, e3". ### Concurrent updates @@ -239,9 +240,9 @@ Each replica "owns" a slot in the version vector and increases its counter when 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 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 @@ -276,7 +277,7 @@ More advanced routing among the replicas is currently left as an exercise for th 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 the these event +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/) @@ -325,10 +326,14 @@ to fast forward the stream of events for the origin replica. (With additional po ## Hot Standby -If all writes occur to one replica 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. +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] 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 index 8de42eacaf..d155d9d6dc 100644 --- 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 @@ -36,9 +36,9 @@ object MultiJournalReplicationSpec { entityId, ReplicaId(replicaId), Map(ReplicaId("R1") -> "journal1.query", ReplicaId("R2") -> "journal2.query"))( - aaContext => + replicationContext => EventSourcedBehavior[Command, String, Set[String]]( - aaContext.persistenceId, + replicationContext.persistenceId, Set.empty[String], (state, command) => command match { 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 index 90c498efb8..c8a88b947c 100644 --- 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 @@ -33,9 +33,9 @@ object ReplicatedEventPublishingSpec { replicaId, allReplicas, PersistenceTestKitReadJournal.Identifier)( - aactx => + replicationctx => EventSourcedBehavior[Command, String, Set[String]]( - aactx.persistenceId, + replicationctx.persistenceId, Set.empty, (state, command) => command match { 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 index 4a4c19766b..beeb5c0863 100644 --- 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 @@ -36,10 +36,10 @@ object ReplicatedEventSourcingSpec { testBehavior(entityId, replicaId, Some(probe)) def eventSourcedBehavior( - aaContext: ReplicationContext, + replicationContext: ReplicationContext, probe: Option[ActorRef[EventAndContext]]): EventSourcedBehavior[Command, String, State] = { EventSourcedBehavior[Command, String, State]( - aaContext.persistenceId, + replicationContext.persistenceId, State(Nil), (state, command) => command match { @@ -47,7 +47,7 @@ object ReplicatedEventSourcingSpec { replyTo ! state Effect.none case GetReplica(replyTo) => - replyTo.tell((aaContext.replicaId, aaContext.allReplicas)) + replyTo.tell((replicationContext.replicaId, replicationContext.allReplicas)) Effect.none case StoreMe(evt, ack) => Effect.persist(evt).thenRun(_ => ack ! Done) @@ -57,7 +57,12 @@ object ReplicatedEventSourcingSpec { Effect.stop() }, (state, event) => { - probe.foreach(_ ! EventAndContext(event, aaContext.origin, aaContext.recoveryRunning, aaContext.concurrent)) + probe.foreach( + _ ! EventAndContext( + event, + replicationContext.origin, + replicationContext.recoveryRunning, + replicationContext.concurrent)) state.copy(all = event :: state.all) }) } @@ -70,7 +75,7 @@ object ReplicatedEventSourcingSpec { entityId, ReplicaId(replicaId), AllReplicas, - PersistenceTestKitReadJournal.Identifier)(aaContext => eventSourcedBehavior(aaContext, probe)) + PersistenceTestKitReadJournal.Identifier)(replicationContext => eventSourcedBehavior(replicationContext, probe)) } 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 index 8415b0dcd9..11b28e9f76 100644 --- 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 @@ -29,15 +29,15 @@ object ReplicationIllegalAccessSpec { def apply(entityId: String, replica: ReplicaId): Behavior[Command] = { ReplicatedEventSourcing.withSharedJournal(entityId, replica, AllReplicas, PersistenceTestKitReadJournal.Identifier)( - aaContext => + replicationContext => EventSourcedBehavior[Command, String, State]( - aaContext.persistenceId, + replicationContext.persistenceId, State(Nil), (_, command) => command match { case AccessInCommandHandler(replyTo) => val exception = try { - aaContext.origin + replicationContext.origin None } catch { case t: Throwable => @@ -48,7 +48,7 @@ object ReplicationIllegalAccessSpec { case AccessInPersistCallback(replyTo) => Effect.persist("cat").thenRun { _ => val exception = try { - aaContext.concurrent + replicationContext.concurrent None } catch { case t: Throwable => @@ -86,8 +86,8 @@ class ReplicationIllegalAccessSpec "detect illegal access in the factory" in { val exception = intercept[UnsupportedOperationException] { ReplicatedEventSourcing.withSharedJournal("id2", R1, AllReplicas, PersistenceTestKitReadJournal.Identifier) { - aaContext => - aaContext.origin + replicationContext => + replicationContext.origin ??? } } 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 index f71b12afbb..d20638c0d1 100644 --- 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 @@ -37,8 +37,8 @@ object ReplicationSnapshotSpec { entityId, replicaId, AllReplicas, - PersistenceTestKitReadJournal.Identifier)(aaContext => - eventSourcedBehavior(aaContext, probe).snapshotWhen((_, _, sequenceNr) => sequenceNr % 2 == 0)) + PersistenceTestKitReadJournal.Identifier)(replicationContext => + eventSourcedBehavior(replicationContext, probe).snapshotWhen((_, _, sequenceNr) => sequenceNr % 2 == 0)) } } 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 index 48b300b5cf..f8bf4e4240 100644 --- 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 @@ -30,10 +30,10 @@ object LwwSpec { entityId, replica, AllReplicas, - PersistenceTestKitReadJournal.Identifier) { aaContext => + PersistenceTestKitReadJournal.Identifier) { replicationContext => EventSourcedBehavior[Command, Event, Registry]( - aaContext.persistenceId, - Registry("", LwwTime(Long.MinValue, aaContext.replicaId)), + replicationContext.persistenceId, + Registry("", LwwTime(Long.MinValue, replicationContext.replicaId)), (state, command) => command match { case Update(s, timestmap, error) => @@ -41,7 +41,7 @@ object LwwSpec { error ! "bad value" Effect.none } else { - Effect.persist(Changed(s, state.updatedTimestamp.increase(timestmap, aaContext.replicaId))) + Effect.persist(Changed(s, state.updatedTimestamp.increase(timestmap, replicationContext.replicaId))) } case Get(replyTo) => replyTo ! state 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 index e17afe3090..2e02fd65bb 100644 --- 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 @@ -31,9 +31,9 @@ object ORSetSpec { entityId, replica, AllReplicas, - PersistenceTestKitReadJournal.Identifier) { aaContext => + PersistenceTestKitReadJournal.Identifier) { replicationContext => EventSourcedBehavior[Command, ORSet.DeltaOp, ORSet[String]]( - aaContext.persistenceId, + replicationContext.persistenceId, ORSet(replica), (state, command) => command match { 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 index f5aaa3f9b2..d0fd54f0f6 100644 --- 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 @@ -104,7 +104,7 @@ object ReplicatedAuctionExampleSpec { //#setup //#command-handler - def commandHandler(setup: AuctionSetup, ctx: ActorContext[AuctionCommand], aaContext: ReplicationContext)( + def commandHandler(setup: AuctionSetup, ctx: ActorContext[AuctionCommand], replicationContext: ReplicationContext)( state: AuctionState, command: AuctionCommand): Effect[AuctionEvent, AuctionState] = { state.phase match { @@ -118,12 +118,12 @@ object ReplicatedAuctionExampleSpec { Effect.none case Finish => ctx.log.info("Finish") - Effect.persist(AuctionFinished(aaContext.replicaId)) + Effect.persist(AuctionFinished(replicationContext.replicaId)) case Close => ctx.log.info("Close") require(shouldClose(setup, state)) // TODO send email (before or after persisting) - Effect.persist(WinnerDecided(aaContext.replicaId, state.highestBid, state.highestCounterOffer)) + Effect.persist(WinnerDecided(replicationContext.replicaId, state.highestBid, state.highestCounterOffer)) case _: OfferBid => // auction finished, no more bids accepted Effect.unhandled @@ -133,12 +133,16 @@ object ReplicatedAuctionExampleSpec { case OfferBid(bidder, offer) => Effect.persist( BidRegistered( - Bid(bidder, offer, Instant.ofEpochMilli(aaContext.currentTimeMillis()), aaContext.replicaId))) + Bid( + bidder, + offer, + Instant.ofEpochMilli(replicationContext.currentTimeMillis()), + replicationContext.replicaId))) case GetHighestBid(replyTo) => replyTo ! state.highestBid Effect.none case Finish => - Effect.persist(AuctionFinished(aaContext.replicaId)) + Effect.persist(AuctionFinished(replicationContext.replicaId)) case Close => ctx.log.warn("Premature close") // Close should only be triggered when we have already finished @@ -166,14 +170,14 @@ object ReplicatedAuctionExampleSpec { } //#event-handler - def eventHandler(ctx: ActorContext[AuctionCommand], aaCtx: ReplicationContext, setup: AuctionSetup)( + def eventHandler(ctx: ActorContext[AuctionCommand], replicationCtx: ReplicationContext, setup: AuctionSetup)( state: AuctionState, event: AuctionEvent): AuctionState = { val newState = state.applyEvent(event) ctx.log.infoN("Applying event {}. New start {}", event, newState) - if (!aaCtx.recoveryRunning) { - eventTriggers(setup, ctx, aaCtx, event, newState) + if (!replicationCtx.recoveryRunning) { + eventTriggers(setup, ctx, replicationCtx, event, newState) } newState @@ -184,7 +188,7 @@ object ReplicatedAuctionExampleSpec { private def eventTriggers( setup: AuctionSetup, ctx: ActorContext[AuctionCommand], - aaCtx: ReplicationContext, + replicationCtx: ReplicationContext, event: AuctionEvent, newState: AuctionState) = { event match { @@ -195,7 +199,7 @@ object ReplicatedAuctionExampleSpec { "AuctionFinished at {}, already finished at [{}]", finished.atReplica, alreadyFinishedAtDc.mkString(", ")) - if (alreadyFinishedAtDc(aaCtx.replicaId)) { + if (alreadyFinishedAtDc(replicationCtx.replicaId)) { if (shouldClose(setup, newState)) ctx.self ! Close } else { ctx.log.info("Sending finish to self") @@ -215,12 +219,13 @@ object ReplicatedAuctionExampleSpec { def behavior(replica: ReplicaId, setup: AuctionSetup): Behavior[AuctionCommand] = Behaviors.setup[AuctionCommand] { ctx => ReplicatedEventSourcing - .withSharedJournal(setup.name, replica, setup.allReplicas, PersistenceTestKitReadJournal.Identifier) { aaCtx => - EventSourcedBehavior( - aaCtx.persistenceId, - initialState(setup), - commandHandler(setup, ctx, aaCtx), - eventHandler(ctx, aaCtx, setup)) + .withSharedJournal(setup.name, replica, setup.allReplicas, PersistenceTestKitReadJournal.Identifier) { + replicationCtx => + EventSourcedBehavior( + replicationCtx.persistenceId, + initialState(setup), + commandHandler(setup, ctx, replicationCtx), + eventHandler(ctx, replicationCtx, setup)) } } } 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 index 2450592182..e4d7f3488f 100644 --- 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 @@ -55,27 +55,27 @@ class ReplicatedBlogExampleSpec implicit val config: PatienceConfig = PatienceConfig(timeout = Span(timeout.duration.toMillis, Millis)) - def behavior(aa: ReplicationContext, ctx: ActorContext[BlogCommand]) = + def behavior(replicationContext: ReplicationContext, ctx: ActorContext[BlogCommand]) = EventSourcedBehavior[BlogCommand, BlogEvent, BlogState]( - aa.persistenceId, + replicationContext.persistenceId, emptyState, (state, cmd) => cmd match { case AddPost(_, content, replyTo) => val evt = PostAdded( - aa.persistenceId.id, + replicationContext.persistenceId.id, content, - state.contentTimestamp.increase(aa.currentTimeMillis(), aa.replicaId)) + state.contentTimestamp.increase(replicationContext.currentTimeMillis(), replicationContext.replicaId)) Effect.persist(evt).thenRun { _ => - replyTo ! AddPostDone(aa.entityId) + replyTo ! AddPostDone(replicationContext.entityId) } case ChangeBody(_, newContent, replyTo) => val evt = BodyChanged( - aa.persistenceId.id, + replicationContext.persistenceId.id, newContent, - state.contentTimestamp.increase(aa.currentTimeMillis(), aa.replicaId)) + state.contentTimestamp.increase(replicationContext.currentTimeMillis(), replicationContext.replicaId)) Effect.persist(evt).thenRun { _ => replyTo ! Done } @@ -89,7 +89,7 @@ class ReplicatedBlogExampleSpec Effect.none }, (state, event) => { - ctx.log.info(s"${aa.entityId}:${aa.replicaId} Received event $event") + ctx.log.info(s"${replicationContext.entityId}:${replicationContext.replicaId} Received event $event") event match { case PostAdded(_, content, timestamp) => if (timestamp.isAfter(state.contentTimestamp)) { @@ -118,8 +118,8 @@ class ReplicatedBlogExampleSpec "cat", ReplicaId("DC-A"), Set(ReplicaId("DC-A"), ReplicaId("DC-B")), - PersistenceTestKitReadJournal.Identifier) { (aa: ReplicationContext) => - behavior(aa, ctx) + PersistenceTestKitReadJournal.Identifier) { replicationContext => + behavior(replicationContext, ctx) } }, "dc-a") @@ -131,8 +131,8 @@ class ReplicatedBlogExampleSpec "cat", ReplicaId("DC-B"), Set(ReplicaId("DC-A"), ReplicaId("DC-B")), - PersistenceTestKitReadJournal.Identifier) { (aa: ReplicationContext) => - behavior(aa, ctx) + PersistenceTestKitReadJournal.Identifier) { replicationContext => + behavior(replicationContext, ctx) } }, "dc-b") 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 1bac5c2683..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 @@ -123,7 +123,7 @@ private[akka] final class ReplayingEvents[C, E, S]( eventForErrorReporting = OptionVal.Some(event) state = state.copy(seqNr = repr.sequenceNr) - val aaMetaAndSelfReplica: Option[(ReplicatedEventMetadata, ReplicaId, ReplicationSetup)] = + val replicatedMetaAndSelfReplica: Option[(ReplicatedEventMetadata, ReplicaId, ReplicationSetup)] = setup.replication match { case Some(replication) => val meta = repr.metadata match { @@ -146,17 +146,17 @@ private[akka] final class ReplayingEvents[C, E, S]( case None => } - aaMetaAndSelfReplica match { - case Some((meta, selfReplica, aa)) if meta.originReplica != selfReplica => + 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)) - aa.clearContext() - case Some((_, _, aa)) => - aa.clearContext() + replication.clearContext() + case Some((_, _, replication)) => + replication.clearContext() state = state.copy(state = newState, eventSeenInInterval = true) case _ => state = state.copy(state = newState, eventSeenInInterval = true) 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 index 755201eb63..62b1e49851 100644 --- 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 @@ -87,7 +87,7 @@ private[akka] final class ReplicationContextImpl( private[akka] final case class ReplicationSetup( replicaId: ReplicaId, allReplicasAndQueryPlugins: Map[ReplicaId, String], - aaContext: ReplicationContextImpl) { + replicationContext: ReplicationContextImpl) { val allReplicas: Set[ReplicaId] = allReplicasAndQueryPlugins.keySet @@ -95,17 +95,17 @@ private[akka] final case class ReplicationSetup( * Must only be called on the same thread that will execute the user code */ def setContext(recoveryRunning: Boolean, originReplica: ReplicaId, concurrent: Boolean): Unit = { - aaContext._currentThread = OptionVal.Some(Thread.currentThread()) - aaContext._recoveryRunning = recoveryRunning - aaContext._concurrent = concurrent - aaContext._origin = OptionVal.Some(originReplica) + replicationContext._currentThread = OptionVal.Some(Thread.currentThread()) + replicationContext._recoveryRunning = recoveryRunning + replicationContext._concurrent = concurrent + replicationContext._origin = OptionVal.Some(originReplica) } def clearContext(): Unit = { - aaContext._currentThread = OptionVal.None - aaContext._recoveryRunning = false - aaContext._concurrent = false - aaContext._origin = OptionVal.None + 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 e0fb66ffe8..05889a0101 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 @@ -111,8 +111,8 @@ 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)) val initialState = setup.replication match { - case Some(aa) => startReplicationStream(setup, state, aa) - case None => state + case Some(replication) => startReplicationStream(setup, state, replication) + case None => state } new running.HandlingCommands(initialState) } @@ -128,7 +128,7 @@ private[akka] object Running { val query = PersistenceQuery(system) replicationSetup.allReplicas.foldLeft(state) { (state, replicaId) => if (replicaId != replicationSetup.replicaId) { - val pid = PersistenceId.replicatedUniqueId(replicationSetup.aaContext.entityId, replicaId) + val pid = PersistenceId.replicatedUniqueId(replicationSetup.replicationContext.entityId, replicaId) val queryPluginId = replicationSetup.allReplicasAndQueryPlugins(replicaId) val replication = query.readJournalFor[EventsByPersistenceIdQuery](queryPluginId) @@ -458,8 +458,8 @@ private[akka] object Running { val eventAdapterManifest = setup.eventAdapter.manifest(event) val newState2 = setup.replication match { - case Some(aa) => - val updatedVersion = stateAfterApply.version.updated(aa.replicaId.id, _currentSequenceNumber) + case Some(replication) => + val updatedVersion = stateAfterApply.version.updated(replication.replicaId.id, _currentSequenceNumber) val r = internalPersist( setup.context, cmd, @@ -467,8 +467,11 @@ private[akka] object Running { eventToPersist, eventAdapterManifest, OptionVal.Some( - ReplicatedEventMetadata(aa.replicaId, _currentSequenceNumber, updatedVersion, concurrent = false))) - .copy(version = updatedVersion) + ReplicatedEventMetadata( + replication.replicaId, + _currentSequenceNumber, + updatedVersion, + concurrent = false))).copy(version = updatedVersion) if (setup.log.isTraceEnabled()) setup.log.traceN( @@ -501,9 +504,9 @@ private[akka] object Running { _currentSequenceNumber = state.seqNr val metadataTemplate: Option[ReplicatedEventMetadata] = setup.replication match { - case Some(aa) => - aa.setContext(recoveryRunning = false, aa.replicaId, concurrent = false) // local events are never concurrent - Some(ReplicatedEventMetadata(aa.replicaId, 0L, state.version, concurrent = false)) // we replace it with actual seqnr later + 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 } @@ -685,7 +688,8 @@ private[akka] object Running { onWriteSuccess(setup.context, p) if (setup.publishEvents) { - val meta = setup.replication.map(aa => new ReplicatedPublishedEventMetaData(aa.replicaId, state.version)) + 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)) } From 7348939ff4c48da8faccf222303893d9c4b61aca Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 5 Aug 2020 10:33:48 +0200 Subject: [PATCH 33/50] DRY ReplicatedEventSourcedBehavior.apply (#29454) * and change signature of withEventPublishing and withDirectReplication --- .../typed/ReplicatedShardingSettings.scala | 8 ++-- .../typed/ShardingDirectReplication.scala | 4 +- .../paradox/typed/replicated-eventsourcing.md | 2 +- .../typed/ReplicatedAuctionExampleTest.java | 30 +++++++----- .../typed/EventPublishingSpec.scala | 2 +- .../internal/EventSourcedBehaviorImpl.scala | 4 +- .../typed/javadsl/EventSourcedBehavior.scala | 10 +++- .../ReplicatedEventSourcedBehavior.scala | 47 ++----------------- .../typed/scaladsl/EventSourcedBehavior.scala | 2 +- 9 files changed, 42 insertions(+), 67 deletions(-) diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala index 799e243960..332241f111 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala @@ -66,13 +66,13 @@ final class ReplicatedShardingSettings[M, E] private ( /** * 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 also have it enabled through [[akka.persistence.typed.scaladsl.EventSourcedBehavior.withEventPublishing]] + * or [[akka.persistence.typed.javadsl.ReplicatedEventSourcedBehavior.withEventPublishing]] * to work. * */ - def withDirectReplication(): ReplicatedShardingSettings[M, E] = - new ReplicatedShardingSettings(replicas, directReplication = true) + def withDirectReplication(enabled: Boolean): ReplicatedShardingSettings[M, E] = + new ReplicatedShardingSettings(replicas, directReplication = enabled) } 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 index f667348a21..97edb4fa0f 100644 --- 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 @@ -22,9 +22,9 @@ import akka.util.ccompat.JavaConverters._ * 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()]] + * 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 [[ReplicatedShardingSettings#withDirectReplication()]] + * If using [[ReplicatedSharding]] the replication can be enabled through [[ReplicatedShardingSettings.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 diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index 66e8932173..f5c1df80b8 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -317,7 +317,7 @@ query is still needed as delivery is not guaranteed, but can be configured to po events will arrive at the replicas through the cluster. To enable this feature you first need to enable event publishing on the @scala[`EventSourcedBehavior`]@java[`ReplicatedEventSourcedBehavior`] with `withEventPublishing` -and then enable direct replication through `withDirectReplication()` on @apidoc[ReplicatedShardingSettings] (if not using +and then enable direct replication through `withDirectReplication(true)` on @apidoc[ReplicatedShardingSettings] (if not using replicated sharding the replication can be run standalone by starting the @apidoc[ShardingDirectReplication] actor). The "event publishing" feature publishes each event to the local system event bus as a side effect after it has been written, 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 index 70d38cf566..f7274a7f08 100644 --- 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 @@ -115,7 +115,7 @@ class ReplicatedAuctionExample this.setup = setup; } - //#setup + // #setup static class AuctionSetup { final String name; final Bid initialBid; // the initial bid is the minimum price bidden at start time by the owner @@ -130,7 +130,7 @@ class ReplicatedAuctionExample this.responsibleForClosing = responsibleForClosing; } } - //#setup + // #setup public static final class Bid implements CborSerializable { public final String bidder; @@ -146,11 +146,13 @@ class ReplicatedAuctionExample } } - //#commands + // #commands 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; @@ -160,6 +162,7 @@ class ReplicatedAuctionExample this.offer = offer; } } + public static final class GetHighestBid implements Command { public final ActorRef replyTo; @@ -167,18 +170,21 @@ class ReplicatedAuctionExample 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 + // #commands - //#events + // #events interface Event extends CborSerializable {} public static final class BidRegistered implements Event { @@ -210,9 +216,9 @@ class ReplicatedAuctionExample this.amount = amount; } } - //#events + // #events - //#state + // #state static class AuctionState implements CborSerializable { final boolean stillRunning; @@ -270,14 +276,14 @@ class ReplicatedAuctionExample return !stillRunning && finishedAtDc.isEmpty(); } } - //#state + // #state @Override public AuctionState emptyState() { return new AuctionState(true, setup.initialBid, setup.initialBid.offer, Collections.emptySet()); } - //#command-handler + // #command-handler @Override public CommandHandler commandHandler() { @@ -348,7 +354,7 @@ class ReplicatedAuctionExample return builder.build(); } - //#command-handler + // #command-handler @Override public EventHandler eventHandler() { @@ -377,7 +383,7 @@ class ReplicatedAuctionExample .build(); } - //#event-triggers + // #event-triggers private void eventTriggers(AuctionFinished event, AuctionState newState) { if (newState.finishedAtDc.contains(getReplicationContext().replicaId().id())) { if (shouldClose(newState)) { @@ -387,7 +393,7 @@ class ReplicatedAuctionExample context.getSelf().tell(Finish.INSTANCE); } } - //#event-triggers + // #event-triggers private boolean shouldClose(AuctionState state) { return setup.responsibleForClosing 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 index 3f33a41381..62cab82ff4 100644 --- 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 @@ -35,7 +35,7 @@ object EventPublishingSpec { }, (state, event) => state + event) .withTagger(evt => if (evt.tagIt) Set("tag") else Set.empty) - .withEventPublishing() + .withEventPublishing(enabled = true) } } 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 56daa2493d..09503e67f2 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 @@ -251,8 +251,8 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( copy(recovery = recovery.toClassic) } - override def withEventPublishing(): EventSourcedBehavior[Command, Event, State] = { - copy(publishEvents = true) + override def withEventPublishing(enabled: Boolean): EventSourcedBehavior[Command, Event, State] = { + copy(publishEvents = enabled) } override private[akka] def withReplication( 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 fc0a1cf57c..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 @@ -178,8 +178,14 @@ abstract class EventSourcedBehavior[Command, Event, State] private[akka] ( /** * INTERNAL API: DeferredBehavior init, not for user extension */ - @InternalApi override def apply(context: typed.TypedActorContext[Command]): Behavior[Command] = { - // Note: duplicated in ReplicatedEventSourcedBehavior to not break source compatibility + @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 index 7346615e8f..f660e813d6 100644 --- 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 @@ -11,9 +11,7 @@ import akka.actor.typed.Behavior import akka.actor.typed.TypedActorContext import akka.annotation.ApiMayChange import akka.annotation.InternalApi -import akka.persistence.typed.internal import akka.persistence.typed.internal.ReplicationContextImpl -import akka.persistence.typed.internal.EffectImpl /** * Base class for replicated event sourced behaviors. @@ -27,7 +25,8 @@ abstract class ReplicatedEventSourcedBehavior[Command, Event, State]( def this(replicationContext: ReplicationContext) = this(replicationContext, Optional.empty()) /** - * Override and return true to publish events to the system event stream as [[akka.persistence.typed.PublishedEvent]] after they have been persisted + * Override and return true to publish events to the system event stream as + * [[akka.persistence.typed.PublishedEvent]] after they have been persisted. */ def withEventPublishing: Boolean = false @@ -37,45 +36,9 @@ abstract class ReplicatedEventSourcedBehavior[Command, Event, State]( * INTERNAL API: DeferredBehavior init, not for user extension */ @InternalApi override def apply(context: TypedActorContext[Command]): Behavior[Command] = { - // Note: duplicated in EventSourcedBehavior to not break source compatibility - val snapshotWhen: (State, Event, Long) => Boolean = (state, event, seqNr) => shouldSnapshot(state, event, seqNr) - - val tagger: Event => Set[String] = { event => - import akka.util.ccompat.JavaConverters._ - val tags = tagsFor(event) - if (tags.isEmpty) Set.empty - else tags.asScala.toSet - } - - val behavior = new internal.EventSourcedBehaviorImpl[Command, Event, State]( - persistenceId, - emptyState, - (state, cmd) => commandHandler()(state, cmd).asInstanceOf[EffectImpl[Event, State]], - eventHandler()(_, _), - getClass) - .snapshotWhen(snapshotWhen) - .withRetention(retentionCriteria.asScala) - .withTagger(tagger) - .eventAdapter(eventAdapter()) - .snapshotAdapter(snapshotAdapter()) - .withJournalPluginId(journalPluginId) - .withSnapshotPluginId(snapshotPluginId) - .withRecovery(recovery.asScala) - // context not user extendable so there should never be any other impls + createEventSourcedBehavior() + // context not user extendable so there should never be any other impls .withReplication(replicationContext.asInstanceOf[ReplicationContextImpl]) - - val handler = signalHandler() - val behaviorWithSignalHandler = - if (handler.isEmpty) behavior - else behavior.receiveSignal(handler.handler) - - val behaviorWithOnPersistFailure = - if (onPersistFailure.isPresent) - behaviorWithSignalHandler.onPersistFailure(onPersistFailure.get) - else - behaviorWithSignalHandler - - if (withEventPublishing) behaviorWithOnPersistFailure.withEventPublishing() - else behaviorWithOnPersistFailure + .withEventPublishing(withEventPublishing) } } 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 0c4c114ed8..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 @@ -222,7 +222,7 @@ object EventSourcedBehavior { * Publish events to the system event stream as [[akka.persistence.typed.PublishedEvent]] after they have been persisted */ @ApiMayChange - def withEventPublishing(): EventSourcedBehavior[Command, Event, State] + def withEventPublishing(enabled: Boolean): EventSourcedBehavior[Command, Event, State] /** * INTERNAL API From 67eb74f07659b17cfea572b57c71c39750c4190d Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Wed, 5 Aug 2020 16:06:33 +0100 Subject: [PATCH 34/50] Embed akka serialization inside jackson serialization (#29455) * Embed akka serialization inside jackson serialization Allows re-use of existing serializers when embedding the types in Jackson * Fix false dead code warning * Move marker trait and add docs * Remove module and document adding annotations * Review feedback --- .../src/main/paradox/serialization-jackson.md | 14 +++++ .../paradox/typed/replicated-eventsourcing.md | 8 ++- .../ReplicatedEventSourcingJacksonSpec.scala | 48 ++++++++++++++++ .../src/main/resources/reference.conf | 1 - .../jackson/AkkaSerializationModule.scala | 42 ++++++++++++++ .../src/test/resources/reference.conf | 1 + .../jackson/JacksonSerializerSpec.scala | 57 +++++++++++++++++-- .../jackson/JsonSerializable.scala | 10 ++++ 8 files changed, 175 insertions(+), 6 deletions(-) create mode 100644 akka-persistence-typed-tests/src/test/scala/akka/persistence/typed/jackson/ReplicatedEventSourcingJacksonSpec.scala create mode 100644 akka-serialization-jackson/src/main/scala/akka/serialization/jackson/AkkaSerializationModule.scala create mode 100644 akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JsonSerializable.scala diff --git a/akka-docs/src/main/paradox/serialization-jackson.md b/akka-docs/src/main/paradox/serialization-jackson.md index 83298aa788..c983cd4145 100644 --- a/akka-docs/src/main/paradox/serialization-jackson.md +++ b/akka-docs/src/main/paradox/serialization-jackson.md @@ -390,6 +390,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/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index f5c1df80b8..eb4905d35d 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -134,7 +134,13 @@ The factory returns a `Behavior` that can be spawned like any other behavior. ### Conflict free replicated data types -TODO example once CRDTs are in +The following CRDTs are included that can be used to build your own data model: + +* @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). ### Last writer wins 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-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 c82bf11a45..715f31bbcc 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 @@ -18,7 +18,6 @@ import java.util.logging.FileHandler import scala.collection.immutable import scala.concurrent.duration._ import scala.concurrent.duration.FiniteDuration - import com.fasterxml.jackson.annotation.JsonSubTypes import com.fasterxml.jackson.annotation.JsonTypeInfo import com.fasterxml.jackson.core.JsonFactory @@ -44,7 +43,6 @@ import com.typesafe.config.ConfigFactory import org.scalatest.BeforeAndAfterAll import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike - import akka.actor.ActorRef import akka.actor.ActorSystem import akka.actor.Address @@ -55,8 +53,11 @@ import akka.actor.setup.ActorSystemSetup import akka.actor.typed.scaladsl.Behaviors import akka.serialization.Serialization import akka.serialization.SerializationExtension -import akka.testkit.TestActors -import akka.testkit.TestKit +import akka.serialization.SerializerWithStringManifest +import akka.testkit.{ TestActors, TestKit } +import com.fasterxml.jackson.annotation.JsonIgnore +import com.fasterxml.jackson.databind.annotation.JsonDeserialize +import com.fasterxml.jackson.databind.annotation.JsonSerialize object ScalaTestMessages { trait TestMessage @@ -115,6 +116,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 ScalaTestEventMigration extends JacksonMigration { @@ -639,6 +676,14 @@ abstract class JacksonSerializerSpec(serializerName: String) } } 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" + } + } """))) with AnyWordSpecLike with Matchers @@ -962,6 +1007,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 From f41f0933722d2abbb32da113fc2a2e364f3f2e1f Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Wed, 5 Aug 2020 16:07:32 +0100 Subject: [PATCH 35/50] Rename replicated settings (#29460) --- ...s.scala => ReplicatedEntityProvider.scala} | 28 +++++++++---------- .../typed/ReplicatedShardingExtension.scala | 2 +- .../typed/ShardingDirectReplication.scala | 2 +- .../ReplicatedShardingExtensionImpl.scala | 4 +-- .../typed/ReplicatedShardingTest.java | 12 ++++---- .../typed/ReplicatedShardingSpec.scala | 10 +++---- .../paradox/typed/replicated-eventsourcing.md | 2 +- 7 files changed, 30 insertions(+), 30 deletions(-) rename akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/{ReplicatedShardingSettings.scala => ReplicatedEntityProvider.scala} (82%) diff --git a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedEntityProvider.scala similarity index 82% rename from akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala rename to akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedEntityProvider.scala index 332241f111..6b2e5a0586 100644 --- a/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedShardingSettings.scala +++ b/akka-cluster-sharding-typed/src/main/scala/akka/cluster/sharding/typed/ReplicatedEntityProvider.scala @@ -18,7 +18,7 @@ import akka.annotation.ApiMayChange import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl @ApiMayChange -object ReplicatedShardingSettings { +object ReplicatedEntityProvider { /** * Java API: @@ -33,7 +33,7 @@ object ReplicatedShardingSettings { JEntityTypeKey[M], ReplicaId, JSet[ReplicaId], - ReplicaSettings[M, E]]): ReplicatedShardingSettings[M, E] = { + ReplicatedEntity[M, E]]): ReplicatedEntityProvider[M, E] = { implicit val classTag: ClassTag[M] = ClassTag(messageClass) apply[M, E](allReplicaIds.asScala.toSet)((key, replica, _) => settingsPerReplicaFactory(key.asInstanceOf[EntityTypeKeyImpl[M]], replica, allReplicaIds)) @@ -46,9 +46,9 @@ object ReplicatedShardingSettings { * @tparam E The type for envelopes used for sending `M`s over sharding */ def apply[M: ClassTag, E](allReplicaIds: Set[ReplicaId])( - settingsPerReplicaFactory: (EntityTypeKey[M], ReplicaId, Set[ReplicaId]) => ReplicaSettings[M, E]) - : ReplicatedShardingSettings[M, E] = { - new ReplicatedShardingSettings(allReplicaIds.map { replicaId => + settingsPerReplicaFactory: (EntityTypeKey[M], ReplicaId, Set[ReplicaId]) => ReplicatedEntity[M, E]) + : ReplicatedEntityProvider[M, E] = { + new ReplicatedEntityProvider(allReplicaIds.map { replicaId => val typeKey = EntityTypeKey[M](replicaId.id) settingsPerReplicaFactory(typeKey, replicaId, allReplicaIds) }.toVector, directReplication = false) @@ -60,8 +60,8 @@ object ReplicatedShardingSettings { * @tparam E The type for envelopes used for sending `M`s over sharding */ @ApiMayChange -final class ReplicatedShardingSettings[M, E] private ( - val replicas: immutable.Seq[ReplicaSettings[M, E]], +final class ReplicatedEntityProvider[M, E] private ( + val replicas: immutable.Seq[ReplicatedEntity[M, E]], val directReplication: Boolean) { /** @@ -71,13 +71,13 @@ final class ReplicatedShardingSettings[M, E] private ( * to work. * */ - def withDirectReplication(enabled: Boolean): ReplicatedShardingSettings[M, E] = - new ReplicatedShardingSettings(replicas, directReplication = enabled) + def withDirectReplication(enabled: Boolean): ReplicatedEntityProvider[M, E] = + new ReplicatedEntityProvider(replicas, directReplication = enabled) } @ApiMayChange -object ReplicaSettings { +object ReplicatedEntity { /** * Java API: Defines the [[akka.cluster.sharding.typed.javadsl.Entity]] to use for a given replica, note that the behavior @@ -85,7 +85,7 @@ object ReplicaSettings { * [[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, E](replicaId: ReplicaId, entity: JEntity[M, E]): ReplicaSettings[M, E] = + def create[M, E](replicaId: ReplicaId, entity: JEntity[M, E]): ReplicatedEntity[M, E] = apply(replicaId, entity.toScala) /** @@ -94,12 +94,12 @@ object ReplicaSettings { * [[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, E](replicaId: ReplicaId, entity: Entity[M, E]): ReplicaSettings[M, E] = - new ReplicaSettings(replicaId, entity) + def apply[M, E](replicaId: ReplicaId, entity: Entity[M, E]): ReplicatedEntity[M, E] = + new ReplicatedEntity(replicaId, entity) } /** * Settings for a specific replica id in replicated sharding */ @ApiMayChange -final class ReplicaSettings[M, E] private (val replicaId: ReplicaId, val entity: Entity[M, E]) +final class ReplicatedEntity[M, E] private (val replicaId: ReplicaId, val entity: Entity[M, E]) 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 index d650106857..fb206cb73f 100644 --- 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 @@ -45,7 +45,7 @@ trait ReplicatedShardingExtension extends Extension { * * Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ReplicatedSharding]] */ - def init[M, E](settings: ReplicatedShardingSettings[M, E]): ReplicatedSharding[M, E] + def init[M, E](settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] } /** 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 index 97edb4fa0f..7904aa6cc9 100644 --- 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 @@ -24,7 +24,7 @@ import akka.util.ccompat.JavaConverters._ * 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 [[ReplicatedShardingSettings.withDirectReplication]] + * 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 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 index 1447c9e559..aff3515bb9 100644 --- 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 @@ -12,7 +12,7 @@ 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.ReplicatedShardingSettings +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 @@ -34,7 +34,7 @@ private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_] private val logger = LoggerFactory.getLogger(getClass) - override def init[M, E](settings: ReplicatedShardingSettings[M, E]): ReplicatedSharding[M, E] = { + override def init[M, E](settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] = { val sharding = ClusterSharding(system) val initializedReplicas = settings.replicas.map { replicaSettings => // start up a sharding instance per replica id 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 index eeab3cfb16..2ea8e53ff2 100644 --- 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 @@ -151,15 +151,15 @@ public class ReplicatedShardingTest extends JUnitSuite { super(context); // #bootstrap - ReplicatedShardingSettings< + ReplicatedEntityProvider< MyReplicatedStringSet.Command, ShardingEnvelope> - replicatedShardingSettings = - ReplicatedShardingSettings.create( + replicatedEntityProvider = + ReplicatedEntityProvider.create( MyReplicatedStringSet.Command.class, ALL_REPLICAS, - // factory for replica settings for a given replica + // factory for replicated entity for a given replica (entityTypeKey, replicaId, allReplicas) -> - ReplicaSettings.create( + ReplicatedEntity.create( replicaId, // use the replica id as typekey for sharding to get one sharding instance // per replica @@ -179,7 +179,7 @@ public class ReplicatedShardingTest extends JUnitSuite { ReplicatedShardingExtension.get(getContext().getSystem()); ReplicatedSharding< MyReplicatedStringSet.Command, ShardingEnvelope> - replicatedSharding = extension.init(replicatedShardingSettings); + replicatedSharding = extension.init(replicatedEntityProvider); // #bootstrap this.replicatedSharding = replicatedSharding; 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 index 90d4d5fa22..8824d5ce55 100644 --- 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 @@ -75,12 +75,12 @@ class ReplicatedShardingSpec def apply(): Behavior[Command] = Behaviors.setup { context => // #bootstrap - val replicatedShardingSettings = - ReplicatedShardingSettings[MyReplicatedStringSet.Command, ShardingEnvelope[MyReplicatedStringSet.Command]]( + val replicatedShardingProvider = + ReplicatedEntityProvider[MyReplicatedStringSet.Command, ShardingEnvelope[MyReplicatedStringSet.Command]]( // all replicas Set(ReplicaId("DC-A"), ReplicaId("DC-B"), ReplicaId("DC-C"))) { (entityTypeKey, replicaId, allReplicaIds) => - // factory for replica settings for a given replica - ReplicaSettings( + // factory for replicated entity for a given replica + ReplicatedEntity( replicaId, // use the provided entity type key for sharding to get one sharding instance per replica Entity(entityTypeKey) { entityContext => @@ -93,7 +93,7 @@ class ReplicatedShardingSpec .withRole(replicaId.id)) } - val replicatedSharding = ReplicatedShardingExtension(context.system).init(replicatedShardingSettings) + val replicatedSharding = ReplicatedShardingExtension(context.system).init(replicatedShardingProvider) // #bootstrap Behaviors.receiveMessage { diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index eb4905d35d..699e9322ce 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -323,7 +323,7 @@ query is still needed as delivery is not guaranteed, but can be configured to po events will arrive at the replicas through the cluster. To enable this feature you first need to enable event publishing on the @scala[`EventSourcedBehavior`]@java[`ReplicatedEventSourcedBehavior`] with `withEventPublishing` -and then enable direct replication through `withDirectReplication(true)` on @apidoc[ReplicatedShardingSettings] (if not using +and then enable direct replication through `withDirectReplication(true)` on @apidoc[ReplicatedEntityProvider] (if not using replicated sharding the replication can be run standalone by starting the @apidoc[ShardingDirectReplication] actor). The "event publishing" feature publishes each event to the local system event bus as a side effect after it has been written, From fb5f5dc1456697ec571953a14c342673379f00a0 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 6 Aug 2020 09:23:11 +0200 Subject: [PATCH 36/50] a few more things when reviewing replicated event sourcing (#29461) --- .../akka/util/AlwaysIncreasingClock.scala | 29 ------------------- .../src/main/scala/akka/util/WallClock.scala | 26 ++++++++++++++++- .../src/main/paradox/typed/guide/modules.md | 2 -- .../src/main/paradox/typed/persistence.md | 10 +++---- .../src/main/resources/reference.conf | 16 +++++----- .../akka/persistence/typed/crdt/ORSet.scala | 6 ++-- .../typed/internal/ReplicationSetup.scala | 3 -- .../persistence/typed/internal/Running.scala | 2 +- .../typed/internal/VersionVector.scala | 5 ++-- 9 files changed, 43 insertions(+), 56 deletions(-) delete mode 100644 akka-actor/src/main/scala/akka/util/AlwaysIncreasingClock.scala diff --git a/akka-actor/src/main/scala/akka/util/AlwaysIncreasingClock.scala b/akka-actor/src/main/scala/akka/util/AlwaysIncreasingClock.scala deleted file mode 100644 index 8bc84f8261..0000000000 --- a/akka-actor/src/main/scala/akka/util/AlwaysIncreasingClock.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Copyright (C) 2020 Lightbend Inc. - */ - -package akka.util - -import java.util.concurrent.atomic.AtomicLong -import java.util.function.LongUnaryOperator - -import akka.annotation.InternalApi - -/** - * 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-actor/src/main/scala/akka/util/WallClock.scala b/akka-actor/src/main/scala/akka/util/WallClock.scala index 1bda0e8485..3254f52e31 100644 --- a/akka-actor/src/main/scala/akka/util/WallClock.scala +++ b/akka-actor/src/main/scala/akka/util/WallClock.scala @@ -5,6 +5,10 @@ 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. @@ -17,7 +21,27 @@ trait WallClock { object WallClock { /** - * Always increasing time source. + * 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-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/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-persistence-typed/src/main/resources/reference.conf b/akka-persistence-typed/src/main/resources/reference.conf index fb72910aca..390064ee69 100644 --- a/akka-persistence-typed/src/main/resources/reference.conf +++ b/akka-persistence-typed/src/main/resources/reference.conf @@ -2,16 +2,16 @@ akka.actor { serialization-identifiers."akka.persistence.typed.serialization.ReplicatedEventSourcingSerializer" = 40 - serializers.active-active = "akka.persistence.typed.serialization.ReplicatedEventSourcingSerializer" + serializers.replicated-event-sourcing = "akka.persistence.typed.serialization.ReplicatedEventSourcingSerializer" serialization-bindings { - "akka.persistence.typed.internal.VersionVector" = active-active - "akka.persistence.typed.crdt.Counter" = active-active - "akka.persistence.typed.crdt.Counter$Updated" = active-active - "akka.persistence.typed.crdt.ORSet" = active-active - "akka.persistence.typed.crdt.ORSet$DeltaOp" = active-active - "akka.persistence.typed.internal.ReplicatedEventMetadata" = active-active - "akka.persistence.typed.internal.ReplicatedSnapshotMetadata" = active-active + "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 } } 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 index f881e1d433..b8e5f1ffc0 100644 --- 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 @@ -32,7 +32,7 @@ object ORSet { */ @InternalApi private[akka] type Dot = VersionVector - sealed trait DeltaOp extends Serializable { + sealed trait DeltaOp { def merge(that: DeltaOp): DeltaOp } @@ -275,13 +275,11 @@ object ORSet { * This class is immutable, i.e. "modifying" methods return a new instance. */ @ApiMayChange -@SerialVersionUID(1L) 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] - with Serializable { + extends OpCrdt[DeltaOp] { type T = ORSet[A] type D = ORSet.DeltaOp 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 index 62b1e49851..e449469a92 100644 --- 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 @@ -14,9 +14,6 @@ import akka.util.ccompat.JavaConverters._ /** * INTERNAL API */ -// FIXME, parts of this can be set during initialisation -// Other fields will be set before executing the event handler as they change per event -// https://github.com/akka/akka/issues/29258 @InternalApi private[akka] final class ReplicationContextImpl( val entityId: String, 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 05889a0101..96aaf7305d 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 @@ -165,7 +165,7 @@ private[akka] object Running { // 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-active-active is supported + // Need to handle this not being available migration from non-replicated is supported val meta = eventEnvelope.eventMetadata.get.asInstanceOf[ReplicatedEventMetadata] val re = ReplicatedEvent[E]( 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 index 42e5b7bbf5..3043a7ad30 100644 --- 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 @@ -13,7 +13,7 @@ import akka.annotation.InternalApi * VersionVector module with helper classes and methods. */ @InternalApi -object VersionVector { +private[akka] object VersionVector { private val emptyVersions: TreeMap[String, Long] = TreeMap.empty val empty: VersionVector = ManyVersionVector(emptyVersions) @@ -71,9 +71,8 @@ object VersionVector { * * This class is immutable, i.e. "modifying" methods return a new instance. */ -@SerialVersionUID(1L) @InternalApi -sealed abstract class VersionVector extends Serializable { +private[akka] sealed abstract class VersionVector { type T = VersionVector From 5e9e490d88e7f54c65e0a1e53b7a550f7d86e617 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Thu, 6 Aug 2020 14:14:48 +0100 Subject: [PATCH 37/50] Add entity type name for replicated entity (#29462) * Add entity type name for replicated entity * Review feedback * Unused import --- .../typed/ReplicatedShardingTest.java | 1 + ...licatedShardingDirectReplicationSpec.scala | 2 +- .../typed/ReplicatedShardingSpec.scala | 1 + .../typed/ReplicatedEventSourcingTest.java | 2 +- .../typed/MyReplicatedBehavior.java | 4 ++-- .../typed/ReplicatedAuctionExampleTest.java | 2 +- .../typed/ReplicatedStringSet.java | 2 +- .../typed/MultiJournalReplicationSpec.scala | 1 + .../typed/ReplicatedEventPublishingSpec.scala | 23 +++++++++++-------- .../typed/ReplicatedEventSourcingSpec.scala | 1 + .../ReplicatedEventSourcingTaggingSpec.scala | 2 +- .../typed/ReplicationIllegalAccessSpec.scala | 19 +++++++++++---- .../typed/ReplicationSnapshotSpec.scala | 7 ++++-- .../persistence/typed/crdt/CounterSpec.scala | 1 + .../akka/persistence/typed/crdt/LwwSpec.scala | 1 + .../persistence/typed/crdt/ORSetSpec.scala | 1 + .../typed/ReplicatedAuctionExampleSpec.scala | 21 +++++++++-------- .../typed/ReplicatedBlogExampleSpec.scala | 2 ++ ...plicatedEventSourcingCompileOnlySpec.scala | 7 +++--- .../persistence/typed/PersistenceId.scala | 15 ++++++++---- .../typed/internal/ReplicationSetup.scala | 3 ++- .../persistence/typed/internal/Running.scala | 5 +++- .../javadsl/ReplicatedEventSourcing.scala | 13 +++++++++-- .../scaladsl/ReplicatedEventSourcing.scala | 11 +++++++-- 24 files changed, 100 insertions(+), 47 deletions(-) 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 index 2ea8e53ff2..be2ec24770 100644 --- 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 @@ -67,6 +67,7 @@ public class ReplicatedShardingTest extends JUnitSuite { static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { return ReplicatedEventSourcing.withSharedJournal( + "StringSet", entityId, replicaId, allReplicas, 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 index 1cd99bb3a2..871317b1ce 100644 --- 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 @@ -37,7 +37,7 @@ class ReplicatedShardingDirectReplicationSpec extends ScalaTestWithActorTestKit upProbe.receiveMessage() // not bullet proof wrt to subscription being complete but good enough val event = PublishedEventImpl( - PersistenceId.replicatedUniqueId("pid", ReplicaId("ReplicaA")), + PersistenceId.replicatedId("ReplicatedShardingSpec", "pid", ReplicaId("ReplicaA")), 1L, "event", System.currentTimeMillis(), 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 index 8824d5ce55..b37a6d26d6 100644 --- 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 @@ -49,6 +49,7 @@ class ReplicatedShardingSpec def apply(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] = ReplicatedEventSourcing.withSharedJournal( + "StringSet", entityId, replicaId, allReplicas, 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 index a22d3bbb85..2b5c951635 100644 --- 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 @@ -80,7 +80,7 @@ public class ReplicatedEventSourcingTest extends JUnitSuite { public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ReplicatedEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.withSharedJournal("ReplicatedEventSourcingTest", entityId, replicaId, allReplicas, 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 index 3ada97caf7..784afca44c 100644 --- 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 @@ -33,7 +33,7 @@ public class MyReplicatedBehavior public static Behavior create( String entityId, ReplicaId replicaId, String queryPluginId) { return ReplicatedEventSourcing.withSharedJournal( - entityId, replicaId, ALL_REPLICAS, queryPluginId, MyReplicatedBehavior::new); + "MyReplicatedEntity", entityId, replicaId, ALL_REPLICAS, queryPluginId, MyReplicatedBehavior::new); } // #factory-shared @@ -44,7 +44,7 @@ public class MyReplicatedBehavior allReplicasAndQueryPlugins.put(DCB, "journalForDCB"); return ReplicatedEventSourcing.create( - entityId, replicaId, allReplicasAndQueryPlugins, MyReplicatedBehavior::new); + "MyReplicatedEntity", entityId, replicaId, allReplicasAndQueryPlugins, MyReplicatedBehavior::new); } private MyReplicatedBehavior(ReplicationContext replicationContext) { 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 index f7274a7f08..cc8c08f274 100644 --- 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 @@ -100,7 +100,7 @@ class ReplicatedAuctionExample public static Behavior create(AuctionSetup setup, ReplicaId replica) { return Behaviors.setup( ctx -> - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.withSharedJournal("Auction", setup.name, replica, ALL_REPLICAS, 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 index 0dc9e078a0..027021d14b 100644 --- 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 @@ -26,7 +26,7 @@ public final class ReplicatedStringSet public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ReplicatedEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.withSharedJournal("StringSet", entityId, replicaId, allReplicas, 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 index d155d9d6dc..2c4f0614af 100644 --- 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 @@ -33,6 +33,7 @@ object MultiJournalReplicationSpec { private val writeJournalPerReplica = Map("R1" -> "journal1.journal", "R2" -> "journal2.journal") def apply(entityId: String, replicaId: String): Behavior[Command] = { ReplicatedEventSourcing( + "MultiJournalSpec", entityId, ReplicaId(replicaId), Map(ReplicaId("R1") -> "journal1.query", ReplicaId("R2") -> "journal2.query"))( 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 index c8a88b947c..051be16df0 100644 --- 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 @@ -20,6 +20,8 @@ import org.scalatest.wordspec.AnyWordSpecLike object ReplicatedEventPublishingSpec { + val EntityType = "EventPublishingSpec" + object MyReplicatedBehavior { trait Command case class Add(text: String, replyTo: ActorRef[Done]) extends Command @@ -29,13 +31,14 @@ object ReplicatedEventPublishingSpec { def apply(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] = Behaviors.setup { ctx => ReplicatedEventSourcing.withSharedJournal( + EntityType, entityId, replicaId, allReplicas, PersistenceTestKitReadJournal.Identifier)( - replicationctx => + replicationContext => EventSourcedBehavior[Command, String, Set[String]]( - replicationctx.persistenceId, + replicationContext.persistenceId, Set.empty, (state, command) => command match { @@ -83,7 +86,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, DCB), + PersistenceId.replicatedId(EntityType, id, DCB), 1L, "two", System.currentTimeMillis(), @@ -104,7 +107,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, DCB), + PersistenceId.replicatedId(EntityType, id, DCB), 2L, // missing 1L "two", System.currentTimeMillis(), @@ -125,7 +128,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, DCC), + PersistenceId.replicatedId(EntityType, id, DCC), 1L, "two", System.currentTimeMillis(), @@ -146,14 +149,14 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId("myId4", DCB), + PersistenceId.replicatedId(EntityType, "myId4", DCB), 1L, "two", System.currentTimeMillis(), Some(new ReplicatedPublishedEventMetaData(DCB, VersionVector.empty))) // simulate another published event from that replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, DCB), + PersistenceId.replicatedId(EntityType, id, DCB), 1L, "two-again", // ofc this would be the same in the real world, different just so we can detect System.currentTimeMillis(), @@ -185,7 +188,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica incarnation2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, DCB), + PersistenceId.replicatedId(EntityType, id, DCB), 1L, "two", System.currentTimeMillis(), @@ -208,7 +211,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica incarnationA1.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, DCB), + PersistenceId.replicatedId(EntityType, id, DCB), 1L, "two", System.currentTimeMillis(), @@ -221,7 +224,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica incarnationA2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(id, DCB), + PersistenceId.replicatedId(EntityType, id, DCB), 2L, "three", System.currentTimeMillis(), 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 index beeb5c0863..e84d7c91e8 100644 --- 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 @@ -72,6 +72,7 @@ object ReplicatedEventSourcingSpec { replicaId: String, probe: Option[ActorRef[EventAndContext]] = None): Behavior[Command] = ReplicatedEventSourcing.withSharedJournal( + "ReplicatedEventSourcingSpec", entityId, ReplicaId(replicaId), AllReplicas, 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 index 033a8dee14..fdfc44ab82 100644 --- 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 @@ -42,7 +42,7 @@ object ReplicatedEventSourcingTaggingSpec { replica: ReplicaId, allReplicas: Set[ReplicaId]): EventSourcedBehavior[Command, String, State] = { // #tagging - ReplicatedEventSourcing.withSharedJournal(entityId, replica, allReplicas, queryPluginId)( + ReplicatedEventSourcing.withSharedJournal("TaggingSpec", entityId, replica, allReplicas, queryPluginId)( replicationContext => EventSourcedBehavior[Command, String, State]( replicationContext.persistenceId, 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 index 11b28e9f76..5857a26fbc 100644 --- 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 @@ -28,7 +28,12 @@ object ReplicationIllegalAccessSpec { case class State(all: List[String]) extends CborSerializable def apply(entityId: String, replica: ReplicaId): Behavior[Command] = { - ReplicatedEventSourcing.withSharedJournal(entityId, replica, AllReplicas, PersistenceTestKitReadJournal.Identifier)( + ReplicatedEventSourcing.withSharedJournal( + "IllegalAccessSpec", + entityId, + replica, + AllReplicas, + PersistenceTestKitReadJournal.Identifier)( replicationContext => EventSourcedBehavior[Command, String, State]( replicationContext.persistenceId, @@ -85,10 +90,14 @@ class ReplicationIllegalAccessSpec } "detect illegal access in the factory" in { val exception = intercept[UnsupportedOperationException] { - ReplicatedEventSourcing.withSharedJournal("id2", R1, AllReplicas, PersistenceTestKitReadJournal.Identifier) { - replicationContext => - replicationContext.origin - ??? + ReplicatedEventSourcing.withSharedJournal( + "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 index d20638c0d1..b22514af28 100644 --- 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 @@ -20,6 +20,8 @@ object ReplicationSnapshotSpec { import ReplicatedEventSourcingSpec._ + val EntityType = "SpapsnotSpec" + def behaviorWithSnapshotting(entityId: String, replicaId: ReplicaId): Behavior[Command] = behaviorWithSnapshotting(entityId, replicaId, None) @@ -34,6 +36,7 @@ object ReplicationSnapshotSpec { replicaId: ReplicaId, probe: Option[ActorRef[EventAndContext]]): Behavior[Command] = { ReplicatedEventSourcing.withSharedJournal( + EntityType, entityId, replicaId, AllReplicas, @@ -81,7 +84,7 @@ class ReplicationSnapshotSpec snapshotTestKit.expectNextPersisted(persistenceIdR2, State(List("r1 2", "r1 1"))) r2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(entityId, R1), + PersistenceId.replicatedId(EntityType, entityId, R1), 1L, "two-again", System.currentTimeMillis(), @@ -95,7 +98,7 @@ class ReplicationSnapshotSpec { val r2 = spawn(behaviorWithSnapshotting(entityId, R2, r2EventProbe.ref)) r2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedUniqueId(entityId, R1), + PersistenceId.replicatedId(EntityType, entityId, R1), 1L, "two-again", System.currentTimeMillis(), 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 index 3f95c87755..16f6eeb6cf 100644 --- 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 @@ -29,6 +29,7 @@ object CounterSpec { eventProbe: Option[ActorRef[Counter.Updated]] = None) = Behaviors.setup[PlainCounter.Command] { context => ReplicatedEventSourcing.withSharedJournal( + "CounterSpec", entityId, replicaId, AllReplicas, 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 index f8bf4e4240..07a2091ca8 100644 --- 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 @@ -27,6 +27,7 @@ object LwwSpec { def apply(entityId: String, replica: ReplicaId): Behavior[Command] = { ReplicatedEventSourcing.withSharedJournal( + "LwwRegistrySpec", entityId, replica, AllReplicas, 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 index 2e02fd65bb..915bb8b64c 100644 --- 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 @@ -28,6 +28,7 @@ object ORSetSpec { def apply(entityId: String, replica: ReplicaId): Behavior[ORSetEntity.Command] = { ReplicatedEventSourcing.withSharedJournal( + "ORSetSpec", entityId, replica, AllReplicas, 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 index d0fd54f0f6..f28493b6c3 100644 --- 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 @@ -218,15 +218,18 @@ object ReplicatedAuctionExampleSpec { def behavior(replica: ReplicaId, setup: AuctionSetup): Behavior[AuctionCommand] = Behaviors.setup[AuctionCommand] { ctx => - ReplicatedEventSourcing - .withSharedJournal(setup.name, replica, setup.allReplicas, PersistenceTestKitReadJournal.Identifier) { - replicationCtx => - EventSourcedBehavior( - replicationCtx.persistenceId, - initialState(setup), - commandHandler(setup, ctx, replicationCtx), - eventHandler(ctx, replicationCtx, setup)) - } + ReplicatedEventSourcing.withSharedJournal( + "auction", + setup.name, + replica, + setup.allReplicas, + PersistenceTestKitReadJournal.Identifier) { replicationCtx => + EventSourcedBehavior( + replicationCtx.persistenceId, + initialState(setup), + commandHandler(setup, ctx, replicationCtx), + eventHandler(ctx, replicationCtx, setup)) + } } } 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 index e4d7f3488f..e9a86ab8bf 100644 --- 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 @@ -115,6 +115,7 @@ class ReplicatedBlogExampleSpec spawn( Behaviors.setup[BlogCommand] { ctx => ReplicatedEventSourcing.withSharedJournal( + "blog", "cat", ReplicaId("DC-A"), Set(ReplicaId("DC-A"), ReplicaId("DC-B")), @@ -128,6 +129,7 @@ class ReplicatedBlogExampleSpec spawn( Behaviors.setup[BlogCommand] { ctx => ReplicatedEventSourcing.withSharedJournal( + "blog", "cat", ReplicaId("DC-B"), Set(ReplicaId("DC-A"), ReplicaId("DC-B")), 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 index 6d98e47bb9..e98b7418c6 100644 --- 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 @@ -24,14 +24,15 @@ object ReplicatedEventSourcingCompileOnlySpec { trait Event //#factory-shared - ReplicatedEventSourcing.withSharedJournal("entityId", DCA, AllReplicas, queryPluginId) { context => + ReplicatedEventSourcing.withSharedJournal("entityTypeHint", "entityId", DCA, AllReplicas, queryPluginId) { context => EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) } //#factory-shared //#factory - ReplicatedEventSourcing("entityId", DCA, Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { context => - EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) + ReplicatedEventSourcing("entityTypeHint", "entityId", DCA, Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { + context => + EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) } //#factory 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 6018759038..47b014a9c4 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 @@ -3,7 +3,7 @@ */ package akka.persistence.typed -import akka.annotation.InternalApi +import akka.annotation.ApiMayChange object PersistenceId { @@ -127,10 +127,15 @@ object PersistenceId { new PersistenceId(id) /** - * Constructs a persistence id from a unique entity id that includes the replica id. + * Constructs a [[PersistenceId]] from the given `entityTypeHint`, `entityId` and `replicaId` by + * concatenating them with the `|` separator. */ - @InternalApi - private[akka] def replicatedUniqueId(entityId: String, replicaId: ReplicaId): PersistenceId = { + @ApiMayChange + def replicatedId(entityTypeHint: String, entityId: String, replicaId: ReplicaId): PersistenceId = { + if (entityTypeHint.contains(DefaultSeparator)) + throw new IllegalArgumentException( + s"entityTypeHint [$entityTypeHint] contains [$DefaultSeparator] which is a reserved character") + if (entityId.contains(DefaultSeparator)) throw new IllegalArgumentException( s"entityId [$entityId] contains [$DefaultSeparator] which is a reserved character") @@ -139,7 +144,7 @@ object PersistenceId { throw new IllegalArgumentException( s"replicaId [${replicaId.id}] contains [$DefaultSeparator] which is a reserved character") - new PersistenceId(entityId + DefaultSeparator + replicaId.id) + new PersistenceId(entityTypeHint + DefaultSeparator + entityId + DefaultSeparator + replicaId.id) } } 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 index e449469a92..76639ab35f 100644 --- 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 @@ -16,6 +16,7 @@ import akka.util.ccompat.JavaConverters._ */ @InternalApi private[akka] final class ReplicationContextImpl( + val entityTypeHint: String, val entityId: String, val replicaId: ReplicaId, val replicasAndQueryPlugins: Map[ReplicaId, String]) @@ -64,7 +65,7 @@ private[akka] final class ReplicationContextImpl( _concurrent } - override def persistenceId: PersistenceId = PersistenceId.replicatedUniqueId(entityId, replicaId) + override def persistenceId: PersistenceId = PersistenceId.replicatedId(entityTypeHint, entityId, replicaId) override def currentTimeMillis(): Long = { WallClock.AlwaysIncreasingClock.currentTimeMillis() 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 96aaf7305d..3af686332b 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 @@ -128,7 +128,10 @@ private[akka] object Running { val query = PersistenceQuery(system) replicationSetup.allReplicas.foldLeft(state) { (state, replicaId) => if (replicaId != replicationSetup.replicaId) { - val pid = PersistenceId.replicatedUniqueId(replicationSetup.replicationContext.entityId, replicaId) + val pid = PersistenceId.replicatedId( + replicationSetup.replicationContext.entityTypeHint, + replicationSetup.replicationContext.entityId, + replicaId) val queryPluginId = replicationSetup.allReplicasAndQueryPlugins(replicaId) val replication = query.readJournalFor[EventsByPersistenceIdQuery](queryPluginId) 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 index f5ed8f29be..4bf71f8590 100644 --- 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 @@ -81,18 +81,25 @@ object ReplicatedEventSourcing { * can be used for each replica. * The events from other replicas are read using PersistentQuery. * + * @param entityType 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 replicaId The unique identity for this entity. The underlying persistence id will include the replica. * @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 withSharedJournal[Command, Event, State]( + entityType: String, entityId: String, replicaId: ReplicaId, allReplicaIds: JSet[ReplicaId], queryPluginId: String, behaviorFactory: JFunction[ReplicationContext, EventSourcedBehavior[Command, Event, State]]) : EventSourcedBehavior[Command, Event, State] = - create(entityId, replicaId, allReplicaIds.asScala.map(id => id -> queryPluginId).toMap.asJava, behaviorFactory) + create( + entityType, + entityId, + replicaId, + allReplicaIds.asScala.map(id => id -> queryPluginId).toMap.asJava, + behaviorFactory) /** * Initialize a replicated event sourced behavior. @@ -106,17 +113,19 @@ object ReplicatedEventSourcing { * 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 entityType 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 replicaId The unique identity for this entity. The underlying persistence id will include the replica. * @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 create[Command, Event, State]( + entityType: String, entityId: String, replicaId: ReplicaId, allReplicasAndQueryPlugins: JMap[ReplicaId, String], eventSourcedBehaviorFactory: JFunction[ReplicationContext, EventSourcedBehavior[Command, Event, State]]) : EventSourcedBehavior[Command, Event, State] = { - val context = new ReplicationContextImpl(entityId, replicaId, allReplicasAndQueryPlugins.asScala.toMap) + val context = new ReplicationContextImpl(entityType, entityId, replicaId, allReplicasAndQueryPlugins.asScala.toMap) eventSourcedBehaviorFactory(context) } 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 index 6cf395f51a..6d9e6a533f 100644 --- 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 @@ -76,18 +76,22 @@ object ReplicatedEventSourcing { * can be used for each replica. * The events from other replicas are read using PersistentQuery. * + * @param entityType 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. * @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 withSharedJournal[Command, Event, State]( + entityType: String, entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId], queryPluginId: String)( eventSourcedBehaviorFactory: ReplicationContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = - apply(entityId, replicaId, allReplicaIds.map(id => id -> queryPluginId).toMap)(eventSourcedBehaviorFactory) + apply(entityType, entityId, replicaId, allReplicaIds.map(id => id -> queryPluginId).toMap)( + eventSourcedBehaviorFactory) /** * Initialize a replicated event sourced behavior. @@ -101,17 +105,20 @@ object ReplicatedEventSourcing { * 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 entityType 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. * @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 apply[Command, Event, State]( + entityType: String, entityId: String, replicaId: ReplicaId, allReplicasAndQueryPlugins: Map[ReplicaId, String])( eventSourcedBehaviorFactory: ReplicationContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = { - val context = new ReplicationContextImpl(entityId, replicaId, allReplicasAndQueryPlugins) + val context = new ReplicationContextImpl(entityType, entityId, replicaId, allReplicasAndQueryPlugins) eventSourcedBehaviorFactory(context).withReplication(context) } From ac469e1a56848906c4e484934f98c4519d2621eb Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 11 Aug 2020 09:49:02 +0200 Subject: [PATCH 38/50] Polish Auction example * adjust the style in the Scala example * AuctionEntity class instead of Setup class that is passed around * add timer in recovery completed --- .../typed/replicated-eventsourcing-auction.md | 121 +++++ .../replicated-eventsourcing-examples.md | 120 +---- .../paradox/typed/replicated-eventsourcing.md | 6 +- .../typed/ReplicatedEventSourcingTest.java | 3 +- .../typed/MyReplicatedBehavior.java | 13 +- .../typed/ReplicatedAuctionExampleTest.java | 154 +++--- .../typed/ReplicatedStringSet.java | 3 +- .../typed/ReplicatedAuctionExampleSpec.scala | 483 ++++++++++-------- 8 files changed, 507 insertions(+), 396 deletions(-) create mode 100644 akka-docs/src/main/paradox/typed/replicated-eventsourcing-auction.md 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..bc2a7040c3 --- /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-examples.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md index e74119b2f2..0d53de9289 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md @@ -2,125 +2,11 @@ The following are more realistic examples of building systems with Replicated Event Sourcing. -## Auction +@@toc { depth=1 } -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. +@@@ index -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 initial state is taken from a `AuctionSetup` instance. 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 } +* [auction](replicated-eventsourcing-auction.md) @@@ -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.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index 699e9322ce..34d126a619 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -199,17 +199,17 @@ 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-examples.md#auction) uses these techniques. +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 deliver order +### 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 [CRDT's](#conflict-free-replicated-data-types) +when using @ref:[CRDT's](#conflict-free-replicated-data-types) and otherwise will be detected via the `ReplicationContext` concurrent method. For example: 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 index 2b5c951635..a5efaf67fa 100644 --- 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 @@ -80,7 +80,8 @@ public class ReplicatedEventSourcingTest extends JUnitSuite { public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ReplicatedEventSourcing.withSharedJournal("ReplicatedEventSourcingTest", + return ReplicatedEventSourcing.withSharedJournal( + "ReplicatedEventSourcingTest", entityId, replicaId, allReplicas, 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 index 784afca44c..911d91486f 100644 --- 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 @@ -33,7 +33,12 @@ public class MyReplicatedBehavior public static Behavior create( String entityId, ReplicaId replicaId, String queryPluginId) { return ReplicatedEventSourcing.withSharedJournal( - "MyReplicatedEntity", entityId, replicaId, ALL_REPLICAS, queryPluginId, MyReplicatedBehavior::new); + "MyReplicatedEntity", + entityId, + replicaId, + ALL_REPLICAS, + queryPluginId, + MyReplicatedBehavior::new); } // #factory-shared @@ -44,7 +49,11 @@ public class MyReplicatedBehavior allReplicasAndQueryPlugins.put(DCB, "journalForDCB"); return ReplicatedEventSourcing.create( - "MyReplicatedEntity", entityId, replicaId, allReplicasAndQueryPlugins, MyReplicatedBehavior::new); + "MyReplicatedEntity", + entityId, + replicaId, + allReplicasAndQueryPlugins, + MyReplicatedBehavior::new); } private MyReplicatedBehavior(ReplicationContext replicationContext) { 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 index cc8c08f274..955f1ff8b2 100644 --- 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 @@ -11,10 +11,18 @@ 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.javadsl.*; +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; @@ -22,6 +30,7 @@ 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; @@ -29,7 +38,7 @@ import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; -import static jdocs.akka.persistence.typed.ReplicatedAuctionExample.*; +import static jdocs.akka.persistence.typed.AuctionEntity.*; import static org.junit.Assert.assertEquals; public class ReplicatedAuctionExampleTest extends JUnitSuite { @@ -41,22 +50,14 @@ public class ReplicatedAuctionExampleTest extends JUnitSuite { @Test public void auctionExample() { - AuctionSetup setupA = - new AuctionSetup( - "old-skis", - new Bid("chbatey", 12, Instant.now(), R1), - Instant.now().plusSeconds(10), - true); + String auctionName = "old-skis"; + Bid initialBid = new Bid("chbatey", 12, Instant.now(), R1); + Instant closeAt = Instant.now().plusSeconds(10); - AuctionSetup setupB = - new AuctionSetup( - "old-skis", - new Bid("chbatey", 12, Instant.now(), R1), - Instant.now().plusSeconds(10), - false); - - ActorRef replicaA = testKit.spawn(create(setupA, R1)); - ActorRef replicaB = testKit.spawn(create(setupA, R2)); + 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)); @@ -87,52 +88,24 @@ public class ReplicatedAuctionExampleTest extends JUnitSuite { } } -class ReplicatedAuctionExample - extends ReplicatedEventSourcedBehavior { +// #setup +class AuctionEntity extends ReplicatedEventSourcedBehavior { 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 AuctionSetup setup; - - public static Behavior create(AuctionSetup setup, ReplicaId replica) { - return Behaviors.setup( - ctx -> - ReplicatedEventSourcing.withSharedJournal("Auction", - setup.name, - replica, - ALL_REPLICAS, - PersistenceTestKitReadJournal.Identifier(), - replicationCtx -> new ReplicatedAuctionExample(replicationCtx, ctx, setup))); - } - - public ReplicatedAuctionExample( - ReplicationContext replicationContext, ActorContext context, AuctionSetup setup) { - super(replicationContext); - this.context = context; - this.setup = setup; - } + private final TimerScheduler timers; + private final Bid initialBid; + private final Instant closingAt; + private final boolean responsibleForClosing; // #setup - static class AuctionSetup { - final String name; - final Bid initialBid; // the initial bid is the minimum price bidden at start time by the owner - final Instant closingAt; - final boolean responsibleForClosing; - public AuctionSetup( - String name, Bid initialBid, Instant closingAt, boolean responsibleForClosing) { - this.name = name; - this.initialBid = initialBid; - this.closingAt = closingAt; - this.responsibleForClosing = responsibleForClosing; - } - } - // #setup - - public static final class Bid implements CborSerializable { + // #commands + public static final class Bid { public final String bidder; public final int offer; public final Instant timestamp; @@ -146,7 +119,6 @@ class ReplicatedAuctionExample } } - // #commands interface Command extends CborSerializable {} public enum Finish implements Command { @@ -278,11 +250,72 @@ class ReplicatedAuctionExample } // #state + // #setup + public static Behavior create( + ReplicaId replica, + String name, + Bid initialBid, + Instant closingAt, + boolean responsibleForClosing) { + return Behaviors.setup( + ctx -> + Behaviors.withTimers( + timers -> + ReplicatedEventSourcing.withSharedJournal( + "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, setup.initialBid, setup.initialBid.offer, Collections.emptySet()); + 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() { @@ -356,6 +389,7 @@ class ReplicatedAuctionExample } // #command-handler + // #event-handler @Override public EventHandler eventHandler() { return newEventHandlerBuilder() @@ -382,6 +416,7 @@ class ReplicatedAuctionExample .onEvent(WinnerDecided.class, (state, event) -> state.close()) .build(); } + // #event-handler // #event-triggers private void eventTriggers(AuctionFinished event, AuctionState newState) { @@ -393,14 +428,17 @@ class ReplicatedAuctionExample context.getSelf().tell(Finish.INSTANCE); } } - // #event-triggers private boolean shouldClose(AuctionState state) { - return setup.responsibleForClosing + 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/ReplicatedStringSet.java b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedStringSet.java index 027021d14b..0aceecac1e 100644 --- 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 @@ -26,7 +26,8 @@ public final class ReplicatedStringSet public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ReplicatedEventSourcing.withSharedJournal("StringSet", + return ReplicatedEventSourcing.withSharedJournal( + "StringSet", entityId, replicaId, allReplicas, 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 index f28493b6c3..724ad79af9 100644 --- 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 @@ -6,231 +6,290 @@ package docs.akka.persistence.typed import java.time.Instant -import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } -import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, _ } -import akka.actor.typed.{ ActorRef, Behavior } -import akka.persistence.testkit.PersistenceTestKitPlugin -import akka.persistence.testkit.query.scaladsl.PersistenceTestKitReadJournal -import akka.persistence.typed.ReplicaId -import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, ReplicatedEventSourcing, ReplicationContext } -import akka.serialization.jackson.CborSerializable -import org.scalatest.concurrent.{ Eventually, ScalaFutures } +import scala.concurrent.duration._ + +import docs.akka.persistence.typed.ReplicatedAuctionExampleSpec.AuctionEntity +import org.scalatest.concurrent.Eventually +import org.scalatest.concurrent.ScalaFutures import org.scalatest.matchers.should.Matchers 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.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 { - type MoneyAmount = Int + //#setup + object AuctionEntity { - case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originReplica: ReplicaId) + //#setup - //#commands - sealed trait AuctionCommand - case object Finish extends AuctionCommand // A timer needs to schedule this event at each replica - final case class OfferBid(bidder: String, offer: MoneyAmount) extends AuctionCommand - final case class GetHighestBid(replyTo: ActorRef[Bid]) extends AuctionCommand - final case class IsClosed(replyTo: ActorRef[Boolean]) extends AuctionCommand - private final case object Close extends AuctionCommand // Internal, should not be sent from the outside - //#commands + //#commands + type MoneyAmount = Int - //#events - sealed trait AuctionEvent extends CborSerializable - final case class BidRegistered(bid: Bid) extends AuctionEvent - final case class AuctionFinished(atReplica: ReplicaId) extends AuctionEvent - final case class WinnerDecided(atReplica: ReplicaId, winningBid: Bid, highestCounterOffer: MoneyAmount) - extends AuctionEvent - //#events + case class Bid(bidder: String, offer: MoneyAmount, timestamp: Instant, originReplica: ReplicaId) - //#phase - sealed trait AuctionPhase - case object Running extends AuctionPhase - final case class Closing(finishedAtReplica: Set[ReplicaId]) extends AuctionPhase - case object Closed extends AuctionPhase - //#phase + 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 - //#state - case class AuctionState(phase: AuctionPhase, highestBid: Bid, highestCounterOffer: MoneyAmount) { + //#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 - def applyEvent(event: AuctionEvent): 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) + //#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 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) } - def withTooLowBid(bid: Bid): AuctionState = { - require(phase != Closed) - require(isHigherBid(highestBid, bid)) - copy(highestCounterOffer = highestCounterOffer.max(bid.offer)) // update highest counter offer + //#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 + .withSharedJournal("auction", name, replica, allReplicas, PersistenceTestKitReadJournal.Identifier) { + replicationCtx => + new AuctionEntity(ctx, replicationCtx, timers, closingAt, responsibleForClosing, allReplicas) + .behavior(initialBid) + } + } } - 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 - case class AuctionSetup( - name: String, - initialBid: Bid, // the initial bid is basically the minimum price bidden at start time by the owner + class AuctionEntity( + context: ActorContext[AuctionEntity.Command], + replicationContext: ReplicationContext, + timers: TimerScheduler[AuctionEntity.Command], closingAt: Instant, responsibleForClosing: Boolean, - allReplicas: Set[ReplicaId]) - //#setup + allReplicas: Set[ReplicaId]) { + import AuctionEntity._ - //#command-handler - def commandHandler(setup: AuctionSetup, ctx: ActorContext[AuctionCommand], replicationContext: ReplicationContext)( - state: AuctionState, - command: AuctionCommand): Effect[AuctionEvent, 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 => - ctx.log.info("Finish") - Effect.persist(AuctionFinished(replicationContext.replicaId)) - case Close => - ctx.log.info("Close") - require(shouldClose(setup, 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 => - ctx.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 - - private def shouldClose(auctionSetup: AuctionSetup, state: AuctionState): Boolean = { - auctionSetup.responsibleForClosing && (state.phase match { - case Closing(alreadyFinishedAtDc) => - val allDone = auctionSetup.allReplicas.diff(alreadyFinishedAtDc).isEmpty - if (!allDone) { - println( - s"Not closing auction as not all DCs have reported finished. All DCs: ${auctionSetup.allReplicas}. Reported finished ${alreadyFinishedAtDc}") - } - allDone - case _ => - false - }) - } - - //#event-handler - def eventHandler(ctx: ActorContext[AuctionCommand], replicationCtx: ReplicationContext, setup: AuctionSetup)( - state: AuctionState, - event: AuctionEvent): AuctionState = { - - val newState = state.applyEvent(event) - ctx.log.infoN("Applying event {}. New start {}", event, newState) - if (!replicationCtx.recoveryRunning) { - eventTriggers(setup, ctx, replicationCtx, event, newState) - } - newState - - } - //#event-handler - - //#event-triggers - private def eventTriggers( - setup: AuctionSetup, - ctx: ActorContext[AuctionCommand], - replicationCtx: ReplicationContext, - event: AuctionEvent, - newState: AuctionState) = { - event match { - case finished: AuctionFinished => - newState.phase match { - case Closing(alreadyFinishedAtDc) => - ctx.log.infoN( - "AuctionFinished at {}, already finished at [{}]", - finished.atReplica, - alreadyFinishedAtDc.mkString(", ")) - if (alreadyFinishedAtDc(replicationCtx.replicaId)) { - if (shouldClose(setup, newState)) ctx.self ! Close - } else { - ctx.log.info("Sending finish to self") - ctx.self ! Finish - } - - case _ => // no trigger for this state - } - case _ => // no trigger for this event - } - } - //#event-triggers - - def initialState(setup: AuctionSetup) = - AuctionState(phase = Running, highestBid = setup.initialBid, highestCounterOffer = setup.initialBid.offer) - - def behavior(replica: ReplicaId, setup: AuctionSetup): Behavior[AuctionCommand] = Behaviors.setup[AuctionCommand] { - ctx => - ReplicatedEventSourcing.withSharedJournal( - "auction", - setup.name, - replica, - setup.allReplicas, - PersistenceTestKitReadJournal.Identifier) { replicationCtx => - EventSourcedBehavior( - replicationCtx.persistenceId, - initialState(setup), - commandHandler(setup, ctx, replicationCtx), - eventHandler(ctx, replicationCtx, setup)) + 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 @@ -240,24 +299,20 @@ class ReplicatedAuctionExampleSpec with LogCapturing with ScalaFutures with Eventually { - import ReplicatedAuctionExampleSpec._ + import ReplicatedAuctionExampleSpec.AuctionEntity._ "Auction example" should { "work" in { val Replicas = Set(ReplicaId("DC-A"), ReplicaId("DC-B")) - val setupA = - AuctionSetup( - "old-skis", - Bid("chbatey", 12, Instant.now(), ReplicaId("DC-A")), - Instant.now().plusSeconds(10), - responsibleForClosing = true, - Replicas) + val auctionName = "old-skis" + val initialBid = Bid("chbatey", 12, Instant.now(), ReplicaId("DC-A")) + val closingAt = Instant.now().plusSeconds(10) - val setupB = setupA.copy(responsibleForClosing = false) - - val dcAReplica: ActorRef[AuctionCommand] = spawn(behavior(ReplicaId("DC-A"), setupA)) - val dcBReplica: ActorRef[AuctionCommand] = spawn(behavior(ReplicaId("DC-B"), setupB)) + 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) From d078a6b65f019551110f2d20caff4ad30cb31c1d Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 11 Aug 2020 12:50:09 +0200 Subject: [PATCH 39/50] LwwTime example * show event and command handlers of the blog example in docs * add blog example for Java --- .../paradox/typed/replicated-eventsourcing.md | 18 +- .../typed/ReplicatedBlogExample.java | 292 ++++++++++++++++++ .../typed/ReplicatedBlogExampleSpec.scala | 236 +++++++------- 3 files changed, 436 insertions(+), 110 deletions(-) create mode 100644 akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedBlogExample.java diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index 34d126a619..2cd709c6d6 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -156,6 +156,21 @@ replica that persisted it. When comparing two @apidoc[LwwTime] the greatest time 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. @@ -188,10 +203,11 @@ Side effects from the event handler are generally discouraged because the event 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 occured +* 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 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..53cea01ed4 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedBlogExample.java @@ -0,0 +1,292 @@ +/* + * 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.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.withSharedJournal( + "StringSet", + 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/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala index e9a86ab8bf..afde7d0de5 100644 --- 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 @@ -4,44 +4,137 @@ package docs.akka.persistence.typed +import org.scalatest.concurrent.Eventually +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.matchers.should.Matchers +import org.scalatest.time.Millis +import org.scalatest.time.Span +import org.scalatest.wordspec.AnyWordSpecLike + import akka.Done -import akka.actor.testkit.typed.scaladsl.{ LogCapturing, ScalaTestWithActorTestKit } +import akka.actor.testkit.typed.scaladsl.LogCapturing +import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.typed.ActorRef -import akka.actor.typed.scaladsl.{ ActorContext, Behaviors } +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.crdt.LwwTime import akka.persistence.typed.scaladsl._ import akka.serialization.jackson.CborSerializable -import org.scalatest.concurrent.{ Eventually, ScalaFutures } -import org.scalatest.matchers.should.Matchers -import org.scalatest.time.{ Millis, Span } -import org.scalatest.wordspec.AnyWordSpecLike object ReplicatedBlogExampleSpec { - final case class BlogState(content: Option[PostContent], contentTimestamp: LwwTime, published: Boolean) { - def withContent(newContent: PostContent, timestamp: LwwTime): BlogState = - copy(content = Some(newContent), contentTimestamp = timestamp) - def isEmpty: Boolean = content.isEmpty + 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.withSharedJournal( + "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 } - val emptyState: BlogState = BlogState(None, LwwTime(Long.MinValue, ReplicaId("")), published = false) - - final case class PostContent(title: String, body: String) - final case class PostSummary(postId: String, title: String) - final case class Published(postId: String) extends BlogEvent - - sealed trait BlogCommand - final case class AddPost(postId: String, content: PostContent, replyTo: ActorRef[AddPostDone]) extends BlogCommand - final case class AddPostDone(postId: String) - final case class GetPost(postId: String, replyTo: ActorRef[PostContent]) extends BlogCommand - final case class ChangeBody(postId: String, newContent: PostContent, replyTo: ActorRef[Done]) extends BlogCommand - final case class Publish(postId: String, replyTo: ActorRef[Done]) extends BlogCommand - - sealed trait BlogEvent extends CborSerializable - final case class PostAdded(postId: String, content: PostContent, timestamp: LwwTime) extends BlogEvent - final case class BodyChanged(postId: String, newContent: PostContent, timestamp: LwwTime) extends BlogEvent } class ReplicatedBlogExampleSpec @@ -51,98 +144,23 @@ class ReplicatedBlogExampleSpec with LogCapturing with ScalaFutures with Eventually { - import ReplicatedBlogExampleSpec._ + import ReplicatedBlogExampleSpec.BlogEntity + import ReplicatedBlogExampleSpec.BlogEntity._ implicit val config: PatienceConfig = PatienceConfig(timeout = Span(timeout.duration.toMillis, Millis)) - def behavior(replicationContext: ReplicationContext, ctx: ActorContext[BlogCommand]) = - EventSourcedBehavior[BlogCommand, BlogEvent, BlogState]( - replicationContext.persistenceId, - emptyState, - (state, cmd) => - cmd match { - case AddPost(_, content, replyTo) => - val evt = - PostAdded( - replicationContext.persistenceId.id, - content, - state.contentTimestamp.increase(replicationContext.currentTimeMillis(), replicationContext.replicaId)) - Effect.persist(evt).thenRun { _ => - replyTo ! AddPostDone(replicationContext.entityId) - } - case ChangeBody(_, newContent, replyTo) => - val evt = - BodyChanged( - replicationContext.persistenceId.id, - 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 - }, - (state, event) => { - 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) - } - }) - "Blog Example" should { "work" in { - val refDcA: ActorRef[BlogCommand] = - spawn( - Behaviors.setup[BlogCommand] { ctx => - ReplicatedEventSourcing.withSharedJournal( - "blog", - "cat", - ReplicaId("DC-A"), - Set(ReplicaId("DC-A"), ReplicaId("DC-B")), - PersistenceTestKitReadJournal.Identifier) { replicationContext => - behavior(replicationContext, ctx) - } - }, - "dc-a") + val refDcA: ActorRef[Command] = + spawn(BlogEntity("cat", ReplicaId("DC-A"), Set(ReplicaId("DC-A"), ReplicaId("DC-B")))) - val refDcB: ActorRef[BlogCommand] = - spawn( - Behaviors.setup[BlogCommand] { ctx => - ReplicatedEventSourcing.withSharedJournal( - "blog", - "cat", - ReplicaId("DC-B"), - Set(ReplicaId("DC-A"), ReplicaId("DC-B")), - PersistenceTestKitReadJournal.Identifier) { replicationContext => - behavior(replicationContext, ctx) - } - }, - "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 - - import scala.concurrent.duration._ implicit val timeout: Timeout = 3.seconds val content = PostContent("cats are the bets", "yep") From 2e0821c2f098bba416db31aa485508cc0c764b41 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 11 Aug 2020 13:39:25 +0200 Subject: [PATCH 40/50] ORSet example, MovieWatchList --- .../paradox/typed/replicated-eventsourcing.md | 32 +++++- .../typed/ReplicatedBlogExample.java | 2 +- .../typed/ReplicatedMovieExample.java | 105 +++++++++++++++++ .../typed/ReplicatedAuctionExampleSpec.scala | 8 +- .../typed/ReplicatedBlogExampleSpec.scala | 8 +- .../ReplicatedMovieWatchListExampleSpec.scala | 106 ++++++++++++++++++ 6 files changed, 244 insertions(+), 17 deletions(-) create mode 100644 akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedMovieExample.java create mode 100644 akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedMovieWatchListExampleSpec.scala diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index 2cd709c6d6..ae05d76266 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -134,13 +134,41 @@ The factory returns a `Behavior` that can be spawned like any other behavior. ### Conflict free replicated data types -The following CRDTs are included that can be used to build your own data model: +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). +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 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 index 53cea01ed4..4af9d46f74 100644 --- 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 @@ -26,7 +26,7 @@ interface ReplicatedBlogExample { public final class BlogEntity extends ReplicatedEventSourcedBehavior< - BlogEntity.Command, BlogEntity.Event, BlogEntity.BlogState> { + BlogEntity.Command, BlogEntity.Event, BlogEntity.BlogState> { private final ActorContext context; 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..16ae730de1 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedMovieExample.java @@ -0,0 +1,105 @@ +/* + * 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.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.withSharedJournal( + "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/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedAuctionExampleSpec.scala index 724ad79af9..2d59b5ce97 100644 --- 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 @@ -9,9 +9,6 @@ import java.time.Instant import scala.concurrent.duration._ import docs.akka.persistence.typed.ReplicatedAuctionExampleSpec.AuctionEntity -import org.scalatest.concurrent.Eventually -import org.scalatest.concurrent.ScalaFutures -import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpecLike import akka.actor.testkit.typed.scaladsl.LogCapturing @@ -295,10 +292,7 @@ object ReplicatedAuctionExampleSpec { class ReplicatedAuctionExampleSpec extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) with AnyWordSpecLike - with Matchers - with LogCapturing - with ScalaFutures - with Eventually { + with LogCapturing { import ReplicatedAuctionExampleSpec.AuctionEntity._ "Auction example" should { 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 index afde7d0de5..79d6543a85 100644 --- 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 @@ -4,9 +4,6 @@ package docs.akka.persistence.typed -import org.scalatest.concurrent.Eventually -import org.scalatest.concurrent.ScalaFutures -import org.scalatest.matchers.should.Matchers import org.scalatest.time.Millis import org.scalatest.time.Span import org.scalatest.wordspec.AnyWordSpecLike @@ -140,10 +137,7 @@ object ReplicatedBlogExampleSpec { class ReplicatedBlogExampleSpec extends ScalaTestWithActorTestKit(PersistenceTestKitPlugin.config) with AnyWordSpecLike - with Matchers - with LogCapturing - with ScalaFutures - with Eventually { + with LogCapturing { import ReplicatedBlogExampleSpec.BlogEntity import ReplicatedBlogExampleSpec.BlogEntity._ 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..ff9c14e8dd --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedMovieWatchListExampleSpec.scala @@ -0,0 +1,106 @@ +/* + * 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.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.withSharedJournal( + "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"))) + } + } + + } + +} From 779e827495351d232a6dafc6591f03c6b585f34a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 11 Aug 2020 14:40:47 +0200 Subject: [PATCH 41/50] Counter example, shopping cart --- .../typed/replicated-eventsourcing-cart.md | 22 +++ .../replicated-eventsourcing-examples.md | 1 + .../typed/ReplicatedBlogExample.java | 2 +- .../typed/ReplicatedShoppingCartExample.java | 159 ++++++++++++++++++ .../typed/ReplicatedBlogExampleSpec.scala | 4 - .../ReplicatedShoppingCartExampleSpec.scala | 123 ++++++++++++++ 6 files changed, 306 insertions(+), 5 deletions(-) create mode 100644 akka-docs/src/main/paradox/typed/replicated-eventsourcing-cart.md create mode 100644 akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedShoppingCartExample.java create mode 100644 akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedShoppingCartExampleSpec.scala 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 index 0d53de9289..e0f17977d1 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-examples.md @@ -7,6 +7,7 @@ The following are more realistic examples of building systems with Replicated Ev @@@ index * [auction](replicated-eventsourcing-auction.md) +* [shopping cart](replicated-eventsourcing-cart.md) @@@ 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 index 4af9d46f74..09c14b60c4 100644 --- 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 @@ -187,7 +187,7 @@ interface ReplicatedBlogExample { return Behaviors.setup( context -> ReplicatedEventSourcing.withSharedJournal( - "StringSet", + "blog", entityId, replicaId, allReplicas, 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..2c49cd2fd9 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/java/jdocs/akka/persistence/typed/ReplicatedShoppingCartExample.java @@ -0,0 +1,159 @@ +/* + * 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.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.withSharedJournal( + "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/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedBlogExampleSpec.scala index 79d6543a85..4d1d9592d6 100644 --- 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 @@ -4,8 +4,6 @@ package docs.akka.persistence.typed -import org.scalatest.time.Millis -import org.scalatest.time.Span import org.scalatest.wordspec.AnyWordSpecLike import akka.Done @@ -141,8 +139,6 @@ class ReplicatedBlogExampleSpec import ReplicatedBlogExampleSpec.BlogEntity import ReplicatedBlogExampleSpec.BlogEntity._ - implicit val config: PatienceConfig = PatienceConfig(timeout = Span(timeout.duration.toMillis, Millis)) - "Blog Example" should { "work" in { val refDcA: ActorRef[Command] = 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..8a311df9a6 --- /dev/null +++ b/akka-persistence-typed-tests/src/test/scala/docs/akka/persistence/typed/ReplicatedShoppingCartExampleSpec.scala @@ -0,0 +1,123 @@ +/* + * 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.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.withSharedJournal( + "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))) + } + } + } +} From 849018b81e026479176655c642db6c8dc2e3bea1 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Thu, 13 Aug 2020 11:27:00 +0100 Subject: [PATCH 42/50] Replicated Sharding improvements (#29483) * WIP * Finishing touches to sharding updates * Review feedback --- .../typed/ReplicatedEntityProvider.scala | 28 +- .../ReplicatedShardingExtensionImpl.scala | 43 ++- .../typed/ReplicatedShardingTest.java | 16 +- .../ReplicatedShardingCompileOnlySpec.java | 103 +++++++ ...licatedShardingDirectReplicationSpec.scala | 4 +- .../typed/ReplicatedShardingSpec.scala | 263 ++++++++++++++---- .../ReplicatedShardingCompileOnlySpec.scala | 74 +++++ .../cluster/sharding/ClusterSharding.scala | 3 +- .../src/main/scala/akka/cluster/Member.scala | 4 +- .../paradox/typed/replicated-eventsourcing.md | 39 ++- .../typed/ReplicatedEventSourcingTest.java | 4 +- .../typed/MyReplicatedBehavior.java | 9 +- .../typed/ReplicatedAuctionExampleTest.java | 5 +- .../typed/ReplicatedBlogExample.java | 5 +- .../typed/ReplicatedMovieExample.java | 5 +- .../typed/ReplicatedShoppingCartExample.java | 5 +- .../typed/ReplicatedStringSet.java | 5 +- .../typed/MultiJournalReplicationSpec.scala | 14 +- .../typed/ReplicatedEventPublishingSpec.scala | 20 +- .../typed/ReplicatedEventSourcingSpec.scala | 4 +- .../ReplicatedEventSourcingTaggingSpec.scala | 5 +- .../typed/ReplicationIllegalAccessSpec.scala | 8 +- .../typed/ReplicationSnapshotSpec.scala | 16 +- .../persistence/typed/crdt/CounterSpec.scala | 5 +- .../akka/persistence/typed/crdt/LwwSpec.scala | 5 +- .../persistence/typed/crdt/ORSetSpec.scala | 5 +- .../typed/ReplicatedAuctionExampleSpec.scala | 14 +- .../typed/ReplicatedBlogExampleSpec.scala | 5 +- ...plicatedEventSourcingCompileOnlySpec.scala | 13 +- .../ReplicatedMovieWatchListExampleSpec.scala | 6 +- .../ReplicatedShoppingCartExampleSpec.scala | 6 +- .../persistence/typed/PersistenceId.scala | 22 -- .../persistence/typed/ReplicationId.scala | 45 +++ .../internal/EventSourcedBehaviorImpl.scala | 3 +- .../typed/internal/ReplicationSetup.scala | 8 +- .../persistence/typed/internal/Running.scala | 7 +- .../javadsl/ReplicatedEventSourcing.scala | 26 +- .../scaladsl/ReplicatedEventSourcing.scala | 29 +- 38 files changed, 615 insertions(+), 266 deletions(-) create mode 100644 akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java create mode 100644 akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.scala create mode 100644 akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicationId.scala 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 index 6b2e5a0586..2a73361093 100644 --- 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 @@ -16,6 +16,7 @@ import java.util.{ Set => JSet } import akka.annotation.ApiMayChange import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl +import akka.persistence.typed.ReplicationId.Separator @ApiMayChange object ReplicatedEntityProvider { @@ -28,29 +29,30 @@ object ReplicatedEntityProvider { */ def create[M, E]( messageClass: Class[M], + typeName: String, allReplicaIds: JSet[ReplicaId], - settingsPerReplicaFactory: akka.japi.function.Function3[ - JEntityTypeKey[M], - ReplicaId, - JSet[ReplicaId], - ReplicatedEntity[M, E]]): ReplicatedEntityProvider[M, E] = { + settingsPerReplicaFactory: akka.japi.function.Function2[JEntityTypeKey[M], ReplicaId, ReplicatedEntity[M, E]]) + : ReplicatedEntityProvider[M, E] = { implicit val classTag: ClassTag[M] = ClassTag(messageClass) - apply[M, E](allReplicaIds.asScala.toSet)((key, replica, _) => - settingsPerReplicaFactory(key.asInstanceOf[EntityTypeKeyImpl[M]], replica, allReplicaIds)) + apply[M, E](typeName, allReplicaIds.asScala.toSet)((key, replica) => + settingsPerReplicaFactory(key.asInstanceOf[EntityTypeKeyImpl[M]], replica)) } /** * Scala API: - * + * @param typeName The type name used in the [[EntityTypeKey]] * @tparam M The type of messages the replicated entity accepts * @tparam E The type for envelopes used for sending `M`s over sharding */ - def apply[M: ClassTag, E](allReplicaIds: Set[ReplicaId])( - settingsPerReplicaFactory: (EntityTypeKey[M], ReplicaId, Set[ReplicaId]) => ReplicatedEntity[M, E]) + def apply[M: ClassTag, E](typeName: String, allReplicaIds: Set[ReplicaId])( + settingsPerReplicaFactory: (EntityTypeKey[M], ReplicaId) => ReplicatedEntity[M, E]) : ReplicatedEntityProvider[M, E] = { new ReplicatedEntityProvider(allReplicaIds.map { replicaId => - val typeKey = EntityTypeKey[M](replicaId.id) - settingsPerReplicaFactory(typeKey, replicaId, allReplicaIds) + 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 = false) } } @@ -61,7 +63,7 @@ object ReplicatedEntityProvider { */ @ApiMayChange final class ReplicatedEntityProvider[M, E] private ( - val replicas: immutable.Seq[ReplicatedEntity[M, E]], + val replicas: immutable.Seq[(ReplicatedEntity[M, E], String)], val directReplication: Boolean) { /** 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 index aff3515bb9..f8f3e2383f 100644 --- 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 @@ -16,12 +16,12 @@ 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.PersistenceId 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._ /** @@ -36,17 +36,23 @@ private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_] override def init[M, E](settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] = { val sharding = ClusterSharding(system) - val initializedReplicas = settings.replicas.map { replicaSettings => - // 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) - (replicaSettings.replicaId, replicaSettings.entity.typeKey, regionOrProxy) + 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 (id, _, regionOrProxy) => id -> regionOrProxy + case (_, replicaId, _, regionOrProxy, _) => replicaId -> regionOrProxy }.toMap if (settings.directReplication) { logger.infoN("Starting Replicated Event Sourcing Direct Replication") @@ -55,7 +61,9 @@ private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_] s"directReplication-${counter.incrementAndGet()}") } - val replicaToTypeKey = initializedReplicas.map { case (id, typeKey, _) => id -> typeKey }.toMap + val replicaToTypeKey = initializedReplicas.map { + case (typeName, id, typeKey, _, dc) => id -> ((typeKey, dc, typeName)) + }.toMap new ReplicatedShardingImpl(sharding, replicaToRegionOrProxy, replicaToTypeKey) } } @@ -67,16 +75,21 @@ private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_] private[akka] final class ReplicatedShardingImpl[M, E]( sharding: ClusterSharding, shardingPerReplica: Map[ReplicaId, ActorRef[E]], - replicaTypeKeys: Map[ReplicaId, EntityTypeKey[M]]) + replicaTypeKeys: Map[ReplicaId, (EntityTypeKey[M], Option[DataCenter], String)]) extends ReplicatedSharding[M, E] { + // FIXME add test coverage for these override def shardingRefs: Map[ReplicaId, ActorRef[E]] = shardingPerReplica override def getShardingRefs: JMap[ReplicaId, ActorRef[E]] = shardingRefs.asJava override def entityRefsFor(entityId: String): Map[ReplicaId, EntityRef[M]] = replicaTypeKeys.map { - case (replicaId, typeKey) => - replicaId -> sharding.entityRefFor(typeKey, PersistenceId.ofUniqueId(entityId).id) + 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, EntityRef[M]] = 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 index be2ec24770..0f9c105b1d 100644 --- 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 @@ -21,6 +21,7 @@ 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; @@ -32,6 +33,7 @@ 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 { @@ -64,13 +66,10 @@ public class ReplicatedShardingTest extends JUnitSuite { } } - static Behavior create( - String entityId, ReplicaId replicaId, Set allReplicas) { + static Behavior create(ReplicationId replicationId) { return ReplicatedEventSourcing.withSharedJournal( - "StringSet", - entityId, - replicaId, - allReplicas, + replicationId, + ALL_REPLICAS, PersistenceTestKitReadJournal.Identifier(), MyReplicatedStringSet::new); } @@ -157,9 +156,10 @@ public class ReplicatedShardingTest extends JUnitSuite { replicatedEntityProvider = ReplicatedEntityProvider.create( MyReplicatedStringSet.Command.class, + "StringSet", ALL_REPLICAS, // factory for replicated entity for a given replica - (entityTypeKey, replicaId, allReplicas) -> + (entityTypeKey, replicaId) -> ReplicatedEntity.create( replicaId, // use the replica id as typekey for sharding to get one sharding instance @@ -169,7 +169,7 @@ public class ReplicatedShardingTest extends JUnitSuite { entityContext -> // factory for the entity for a given entity in that replica MyReplicatedStringSet.create( - entityContext.getEntityId(), replicaId, allReplicas)) + 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 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..a65a7337c5 --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java @@ -0,0 +1,103 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package jdocs.akka.cluster.sharding.typed; + +import akka.actor.typed.ActorRef; +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.scaladsl.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"); + Map>> shardingRefs = + replicatedSharding.getShardingRefs(); + // #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 index 871317b1ce..550ca2e296 100644 --- 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 @@ -10,10 +10,10 @@ 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.PersistenceId 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 { @@ -37,7 +37,7 @@ class ReplicatedShardingDirectReplicationSpec extends ScalaTestWithActorTestKit upProbe.receiveMessage() // not bullet proof wrt to subscription being complete but good enough val event = PublishedEventImpl( - PersistenceId.replicatedId("ReplicatedShardingSpec", "pid", ReplicaId("ReplicaA")), + ReplicationId("ReplicatedShardingSpec", "pid", ReplicaId("ReplicaA")).persistenceId, 1L, "event", System.currentTimeMillis(), 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 index b37a6d26d6..af33673f36 100644 --- 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 @@ -6,12 +6,18 @@ 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 @@ -24,35 +30,54 @@ 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 object ReplicatedShardingSpec { - def config = ConfigFactory.parseString(""" - akka.loglevel = DEBUG + def commonConfig = ConfigFactory.parseString(""" + akka.loglevel = INFO akka.loggers = ["akka.testkit.SilenceAllTestEventListener"] akka.actor.provider = "cluster" - # pretend we're a node in all dc:s - akka.cluster.roles = ["DC-A", "DC-B", "DC-C"] akka.remote.classic.netty.tcp.port = 0 akka.remote.artery.canonical.port = 0""").withFallback(PersistenceTestKitPlugin.config) -} -class ReplicatedShardingSpec - extends ScalaTestWithActorTestKit(ReplicatedShardingSpec.config) - with AnyWordSpecLike - with LogCapturing { + 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(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] = - ReplicatedEventSourcing.withSharedJournal( - "StringSet", - entityId, - replicaId, - allReplicas, + def apply(replicationId: ReplicationId): Behavior[Command] = + ReplicatedEventSourcing.withSharedJournal( // it isn't really shared as it is in memory + replicationId, + AllReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => EventSourcedBehavior[Command, String, Set[String]]( replicationContext.persistenceId, @@ -65,81 +90,205 @@ class ReplicatedShardingSpec replyTo ! Texts(state) Effect.none }, - (state, event) => state + event).withJournalPluginId(PersistenceTestKitPlugin.PluginId) + (state, event) => state + event) + .withJournalPluginId(PersistenceTestKitPlugin.PluginId) + .withEventPublishing(true) } + + def provider(replicationType: ReplicationType) = + ReplicatedEntityProvider[MyReplicatedStringSet.Command, ShardingEnvelope[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 ProxyActor { - sealed trait Command - case class ForwardToRandom(entityId: String, msg: MyReplicatedStringSet.Command) extends Command - case class ForwardToAll(entityId: String, msg: MyReplicatedStringSet.Command) extends Command + 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(): Behavior[Command] = Behaviors.setup { context => - // #bootstrap - val replicatedShardingProvider = - ReplicatedEntityProvider[MyReplicatedStringSet.Command, ShardingEnvelope[MyReplicatedStringSet.Command]]( - // all replicas - Set(ReplicaId("DC-A"), ReplicaId("DC-B"), ReplicaId("DC-C"))) { (entityTypeKey, replicaId, allReplicaIds) => - // factory for replicated entity for a given replica - ReplicatedEntity( - replicaId, - // use the provided entity type key for sharding to get one sharding instance per replica - Entity(entityTypeKey) { entityContext => - // factory for the entity for a given entity in that replica - MyReplicatedStringSet(entityContext.entityId, replicaId, allReplicaIds) - } - // 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)) + def apply(id: ReplicationId, allReplicas: Set[ReplicaId]): Behavior[Command] = + ReplicatedEventSourcing.withSharedJournal( // 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, ShardingEnvelope[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) + } +} - val replicatedSharding = ReplicatedShardingExtension(context.system).init(replicatedShardingProvider) - // #bootstrap +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 ForwardToAll(entityId, cmd) => - // #all-entity-refs - replicatedSharding.entityRefsFor(entityId).foreach { + 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 } - // #all-entity-refs Behaviors.same - case ForwardToRandom(entityId, cmd) => - val refs = replicatedSharding.entityRefsFor(entityId) + case ForwardToRandomInt(entityId, cmd) => + val refs = + replicatedShardingIntSet.entityRefsFor(entityId) val chosenIdx = ThreadLocalRandom.current().nextInt(refs.size) - refs.values.toIndexedSeq(chosenIdx) ! cmd; + 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 { - val node = Cluster(system) - node.manager ! Join(node.selfMember.address) + Cluster(system).manager ! Join(Cluster(system).selfMember.address) + Cluster(system2).manager ! Join(Cluster(system).selfMember.address) + eventually { - node.selfMember.status should ===(MemberStatus.Up) + Cluster(system).state.members.size should ===(2) + Cluster(system).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up)) + } + eventually { + Cluster(system2).state.members.size should ===(2) + Cluster(system2).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up)) } } - "forward to replicas" in { - val proxy = spawn(ProxyActor()) + "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) + } - proxy ! ProxyActor.ForwardToAll("id1", MyReplicatedStringSet.Add("to-all")) - proxy ! ProxyActor.ForwardToRandom("id1", MyReplicatedStringSet.Add("to-random")) + "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.ForwardToAll("id1", MyReplicatedStringSet.GetTexts(probe.ref)) - val responses: Seq[MyReplicatedStringSet.Texts] = probe.receiveMessages(3) + 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..45bb443191 --- /dev/null +++ b/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.scala @@ -0,0 +1,74 @@ +/* + * Copyright (C) 2020 Lightbend Inc. + */ + +package docs.akka.cluster.sharding.typed + +import akka.actor.typed.ActorRef +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.ShardingEnvelope +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, ShardingEnvelope[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[Command, ShardingEnvelope[Command]]( + "MyEntityType", + Set(ReplicaId("DC-A"), ReplicaId("DC-B"))) { (entityTypeKey, replicaId) => + ReplicatedEntity(replicaId, Entity(entityTypeKey) { entityContext => + val replicationId = ReplicationId.fromString(entityContext.entityId) + MyEventSourcedBehavior(replicationId) + }.withDataCenter(replicaId.id)) + } + //#bootstrap-dc + + //#bootstrap-role + val provider = ReplicatedEntityProvider[Command, ShardingEnvelope[Command]]( + "MyEntityType", + Set(ReplicaId("DC-A"), ReplicaId("DC-B"))) { (entityTypeKey, replicaId) => + ReplicatedEntity(replicaId, Entity(entityTypeKey) { entityContext => + val replicationId = ReplicationId.fromString(entityContext.entityId) + MyEventSourcedBehavior(replicationId) + }.withRole(replicaId.id)) + } + //#bootstrap-role + + //#sending-messages + val myReplicatedSharding: ReplicatedSharding[Command, ShardingEnvelope[Command]] = + ReplicatedShardingExtension(system).init(provider) + + val entityRefs: Map[ReplicaId, EntityRef[Command]] = myReplicatedSharding.entityRefsFor("myEntityId") + val actorRefs: Map[ReplicaId, ActorRef[ShardingEnvelope[Command]]] = myReplicatedSharding.shardingRefs + //#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/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/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index ae05d76266..72361f204b 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -298,27 +298,50 @@ When comparing two version vectors `v1` and `v2`: ## Sharded Replicated Event Sourced entities -To simplify what probably are the most common use cases for how you will want to distribute the replicated actors there is a minimal API for running multiple instances of @ref[Akka Cluster Sharding](cluster-sharding.md), -each instance holding the entities for a single replica. +There are three ways to integrate replicated event sourced entities with sharding: -The distribution of the replicas can be controlled either through cluster roles or using the @ref[multi datacenter](cluster-dc.md) support in Akka Cluster. +* 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 -The API consists of bootstrapping logic for starting the sharding instances through @apidoc[ReplicatedShardingExtension] available from the + +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/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala) { #bootstrap } +: @@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/akka/cluster/sharding/typed/ReplicatedShardingTest.java) { #bootstrap } +: @@snip [ReplicatedShardingTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java) { #bootstrap } + +This will run a single instance of sharding and the replicas will be differentiated by having the replica id in the sharding entity id. +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/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala) { #all-entity-refs } +: @@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/akka/cluster/sharding/typed/ReplicatedShardingTest.java) { #all-entity-refs } +: @@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)). 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 index a5efaf67fa..7cbc460f10 100644 --- 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 @@ -81,9 +81,7 @@ public class ReplicatedEventSourcingTest extends JUnitSuite { public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { return ReplicatedEventSourcing.withSharedJournal( - "ReplicatedEventSourcingTest", - entityId, - replicaId, + new ReplicationId("ReplicatedEventSourcingTest", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), TestBehavior::new); 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 index 911d91486f..5a04ddf503 100644 --- 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 @@ -6,6 +6,7 @@ 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.*; @@ -33,9 +34,7 @@ public class MyReplicatedBehavior public static Behavior create( String entityId, ReplicaId replicaId, String queryPluginId) { return ReplicatedEventSourcing.withSharedJournal( - "MyReplicatedEntity", - entityId, - replicaId, + new ReplicationId("MyReplicatedEntity", entityId, replicaId), ALL_REPLICAS, queryPluginId, MyReplicatedBehavior::new); @@ -49,9 +48,7 @@ public class MyReplicatedBehavior allReplicasAndQueryPlugins.put(DCB, "journalForDCB"); return ReplicatedEventSourcing.create( - "MyReplicatedEntity", - entityId, - replicaId, + new ReplicationId("MyReplicatedEntity", entityId, replicaId), allReplicasAndQueryPlugins, MyReplicatedBehavior::new); } 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 index 955f1ff8b2..8d1091b7c3 100644 --- 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 @@ -16,6 +16,7 @@ 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; @@ -262,9 +263,7 @@ class AuctionEntity extends ReplicatedEventSourcedBehavior ReplicatedEventSourcing.withSharedJournal( - "Auction", - name, - replica, + new ReplicationId("Auction", name, replica), ALL_REPLICAS, PersistenceTestKitReadJournal.Identifier(), replicationCtx -> 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 index 09c14b60c4..b32c811ea5 100644 --- 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 @@ -11,6 +11,7 @@ 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; @@ -187,9 +188,7 @@ interface ReplicatedBlogExample { return Behaviors.setup( context -> ReplicatedEventSourcing.withSharedJournal( - "blog", - entityId, - replicaId, + new ReplicationId("blog", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), replicationContext -> new BlogEntity(context, replicationContext))); 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 index 16ae730de1..94b219c6c1 100644 --- 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 @@ -8,6 +8,7 @@ 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; @@ -61,9 +62,7 @@ interface ReplicatedMovieExample { public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { return ReplicatedEventSourcing.withSharedJournal( - "movies", - entityId, - replicaId, + new ReplicationId("movies", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), MovieWatchList::new); 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 index 2c49cd2fd9..629065b614 100644 --- 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 @@ -8,6 +8,7 @@ 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; @@ -85,9 +86,7 @@ interface ReplicatedShoppingCartExample { public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { return ReplicatedEventSourcing.withSharedJournal( - "blog", - entityId, - replicaId, + new ReplicationId("blog", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), ShoppingCart::new); 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 index 0aceecac1e..c9198f65d0 100644 --- 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 @@ -7,6 +7,7 @@ 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; @@ -27,9 +28,7 @@ public final class ReplicatedStringSet public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { return ReplicatedEventSourcing.withSharedJournal( - "StringSet", - entityId, - replicaId, + new ReplicationId("StringSet", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), ReplicatedStringSet::new); 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 index 2c4f0614af..4257e324f8 100644 --- 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 @@ -33,9 +33,7 @@ object MultiJournalReplicationSpec { private val writeJournalPerReplica = Map("R1" -> "journal1.journal", "R2" -> "journal2.journal") def apply(entityId: String, replicaId: String): Behavior[Command] = { ReplicatedEventSourcing( - "MultiJournalSpec", - entityId, - ReplicaId(replicaId), + ReplicationId("MultiJournalSpec", entityId, ReplicaId(replicaId)), Map(ReplicaId("R1") -> "journal1.query", ReplicaId("R2") -> "journal2.query"))( replicationContext => EventSourcedBehavior[Command, String, Set[String]]( @@ -100,11 +98,17 @@ class MultiJournalReplicationSpec val readJournal2 = PersistenceQuery(system).readJournalFor[CurrentEventsByPersistenceIdQuery]("journal2.query") val eventsForJournal1 = - readJournal1.currentEventsByPersistenceId("id1|R1", 0L, Long.MaxValue).runWith(Sink.seq).futureValue + 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("id1|R2", 0L, Long.MaxValue).runWith(Sink.seq).futureValue + 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 index 051be16df0..368f9885dc 100644 --- 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 @@ -31,9 +31,7 @@ object ReplicatedEventPublishingSpec { def apply(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] = Behaviors.setup { ctx => ReplicatedEventSourcing.withSharedJournal( - EntityType, - entityId, - replicaId, + ReplicationId(EntityType, entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier)( replicationContext => @@ -86,7 +84,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedId(EntityType, id, DCB), + ReplicationId(EntityType, id, DCB).persistenceId, 1L, "two", System.currentTimeMillis(), @@ -107,7 +105,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedId(EntityType, id, DCB), + ReplicationId(EntityType, id, DCB).persistenceId, 2L, // missing 1L "two", System.currentTimeMillis(), @@ -128,7 +126,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedId(EntityType, id, DCC), + ReplicationId(EntityType, id, DCC).persistenceId, 1L, "two", System.currentTimeMillis(), @@ -149,14 +147,14 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica actor.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedId(EntityType, "myId4", DCB), + 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( - PersistenceId.replicatedId(EntityType, id, DCB), + 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(), @@ -188,7 +186,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica incarnation2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedId(EntityType, id, DCB), + ReplicationId(EntityType, id, DCB).persistenceId, 1L, "two", System.currentTimeMillis(), @@ -211,7 +209,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica incarnationA1.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedId(EntityType, id, DCB), + ReplicationId(EntityType, id, DCB).persistenceId, 1L, "two", System.currentTimeMillis(), @@ -224,7 +222,7 @@ class ReplicatedEventPublishingSpec // simulate a published event from another replica incarnationA2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedId(EntityType, id, DCB), + ReplicationId(EntityType, id, DCB).persistenceId, 2L, "three", System.currentTimeMillis(), 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 index e84d7c91e8..c6ba98a771 100644 --- 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 @@ -72,9 +72,7 @@ object ReplicatedEventSourcingSpec { replicaId: String, probe: Option[ActorRef[EventAndContext]] = None): Behavior[Command] = ReplicatedEventSourcing.withSharedJournal( - "ReplicatedEventSourcingSpec", - entityId, - ReplicaId(replicaId), + ReplicationId("ReplicatedEventSourcingSpec", entityId, ReplicaId(replicaId)), AllReplicas, PersistenceTestKitReadJournal.Identifier)(replicationContext => eventSourcedBehavior(replicationContext, probe)) 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 index fdfc44ab82..626c07daf1 100644 --- 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 @@ -42,7 +42,10 @@ object ReplicatedEventSourcingTaggingSpec { replica: ReplicaId, allReplicas: Set[ReplicaId]): EventSourcedBehavior[Command, String, State] = { // #tagging - ReplicatedEventSourcing.withSharedJournal("TaggingSpec", entityId, replica, allReplicas, queryPluginId)( + ReplicatedEventSourcing.withSharedJournal( + ReplicationId("TaggingSpec", entityId, replica), + allReplicas, + queryPluginId)( replicationContext => EventSourcedBehavior[Command, String, State]( replicationContext.persistenceId, 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 index 5857a26fbc..79503a5d3b 100644 --- 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 @@ -29,9 +29,7 @@ object ReplicationIllegalAccessSpec { def apply(entityId: String, replica: ReplicaId): Behavior[Command] = { ReplicatedEventSourcing.withSharedJournal( - "IllegalAccessSpec", - entityId, - replica, + ReplicationId("IllegalAccessSpec", entityId, replica), AllReplicas, PersistenceTestKitReadJournal.Identifier)( replicationContext => @@ -91,9 +89,7 @@ class ReplicationIllegalAccessSpec "detect illegal access in the factory" in { val exception = intercept[UnsupportedOperationException] { ReplicatedEventSourcing.withSharedJournal( - "IllegalAccessSpec", - "id2", - R1, + ReplicationId("IllegalAccessSpec", "id2", R1), AllReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => replicationContext.origin 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 index b22514af28..3fe5722b95 100644 --- 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 @@ -20,7 +20,7 @@ object ReplicationSnapshotSpec { import ReplicatedEventSourcingSpec._ - val EntityType = "SpapsnotSpec" + val EntityType = "SnapshotSpec" def behaviorWithSnapshotting(entityId: String, replicaId: ReplicaId): Behavior[Command] = behaviorWithSnapshotting(entityId, replicaId, None) @@ -36,9 +36,7 @@ object ReplicationSnapshotSpec { replicaId: ReplicaId, probe: Option[ActorRef[EventAndContext]]): Behavior[Command] = { ReplicatedEventSourcing.withSharedJournal( - EntityType, - entityId, - replicaId, + ReplicationId(EntityType, entityId, replicaId), AllReplicas, PersistenceTestKitReadJournal.Identifier)(replicationContext => eventSourcedBehavior(replicationContext, probe).snapshotWhen((_, _, sequenceNr) => sequenceNr % 2 == 0)) @@ -67,8 +65,8 @@ class ReplicationSnapshotSpec "ReplicatedEventSourcing" should { "recover state from snapshots" in { val entityId = nextEntityId - val persistenceIdR1 = s"$entityId|R1" - val persistenceIdR2 = s"$entityId|R2" + val persistenceIdR1 = s"$EntityType|$entityId|R1" + val persistenceIdR2 = s"$EntityType|$entityId|R2" val probe = createTestProbe[Done]() val r2EventProbe = createTestProbe[EventAndContext]() @@ -84,7 +82,7 @@ class ReplicationSnapshotSpec snapshotTestKit.expectNextPersisted(persistenceIdR2, State(List("r1 2", "r1 1"))) r2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedId(EntityType, entityId, R1), + ReplicationId(EntityType, entityId, R1).persistenceId, 1L, "two-again", System.currentTimeMillis(), @@ -98,7 +96,7 @@ class ReplicationSnapshotSpec { val r2 = spawn(behaviorWithSnapshotting(entityId, R2, r2EventProbe.ref)) r2.asInstanceOf[ActorRef[Any]] ! internal.PublishedEventImpl( - PersistenceId.replicatedId(EntityType, entityId, R1), + ReplicationId(EntityType, entityId, R1).persistenceId, 1L, "two-again", System.currentTimeMillis(), @@ -109,8 +107,6 @@ class ReplicationSnapshotSpec 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 index 16f6eeb6cf..aeecc51881 100644 --- 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 @@ -7,6 +7,7 @@ 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 } @@ -29,9 +30,7 @@ object CounterSpec { eventProbe: Option[ActorRef[Counter.Updated]] = None) = Behaviors.setup[PlainCounter.Command] { context => ReplicatedEventSourcing.withSharedJournal( - "CounterSpec", - entityId, - replicaId, + ReplicationId("CounterSpec", entityId, replicaId), AllReplicas, PersistenceTestKitReadJournal.Identifier) { ctx => EventSourcedBehavior[PlainCounter.Command, Counter.Updated, Counter]( 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 index 07a2091ca8..1de011f629 100644 --- 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 @@ -6,6 +6,7 @@ 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 @@ -27,9 +28,7 @@ object LwwSpec { def apply(entityId: String, replica: ReplicaId): Behavior[Command] = { ReplicatedEventSourcing.withSharedJournal( - "LwwRegistrySpec", - entityId, - replica, + ReplicationId("LwwRegistrySpec", entityId, replica), AllReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => EventSourcedBehavior[Command, Event, Registry]( 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 index 915bb8b64c..95a2f383a9 100644 --- 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 @@ -10,6 +10,7 @@ import akka.persistence.typed.scaladsl.{ Effect, EventSourcedBehavior, Replicate 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 @@ -28,9 +29,7 @@ object ORSetSpec { def apply(entityId: String, replica: ReplicaId): Behavior[ORSetEntity.Command] = { ReplicatedEventSourcing.withSharedJournal( - "ORSetSpec", - entityId, - replica, + ReplicationId("ORSetSpec", entityId, replica), AllReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => EventSourcedBehavior[Command, ORSet.DeltaOp, ORSet[String]]( 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 index 2d59b5ce97..fcae4417a1 100644 --- 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 @@ -23,6 +23,7 @@ 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 @@ -142,12 +143,13 @@ object ReplicatedAuctionExampleSpec { responsibleForClosing: Boolean, allReplicas: Set[ReplicaId]): Behavior[Command] = Behaviors.setup[Command] { ctx => Behaviors.withTimers { timers => - ReplicatedEventSourcing - .withSharedJournal("auction", name, replica, allReplicas, PersistenceTestKitReadJournal.Identifier) { - replicationCtx => - new AuctionEntity(ctx, replicationCtx, timers, closingAt, responsibleForClosing, allReplicas) - .behavior(initialBid) - } + ReplicatedEventSourcing.withSharedJournal( + ReplicationId("auction", name, replica), + allReplicas, + PersistenceTestKitReadJournal.Identifier) { replicationCtx => + new AuctionEntity(ctx, replicationCtx, timers, closingAt, responsibleForClosing, allReplicas) + .behavior(initialBid) + } } } 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 index 4d1d9592d6..53d3949e0c 100644 --- 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 @@ -16,6 +16,7 @@ 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 @@ -52,9 +53,7 @@ object ReplicatedBlogExampleSpec { def apply(entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId]): Behavior[Command] = { Behaviors.setup[Command] { ctx => ReplicatedEventSourcing.withSharedJournal( - "blog", - entityId, - replicaId, + ReplicationId("blog", entityId, replicaId), allReplicaIds, PersistenceTestKitReadJournal.Identifier) { replicationContext => EventSourcedBehavior[Command, Event, BlogState]( 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 index e98b7418c6..f6edecc687 100644 --- 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 @@ -5,6 +5,7 @@ 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 @@ -24,15 +25,19 @@ object ReplicatedEventSourcingCompileOnlySpec { trait Event //#factory-shared - ReplicatedEventSourcing.withSharedJournal("entityTypeHint", "entityId", DCA, AllReplicas, queryPluginId) { context => + ReplicatedEventSourcing.withSharedJournal( + ReplicationId("entityTypeHint", "entityId", DCA), + AllReplicas, + queryPluginId) { context => EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) } //#factory-shared //#factory - ReplicatedEventSourcing("entityTypeHint", "entityId", DCA, Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { - context => - EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) + ReplicatedEventSourcing( + 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 index ff9c14e8dd..c67a0bc203 100644 --- 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 @@ -5,7 +5,6 @@ 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 @@ -13,6 +12,7 @@ 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 @@ -29,9 +29,7 @@ object ReplicatedMovieWatchListExampleSpec { def apply(entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId]): Behavior[Command] = { ReplicatedEventSourcing.withSharedJournal( - "movies", - entityId, - replicaId, + ReplicationId("movies", entityId, replicaId), allReplicaIds, PersistenceTestKitReadJournal.Identifier) { replicationContext => EventSourcedBehavior[Command, ORSet.DeltaOp, ORSet[String]]( 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 index 8a311df9a6..a8ed48b856 100644 --- 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 @@ -8,7 +8,6 @@ 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 @@ -16,6 +15,7 @@ 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 @@ -42,9 +42,7 @@ object ReplicatedShoppingCartExampleSpec { def apply(entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId]): Behavior[Command] = { ReplicatedEventSourcing.withSharedJournal( - "blog", - entityId, - replicaId, + ReplicationId("blog", entityId, replicaId), allReplicaIds, PersistenceTestKitReadJournal.Identifier) { replicationContext => EventSourcedBehavior[Command, Event, State]( 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 47b014a9c4..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 @@ -3,7 +3,6 @@ */ package akka.persistence.typed -import akka.annotation.ApiMayChange object PersistenceId { @@ -125,27 +124,6 @@ object PersistenceId { */ def ofUniqueId(id: String): PersistenceId = new PersistenceId(id) - - /** - * Constructs a [[PersistenceId]] from the given `entityTypeHint`, `entityId` and `replicaId` by - * concatenating them with the `|` separator. - */ - @ApiMayChange - def replicatedId(entityTypeHint: String, entityId: String, replicaId: ReplicaId): PersistenceId = { - if (entityTypeHint.contains(DefaultSeparator)) - throw new IllegalArgumentException( - s"entityTypeHint [$entityTypeHint] contains [$DefaultSeparator] which is a reserved character") - - if (entityId.contains(DefaultSeparator)) - throw new IllegalArgumentException( - s"entityId [$entityId] contains [$DefaultSeparator] which is a reserved character") - - if (replicaId.id.contains(DefaultSeparator)) - throw new IllegalArgumentException( - s"replicaId [${replicaId.id}] contains [$DefaultSeparator] which is a reserved character") - - new PersistenceId(entityTypeHint + DefaultSeparator + entityId + DefaultSeparator + replicaId.id) - } } /** 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..a8e9a392ba --- /dev/null +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicationId.scala @@ -0,0 +1,45 @@ +/* + * 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) +} 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 09503e67f2..6f0fb58463 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 @@ -257,7 +257,8 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( override private[akka] def withReplication( context: ReplicationContextImpl): EventSourcedBehavior[Command, Event, State] = { - copy(replication = Some(ReplicationSetup(context.replicaId, context.replicasAndQueryPlugins, context))) + copy( + replication = Some(ReplicationSetup(context.replicationId.replicaId, context.replicasAndQueryPlugins, context))) } } 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 index 76639ab35f..a118c27dea 100644 --- 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 @@ -7,6 +7,7 @@ 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._ @@ -16,14 +17,11 @@ import akka.util.ccompat.JavaConverters._ */ @InternalApi private[akka] final class ReplicationContextImpl( - val entityTypeHint: String, - val entityId: String, - val replicaId: ReplicaId, + 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 @@ -65,7 +63,7 @@ private[akka] final class ReplicationContextImpl( _concurrent } - override def persistenceId: PersistenceId = PersistenceId.replicatedId(entityTypeHint, entityId, replicaId) + override def persistenceId: PersistenceId = replicationId.persistenceId override def currentTimeMillis(): Long = { WallClock.AlwaysIncreasingClock.currentTimeMillis() 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 3af686332b..9edd3c3696 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 @@ -35,6 +35,7 @@ import akka.persistence.journal.Tagged 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, @@ -128,8 +129,8 @@ private[akka] object Running { val query = PersistenceQuery(system) replicationSetup.allReplicas.foldLeft(state) { (state, replicaId) => if (replicaId != replicationSetup.replicaId) { - val pid = PersistenceId.replicatedId( - replicationSetup.replicationContext.entityTypeHint, + val pid = ReplicationId( + replicationSetup.replicationContext.replicationId.typeName, replicationSetup.replicationContext.entityId, replicaId) val queryPluginId = replicationSetup.allReplicasAndQueryPlugins(replicaId) @@ -147,7 +148,7 @@ private[akka] object Running { Source.futureSource { setup.context.self.ask[Long](replyTo => GetSeenSequenceNr(replicaId, replyTo)).map { seqNr => replication - .eventsByPersistenceId(pid.id, seqNr + 1, Long.MaxValue) + .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 => 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 index 4bf71f8590..3f3e6c049c 100644 --- 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 @@ -11,8 +11,8 @@ 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._ /** @@ -23,6 +23,8 @@ import akka.util.ccompat.JavaConverters._ @DoNotInherit trait ReplicationContext { + def replicationId: ReplicationId + /** * @return The replica id of this replicated event sourced actor */ @@ -81,25 +83,15 @@ object ReplicatedEventSourcing { * can be used for each replica. * The events from other replicas are read using PersistentQuery. * - * @param entityType 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 replicaId The unique identity for this entity. The underlying persistence id will include the replica. - * @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 withSharedJournal[Command, Event, State]( - entityType: String, - entityId: String, - replicaId: ReplicaId, + replicationId: ReplicationId, allReplicaIds: JSet[ReplicaId], queryPluginId: String, behaviorFactory: JFunction[ReplicationContext, EventSourcedBehavior[Command, Event, State]]) : EventSourcedBehavior[Command, Event, State] = - create( - entityType, - entityId, - replicaId, - allReplicaIds.asScala.map(id => id -> queryPluginId).toMap.asJava, - behaviorFactory) + create(replicationId, allReplicaIds.asScala.map(id => id -> queryPluginId).toMap.asJava, behaviorFactory) /** * Initialize a replicated event sourced behavior. @@ -113,19 +105,15 @@ object ReplicatedEventSourcing { * 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 entityType 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 replicaId The unique identity for this entity. The underlying persistence id will include the replica. * @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 create[Command, Event, State]( - entityType: String, - entityId: String, - replicaId: ReplicaId, + replicationId: ReplicationId, allReplicasAndQueryPlugins: JMap[ReplicaId, String], eventSourcedBehaviorFactory: JFunction[ReplicationContext, EventSourcedBehavior[Command, Event, State]]) : EventSourcedBehavior[Command, Event, State] = { - val context = new ReplicationContextImpl(entityType, entityId, replicaId, allReplicasAndQueryPlugins.asScala.toMap) + val context = new ReplicationContextImpl(replicationId, allReplicasAndQueryPlugins.asScala.toMap) eventSourcedBehaviorFactory(context) } 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 index 6d9e6a533f..d5ceaff944 100644 --- 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 @@ -7,6 +7,7 @@ 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 /** @@ -17,6 +18,8 @@ import akka.persistence.typed.internal.ReplicationContextImpl @DoNotInherit trait ReplicationContext { + def replicationId: ReplicationId + /** * @return The unique id of this replica, including the replica id */ @@ -25,7 +28,7 @@ trait ReplicationContext { /** * @return The replica id of this replicated event sourced actor */ - def replicaId: ReplicaId + def replicaId: ReplicaId = replicationId.replicaId /** * @return The ids of all replicas of this replicated event sourced actor @@ -35,7 +38,7 @@ trait ReplicationContext { /** * @return The entity id of this replicated event sourced actor (not including the replica id) */ - def entityId: String + def entityId: String = replicationId.entityId /** * Must only be called from the event handler @@ -76,22 +79,16 @@ object ReplicatedEventSourcing { * can be used for each replica. * The events from other replicas are read using PersistentQuery. * - * @param entityType 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. * @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 withSharedJournal[Command, Event, State]( - entityType: String, - entityId: String, - replicaId: ReplicaId, + replicationId: ReplicationId, allReplicaIds: Set[ReplicaId], queryPluginId: String)( eventSourcedBehaviorFactory: ReplicationContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = - apply(entityType, entityId, replicaId, allReplicaIds.map(id => id -> queryPluginId).toMap)( - eventSourcedBehaviorFactory) + apply(replicationId, allReplicaIds.map(id => id -> queryPluginId).toMap)(eventSourcedBehaviorFactory) /** * Initialize a replicated event sourced behavior. @@ -104,21 +101,13 @@ object ReplicatedEventSourcing { * 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 entityType 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. * @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 apply[Command, Event, State]( - entityType: String, - entityId: String, - replicaId: ReplicaId, - allReplicasAndQueryPlugins: Map[ReplicaId, String])( + def apply[Command, Event, State](replicationId: ReplicationId, allReplicasAndQueryPlugins: Map[ReplicaId, String])( eventSourcedBehaviorFactory: ReplicationContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = { - val context = new ReplicationContextImpl(entityType, entityId, replicaId, allReplicasAndQueryPlugins) + val context = new ReplicationContextImpl(replicationId, allReplicasAndQueryPlugins) eventSourcedBehaviorFactory(context).withReplication(context) } From 260276fd90dce7ffc1581b9238a0e380154c941c Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Fri, 14 Aug 2020 12:26:54 +0100 Subject: [PATCH 43/50] Turn on direct replication by default (#29493) --- .../cluster/sharding/typed/ReplicatedEntityProvider.scala | 2 +- akka-docs/src/main/paradox/typed/replicated-eventsourcing.md | 2 +- .../persistence/typed/internal/EventSourcedBehaviorImpl.scala | 2 +- .../typed/javadsl/ReplicatedEventSourcedBehavior.scala | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) 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 index 2a73361093..6773b6ef65 100644 --- 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 @@ -53,7 +53,7 @@ object ReplicatedEntityProvider { val typeKey = EntityTypeKey[M](s"$typeName${Separator}${replicaId.id}") (settingsPerReplicaFactory(typeKey, replicaId), typeName) - }.toVector, directReplication = false) + }.toVector, directReplication = true) } } diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index 72361f204b..96760fd284 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -314,7 +314,7 @@ Scala Java : @@snip [ReplicatedShardingTest.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/ReplicatedShardingCompileOnlySpec.java) { #bootstrap } -This will run a single instance of sharding and the replicas will be differentiated by having the replica id in the sharding entity id. +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. 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 6f0fb58463..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 @@ -98,7 +98,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State]( supervisionStrategy: SupervisorStrategy = SupervisorStrategy.stop, override val signalHandler: PartialFunction[(State, Signal), Unit] = PartialFunction.empty, replication: Option[ReplicationSetup] = None, - publishEvents: Boolean = false) + publishEvents: Boolean = true) extends EventSourcedBehavior[Command, Event, State] { import EventSourcedBehaviorImpl.WriterIdentity 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 index f660e813d6..06f6288330 100644 --- 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 @@ -25,10 +25,10 @@ abstract class ReplicatedEventSourcedBehavior[Command, Event, State]( def this(replicationContext: ReplicationContext) = this(replicationContext, Optional.empty()) /** - * Override and return true to publish events to the system event stream as + * 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 = false + def withEventPublishing: Boolean = true protected def getReplicationContext(): ReplicationContext = replicationContext From eae102acb108bfe659009c5d1fcaf8f819aaf335 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Fri, 14 Aug 2020 15:02:38 +0100 Subject: [PATCH 44/50] Make ShardingDirectReplication private (#29492) * Make ShardingDirectReplication private And provider user API to provide ReplicaId so that local messages don't need to be forwarded to sharding to just be dropped * Update docs to say direct replication is on my default * Doh --- .../typed/ReplicatedShardingExtension.scala | 11 +++++ .../typed/ShardingDirectReplication.scala | 46 ++----------------- .../ReplicatedShardingExtensionImpl.scala | 12 ++++- ...licatedShardingDirectReplicationSpec.scala | 2 +- .../paradox/typed/replicated-eventsourcing.md | 13 ++---- 5 files changed, 32 insertions(+), 52 deletions(-) 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 index fb206cb73f..37654f6bb6 100644 --- 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 @@ -46,6 +46,17 @@ trait ReplicatedShardingExtension extends Extension { * Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ReplicatedSharding]] */ def init[M, E](settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] + + /** + * 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 + * @tparam E The type of envelope used for routing messages to actors, the same for all replicas + * + * Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ReplicatedSharding]] + */ + def init[M, E](thisReplica: ReplicaId, settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] } /** 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 index 7904aa6cc9..47e228ecb7 100644 --- 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 @@ -9,15 +9,14 @@ 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.ApiMayChange import akka.annotation.DoNotInherit import akka.annotation.InternalApi import akka.persistence.typed.PublishedEvent import akka.persistence.typed.ReplicaId -import akka.util.ccompat.JavaConverters._ - /** + * 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. * @@ -35,8 +34,8 @@ import akka.util.ccompat.JavaConverters._ * 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. */ -@ApiMayChange -object ShardingDirectReplication { +@InternalApi +private[akka] object ShardingDirectReplication { /** * Not for user extension @@ -52,42 +51,7 @@ object ShardingDirectReplication { private final case class WrappedPublishedEvent(publishedEvent: PublishedEvent) extends Command - /** - * Java API: - * Factory for when the self replica id is unknown (or multiple) - * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system - */ - def create[T](replicaShardingProxies: java.util.Map[ReplicaId, ActorRef[T]]): Behavior[Command] = - apply(None, replicaShardingProxies.asScala.toMap) - - /** - * Java API: - * @param selfReplica The replica id of the replica that runs on this node - * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system - */ - def create[T]( - selfReplica: ReplicaId, - replicaShardingProxies: java.util.Map[ReplicaId, ActorRef[T]]): Behavior[Command] = - apply(Some(selfReplica), replicaShardingProxies.asScala.toMap) - - /** - * Scala API: - * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system - */ - def apply[T](replicaShardingProxies: Map[ReplicaId, ActorRef[T]]): Behavior[Command] = - apply(None, replicaShardingProxies) - - /** - * Scala API: - * @param selfReplica The replica id of the replica that runs on this node - * @param replicaShardingProxies A replica id to sharding proxy mapping for each replica in the system - */ - def apply[T](selfReplica: ReplicaId, replicaShardingProxies: Map[ReplicaId, ActorRef[T]]): Behavior[Command] = - apply(Some(selfReplica), replicaShardingProxies) - - private def apply[T]( - selfReplica: Option[ReplicaId], - replicaShardingProxies: Map[ReplicaId, ActorRef[T]]): Behavior[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", 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 index f8f3e2383f..a9f218a0a3 100644 --- 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 @@ -34,7 +34,15 @@ private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_] private val logger = LoggerFactory.getLogger(getClass) - override def init[M, E](settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] = { + override def init[M, E](settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] = + initInternal(None, settings) + + override def init[M, E](thisReplica: ReplicaId, settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] = + initInternal(Some(thisReplica), settings) + + private def initInternal[M, E]( + thisReplica: Option[ReplicaId], + settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] = { val sharding = ClusterSharding(system) val initializedReplicas = settings.replicas.map { case (replicaSettings, typeName) => @@ -57,7 +65,7 @@ private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_] if (settings.directReplication) { logger.infoN("Starting Replicated Event Sourcing Direct Replication") system.systemActorOf( - ShardingDirectReplication(replicaToRegionOrProxy), + ShardingDirectReplication(thisReplica, replicaToRegionOrProxy), s"directReplication-${counter.incrementAndGet()}") } 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 index 550ca2e296..ae19cb35ad 100644 --- 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 @@ -26,7 +26,7 @@ class ReplicatedShardingDirectReplicationSpec extends ScalaTestWithActorTestKit val replicationActor = spawn( ShardingDirectReplication( - typed.ReplicaId("ReplicaA"), + Some(typed.ReplicaId("ReplicaA")), replicaShardingProxies = Map( ReplicaId("ReplicaA") -> replicaAProbe.ref, ReplicaId("ReplicaB") -> replicaBProbe.ref, diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index 96760fd284..8a9413d935 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -384,18 +384,15 @@ with a single stream of tagged events from all replicas without duplicates. ## Direct Replication of Events -Normally an event has to be written in the journal and then picked up by the trailing read journal in the other replicas. -As an optimization the replicated events can be published across the Akka cluster to the replicas. The read side -query is still needed as delivery is not guaranteed, but can be configured to poll the database less often since most +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. -To enable this feature you first need to enable event publishing on the @scala[`EventSourcedBehavior`]@java[`ReplicatedEventSourcedBehavior`] with `withEventPublishing` -and then enable direct replication through `withDirectReplication(true)` on @apidoc[ReplicatedEntityProvider] (if not using - replicated sharding the replication can be run standalone by starting the @apidoc[ShardingDirectReplication] actor). +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, -the @apidoc[ShardingDirectReplication] actor subscribes to these events and forwards them to the replicas allowing them -to fast forward the stream of events for the origin replica. (With additional potential future support in journals for fast forwarding [#29311](https://github.com/akka/akka/issues/29311)). ## Hot Standby From e7e2757d1adb8398207f301b2fc18f01381c0c76 Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Fri, 14 Aug 2020 17:13:35 +0100 Subject: [PATCH 45/50] Test coverage and fixes for direct replication (#29491) * Test coverage and fixes for direct replication * Serializer test and review feedback * use testkit serialization testkit --- .../typed/ShardingDirectReplication.scala | 7 +- .../typed/ReplicatedShardingSpec.scala | 190 ++ .../typed/ReplicatedShardingSpec.scala | 1 + .../ReplicatedEventSourcing.java | 2322 ++++++++++++++++- .../protobuf/ReplicatedEventSourcing.proto | 13 + .../src/main/resources/reference.conf | 1 + .../persistence/typed/ReplicationId.scala | 6 + .../persistence/typed/internal/Running.scala | 31 +- .../ReplicatedEventSourcingSerializer.scala | 60 +- ...eplicatedEventSourcingSerializerSpec.scala | 50 + 10 files changed, 2666 insertions(+), 15 deletions(-) create mode 100644 akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala create mode 100644 akka-persistence-typed/src/test/scala/akka/persistence/typed/serialization/ReplicatedEventSourcingSerializerSpec.scala 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 index 47e228ecb7..1b87757542 100644 --- 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 @@ -13,6 +13,7 @@ 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 @@ -67,9 +68,11 @@ private[akka] object ShardingDirectReplication { event.sequenceNumber) replicaShardingProxies.foreach { case (replica, proxy) => - val envelopedEvent = ShardingEnvelope(event.persistenceId.id, event) - if (!selfReplica.contains(replica)) + 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) => 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..548b9945be --- /dev/null +++ b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala @@ -0,0 +1,190 @@ +/* + * 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.withSharedJournal(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, ShardingEnvelope[Command]] = { + ReplicatedEntityProvider[Command, ShardingEnvelope[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, ShardingEnvelope[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/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala b/akka-cluster-sharding-typed/src/test/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala index af33673f36..93b4d6fa8e 100644 --- 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 @@ -181,6 +181,7 @@ object ProxyActor { 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) => 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 index 2bc39615d6..c1fd5de3d4 100644 --- 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 @@ -10004,6 +10004,2293 @@ public final class ReplicatedEventSourcing { } } + 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 @@ -10044,6 +12331,14 @@ public final class ReplicatedEventSourcing { 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; @@ -10074,9 +12369,15 @@ public final class ReplicatedEventSourcing { + "\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\014ORSetDeltaOp\022\007\n\003Add\020\000\022\n\n\006Remove\020" - + "\001\022\010\n\004Full\020\002B(\n$akka.persistence.typed.se" - + "rializationH\001" + + " \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( @@ -10164,6 +12465,21 @@ public final class ReplicatedEventSourcing { 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(); } diff --git a/akka-persistence-typed/src/main/protobuf/ReplicatedEventSourcing.proto b/akka-persistence-typed/src/main/protobuf/ReplicatedEventSourcing.proto index ab6f433448..a5f8e734ef 100644 --- a/akka-persistence-typed/src/main/protobuf/ReplicatedEventSourcing.proto +++ b/akka-persistence-typed/src/main/protobuf/ReplicatedEventSourcing.proto @@ -63,4 +63,17 @@ message ReplicatedSnapshotMetadata { } 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 390064ee69..afc0976127 100644 --- a/akka-persistence-typed/src/main/resources/reference.conf +++ b/akka-persistence-typed/src/main/resources/reference.conf @@ -12,6 +12,7 @@ akka.actor { "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 } } 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 index a8e9a392ba..ed10be3dee 100644 --- a/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicationId.scala +++ b/akka-persistence-typed/src/main/scala/akka/persistence/typed/ReplicationId.scala @@ -42,4 +42,10 @@ final class ReplicationId(val typeName: String, val entityId: String, val replic 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/internal/Running.scala b/akka-persistence-typed/src/main/scala/akka/persistence/typed/internal/Running.scala index 9edd3c3696..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 @@ -368,13 +368,14 @@ private[akka] object Running { } this } else { - if (log.isTraceEnabled) + 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)) @@ -442,6 +443,7 @@ private[akka] object Running { state, numberOfEvents = 1, shouldSnapshotAfterPersist, + shouldPublish = false, Nil) } @@ -489,8 +491,15 @@ private[akka] object Running { } val shouldSnapshotAfterPersist = setup.shouldSnapshot(newState2.state, event, newState2.seqNr) - (persistingEvents(newState2, state, numberOfEvents = 1, shouldSnapshotAfterPersist, sideEffects), false) - + ( + persistingEvents( + newState2, + state, + numberOfEvents = 1, + shouldSnapshotAfterPersist, + shouldPublish = true, + sideEffects), + false) } finally { setup.replication.foreach(_.clearContext()) } @@ -545,7 +554,15 @@ private[akka] object Running { val newState2 = internalPersistAll(setup.context, cmd, currentState, eventsToPersist.reverse) - (persistingEvents(newState2, state, events.size, shouldSnapshotAfterPersist, sideEffects), false) + ( + persistingEvents( + newState2, + state, + events.size, + shouldSnapshotAfterPersist, + shouldPublish = true, + sideEffects = sideEffects), + false) } finally { setup.replication.foreach(_.clearContext()) } @@ -614,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 */ @@ -625,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) @@ -691,7 +710,7 @@ private[akka] object Running { onWriteSuccess(setup.context, p) - if (setup.publishEvents) { + if (setup.publishEvents && shouldPublish) { val meta = setup.replication.map(replication => new ReplicatedPublishedEventMetaData(replication.replicaId, state.version)) context.system.eventStream ! EventStream.Publish( 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 index 5c4b4dc25d..556244179d 100644 --- 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 @@ -10,10 +10,13 @@ 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 @@ -77,6 +80,8 @@ import scala.collection.immutable.TreeMap 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 @@ -91,12 +96,14 @@ import scala.collection.immutable.TreeMap 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 => replicatedEventMetadataToProtoByteArray(m) + case m: ReplicatedEventMetadata => replicatedEventMetadataToProto(m).toByteArray case m: ReplicatedSnapshotMetadata => replicatedSnapshotMetadataToByteArray(m) case m: VersionVector => versionVectorToProto(m).toByteArray @@ -110,6 +117,8 @@ import scala.collection.immutable.TreeMap 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}") } @@ -130,11 +139,50 @@ import scala.collection.immutable.TreeMap 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)) @@ -217,7 +265,7 @@ import scala.collection.immutable.TreeMap b.build() } - def replicatedEventMetadataToProtoByteArray(rem: ReplicatedEventMetadata): Array[Byte] = { + def replicatedEventMetadataToProto(rem: ReplicatedEventMetadata): ReplicatedEventSourcing.ReplicatedEventMetadata = { ReplicatedEventSourcing.ReplicatedEventMetadata .newBuilder() .setOriginSequenceNr(rem.originSequenceNr) @@ -225,7 +273,6 @@ import scala.collection.immutable.TreeMap .setOriginReplica(rem.originReplica.id) .setVersionVector(versionVectorToProto(rem.version)) .build() - .toByteArray } def replicatedSnapshotMetadataToByteArray(rsm: ReplicatedSnapshotMetadata): Array[Byte] = { @@ -332,6 +379,10 @@ import scala.collection.immutable.TreeMap 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, @@ -340,7 +391,8 @@ import scala.collection.immutable.TreeMap } def replicatedSnapshotMetadataFromBinary(bytes: Array[Byte]): ReplicatedSnapshotMetadata = { - val parsed = ReplicatedEventSourcing.ReplicatedSnapshotMetadata.parseFrom(bytes) + 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/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) + } + } + +} From 9ba9571b4a2992c8ad12de37d72588d2de192b4a Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Mon, 17 Aug 2020 08:29:12 +0100 Subject: [PATCH 46/50] Ignore java warnins in persistence typed and mima Warnings are from protobuf --- .../29217-replicated-event-sourcing.excludes | 0 .../29217-active-active-event-sourcing.excludes | 5 +++++ project/AkkaDisciplinePlugin.scala | 2 ++ 3 files changed, 7 insertions(+) rename akka-persistence-typed/src/main/mima-filters/{2.6.7.backwards.excludes => 2.6.8.backwards.excludes}/29217-replicated-event-sourcing.excludes (100%) rename akka-persistence/src/main/mima-filters/{2.6.7.backwards.excludes => 2.6.8.backwards.excludes}/29217-active-active-event-sourcing.excludes (55%) diff --git a/akka-persistence-typed/src/main/mima-filters/2.6.7.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 similarity index 100% rename from akka-persistence-typed/src/main/mima-filters/2.6.7.backwards.excludes/29217-replicated-event-sourcing.excludes rename to akka-persistence-typed/src/main/mima-filters/2.6.8.backwards.excludes/29217-replicated-event-sourcing.excludes diff --git a/akka-persistence/src/main/mima-filters/2.6.7.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 similarity index 55% rename from akka-persistence/src/main/mima-filters/2.6.7.backwards.excludes/29217-active-active-event-sourcing.excludes rename to akka-persistence/src/main/mima-filters/2.6.8.backwards.excludes/29217-active-active-event-sourcing.excludes index 3e13002376..4ac25cf0b7 100644 --- a/akka-persistence/src/main/mima-filters/2.6.7.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 @@ -9,3 +9,8 @@ ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.persistence.Persisten # 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/project/AkkaDisciplinePlugin.scala b/project/AkkaDisciplinePlugin.scala index 9e3a9e1ce9..a365c25d4c 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( From 90e09549e858eaeeae148b3f3d97290b3254646f Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Mon, 17 Aug 2020 10:04:43 +0100 Subject: [PATCH 47/50] 2.13 compilation --- .../sharding/typed/ReplicatedShardingSpec.scala | 11 ++++++----- .../persistence/testkit/internal/TestKitStorage.scala | 2 +- 2 files changed, 7 insertions(+), 6 deletions(-) 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 index 93b4d6fa8e..51908fe605 100644 --- 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 @@ -35,7 +35,8 @@ 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 @@ -247,12 +248,12 @@ abstract class ReplicatedShardingSpec(replicationType: ReplicationType, configA: Cluster(system2).manager ! Join(Cluster(system).selfMember.address) eventually { - Cluster(system).state.members.size should ===(2) - Cluster(system).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up)) + 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.size should ===(2) - Cluster(system2).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up)) + Cluster(system2).state.members.unsorted.size should ===(2) + Cluster(system2).state.members.unsorted.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up)) } } 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 47c6ddc7fb..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,7 +8,7 @@ import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicReference import scala.collection.immutable -import scala.collection.JavaConverters._ +import akka.util.ccompat.JavaConverters._ import akka.annotation.InternalApi import akka.persistence.testkit.ProcessingPolicy From d1114495dd2399a6e5989bd1002018cefb7b278c Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Mon, 24 Aug 2020 10:52:28 +0100 Subject: [PATCH 48/50] Feedback from creating replicated entity sample (#29510) * Fix javadsl and remove shard regions from ReplicatedSharding * Simplyfy sharding API for replicated event sourcing As the ShardRegion access has been removed then we will only initially support Entity's with ShardingEnvelope meaning we can remove the type param. Also provide convenience constructors for running a replica on a role and a replica in each DC * Compile * Review feedback * feedback --- .../typed/ReplicatedEntityProvider.scala | 110 +++++++++++++++--- .../typed/ReplicatedShardingExtension.scala | 31 +---- .../typed/internal/ClusterShardingImpl.scala | 5 + .../ReplicatedShardingExtensionImpl.scala | 24 ++-- .../internal/testkit/TestEntityRefImpl.scala | 3 + .../typed/javadsl/ClusterSharding.scala | 2 +- .../typed/scaladsl/ClusterSharding.scala | 5 + .../typed/ReplicatedShardingSpec.scala | 2 +- .../typed/ReplicatedShardingTest.java | 59 +++++----- .../ReplicatedShardingCompileOnlySpec.java | 12 +- .../typed/ReplicatedShardingSpec.scala | 10 +- .../ReplicatedShardingCompileOnlySpec.scala | 39 +++---- .../typed/ReplicatedEventSourcingTest.java | 2 +- .../typed/MyReplicatedBehavior.java | 4 +- .../typed/ReplicatedAuctionExampleTest.java | 2 +- .../typed/ReplicatedBlogExample.java | 2 +- .../typed/ReplicatedMovieExample.java | 2 +- .../typed/ReplicatedShoppingCartExample.java | 2 +- .../typed/ReplicatedStringSet.java | 2 +- .../typed/ReplicatedEventPublishingSpec.scala | 2 +- .../typed/ReplicatedEventSourcingSpec.scala | 2 +- .../ReplicatedEventSourcingTaggingSpec.scala | 2 +- .../typed/ReplicationIllegalAccessSpec.scala | 4 +- .../typed/ReplicationSnapshotSpec.scala | 2 +- .../persistence/typed/crdt/CounterSpec.scala | 2 +- .../akka/persistence/typed/crdt/LwwSpec.scala | 2 +- .../persistence/typed/crdt/ORSetSpec.scala | 2 +- .../typed/ReplicatedAuctionExampleSpec.scala | 2 +- .../typed/ReplicatedBlogExampleSpec.scala | 2 +- ...plicatedEventSourcingCompileOnlySpec.scala | 2 +- .../ReplicatedMovieWatchListExampleSpec.scala | 2 +- .../ReplicatedShoppingCartExampleSpec.scala | 2 +- .../javadsl/ReplicatedEventSourcing.scala | 15 ++- .../scaladsl/ReplicatedEventSourcing.scala | 9 +- 34 files changed, 208 insertions(+), 162 deletions(-) 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 index 6773b6ef65..4a02335d49 100644 --- 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 @@ -14,8 +14,10 @@ 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 @@ -24,29 +26,33 @@ 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 - * @tparam E The type for envelopes used for sending `M`s over sharding */ - def create[M, E]( + def create[M]( messageClass: Class[M], typeName: String, allReplicaIds: JSet[ReplicaId], - settingsPerReplicaFactory: akka.japi.function.Function2[JEntityTypeKey[M], ReplicaId, ReplicatedEntity[M, E]]) - : ReplicatedEntityProvider[M, E] = { + settingsPerReplicaFactory: akka.japi.function.Function2[JEntityTypeKey[M], ReplicaId, ReplicatedEntity[M]]) + : ReplicatedEntityProvider[M] = { implicit val classTag: ClassTag[M] = ClassTag(messageClass) - apply[M, E](typeName, allReplicaIds.asScala.toSet)((key, replica) => + 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 - * @tparam E The type for envelopes used for sending `M`s over sharding */ - def apply[M: ClassTag, E](typeName: String, allReplicaIds: Set[ReplicaId])( - settingsPerReplicaFactory: (EntityTypeKey[M], ReplicaId) => ReplicatedEntity[M, E]) - : ReplicatedEntityProvider[M, E] = { + 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") @@ -55,15 +61,86 @@ object ReplicatedEntityProvider { (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 - * @tparam E The type for envelopes used for sending `M`s over sharding */ @ApiMayChange -final class ReplicatedEntityProvider[M, E] private ( - val replicas: immutable.Seq[(ReplicatedEntity[M, E], String)], +final class ReplicatedEntityProvider[M] private ( + val replicas: immutable.Seq[(ReplicatedEntity[M], String)], val directReplication: Boolean) { /** @@ -73,7 +150,7 @@ final class ReplicatedEntityProvider[M, E] private ( * to work. * */ - def withDirectReplication(enabled: Boolean): ReplicatedEntityProvider[M, E] = + def withDirectReplication(enabled: Boolean): ReplicatedEntityProvider[M] = new ReplicatedEntityProvider(replicas, directReplication = enabled) } @@ -87,7 +164,7 @@ object ReplicatedEntity { * [[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, E](replicaId: ReplicaId, entity: JEntity[M, E]): ReplicatedEntity[M, E] = + def create[M](replicaId: ReplicaId, entity: JEntity[M, ShardingEnvelope[M]]): ReplicatedEntity[M] = apply(replicaId, entity.toScala) /** @@ -96,12 +173,13 @@ object ReplicatedEntity { * [[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, E](replicaId: ReplicaId, entity: Entity[M, E]): ReplicatedEntity[M, E] = + 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, E] private (val replicaId: ReplicaId, val entity: Entity[M, E]) +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 index 37654f6bb6..af35f26cc7 100644 --- 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 @@ -14,8 +14,6 @@ import akka.cluster.sharding.typed.scaladsl.EntityRef import akka.persistence.typed.ReplicaId import java.util.{ Map => JMap } -import akka.actor.typed.ActorRef - /** * 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. @@ -41,22 +39,20 @@ 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 - * @tparam E The type of envelope used for routing messages to actors, the same for all replicas * * Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ReplicatedSharding]] */ - def init[M, E](settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] + 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 - * @tparam E The type of envelope used for routing messages to actors, the same for all replicas * * Note, multiple calls on the same node will not start new sharding instances but will return a new instance of [[ReplicatedSharding]] */ - def init[M, E](thisReplica: ReplicaId, settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] + def init[M](thisReplica: ReplicaId, settings: ReplicatedEntityProvider[M]): ReplicatedSharding[M] } /** @@ -66,34 +62,15 @@ trait ReplicatedShardingExtension extends Extension { */ @DoNotInherit @ApiMayChange -trait ReplicatedSharding[M, E] { - - /** - * Scala API: Returns the actor refs for the shard region or proxies of sharding for each replica for user defined - * routing/replica selection. - */ - def shardingRefs: Map[ReplicaId, ActorRef[E]] - - /** - * Java API: Returns the actor refs for the shard region or proxies of sharding for each replica for user defined - * routing/replica selection. - */ - def getShardingRefs: JMap[ReplicaId, ActorRef[E]] +trait ReplicatedSharding[M] { /** * Scala API: Returns the entity ref for each replica for user defined routing/replica selection - * - * 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 [[#shardingRefs]] */ def entityRefsFor(entityId: String): Map[ReplicaId, EntityRef[M]] /** * Java API: Returns the entity ref for each replica for user defined routing/replica selection - * - * 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 [[#getShardingRefs]] */ - def getEntityRefsFor(entityId: String): JMap[ReplicaId, EntityRef[M]] - + def getEntityRefsFor(entityId: String): JMap[ReplicaId, javadsl.EntityRef[M]] } 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 index a9f218a0a3..57ef3965e1 100644 --- 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 @@ -7,7 +7,6 @@ package akka.cluster.sharding.typed.internal import java.util.concurrent.atomic.AtomicLong import java.util.{ Map => JMap } -import akka.actor.typed.ActorRef import akka.actor.typed.ActorSystem import akka.annotation.InternalApi import akka.cluster.sharding.typed.ReplicatedShardingExtension @@ -34,15 +33,15 @@ private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_] private val logger = LoggerFactory.getLogger(getClass) - override def init[M, E](settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] = + override def init[M](settings: ReplicatedEntityProvider[M]): ReplicatedSharding[M] = initInternal(None, settings) - override def init[M, E](thisReplica: ReplicaId, settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] = + override def init[M](thisReplica: ReplicaId, settings: ReplicatedEntityProvider[M]): ReplicatedSharding[M] = initInternal(Some(thisReplica), settings) - private def initInternal[M, E]( + private def initInternal[M]( thisReplica: Option[ReplicaId], - settings: ReplicatedEntityProvider[M, E]): ReplicatedSharding[M, E] = { + settings: ReplicatedEntityProvider[M]): ReplicatedSharding[M] = { val sharding = ClusterSharding(system) val initializedReplicas = settings.replicas.map { case (replicaSettings, typeName) => @@ -72,7 +71,7 @@ private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_] val replicaToTypeKey = initializedReplicas.map { case (typeName, id, typeKey, _, dc) => id -> ((typeKey, dc, typeName)) }.toMap - new ReplicatedShardingImpl(sharding, replicaToRegionOrProxy, replicaToTypeKey) + new ReplicatedShardingImpl(sharding, replicaToTypeKey) } } @@ -80,15 +79,10 @@ private[akka] final class ReplicatedShardingExtensionImpl(system: ActorSystem[_] * INTERNAL API */ @InternalApi -private[akka] final class ReplicatedShardingImpl[M, E]( +private[akka] final class ReplicatedShardingImpl[M]( sharding: ClusterSharding, - shardingPerReplica: Map[ReplicaId, ActorRef[E]], replicaTypeKeys: Map[ReplicaId, (EntityTypeKey[M], Option[DataCenter], String)]) - extends ReplicatedSharding[M, E] { - - // FIXME add test coverage for these - override def shardingRefs: Map[ReplicaId, ActorRef[E]] = shardingPerReplica - override def getShardingRefs: JMap[ReplicaId, ActorRef[E]] = shardingRefs.asJava + extends ReplicatedSharding[M] { override def entityRefsFor(entityId: String): Map[ReplicaId, EntityRef[M]] = replicaTypeKeys.map { @@ -100,7 +94,7 @@ private[akka] final class ReplicatedShardingImpl[M, E]( }) } - override def getEntityRefsFor(entityId: String): JMap[ReplicaId, EntityRef[M]] = - entityRefsFor(entityId).asJava + 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 2211719e48..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 @@ -441,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 9e46aaabdb..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 @@ -500,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 index 548b9945be..156037b46d 100644 --- 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 @@ -70,7 +70,7 @@ object ReplicatedShardingSpec extends MultiNodeConfig { def apply(id: ReplicationId, ctx: ActorContext[Command]): EventSourcedBehavior[Command, String, State] = { // Relies on direct replication as there is no proxy query journal - ReplicatedEventSourcing.withSharedJournal(id, AllReplicas, PersistenceTestKitReadJournal.Identifier) { + ReplicatedEventSourcing.commonJournalConfig(id, AllReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => ctx.log.info("Creating replica {}", replicationContext.replicationId) EventSourcedBehavior[Command, String, State]( 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 index 0f9c105b1d..da813a698b 100644 --- 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 @@ -15,7 +15,7 @@ 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.scaladsl.EntityRef; +import akka.cluster.sharding.typed.javadsl.EntityRef; import akka.cluster.typed.Cluster; import akka.cluster.typed.Join; import akka.persistence.testkit.PersistenceTestKitPlugin; @@ -67,7 +67,7 @@ public class ReplicatedShardingTest extends JUnitSuite { } static Behavior create(ReplicationId replicationId) { - return ReplicatedEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.commonJournalConfig( replicationId, ALL_REPLICAS, PersistenceTestKitReadJournal.Identifier(), @@ -143,44 +143,39 @@ public class ReplicatedShardingTest extends JUnitSuite { Arrays.asList( new ReplicaId("DC-A"), new ReplicaId("DC-B"), new ReplicaId("DC-C")))); - private final ReplicatedSharding< - MyReplicatedStringSet.Command, ShardingEnvelope> - replicatedSharding; + private final ReplicatedSharding replicatedSharding; private ProxyActor(ActorContext context) { super(context); // #bootstrap - ReplicatedEntityProvider< - MyReplicatedStringSet.Command, ShardingEnvelope> - 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()))); + 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< - MyReplicatedStringSet.Command, ShardingEnvelope> - replicatedSharding = extension.init(replicatedEntityProvider); + ReplicatedSharding replicatedSharding = + extension.init(replicatedEntityProvider); // #bootstrap this.replicatedSharding = replicatedSharding; 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 index a65a7337c5..afc488bf64 100644 --- 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 @@ -4,12 +4,11 @@ package jdocs.akka.cluster.sharding.typed; -import akka.actor.typed.ActorRef; 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.scaladsl.EntityRef; +import akka.cluster.sharding.typed.javadsl.EntityRef; import akka.persistence.typed.ReplicaId; import akka.persistence.typed.ReplicationId; @@ -30,7 +29,7 @@ public class ReplicatedShardingCompileOnlySpec { new HashSet<>( Arrays.asList(new ReplicaId("DC-A"), new ReplicaId("DC-B"), new ReplicaId("DC-C")))); - public static ReplicatedEntityProvider> provider() { + public static ReplicatedEntityProvider provider() { // #bootstrap return ReplicatedEntityProvider.create( Command.class, @@ -67,7 +66,7 @@ public class ReplicatedShardingCompileOnlySpec { // #bootstrap-dc } - public static ReplicatedEntityProvider> role() { + public static ReplicatedEntityProvider role() { // #bootstrap-role return ReplicatedEntityProvider.create( Command.class, @@ -90,13 +89,10 @@ public class ReplicatedShardingCompileOnlySpec { // #sending-messages ReplicatedShardingExtension extension = ReplicatedShardingExtension.get(system); - ReplicatedSharding> replicatedSharding = - extension.init(provider()); + ReplicatedSharding replicatedSharding = extension.init(provider()); Map> myEntityId = replicatedSharding.getEntityRefsFor("myEntityId"); - Map>> shardingRefs = - replicatedSharding.getShardingRefs(); // #sending-messages } 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 index 51908fe605..6fd6d0925a 100644 --- 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 @@ -76,7 +76,7 @@ object ReplicatedShardingSpec { case class Texts(texts: Set[String]) extends CborSerializable def apply(replicationId: ReplicationId): Behavior[Command] = - ReplicatedEventSourcing.withSharedJournal( // it isn't really shared as it is in memory + ReplicatedEventSourcing.commonJournalConfig( // it isn't really shared as it is in memory replicationId, AllReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => @@ -97,7 +97,7 @@ object ReplicatedShardingSpec { } def provider(replicationType: ReplicationType) = - ReplicatedEntityProvider[MyReplicatedStringSet.Command, ShardingEnvelope[MyReplicatedStringSet.Command]]( + ReplicatedEntityProvider[MyReplicatedStringSet.Command]( // all replicas "StringSet", AllReplicas) { (entityTypeKey, replicaId) => @@ -127,7 +127,7 @@ object ReplicatedShardingSpec { case class Ints(ints: Set[Int]) extends CborSerializable def apply(id: ReplicationId, allReplicas: Set[ReplicaId]): Behavior[Command] = - ReplicatedEventSourcing.withSharedJournal( // it isn't really shared as it is in memory + ReplicatedEventSourcing.commonJournalConfig( // it isn't really shared as it is in memory id, allReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => @@ -148,9 +148,7 @@ object ReplicatedShardingSpec { } def provider(replicationType: ReplicationType) = - ReplicatedEntityProvider[MyReplicatedIntSet.Command, ShardingEnvelope[MyReplicatedIntSet.Command]]( - "IntSet", - AllReplicas) { (entityTypeKey, replicaId) => + ReplicatedEntityProvider[MyReplicatedIntSet.Command]("IntSet", AllReplicas) { (entityTypeKey, replicaId) => val entity = { val e = Entity(entityTypeKey) { entityContext => val replicationId = ReplicationId.fromString(entityContext.entityId) 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 index 45bb443191..3954ea139c 100644 --- 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 @@ -4,14 +4,12 @@ package docs.akka.cluster.sharding.typed -import akka.actor.typed.ActorRef 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.ShardingEnvelope import akka.cluster.sharding.typed.scaladsl.Entity import akka.cluster.sharding.typed.scaladsl.EntityRef import akka.persistence.typed.ReplicaId @@ -30,45 +28,34 @@ object ReplicatedShardingCompileOnlySpec { } //#bootstrap - ReplicatedEntityProvider[Command, ShardingEnvelope[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) - }) + 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[Command, ShardingEnvelope[Command]]( - "MyEntityType", - Set(ReplicaId("DC-A"), ReplicaId("DC-B"))) { (entityTypeKey, replicaId) => - ReplicatedEntity(replicaId, Entity(entityTypeKey) { entityContext => - val replicationId = ReplicationId.fromString(entityContext.entityId) - MyEventSourcedBehavior(replicationId) - }.withDataCenter(replicaId.id)) + ReplicatedEntityProvider.perDataCenter("MyEntityType", Set(ReplicaId("DC-A"), ReplicaId("DC-B"))) { replicationId => + MyEventSourcedBehavior(replicationId) } //#bootstrap-dc //#bootstrap-role - val provider = ReplicatedEntityProvider[Command, ShardingEnvelope[Command]]( - "MyEntityType", - Set(ReplicaId("DC-A"), ReplicaId("DC-B"))) { (entityTypeKey, replicaId) => - ReplicatedEntity(replicaId, Entity(entityTypeKey) { entityContext => - val replicationId = ReplicationId.fromString(entityContext.entityId) + val provider = ReplicatedEntityProvider.perRole("MyEntityType", Set(ReplicaId("DC-A"), ReplicaId("DC-B"))) { + replicationId => MyEventSourcedBehavior(replicationId) - }.withRole(replicaId.id)) } //#bootstrap-role //#sending-messages - val myReplicatedSharding: ReplicatedSharding[Command, ShardingEnvelope[Command]] = + val myReplicatedSharding: ReplicatedSharding[Command] = ReplicatedShardingExtension(system).init(provider) val entityRefs: Map[ReplicaId, EntityRef[Command]] = myReplicatedSharding.entityRefsFor("myEntityId") - val actorRefs: Map[ReplicaId, ActorRef[ShardingEnvelope[Command]]] = myReplicatedSharding.shardingRefs //#sending-messages } 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 index 7cbc460f10..8badd779ad 100644 --- 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 @@ -80,7 +80,7 @@ public class ReplicatedEventSourcingTest extends JUnitSuite { public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ReplicatedEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.commonJournalConfig( new ReplicationId("ReplicatedEventSourcingTest", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), 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 index 5a04ddf503..25166dccd7 100644 --- 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 @@ -33,7 +33,7 @@ public class MyReplicatedBehavior // #factory-shared public static Behavior create( String entityId, ReplicaId replicaId, String queryPluginId) { - return ReplicatedEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.commonJournalConfig( new ReplicationId("MyReplicatedEntity", entityId, replicaId), ALL_REPLICAS, queryPluginId, @@ -47,7 +47,7 @@ public class MyReplicatedBehavior allReplicasAndQueryPlugins.put(DCA, "journalForDCA"); allReplicasAndQueryPlugins.put(DCB, "journalForDCB"); - return ReplicatedEventSourcing.create( + return ReplicatedEventSourcing.perReplicaJournalConfig( new ReplicationId("MyReplicatedEntity", entityId, replicaId), allReplicasAndQueryPlugins, MyReplicatedBehavior::new); 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 index 8d1091b7c3..e5b2497ddf 100644 --- 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 @@ -262,7 +262,7 @@ class AuctionEntity extends ReplicatedEventSourcedBehavior Behaviors.withTimers( timers -> - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( new ReplicationId("Auction", name, replica), ALL_REPLICAS, PersistenceTestKitReadJournal.Identifier(), 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 index b32c811ea5..fb550b04fe 100644 --- 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 @@ -187,7 +187,7 @@ interface ReplicatedBlogExample { String entityId, ReplicaId replicaId, Set allReplicas) { return Behaviors.setup( context -> - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( new ReplicationId("blog", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), 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 index 94b219c6c1..da80971b95 100644 --- 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 @@ -61,7 +61,7 @@ interface ReplicatedMovieExample { public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ReplicatedEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.commonJournalConfig( new ReplicationId("movies", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), 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 index 629065b614..069f0982aa 100644 --- 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 @@ -85,7 +85,7 @@ interface ReplicatedShoppingCartExample { public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ReplicatedEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.commonJournalConfig( new ReplicationId("blog", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), 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 index c9198f65d0..011108922f 100644 --- 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 @@ -27,7 +27,7 @@ public final class ReplicatedStringSet public static Behavior create( String entityId, ReplicaId replicaId, Set allReplicas) { - return ReplicatedEventSourcing.withSharedJournal( + return ReplicatedEventSourcing.commonJournalConfig( new ReplicationId("StringSet", entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier(), 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 index 368f9885dc..eb8e701238 100644 --- 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 @@ -30,7 +30,7 @@ object ReplicatedEventPublishingSpec { def apply(entityId: String, replicaId: ReplicaId, allReplicas: Set[ReplicaId]): Behavior[Command] = Behaviors.setup { ctx => - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId(EntityType, entityId, replicaId), allReplicas, PersistenceTestKitReadJournal.Identifier)( 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 index c6ba98a771..919bda434b 100644 --- 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 @@ -71,7 +71,7 @@ object ReplicatedEventSourcingSpec { entityId: String, replicaId: String, probe: Option[ActorRef[EventAndContext]] = None): Behavior[Command] = - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("ReplicatedEventSourcingSpec", entityId, ReplicaId(replicaId)), AllReplicas, PersistenceTestKitReadJournal.Identifier)(replicationContext => eventSourcedBehavior(replicationContext, probe)) 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 index 626c07daf1..f5a7e61601 100644 --- 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 @@ -42,7 +42,7 @@ object ReplicatedEventSourcingTaggingSpec { replica: ReplicaId, allReplicas: Set[ReplicaId]): EventSourcedBehavior[Command, String, State] = { // #tagging - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("TaggingSpec", entityId, replica), allReplicas, queryPluginId)( 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 index 79503a5d3b..cf457d5e4d 100644 --- 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 @@ -28,7 +28,7 @@ object ReplicationIllegalAccessSpec { case class State(all: List[String]) extends CborSerializable def apply(entityId: String, replica: ReplicaId): Behavior[Command] = { - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("IllegalAccessSpec", entityId, replica), AllReplicas, PersistenceTestKitReadJournal.Identifier)( @@ -88,7 +88,7 @@ class ReplicationIllegalAccessSpec } "detect illegal access in the factory" in { val exception = intercept[UnsupportedOperationException] { - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("IllegalAccessSpec", "id2", R1), AllReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => 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 index 3fe5722b95..620c8ffbb4 100644 --- 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 @@ -35,7 +35,7 @@ object ReplicationSnapshotSpec { entityId: String, replicaId: ReplicaId, probe: Option[ActorRef[EventAndContext]]): Behavior[Command] = { - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId(EntityType, entityId, replicaId), AllReplicas, PersistenceTestKitReadJournal.Identifier)(replicationContext => 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 index aeecc51881..3d6a9468c9 100644 --- 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 @@ -29,7 +29,7 @@ object CounterSpec { snapshotEvery: Long = 100, eventProbe: Option[ActorRef[Counter.Updated]] = None) = Behaviors.setup[PlainCounter.Command] { context => - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("CounterSpec", entityId, replicaId), AllReplicas, PersistenceTestKitReadJournal.Identifier) { ctx => 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 index 1de011f629..f7a1cfbc73 100644 --- 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 @@ -27,7 +27,7 @@ object LwwSpec { object LwwRegistry { def apply(entityId: String, replica: ReplicaId): Behavior[Command] = { - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("LwwRegistrySpec", entityId, replica), AllReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => 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 index 95a2f383a9..6f9a384ad8 100644 --- 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 @@ -28,7 +28,7 @@ object ORSetSpec { def apply(entityId: String, replica: ReplicaId): Behavior[ORSetEntity.Command] = { - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("ORSetSpec", entityId, replica), AllReplicas, PersistenceTestKitReadJournal.Identifier) { replicationContext => 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 index fcae4417a1..71d868a6a1 100644 --- 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 @@ -143,7 +143,7 @@ object ReplicatedAuctionExampleSpec { responsibleForClosing: Boolean, allReplicas: Set[ReplicaId]): Behavior[Command] = Behaviors.setup[Command] { ctx => Behaviors.withTimers { timers => - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("auction", name, replica), allReplicas, PersistenceTestKitReadJournal.Identifier) { replicationCtx => 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 index 53d3949e0c..cba33aee8e 100644 --- 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 @@ -52,7 +52,7 @@ object ReplicatedBlogExampleSpec { def apply(entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId]): Behavior[Command] = { Behaviors.setup[Command] { ctx => - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("blog", entityId, replicaId), allReplicaIds, PersistenceTestKitReadJournal.Identifier) { replicationContext => 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 index f6edecc687..002f2cd0ef 100644 --- 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 @@ -25,7 +25,7 @@ object ReplicatedEventSourcingCompileOnlySpec { trait Event //#factory-shared - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("entityTypeHint", "entityId", DCA), AllReplicas, queryPluginId) { context => 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 index c67a0bc203..53a726bb3a 100644 --- 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 @@ -28,7 +28,7 @@ object ReplicatedMovieWatchListExampleSpec { final case class MovieList(movieIds: Set[String]) def apply(entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId]): Behavior[Command] = { - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("movies", entityId, replicaId), allReplicaIds, PersistenceTestKitReadJournal.Identifier) { replicationContext => 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 index a8ed48b856..53a9fd2106 100644 --- 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 @@ -41,7 +41,7 @@ object ReplicatedShoppingCartExampleSpec { final case class State(items: Map[ProductId, Counter]) def apply(entityId: String, replicaId: ReplicaId, allReplicaIds: Set[ReplicaId]): Behavior[Command] = { - ReplicatedEventSourcing.withSharedJournal( + ReplicatedEventSourcing.commonJournalConfig( ReplicationId("blog", entityId, replicaId), allReplicaIds, PersistenceTestKitReadJournal.Identifier) { replicationContext => 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 index 3f3e6c049c..1ba2d88151 100644 --- 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 @@ -72,7 +72,8 @@ trait ReplicationContext { object ReplicatedEventSourcing { /** - * Initialize a replicated event sourced behavior where all entity replicas are stored in the same journal. + * 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. @@ -85,16 +86,20 @@ object ReplicatedEventSourcing { * * @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 withSharedJournal[Command, Event, State]( + def commonJournalConfig[Command, Event, State]( replicationId: ReplicationId, allReplicaIds: JSet[ReplicaId], queryPluginId: String, behaviorFactory: JFunction[ReplicationContext, EventSourcedBehavior[Command, Event, State]]) : EventSourcedBehavior[Command, Event, State] = - create(replicationId, allReplicaIds.asScala.map(id => id -> queryPluginId).toMap.asJava, behaviorFactory) + perReplicaJournalConfig( + replicationId, + allReplicaIds.asScala.map(id => id -> queryPluginId).toMap.asJava, + behaviorFactory) /** - * Initialize a replicated event sourced behavior. + * 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. @@ -108,7 +113,7 @@ object ReplicatedEventSourcing { * @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 create[Command, Event, State]( + def perReplicaJournalConfig[Command, Event, State]( replicationId: ReplicationId, allReplicasAndQueryPlugins: JMap[ReplicaId, String], eventSourcedBehaviorFactory: JFunction[ReplicationContext, 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 index d5ceaff944..c9d12d6655 100644 --- 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 @@ -82,13 +82,14 @@ object ReplicatedEventSourcing { * @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 withSharedJournal[Command, Event, State]( + def commonJournalConfig[Command, Event, State]( replicationId: ReplicationId, allReplicaIds: Set[ReplicaId], queryPluginId: String)( eventSourcedBehaviorFactory: ReplicationContext => EventSourcedBehavior[Command, Event, State]) : EventSourcedBehavior[Command, Event, State] = - apply(replicationId, allReplicaIds.map(id => id -> queryPluginId).toMap)(eventSourcedBehaviorFactory) + perReplicaJournalConfig(replicationId, allReplicaIds.map(id => id -> queryPluginId).toMap)( + eventSourcedBehaviorFactory) /** * Initialize a replicated event sourced behavior. @@ -104,7 +105,9 @@ object ReplicatedEventSourcing { * @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 apply[Command, Event, State](replicationId: ReplicationId, allReplicasAndQueryPlugins: Map[ReplicaId, String])( + 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) From 2d9660f847551b0dfc5825b1887e326b9c41e0ea Mon Sep 17 00:00:00 2001 From: Christopher Batey Date: Mon, 24 Aug 2020 11:29:56 +0100 Subject: [PATCH 49/50] Final review feedback --- .../replicated-event-sourcing.excludes | 2 ++ .../sharding/typed/ReplicatedShardingSpec.scala | 17 ++++++++--------- .../scala/akka/cluster/sharding/Shard.scala | 2 ++ akka-docs/src/main/paradox/project/examples.md | 2 +- akka-docs/src/main/paradox/typed/cluster-dc.md | 2 +- .../typed/replicated-eventsourcing-auction.md | 2 +- .../paradox/typed/replicated-eventsourcing.md | 7 ++++--- .../typed/MultiJournalReplicationSpec.scala | 11 ++++++----- ...ReplicatedEventSourcingCompileOnlySpec.scala | 2 +- build.sbt | 1 + 10 files changed, 27 insertions(+), 21 deletions(-) create mode 100644 akka-cluster-sharding-typed/src/main/mima-filters/2.6.8.backwards.excludes/replicated-event-sourcing.excludes 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/multi-jvm/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala b/akka-cluster-sharding-typed/src/multi-jvm/scala/akka/cluster/sharding/typed/ReplicatedShardingSpec.scala index 156037b46d..3d39c47991 100644 --- 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 @@ -96,14 +96,13 @@ object ReplicatedShardingSpec extends MultiNodeConfig { } } - def provider(): ReplicatedEntityProvider[Command, ShardingEnvelope[Command]] = { - ReplicatedEntityProvider[Command, ShardingEnvelope[Command]]("TestRES", AllReplicas) { - (entityTypeKey, replicaId) => - ReplicatedEntity(replicaId, Entity(entityTypeKey) { entityContext => - Behaviors.setup { ctx => - TestRES(ReplicationId.fromString(entityContext.entityId), ctx) - } - }) + 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 } } @@ -134,7 +133,7 @@ abstract class ReplicatedShardingSpec } "start replicated entities" in { - val replicatedSharding: ReplicatedSharding[TestRES.Command, ShardingEnvelope[TestRES.Command]] = + val replicatedSharding: ReplicatedSharding[TestRES.Command] = ReplicatedShardingExtension(typedSystem).init(TestRES.provider()) runOn(first) { 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 925da2e7f3..13b7f8bc25 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,6 +358,8 @@ private[akka] object Shard { // only called once during handoff def activeEntities(): Set[ActorRef] = byRef.keySet.asScala.toSet + def nrActiveEntities() = byRef.size() + // only called for getting shard stats def activeEntityIds(): Set[EntityId] = byRef.values.asScala.toSet diff --git a/akka-docs/src/main/paradox/project/examples.md b/akka-docs/src/main/paradox/project/examples.md index d790e8590e..d108262df8 100644 --- a/akka-docs/src/main/paradox/project/examples.md +++ b/akka-docs/src/main/paradox/project/examples.md @@ -61,7 +61,7 @@ from the events, or publish the events to other services. ## Multi-DC Persistence -This commercial feature has now been superseeded by @ref[Replicated Event Sourcing](../typed/replicated-eventsourcing.md) +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/typed/cluster-dc.md b/akka-docs/src/main/paradox/typed/cluster-dc.md index ca0003bb71..ea5d3921ce 100644 --- a/akka-docs/src/main/paradox/typed/cluster-dc.md +++ b/akka-docs/src/main/paradox/typed/cluster-dc.md @@ -193,7 +193,7 @@ Especially when used together with Akka Persistence that is based on the single- 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 replicated persistent -entities see @ref[Replciated Event Sourcing](replicated-eventsourcing.md). +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/replicated-eventsourcing-auction.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-auction.md index bc2a7040c3..a10bc9ba9f 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing-auction.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing-auction.md @@ -25,7 +25,7 @@ Scala 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. +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: diff --git a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md index 8a9413d935..52770e4b95 100644 --- a/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md +++ b/akka-docs/src/main/paradox/typed/replicated-eventsourcing.md @@ -8,7 +8,8 @@ warning or deprecation period. It is also not recommended to use this module in @@@ -@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. +@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. @@ -28,7 +29,7 @@ The motivations are: * Balance the load over many servers However, the event handler must be able to **handle concurrent events** as when replication is enabled -there is no longer the single writer principle as there is with a normal `EventSourcedBehavior`. +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 @@ -36,7 +37,7 @@ 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 +## Relaxing the single-writer principle for availability Taking the example of using Replicated Event Sourcing to run a replica per data center. 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 index 4257e324f8..cc7867f5dd 100644 --- 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 @@ -32,10 +32,10 @@ object MultiJournalReplicationSpec { private val writeJournalPerReplica = Map("R1" -> "journal1.journal", "R2" -> "journal2.journal") def apply(entityId: String, replicaId: String): Behavior[Command] = { - ReplicatedEventSourcing( - ReplicationId("MultiJournalSpec", entityId, ReplicaId(replicaId)), - Map(ReplicaId("R1") -> "journal1.query", ReplicaId("R2") -> "journal2.query"))( - replicationContext => + 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], @@ -47,7 +47,8 @@ object MultiJournalReplicationSpec { case StoreMe(evt, ack) => Effect.persist(evt).thenRun(_ => ack ! Done) }, - (state, event) => state + event)).withJournalPluginId(writeJournalPerReplica(replicaId)) + (state, event) => state + event)) + .withJournalPluginId(writeJournalPerReplica(replicaId)) } } 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 index 002f2cd0ef..8ab4ebec6e 100644 --- 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 @@ -34,7 +34,7 @@ object ReplicatedEventSourcingCompileOnlySpec { //#factory-shared //#factory - ReplicatedEventSourcing( + ReplicatedEventSourcing.perReplicaJournalConfig( ReplicationId("entityTypeHint", "entityId", DCA), Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { context => EventSourcedBehavior[Command, State, Event](???, ???, ???, ???) diff --git a/build.sbt b/build.sbt index 0794926692..b8c1e1ca4e 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, From bb985b82c6c140dd4697fe3a9b8d93389c98a647 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 3 Sep 2020 12:48:53 +0200 Subject: [PATCH 50/50] fix compilation errors --- .../src/main/scala/akka/cluster/sharding/Shard.scala | 2 +- .../akka/persistence/typed/ReplicatedAuctionExampleTest.java | 4 +++- .../test/scala/akka/persistence/typed/crdt/CounterSpec.scala | 4 ++-- 3 files changed, 6 insertions(+), 4 deletions(-) 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 6b421229f5..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 @@ -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-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 index e5b2497ddf..2401d1bfb9 100644 --- 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 @@ -90,7 +90,9 @@ public class ReplicatedAuctionExampleTest extends JUnitSuite { } // #setup -class AuctionEntity extends ReplicatedEventSourcedBehavior { +class AuctionEntity + extends ReplicatedEventSourcedBehavior< + AuctionEntity.Command, AuctionEntity.Event, AuctionEntity.AuctionState> { public static ReplicaId R1 = new ReplicaId("R1"); public static ReplicaId R2 = new ReplicaId("R2"); 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 index 3d6a9468c9..54f4e7ab7d 100644 --- 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 @@ -44,8 +44,8 @@ object CounterSpec { 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() + context.log.info("Get request. {} {}", state.value, state.value.longValue) + replyTo ! state.value.longValue Effect.none }, (counter, event) => {