Merge pull request #25804 from akka/wip-25750-AbstractBehavior-patriknw
Rename MutableBehavior to AbstractBehavior, #25750
This commit is contained in:
commit
abb3429bc8
22 changed files with 212 additions and 138 deletions
|
|
@ -18,7 +18,7 @@ public class ReceiveBuilderTest extends JUnitSuite {
|
|||
|
||||
@Test
|
||||
public void testMutableCounter() {
|
||||
Behavior<BehaviorBuilderTest.CounterMessage> mutable = Behaviors.setup(ctx -> new MutableBehavior<BehaviorBuilderTest.CounterMessage>() {
|
||||
Behavior<BehaviorBuilderTest.CounterMessage> mutable = Behaviors.setup(ctx -> new AbstractBehavior<BehaviorBuilderTest.CounterMessage>() {
|
||||
int currentValue = 0;
|
||||
|
||||
private Behavior<BehaviorBuilderTest.CounterMessage> receiveIncrease(BehaviorBuilderTest.Increase msg) {
|
||||
|
|
@ -32,7 +32,7 @@ public class ReceiveBuilderTest extends JUnitSuite {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Behaviors.Receive<BehaviorBuilderTest.CounterMessage> createReceive() {
|
||||
public Receive<BehaviorBuilderTest.CounterMessage> createReceive() {
|
||||
return receiveBuilder()
|
||||
.onMessage(BehaviorBuilderTest.Increase.class, this::receiveIncrease)
|
||||
.onMessage(BehaviorBuilderTest.Get.class, this::receiveGet)
|
||||
|
|
@ -41,16 +41,16 @@ public class ReceiveBuilderTest extends JUnitSuite {
|
|||
});
|
||||
}
|
||||
|
||||
private static class MyMutableBehavior extends MutableBehavior<BehaviorBuilderTest.CounterMessage> {
|
||||
private static class MyAbstractBehavior extends AbstractBehavior<BehaviorBuilderTest.CounterMessage> {
|
||||
private int value;
|
||||
|
||||
public MyMutableBehavior(int initialValue) {
|
||||
public MyAbstractBehavior(int initialValue) {
|
||||
super();
|
||||
this.value = initialValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Behaviors.Receive<BehaviorBuilderTest.CounterMessage> createReceive() {
|
||||
public Receive<BehaviorBuilderTest.CounterMessage> createReceive() {
|
||||
assertEquals(42, value);
|
||||
return receiveBuilder().build();
|
||||
}
|
||||
|
|
@ -58,7 +58,7 @@ public class ReceiveBuilderTest extends JUnitSuite {
|
|||
|
||||
@Test
|
||||
public void testInitializationOrder() throws Exception {
|
||||
MyMutableBehavior mutable = new MyMutableBehavior(42);
|
||||
MyAbstractBehavior mutable = new MyAbstractBehavior(42);
|
||||
assertEquals(Behaviors.unhandled(), mutable.receive(null, new BehaviorBuilderTest.Increase()));
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -170,7 +170,7 @@ public class InteractionPatternsTest extends JUnitSuite {
|
|||
}
|
||||
}
|
||||
|
||||
public static class Translator extends MutableBehavior<Command> {
|
||||
public static class Translator extends AbstractBehavior<Command> {
|
||||
private final ActorContext<Command> ctx;
|
||||
private final ActorRef<Backend.Request> backend;
|
||||
private final ActorRef<Backend.Response> backendResponseAdapter;
|
||||
|
|
@ -194,7 +194,7 @@ public class InteractionPatternsTest extends JUnitSuite {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Behaviors.Receive<Command> createReceive() {
|
||||
public Receive<Command> createReceive() {
|
||||
return receiveBuilder()
|
||||
.onMessage(Translate.class, cmd -> {
|
||||
taskIdCounter += 1;
|
||||
|
|
@ -522,7 +522,7 @@ public class InteractionPatternsTest extends JUnitSuite {
|
|||
}
|
||||
|
||||
// per session actor behavior
|
||||
class PrepareToLeaveHome extends MutableBehavior<Object> {
|
||||
class PrepareToLeaveHome extends AbstractBehavior<Object> {
|
||||
private final String whoIsLeaving;
|
||||
private final ActorRef<ReadyToLeaveHome> respondTo;
|
||||
private final ActorRef<GetKeys> keyCabinet;
|
||||
|
|
@ -537,7 +537,7 @@ public class InteractionPatternsTest extends JUnitSuite {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Behaviors.Receive<Object> createReceive() {
|
||||
public Receive<Object> createReceive() {
|
||||
return receiveBuilder()
|
||||
.onMessage(Wallet.class, (wallet) -> {
|
||||
this.wallet = Optional.of(wallet);
|
||||
|
|
|
|||
|
|
@ -10,17 +10,17 @@ import akka.actor.typed.ActorRef;
|
|||
import akka.actor.typed.ActorSystem;
|
||||
import akka.actor.typed.Behavior;
|
||||
import akka.actor.typed.Terminated;
|
||||
import akka.actor.typed.javadsl.Behaviors;
|
||||
import akka.actor.typed.javadsl.Behaviors.Receive;
|
||||
import akka.actor.typed.javadsl.AbstractBehavior;
|
||||
import akka.actor.typed.javadsl.ActorContext;
|
||||
import akka.actor.typed.javadsl.MutableBehavior;
|
||||
import akka.actor.typed.javadsl.Behaviors;
|
||||
import akka.actor.typed.javadsl.Receive;
|
||||
//#imports
|
||||
import java.net.URLEncoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
public class MutableIntroTest {
|
||||
public class OOIntroTest {
|
||||
|
||||
//#chatroom-actor
|
||||
public static class ChatRoom {
|
||||
|
|
@ -89,7 +89,7 @@ public class MutableIntroTest {
|
|||
return Behaviors.setup(ChatRoomBehavior::new);
|
||||
}
|
||||
|
||||
public static class ChatRoomBehavior extends MutableBehavior<RoomCommand> {
|
||||
public static class ChatRoomBehavior extends AbstractBehavior<RoomCommand> {
|
||||
final ActorContext<RoomCommand> ctx;
|
||||
final List<ActorRef<SessionCommand>> sessions = new ArrayList<>();
|
||||
|
||||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.actor.typed
|
||||
|
||||
import akka.actor.typed.scaladsl.{ Behaviors ⇒ SBehaviors }
|
||||
import akka.actor.typed.scaladsl.{ MutableBehavior ⇒ SMutableBehavior }
|
||||
import akka.actor.typed.scaladsl.{ AbstractBehavior ⇒ SAbstractBehavior }
|
||||
import akka.actor.typed.javadsl.{ ActorContext ⇒ JActorContext, Behaviors ⇒ JBehaviors }
|
||||
import akka.japi.function.{ Function ⇒ F1e, Function2 ⇒ F2, Procedure2 ⇒ P2 }
|
||||
import akka.japi.pf.{ FI, PFBuilder }
|
||||
|
|
@ -451,7 +451,7 @@ class MutableScalaBehaviorSpec extends Messages with Become with Stoppable {
|
|||
|
||||
def behv(monitor: ActorRef[Event]): Behavior[Command] =
|
||||
SBehaviors.setup[Command] { ctx ⇒
|
||||
new SMutableBehavior[Command] {
|
||||
new SAbstractBehavior[Command] {
|
||||
private var state: State = StateA
|
||||
|
||||
override def onMessage(msg: Command): Behavior[Command] = {
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import java.io.IOException
|
|||
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger }
|
||||
|
||||
import akka.actor.ActorInitializationException
|
||||
import akka.actor.typed.scaladsl.{ Behaviors, MutableBehavior }
|
||||
import akka.actor.typed.scaladsl.{ Behaviors, AbstractBehavior }
|
||||
import akka.actor.typed.scaladsl.Behaviors._
|
||||
import akka.testkit.EventFilter
|
||||
import akka.actor.testkit.typed.scaladsl._
|
||||
|
|
@ -64,7 +64,7 @@ object SupervisionSpec {
|
|||
Behaviors.same
|
||||
}
|
||||
|
||||
class FailingConstructor(monitor: ActorRef[Event]) extends MutableBehavior[Command] {
|
||||
class FailingConstructor(monitor: ActorRef[Event]) extends AbstractBehavior[Command] {
|
||||
monitor ! Started
|
||||
throw new RuntimeException("simulated exc from constructor") with NoStackTrace
|
||||
|
||||
|
|
@ -257,7 +257,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
|
|||
|
||||
class FailingConstructorTestSetup(failCount: Int) {
|
||||
val failCounter = new AtomicInteger(0)
|
||||
class FailingConstructor(monitor: ActorRef[Event]) extends MutableBehavior[Command] {
|
||||
class FailingConstructor(monitor: ActorRef[Event]) extends AbstractBehavior[Command] {
|
||||
monitor ! Started
|
||||
if (failCounter.getAndIncrement() < failCount) {
|
||||
throw TE("simulated exc from constructor")
|
||||
|
|
@ -732,7 +732,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
|
|||
}
|
||||
}
|
||||
|
||||
"fail when exception from MutableBehavior constructor" in new FailingConstructorTestSetup(failCount = 1) {
|
||||
"fail when exception from AbstractBehavior constructor" in new FailingConstructorTestSetup(failCount = 1) {
|
||||
val probe = TestProbe[Event]("evt")
|
||||
val behv = supervise(setup[Command](_ ⇒ new FailingConstructor(probe.ref)))
|
||||
.onFailure[Exception](SupervisorStrategy.restart)
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.actor.typed
|
|||
|
||||
import akka.Done
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.actor.typed.scaladsl.MutableBehavior
|
||||
import akka.actor.typed.scaladsl.AbstractBehavior
|
||||
import akka.actor.typed.scaladsl.adapter._
|
||||
import akka.testkit.EventFilter
|
||||
import akka.actor.testkit.typed.scaladsl.TestProbe
|
||||
|
|
@ -28,7 +28,7 @@ object WatchSpec {
|
|||
case (_, Stop) ⇒ Behaviors.stopped
|
||||
}
|
||||
|
||||
val mutableTerminatorBehavior = new MutableBehavior[Stop.type] {
|
||||
val mutableTerminatorBehavior = new AbstractBehavior[Stop.type] {
|
||||
override def onMessage(msg: Stop.type) = msg match {
|
||||
case Stop ⇒ Behaviors.stopped
|
||||
}
|
||||
|
|
|
|||
|
|
@ -120,7 +120,7 @@ object StashSpec {
|
|||
active(Vector.empty)
|
||||
}
|
||||
|
||||
class MutableStash(ctx: ActorContext[Command]) extends MutableBehavior[Command] {
|
||||
class MutableStash(ctx: ActorContext[Command]) extends AbstractBehavior[Command] {
|
||||
|
||||
private val buffer = StashBuffer.apply[Command](capacity = 10)
|
||||
private var stashing = false
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import java.net.URLEncoder
|
|||
import java.nio.charset.StandardCharsets
|
||||
|
||||
import akka.actor.typed._
|
||||
import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, MutableBehavior }
|
||||
import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, AbstractBehavior }
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -17,7 +17,7 @@ import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
|
|||
import org.scalatest.WordSpecLike
|
||||
//#imports
|
||||
|
||||
object MutableIntroSpec {
|
||||
object OOIntroSpec {
|
||||
|
||||
//#chatroom-actor
|
||||
object ChatRoom {
|
||||
|
|
@ -46,7 +46,7 @@ object MutableIntroSpec {
|
|||
def behavior(): Behavior[RoomCommand] =
|
||||
Behaviors.setup[RoomCommand](ctx ⇒ new ChatRoomBehavior(ctx))
|
||||
|
||||
class ChatRoomBehavior(ctx: ActorContext[RoomCommand]) extends MutableBehavior[RoomCommand] {
|
||||
class ChatRoomBehavior(ctx: ActorContext[RoomCommand]) extends AbstractBehavior[RoomCommand] {
|
||||
private var sessions: List[ActorRef[SessionCommand]] = List.empty
|
||||
|
||||
override def onMessage(msg: RoomCommand): Behavior[RoomCommand] = {
|
||||
|
|
@ -87,9 +87,9 @@ object MutableIntroSpec {
|
|||
|
||||
}
|
||||
|
||||
class MutableIntroSpec extends ScalaTestWithActorTestKit with WordSpecLike {
|
||||
class OOIntroSpec extends ScalaTestWithActorTestKit with WordSpecLike {
|
||||
|
||||
import MutableIntroSpec._
|
||||
import OOIntroSpec._
|
||||
|
||||
"A chat room" must {
|
||||
"chat" in {
|
||||
|
|
@ -5,21 +5,24 @@
|
|||
package akka.actor.typed.javadsl
|
||||
|
||||
import akka.actor.typed.{ Behavior, ExtensibleBehavior, Signal }
|
||||
import akka.actor.typed.javadsl.Behaviors.Receive
|
||||
import akka.util.OptionVal
|
||||
|
||||
/**
|
||||
* Mutable behavior can be implemented by extending this class and implement the
|
||||
* abstract method [[MutableBehavior#onMessage]] and optionally override
|
||||
* [[MutableBehavior#onSignal]].
|
||||
* An actor `Behavior` can be implemented by extending this class and implement the
|
||||
* abstract method [[AbstractBehavior#createReceive]]. Mutable state can be defined
|
||||
* as instance variables of the class.
|
||||
*
|
||||
* Instances of this behavior should be created via [[Behaviors#setup]] and if
|
||||
* This is an Object-oriented style of defining a `Behavior`. A more functional style
|
||||
* alternative is provided by the factory methods in [[Behaviors]], for example
|
||||
* [[Behaviors.receiveMessage]].
|
||||
*
|
||||
* Instances of this behavior should be created via [[Behaviors.setup]] and if
|
||||
* the [[ActorContext]] is needed it can be passed as a constructor parameter
|
||||
* from the factory function.
|
||||
*
|
||||
* @see [[Behaviors#setup]]
|
||||
* @see [[Behaviors.setup]]
|
||||
*/
|
||||
abstract class MutableBehavior[T] extends ExtensibleBehavior[T] {
|
||||
abstract class AbstractBehavior[T] extends ExtensibleBehavior[T] {
|
||||
private var _receive: OptionVal[Receive[T]] = OptionVal.None
|
||||
private def receive: Receive[T] = _receive match {
|
||||
case OptionVal.None ⇒
|
||||
|
|
@ -37,7 +40,15 @@ abstract class MutableBehavior[T] extends ExtensibleBehavior[T] {
|
|||
override final def receiveSignal(ctx: akka.actor.typed.ActorContext[T], msg: Signal): Behavior[T] =
|
||||
receive.receiveSignal(ctx, msg)
|
||||
|
||||
/**
|
||||
* Implement this to define how messages and signals are processed. Use the
|
||||
* [[AbstractBehavior.receiveBuilder]] to define the message dispatch.
|
||||
*/
|
||||
def createReceive: Receive[T]
|
||||
|
||||
/**
|
||||
* Create a [[ReceiveBuilder]] to define the message dispatch of the `Behavior`.
|
||||
* Typically used from [[AbstractBehavior.createReceive]].
|
||||
*/
|
||||
def receiveBuilder: ReceiveBuilder[T] = ReceiveBuilder.create
|
||||
}
|
||||
|
|
@ -234,7 +234,7 @@ trait ActorContext[T] extends akka.actor.typed.ActorContext[T] {
|
|||
*
|
||||
* A message adapter (and the returned `ActorRef`) has the same lifecycle as
|
||||
* this actor. It's recommended to register the adapters in a top level
|
||||
* `Behaviors.setup` or constructor of `MutableBehavior` but it's possible to
|
||||
* `Behaviors.setup` or constructor of `AbstractBehavior` but it's possible to
|
||||
* register them later also if needed. Message adapters don't have to be stopped since
|
||||
* they consume no resources other than an entry in an internal `Map` and the number
|
||||
* of adapters are bounded since it's only possible to have one per message class.
|
||||
|
|
|
|||
|
|
@ -7,10 +7,10 @@ package akka.actor.typed.javadsl
|
|||
import java.util.Collections
|
||||
import java.util.function.{ Function ⇒ JFunction }
|
||||
|
||||
import akka.actor.typed.{ ActorRef, Behavior, BehaviorInterceptor, ExtensibleBehavior, Signal, SupervisorStrategy }
|
||||
import akka.actor.typed.{ ActorRef, Behavior, BehaviorInterceptor, Signal, SupervisorStrategy }
|
||||
import akka.actor.typed.internal.{ BehaviorImpl, Supervisor, TimerSchedulerImpl, WithMdcBehaviorInterceptor }
|
||||
import akka.actor.typed.scaladsl
|
||||
import akka.annotation.{ ApiMayChange, DoNotInherit }
|
||||
import akka.annotation.ApiMayChange
|
||||
import akka.japi.function.{ Function2 ⇒ JapiFunction2 }
|
||||
import akka.japi.pf.PFBuilder
|
||||
|
||||
|
|
@ -97,11 +97,10 @@ object Behaviors {
|
|||
* [[ActorContext]] that allows access to the system, spawning and watching
|
||||
* other actors, etc.
|
||||
*
|
||||
* This constructor is called immutable because the behavior instance doesn't
|
||||
* have or close over any mutable state. Processing the next message
|
||||
* results in a new behavior that can potentially be different from this one.
|
||||
* State is updated by returning a new behavior that holds the new immutable
|
||||
* state.
|
||||
* Compared to using [[AbstractBehavior]] this factory is a more functional style
|
||||
* of defining the `Behavior`. Processing the next message results in a new behavior
|
||||
* that can potentially be different from this one. State is maintained by returning
|
||||
* a new behavior that holds the new immutable state.
|
||||
*/
|
||||
def receive[T](onMessage: JapiFunction2[ActorContext[T], T, Behavior[T]]): Behavior[T] =
|
||||
new BehaviorImpl.ReceiveBehavior((ctx, msg) ⇒ onMessage.apply(ctx.asJava, msg))
|
||||
|
|
@ -117,11 +116,10 @@ object Behaviors {
|
|||
* [[ActorContext]] that allows access to the system, spawning and watching
|
||||
* other actors, etc.
|
||||
*
|
||||
* This constructor is called immutable because the behavior instance doesn't
|
||||
* have or close over any mutable state. Processing the next message
|
||||
* results in a new behavior that can potentially be different from this one.
|
||||
* State is updated by returning a new behavior that holds the new immutable
|
||||
* state.
|
||||
* Compared to using [[AbstractBehavior]] this factory is a more functional style
|
||||
* of defining the `Behavior`. Processing the next message results in a new behavior
|
||||
* that can potentially be different from this one. State is maintained by returning
|
||||
* a new behavior that holds the new immutable state.
|
||||
*/
|
||||
def receiveMessage[T](onMessage: akka.japi.Function[T, Behavior[T]]): Behavior[T] =
|
||||
new BehaviorImpl.ReceiveBehavior((_, msg) ⇒ onMessage.apply(msg))
|
||||
|
|
@ -133,11 +131,10 @@ object Behaviors {
|
|||
* [[ActorContext]] that allows access to the system, spawning and watching
|
||||
* other actors, etc.
|
||||
*
|
||||
* This constructor is called immutable because the behavior instance doesn't
|
||||
* have or close over any mutable state. Processing the next message
|
||||
* results in a new behavior that can potentially be different from this one.
|
||||
* State is updated by returning a new behavior that holds the new immutable
|
||||
* state.
|
||||
* Compared to using [[AbstractBehavior]] this factory is a more functional style
|
||||
* of defining the `Behavior`. Processing the next message results in a new behavior
|
||||
* that can potentially be different from this one. State is maintained by returning
|
||||
* a new behavior that holds the new immutable state.
|
||||
*/
|
||||
def receive[T](
|
||||
onMessage: JapiFunction2[ActorContext[T], T, Behavior[T]],
|
||||
|
|
@ -151,10 +148,10 @@ object Behaviors {
|
|||
* Constructs an actor behavior builder that can build a behavior that can react to both
|
||||
* incoming messages and lifecycle signals.
|
||||
*
|
||||
* This constructor is called immutable because the behavior instance does not
|
||||
* need and in fact should not use (close over) mutable variables, but instead
|
||||
* return a potentially different behavior encapsulating any state changes.
|
||||
* If no change is desired, use {@link #same}.
|
||||
* Compared to using [[AbstractBehavior]] this factory is a more functional style
|
||||
* of defining the `Behavior`. Processing the next message results in a new behavior
|
||||
* that can potentially be different from this one. State is maintained by returning
|
||||
* a new behavior that holds the new immutable state.
|
||||
*
|
||||
* @param type the supertype of all messages accepted by this behavior
|
||||
* @return the behavior builder
|
||||
|
|
@ -274,10 +271,6 @@ object Behaviors {
|
|||
def withTimers[T](factory: akka.japi.function.Function[TimerScheduler[T], Behavior[T]]): Behavior[T] =
|
||||
TimerSchedulerImpl.withTimers(timers ⇒ factory.apply(timers))
|
||||
|
||||
/** A specialized "receive" behavior that is implemented using message matching builders. */
|
||||
@DoNotInherit
|
||||
trait Receive[T] extends ExtensibleBehavior[T]
|
||||
|
||||
/**
|
||||
* Per message MDC (Mapped Diagnostic Context) logging.
|
||||
*
|
||||
|
|
|
|||
|
|
@ -0,0 +1,58 @@
|
|||
/**
|
||||
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.actor.typed.javadsl
|
||||
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.actor.typed.ExtensibleBehavior
|
||||
import akka.actor.typed.Signal
|
||||
import akka.annotation.DoNotInherit
|
||||
|
||||
/**
|
||||
* A specialized "receive" behavior that is implemented using message matching builders,
|
||||
* such as [[ReceiveBuilder]], from [[AbstractBehavior]].
|
||||
*/
|
||||
@DoNotInherit
|
||||
abstract class Receive[T] extends ExtensibleBehavior[T] {
|
||||
// Note that this class may be opened for extensions to support other builder patterns,
|
||||
// or message dispatch without builders.
|
||||
|
||||
/**
|
||||
* Process an incoming message and return the next behavior.
|
||||
*
|
||||
* The returned behavior can in addition to normal behaviors be one of the
|
||||
* canned special objects:
|
||||
*
|
||||
* * returning `stopped` will terminate this Behavior
|
||||
* * returning `same` designates to reuse the current Behavior
|
||||
* * returning `unhandled` keeps the same Behavior and signals that the message was not yet handled
|
||||
*
|
||||
*/
|
||||
@throws(classOf[Exception])
|
||||
def receiveMessage(msg: T): Behavior[T]
|
||||
|
||||
/**
|
||||
* Process an incoming [[akka.actor.typed.Signal]] and return the next behavior. This means
|
||||
* that all lifecycle hooks, ReceiveTimeout, Terminated and Failed messages
|
||||
* can initiate a behavior change.
|
||||
*
|
||||
* The returned behavior can in addition to normal behaviors be one of the
|
||||
* canned special objects:
|
||||
*
|
||||
* * returning `stopped` will terminate this Behavior
|
||||
* * returning `same` designates to reuse the current Behavior
|
||||
* * returning `unhandled` keeps the same Behavior and signals that the message was not yet handled
|
||||
*/
|
||||
@throws(classOf[Exception])
|
||||
def receiveSignal(sig: Signal): Behavior[T]
|
||||
|
||||
@throws(classOf[Exception])
|
||||
override final def receive(ctx: akka.actor.typed.ActorContext[T], msg: T): Behavior[T] =
|
||||
receiveMessage(msg)
|
||||
|
||||
@throws(classOf[Exception])
|
||||
override final def receiveSignal(ctx: akka.actor.typed.ActorContext[T], sig: Signal): Behavior[T] =
|
||||
receiveSignal(sig)
|
||||
|
||||
}
|
||||
|
|
@ -6,14 +6,11 @@ package akka.actor.typed.javadsl
|
|||
|
||||
import scala.annotation.tailrec
|
||||
import akka.japi.function.{ Creator, Function, Predicate }
|
||||
import akka.actor.typed.javadsl.Behaviors.Receive
|
||||
import akka.actor.typed.{ Behavior, Signal }
|
||||
import ReceiveBuilder._
|
||||
import akka.actor.typed
|
||||
import akka.annotation.InternalApi
|
||||
|
||||
/**
|
||||
* Used when implementing [[MutableBehavior]].
|
||||
* Used when implementing [[AbstractBehavior]].
|
||||
*
|
||||
* When handling a message or signal, this [[Behavior]] will consider all handlers in the order they were added,
|
||||
* looking for the first handler for which both the type and the (optional) predicate match.
|
||||
|
|
@ -21,10 +18,12 @@ import akka.annotation.InternalApi
|
|||
* @tparam T the common superclass of all supported messages.
|
||||
*/
|
||||
class ReceiveBuilder[T] private (
|
||||
private val messageHandlers: List[Case[T, T]],
|
||||
private val signalHandlers: List[Case[T, Signal]]
|
||||
private val messageHandlers: List[ReceiveBuilder.Case[T, T]],
|
||||
private val signalHandlers: List[ReceiveBuilder.Case[T, Signal]]
|
||||
) {
|
||||
|
||||
import ReceiveBuilder.Case
|
||||
|
||||
def build(): Receive[T] = new BuiltReceive(messageHandlers.reverse, signalHandlers.reverse)
|
||||
|
||||
/**
|
||||
|
|
@ -144,18 +143,17 @@ object ReceiveBuilder {
|
|||
}
|
||||
|
||||
/**
|
||||
* Receive type for [[MutableBehavior]]
|
||||
* Receive type for [[AbstractBehavior]]
|
||||
*/
|
||||
private final class BuiltReceive[T](
|
||||
private val messageHandlers: List[Case[T, T]],
|
||||
private val signalHandlers: List[Case[T, Signal]]
|
||||
private val messageHandlers: List[ReceiveBuilder.Case[T, T]],
|
||||
private val signalHandlers: List[ReceiveBuilder.Case[T, Signal]]
|
||||
) extends Receive[T] {
|
||||
import ReceiveBuilder.Case
|
||||
|
||||
override def receive(ctx: typed.ActorContext[T], msg: T): Behavior[T] = receive[T](msg, messageHandlers)
|
||||
// override def receiveMessage(msg: T): Behavior[T] = receive[T](msg, messageHandlers)
|
||||
override def receiveMessage(msg: T): Behavior[T] = receive[T](msg, messageHandlers)
|
||||
|
||||
override def receiveSignal(ctx: typed.ActorContext[T], msg: Signal): Behavior[T] = receive[Signal](msg, signalHandlers)
|
||||
// override def receiveSignal(msg: Signal): Behavior[T] = receive[Signal](msg, signalHandlers)
|
||||
override def receiveSignal(msg: Signal): Behavior[T] = receive[Signal](msg, signalHandlers)
|
||||
|
||||
@tailrec
|
||||
private def receive[M](msg: M, handlers: List[Case[T, M]]): Behavior[T] =
|
||||
|
|
|
|||
|
|
@ -7,20 +7,22 @@ package akka.actor.typed.scaladsl
|
|||
import akka.actor.typed.{ Behavior, ExtensibleBehavior, Signal }
|
||||
|
||||
/**
|
||||
* Mutable behavior can be implemented by extending this class and implement the
|
||||
* abstract method [[MutableBehavior#onMessage]] and optionally override
|
||||
* [[MutableBehavior#onSignal]].
|
||||
* An actor `Behavior` can be implemented by extending this class and implement the
|
||||
* abstract method [[AbstractBehavior#onMessage]] and optionally override
|
||||
* [[AbstractBehavior#onSignal]]. Mutable state can be defined as instance variables
|
||||
* of the class.
|
||||
*
|
||||
* Instances of this behavior should be created via [[Behaviors#setup]] and if
|
||||
* This is an Object-oriented style of defining a `Behavior`. A more functional style
|
||||
* alternative is provided by the factory methods in [[Behaviors]], for example
|
||||
* [[Behaviors.receiveMessage]].
|
||||
*
|
||||
* Instances of this behavior should be created via [[Behaviors.setup]] and if
|
||||
* the [[ActorContext]] is needed it can be passed as a constructor parameter
|
||||
* from the factory function.
|
||||
*
|
||||
* @see [[Behaviors#setup]]
|
||||
* @see [[Behaviors.setup]]
|
||||
*/
|
||||
abstract class MutableBehavior[T] extends ExtensibleBehavior[T] {
|
||||
@throws(classOf[Exception])
|
||||
override final def receive(ctx: akka.actor.typed.ActorContext[T], msg: T): Behavior[T] =
|
||||
onMessage(msg)
|
||||
abstract class AbstractBehavior[T] extends ExtensibleBehavior[T] {
|
||||
|
||||
/**
|
||||
* Implement this method to process an incoming message and return the next behavior.
|
||||
|
|
@ -36,10 +38,6 @@ abstract class MutableBehavior[T] extends ExtensibleBehavior[T] {
|
|||
@throws(classOf[Exception])
|
||||
def onMessage(msg: T): Behavior[T]
|
||||
|
||||
@throws(classOf[Exception])
|
||||
override final def receiveSignal(ctx: akka.actor.typed.ActorContext[T], msg: Signal): Behavior[T] =
|
||||
onSignal.applyOrElse(msg, { case _ ⇒ Behavior.unhandled }: PartialFunction[Signal, Behavior[T]])
|
||||
|
||||
/**
|
||||
* Override this method to process an incoming [[akka.actor.typed.Signal]] and return the next behavior.
|
||||
* This means that all lifecycle hooks, ReceiveTimeout, Terminated and Failed messages
|
||||
|
|
@ -55,4 +53,12 @@ abstract class MutableBehavior[T] extends ExtensibleBehavior[T] {
|
|||
*/
|
||||
@throws(classOf[Exception])
|
||||
def onSignal: PartialFunction[Signal, Behavior[T]] = PartialFunction.empty
|
||||
|
||||
@throws(classOf[Exception])
|
||||
override final def receive(ctx: akka.actor.typed.ActorContext[T], msg: T): Behavior[T] =
|
||||
onMessage(msg)
|
||||
|
||||
@throws(classOf[Exception])
|
||||
override final def receiveSignal(ctx: akka.actor.typed.ActorContext[T], msg: Signal): Behavior[T] =
|
||||
onSignal.applyOrElse(msg, { case _ ⇒ Behavior.unhandled }: PartialFunction[Signal, Behavior[T]])
|
||||
}
|
||||
|
|
@ -238,7 +238,7 @@ trait ActorContext[T] extends akka.actor.typed.ActorContext[T] { this: akka.acto
|
|||
*
|
||||
* A message adapter (and the returned `ActorRef`) has the same lifecycle as
|
||||
* this actor. It's recommended to register the adapters in a top level
|
||||
* `Behaviors.setup` or constructor of `MutableBehavior` but it's possible to
|
||||
* `Behaviors.setup` or constructor of `AbstractBehavior` but it's possible to
|
||||
* register them later also if needed. Message adapters don't have to be stopped since
|
||||
* they consume no resources other than an entry in an internal `Map` and the number
|
||||
* of adapters are bounded since it's only possible to have one per message class.
|
||||
|
|
|
|||
|
|
@ -85,9 +85,10 @@ object Behaviors {
|
|||
* [[ActorContext]] that allows access to the system, spawning and watching
|
||||
* other actors, etc.
|
||||
*
|
||||
* This constructor is called immutable because the behavior instance does not
|
||||
* need and in fact should not use (close over) mutable variables, but instead
|
||||
* return a potentially different behavior encapsulating any state changes.
|
||||
* Compared to using [[AbstractBehavior]] this factory is a more functional style
|
||||
* of defining the `Behavior`. Processing the next message results in a new behavior
|
||||
* that can potentially be different from this one. State is maintained by returning
|
||||
* a new behavior that holds the new immutable state.
|
||||
*/
|
||||
def receive[T](onMessage: (ActorContext[T], T) ⇒ Behavior[T]): Receive[T] =
|
||||
new ReceiveImpl(onMessage)
|
||||
|
|
@ -103,15 +104,16 @@ object Behaviors {
|
|||
* [[ActorContext]] that allows access to the system, spawning and watching
|
||||
* other actors, etc.
|
||||
*
|
||||
* This constructor is called immutable because the behavior instance does not
|
||||
* need and in fact should not use (close over) mutable variables, but instead
|
||||
* return a potentially different behavior encapsulating any state changes.
|
||||
* Compared to using [[AbstractBehavior]] this factory is a more functional style
|
||||
* of defining the `Behavior`. Processing the next message results in a new behavior
|
||||
* that can potentially be different from this one. State is maintained by returning
|
||||
* a new behavior that holds the new immutable state.
|
||||
*/
|
||||
def receiveMessage[T](onMessage: T ⇒ Behavior[T]): Receive[T] =
|
||||
new ReceiveMessageImpl(onMessage)
|
||||
|
||||
/**
|
||||
* Construct an immutable actor behavior from a partial message handler which treats undefined messages as unhandled.
|
||||
* Construct an actor `Behavior` from a partial message handler which treats undefined messages as unhandled.
|
||||
*
|
||||
* Behaviors can also be composed with [[Behavior#orElse]].
|
||||
*/
|
||||
|
|
@ -121,7 +123,7 @@ object Behaviors {
|
|||
}
|
||||
|
||||
/**
|
||||
* Construct an immutable actor behavior from a partial message handler which treats undefined messages as unhandled.
|
||||
* Construct an actor `Behavior` from a partial message handler which treats undefined messages as unhandled.
|
||||
*
|
||||
* Behaviors can also be composed with [[Behavior#orElse]].
|
||||
*/
|
||||
|
|
@ -131,7 +133,7 @@ object Behaviors {
|
|||
}
|
||||
|
||||
/**
|
||||
* Construct an actor behavior that can react to lifecycle signals only.
|
||||
* Construct an actor `Behavior` that can react to lifecycle signals only.
|
||||
*/
|
||||
def receiveSignal[T](handler: PartialFunction[(ActorContext[T], Signal), Behavior[T]]): Behavior[T] =
|
||||
receive[T]((_, _) ⇒ same).receiveSignal(handler)
|
||||
|
|
@ -247,18 +249,13 @@ object Behaviors {
|
|||
def withMdc[T](staticMdc: Map[String, Any], mdcForMessage: T ⇒ Map[String, Any])(behavior: Behavior[T]): Behavior[T] =
|
||||
WithMdcBehaviorInterceptor[T](staticMdc, mdcForMessage, behavior)
|
||||
|
||||
// TODO
|
||||
// final case class Selective[T](timeout: FiniteDuration, selector: PartialFunction[T, Behavior[T]], onTimeout: () ⇒ Behavior[T])
|
||||
|
||||
/**
|
||||
* Immutable behavior that exposes additional fluent DSL methods
|
||||
* to further change the message or signal reception behavior.
|
||||
* `Behavior` that exposes additional fluent DSL methods to further change the message or
|
||||
* signal reception behavior. It's returned by for example [[Behaviors.receiveMessage]].
|
||||
*/
|
||||
@DoNotInherit
|
||||
trait Receive[T] extends ExtensibleBehavior[T] {
|
||||
def receiveSignal(onSignal: PartialFunction[(ActorContext[T], Signal), Behavior[T]]): Behavior[T]
|
||||
|
||||
// TODO orElse can be defined here
|
||||
}
|
||||
|
||||
@InternalApi
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.actor.typed
|
|||
import java.util.concurrent.{ CountDownLatch, TimeUnit }
|
||||
|
||||
import akka.Done
|
||||
import akka.actor.typed.scaladsl.{ Behaviors, MutableBehavior }
|
||||
import akka.actor.typed.scaladsl.{ Behaviors, AbstractBehavior }
|
||||
import akka.actor.typed.scaladsl.{ ActorContext ⇒ SActorContext }
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
|
|
|||
|
|
@ -25,8 +25,9 @@ import akka.actor.typed.Behavior;
|
|||
import akka.cluster.ddata.typed.javadsl.Replicator.Command;
|
||||
import akka.actor.typed.javadsl.Behaviors;
|
||||
import akka.actor.typed.javadsl.Adapter;
|
||||
import akka.actor.typed.javadsl.MutableBehavior;
|
||||
import akka.actor.typed.javadsl.AbstractBehavior;
|
||||
import akka.actor.typed.javadsl.ActorContext;
|
||||
import akka.actor.typed.javadsl.Receive;
|
||||
|
||||
// #sample
|
||||
|
||||
|
|
@ -83,7 +84,7 @@ public class ReplicatorTest extends JUnitSuite {
|
|||
|
||||
static final Key<GCounter> Key = GCounterKey.create("counter");
|
||||
|
||||
static class Counter extends MutableBehavior<ClientCommand> {
|
||||
static class Counter extends AbstractBehavior<ClientCommand> {
|
||||
private final ActorRef<Replicator.Command> replicator;
|
||||
private final Cluster node;
|
||||
final ActorRef<Replicator.UpdateResponse<GCounter>> updateResponseAdapter;
|
||||
|
|
@ -132,7 +133,7 @@ public class ReplicatorTest extends JUnitSuite {
|
|||
// #sample
|
||||
|
||||
@Override
|
||||
public Behaviors.Receive<ClientCommand> createReceive() {
|
||||
public Receive<ClientCommand> createReceive() {
|
||||
return receiveBuilder()
|
||||
.onMessage(Increment.class, this::onIncrement)
|
||||
.onMessage(InternalUpdateResponse.class, msg -> Behaviors.same())
|
||||
|
|
|
|||
|
|
@ -10,7 +10,8 @@ import akka.actor.typed.ActorSystem;
|
|||
import akka.actor.typed.Behavior;
|
||||
import akka.actor.typed.javadsl.ActorContext;
|
||||
import akka.actor.typed.javadsl.Behaviors;
|
||||
import akka.actor.typed.javadsl.MutableBehavior;
|
||||
import akka.actor.typed.javadsl.AbstractBehavior;
|
||||
import akka.actor.typed.javadsl.Receive;
|
||||
import akka.actor.typed.receptionist.Receptionist;
|
||||
import akka.actor.typed.receptionist.ServiceKey;
|
||||
import akka.cluster.ClusterEvent;
|
||||
|
|
@ -32,7 +33,7 @@ public class ReceptionistExampleTest extends JUnitSuite {
|
|||
|
||||
static class RandomRouter {
|
||||
|
||||
private static class RouterBehavior<T> extends MutableBehavior<Object> {
|
||||
private static class RouterBehavior<T> extends AbstractBehavior<Object> {
|
||||
private final Class<T> messageClass;
|
||||
private final ServiceKey<T> serviceKey;
|
||||
private final List<ActorRef<T>> routees = new ArrayList<>();
|
||||
|
|
@ -44,7 +45,7 @@ public class ReceptionistExampleTest extends JUnitSuite {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Behaviors.Receive<Object> createReceive() {
|
||||
public Receive<Object> createReceive() {
|
||||
return receiveBuilder()
|
||||
.onMessage(Receptionist.Listing.class, listing -> listing.isForKey(serviceKey), (listing) -> {
|
||||
routees.clear();
|
||||
|
|
@ -74,7 +75,7 @@ public class ReceptionistExampleTest extends JUnitSuite {
|
|||
}
|
||||
}
|
||||
|
||||
private static class ClusterRouterBehavior<T> extends MutableBehavior<Object> {
|
||||
private static class ClusterRouterBehavior<T> extends AbstractBehavior<Object> {
|
||||
private final Class<T> messageClass;
|
||||
private final ServiceKey<T> serviceKey;
|
||||
private final List<ActorRef<T>> routees = new ArrayList<>();
|
||||
|
|
@ -110,7 +111,7 @@ public class ReceptionistExampleTest extends JUnitSuite {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Behaviors.Receive<Object> createReceive() {
|
||||
public Receive<Object> createReceive() {
|
||||
return receiveBuilder()
|
||||
.onMessage(Receptionist.Listing.class, listing -> listing.isForKey(serviceKey), listing -> {
|
||||
routees.clear();
|
||||
|
|
|
|||
|
|
@ -139,8 +139,6 @@ The console output may look like this:
|
|||
|
||||
## A More Complex Example
|
||||
|
||||
### Functional Style
|
||||
|
||||
The next example is more realistic and demonstrates some important patterns:
|
||||
|
||||
* Using a sealed trait and case class/objects to represent multiple messages an actor can receive
|
||||
|
|
@ -148,6 +146,12 @@ The next example is more realistic and demonstrates some important patterns:
|
|||
* Handling state by changing behavior
|
||||
* Using multiple typed actors to represent different parts of a protocol in a type safe way
|
||||
|
||||
### Functional Style
|
||||
|
||||
First we will show this example in a functional style, and then the same example is shown with an
|
||||
@ref:[Object-oriented style](#object-oriented-style). Which style you choose to use is a matter of
|
||||
taste and both styles can be mixed depending on which is best for a specific actor.
|
||||
|
||||
Consider an Actor that runs a chat room: client Actors may connect by sending
|
||||
a message that contains their screen name and then they can post messages. The
|
||||
chat room Actor will disseminate all posted messages to all currently connected
|
||||
|
|
@ -281,12 +285,15 @@ where you pass a function to a factory which then constructs a behavior, for sta
|
|||
actors this means passing immutable state around as parameters and switching to a new behavior
|
||||
whenever you need to act on a changed state. An alternative way to express the same is a more
|
||||
object oriented style where a concrete class for the actor behavior is defined and mutable
|
||||
state is kept inside of it as fields.
|
||||
state is kept inside of it as fields. Which style you choose to use is a matter of
|
||||
taste and both styles can be mixed depending on which is best for a specific actor.
|
||||
|
||||
Some reasons why you may want to do this are:
|
||||
Some reasons why you may want to use the object-oriented style:
|
||||
|
||||
@@@ div {.group-java}
|
||||
|
||||
* you are more familiar with an object-oriented style of structuring the code with methods
|
||||
in a class rather than functions
|
||||
* Java lambdas can only close over final or effectively final fields, making it
|
||||
impractical to use this style in behaviors that mutate their fields
|
||||
* some state is not immutable, e.g. immutable collections are not widely used in Java
|
||||
|
|
@ -299,6 +306,8 @@ Some reasons why you may want to do this are:
|
|||
|
||||
@@@ div {.group-scala}
|
||||
|
||||
* you are more familiar with an object-oriented style of structuring the code with methods
|
||||
in a class rather than functions
|
||||
* some state is not immutable
|
||||
* it could be more familiar and easier to migrate existing untyped actors to this style
|
||||
* mutable state can sometimes have better performance, e.g. mutable collections and
|
||||
|
|
@ -307,21 +316,21 @@ Some reasons why you may want to do this are:
|
|||
|
||||
@@@
|
||||
|
||||
#### MutableBehavior API
|
||||
#### AbstractBehavior API
|
||||
|
||||
Defining a class based actor behavior starts with extending
|
||||
@scala[`akka.actor.typed.scaladsl.MutableBehavior[T]`]
|
||||
@java[`akka.actor.typed.javadsl.MutableBehavior<T>`] where `T` is the type of messages
|
||||
@scala[`akka.actor.typed.scaladsl.AbstractBehavior[T]`]
|
||||
@java[`akka.actor.typed.javadsl.AbstractBehavior<T>`] where `T` is the type of messages
|
||||
the behavior will accept.
|
||||
|
||||
Let's repeat the chat room sample from @ref:[A more complex example above](#a-more-complex-example) but implemented
|
||||
using `MutableBehavior`. The protocol for interacting with the actor looks the same:
|
||||
using `AbstractBehavior`. The protocol for interacting with the actor looks the same:
|
||||
|
||||
Scala
|
||||
: @@snip [MutableIntroSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/MutableIntroSpec.scala) { #chatroom-protocol }
|
||||
: @@snip [OOIntroSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/OOIntroSpec.scala) { #chatroom-protocol }
|
||||
|
||||
Java
|
||||
: @@snip [MutableIntroTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/MutableIntroTest.java) { #chatroom-protocol }
|
||||
: @@snip [OOIntroTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/OOIntroTest.java) { #chatroom-protocol }
|
||||
|
||||
Initially the client Actors only get access to an @scala[`ActorRef[GetSession]`]@java[`ActorRef<GetSession>`]
|
||||
which allows them to make the first step. Once a client’s session has been
|
||||
|
|
@ -335,20 +344,20 @@ that the client has revealed its own address, via the `replyTo` argument, so tha
|
|||
This illustrates how Actors can express more than just the equivalent of method
|
||||
calls on Java objects. The declared message types and their contents describe a
|
||||
full protocol that can involve multiple Actors and that can evolve over
|
||||
multiple steps. Here's the `MutableBehavior` implementation of the chat room protocol:
|
||||
multiple steps. Here's the `AbstractBehavior` implementation of the chat room protocol:
|
||||
|
||||
Scala
|
||||
: @@snip [MutableIntroSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/MutableIntroSpec.scala) { #chatroom-behavior }
|
||||
: @@snip [OOIntroSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/OOIntroSpec.scala) { #chatroom-behavior }
|
||||
|
||||
Java
|
||||
: @@snip [MutableIntroTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/MutableIntroTest.java) { #chatroom-behavior }
|
||||
: @@snip [OOIntroTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/OOIntroTest.java) { #chatroom-behavior }
|
||||
|
||||
The state is managed through fields in the class, just like with a regular object oriented class.
|
||||
As the state is mutable, we never return a different behavior from the message logic, but can return
|
||||
the `MutableBehavior` instance itself (`this`) as a behavior to use for processing the next message coming in.
|
||||
the `AbstractBehavior` instance itself (`this`) as a behavior to use for processing the next message coming in.
|
||||
We could also return `Behavior.same` to achieve the same.
|
||||
|
||||
It is also possible to return a new different `MutableBehavior`, for example to represent a different state in a
|
||||
It is also possible to return a new different `AbstractBehavior`, for example to represent a different state in a
|
||||
finite state machine (FSM), or use one of the functional behavior factories to combine the object oriented
|
||||
with the functional style for different parts of the lifecycle of the same Actor behavior.
|
||||
|
||||
|
|
@ -361,7 +370,7 @@ screen name.
|
|||
|
||||
To implement the logic where we spawn a child for the session we need access
|
||||
to the `ActorContext`. This is injected as a constructor parameter upon creation
|
||||
of the behavior, note how we combine the `MutableBehavior` with `Behaviors.setup`
|
||||
of the behavior, note how we combine the `AbstractBehavior` with `Behaviors.setup`
|
||||
to do this in the `behavior` method.
|
||||
|
||||
The behavior that we declare here can handle both subtypes of `RoomCommand`.
|
||||
|
|
@ -390,14 +399,14 @@ problematic, so passing an @scala[`ActorRef[PublishSessionMessage]`]@java[`Actor
|
|||
#### Trying it out
|
||||
|
||||
In order to see this chat room in action we need to write a client Actor that can use it, for this
|
||||
stateless actor it doesn't make much sense to use the `MutableBehavior` so let's just reuse the
|
||||
stateless actor it doesn't make much sense to use the `AbstractBehavior` so let's just reuse the
|
||||
functional style gabbler from the sample above:
|
||||
|
||||
Scala
|
||||
: @@snip [MutableIntroSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/MutableIntroSpec.scala) { #chatroom-gabbler }
|
||||
: @@snip [OOIntroSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/OOIntroSpec.scala) { #chatroom-gabbler }
|
||||
|
||||
Java
|
||||
: @@snip [MutableIntroTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/MutableIntroTest.java) { #chatroom-gabbler }
|
||||
: @@snip [OOIntroTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/OOIntroTest.java) { #chatroom-gabbler }
|
||||
|
||||
Now to try things out we must start both a chat room and a gabbler and of
|
||||
course we do this inside an Actor system. Since there can be only one guardian
|
||||
|
|
@ -408,10 +417,10 @@ choice:
|
|||
|
||||
|
||||
Scala
|
||||
: @@snip [MutableIntroSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/MutableIntroSpec.scala) { #chatroom-main }
|
||||
: @@snip [OOIntroSpec.scala](/akka-actor-typed-tests/src/test/scala/docs/akka/typed/OOIntroSpec.scala) { #chatroom-main }
|
||||
|
||||
Java
|
||||
: @@snip [MutableIntroTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/MutableIntroTest.java) { #chatroom-main }
|
||||
: @@snip [OOIntroTest.java](/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/OOIntroTest.java) { #chatroom-main }
|
||||
|
||||
In good tradition we call the `main` Actor what it is, it directly
|
||||
corresponds to the `main` method in a traditional Java application. This
|
||||
|
|
|
|||
|
|
@ -119,7 +119,7 @@ their registration order, i.e. the last registered first.
|
|||
|
||||
A message adapter (and the returned `ActorRef`) has the same lifecycle as
|
||||
the receiving actor. It's recommended to register the adapters in a top level
|
||||
`Behaviors.setup` or constructor of `MutableBehavior` but it's possible to
|
||||
`Behaviors.setup` or constructor of `AbstractBehavior` but it's possible to
|
||||
register them later also if needed.
|
||||
|
||||
The adapter function is running in the receiving actor and can safely access state of it, but if it throws an exception the actor is stopped.
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.persistence.typed.internal
|
|||
import akka.Done
|
||||
import akka.actor.typed.Behavior
|
||||
import akka.actor.typed.scaladsl.Behaviors
|
||||
import akka.actor.typed.scaladsl.MutableBehavior
|
||||
import akka.actor.typed.scaladsl.AbstractBehavior
|
||||
import akka.annotation.InternalApi
|
||||
import akka.persistence.JournalProtocol._
|
||||
import akka.persistence._
|
||||
|
|
@ -179,7 +179,7 @@ private[akka] object EventsourcedRunning {
|
|||
numberOfEvents: Int,
|
||||
shouldSnapshotAfterPersist: Boolean,
|
||||
var sideEffects: immutable.Seq[SideEffect[S]])
|
||||
extends MutableBehavior[EventsourcedBehavior.InternalProtocol] {
|
||||
extends AbstractBehavior[EventsourcedBehavior.InternalProtocol] {
|
||||
|
||||
private var eventCounter = 0
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue