Merge pull request #25696 from akka/wip-PersistentBehavior-rename-patriknw
rename PersistentBehaviors.receive to PersistentBehavior.apply
This commit is contained in:
commit
63b7cc40cc
20 changed files with 76 additions and 76 deletions
|
|
@ -11,7 +11,7 @@ import akka.actor.typed.Props
|
|||
import akka.cluster.sharding.typed.ClusterShardingSettings
|
||||
import akka.cluster.typed.Cluster
|
||||
import akka.cluster.typed.Join
|
||||
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehaviors }
|
||||
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehavior }
|
||||
import akka.actor.testkit.typed.scaladsl.TestProbe
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.{ WordSpec, WordSpecLike }
|
||||
|
|
@ -44,7 +44,7 @@ object ClusterShardingPersistenceSpec {
|
|||
final case object StopPlz extends Command
|
||||
|
||||
def persistentActor(entityId: String): Behavior[Command] =
|
||||
PersistentBehaviors.receive[Command, String, String](
|
||||
PersistentBehavior[Command, String, String](
|
||||
entityId,
|
||||
emptyState = "",
|
||||
commandHandler = (state, cmd) ⇒ cmd match {
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.cluster.typed
|
|||
|
||||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.{ ActorRef, Behavior, Props }
|
||||
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehaviors }
|
||||
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehavior }
|
||||
import akka.actor.testkit.typed.scaladsl.TestProbe
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.WordSpecLike
|
||||
|
|
@ -36,7 +36,7 @@ object ClusterSingletonPersistenceSpec {
|
|||
private final case object StopPlz extends Command
|
||||
|
||||
val persistentActor: Behavior[Command] =
|
||||
PersistentBehaviors.receive[Command, String, String](
|
||||
PersistentBehavior[Command, String, String](
|
||||
persistenceId = "TheSingleton",
|
||||
emptyState = "",
|
||||
commandHandler = (state, cmd) ⇒ cmd match {
|
||||
|
|
|
|||
|
|
@ -30,10 +30,10 @@ This module is currently marked as @ref:[may change](../common/may-change.md) in
|
|||
Let's start with a simple example. The minimum required for a `PersistentBehavior` is:
|
||||
|
||||
Scala
|
||||
: @@snip [BasicPersistentBehaviorsCompileOnly.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorsCompileOnly.scala) { #structure }
|
||||
: @@snip [BasicPersistentBehaviorCompileOnly.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala) { #structure }
|
||||
|
||||
Java
|
||||
: @@snip [BasicPersistentBehaviorsTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorsTest.java) { #structure }
|
||||
: @@snip [BasicPersistentBehaviorTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java) { #structure }
|
||||
|
||||
The first important thing to notice is the `Behavior` of a persistent actor is typed to the type of the `Command`
|
||||
because this is the type of message a persistent actor should receive. In Akka Typed this is now enforced by the type system.
|
||||
|
|
@ -212,7 +212,7 @@ Scala
|
|||
Java
|
||||
: @@snip [InDepthPersistentBehaviorTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/InDepthPersistentBehaviorTest.java) { #event-handler }
|
||||
|
||||
And finally the behavior is created @scala[from the `PersistentBehaviors.receive`]:
|
||||
And finally the behavior is created @scala[from the `PersistentBehavior.apply`]:
|
||||
|
||||
Scala
|
||||
: @@snip [InDepthPersistentBehaviorSpec.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/InDepthPersistentBehaviorSpec.scala) { #behavior }
|
||||
|
|
@ -238,10 +238,10 @@ Most of them time this will be done with the `thenRun` method on the `Effect` ab
|
|||
factor out common `SideEffect`s. For example:
|
||||
|
||||
Scala
|
||||
: @@snip [BasicPersistentBehaviorsCompileOnly.scala](/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala) { #commonChainedEffects }
|
||||
: @@snip [BasicPersistentBehaviorCompileOnly.scala](/akka-persistence-typed/src/test/scala/akka/persistence/typed/scaladsl/PersistentActorCompileOnlyTest.scala) { #commonChainedEffects }
|
||||
|
||||
Java
|
||||
: @@snip [BasicPersistentBehaviorsCompileOnly.scala](/akka-persistence-typed/src/test/java/akka/persistence/typed/javadsl/PersistentActorCompileOnlyTest.java) { #commonChainedEffects }
|
||||
: @@snip [BasicPersistentBehaviorCompileOnly.scala](/akka-persistence-typed/src/test/java/akka/persistence/typed/javadsl/PersistentActorCompileOnlyTest.java) { #commonChainedEffects }
|
||||
|
||||
### Side effects ordering and guarantees
|
||||
|
||||
|
|
@ -261,10 +261,10 @@ It is strongly discouraged to perform side effects in `applyEvent`,
|
|||
so side effects should be performed once recovery has completed @scala[in the `onRecoveryCompleted` callback.] @java[by overriding `onRecoveryCompleted`]
|
||||
|
||||
Scala
|
||||
: @@snip [BasicPersistentBehaviorsCompileOnly.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorsCompileOnly.scala) { #recovery }
|
||||
: @@snip [BasicPersistentBehaviorCompileOnly.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala) { #recovery }
|
||||
|
||||
Java
|
||||
: @@snip [BasicPersistentBehaviorsTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorsTest.java) { #recovery }
|
||||
: @@snip [BasicPersistentBehaviorTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java) { #recovery }
|
||||
|
||||
The `onRecoveryCompleted` takes @scala[an `ActorContext` and] the current `State`,
|
||||
and doesn't return anything.
|
||||
|
|
@ -274,10 +274,10 @@ and doesn't return anything.
|
|||
Persistence typed allows you to use event tags without using @ref[`EventAdapter`](../persistence.md#event-adapters):
|
||||
|
||||
Scala
|
||||
: @@snip [BasicPersistentActorCompileOnly.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorsCompileOnly.scala) { #tagging }
|
||||
: @@snip [BasicPersistentActorCompileOnly.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala) { #tagging }
|
||||
|
||||
Java
|
||||
: @@snip [BasicPersistentBehaviorsTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorsTest.java) { #tagging }
|
||||
: @@snip [BasicPersistentBehaviorTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java) { #tagging }
|
||||
|
||||
## Event adapters
|
||||
|
||||
|
|
@ -307,10 +307,10 @@ other behaviors such as `Behaviors.setup` in order to access the `ActorContext`
|
|||
to access the actor logging upon taking snapshots for debug purpose.
|
||||
|
||||
Scala
|
||||
: @@snip [BasicPersistentActorCompileOnly.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorsCompileOnly.scala) { #wrapPersistentBehavior }
|
||||
: @@snip [BasicPersistentActorCompileOnly.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala) { #wrapPersistentBehavior }
|
||||
|
||||
Java
|
||||
: @@snip [BasicPersistentBehaviorsTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorsTest.java) { #wrapPersistentBehavior }
|
||||
: @@snip [BasicPersistentBehaviorTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java) { #wrapPersistentBehavior }
|
||||
|
||||
|
||||
## Journal failures
|
||||
|
|
@ -321,10 +321,10 @@ any `BackoffSupervisorStrategy`. It is not possible to use the normal supervisio
|
|||
|
||||
|
||||
Scala
|
||||
: @@snip [BasicPersistentBehaviorsSpec.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorsCompileOnly.scala) { #supervision }
|
||||
: @@snip [BasicPersistentBehaviorSpec.scala](/akka-persistence-typed/src/test/scala/docs/akka/persistence/typed/BasicPersistentBehaviorCompileOnly.scala) { #supervision }
|
||||
|
||||
Java
|
||||
: @@snip [BasicPersistentBehaviorsTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorsTest.java) { #supervision }
|
||||
: @@snip [BasicPersistentBehaviorTest.java](/akka-persistence-typed/src/test/java/jdocs/akka/persistence/typed/BasicPersistentBehaviorTest.java) { #supervision }
|
||||
|
||||
## Journal rejections
|
||||
|
||||
|
|
|
|||
|
|
@ -15,7 +15,7 @@ import akka.persistence._
|
|||
import akka.persistence.typed.EventAdapter
|
||||
import akka.persistence.typed.internal.EventsourcedBehavior.MDC
|
||||
import akka.persistence.typed.internal.EventsourcedBehavior.{ InternalProtocol, WriterIdentity }
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior
|
||||
import akka.util.Collections.EmptyImmutableSeq
|
||||
import akka.util.OptionVal
|
||||
import scala.util.Try
|
||||
|
|
@ -31,8 +31,8 @@ private[persistence] final class EventsourcedSetup[C, E, S](
|
|||
val context: ActorContext[InternalProtocol],
|
||||
val persistenceId: String,
|
||||
val emptyState: S,
|
||||
val commandHandler: PersistentBehaviors.CommandHandler[C, E, S],
|
||||
val eventHandler: PersistentBehaviors.EventHandler[S, E],
|
||||
val commandHandler: PersistentBehavior.CommandHandler[C, E, S],
|
||||
val eventHandler: PersistentBehavior.EventHandler[S, E],
|
||||
val writerIdentity: WriterIdentity,
|
||||
val recoveryCompleted: S ⇒ Unit,
|
||||
val onSnapshot: (SnapshotMetadata, Try[Done]) ⇒ Unit,
|
||||
|
|
|
|||
|
|
@ -34,17 +34,17 @@ private[akka] object PersistentBehaviorImpl {
|
|||
private[akka] final case class PersistentBehaviorImpl[Command, Event, State](
|
||||
persistenceId: String,
|
||||
emptyState: State,
|
||||
commandHandler: PersistentBehaviors.CommandHandler[Command, Event, State],
|
||||
eventHandler: PersistentBehaviors.EventHandler[State, Event],
|
||||
journalPluginId: Option[String] = None,
|
||||
snapshotPluginId: Option[String] = None,
|
||||
recoveryCompleted: State ⇒ Unit = ConstantFun.scalaAnyToUnit,
|
||||
tagger: Event ⇒ Set[String] = (_: Event) ⇒ Set.empty[String],
|
||||
eventAdapter: EventAdapter[Event, Any] = NoOpEventAdapter.instance[Event],
|
||||
snapshotWhen: (State, Event, Long) ⇒ Boolean = ConstantFun.scalaAnyThreeToFalse,
|
||||
recovery: Recovery = Recovery(),
|
||||
supervisionStrategy: SupervisorStrategy = SupervisorStrategy.stop,
|
||||
onSnapshot: (SnapshotMetadata, Try[Done]) ⇒ Unit = ConstantFun.scalaAnyTwoToUnit
|
||||
commandHandler: PersistentBehavior.CommandHandler[Command, Event, State],
|
||||
eventHandler: PersistentBehavior.EventHandler[State, Event],
|
||||
journalPluginId: Option[String] = None,
|
||||
snapshotPluginId: Option[String] = None,
|
||||
recoveryCompleted: State ⇒ Unit = ConstantFun.scalaAnyToUnit,
|
||||
tagger: Event ⇒ Set[String] = (_: Event) ⇒ Set.empty[String],
|
||||
eventAdapter: EventAdapter[Event, Any] = NoOpEventAdapter.instance[Event],
|
||||
snapshotWhen: (State, Event, Long) ⇒ Boolean = ConstantFun.scalaAnyThreeToFalse,
|
||||
recovery: Recovery = Recovery(),
|
||||
supervisionStrategy: SupervisorStrategy = SupervisorStrategy.stop,
|
||||
onSnapshot: (SnapshotMetadata, Try[Done]) ⇒ Unit = ConstantFun.scalaAnyTwoToUnit
|
||||
) extends PersistentBehavior[Command, Event, State] with EventsourcedStashReferenceManagement {
|
||||
|
||||
override def apply(context: typed.ActorContext[Command]): Behavior[Command] = {
|
||||
|
|
|
|||
|
|
@ -147,7 +147,7 @@ abstract class PersistentBehavior[Command, Event, State >: Null] private (val pe
|
|||
else tags.asScala.toSet
|
||||
}
|
||||
|
||||
val behavior = scaladsl.PersistentBehaviors.receive[Command, Event, State](
|
||||
val behavior = scaladsl.PersistentBehavior[Command, Event, State](
|
||||
persistenceId,
|
||||
emptyState,
|
||||
(state, cmd) ⇒ commandHandler()(state, cmd).asInstanceOf[EffectImpl[Event, State]],
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ import akka.persistence.typed.internal._
|
|||
|
||||
import scala.util.Try
|
||||
|
||||
object PersistentBehaviors {
|
||||
object PersistentBehavior {
|
||||
|
||||
/**
|
||||
* Type alias for the command handler function for reacting on events having been persisted.
|
||||
|
|
@ -37,7 +37,7 @@ object PersistentBehaviors {
|
|||
/**
|
||||
* Create a `Behavior` for a persistent actor.
|
||||
*/
|
||||
def receive[Command, Event, State](
|
||||
def apply[Command, Event, State](
|
||||
persistenceId: String,
|
||||
emptyState: State,
|
||||
commandHandler: (State, Command) ⇒ Effect[Event, State],
|
||||
|
|
@ -16,7 +16,7 @@ import java.time.Duration;
|
|||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
public class BasicPersistentBehaviorsTest {
|
||||
public class BasicPersistentBehaviorTest {
|
||||
|
||||
//#structure
|
||||
public interface Command {}
|
||||
|
|
@ -6,8 +6,8 @@ package akka.persistence.typed
|
|||
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.actor.typed.scaladsl.adapter.TypedActorSystemOps
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors.CommandHandler
|
||||
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehavior, PersistentBehaviors }
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior.CommandHandler
|
||||
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehavior }
|
||||
import akka.testkit.TestLatch
|
||||
import akka.actor.testkit.typed.scaladsl.TestProbe
|
||||
|
||||
|
|
@ -26,7 +26,7 @@ object ManyRecoveriesSpec {
|
|||
name: String,
|
||||
probe: TestProbe[String],
|
||||
latch: Option[TestLatch]): PersistentBehavior[Cmd, Evt, String] =
|
||||
PersistentBehaviors.receive[Cmd, Evt, String](
|
||||
PersistentBehavior[Cmd, Evt, String](
|
||||
persistenceId = name,
|
||||
emptyState = "",
|
||||
commandHandler = CommandHandler.command {
|
||||
|
|
|
|||
|
|
@ -11,8 +11,8 @@ import akka.actor.typed.scaladsl.adapter.{ TypedActorRefOps, TypedActorSystemOps
|
|||
import akka.actor.typed.{ ActorRef, Behavior }
|
||||
import akka.persistence.Persistence
|
||||
import akka.persistence.RecoveryPermitter.{ RecoveryPermitGranted, RequestRecoveryPermit, ReturnRecoveryPermit }
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors.CommandHandler
|
||||
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehaviors }
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior.CommandHandler
|
||||
import akka.persistence.typed.scaladsl.{ Effect, PersistentBehavior }
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
|
@ -43,7 +43,7 @@ object RecoveryPermitterSpec {
|
|||
commandProbe: TestProbe[Any],
|
||||
eventProbe: TestProbe[Any],
|
||||
throwOnRecovery: Boolean = false): Behavior[Command] =
|
||||
PersistentBehaviors.receive[Command, Event, State](
|
||||
PersistentBehavior[Command, Event, State](
|
||||
persistenceId = name,
|
||||
emptyState = EmptyState,
|
||||
commandHandler = CommandHandler.command {
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import java.util.UUID
|
|||
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
||||
import akka.actor.typed.scaladsl.adapter.{ TypedActorRefOps, TypedActorSystemOps }
|
||||
import akka.event.Logging
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors.CommandHandler
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior.CommandHandler
|
||||
import akka.actor.testkit.typed.scaladsl.TestProbe
|
||||
import org.scalatest.WordSpecLike
|
||||
|
||||
|
|
@ -26,7 +26,7 @@ object OptionalSnapshotStoreSpec {
|
|||
def persistentBehavior(
|
||||
probe: TestProbe[State],
|
||||
name: String = UUID.randomUUID().toString) =
|
||||
PersistentBehaviors.receive[Command, Event, State](
|
||||
PersistentBehavior[Command, Event, State](
|
||||
persistenceId = name,
|
||||
emptyState = State(),
|
||||
commandHandler = CommandHandler.command {
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import java.util.UUID
|
|||
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.actor.typed.{ ActorRef, SupervisorStrategy }
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors.CommandHandler
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior.CommandHandler
|
||||
import akka.actor.testkit.typed.TE
|
||||
import akka.actor.testkit.typed.scaladsl.TestProbe
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
|
@ -61,7 +61,7 @@ object PerformanceSpec {
|
|||
def behavior(name: String, probe: TestProbe[Command])(other: (Command, Parameters) ⇒ Effect[String, String]) = {
|
||||
Behaviors.supervise({
|
||||
val parameters = Parameters()
|
||||
PersistentBehaviors.receive[Command, String, String](
|
||||
PersistentBehavior[Command, String, String](
|
||||
persistenceId = name,
|
||||
"",
|
||||
commandHandler = CommandHandler.command {
|
||||
|
|
|
|||
|
|
@ -13,7 +13,7 @@ import akka.persistence.typed.SideEffect
|
|||
|
||||
object PersistentActorCompileOnlyTest {
|
||||
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors._
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior._
|
||||
|
||||
object Simple {
|
||||
//#command
|
||||
|
|
@ -43,7 +43,7 @@ object PersistentActorCompileOnlyTest {
|
|||
|
||||
//#behavior
|
||||
val simpleBehavior: PersistentBehavior[SimpleCommand, SimpleEvent, ExampleState] =
|
||||
PersistentBehaviors.receive[SimpleCommand, SimpleEvent, ExampleState](
|
||||
PersistentBehavior[SimpleCommand, SimpleEvent, ExampleState](
|
||||
persistenceId = "sample-id-1",
|
||||
emptyState = ExampleState(Nil),
|
||||
commandHandler = commandHandler,
|
||||
|
|
@ -63,7 +63,7 @@ object PersistentActorCompileOnlyTest {
|
|||
|
||||
case class ExampleState(events: List[String] = Nil)
|
||||
|
||||
PersistentBehaviors.receive[MyCommand, MyEvent, ExampleState](
|
||||
PersistentBehavior[MyCommand, MyEvent, ExampleState](
|
||||
persistenceId = "sample-id-1",
|
||||
|
||||
emptyState = ExampleState(Nil),
|
||||
|
|
@ -108,7 +108,7 @@ object PersistentActorCompileOnlyTest {
|
|||
}
|
||||
|
||||
val behavior: Behavior[Command] = Behaviors.setup(ctx ⇒
|
||||
PersistentBehaviors.receive[Command, Event, EventsInFlight](
|
||||
PersistentBehavior[Command, Event, EventsInFlight](
|
||||
persistenceId = "recovery-complete-id",
|
||||
|
||||
emptyState = EventsInFlight(0, Map.empty),
|
||||
|
|
@ -150,7 +150,7 @@ object PersistentActorCompileOnlyTest {
|
|||
sealed trait Event
|
||||
case class MoodChanged(to: Mood) extends Event
|
||||
|
||||
val b: Behavior[Command] = PersistentBehaviors.receive[Command, Event, Mood](
|
||||
val b: Behavior[Command] = PersistentBehavior[Command, Event, Mood](
|
||||
persistenceId = "myPersistenceId",
|
||||
emptyState = Happy,
|
||||
commandHandler = { (state, command) ⇒
|
||||
|
|
@ -192,7 +192,7 @@ object PersistentActorCompileOnlyTest {
|
|||
|
||||
case class State(tasksInFlight: List[Task])
|
||||
|
||||
PersistentBehaviors.receive[Command, Event, State](
|
||||
PersistentBehavior[Command, Event, State](
|
||||
persistenceId = "asdf",
|
||||
emptyState = State(Nil),
|
||||
commandHandler = CommandHandler.command {
|
||||
|
|
@ -220,7 +220,7 @@ object PersistentActorCompileOnlyTest {
|
|||
def worker(task: Task): Behavior[Nothing] = ???
|
||||
|
||||
val behavior: Behavior[Command] = Behaviors.setup(ctx ⇒
|
||||
PersistentBehaviors.receive[Command, Event, State](
|
||||
PersistentBehavior[Command, Event, State](
|
||||
persistenceId = "asdf",
|
||||
emptyState = State(Nil),
|
||||
commandHandler = (_, cmd) ⇒ cmd match {
|
||||
|
|
@ -283,7 +283,7 @@ object PersistentActorCompileOnlyTest {
|
|||
.persist[Event, List[Id]](ItemAdded(id))
|
||||
.thenRun(_ ⇒ metadataRegistry ! GetMetaData(id, adapt))
|
||||
|
||||
PersistentBehaviors.receive[Command, Event, List[Id]](
|
||||
PersistentBehavior[Command, Event, List[Id]](
|
||||
persistenceId = "basket-1",
|
||||
emptyState = Nil,
|
||||
commandHandler = { (state, cmd) ⇒
|
||||
|
|
@ -375,7 +375,7 @@ object PersistentActorCompileOnlyTest {
|
|||
case (state, Remembered(_)) ⇒ state
|
||||
}
|
||||
|
||||
PersistentBehaviors.receive[Command, Event, Mood](
|
||||
PersistentBehavior[Command, Event, Mood](
|
||||
persistenceId = "myPersistenceId",
|
||||
emptyState = Sad,
|
||||
commandHandler,
|
||||
|
|
@ -403,7 +403,7 @@ object PersistentActorCompileOnlyTest {
|
|||
case (state, Done) ⇒ state
|
||||
}
|
||||
|
||||
PersistentBehaviors.receive[Command, Event, State](
|
||||
PersistentBehavior[Command, Event, State](
|
||||
persistenceId = "myPersistenceId",
|
||||
emptyState = new State,
|
||||
commandHandler,
|
||||
|
|
@ -415,7 +415,7 @@ object PersistentActorCompileOnlyTest {
|
|||
class First extends State
|
||||
class Second extends State
|
||||
|
||||
PersistentBehaviors.receive[String, String, State](
|
||||
PersistentBehavior[String, String, State](
|
||||
persistenceId = "myPersistenceId",
|
||||
emptyState = new First,
|
||||
commandHandler = CommandHandler.command {
|
||||
|
|
@ -440,7 +440,7 @@ object PersistentActorCompileOnlyTest {
|
|||
// #actor-context
|
||||
val behavior: Behavior[String] =
|
||||
Behaviors.setup { ctx ⇒
|
||||
PersistentBehaviors.receive[String, String, State](
|
||||
PersistentBehavior[String, String, State](
|
||||
persistenceId = "myPersistenceId",
|
||||
emptyState = new State,
|
||||
commandHandler = CommandHandler.command {
|
||||
|
|
|
|||
|
|
@ -66,7 +66,7 @@ class PersistentBehaviorFailureSpec extends ScalaTestWithActorTestKit(Persistent
|
|||
|
||||
implicit val testSettings = TestKitSettings(system)
|
||||
|
||||
def failingPersistentActor(pid: String, probe: ActorRef[String]): Behavior[String] = PersistentBehaviors.receive[String, String, String](
|
||||
def failingPersistentActor(pid: String, probe: ActorRef[String]): Behavior[String] = PersistentBehavior[String, String, String](
|
||||
pid, "",
|
||||
(_, cmd) ⇒ {
|
||||
probe.tell("persisting")
|
||||
|
|
|
|||
|
|
@ -124,7 +124,7 @@ object PersistentBehaviorSpec {
|
|||
loggingActor: ActorRef[String],
|
||||
probe: ActorRef[(State, Event)],
|
||||
snapshotProbe: ActorRef[Try[Done]]): PersistentBehavior[Command, Event, State] = {
|
||||
PersistentBehaviors.receive[Command, Event, State](
|
||||
PersistentBehavior[Command, Event, State](
|
||||
persistenceId,
|
||||
emptyState = State(0, Vector.empty),
|
||||
commandHandler = (state, cmd) ⇒ cmd match {
|
||||
|
|
|
|||
|
|
@ -6,8 +6,8 @@ package docs.akka.persistence.typed
|
|||
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors.CommandHandler
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior.CommandHandler
|
||||
|
||||
object AccountExample1 {
|
||||
|
||||
|
|
@ -89,7 +89,7 @@ object AccountExample1 {
|
|||
}
|
||||
|
||||
def behavior(accountNumber: String): Behavior[AccountCommand] =
|
||||
PersistentBehaviors.receive[AccountCommand, AccountEvent, Option[Account]](
|
||||
PersistentBehavior[AccountCommand, AccountEvent, Option[Account]](
|
||||
persistenceId = accountNumber,
|
||||
emptyState = None,
|
||||
commandHandler = commandHandler,
|
||||
|
|
|
|||
|
|
@ -6,8 +6,8 @@ package docs.akka.persistence.typed
|
|||
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors.CommandHandler
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior.CommandHandler
|
||||
|
||||
object AccountExample2 {
|
||||
|
||||
|
|
@ -92,7 +92,7 @@ object AccountExample2 {
|
|||
(state, event) ⇒ state.applyEvent(event)
|
||||
|
||||
def behavior(accountNumber: String): Behavior[AccountCommand] =
|
||||
PersistentBehaviors.receive[AccountCommand, AccountEvent, Account](
|
||||
PersistentBehavior[AccountCommand, AccountEvent, Account](
|
||||
persistenceId = accountNumber,
|
||||
emptyState = EmptyAccount,
|
||||
commandHandler = commandHandler,
|
||||
|
|
|
|||
|
|
@ -7,11 +7,11 @@ package docs.akka.persistence.typed
|
|||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.{ Behavior, SupervisorStrategy }
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
object BasicPersistentBehaviorsCompileOnly {
|
||||
object BasicPersistentBehaviorCompileOnly {
|
||||
|
||||
//#structure
|
||||
sealed trait Command
|
||||
|
|
@ -19,7 +19,7 @@ object BasicPersistentBehaviorsCompileOnly {
|
|||
case class State()
|
||||
|
||||
val behavior: Behavior[Command] =
|
||||
PersistentBehaviors.receive[Command, Event, State](
|
||||
PersistentBehavior[Command, Event, State](
|
||||
persistenceId = "abc",
|
||||
emptyState = State(),
|
||||
commandHandler =
|
||||
|
|
@ -36,7 +36,7 @@ object BasicPersistentBehaviorsCompileOnly {
|
|||
|
||||
//#recovery
|
||||
val recoveryBehavior: Behavior[Command] =
|
||||
PersistentBehaviors.receive[Command, Event, State](
|
||||
PersistentBehavior[Command, Event, State](
|
||||
persistenceId = "abc",
|
||||
emptyState = State(),
|
||||
commandHandler =
|
||||
|
|
@ -52,7 +52,7 @@ object BasicPersistentBehaviorsCompileOnly {
|
|||
|
||||
//#tagging
|
||||
val taggingBehavior: Behavior[Command] =
|
||||
PersistentBehaviors.receive[Command, Event, State](
|
||||
PersistentBehavior[Command, Event, State](
|
||||
persistenceId = "abc",
|
||||
emptyState = State(),
|
||||
commandHandler =
|
||||
|
|
@ -65,7 +65,7 @@ object BasicPersistentBehaviorsCompileOnly {
|
|||
//#tagging
|
||||
|
||||
//#wrapPersistentBehavior
|
||||
val samplePersistentBehavior = PersistentBehaviors.receive[Command, Event, State](
|
||||
val samplePersistentBehavior = PersistentBehavior[Command, Event, State](
|
||||
persistenceId = "abc",
|
||||
emptyState = State(),
|
||||
commandHandler =
|
||||
|
|
@ -6,7 +6,7 @@ package docs.akka.persistence.typed
|
|||
|
||||
import akka.Done
|
||||
import akka.actor.typed.{ ActorRef, Behavior }
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
|
||||
object InDepthPersistentBehaviorSpec {
|
||||
|
|
@ -119,7 +119,7 @@ object InDepthPersistentBehaviorSpec {
|
|||
|
||||
//#behavior
|
||||
def behavior(entityId: String): Behavior[BlogCommand] =
|
||||
PersistentBehaviors.receive[BlogCommand, BlogEvent, BlogState](
|
||||
PersistentBehavior[BlogCommand, BlogEvent, BlogState](
|
||||
persistenceId = "Blog-" + entityId,
|
||||
emptyState = BlogState.empty,
|
||||
commandHandler,
|
||||
|
|
|
|||
|
|
@ -7,8 +7,8 @@ package docs.akka.persistence.typed
|
|||
import akka.actor.typed.ActorRef
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.persistence.typed.scaladsl.Effect
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors
|
||||
import akka.persistence.typed.scaladsl.PersistentBehaviors.CommandHandler
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior
|
||||
import akka.persistence.typed.scaladsl.PersistentBehavior.CommandHandler
|
||||
|
||||
object MovieWatchList {
|
||||
sealed trait Command
|
||||
|
|
@ -43,7 +43,7 @@ object MovieWatchList {
|
|||
}
|
||||
|
||||
def behavior(userId: String): Behavior[Command] = {
|
||||
PersistentBehaviors.receive[Command, Event, MovieList](
|
||||
PersistentBehavior[Command, Event, MovieList](
|
||||
persistenceId = "movies-" + userId,
|
||||
emptyState = MovieList(Set.empty),
|
||||
commandHandler,
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue