#23848 Align typed persistence API with ReplicatedEntity API

* Single effect factory companion, rather than concrete types - Improves discoverability and hides implementation details (hopefully making maintenance, ensuring bin comp. etc. easier)
* Actions renamed to CommandHandler
* Rename applyEvent to eventHandler - Especially makes sense after renaming actions to command handler
* Rearrange parameters from most stable to least stable
* onRecoveryCompleted cannot modify state
* Full types for function parameters to make the API easier to understand
This commit is contained in:
Johan Andrén 2017-11-02 14:39:56 +01:00 committed by GitHub
parent 9506c7d949
commit c8748e8cf0
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
5 changed files with 238 additions and 177 deletions

View file

@ -53,14 +53,14 @@ object ClusterShardingPersistenceSpec {
PersistentActor.persistentEntity[Command, String, String]( PersistentActor.persistentEntity[Command, String, String](
persistenceIdFromActorName = name "Test-" + name, persistenceIdFromActorName = name "Test-" + name,
initialState = "", initialState = "",
actions = Actions((ctx, cmd, state) cmd match { commandHandler = CommandHandler((ctx, state, cmd) cmd match {
case Add(s) Persist(s) case Add(s) Effect.persist(s)
case Get(replyTo) case Get(replyTo)
replyTo ! state replyTo ! state
PersistNothing() Effect.done
case StopPlz Stop() case StopPlz Effect.stop
}), }),
applyEvent = (evt, state) if (state.isEmpty) evt else state + "|" + evt) eventHandler = (state, evt) if (state.isEmpty) evt else state + "|" + evt)
val typeKey = EntityTypeKey[Command]("test") val typeKey = EntityTypeKey[Command]("test")

View file

@ -31,12 +31,12 @@ object PersistentActorCompileOnlyTest {
initialState = ExampleState(Nil), initialState = ExampleState(Nil),
actions = Actions.command { commandHandler = CommandHandler.command {
case Cmd(data) Persist(Evt(data)) case Cmd(data) Effect.persist(Evt(data))
}, },
applyEvent = { eventHandler = {
case (Evt(data), state) state.copy(data :: state.events) case (state, Evt(data)) state.copy(data :: state.events)
}) })
} }
@ -56,14 +56,14 @@ object PersistentActorCompileOnlyTest {
initialState = ExampleState(Nil), initialState = ExampleState(Nil),
actions = Actions.command { commandHandler = CommandHandler.command {
case Cmd(data, sender) case Cmd(data, sender)
Persist(Evt(data)) Effect.persist(Evt(data))
.andThen { sender ! Ack } .andThen { sender ! Ack }
}, },
applyEvent = { eventHandler = {
case (Evt(data), state) state.copy(data :: state.events) case (state, Evt(data)) state.copy(data :: state.events)
}) })
} }
@ -98,16 +98,16 @@ object PersistentActorCompileOnlyTest {
initialState = EventsInFlight(0, Map.empty), initialState = EventsInFlight(0, Map.empty),
actions = Actions((ctx, cmd, state) cmd match { commandHandler = CommandHandler((ctx, state, cmd) cmd match {
case DoSideEffect(data) case DoSideEffect(data)
Persist(IntentRecorded(state.nextCorrelationId, data)).andThen { Effect.persist(IntentRecorded(state.nextCorrelationId, data)).andThen {
performSideEffect(ctx.self, state.nextCorrelationId, data) performSideEffect(ctx.self, state.nextCorrelationId, data)
} }
case AcknowledgeSideEffect(correlationId) case AcknowledgeSideEffect(correlationId)
Persist(SideEffectAcknowledged(correlationId)) Effect.persist(SideEffectAcknowledged(correlationId))
}), }),
applyEvent = (evt, state) evt match { eventHandler = (state, evt) evt match {
case IntentRecorded(correlationId, data) case IntentRecorded(correlationId, data)
EventsInFlight( EventsInFlight(
nextCorrelationId = correlationId + 1, nextCorrelationId = correlationId + 1,
@ -119,7 +119,6 @@ object PersistentActorCompileOnlyTest {
state.dataByCorrelationId.foreach { state.dataByCorrelationId.foreach {
case (correlationId, data) performSideEffect(ctx.self, correlationId, data) case (correlationId, data) performSideEffect(ctx.self, correlationId, data)
} }
state
} }
} }
@ -140,22 +139,22 @@ object PersistentActorCompileOnlyTest {
val b: Behavior[Command] = PersistentActor.immutable[Command, Event, Mood]( val b: Behavior[Command] = PersistentActor.immutable[Command, Event, Mood](
persistenceId = "myPersistenceId", persistenceId = "myPersistenceId",
initialState = Happy, initialState = Happy,
actions = Actions.byState { commandHandler = CommandHandler.byState {
case Happy Actions.command { case Happy CommandHandler.command {
case Greet(whom) case Greet(whom)
println(s"Super happy to meet you $whom!") println(s"Super happy to meet you $whom!")
PersistNothing() Effect.done
case MoodSwing Persist(MoodChanged(Sad)) case MoodSwing Effect.persist(MoodChanged(Sad))
} }
case Sad Actions.command { case Sad CommandHandler.command {
case Greet(whom) case Greet(whom)
println(s"hi $whom") println(s"hi $whom")
PersistNothing() Effect.done
case MoodSwing Persist(MoodChanged(Happy)) case MoodSwing Effect.persist(MoodChanged(Happy))
} }
}, },
applyEvent = { eventHandler = {
case (MoodChanged(to), _) to case (_, MoodChanged(to)) to
}) })
// FIXME this doesn't work, wrapping is not supported // FIXME this doesn't work, wrapping is not supported
@ -181,11 +180,11 @@ object PersistentActorCompileOnlyTest {
PersistentActor.immutable[Command, Event, State]( PersistentActor.immutable[Command, Event, State](
persistenceId = "asdf", persistenceId = "asdf",
initialState = State(Nil), initialState = State(Nil),
actions = Actions.command { commandHandler = CommandHandler.command {
case RegisterTask(task) Persist(TaskRegistered(task)) case RegisterTask(task) Effect.persist(TaskRegistered(task))
case TaskDone(task) Persist(TaskRemoved(task)) case TaskDone(task) Effect.persist(TaskRemoved(task))
}, },
applyEvent = (evt, state) evt match { eventHandler = (state, evt) evt match {
case TaskRegistered(task) State(task :: state.tasksInFlight) case TaskRegistered(task) State(task :: state.tasksInFlight)
case TaskRemoved(task) State(state.tasksInFlight.filter(_ != task)) case TaskRemoved(task) State(state.tasksInFlight.filter(_ != task))
}).snapshotOnState(_.tasksInFlight.isEmpty) }).snapshotOnState(_.tasksInFlight.isEmpty)
@ -208,16 +207,17 @@ object PersistentActorCompileOnlyTest {
PersistentActor.immutable[Command, Event, State]( PersistentActor.immutable[Command, Event, State](
persistenceId = "asdf", persistenceId = "asdf",
initialState = State(Nil), initialState = State(Nil),
actions = Actions((ctx, cmd, _) cmd match { commandHandler = CommandHandler((ctx, _, cmd) cmd match {
case RegisterTask(task) Persist(TaskRegistered(task)) case RegisterTask(task)
.andThen { Effect.persist(TaskRegistered(task))
val child = ctx.spawn[Nothing](worker(task), task) .andThen {
// This assumes *any* termination of the child may trigger a `TaskDone`: val child = ctx.spawn[Nothing](worker(task), task)
ctx.watchWith(child, TaskDone(task)) // This assumes *any* termination of the child may trigger a `TaskDone`:
} ctx.watchWith(child, TaskDone(task))
case TaskDone(task) Persist(TaskRemoved(task)) }
case TaskDone(task) Effect.persist(TaskRemoved(task))
}), }),
applyEvent = (evt, state) evt match { eventHandler = (state, evt) evt match {
case TaskRegistered(task) State(task :: state.tasksInFlight) case TaskRegistered(task) State(task :: state.tasksInFlight)
case TaskRemoved(task) State(state.tasksInFlight.filter(_ != task)) case TaskRemoved(task) State(state.tasksInFlight.filter(_ != task))
}) })
@ -239,21 +239,21 @@ object PersistentActorCompileOnlyTest {
persistenceId = "asdf", persistenceId = "asdf",
initialState = State(Nil), initialState = State(Nil),
// The 'onSignal' seems to break type inference here.. not sure if that can be avoided? // The 'onSignal' seems to break type inference here.. not sure if that can be avoided?
actions = Actions[RegisterTask, Event, State]((ctx, cmd, state) cmd match { commandHandler = CommandHandler[RegisterTask, Event, State]((ctx, state, cmd) cmd match {
case RegisterTask(task) Persist(TaskRegistered(task)) case RegisterTask(task) Effect.persist(TaskRegistered(task))
.andThen { .andThen {
val child = ctx.spawn[Nothing](worker(task), task) val child = ctx.spawn[Nothing](worker(task), task)
// This assumes *any* termination of the child may trigger a `TaskDone`: // This assumes *any* termination of the child may trigger a `TaskDone`:
ctx.watch(child) ctx.watch(child)
} }
}).onSignal { }).onSignal {
case (ctx, Terminated(actorRef), _) case (ctx, _, Terminated(actorRef))
// watchWith (as in the above example) is nicer because it means we don't have to // watchWith (as in the above example) is nicer because it means we don't have to
// 'manually' associate the task and the child actor, but we wanted to demonstrate // 'manually' associate the task and the child actor, but we wanted to demonstrate
// signals here: // signals here:
Persist(TaskRemoved(actorRef.path.name)) Effect.persist(TaskRemoved(actorRef.path.name))
}, },
applyEvent = (evt, state) evt match { eventHandler = (state, evt) evt match {
case TaskRegistered(task) State(task :: state.tasksInFlight) case TaskRegistered(task) State(task :: state.tasksInFlight)
case TaskRemoved(task) State(state.tasksInFlight.filter(_ != task)) case TaskRemoved(task) State(state.tasksInFlight.filter(_ != task))
}) })
@ -305,38 +305,42 @@ object PersistentActorCompileOnlyTest {
PersistentActor.immutable[Command, Event, List[Id]]( PersistentActor.immutable[Command, Event, List[Id]](
persistenceId = "basket-1", persistenceId = "basket-1",
initialState = Nil, initialState = Nil,
actions = commandHandler =
Actions.byState(state CommandHandler.byState(state
if (isFullyHydrated(basket, state)) Actions { (ctx, cmd, state) if (isFullyHydrated(basket, state)) CommandHandler { (ctx, state, cmd)
cmd match { cmd match {
case AddItem(id) addItem(id, ctx.self) case AddItem(id) addItem(id, ctx.self)
case RemoveItem(id) Persist(ItemRemoved(id)) case RemoveItem(id) Effect.persist(ItemRemoved(id))
case GotMetaData(data) case GotMetaData(data)
basket = basket.updatedWith(data); PersistNothing() basket = basket.updatedWith(data)
case GetTotalPrice(sender) sender ! basket.items.map(_.price).sum; PersistNothing() Effect.done
case GetTotalPrice(sender)
sender ! basket.items.map(_.price).sum
Effect.done
} }
} }
else Actions { (ctx, cmd, state) else CommandHandler { (ctx, state, cmd)
cmd match { cmd match {
case AddItem(id) addItem(id, ctx.self) case AddItem(id) addItem(id, ctx.self)
case RemoveItem(id) Persist(ItemRemoved(id)) case RemoveItem(id) Effect.persist(ItemRemoved(id))
case GotMetaData(data) case GotMetaData(data)
basket = basket.updatedWith(data) basket = basket.updatedWith(data)
if (isFullyHydrated(basket, state)) { if (isFullyHydrated(basket, state)) {
stash.foreach(ctx.self ! _) stash.foreach(ctx.self ! _)
stash = Nil stash = Nil
} }
PersistNothing() Effect.done
case cmd: GetTotalPrice stash :+= cmd; PersistNothing() case cmd: GetTotalPrice
stash :+= cmd
Effect.done
} }
}), }),
applyEvent = (evt, state) evt match { eventHandler = (state, evt) evt match {
case ItemAdded(id) id +: state case ItemAdded(id) id +: state
case ItemRemoved(id) state.filter(_ != id) case ItemRemoved(id) state.filter(_ != id)
}).onRecoveryCompleted((ctx, state) { }).onRecoveryCompleted((ctx, state) {
val ad = ctx.spawnAdapter((m: MetaData) GotMetaData(m)) val ad = ctx.spawnAdapter((m: MetaData) GotMetaData(m))
state.foreach(id metadataRegistry ! GetMetaData(id, ad)) state.foreach(id metadataRegistry ! GetMetaData(id, ad))
state
}) })
} }
} }
@ -358,17 +362,17 @@ object PersistentActorCompileOnlyTest {
case class Remembered(memory: String) extends Event case class Remembered(memory: String) extends Event
def changeMoodIfNeeded(currentState: Mood, newMood: Mood): Effect[Event, Mood] = def changeMoodIfNeeded(currentState: Mood, newMood: Mood): Effect[Event, Mood] =
if (currentState == newMood) PersistNothing() if (currentState == newMood) Effect.done
else Persist(MoodChanged(newMood)) else Effect.persist(MoodChanged(newMood))
PersistentActor.immutable[Command, Event, Mood]( PersistentActor.immutable[Command, Event, Mood](
persistenceId = "myPersistenceId", persistenceId = "myPersistenceId",
initialState = Sad, initialState = Sad,
actions = Actions { (_, cmd, state) commandHandler = CommandHandler { (_, state, cmd)
cmd match { cmd match {
case Greet(whom) case Greet(whom)
println(s"Hi there, I'm $state!") println(s"Hi there, I'm $state!")
PersistNothing() Effect.done
case CheerUp(sender) case CheerUp(sender)
changeMoodIfNeeded(state, Happy) changeMoodIfNeeded(state, Happy)
.andThen { sender ! Ack } .andThen { sender ! Ack }
@ -376,15 +380,13 @@ object PersistentActorCompileOnlyTest {
// A more elaborate example to show we still have full control over the effects // A more elaborate example to show we still have full control over the effects
// if needed (e.g. when some logic is factored out but you want to add more effects) // if needed (e.g. when some logic is factored out but you want to add more effects)
val commonEffects = changeMoodIfNeeded(state, Happy) val commonEffects = changeMoodIfNeeded(state, Happy)
CompositeEffect( Effect.persistAll(commonEffects.events :+ Remembered(memory), commonEffects.sideEffects)
PersistAll[Event, Mood](commonEffects.events :+ Remembered(memory)),
commonEffects.sideEffects)
} }
}, },
applyEvent = { eventHandler = {
case (MoodChanged(to), _) to case (_, MoodChanged(to)) to
case (Remembered(_), state) state case (state, Remembered(_)) state
}) })
} }
@ -396,22 +398,20 @@ object PersistentActorCompileOnlyTest {
sealed trait Event sealed trait Event
case object Done extends Event case object Done extends Event
type State = Unit class State
PersistentActor.immutable[Command, Event, State]( PersistentActor.immutable[Command, Event, State](
persistenceId = "myPersistenceId", persistenceId = "myPersistenceId",
initialState = (), initialState = new State,
actions = Actions { (_, cmd, _) commandHandler = CommandHandler.command {
cmd match { case Enough
case Enough Effect.persist(Done)
Persist(Done) .andThen(println("yay"))
.andThen( .andThenStop
SideEffect(_ println("yay")),
Stop())
}
}, },
applyEvent = { eventHandler = {
case (Done, _) () case (state, Done) state
}) })
} }

View file

@ -45,12 +45,12 @@ object PersistentActorSpec {
PersistentActor.immutable[Command, Event, State]( PersistentActor.immutable[Command, Event, State](
persistenceId, persistenceId,
initialState = State(0, Vector.empty), initialState = State(0, Vector.empty),
actions = Actions[Command, Event, State]((ctx, cmd, state) cmd match { commandHandler = CommandHandler[Command, Event, State]((ctx, state, cmd) cmd match {
case Increment case Increment
Persist(Incremented(1)) Effect.persist(Incremented(1))
case GetValue(replyTo) case GetValue(replyTo)
replyTo ! state replyTo ! state
PersistNothing() Effect.done
case IncrementLater case IncrementLater
// purpose is to test signals // purpose is to test signals
val delay = ctx.spawnAnonymous(Actor.withTimers[Tick.type] { timers val delay = ctx.spawnAnonymous(Actor.withTimers[Tick.type] { timers
@ -60,19 +60,19 @@ object PersistentActorSpec {
}) })
}) })
ctx.watch(delay) ctx.watch(delay)
PersistNothing() Effect.done
case IncrementAfterReceiveTimeout case IncrementAfterReceiveTimeout
ctx.setReceiveTimeout(10.millis, Timeout) ctx.setReceiveTimeout(10.millis, Timeout)
PersistNothing() Effect.done
case Timeout case Timeout
ctx.cancelReceiveTimeout() ctx.cancelReceiveTimeout()
Persist(Incremented(100)) Effect.persist(Incremented(100))
}) })
.onSignal { .onSignal {
case (_, Terminated(_), _) case (_, _, Terminated(_))
Persist(Incremented(10)) Effect.persist(Incremented(10))
}, },
applyEvent = (evt, state) evt match { eventHandler = (state, evt) evt match {
case Incremented(delta) case Incremented(delta)
State(state.value + delta, state.history :+ state.value) State(state.value + delta, state.history :+ state.value)
}) })

View file

@ -50,9 +50,9 @@ import akka.typed.internal.adapter.ActorRefAdapter
private var state: S = behavior.initialState private var state: S = behavior.initialState
private val actions: Actions[C, E, S] = behavior.actions private val commandHandler: CommandHandler[C, E, S] = behavior.commandHandler
private val eventHandler: (E, S) S = behavior.applyEvent private val eventHandler: (S, E) S = behavior.eventHandler
private val ctxAdapter = new ActorContextAdapter[C](context) private val ctxAdapter = new ActorContextAdapter[C](context)
private val ctx = ctxAdapter.asScala private val ctx = ctxAdapter.asScala
@ -62,17 +62,17 @@ import akka.typed.internal.adapter.ActorRefAdapter
state = snapshot.asInstanceOf[S] state = snapshot.asInstanceOf[S]
case RecoveryCompleted case RecoveryCompleted
state = behavior.recoveryCompleted(ctx, state) behavior.recoveryCompleted(ctx, state)
case event: E @unchecked case event: E @unchecked
state = applyEvent(state, event) state = applyEvent(state, event)
} }
def applyEvent(s: S, event: E): S = def applyEvent(s: S, event: E): S =
eventHandler.apply(event, s) eventHandler.apply(s, event)
private val unhandledSignal: PartialFunction[(ActorContext[C], Signal, S), Effect[E, S]] = { private val unhandledSignal: PartialFunction[(ActorContext[C], S, Signal), Effect[E, S]] = {
case sig Unhandled() case sig Effect.unhandled
} }
override def receiveCommand: Receive = { override def receiveCommand: Receive = {
@ -84,13 +84,13 @@ import akka.typed.internal.adapter.ActorRefAdapter
val effects = msg match { val effects = msg match {
case a.Terminated(ref) case a.Terminated(ref)
val sig = Terminated(ActorRefAdapter(ref))(null) val sig = Terminated(ActorRefAdapter(ref))(null)
actions.sigHandler(state).applyOrElse((ctx, sig, state), unhandledSignal) commandHandler.sigHandler(state).applyOrElse((ctx, state, sig), unhandledSignal)
case a.ReceiveTimeout case a.ReceiveTimeout
actions.commandHandler(ctx, ctxAdapter.receiveTimeoutMsg, state) commandHandler.commandHandler(ctx, state, ctxAdapter.receiveTimeoutMsg)
// TODO note that PostStop and PreRestart signals are not handled, we wouldn't be able to persist there // TODO note that PostStop and PreRestart signals are not handled, we wouldn't be able to persist there
case cmd: C @unchecked case cmd: C @unchecked
// FIXME we could make it more safe by using ClassTag for C // FIXME we could make it more safe by using ClassTag for C
actions.commandHandler(ctx, cmd, state) commandHandler.commandHandler(ctx, state, cmd)
} }
applyEffects(msg, effects) applyEffects(msg, effects)
@ -121,16 +121,16 @@ import akka.typed.internal.adapter.ActorRefAdapter
persistAll(scala.collection.immutable.Seq(events)) { _ persistAll(scala.collection.immutable.Seq(events)) { _
sideEffects.foreach(applySideEffect) sideEffects.foreach(applySideEffect)
} }
case PersistNothing() case _: PersistNothing.type @unchecked
case Unhandled() case _: Unhandled.type @unchecked
super.unhandled(msg) super.unhandled(msg)
case c: ChainableEffect[_, S] case c: ChainableEffect[_, S]
applySideEffect(c) applySideEffect(c)
} }
def applySideEffect(effect: ChainableEffect[_, S]): Unit = effect match { def applySideEffect(effect: ChainableEffect[_, S]): Unit = effect match {
case Stop() context.stop(self) case _: Stop.type @unchecked context.stop(self)
case SideEffect(callbacks) callbacks.apply(state) case SideEffect(callbacks) callbacks.apply(state)
} }
} }

View file

@ -4,23 +4,23 @@
package akka.typed.persistence.scaladsl package akka.typed.persistence.scaladsl
import scala.collection.{ immutable im } import scala.collection.{ immutable im }
import akka.annotation.DoNotInherit import akka.annotation.{ ApiMayChange, DoNotInherit, InternalApi }
import akka.annotation.InternalApi
import akka.typed.Behavior.UntypedBehavior import akka.typed.Behavior.UntypedBehavior
import akka.typed.Signal import akka.typed.Signal
import akka.typed.persistence.internal.PersistentActorImpl import akka.typed.persistence.internal.PersistentActorImpl
import akka.typed.scaladsl.ActorContext import akka.typed.scaladsl.ActorContext
object PersistentActor { object PersistentActor {
/** /**
* Create a `Behavior` for a persistent actor. * Create a `Behavior` for a persistent actor.
*/ */
def immutable[Command, Event, State]( def immutable[Command, Event, State](
persistenceId: String, persistenceId: String,
initialState: State, initialState: State,
actions: Actions[Command, Event, State], commandHandler: CommandHandler[Command, Event, State],
applyEvent: (Event, State) State): PersistentBehavior[Command, Event, State] = eventHandler: (State, Event) State): PersistentBehavior[Command, Event, State] =
persistentEntity(_ persistenceId, initialState, actions, applyEvent) persistentEntity(_ persistenceId, initialState, commandHandler, eventHandler)
/** /**
* Create a `Behavior` for a persistent actor in Cluster Sharding, when the persistenceId is not known * Create a `Behavior` for a persistent actor in Cluster Sharding, when the persistenceId is not known
@ -32,125 +32,186 @@ object PersistentActor {
def persistentEntity[Command, Event, State]( def persistentEntity[Command, Event, State](
persistenceIdFromActorName: String String, persistenceIdFromActorName: String String,
initialState: State, initialState: State,
actions: Actions[Command, Event, State], commandHandler: CommandHandler[Command, Event, State],
applyEvent: (Event, State) State): PersistentBehavior[Command, Event, State] = eventHandler: (State, Event) State): PersistentBehavior[Command, Event, State] =
new PersistentBehavior(persistenceIdFromActorName, initialState, actions, applyEvent, new PersistentBehavior(persistenceIdFromActorName, initialState, commandHandler, eventHandler,
recoveryCompleted = (_, state) state) recoveryCompleted = (_, _) ())
sealed abstract class Effect[+Event, State]() { /**
* Factories for effects - how a persitent actor reacts on a command
*/
object Effect {
def persist[Event, State](event: Event): Effect[Event, State] =
new Persist[Event, State](event)
def persistAll[Event, State](events: im.Seq[Event]): Effect[Event, State] =
new PersistAll[Event, State](events)
def persistAll[Event, State](events: im.Seq[Event], sideEffects: im.Seq[ChainableEffect[Event, State]]): Effect[Event, State] =
new CompositeEffect[Event, State](Some(new PersistAll[Event, State](events)), sideEffects)
/**
* Do not persist anything
*/
def done[Event, State]: Effect[Event, State] = PersistNothing.asInstanceOf[Effect[Event, State]]
/**
* This command is not handled, but it is not an error that it isn't.
*/
def unhandled[Event, State]: Effect[Event, State] = Unhandled.asInstanceOf[Effect[Event, State]]
/**
* Stop this persistent actor
*/
def stop[Event, State]: ChainableEffect[Event, State] = Stop.asInstanceOf[ChainableEffect[Event, State]]
}
/**
* Instances are created through the factories in the [[Effect]] companion object.
*
* Not for user extension.
*/
@DoNotInherit
sealed abstract class Effect[+Event, State] {
/* All events that will be persisted in this effect */ /* All events that will be persisted in this effect */
def events: im.Seq[Event] = Nil def events: im.Seq[Event] = Nil
/* All side effects that will be performed in this effect */ /* All side effects that will be performed in this effect */
def sideEffects: im.Seq[ChainableEffect[_, State]] = def sideEffects[E >: Event]: im.Seq[ChainableEffect[E, State]] = Nil
if (isInstanceOf[ChainableEffect[_, State]]) im.Seq(asInstanceOf[ChainableEffect[_, State]])
else Nil
def andThen(sideEffects: ChainableEffect[_, State]*): Effect[Event, State] =
CompositeEffect(if (events.isEmpty) None else Some(this), sideEffects.toList)
/** Convenience method to register a side effect with just a callback function */ /** Convenience method to register a side effect with just a callback function */
def andThen(callback: State Unit): Effect[Event, State] = def andThen(callback: State Unit): Effect[Event, State] =
andThen(SideEffect[Event, State](callback)) CompositeEffect(this, SideEffect[Event, State](callback))
/** Convenience method to register a side effect with just a lazy expression */ /** Convenience method to register a side effect with just a lazy expression */
def andThen(callback: Unit): Effect[Event, State] = def andThen(callback: Unit): Effect[Event, State] =
andThen(SideEffect[Event, State]((_: State) callback)) CompositeEffect(this, SideEffect[Event, State]((_: State) callback))
/** The side effect is to stop the actor */
def andThenStop: Effect[Event, State] =
CompositeEffect(this, Effect.stop[Event, State])
} }
case class CompositeEffect[Event, State](persistingEffect: Option[Effect[Event, State]], override val sideEffects: im.Seq[ChainableEffect[_, State]]) extends Effect[Event, State] { @InternalApi
private[akka] object CompositeEffect {
def apply[Event, State](effect: Effect[Event, State], sideEffects: ChainableEffect[Event, State]): Effect[Event, State] =
CompositeEffect[Event, State](
if (effect.events.isEmpty) None else Some(effect),
sideEffects :: Nil)
}
@InternalApi
private[akka] final case class CompositeEffect[Event, State](
persistingEffect: Option[Effect[Event, State]],
_sideEffects: im.Seq[ChainableEffect[Event, State]]) extends Effect[Event, State] {
override val events = persistingEffect.map(_.events).getOrElse(Nil) override val events = persistingEffect.map(_.events).getOrElse(Nil)
override def andThen(additionalSideEffects: ChainableEffect[_, State]*): CompositeEffect[Event, State] =
copy(sideEffects = sideEffects ++ additionalSideEffects) override def sideEffects[E >: Event]: im.Seq[ChainableEffect[E, State]] = _sideEffects.asInstanceOf[im.Seq[ChainableEffect[E, State]]]
}
object CompositeEffect {
def apply[Event, State](persistAll: PersistAll[Event, State], sideEffects: im.Seq[ChainableEffect[_, State]]): CompositeEffect[Event, State] =
CompositeEffect(Some(persistAll), sideEffects)
} }
case class PersistNothing[Event, State]() extends Effect[Event, State] @InternalApi
private[akka] case object PersistNothing extends Effect[Nothing, Nothing]
case class Persist[Event, State](event: Event) extends Effect[Event, State] { @InternalApi
private[akka] case class Persist[Event, State](event: Event) extends Effect[Event, State] {
override val events = event :: Nil override val events = event :: Nil
} }
case class PersistAll[Event, State](override val events: im.Seq[Event]) extends Effect[Event, State] @InternalApi
private[akka] case class PersistAll[Event, State](override val events: im.Seq[Event]) extends Effect[Event, State]
trait ChainableEffect[Event, State] {
self: Effect[Event, State]
}
case class SideEffect[Event, State](effect: State Unit) extends Effect[Event, State] with ChainableEffect[Event, State]
case class Stop[Event, State]() extends Effect[Event, State] with ChainableEffect[Event, State]()
case class Unhandled[Event, State]() extends Effect[Event, State]
type CommandHandler[Command, Event, State] = Function3[ActorContext[Command], Command, State, Effect[Event, State]]
type SignalHandler[Command, Event, State] = PartialFunction[(ActorContext[Command], Signal, State), Effect[Event, State]]
/** /**
* `Actions` defines command handlers and partial function for other signals, * Not for user extension
*/
@DoNotInherit
sealed abstract class ChainableEffect[Event, State] extends Effect[Event, State]
@InternalApi
private[akka] case class SideEffect[Event, State](effect: State Unit) extends ChainableEffect[Event, State]
@InternalApi
private[akka] case object Stop extends ChainableEffect[Nothing, Nothing]
@InternalApi
private[akka] case object Unhandled extends Effect[Nothing, Nothing]
type CommandToEffect[Command, Event, State] = (ActorContext[Command], State, Command) Effect[Event, State]
type SignalHandler[Command, Event, State] = PartialFunction[(ActorContext[Command], State, Signal), Effect[Event, State]]
/**
* The `CommandHandler` defines how to act on commands and partial function for other signals,
* e.g. `Termination` messages if `watch` is used. * e.g. `Termination` messages if `watch` is used.
* *
* Note that you can have different actions based on current state by using * Note that you can have different command handlers based on current state by using
* [[Actions#byState]]. * [[CommandHandler#byState]].
*/ */
object Actions { object CommandHandler {
def apply[Command, Event, State](commandHandler: CommandHandler[Command, Event, State]): Actions[Command, Event, State] =
new Actions(commandHandler, Map.empty) /**
* Create a command handler that will be applied for commands.
*
* @see [[Effect]] for possible effects of a command.
*/
// Note: using full parameter type instead of type aliase here to make API more straight forward to figure out in an IDE
def apply[Command, Event, State](commandHandler: (ActorContext[Command], State, Command) Effect[Event, State]): CommandHandler[Command, Event, State] =
new CommandHandler(commandHandler, Map.empty)
/** /**
* Convenience for simple commands that don't need the state and context. * Convenience for simple commands that don't need the state and context.
*
* @see [[Effect]] for possible effects of a command.
*/ */
def command[Command, Event, State](commandHandler: Command Effect[Event, State]): Actions[Command, Event, State] = def command[Command, Event, State](commandHandler: Command Effect[Event, State]): CommandHandler[Command, Event, State] =
apply((_, cmd, _) commandHandler(cmd)) apply((_, _, cmd) commandHandler(cmd))
/** /**
* Select different actions based on current state. * Select different command handlers based on current state.
*/ */
def byState[Command, Event, State](choice: State Actions[Command, Event, State]): Actions[Command, Event, State] = def byState[Command, Event, State](choice: State CommandHandler[Command, Event, State]): CommandHandler[Command, Event, State] =
new ByStateActions(choice, signalHandler = PartialFunction.empty) new ByStateCommandHandler(choice, signalHandler = PartialFunction.empty)
} }
/** /**
* INTERNAL API * INTERNAL API
*/ */
@InternalApi private[akka] final class ByStateActions[Command, Event, State]( @InternalApi private[akka] final class ByStateCommandHandler[Command, Event, State](
choice: State Actions[Command, Event, State], choice: State CommandHandler[Command, Event, State],
signalHandler: SignalHandler[Command, Event, State]) signalHandler: SignalHandler[Command, Event, State])
extends Actions[Command, Event, State]( extends CommandHandler[Command, Event, State](
commandHandler = (ctx, cmd, state) choice(state).commandHandler(ctx, cmd, state), commandHandler = (ctx, state, cmd) choice(state).commandHandler(ctx, state, cmd),
signalHandler) { signalHandler) {
// SignalHandler may be registered in the wrapper or in the wrapped // SignalHandler may be registered in the wrapper or in the wrapped
private[akka] override def sigHandler(state: State): SignalHandler[Command, Event, State] = private[akka] override def sigHandler(state: State): SignalHandler[Command, Event, State] =
choice(state).sigHandler(state).orElse(signalHandler) choice(state).sigHandler(state).orElse(signalHandler)
// override to preserve the ByStateActions // override to preserve the ByStateCommandHandler
private[akka] override def withSignalHandler( private[akka] override def withSignalHandler(
handler: SignalHandler[Command, Event, State]): Actions[Command, Event, State] = handler: SignalHandler[Command, Event, State]): CommandHandler[Command, Event, State] =
new ByStateActions(choice, handler) new ByStateCommandHandler(choice, handler)
} }
/** /**
* `Actions` defines command handlers and partial function for other signals, * `CommandHandler` defines command handlers and partial function for other signals,
* e.g. `Termination` messages if `watch` is used. * e.g. `Termination` messages if `watch` is used.
* `Actions` is an immutable class. * `CommandHandler` is an immutable class.
*/ */
@DoNotInherit class Actions[Command, Event, State] private[akka] ( @DoNotInherit class CommandHandler[Command, Event, State] private[akka] (
val commandHandler: CommandHandler[Command, Event, State], val commandHandler: CommandToEffect[Command, Event, State],
val signalHandler: SignalHandler[Command, Event, State]) { val signalHandler: SignalHandler[Command, Event, State]) {
@InternalApi private[akka] def sigHandler(state: State): SignalHandler[Command, Event, State] = @InternalApi private[akka] def sigHandler(state: State): SignalHandler[Command, Event, State] =
signalHandler signalHandler
def onSignal(handler: SignalHandler[Command, Event, State]): Actions[Command, Event, State] = // Note: using full parameter type instead of type alias here to make API more straight forward to figure out in an IDE
def onSignal(handler: PartialFunction[(ActorContext[Command], State, Signal), Effect[Event, State]]): CommandHandler[Command, Event, State] =
withSignalHandler(signalHandler.orElse(handler)) withSignalHandler(signalHandler.orElse(handler))
/** INTERNAL API */ /** INTERNAL API */
@InternalApi private[akka] def withSignalHandler( @InternalApi private[akka] def withSignalHandler(
handler: SignalHandler[Command, Event, State]): Actions[Command, Event, State] = handler: SignalHandler[Command, Event, State]): CommandHandler[Command, Event, State] =
new Actions(commandHandler, handler) new CommandHandler(commandHandler, handler)
} }
@ -159,9 +220,9 @@ object PersistentActor {
class PersistentBehavior[Command, Event, State]( class PersistentBehavior[Command, Event, State](
@InternalApi private[akka] val persistenceIdFromActorName: String String, @InternalApi private[akka] val persistenceIdFromActorName: String String,
val initialState: State, val initialState: State,
val actions: PersistentActor.Actions[Command, Event, State], val commandHandler: PersistentActor.CommandHandler[Command, Event, State],
val applyEvent: (Event, State) State, val eventHandler: (State, Event) State,
val recoveryCompleted: (ActorContext[Command], State) State) extends UntypedBehavior[Command] { val recoveryCompleted: (ActorContext[Command], State) Unit) extends UntypedBehavior[Command] {
import PersistentActor._ import PersistentActor._
/** INTERNAL API */ /** INTERNAL API */
@ -171,7 +232,7 @@ class PersistentBehavior[Command, Event, State](
* The `callback` function is called to notify the actor that the recovery process * The `callback` function is called to notify the actor that the recovery process
* is finished. * is finished.
*/ */
def onRecoveryCompleted(callback: (ActorContext[Command], State) State): PersistentBehavior[Command, Event, State] = def onRecoveryCompleted(callback: (ActorContext[Command], State) Unit): PersistentBehavior[Command, Event, State] =
copy(recoveryCompleted = callback) copy(recoveryCompleted = callback)
/** /**
@ -185,10 +246,10 @@ class PersistentBehavior[Command, Event, State](
def snapshotOn(predicate: (State, Event) Boolean): PersistentBehavior[Command, Event, State] = ??? def snapshotOn(predicate: (State, Event) Boolean): PersistentBehavior[Command, Event, State] = ???
private def copy( private def copy(
persistenceIdFromActorName: String String = persistenceIdFromActorName, persistenceIdFromActorName: String String = persistenceIdFromActorName,
initialState: State = initialState, initialState: State = initialState,
actions: Actions[Command, Event, State] = actions, commandHandler: CommandHandler[Command, Event, State] = commandHandler,
applyEvent: (Event, State) State = applyEvent, eventHandler: (State, Event) State = eventHandler,
recoveryCompleted: (ActorContext[Command], State) State = recoveryCompleted): PersistentBehavior[Command, Event, State] = recoveryCompleted: (ActorContext[Command], State) Unit = recoveryCompleted): PersistentBehavior[Command, Event, State] =
new PersistentBehavior(persistenceIdFromActorName, initialState, actions, applyEvent, recoveryCompleted) new PersistentBehavior(persistenceIdFromActorName, initialState, commandHandler, eventHandler, recoveryCompleted)
} }