Merge pull request #25804 from akka/wip-25750-AbstractBehavior-patriknw

Rename MutableBehavior to AbstractBehavior, #25750
This commit is contained in:
Patrik Nordwall 2018-10-18 13:16:48 +02:00 committed by GitHub
commit abb3429bc8
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
22 changed files with 212 additions and 138 deletions

View file

@ -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()));
}
}

View file

@ -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);

View file

@ -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<>();

View file

@ -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] = {

View file

@ -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)

View file

@ -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
}

View file

@ -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

View file

@ -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 {

View file

@ -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
}

View file

@ -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.

View file

@ -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.
*

View file

@ -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)
}

View file

@ -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] =

View file

@ -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]])
}

View file

@ -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.

View file

@ -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

View file

@ -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._

View file

@ -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())

View file

@ -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();

View file

@ -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 clients 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

View file

@ -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.

View file

@ -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