HUGE CHANGE - moving behaviorStack into ActorCell
This commit is contained in:
parent
e14f9d0130
commit
26f6c48ae1
31 changed files with 71 additions and 90 deletions
|
|
@ -393,7 +393,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
override def postRestart(reason: Throwable) = latch.countDown()
|
||||
}))
|
||||
|
||||
protected def receive = { case "sendKill" ⇒ ref ! Kill }
|
||||
def receive = { case "sendKill" ⇒ ref ! Kill }
|
||||
}))
|
||||
|
||||
boss ! "sendKill"
|
||||
|
|
|
|||
|
|
@ -131,7 +131,7 @@ class ActorWithStashSpec extends AkkaSpec(ActorWithStashSpec.testConf) with Defa
|
|||
val hasMsgLatch = new TestLatch
|
||||
|
||||
val slaveProps = myProps(new Actor with Stash {
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case "crash" ⇒
|
||||
throw new Exception("Crashing...")
|
||||
|
||||
|
|
|
|||
|
|
@ -147,7 +147,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
|
|||
object Hello
|
||||
object Bye
|
||||
val tester = system.actorOf(Props(new Actor {
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Hello ⇒ lock ! "hello"
|
||||
case "world" ⇒ answerLatch.open
|
||||
case Bye ⇒ lock ! "bye"
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
context.setReceiveTimeout(500 milliseconds)
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case ReceiveTimeout ⇒ timeoutLatch.open
|
||||
}
|
||||
}))
|
||||
|
|
@ -38,7 +38,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
context.setReceiveTimeout(500 milliseconds)
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Tick ⇒ ()
|
||||
case ReceiveTimeout ⇒ timeoutLatch.open
|
||||
}
|
||||
|
|
@ -58,7 +58,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
context.setReceiveTimeout(500 milliseconds)
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Tick ⇒ ()
|
||||
case ReceiveTimeout ⇒
|
||||
count.incrementAndGet
|
||||
|
|
@ -78,7 +78,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutLatch = TestLatch()
|
||||
|
||||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case ReceiveTimeout ⇒ timeoutLatch.open
|
||||
}
|
||||
}))
|
||||
|
|
|
|||
|
|
@ -40,7 +40,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Ping ⇒ countDownLatch.countDown()
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
}
|
||||
|
|
@ -83,7 +83,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
}
|
||||
|
||||
|
|
@ -110,7 +110,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Ping ⇒
|
||||
if (!pingLatch.isOpen) pingLatch.open else secondPingLatch.open
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
|
|
@ -166,7 +166,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Ping ⇒ countDownLatch.countDown()
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
}
|
||||
|
|
@ -221,7 +221,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Ping ⇒ countDownLatch.countDown()
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ object SupervisorHierarchySpec {
|
|||
*/
|
||||
class CountDownActor(countDown: CountDownLatch, override val supervisorStrategy: SupervisorStrategy) extends Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case p: Props ⇒ sender ! context.actorOf(p)
|
||||
}
|
||||
// test relies on keeping children around during restart
|
||||
|
|
@ -67,7 +67,7 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, SupervisorStrategy.defaultStrategy))))
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case "killCrasher" ⇒ crasher ! Kill
|
||||
case Terminated(_) ⇒ countDownMax.countDown()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul
|
|||
|
||||
val workerProps = Props(new Actor {
|
||||
override def postRestart(cause: Throwable) { countDownLatch.countDown() }
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case "status" ⇒ this.sender ! "OK"
|
||||
case _ ⇒ this.context.stop(self)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -400,9 +400,9 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
|||
val a = newTestActor(dispatcher.id)
|
||||
val f1 = a ? Reply("foo")
|
||||
val f2 = a ? Reply("bar")
|
||||
val f3 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(ActorInterruptedException(ie)) }
|
||||
val f3 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(new ActorInterruptedException(ie)) }
|
||||
val f4 = a ? Reply("foo2")
|
||||
val f5 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(ActorInterruptedException(ie)) }
|
||||
val f5 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(new ActorInterruptedException(ie)) }
|
||||
val f6 = a ? Reply("bar2")
|
||||
|
||||
assert(Await.result(f1, timeout.duration) === "foo")
|
||||
|
|
|
|||
|
|
@ -152,7 +152,7 @@ case class DeathPactException private[akka] (dead: ActorRef)
|
|||
* When an InterruptedException is thrown inside an Actor, it is wrapped as an ActorInterruptedException as to
|
||||
* avoid cascading interrupts to other threads than the originally interrupted one.
|
||||
*/
|
||||
case class ActorInterruptedException private[akka] (cause: Throwable) extends AkkaException(cause.getMessage, cause) with NoStackTrace
|
||||
class ActorInterruptedException private[akka] (cause: Throwable) extends AkkaException(cause.getMessage, cause) with NoStackTrace
|
||||
|
||||
/**
|
||||
* This message is published to the EventStream whenever an Actor receives a message it doesn't understand
|
||||
|
|
@ -173,6 +173,7 @@ object Status {
|
|||
|
||||
/**
|
||||
* This class/message type is preferably used to indicate failure of some operation performed.
|
||||
* As an example, it is used to signal failure with AskSupport is used (ask/?).
|
||||
*/
|
||||
case class Failure(cause: Throwable) extends Status
|
||||
}
|
||||
|
|
@ -317,7 +318,7 @@ trait Actor {
|
|||
* This defines the initial actor behavior, it must return a partial function
|
||||
* with the actor logic.
|
||||
*/
|
||||
protected def receive: Receive
|
||||
def receive: Receive
|
||||
|
||||
/**
|
||||
* User overridable definition the strategy to use for supervising
|
||||
|
|
@ -377,45 +378,5 @@ trait Actor {
|
|||
case _ ⇒ context.system.eventStream.publish(UnhandledMessage(message, sender, self))
|
||||
}
|
||||
}
|
||||
|
||||
// =========================================
|
||||
// ==== INTERNAL IMPLEMENTATION DETAILS ====
|
||||
// =========================================
|
||||
|
||||
/**
|
||||
* For Akka internal use only.
|
||||
*/
|
||||
private[akka] final def apply(msg: Any) = {
|
||||
//FIXME replace with behaviorStack.head.applyOrElse(msg, unhandled) + "-optimize"
|
||||
val head = behaviorStack.head
|
||||
if (head.isDefinedAt(msg)) head.apply(msg) else unhandled(msg)
|
||||
}
|
||||
|
||||
/**
|
||||
* For Akka internal use only.
|
||||
*/
|
||||
private[akka] def pushBehavior(behavior: Receive): Unit = {
|
||||
behaviorStack = behaviorStack.push(behavior)
|
||||
}
|
||||
|
||||
/**
|
||||
* For Akka internal use only.
|
||||
*/
|
||||
private[akka] def popBehavior(): Unit = {
|
||||
val original = behaviorStack
|
||||
val popped = original.pop
|
||||
behaviorStack = if (popped.isEmpty) original else popped
|
||||
}
|
||||
|
||||
/**
|
||||
* For Akka internal use only.
|
||||
*/
|
||||
private[akka] def clearBehaviorStack(): Unit =
|
||||
behaviorStack = Stack.empty[Receive].push(behaviorStack.last)
|
||||
|
||||
/**
|
||||
* For Akka internal use only.
|
||||
*/
|
||||
private var behaviorStack: Stack[Receive] = Stack.empty[Receive].push(receive)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -409,6 +409,8 @@ private[akka] class ActorCell(
|
|||
|
||||
var actor: Actor = _
|
||||
|
||||
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
|
||||
|
||||
@volatile //This must be volatile since it isn't protected by the mailbox status
|
||||
var mailbox: Mailbox = _
|
||||
|
||||
|
|
@ -489,8 +491,7 @@ private[akka] class ActorCell(
|
|||
|
||||
//This method is in charge of setting up the contextStack and create a new instance of the Actor
|
||||
protected def newActor(): Actor = {
|
||||
val stackBefore = contextStack.get
|
||||
contextStack.set(stackBefore.push(this))
|
||||
contextStack.set(contextStack.get.push(this))
|
||||
try {
|
||||
val instance = props.creator()
|
||||
|
||||
|
|
@ -511,6 +512,7 @@ private[akka] class ActorCell(
|
|||
def create(): Unit = if (isNormal) {
|
||||
try {
|
||||
val created = newActor()
|
||||
behaviorStack = Stack.empty.push(created.receive)
|
||||
actor = created
|
||||
created.preStart()
|
||||
checkReceiveTimeout
|
||||
|
|
@ -612,7 +614,7 @@ private[akka] class ActorCell(
|
|||
cancelReceiveTimeout() // FIXME: leave this here???
|
||||
messageHandle.message match {
|
||||
case msg: AutoReceivedMessage ⇒ autoReceiveMessage(messageHandle)
|
||||
case msg ⇒ actor(msg)
|
||||
case msg ⇒ receiveMessage(msg)
|
||||
}
|
||||
currentMessage = null // reset current message after successful invocation
|
||||
} catch {
|
||||
|
|
@ -628,14 +630,14 @@ private[akka] class ActorCell(
|
|||
if (actor ne null) actor.supervisorStrategy.handleSupervisorFailing(self, children)
|
||||
} finally {
|
||||
t match { // Wrap InterruptedExceptions and rethrow
|
||||
case _: InterruptedException ⇒ parent.tell(Failed(ActorInterruptedException(t)), self); throw t
|
||||
case _: InterruptedException ⇒ parent.tell(Failed(new ActorInterruptedException(t)), self); throw t
|
||||
case _ ⇒ parent.tell(Failed(t), self)
|
||||
}
|
||||
}
|
||||
|
||||
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit = {
|
||||
if (discardOld) unbecome()
|
||||
actor.pushBehavior(behavior)
|
||||
behaviorStack = behaviorStack.push(behavior)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -651,9 +653,13 @@ private[akka] class ActorCell(
|
|||
become(newReceive, discardOld)
|
||||
}
|
||||
|
||||
def unbecome(): Unit = actor.popBehavior()
|
||||
def unbecome(): Unit = {
|
||||
val original = behaviorStack
|
||||
val popped = original.pop
|
||||
behaviorStack = if (popped.isEmpty) original else popped
|
||||
}
|
||||
|
||||
def autoReceiveMessage(msg: Envelope) {
|
||||
def autoReceiveMessage(msg: Envelope): Unit = {
|
||||
if (system.settings.DebugAutoReceive)
|
||||
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "received AutoReceiveMessage " + msg))
|
||||
|
||||
|
|
@ -667,6 +673,12 @@ private[akka] class ActorCell(
|
|||
}
|
||||
}
|
||||
|
||||
final def receiveMessage(msg: Any): Unit = {
|
||||
//FIXME replace with behaviorStack.head.applyOrElse(msg, unhandled) + "-optimize"
|
||||
val head = behaviorStack.head
|
||||
if (head.isDefinedAt(msg)) head.apply(msg) else actor.unhandled(msg)
|
||||
}
|
||||
|
||||
private def doTerminate() {
|
||||
val a = actor
|
||||
try {
|
||||
|
|
@ -682,7 +694,7 @@ private[akka] class ActorCell(
|
|||
if (system.settings.DebugLifecycle)
|
||||
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped"))
|
||||
} finally {
|
||||
if (a ne null) a.clearBehaviorStack()
|
||||
behaviorStack = Stack.empty
|
||||
clearActorFields(a)
|
||||
actor = null
|
||||
}
|
||||
|
|
@ -694,6 +706,7 @@ private[akka] class ActorCell(
|
|||
actor.supervisorStrategy.handleSupervisorRestarted(cause, self, children)
|
||||
|
||||
val freshActor = newActor()
|
||||
behaviorStack = Stack.empty.push(freshActor.receive)
|
||||
actor = freshActor // this must happen before postRestart has a chance to fail
|
||||
if (freshActor eq failedActor) setActorFields(freshActor, this, self) // If the creator returns the same instance, we need to restore our nulled out fields.
|
||||
|
||||
|
|
|
|||
|
|
@ -15,7 +15,9 @@ object ActorPath {
|
|||
}
|
||||
|
||||
/**
|
||||
* This Regular Expression is used to validate a path element (Actor Name)
|
||||
* This Regular Expression is used to validate a path element (Actor Name).
|
||||
* Since Actors form a tree, it is addressable using an URL, therefor an Actor Name has to conform to:
|
||||
* http://www.ietf.org/rfc/rfc2396.txt
|
||||
*/
|
||||
val ElementRegex = """[-\w:@&=+,.!~*'_;][-\w:@&=+,.!~*'$_;]*""".r
|
||||
}
|
||||
|
|
|
|||
|
|
@ -335,9 +335,10 @@ private[akka] class LocalActorRef private[akka] (
|
|||
|
||||
/**
|
||||
* Memento pattern for serializing ActorRefs transparently
|
||||
* INTERNAL API
|
||||
*/
|
||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
case class SerializedActorRef private (path: String) {
|
||||
private[akka] case class SerializedActorRef private (path: String) {
|
||||
import akka.serialization.JavaSerializer.currentSystem
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
|
|
@ -350,8 +351,11 @@ case class SerializedActorRef private (path: String) {
|
|||
someSystem.actorFor(path)
|
||||
}
|
||||
}
|
||||
//FIXME: Should SerializedActorRef be private[akka] ?
|
||||
object SerializedActorRef {
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object SerializedActorRef {
|
||||
def apply(path: ActorPath): SerializedActorRef = {
|
||||
Serialization.currentTransportAddress.value match {
|
||||
case null ⇒ new SerializedActorRef(path.toString)
|
||||
|
|
|
|||
|
|
@ -437,7 +437,7 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
* Main actor receive() method
|
||||
* *******************************************
|
||||
*/
|
||||
override final protected def receive: Receive = {
|
||||
override final def receive: Receive = {
|
||||
case TimeoutMarker(gen) ⇒
|
||||
if (generation == gen) {
|
||||
processMsg(StateTimeout, "state timeout")
|
||||
|
|
|
|||
|
|
@ -176,9 +176,10 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
|||
/**
|
||||
* Sort so that subtypes always precede their supertypes, but without
|
||||
* obeying any order between unrelated subtypes (insert sort).
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
//FIXME Should this really be public API?
|
||||
def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] =
|
||||
private[akka] def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] =
|
||||
(new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca) ⇒
|
||||
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
|
||||
case -1 ⇒ buf append ca
|
||||
|
|
@ -195,7 +196,7 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
|||
}
|
||||
|
||||
/**
|
||||
* An Akka SupervisorStrategy is
|
||||
* An Akka SupervisorStrategy is the policy to apply for crashing children
|
||||
*/
|
||||
abstract class SupervisorStrategy {
|
||||
|
||||
|
|
|
|||
|
|
@ -987,7 +987,7 @@ final class IOManagerActor extends Actor with ActorLogging {
|
|||
}
|
||||
}
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Select ⇒
|
||||
select()
|
||||
if (running) self ! Select
|
||||
|
|
|
|||
|
|
@ -236,7 +236,7 @@ private[akka] class ContinuousCancellable extends AtomicReference[HWTimeout](Con
|
|||
|
||||
def isCancelled(): Boolean = get match {
|
||||
case null ⇒ true
|
||||
case some ⇒ isCancelled()
|
||||
case some ⇒ some.isCancelled()
|
||||
}
|
||||
|
||||
def cancel(): Unit =
|
||||
|
|
|
|||
|
|
@ -156,7 +156,7 @@ abstract class UntypedActor extends Actor {
|
|||
*/
|
||||
override def postRestart(reason: Throwable): Unit = super.postRestart(reason)
|
||||
|
||||
final protected def receive = { case msg ⇒ onReceive(msg) }
|
||||
final def receive = { case msg ⇒ onReceive(msg) }
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -134,7 +134,7 @@ trait Producer extends ProducerSupport { this: Actor ⇒
|
|||
* Default implementation of Actor.receive. Any messages received by this actors
|
||||
* will be produced to the endpoint specified by <code>endpointUri</code>.
|
||||
*/
|
||||
protected def receive = produce
|
||||
def receive = produce
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ private[camelexamples] class SysOutConsumer extends Consumer {
|
|||
override def activationTimeout = 10 seconds
|
||||
def endpointUri = "file://data/input/CamelConsumer"
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case msg: CamelMessage ⇒ {
|
||||
printf("Received '%s'\n", msg.bodyAs[String])
|
||||
}
|
||||
|
|
@ -30,12 +30,12 @@ private[camelexamples] class TroubleMaker extends Consumer {
|
|||
def endpointUri = "WRONG URI"
|
||||
|
||||
println("Trying to instantiate conumer with uri: " + endpointUri)
|
||||
protected def receive = { case _ ⇒ }
|
||||
def receive = { case _ ⇒ }
|
||||
}
|
||||
|
||||
private[camelexamples] class SysOutActor(implicit camel: Camel) extends Actor {
|
||||
implicit val camelContext = camel.context
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case msg: CamelMessage ⇒ {
|
||||
printf("Received '%s'\n", msg.bodyAs[String])
|
||||
}
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ private[camelexamples] object SupervisedConsumersExample extends App {
|
|||
|
||||
system.actorOf(Props(new Actor {
|
||||
context.watch(context.actorOf(Props[EndpointManager]))
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Terminated(ref) ⇒ system.shutdown()
|
||||
}
|
||||
}))
|
||||
|
|
@ -30,7 +30,7 @@ private[camelexamples] class EndpointManager extends Actor {
|
|||
watch(actorOf(Props[SysOutConsumer]))
|
||||
watch(actorOf(Props[TroubleMaker]))
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Terminated(ref) ⇒ {
|
||||
printf("Hey! One of the endpoints has died: %s. I am doing sepuku...\n", ref)
|
||||
self ! PoisonPill
|
||||
|
|
|
|||
|
|
@ -244,7 +244,7 @@ object ProducerFeatureTest {
|
|||
}
|
||||
|
||||
class TestResponder extends Actor {
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case msg: CamelMessage ⇒ msg.body match {
|
||||
case "fail" ⇒ context.sender ! akka.actor.Status.Failure(new AkkaCamelException(new Exception("failure"), msg.headers))
|
||||
case _ ⇒
|
||||
|
|
@ -256,7 +256,7 @@ object ProducerFeatureTest {
|
|||
}
|
||||
|
||||
class ReplyingForwardTarget extends Actor {
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case msg: CamelMessage ⇒
|
||||
context.sender ! (msg.addHeader("test" -> "result"))
|
||||
case msg: akka.actor.Status.Failure ⇒
|
||||
|
|
|
|||
|
|
@ -334,7 +334,7 @@ trait ActorProducerFixture extends MockitoSugar with BeforeAndAfterAll with Befo
|
|||
}
|
||||
|
||||
def echoActor = system.actorOf(Props(new Actor {
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case msg ⇒ sender ! "received " + msg
|
||||
}
|
||||
}))
|
||||
|
|
|
|||
|
|
@ -443,7 +443,7 @@ An Actor has to implement the ``receive`` method to receive messages:
|
|||
|
||||
.. code-block:: scala
|
||||
|
||||
protected def receive: PartialFunction[Any, Unit]
|
||||
def receive: PartialFunction[Any, Unit]
|
||||
|
||||
Note: Akka has an alias to the ``PartialFunction[Any, Unit]`` type called
|
||||
``Receive`` (``akka.actor.Actor.Receive``), so you can use this type instead for
|
||||
|
|
|
|||
|
|
@ -56,7 +56,7 @@ class TestActorRef[T <: Actor](
|
|||
* thrown will be available to you, while still being able to use
|
||||
* become/unbecome.
|
||||
*/
|
||||
def receive(o: Any) { underlyingActor.apply(o) }
|
||||
def receive(o: Any): Unit = underlying.receiveMessage(o)
|
||||
|
||||
/**
|
||||
* Retrieve reference to the underlying actor, where the static type matches the factory used inside the
|
||||
|
|
|
|||
|
|
@ -138,7 +138,7 @@ class ConcurrentSocketActorSpec extends AkkaSpec {
|
|||
}
|
||||
}
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case _ ⇒
|
||||
val payload = "%s".format(messageNumber)
|
||||
messageNumber += 1
|
||||
|
|
|
|||
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Loading…
Add table
Add a link
Reference in a new issue