merge master into parental-supervision, fixing up resulting breakage

This commit is contained in:
Roland 2011-10-21 15:11:43 +02:00
commit d55f02e6c1
95 changed files with 759 additions and 731 deletions

View file

@ -2,6 +2,6 @@ package akka.actor;
public class JavaAPITestActor extends UntypedActor { public class JavaAPITestActor extends UntypedActor {
public void onReceive(Object msg) { public void onReceive(Object msg) {
tryReply("got it!"); getChannel().tryTell("got it!");
} }
} }

View file

@ -15,7 +15,7 @@ object ActorFireForgetRequestReplySpec {
class ReplyActor extends Actor { class ReplyActor extends Actor {
def receive = { def receive = {
case "Send" case "Send"
reply("Reply") channel ! "Reply"
case "SendImplicit" case "SendImplicit"
channel ! "ReplyImplicit" channel ! "ReplyImplicit"
} }

View file

@ -25,7 +25,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS
val currentGen = generationProvider.getAndIncrement() val currentGen = generationProvider.getAndIncrement()
override def preStart() { report("preStart") } override def preStart() { report("preStart") }
override def postStop() { report("postStop") } override def postStop() { report("postStop") }
def receive = { case "status" this reply message("OK") } def receive = { case "status" channel ! message("OK") }
} }
"An Actor" must { "An Actor" must {

View file

@ -35,7 +35,7 @@ object ActorRefSpec {
val worker = context.actorOf(Props[WorkerActor]) val worker = context.actorOf(Props[WorkerActor])
worker ! ReplyTo(channel) worker ! ReplyTo(channel)
case "workDone" replyTo ! "complexReply" case "workDone" replyTo ! "complexReply"
case "simpleRequest" reply("simpleReply") case "simpleRequest" channel ! "simpleReply"
} }
} }
@ -43,7 +43,7 @@ object ActorRefSpec {
def receive = { def receive = {
case "work" { case "work" {
work work
reply("workDone") channel ! "workDone"
self.stop() self.stop()
} }
case ReplyTo(replyTo) { case ReplyTo(replyTo) {
@ -74,7 +74,7 @@ object ActorRefSpec {
class OuterActor(val inner: ActorRef) extends Actor { class OuterActor(val inner: ActorRef) extends Actor {
def receive = { def receive = {
case "self" reply(self) case "self" channel ! self
case x inner forward x case x inner forward x
} }
} }
@ -83,7 +83,7 @@ object ActorRefSpec {
val fail = new InnerActor val fail = new InnerActor
def receive = { def receive = {
case "self" reply(self) case "self" channel ! self
case x inner forward x case x inner forward x
} }
} }
@ -94,8 +94,8 @@ object ActorRefSpec {
class InnerActor extends Actor { class InnerActor extends Actor {
def receive = { def receive = {
case "innerself" reply(self) case "innerself" channel ! self
case other reply(other) case other channel ! other
} }
} }
@ -103,8 +103,8 @@ object ActorRefSpec {
val fail = new InnerActor val fail = new InnerActor
def receive = { def receive = {
case "innerself" reply(self) case "innerself" channel ! self
case other reply(other) case other channel ! other
} }
} }
@ -321,7 +321,7 @@ class ActorRefSpec extends AkkaSpec {
"support nested actorOfs" in { "support nested actorOfs" in {
val a = actorOf(new Actor { val a = actorOf(new Actor {
val nested = actorOf(new Actor { def receive = { case _ } }) val nested = actorOf(new Actor { def receive = { case _ } })
def receive = { case _ reply(nested) } def receive = { case _ channel ! nested }
}) })
val nested = (a ? "any").as[ActorRef].get val nested = (a ? "any").as[ActorRef].get
@ -369,8 +369,8 @@ class ActorRefSpec extends AkkaSpec {
val timeout = Timeout(20000) val timeout = Timeout(20000)
val ref = actorOf(Props(new Actor { val ref = actorOf(Props(new Actor {
def receive = { def receive = {
case 5 tryReply("five") case 5 channel.tryTell("five")
case null tryReply("null") case null channel.tryTell("null")
} }
})) }))

View file

@ -44,7 +44,7 @@ object Chameneos {
case Exit case Exit
colour = FADED colour = FADED
sender.get ! MeetingCount(meetings) sender ! MeetingCount(meetings)
} }
def complement(otherColour: Colour): Colour = colour match { def complement(otherColour: Colour): Colour = colour match {
@ -97,7 +97,7 @@ object Chameneos {
n -= 1 n -= 1
chameneo ! msg chameneo ! msg
waitingChameneo = None waitingChameneo = None
case None waitingChameneo = sender case None waitingChameneo = Some(sender)
} }
} else { } else {
waitingChameneo.foreach(_ ! Exit) waitingChameneo.foreach(_ ! Exit)

View file

@ -12,59 +12,70 @@ import java.util.concurrent.atomic._
class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender { class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender {
"The Death Watch" must { "The Death Watch" must {
def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(2 seconds, actorRef + ": Stopped") { def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(5 seconds, actorRef + ": Stopped or Already terminated when linking") {
case Terminated(`actorRef`, ex: ActorKilledException) if ex.getMessage == "Stopped" true case Terminated(`actorRef`) true
} }
"notify with one Terminated message when an Actor is stopped" in { "notify with one Terminated message when an Actor is stopped" in {
val terminal = actorOf(Props(context { case _ context.self.stop() })) val terminal = actorOf(Props(context { case _ }))
testActor startsMonitoring terminal testActor startsMonitoring terminal
terminal ! "anything" testActor ! "ping"
expectMsg("ping")
terminal ! PoisonPill
expectTerminationOf(terminal) expectTerminationOf(terminal)
terminal.stop()
} }
"notify with all monitors with one Terminated message when an Actor is stopped" in { "notify with all monitors with one Terminated message when an Actor is stopped" in {
val monitor1, monitor2 = actorOf(Props(context { case t: Terminated testActor ! t })) val terminal = actorOf(Props(context { case _ }))
val terminal = actorOf(Props(context { case _ context.self.stop() })) val monitor1, monitor2, monitor3 =
actorOf(Props(new Actor {
watch(terminal)
def receive = { case t: Terminated testActor ! t }
}))
monitor1 startsMonitoring terminal terminal ! PoisonPill
monitor2 startsMonitoring terminal
testActor startsMonitoring terminal
terminal ! "anything"
expectTerminationOf(terminal) expectTerminationOf(terminal)
expectTerminationOf(terminal) expectTerminationOf(terminal)
expectTerminationOf(terminal) expectTerminationOf(terminal)
terminal.stop()
monitor1.stop() monitor1.stop()
monitor2.stop() monitor2.stop()
monitor3.stop()
} }
"notify with _current_ monitors with one Terminated message when an Actor is stopped" in { "notify with _current_ monitors with one Terminated message when an Actor is stopped" in {
val monitor1, monitor2 = actorOf(Props(context { case t: Terminated testActor ! t })) val terminal = actorOf(Props(context { case _ }))
val terminal = actorOf(Props(context { case _ context.self.stop() })) val monitor1, monitor3 =
actorOf(Props(new Actor {
watch(terminal)
def receive = { case t: Terminated testActor ! t }
}))
val monitor2 = actorOf(Props(new Actor {
watch(terminal)
unwatch(terminal)
def receive = {
case "ping" sender ! "pong"
case t: Terminated testActor ! t
}
}))
monitor1 startsMonitoring terminal monitor2 ! "ping"
monitor2 startsMonitoring terminal
testActor startsMonitoring terminal
monitor2 stopsMonitoring terminal expectMsg("pong") //Needs to be here since startsMonitoring and stopsMonitoring are asynchronous
terminal ! "anything" terminal ! PoisonPill
expectTerminationOf(terminal) expectTerminationOf(terminal)
expectTerminationOf(terminal) expectTerminationOf(terminal)
terminal.stop()
monitor1.stop() monitor1.stop()
monitor2.stop() monitor2.stop()
monitor3.stop()
} }
"notify with a Terminated message once when an Actor is stopped but not when restarted" in { "notify with a Terminated message once when an Actor is stopped but not when restarted" in {
@ -73,7 +84,10 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
val terminalProps = Props(context { case x context.channel ! x }) val terminalProps = Props(context { case x context.channel ! x })
val terminal = (supervisor ? terminalProps).as[ActorRef].get val terminal = (supervisor ? terminalProps).as[ActorRef].get
testActor startsMonitoring terminal val monitor = actorOf(Props(new Actor {
watch(terminal)
def receive = { case t: Terminated testActor ! t }
}))
terminal ! Kill terminal ! Kill
terminal ! Kill terminal ! Kill
@ -81,24 +95,34 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
terminal ! Kill terminal ! Kill
expectTerminationOf(terminal) expectTerminationOf(terminal)
terminal.isShutdown must be === true
terminal.stop()
supervisor.stop() supervisor.stop()
} }
} }
"fail a monitor which does not handle Terminated()" in { "fail a monitor which does not handle Terminated()" in {
filterEvents(EventFilter[ActorKilledException], EventFilter[DeathPactException]) { filterEvents(EventFilter[ActorKilledException], EventFilter[DeathPactException]) {
val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) val supervisor = actorOf(Props[Supervisor]
.withFaultHandler(new OneForOneStrategy(FaultHandlingStrategy.makeDecider(List(classOf[Exception])), Some(0)) {
override def handleFailure(fail: Failed, stats: ChildRestartStats, children: Iterable[(ActorRef, ChildRestartStats)]) = {
testActor ! fail
super.handleFailure(fail, stats, children)
}
}))
val failed, brother = (supervisor ? Props.empty).as[ActorRef].get val failed, brother = (supervisor ? Props.empty).as[ActorRef].get
brother startsMonitoring failed brother startsMonitoring failed
testActor startsMonitoring brother testActor startsMonitoring brother
failed ! Kill failed ! Kill
expectMsgPF() { val result = receiveWhile(3 seconds, messages = 3) {
case Terminated(brother, DeathPactException(failed, _: ActorKilledException)) true case Failed(`failed`, _: ActorKilledException) 1
case Failed(`brother`, DeathPactException(`failed`)) 2
case Terminated(`brother`) 3
} }
testActor must not be 'shutdown
result must be(Seq(1, 2, 3))
} }
} }
} }

View file

@ -55,8 +55,12 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
"not fail when listener goes away" in { "not fail when listener goes away" in {
val forward = actorOf(new Forwarder(testActor)) val forward = actorOf(new Forwarder(testActor))
val sup = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) val fsm = actorOf(new MyFSM(testActor))
val fsm = sup startsMonitoring actorOf(new MyFSM(testActor)) val sup = actorOf(Props(new Actor {
self startsMonitoring fsm
def receive = { case _ }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None)))
within(300 millis) { within(300 millis) {
fsm ! SubscribeTransitionCallBack(forward) fsm ! SubscribeTransitionCallBack(forward)
expectMsg(CurrentState(fsm, 0)) expectMsg(CurrentState(fsm, 0))

View file

@ -15,7 +15,7 @@ object ForwardActorSpec {
def createForwardingChain(app: AkkaApplication): ActorRef = { def createForwardingChain(app: AkkaApplication): ActorRef = {
val replier = app.actorOf(new Actor { val replier = app.actorOf(new Actor {
def receive = { case x reply(x) } def receive = { case x channel ! x }
}) })
def mkforwarder(forwardTo: ActorRef) = app.actorOf( def mkforwarder(forwardTo: ActorRef) = app.actorOf(

View file

@ -48,7 +48,7 @@ object IOActorSpec {
def receiveIO = { def receiveIO = {
case length: Int case length: Int
val bytes = socket.read(length) val bytes = socket.read(length)
reply(bytes) channel ! bytes
} }
} }
} }
@ -108,9 +108,9 @@ object IOActorSpec {
case msg: NewClient createWorker forward msg case msg: NewClient createWorker forward msg
case ('set, key: String, value: ByteString) case ('set, key: String, value: ByteString)
kvs += (key -> value) kvs += (key -> value)
tryReply(()) channel.tryTell(())(self)
case ('get, key: String) tryReply(kvs.get(key)) case ('get, key: String) channel.tryTell(kvs.get(key))(self)
case 'getall tryReply(kvs) case 'getall channel.tryTell(kvs)(self)
} }
} }
@ -123,18 +123,20 @@ object IOActorSpec {
socket = connect(ioManager, host, port) socket = connect(ioManager, host, port)
} }
def reply(msg: Any) = channel.tryTell(msg)(self)
def receiveIO = { def receiveIO = {
case ('set, key: String, value: ByteString) case ('set, key: String, value: ByteString)
socket write (ByteString("SET " + key + " " + value.length + "\r\n") ++ value) socket write (ByteString("SET " + key + " " + value.length + "\r\n") ++ value)
tryReply(readResult) reply(readResult)
case ('get, key: String) case ('get, key: String)
socket write ByteString("GET " + key + "\r\n") socket write ByteString("GET " + key + "\r\n")
tryReply(readResult) reply(readResult)
case 'getall case 'getall
socket write ByteString("GETALL\r\n") socket write ByteString("GETALL\r\n")
tryReply(readResult) reply(readResult)
} }
def readResult = { def readResult = {

View file

@ -62,7 +62,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
app.eventHandler.addListener(testActor) app.eventHandler.addListener(testActor)
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = loggable(this) { def receive = loggable(this) {
case _ reply("x") case _ channel ! "x"
} }
}) })
actor ! "buh" actor ! "buh"
@ -91,7 +91,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
app.eventHandler.addListener(testActor) app.eventHandler.addListener(testActor)
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = loggable(this)(loggable(this) { def receive = loggable(this)(loggable(this) {
case _ reply("x") case _ channel ! "x"
}) })
}) })
actor ! "buh" actor ! "buh"

View file

@ -209,7 +209,7 @@ class RestartStrategySpec extends AkkaSpec {
val boss = actorOf(Props(new Actor { val boss = actorOf(Props(new Actor {
def receive = { def receive = {
case p: Props reply(context.actorOf(p)) case p: Props channel ! context.actorOf(p)
case t: Terminated maxNoOfRestartsLatch.open case t: Terminated maxNoOfRestartsLatch.open
} }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, Some(1000)))) }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, Some(1000))))

View file

@ -5,6 +5,6 @@ package akka.actor
class Supervisor extends Actor { class Supervisor extends Actor {
def receive = { def receive = {
case x: Props reply(context.actorOf(x)) case x: Props channel ! context.actorOf(x)
} }
} }

View file

@ -13,7 +13,7 @@ object SupervisorHierarchySpec {
class CountDownActor(countDown: CountDownLatch) extends Actor { class CountDownActor(countDown: CountDownLatch) extends Actor {
protected def receive = { protected def receive = {
case p: Props reply(context.actorOf(p)) case p: Props channel ! context.actorOf(p)
} }
override def postRestart(reason: Throwable) = { override def postRestart(reason: Throwable) = {
countDown.countDown() countDown.countDown()
@ -55,8 +55,8 @@ class SupervisorHierarchySpec extends AkkaSpec {
self startsMonitoring crasher self startsMonitoring crasher
protected def receive = { protected def receive = {
case "killCrasher" crasher ! Kill case "killCrasher" crasher ! Kill
case Terminated(_, _) countDownMax.countDown() case Terminated(_) countDownMax.countDown()
} }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 1, 5000))) }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 1, 5000)))

View file

@ -21,7 +21,7 @@ class SupervisorMiscSpec extends AkkaSpec {
val workerProps = Props(new Actor { val workerProps = Props(new Actor {
override def postRestart(cause: Throwable) { countDownLatch.countDown() } override def postRestart(cause: Throwable) { countDownLatch.countDown() }
protected def receive = { protected def receive = {
case "status" this.reply("OK") case "status" this.channel ! "OK"
case _ this.self.stop() case _ this.self.stop()
} }
}) })

View file

@ -4,16 +4,13 @@
package akka.actor package akka.actor
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import org.scalatest.BeforeAndAfterAll
import akka.testkit.Testing.sleepFor import akka.testkit.Testing.sleepFor
import akka.util.duration._ import akka.util.duration._
import akka.{ Die, Ping } import akka.{ Die, Ping }
import akka.actor.Actor._ import akka.actor.Actor._
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._
import akka.testkit.EventFilter import akka.testkit.{ EventFilter, ImplicitSender }
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.LinkedBlockingQueue
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
@ -42,7 +39,7 @@ object SupervisorSpec {
def receive = { def receive = {
case Ping case Ping
messageLog.put(PingMessage) messageLog.put(PingMessage)
tryReply(PongMessage) channel.tryTell(PongMessage)
case Die case Die
throw new RuntimeException(ExceptionMessage) throw new RuntimeException(ExceptionMessage)
} }
@ -59,14 +56,14 @@ object SupervisorSpec {
var s: UntypedChannel = _ var s: UntypedChannel = _
def receive = { def receive = {
case Die temp ! Die; s = context.channel case Die temp ! Die; s = context.channel
case Terminated(`temp`, cause) s ! cause case Terminated(`temp`) s ! "terminated"
} }
} }
} }
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach { class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender {
import SupervisorSpec._ import SupervisorSpec._
@ -150,9 +147,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach {
"not restart programmatically linked temporary actor" in { "not restart programmatically linked temporary actor" in {
val master = actorOf(Props[Master].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) val master = actorOf(Props[Master].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0))))
(master.?(Die, TimeoutMillis)).get match { master ! Die
case r: RuntimeException r === ExceptionMessage expectMsg(3 seconds, "terminated")
}
sleepFor(1 second) sleepFor(1 second)
messageLogPoll must be(null) messageLogPoll must be(null)
@ -298,7 +294,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach {
if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!") if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!")
def receive = { def receive = {
case Ping tryReply(PongMessage) case Ping channel.tryTell(PongMessage)
case Die throw new RuntimeException("Expected") case Die throw new RuntimeException("Expected")
} }
}) })

View file

@ -22,7 +22,7 @@ class SupervisorTreeSpec extends AkkaSpec with ImplicitSender {
within(5 seconds) { within(5 seconds) {
val p = Props(new Actor { val p = Props(new Actor {
def receive = { def receive = {
case p: Props this reply context.actorOf(p) case p: Props channel ! context.actorOf(p)
} }
override def preRestart(cause: Throwable, msg: Option[Any]) { testActor ! self.address } override def preRestart(cause: Throwable, msg: Option[Any]) { testActor ! self.address }
}).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 1000)) }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 1000))

View file

@ -50,11 +50,11 @@ object Ticket669Spec {
} }
override def preRestart(reason: scala.Throwable, msg: Option[Any]) { override def preRestart(reason: scala.Throwable, msg: Option[Any]) {
tryReply("failure1") channel.tryTell("failure1")
} }
override def postStop() { override def postStop() {
tryReply("failure2") channel.tryTell("failure2")
} }
} }
} }

View file

@ -71,8 +71,8 @@ object ActorModelSpec {
case Meet(sign, wait) ack; sign.countDown(); wait.await(); busy.switchOff() case Meet(sign, wait) ack; sign.countDown(); wait.await(); busy.switchOff()
case Wait(time) ack; Thread.sleep(time); busy.switchOff() case Wait(time) ack; Thread.sleep(time); busy.switchOff()
case WaitAck(time, l) ack; Thread.sleep(time); l.countDown(); busy.switchOff() case WaitAck(time, l) ack; Thread.sleep(time); l.countDown(); busy.switchOff()
case Reply(msg) ack; reply(msg); busy.switchOff() case Reply(msg) ack; channel ! msg; busy.switchOff()
case TryReply(msg) ack; tryReply(msg); busy.switchOff() case TryReply(msg) ack; channel.tryTell(msg); busy.switchOff()
case Forward(to, msg) ack; to.forward(msg); busy.switchOff() case Forward(to, msg) ack; to.forward(msg); busy.switchOff()
case CountDown(latch) ack; latch.countDown(); busy.switchOff() case CountDown(latch) ack; latch.countDown(); busy.switchOff()
case Increment(count) ack; count.incrementAndGet(); busy.switchOff() case Increment(count) ack; count.incrementAndGet(); busy.switchOff()
@ -128,10 +128,10 @@ object ActorModelSpec {
super.unregister(actor) super.unregister(actor)
} }
protected[akka] abstract override def dispatch(invocation: Envelope) { protected[akka] abstract override def dispatch(receiver: ActorCell, invocation: Envelope) {
val stats = getStats(invocation.receiver.self) val stats = getStats(receiver.self)
stats.msgsReceived.incrementAndGet() stats.msgsReceived.incrementAndGet()
super.dispatch(invocation) super.dispatch(receiver, invocation)
} }
protected[akka] abstract override def start() { protected[akka] abstract override def start() {
@ -381,31 +381,11 @@ abstract class ActorModelSpec extends AkkaSpec {
} catch { } catch {
case e case e
System.err.println("Error: " + e.getMessage + " missing count downs == " + cachedMessage.latch.getCount() + " out of " + num) System.err.println("Error: " + e.getMessage + " missing count downs == " + cachedMessage.latch.getCount() + " out of " + num)
//app.eventHandler.error(new Exception with NoStackTrace, null, cachedMessage.latch.getCount())
} }
} }
for (run 1 to 3) { for (run 1 to 3) {
flood(40000) flood(40000)
try { assertDispatcher(dispatcher)(starts = run, stops = run)
assertDispatcher(dispatcher)(starts = run, stops = run)
} catch {
case e
// FIXME: registry has been removed
// app.registry.local.foreach {
// case actor: LocalActorRef
// val cell = actor.underlying
// val mbox = cell.mailbox
// System.err.println("Left in the registry: " + actor.address + " => " + cell + " => " + mbox.hasMessages + " " + mbox.hasSystemMessages + " " + mbox.numberOfMessages + " " + mbox.isScheduled)
// var message = mbox.dequeue()
// while (message ne null) {
// System.err.println("Lingering message for " + cell + " " + message)
// message = mbox.dequeue()
// }
// }
throw e
}
} }
} }

View file

@ -9,7 +9,7 @@ import akka.actor.{ Props, Actor }
object DispatcherActorSpec { object DispatcherActorSpec {
class TestActor extends Actor { class TestActor extends Actor {
def receive = { def receive = {
case "Hello" reply("World") case "Hello" channel ! "World"
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance") case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
} }
} }
@ -67,20 +67,18 @@ class DispatcherActorSpec extends AkkaSpec {
val works = new AtomicBoolean(true) val works = new AtomicBoolean(true)
val latch = new CountDownLatch(100) val latch = new CountDownLatch(100)
val thereWeAre = new CountDownLatch(1)
val start = new CountDownLatch(1) val start = new CountDownLatch(1)
val fastOne = actorOf( val fastOne = actorOf(
Props(context { case "sabotage" works.set(false) }).withDispatcher(throughputDispatcher)) Props(context { case "sabotage" works.set(false) }).withDispatcher(throughputDispatcher))
val slowOne = actorOf( val slowOne = actorOf(
Props(context { Props(context {
case "hogexecutor" thereWeAre.countDown(); start.await case "hogexecutor" context.channel ! "OK"; start.await
case "ping" if (works.get) latch.countDown() case "ping" if (works.get) latch.countDown()
}).withDispatcher(throughputDispatcher)) }).withDispatcher(throughputDispatcher))
slowOne ! "hogexecutor" assert((slowOne ? "hogexecutor").get === "OK")
(1 to 100) foreach { _ slowOne ! "ping" } (1 to 100) foreach { _ slowOne ! "ping" }
assert(thereWeAre.await(2, TimeUnit.SECONDS))
fastOne ! "sabotage" fastOne ! "sabotage"
start.countDown() start.countDown()
latch.await(10, TimeUnit.SECONDS) latch.await(10, TimeUnit.SECONDS)

View file

@ -12,7 +12,7 @@ import org.scalatest.BeforeAndAfterEach
object PinnedActorSpec { object PinnedActorSpec {
class TestActor extends Actor { class TestActor extends Actor {
def receive = { def receive = {
case "Hello" reply("World") case "Hello" channel ! "World"
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance") case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
} }
} }

View file

@ -17,20 +17,16 @@ import org.scalatest.junit.JUnitSuite
object FutureSpec { object FutureSpec {
class TestActor extends Actor { class TestActor extends Actor {
def receive = { def receive = {
case "Hello" case "Hello" channel ! "World"
reply("World") case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
case "NoReply" {} case "NoReply"
case "Failure"
throw new RuntimeException("Expected exception; to test fault-tolerance")
} }
} }
class TestDelayActor(await: StandardLatch) extends Actor { class TestDelayActor(await: StandardLatch) extends Actor {
def receive = { def receive = {
case "Hello" case "Hello" await.await; channel ! "World"
await.await case "NoReply" await.await
reply("World")
case "NoReply" { await.await }
case "Failure" case "Failure"
await.await await.await
throw new RuntimeException("Expected exception; to test fault-tolerance") throw new RuntimeException("Expected exception; to test fault-tolerance")
@ -140,7 +136,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"will return a result" must { "will return a result" must {
behave like futureWithResult { test behave like futureWithResult { test
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String reply(s.toUpperCase) } }) val actor2 = actorOf(new Actor { def receive = { case s: String channel ! s.toUpperCase } })
val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s } val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s }
future.await future.await
test(future, "WORLD") test(future, "WORLD")
@ -152,7 +148,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
behave like futureWithException[ArithmeticException] { test behave like futureWithException[ArithmeticException] { test
filterException[ArithmeticException] { filterException[ArithmeticException] {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String reply(s.length / 0) } }) val actor2 = actorOf(new Actor { def receive = { case s: String channel ! s.length / 0 } })
val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s } val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s }
future.await future.await
test(future, "/ by zero") test(future, "/ by zero")
@ -165,7 +161,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
behave like futureWithException[MatchError] { test behave like futureWithException[MatchError] { test
filterException[MatchError] { filterException[MatchError] {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String reply(s.toUpperCase) } }) val actor2 = actorOf(new Actor { def receive = { case s: String channel ! s.toUpperCase } })
val future = actor1 ? "Hello" flatMap { case i: Int actor2 ? i } val future = actor1 ? "Hello" flatMap { case i: Int actor2 ? i }
future.await future.await
test(future, "World (of class java.lang.String)") test(future, "World (of class java.lang.String)")
@ -182,8 +178,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
filterException[ClassCastException] { filterException[ClassCastException] {
val actor = actorOf(new Actor { val actor = actorOf(new Actor {
def receive = { def receive = {
case s: String reply(s.length) case s: String channel ! s.length
case i: Int reply((i * 2).toString) case i: Int channel ! (i * 2).toString
} }
}) })
@ -214,8 +210,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
case class Res[T](res: T) case class Res[T](res: T)
val actor = actorOf(new Actor { val actor = actorOf(new Actor {
def receive = { def receive = {
case Req(s: String) reply(Res(s.length)) case Req(s: String) channel ! Res(s.length)
case Req(i: Int) reply(Res((i * 2).toString)) case Req(i: Int) channel ! Res((i * 2).toString)
} }
}) })
@ -301,7 +297,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"fold" in { "fold" in {
val actors = (1 to 10).toList map { _ val actors = (1 to 10).toList map { _
actorOf(new Actor { actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); tryReply(add) } def receive = { case (add: Int, wait: Int) Thread.sleep(wait); channel.tryTell(add) }
}) })
} }
val timeout = 10000 val timeout = 10000
@ -312,7 +308,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"fold by composing" in { "fold by composing" in {
val actors = (1 to 10).toList map { _ val actors = (1 to 10).toList map { _
actorOf(new Actor { actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); tryReply(add) } def receive = { case (add: Int, wait: Int) Thread.sleep(wait); channel.tryTell(add) }
}) })
} }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 200), 10000).mapTo[Int] } def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 200), 10000).mapTo[Int] }
@ -327,7 +323,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
case (add: Int, wait: Int) case (add: Int, wait: Int)
Thread.sleep(wait) Thread.sleep(wait)
if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected") if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected")
tryReply(add) channel.tryTell(add)
} }
}) })
} }
@ -359,7 +355,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"shouldReduceResults" in { "shouldReduceResults" in {
val actors = (1 to 10).toList map { _ val actors = (1 to 10).toList map { _
actorOf(new Actor { actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); tryReply(add) } def receive = { case (add: Int, wait: Int) Thread.sleep(wait); channel.tryTell(add) }
}) })
} }
val timeout = 10000 val timeout = 10000
@ -375,7 +371,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
case (add: Int, wait: Int) case (add: Int, wait: Int)
Thread.sleep(wait) Thread.sleep(wait)
if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected") if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected")
tryReply(add) channel.tryTell(add)
} }
}) })
} }
@ -404,7 +400,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
var counter = 1 var counter = 1
def receive = { def receive = {
case 'GetNext case 'GetNext
reply(counter) channel ! counter
counter += 2 counter += 2
} }
}) })

View file

@ -80,12 +80,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
result result
} }
def createMessageInvocation(msg: Any): Envelope = { def createMessageInvocation(msg: Any): Envelope = Envelope(msg, NullChannel)
new Envelope(
actorOf(new Actor { //Dummy actor
def receive = { case _ }
}).asInstanceOf[LocalActorRef].underlying, msg, NullChannel)
}
def ensureInitialMailboxState(config: MailboxType, q: Mailbox) { def ensureInitialMailboxState(config: MailboxType, q: Mailbox) {
q must not be null q must not be null

View file

@ -29,7 +29,7 @@ class PriorityDispatcherSpec extends AkkaSpec {
def receive = { def receive = {
case i: Int acc = i :: acc case i: Int acc = i :: acc
case 'Result tryReply(acc) case 'Result channel.tryTell(acc)
} }
}).withDispatcher(dispatcher)).asInstanceOf[LocalActorRef] }).withDispatcher(dispatcher)).asInstanceOf[LocalActorRef]

View file

@ -41,7 +41,7 @@ class ActorPoolSpec extends AkkaSpec {
case _ case _
count.incrementAndGet count.incrementAndGet
latch.countDown() latch.countDown()
tryReply("success") channel.tryTell("success")
} }
})) }))
@ -88,7 +88,7 @@ class ActorPoolSpec extends AkkaSpec {
def receive = { def receive = {
case req: String { case req: String {
sleepFor(10 millis) sleepFor(10 millis)
tryReply("Response") channel.tryTell("Response")
} }
} }
})) }))

View file

@ -73,7 +73,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val actor = app.actorOf(Props(new Actor { val actor = app.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement() lazy val id = counter.getAndIncrement()
def receive = { def receive = {
case "hit" reply(id) case "hit" channel ! id
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
} }
}), address) }), address)
@ -187,7 +187,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val actor = app.actorOf(Props(new Actor { val actor = app.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement() lazy val id = counter.getAndIncrement()
def receive = { def receive = {
case "hit" reply(id) case "hit" channel ! id
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
} }
}), address) }), address)

View file

@ -458,7 +458,7 @@ class RoutingSpec extends AkkaSpec {
case Stop(None) self.stop() case Stop(None) self.stop()
case Stop(Some(_id)) if (_id == id) self.stop() case Stop(Some(_id)) if (_id == id) self.stop()
case _id: Int if (_id == id) case _id: Int if (_id == id)
case _ Thread sleep 100 * id; tryReply(id) case _ Thread sleep 100 * id; channel.tryTell(id)
} }
override def postStop = { override def postStop = {

View file

@ -21,7 +21,7 @@ class Ticket703Spec extends AkkaSpec {
def receive = { def receive = {
case req: String case req: String
Thread.sleep(6000L) Thread.sleep(6000L)
tryReply("Response") channel.tryTell("Response")
} }
})) }))
}).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 5, 1000))) }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 5, 1000)))

View file

@ -5,6 +5,7 @@ package akka
import akka.config._ import akka.config._
import akka.actor._ import akka.actor._
import event._
import java.net.InetAddress import java.net.InetAddress
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import akka.dispatch.{ Dispatcher, Dispatchers, Future, DefaultPromise } import akka.dispatch.{ Dispatcher, Dispatchers, Future, DefaultPromise }
@ -16,9 +17,6 @@ import akka.dispatch.UnboundedMailbox
import akka.routing.Routing import akka.routing.Routing
import akka.remote.RemoteSupport import akka.remote.RemoteSupport
import akka.serialization.Serialization import akka.serialization.Serialization
import akka.event.EventHandler
import akka.event.EventHandlerLogging
import akka.event.Logging
import java.net.InetSocketAddress import java.net.InetSocketAddress
object AkkaApplication { object AkkaApplication {
@ -119,6 +117,9 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
val RemoteTransport = getString("akka.remote.layer", "akka.remote.netty.NettyRemoteSupport") val RemoteTransport = getString("akka.remote.layer", "akka.remote.netty.NettyRemoteSupport")
val RemoteServerPort = getInt("akka.remote.server.port", 2552) val RemoteServerPort = getInt("akka.remote.server.port", 2552)
val FailureDetectorThreshold: Int = getInt("akka.remote.failure-detector.threshold", 8)
val FailureDetectorMaxSampleSize: Int = getInt("akka.remote.failure-detector.max-sample-size", 1000)
} }
object MistSettings { object MistSettings {
@ -193,6 +194,9 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
val log: Logging = new EventHandlerLogging(eventHandler, this) val log: Logging = new EventHandlerLogging(eventHandler, this)
// TODO think about memory consistency effects when doing funky stuff inside constructor // TODO think about memory consistency effects when doing funky stuff inside constructor
val deadLetters = new DeadLetterActorRef(this)
// TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor
val deployer = new Deployer(this) val deployer = new Deployer(this)
val deathWatch = provider.createDeathWatch() val deathWatch = provider.createDeathWatch()

View file

@ -55,7 +55,7 @@ case class HotSwap(code: ActorRef ⇒ Actor.Receive, discardOld: Boolean = true)
case class Failed(@BeanProperty actor: ActorRef, case class Failed(@BeanProperty actor: ActorRef,
@BeanProperty cause: Throwable) extends AutoReceivedMessage with PossiblyHarmful @BeanProperty cause: Throwable) extends AutoReceivedMessage with PossiblyHarmful
case class ChildTerminated(@BeanProperty child: ActorRef, @BeanProperty cause: Throwable) extends AutoReceivedMessage with PossiblyHarmful case class ChildTerminated(@BeanProperty child: ActorRef) extends AutoReceivedMessage with PossiblyHarmful
case object RevertHotSwap extends AutoReceivedMessage with PossiblyHarmful case object RevertHotSwap extends AutoReceivedMessage with PossiblyHarmful
@ -63,7 +63,7 @@ case object PoisonPill extends AutoReceivedMessage with PossiblyHarmful
case object Kill extends AutoReceivedMessage with PossiblyHarmful case object Kill extends AutoReceivedMessage with PossiblyHarmful
case class Terminated(@BeanProperty actor: ActorRef, @BeanProperty cause: Throwable) extends PossiblyHarmful case class Terminated(@BeanProperty actor: ActorRef) extends PossiblyHarmful
case object ReceiveTimeout extends PossiblyHarmful case object ReceiveTimeout extends PossiblyHarmful
@ -95,8 +95,8 @@ class InvalidMessageException private[akka] (message: String, cause: Throwable =
def this(msg: String) = this(msg, null); def this(msg: String) = this(msg, null);
} }
case class DeathPactException private[akka] (dead: ActorRef, cause: Throwable) case class DeathPactException private[akka] (dead: ActorRef)
extends AkkaException("monitored actor " + dead + " terminated", cause) extends AkkaException("monitored actor " + dead + " terminated")
with NoStackTrace with NoStackTrace
// must not pass InterruptedException to other threads // must not pass InterruptedException to other threads
@ -175,6 +175,11 @@ object Actor {
case _ new LoggingReceive(source, r) case _ new LoggingReceive(source, r)
} }
} }
object emptyBehavior extends Receive {
def isDefinedAt(x: Any) = false
def apply(x: Any) = throw new UnsupportedOperationException("empty behavior apply()")
}
} }
/** /**
@ -223,12 +228,10 @@ trait Actor {
implicit def app = context.app implicit def app = context.app
private def config = context.app.AkkaConfig
/** /**
* The default timeout, based on the config setting 'akka.actor.timeout' * The default timeout, based on the config setting 'akka.actor.timeout'
*/ */
implicit def defaultTimeout = config.ActorTimeout implicit def defaultTimeout = app.AkkaConfig.ActorTimeout
/** /**
* Wrap a Receive partial function in a logging enclosure, which sends a * Wrap a Receive partial function in a logging enclosure, which sends a
@ -244,7 +247,7 @@ trait Actor {
* This method does NOT modify the given Receive unless * This method does NOT modify the given Receive unless
* akka.actor.debug.receive is set within akka.conf. * akka.actor.debug.receive is set within akka.conf.
*/ */
def loggable(self: AnyRef)(r: Receive): Receive = if (config.AddLoggingReceive) LoggingReceive(self, r) else r def loggable(self: AnyRef)(r: Receive): Receive = if (app.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
/** /**
* Some[ActorRef] representation of the 'self' ActorRef reference. * Some[ActorRef] representation of the 'self' ActorRef reference.
@ -252,7 +255,7 @@ trait Actor {
* Mainly for internal use, functions as the implicit sender references when invoking * Mainly for internal use, functions as the implicit sender references when invoking
* the 'forward' function. * the 'forward' function.
*/ */
def someSelf: Some[ActorRef with ScalaActorRef] = Some(context.self) def someSelf: Some[ActorRef with ScalaActorRef] = Some(context.self) //TODO FIXME we might not need this when we switch to sender-in-scope-always
/* /*
* Option[ActorRef] representation of the 'self' ActorRef reference. * Option[ActorRef] representation of the 'self' ActorRef reference.
@ -260,7 +263,7 @@ trait Actor {
* Mainly for internal use, functions as the implicit sender references when invoking * Mainly for internal use, functions as the implicit sender references when invoking
* one of the message send functions ('!' and '?'). * one of the message send functions ('!' and '?').
*/ */
def optionSelf: Option[ActorRef with ScalaActorRef] = someSelf def optionSelf: Option[ActorRef with ScalaActorRef] = someSelf //TODO FIXME we might not need this when we switch to sender-in-scope-always
/** /**
* The 'self' field holds the ActorRef for this actor. * The 'self' field holds the ActorRef for this actor.
@ -276,20 +279,15 @@ trait Actor {
* The reference sender Actor of the last received message. * The reference sender Actor of the last received message.
* Is defined if the message was sent from another Actor, else None. * Is defined if the message was sent from another Actor, else None.
*/ */
def sender: Option[ActorRef] = context.sender @inline
final def sender: ActorRef = context.sender
/**
* The reference sender future of the last received message.
* Is defined if the message was sent with sent with '?'/'ask', else None.
*/
def senderFuture(): Option[Promise[Any]] = context.senderFuture
/** /**
* Abstraction for unification of sender and senderFuture for later reply * Abstraction for unification of sender and senderFuture for later reply
*/ */
def channel: UntypedChannel = context.channel def channel: UntypedChannel = context.channel
// just for current compatibility // TODO FIXME REMOVE ME just for current compatibility
implicit def forwardable: ForwardableChannel = ForwardableChannel(channel) implicit def forwardable: ForwardableChannel = ForwardableChannel(channel)
/** /**
@ -306,31 +304,6 @@ trait Actor {
*/ */
def receiveTimeout_=(timeout: Option[Long]) = context.receiveTimeout = timeout def receiveTimeout_=(timeout: Option[Long]) = context.receiveTimeout = timeout
/**
* Akka Scala & Java API
* Use <code>reply(..)</code> to reply with a message to the original sender of the message currently
* being processed. This method fails if the original sender of the message could not be determined with an
* IllegalStateException.
*
* If you don't want deal with this IllegalStateException, but just a boolean, just use the <code>tryReply(...)</code>
* version.
*
* <p/>
* Throws an IllegalStateException if unable to determine what to reply to.
*/
def reply(message: Any) = channel.!(message)(self)
/**
* Akka Scala & Java API
* Use <code>tryReply(..)</code> to try reply with a message to the original sender of the message currently
* being processed. This method
* <p/>
* Returns true if reply was sent, and false if unable to determine what to reply to.
*
* If you would rather have an exception, check the <code>reply(..)</code> version.
*/
def tryReply(message: Any): Boolean = channel.tryTell(message)(self)
/** /**
* Same as ActorContext.children * Same as ActorContext.children
*/ */
@ -352,7 +325,7 @@ trait Actor {
* def receive = { * def receive = {
* case Ping =&gt; * case Ping =&gt;
* println("got a 'Ping' message") * println("got a 'Ping' message")
* reply("pong") * channel ! "pong"
* *
* case OneWay =&gt; * case OneWay =&gt;
* println("got a 'OneWay' message") * println("got a 'OneWay' message")
@ -403,8 +376,8 @@ trait Actor {
*/ */
def unhandled(message: Any) { def unhandled(message: Any) {
message match { message match {
case Terminated(dead, cause) throw new DeathPactException(dead, cause) case Terminated(dead) throw new DeathPactException(dead)
case _ throw new UnhandledMessageException(message, self) case _ throw new UnhandledMessageException(message, self)
} }
} }
@ -426,16 +399,26 @@ trait Actor {
if (h.nonEmpty) context.hotswap = h.pop if (h.nonEmpty) context.hotswap = h.pop
} }
/**
* Registers this actor as a Monitor for the provided ActorRef
* @returns the provided ActorRef
*/
def watch(subject: ActorRef): ActorRef = self startsMonitoring subject
/**
* Unregisters this actor as Monitor for the provided ActorRef
* @returns the provided ActorRef
*/
def unwatch(subject: ActorRef): ActorRef = self stopsMonitoring subject
// ========================================= // =========================================
// ==== INTERNAL IMPLEMENTATION DETAILS ==== // ==== INTERNAL IMPLEMENTATION DETAILS ====
// ========================================= // =========================================
private[akka] final def apply(msg: Any) = { private[akka] final def apply(msg: Any) = {
if (msg.isInstanceOf[AnyRef] && (msg.asInstanceOf[AnyRef] eq null))
throw new InvalidMessageException("Message from [" + channel + "] to [" + self + "] is null")
def autoReceiveMessage(msg: AutoReceivedMessage) { def autoReceiveMessage(msg: AutoReceivedMessage) {
if (config.DebugAutoReceive) app.eventHandler.debug(this, "received AutoReceiveMessage " + msg) if (app.AkkaConfig.DebugAutoReceive) app.eventHandler.debug(this, "received AutoReceiveMessage " + msg)
msg match { msg match {
case HotSwap(code, discardOld) become(code(self), discardOld) case HotSwap(code, discardOld) become(code(self), discardOld)

View file

@ -34,9 +34,7 @@ private[akka] trait ActorContext extends ActorRefFactory with TypedActorFactory
def currentMessage_=(invocation: Envelope): Unit def currentMessage_=(invocation: Envelope): Unit
def sender: Option[ActorRef] def sender: ActorRef
def senderFuture(): Option[Promise[Any]]
def channel: UntypedChannel def channel: UntypedChannel
@ -126,7 +124,7 @@ private[akka] class ActorCell(
def children: Iterable[ActorRef] = _children.keys def children: Iterable[ActorRef] = _children.keys
def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = dispatcher dispatch Envelope(this, message, channel) def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = dispatcher.dispatch(this, Envelope(message, channel))
def postMessageToMailboxAndCreateFutureResultWithTimeout( def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any, message: Any,
@ -136,20 +134,14 @@ private[akka] class ActorCell(
case f: ActorPromise f case f: ActorPromise f
case _ new ActorPromise(timeout)(dispatcher) case _ new ActorPromise(timeout)(dispatcher)
} }
dispatcher dispatch Envelope(this, message, future) dispatcher.dispatch(this, Envelope(message, future))
future future
} }
def sender: Option[ActorRef] = currentMessage match { def sender: ActorRef = currentMessage match {
case null None case null app.deadLetters
case msg if msg.channel.isInstanceOf[ActorRef] Some(msg.channel.asInstanceOf[ActorRef]) case msg if msg.channel.isInstanceOf[ActorRef] msg.channel.asInstanceOf[ActorRef]
case _ None case _ app.deadLetters
}
def senderFuture(): Option[Promise[Any]] = currentMessage match {
case null None
case msg if msg.channel.isInstanceOf[ActorPromise] Some(msg.channel.asInstanceOf[ActorPromise])
case _ None
} }
def channel: UntypedChannel = currentMessage match { def channel: UntypedChannel = currentMessage match {
@ -250,10 +242,8 @@ private[akka] class ActorCell(
} }
} finally { } finally {
try { try {
// when changing this, remember to update the match in the BubbleWalker supervisor ! ChildTerminated(self)
val cause = new ActorKilledException("Stopped") //FIXME TODO make this an object, can be reused everywhere app.deathWatch.publish(Terminated(self))
supervisor ! ChildTerminated(self, cause)
app.deathWatch.publish(Terminated(self, cause))
} finally { } finally {
currentMessage = null currentMessage = null
clearActorContext() clearActorContext()

View file

@ -9,6 +9,7 @@ import akka.util._
import scala.collection.immutable.Stack import scala.collection.immutable.Stack
import java.lang.{ UnsupportedOperationException, IllegalStateException } import java.lang.{ UnsupportedOperationException, IllegalStateException }
import akka.AkkaApplication import akka.AkkaApplication
import akka.event.ActorEventBus
/** /**
* ActorRef is an immutable and serializable handle to an Actor. * ActorRef is an immutable and serializable handle to an Actor.
@ -68,7 +69,7 @@ abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyCha
* Use this method with care. In most cases it is better to use 'tell' together with the 'getContext().getSender()' to * Use this method with care. In most cases it is better to use 'tell' together with the 'getContext().getSender()' to
* implement request/response message exchanges. * implement request/response message exchanges.
* <p/> * <p/>
* If you are sending messages using <code>ask</code> then you <b>have to</b> use <code>getContext().reply(..)</code> * If you are sending messages using <code>ask</code> then you <b>have to</b> use <code>getContext().channel().tell(...)</code>
* to send a reply message to the original sender. If not then the sender will block until the timeout expires. * to send a reply message to the original sender. If not then the sender will block until the timeout expires.
*/ */
def ask(message: AnyRef, timeout: Long, sender: ActorRef): Future[AnyRef] = def ask(message: AnyRef, timeout: Long, sender: ActorRef): Future[AnyRef] =
@ -361,6 +362,8 @@ trait UnsupportedActorRef extends ActorRef with ScalaActorRef {
private def unsupported = throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName)) private def unsupported = throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName))
} }
case class DeadLetter(message: Any, channel: UntypedChannel)
class DeadLetterActorRef(app: AkkaApplication) extends UnsupportedActorRef { class DeadLetterActorRef(app: AkkaApplication) extends UnsupportedActorRef {
val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher) val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher)
override val address: String = "akka:internal:DeadLetterActorRef" override val address: String = "akka:internal:DeadLetterActorRef"
@ -375,10 +378,10 @@ class DeadLetterActorRef(app: AkkaApplication) extends UnsupportedActorRef {
override def stop(): Unit = () override def stop(): Unit = ()
protected[akka] override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = app.eventHandler.debug(this, message) protected[akka] override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = app.eventHandler.notify(DeadLetter(message, channel))
protected[akka] override def postMessageToMailboxAndCreateFutureResultWithTimeout( protected[akka] override def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any, message: Any,
timeout: Timeout, timeout: Timeout,
channel: UntypedChannel): Future[Any] = { app.eventHandler.debug(this, message); brokenPromise } channel: UntypedChannel): Future[Any] = { app.eventHandler.notify(DeadLetter(message, channel)); brokenPromise }
} }

View file

@ -100,12 +100,9 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
protected[akka] override def postMessageToMailbox(msg: Any, channel: UntypedChannel) { protected[akka] override def postMessageToMailbox(msg: Any, channel: UntypedChannel) {
msg match { msg match {
case Failed(child, ex) child.stop() case Failed(child, ex) child.stop()
case ChildTerminated(child, ex) ex match { case ChildTerminated(child) terminationFuture.completeWithResult(AkkaApplication.Stopped)
case a: ActorKilledException if a.getMessage == "Stopped" terminationFuture.completeWithResult(AkkaApplication.Stopped) case _ app.eventHandler.error(this, this + " received unexpected message " + msg)
case x terminationFuture.completeWithResult(AkkaApplication.Failed(x))
}
case _ app.eventHandler.error(this, this + " received unexpected message " + msg)
} }
} }
@ -224,7 +221,7 @@ class LocalDeathWatch extends DeathWatch with ActorClassification {
override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = { override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = {
if (!super.subscribe(subscriber, to)) { if (!super.subscribe(subscriber, to)) {
subscriber ! Terminated(subscriber, new ActorKilledException("Already terminated when linking")) subscriber ! Terminated(to)
false false
} else true } else true
} }

View file

@ -502,7 +502,7 @@ trait FSM[S, D] extends ListenerManagement {
nextState.stopReason match { nextState.stopReason match {
case None makeTransition(nextState) case None makeTransition(nextState)
case _ case _
nextState.replies.reverse foreach reply nextState.replies.reverse foreach { r channel ! r }
terminate(nextState) terminate(nextState)
self.stop() self.stop()
} }
@ -512,7 +512,7 @@ trait FSM[S, D] extends ListenerManagement {
if (!stateFunctions.contains(nextState.stateName)) { if (!stateFunctions.contains(nextState.stateName)) {
terminate(stay withStopReason Failure("Next state %s does not exist".format(nextState.stateName))) terminate(stay withStopReason Failure("Next state %s does not exist".format(nextState.stateName)))
} else { } else {
nextState.replies.reverse foreach reply nextState.replies.reverse foreach { r channel ! r }
if (currentState.stateName != nextState.stateName) { if (currentState.stateName != nextState.stateName) {
handleTransition(currentState.stateName, nextState.stateName) handleTransition(currentState.stateName, nextState.stateName)
notifyListeners(Transition(self, currentState.stateName, nextState.stateName)) notifyListeners(Transition(self, currentState.stateName, nextState.stateName))

View file

@ -133,7 +133,7 @@ abstract class FaultHandlingStrategy {
/** /**
* Returns whether it processed the failure or not * Returns whether it processed the failure or not
*/ */
final def handleFailure(fail: Failed, stats: ChildRestartStats, children: Iterable[(ActorRef, ChildRestartStats)]): Boolean = { def handleFailure(fail: Failed, stats: ChildRestartStats, children: Iterable[(ActorRef, ChildRestartStats)]): Boolean = {
val cause = fail.cause val cause = fail.cause
val action = if (decider.isDefinedAt(cause)) decider(cause) else Escalate val action = if (decider.isDefinedAt(cause)) decider(cause) else Escalate
action match { action match {

View file

@ -42,7 +42,7 @@ object Props {
*/ */
def apply(): Props = default def apply(): Props = default
def empty = Props(context { case null }) val empty = Props(new Actor { def receive = Actor.emptyBehavior })
/** /**
* Returns a Props that has default values except for "creator" which will be a function that creates an instance * Returns a Props that has default values except for "creator" which will be a function that creates an instance

View file

@ -130,7 +130,15 @@ trait TypedActorFactory { this: ActorRefFactory ⇒
* all interfaces (Class.getInterfaces) if it's not an interface class * all interfaces (Class.getInterfaces) if it's not an interface class
*/ */
def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props): R = def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props): R =
typedActor.createProxyAndTypedActor(this, interface, impl.newInstance, props, interface.getClassLoader) typedActor.createProxyAndTypedActor(this, interface, impl.newInstance, props, Props.randomAddress, interface.getClassLoader)
/**
* Creates a new TypedActor proxy using the supplied Props,
* the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or
* all interfaces (Class.getInterfaces) if it's not an interface class
*/
def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props, address: String): R =
typedActor.createProxyAndTypedActor(this, interface, impl.newInstance, props, address, interface.getClassLoader)
/** /**
* Creates a new TypedActor proxy using the supplied Props, * Creates a new TypedActor proxy using the supplied Props,
@ -138,10 +146,31 @@ trait TypedActorFactory { this: ActorRefFactory ⇒
* all interfaces (Class.getInterfaces) if it's not an interface class * all interfaces (Class.getInterfaces) if it's not an interface class
*/ */
def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props): R = def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props): R =
typedActor.createProxyAndTypedActor(this, interface, impl.create, props, interface.getClassLoader) typedActor.createProxyAndTypedActor(this, interface, impl.create, props, Props.randomAddress, interface.getClassLoader)
/**
* Creates a new TypedActor proxy using the supplied Props,
* the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or
* all interfaces (Class.getInterfaces) if it's not an interface class
*/
def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props, address: String): R =
typedActor.createProxyAndTypedActor(this, interface, impl.create, props, address, interface.getClassLoader)
/**
* Creates a new TypedActor proxy using the supplied Props,
* the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or
* all interfaces (Class.getInterfaces) if it's not an interface class
*/
def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props, loader: ClassLoader): R = def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props, loader: ClassLoader): R =
typedActor.createProxyAndTypedActor(this, interface, impl.newInstance, props, loader) typedActor.createProxyAndTypedActor(this, interface, impl.newInstance, props, Props.randomAddress, loader)
/**
* Creates a new TypedActor proxy using the supplied Props,
* the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or
* all interfaces (Class.getInterfaces) if it's not an interface class
*/
def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props, address: String, loader: ClassLoader): R =
typedActor.createProxyAndTypedActor(this, interface, impl.newInstance, props, address, loader)
/** /**
* Creates a new TypedActor proxy using the supplied Props, * Creates a new TypedActor proxy using the supplied Props,
@ -149,44 +178,73 @@ trait TypedActorFactory { this: ActorRefFactory ⇒
* all interfaces (Class.getInterfaces) if it's not an interface class * all interfaces (Class.getInterfaces) if it's not an interface class
*/ */
def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props, loader: ClassLoader): R = def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props, loader: ClassLoader): R =
typedActor.createProxyAndTypedActor(this, interface, impl.create, props, loader) typedActor.createProxyAndTypedActor(this, interface, impl.create, props, Props.randomAddress, loader)
/**
* Creates a new TypedActor proxy using the supplied Props,
* the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or
* all interfaces (Class.getInterfaces) if it's not an interface class
*/
def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props, address: String, loader: ClassLoader): R =
typedActor.createProxyAndTypedActor(this, interface, impl.create, props, address, loader)
/** /**
* Creates a new TypedActor proxy using the supplied Props, * Creates a new TypedActor proxy using the supplied Props,
* the interfaces usable by the returned proxy is the supplied implementation class' interfaces (Class.getInterfaces) * the interfaces usable by the returned proxy is the supplied implementation class' interfaces (Class.getInterfaces)
*/ */
def typedActorOf[R <: AnyRef, T <: R](impl: Class[T], props: Props, loader: ClassLoader): R = def typedActorOf[R <: AnyRef, T <: R](impl: Class[T], props: Props, loader: ClassLoader): R =
typedActor.createProxyAndTypedActor(this, impl, impl.newInstance, props, loader) typedActor.createProxyAndTypedActor(this, impl, impl.newInstance, props, Props.randomAddress, loader)
/** /**
* Creates a new TypedActor proxy using the supplied Props, * Creates a new TypedActor proxy using the supplied Props,
* the interfaces usable by the returned proxy is the supplied implementation class' interfaces (Class.getInterfaces) * the interfaces usable by the returned proxy is the supplied implementation class' interfaces (Class.getInterfaces)
*/ */
def typedActorOf[R <: AnyRef, T <: R](props: Props = Props(), loader: ClassLoader = null)(implicit m: Manifest[T]): R = { def typedActorOf[R <: AnyRef, T <: R](impl: Class[T], props: Props, address: String, loader: ClassLoader): R =
typedActor.createProxyAndTypedActor(this, impl, impl.newInstance, props, address, loader)
/**
* Creates a new TypedActor proxy using the supplied Props,
* the interfaces usable by the returned proxy is the supplied implementation class' interfaces (Class.getInterfaces)
*/
def typedActorOf[R <: AnyRef, T <: R](props: Props = Props(), address: String = Props.randomAddress, loader: ClassLoader = null)(implicit m: Manifest[T]): R = {
val clazz = m.erasure.asInstanceOf[Class[T]] val clazz = m.erasure.asInstanceOf[Class[T]]
typedActor.createProxyAndTypedActor(this, clazz, clazz.newInstance, props, if (loader eq null) clazz.getClassLoader else loader) typedActor.createProxyAndTypedActor(this, clazz, clazz.newInstance, props, address, if (loader eq null) clazz.getClassLoader else loader)
} }
/** /**
* Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself, * Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself,
* to create TypedActor proxies, use typedActorOf * to create TypedActor proxies, use typedActorOf
*/ */
def createProxy[R <: AnyRef](constructor: Actor, props: Props = Props(), loader: ClassLoader = null)(implicit m: Manifest[R]): R = def createProxy[R <: AnyRef](constructor: Actor, props: Props = Props(), address: String = Props.randomAddress, loader: ClassLoader = null)(implicit m: Manifest[R]): R =
typedActor.createProxy[R](this, typedActor.extractInterfaces(m.erasure), (ref: AtomVar[R]) constructor, props, if (loader eq null) m.erasure.getClassLoader else loader) typedActor.createProxy[R](this, typedActor.extractInterfaces(m.erasure), (ref: AtomVar[R]) constructor, props, Props.randomAddress, if (loader eq null) m.erasure.getClassLoader else loader)
/** /**
* Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself, * Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself,
* to create TypedActor proxies, use typedActorOf * to create TypedActor proxies, use typedActorOf
*/ */
def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: Creator[Actor], props: Props, loader: ClassLoader): R = def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: Creator[Actor], props: Props, loader: ClassLoader): R =
typedActor.createProxy(this, interfaces, (ref: AtomVar[R]) constructor.create, props, loader) typedActor.createProxy(this, interfaces, (ref: AtomVar[R]) constructor.create, props, Props.randomAddress, loader)
/**
* Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself,
* to create TypedActor proxies, use typedActorOf
*/
def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: Creator[Actor], props: Props, address: String, loader: ClassLoader): R =
typedActor.createProxy(this, interfaces, (ref: AtomVar[R]) constructor.create, props, address, loader)
/** /**
* Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself, * Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself,
* to create TypedActor proxies, use typedActorOf * to create TypedActor proxies, use typedActorOf
*/ */
def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: Actor, props: Props, loader: ClassLoader): R = def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: Actor, props: Props, loader: ClassLoader): R =
typedActor.createProxy[R](this, interfaces, (ref: AtomVar[R]) constructor, props, loader) typedActor.createProxy[R](this, interfaces, (ref: AtomVar[R]) constructor, props, Props.randomAddress, loader)
/**
* Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself,
* to create TypedActor proxies, use typedActorOf
*/
def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: Actor, props: Props, address: String, loader: ClassLoader): R =
typedActor.createProxy[R](this, interfaces, (ref: AtomVar[R]) constructor, props, address, loader)
} }
@ -244,15 +302,15 @@ class TypedActor(val app: AkkaApplication) {
} }
else null else null
private[akka] def createProxy[R <: AnyRef](supervisor: ActorRefFactory, interfaces: Array[Class[_]], constructor: (AtomVar[R]) Actor, props: Props, loader: ClassLoader): R = { private[akka] def createProxy[R <: AnyRef](supervisor: ActorRefFactory, interfaces: Array[Class[_]], constructor: (AtomVar[R]) Actor, props: Props, address: String, loader: ClassLoader): R = {
val proxyVar = new AtomVar[R] val proxyVar = new AtomVar[R]
configureAndProxyLocalActorRef[R](supervisor, interfaces, proxyVar, props.withCreator(constructor(proxyVar)), loader) configureAndProxyLocalActorRef[R](supervisor, interfaces, proxyVar, props.withCreator(constructor(proxyVar)), address, loader)
} }
private[akka] def createProxyAndTypedActor[R <: AnyRef, T <: R](supervisor: ActorRefFactory, interface: Class[_], constructor: T, props: Props, loader: ClassLoader): R = private[akka] def createProxyAndTypedActor[R <: AnyRef, T <: R](supervisor: ActorRefFactory, interface: Class[_], constructor: T, props: Props, address: String, loader: ClassLoader): R =
createProxy[R](supervisor, extractInterfaces(interface), (ref: AtomVar[R]) new TypedActor[R, T](ref, constructor), props, loader) createProxy[R](supervisor, extractInterfaces(interface), (ref: AtomVar[R]) new TypedActor[R, T](ref, constructor), props, address, loader)
private[akka] def configureAndProxyLocalActorRef[T <: AnyRef](supervisor: ActorRefFactory, interfaces: Array[Class[_]], proxyVar: AtomVar[T], props: Props, loader: ClassLoader): T = { private[akka] def configureAndProxyLocalActorRef[T <: AnyRef](supervisor: ActorRefFactory, interfaces: Array[Class[_]], proxyVar: AtomVar[T], props: Props, address: String, loader: ClassLoader): T = {
//Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling //Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling
val actorVar = new AtomVar[ActorRef](null) val actorVar = new AtomVar[ActorRef](null)
val timeout = props.timeout match { val timeout = props.timeout match {
@ -261,7 +319,7 @@ class TypedActor(val app: AkkaApplication) {
} }
val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)(timeout)).asInstanceOf[T] val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)(timeout)).asInstanceOf[T]
proxyVar.set(proxy) // Chicken and egg situation we needed to solve, set the proxy so that we can set the self-reference inside each receive proxyVar.set(proxy) // Chicken and egg situation we needed to solve, set the proxy so that we can set the self-reference inside each receive
val ref = supervisor.actorOf(props) val ref = supervisor.actorOf(props, address)
actorVar.set(ref) //Make sure the InvocationHandler gets ahold of the actor reference, this is not a problem since the proxy hasn't escaped this method yet actorVar.set(ref) //Make sure the InvocationHandler gets ahold of the actor reference, this is not a problem since the proxy hasn't escaped this method yet
proxyVar.get proxyVar.get
} }
@ -286,7 +344,7 @@ class TypedActor(val app: AkkaApplication) {
case p: ActorPromise p completeWith m(me).asInstanceOf[Future[Any]] case p: ActorPromise p completeWith m(me).asInstanceOf[Future[Any]]
case _ throw new IllegalStateException("Future-returning TypedActor didn't use ?/ask so cannot reply") case _ throw new IllegalStateException("Future-returning TypedActor didn't use ?/ask so cannot reply")
} }
} else reply(m(me)) } else channel ! m(me)
} finally { } finally {
TypedActor.selfReference set null TypedActor.selfReference set null

View file

@ -21,17 +21,13 @@ import akka.dispatch.{ MessageDispatcher, Promise }
* *
* if (msg.equals("UseReply")) { * if (msg.equals("UseReply")) {
* // Reply to original sender of message using the 'reply' method * // Reply to original sender of message using the 'reply' method
* reply(msg + ":" + getSelf().getAddress()); * getContext().getChannel().tell(msg + ":" + getSelf().getAddress());
* *
* } else if (msg.equals("UseSender") && getSender().isDefined()) { * } else if (msg.equals("UseSender") && getSender().isDefined()) {
* // Reply to original sender of message using the sender reference * // Reply to original sender of message using the sender reference
* // also passing along my own reference (the self) * // also passing along my own reference (the self)
* getSender().get().tell(msg, getSelf()); * getSender().get().tell(msg, getSelf());
* *
* } else if (msg.equals("UseSenderFuture") && getSenderFuture().isDefined()) {
* // Reply to original sender of message using the sender future reference
* getSenderFuture().get().completeWithResult(msg);
*
* } else if (msg.equals("SendToSelf")) { * } else if (msg.equals("SendToSelf")) {
* // Send message to the actor itself recursively * // Send message to the actor itself recursively
* getSelf().tell(msg) * getSelf().tell(msg)
@ -71,13 +67,7 @@ abstract class UntypedActor extends Actor {
* The reference sender Actor of the last received message. * The reference sender Actor of the last received message.
* Is defined if the message was sent from another Actor, else None. * Is defined if the message was sent from another Actor, else None.
*/ */
def getSender: Option[ActorRef] = sender def getSender: ActorRef = sender
/**
* The reference sender future of the last received message.
* Is defined if the message was sent with sent with '?'/'ask', else None.
*/
def getSenderFuture: Option[Promise[Any]] = senderFuture
/** /**
* Abstraction for unification of sender and senderFuture for later reply * Abstraction for unification of sender and senderFuture for later reply

View file

@ -17,12 +17,8 @@ import scala.annotation.tailrec
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
final case class Envelope(val receiver: ActorCell, val message: Any, val channel: UntypedChannel) { final case class Envelope(val message: Any, val channel: UntypedChannel) {
if (receiver eq null) throw new IllegalArgumentException("Receiver can't be null") if (message.isInstanceOf[AnyRef] && (message.asInstanceOf[AnyRef] eq null)) throw new InvalidMessageException("Message is null")
final def invoke() {
receiver invoke this
}
} }
object SystemMessage { object SystemMessage {
@ -295,7 +291,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable
/** /**
* Will be called when the dispatcher is to queue an invocation for execution * Will be called when the dispatcher is to queue an invocation for execution
*/ */
protected[akka] def dispatch(invocation: Envelope) protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope)
/** /**
* Suggest to register the provided mailbox for execution * Suggest to register the provided mailbox for execution

View file

@ -57,12 +57,7 @@ class BalancingDispatcher(
class SharingMailbox(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue { class SharingMailbox(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue {
final def enqueue(handle: Envelope) = messageQueue.enqueue(handle) final def enqueue(handle: Envelope) = messageQueue.enqueue(handle)
final def dequeue(): Envelope = { final def dequeue(): Envelope = messageQueue.dequeue()
val envelope = messageQueue.dequeue()
if (envelope eq null) null
else if (envelope.receiver eq actor) envelope
else envelope.copy(receiver = actor)
}
final def numberOfMessages: Int = messageQueue.numberOfMessages final def numberOfMessages: Int = messageQueue.numberOfMessages
@ -95,6 +90,8 @@ class BalancingDispatcher(
while (messages ne null) { while (messages ne null) {
deadLetterMailbox.systemEnqueue(messages) //Send to dead letter queue deadLetterMailbox.systemEnqueue(messages) //Send to dead letter queue
messages = messages.next messages = messages.next
if (messages eq null) //Make sure that any system messages received after the current drain are also sent to the dead letter mbox
messages = mailBox.systemDrain()
} }
} }
} }
@ -106,8 +103,7 @@ class BalancingDispatcher(
} else true } else true
} }
override protected[akka] def dispatch(invocation: Envelope) = { override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
val receiver = invocation.receiver
messageQueue enqueue invocation messageQueue enqueue invocation
val buddy = buddies.pollFirst() val buddy = buddies.pollFirst()

View file

@ -78,8 +78,8 @@ class Dispatcher(
protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name) protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name)
protected[akka] val executorService = new AtomicReference[ExecutorService](new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService)) protected[akka] val executorService = new AtomicReference[ExecutorService](new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService))
protected[akka] def dispatch(invocation: Envelope) = { protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
val mbox = invocation.receiver.mailbox val mbox = receiver.mailbox
mbox enqueue invocation mbox enqueue invocation
registerForExecution(mbox, true, false) registerForExecution(mbox, true, false)
} }

View file

@ -30,7 +30,7 @@ private[dispatch] object Mailbox {
// secondary status: Scheduled bit may be added to Open/Suspended // secondary status: Scheduled bit may be added to Open/Suspended
final val Scheduled = 4 final val Scheduled = 4
// static constant for enabling println debugging of message processing (for hardcore bugs) // mailbox debugging helper using println (see below)
final val debug = false final val debug = false
} }
@ -172,8 +172,8 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag
var processedMessages = 0 var processedMessages = 0
val deadlineNs = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + TimeUnit.MILLISECONDS.toNanos(dispatcher.throughputDeadlineTime) else 0 val deadlineNs = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + TimeUnit.MILLISECONDS.toNanos(dispatcher.throughputDeadlineTime) else 0
do { do {
if (debug) println(actor + " processing message " + nextMessage.message + " from " + nextMessage.channel) if (debug) println(actor + " processing message " + nextMessage)
nextMessage.invoke actor invoke nextMessage
processAllSystemMessages() //After we're done, process all system messages processAllSystemMessages() //After we're done, process all system messages
@ -185,7 +185,7 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag
} else null //Abort } else null //Abort
} while (nextMessage ne null) } while (nextMessage ne null)
} else { //If we only run one message per process } else { //If we only run one message per process
nextMessage.invoke //Just run it actor invoke nextMessage //Just run it
processAllSystemMessages() //After we're done, process all system messages processAllSystemMessages() //After we're done, process all system messages
} }
} }
@ -202,7 +202,6 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag
// dont ever execute normal message when system message present! // dont ever execute normal message when system message present!
if (nextMessage eq null) nextMessage = systemDrain() if (nextMessage eq null) nextMessage = systemDrain()
} }
if (debug) println(actor + " has finished processing system messages")
} catch { } catch {
case e case e
actor.app.eventHandler.error(e, this, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!") actor.app.eventHandler.error(e, this, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!")
@ -244,7 +243,7 @@ trait DefaultSystemMessageQueue { self: Mailbox ⇒
@tailrec @tailrec
final def systemEnqueue(message: SystemMessage): Unit = { final def systemEnqueue(message: SystemMessage): Unit = {
if (Mailbox.debug) println(actor + " having system message enqueued: " + message) if (Mailbox.debug) println(actor + " having enqueued " + message)
val head = systemQueueGet val head = systemQueueGet
/* /*
* this write is safely published by the compareAndSet contained within * this write is safely published by the compareAndSet contained within

View file

@ -182,10 +182,7 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒
if (monitored.isShutdown) false if (monitored.isShutdown) false
else { else {
if (mappings.putIfAbsent(monitored, Vector(monitor)) ne null) associate(monitored, monitor) if (mappings.putIfAbsent(monitored, Vector(monitor)) ne null) associate(monitored, monitor)
else { else if (monitored.isShutdown) !dissociate(monitored, monitor) else true
if (monitored.isShutdown) !dissociate(monitored, monitor)
else true
}
} }
case raw: Vector[_] case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]] val v = raw.asInstanceOf[Vector[ActorRef]]
@ -194,10 +191,7 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒
else { else {
val added = v :+ monitor val added = v :+ monitor
if (!mappings.replace(monitored, v, added)) associate(monitored, monitor) if (!mappings.replace(monitored, v, added)) associate(monitored, monitor)
else { else if (monitored.isShutdown) !dissociate(monitored, monitor) else true
if (monitored.isShutdown) !dissociate(monitored, monitor)
else true
}
} }
} }
} }
@ -241,13 +235,11 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒
case raw: Vector[_] case raw: Vector[_]
val v = raw.asInstanceOf[Vector[ActorRef]] val v = raw.asInstanceOf[Vector[ActorRef]]
val removed = v.filterNot(monitor ==) val removed = v.filterNot(monitor ==)
if (removed eq v) false if (removed eq raw) false
else if (removed.isEmpty) { else if (removed.isEmpty) {
if (!mappings.remove(monitored, v)) dissociate(monitored, monitor) if (!mappings.remove(monitored, v)) dissociate(monitored, monitor) else true
else true
} else { } else {
if (!mappings.replace(monitored, v, removed)) dissociate(monitored, monitor) if (!mappings.replace(monitored, v, removed)) dissociate(monitored, monitor) else true
else true
} }
} }
} }
@ -263,10 +255,8 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒
protected def mapSize: Int protected def mapSize: Int
def publish(event: Event): Unit = mappings.get(classify(event)) match { def publish(event: Event): Unit = mappings.get(classify(event)) match {
case null case null
case raw: Vector[_] case raw: Vector[_] raw.asInstanceOf[Vector[ActorRef]] foreach { _ ! event }
val v = raw.asInstanceOf[Vector[ActorRef]]
v foreach { _ ! event }
} }
def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber) def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber)

View file

@ -14,6 +14,7 @@ import java.net.InetSocketAddress
/** /**
* An Iterable that also contains a version. * An Iterable that also contains a version.
*/ */
// FIXME REMOVE VersionedIterable
trait VersionedIterable[A] { trait VersionedIterable[A] {
val version: Long val version: Long

View file

@ -103,8 +103,8 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒
protected def _route(): Actor.Receive = { protected def _route(): Actor.Receive = {
// for testing... // for testing...
case Stat case Stat
tryReply(Stats(_delegates length)) channel.tryTell(Stats(_delegates length))
case Terminated(victim, _) case Terminated(victim)
_delegates = _delegates filterNot { victim == } _delegates = _delegates filterNot { victim == }
case msg case msg
resizeIfAppropriate() resizeIfAppropriate()

View file

@ -24,7 +24,7 @@ object TypedCamelTestSupport {
def countdown: Handler = { def countdown: Handler = {
case SetExpectedMessageCount(num) { case SetExpectedMessageCount(num) {
latch = new CountDownLatch(num) latch = new CountDownLatch(num)
reply(latch) channel ! latch
} }
case msg latch.countDown case msg latch.countDown
} }
@ -32,7 +32,7 @@ object TypedCamelTestSupport {
trait Respond { this: Actor trait Respond { this: Actor
def respond: Handler = { def respond: Handler = {
case msg: Message reply(response(msg)) case msg: Message channel ! response(msg)
} }
def response(msg: Message): Any = "Hello %s" format msg.body def response(msg: Message): Any = "Hello %s" format msg.body
@ -42,8 +42,8 @@ object TypedCamelTestSupport {
val messages = Buffer[Any]() val messages = Buffer[Any]()
def retain: Handler = { def retain: Handler = {
case GetRetainedMessage reply(messages.last) case GetRetainedMessage channel ! messages.last
case GetRetainedMessages(p) reply(messages.toList.filter(p)) case GetRetainedMessages(p) channel ! messages.filter(p).toList
case msg { case msg {
messages += msg messages += msg
msg msg

View file

@ -127,11 +127,11 @@ private[camel] class ActivationTracker extends Actor {
def receive = { def receive = {
case SetExpectedActivationCount(num) { case SetExpectedActivationCount(num) {
activationLatch = new CountDownLatch(num) activationLatch = new CountDownLatch(num)
reply(activationLatch) channel ! activationLatch
} }
case SetExpectedDeactivationCount(num) { case SetExpectedDeactivationCount(num) {
deactivationLatch = new CountDownLatch(num) deactivationLatch = new CountDownLatch(num)
reply(deactivationLatch) channel ! deactivationLatch
} }
case EndpointActivated activationLatch.countDown case EndpointActivated activationLatch.countDown
case EndpointDeactivated deactivationLatch.countDown case EndpointDeactivated deactivationLatch.countDown

View file

@ -159,7 +159,7 @@ trait ProducerSupport { this: Actor ⇒
* actor). * actor).
*/ */
protected def receiveAfterProduce: Receive = { protected def receiveAfterProduce: Receive = {
case msg if (!oneway) reply(msg) case msg if (!oneway) channel ! msg
} }
/** /**

View file

@ -15,7 +15,7 @@ public class SampleUntypedConsumer extends UntypedConsumerActor {
Message msg = (Message)message; Message msg = (Message)message;
String body = msg.getBodyAs(String.class); String body = msg.getBodyAs(String.class);
String header = msg.getHeaderAs("test", String.class); String header = msg.getHeaderAs("test", String.class);
tryReply(String.format("%s %s", body, header)); channel.tryTell(String.format("%s %s", body, header));
} }
} }

View file

@ -17,7 +17,7 @@ public class SampleUntypedConsumerBlocking extends UntypedConsumerActor {
Message msg = (Message)message; Message msg = (Message)message;
String body = msg.getBodyAs(String.class); String body = msg.getBodyAs(String.class);
String header = msg.getHeaderAs("test", String.class); String header = msg.getHeaderAs("test", String.class);
tryReply(String.format("%s %s", body, header)); channel.tryTell(String.format("%s %s", body, header));
} }
} }

View file

@ -36,7 +36,7 @@ object CamelTestSupport {
def countdown: Handler = { def countdown: Handler = {
case SetExpectedMessageCount(num) { case SetExpectedMessageCount(num) {
latch = new CountDownLatch(num) latch = new CountDownLatch(num)
reply(latch) channel ! latch
} }
case msg latch.countDown case msg latch.countDown
} }
@ -44,7 +44,7 @@ object CamelTestSupport {
trait Respond { this: Actor trait Respond { this: Actor
def respond: Handler = { def respond: Handler = {
case msg: Message reply(response(msg)) case msg: Message channel ! response(msg)
} }
def response(msg: Message): Any = "Hello %s" format msg.body def response(msg: Message): Any = "Hello %s" format msg.body
@ -54,8 +54,8 @@ object CamelTestSupport {
val messages = Buffer[Any]() val messages = Buffer[Any]()
def retain: Handler = { def retain: Handler = {
case GetRetainedMessage reply(messages.last) case GetRetainedMessage channel ! messages.last
case GetRetainedMessages(p) reply(messages.toList.filter(p)) case GetRetainedMessages(p) channel ! messages.filter(p).toList
case msg { case msg {
messages += msg messages += msg
msg msg

View file

@ -211,7 +211,7 @@ object ConsumerScalaTest {
class TestConsumer(uri: String) extends Actor with Consumer { class TestConsumer(uri: String) extends Actor with Consumer {
def endpointUri = uri def endpointUri = uri
protected def receive = { protected def receive = {
case msg: Message reply("received %s" format msg.body) case msg: Message channel ! "received %s" format msg.body
} }
} }
@ -226,7 +226,7 @@ object ConsumerScalaTest {
def endpointUri = uri def endpointUri = uri
override def autoack = false override def autoack = false
protected def receive = { protected def receive = {
case msg: Message reply(Ack) case msg: Message channel ! Ack
} }
} }
@ -247,15 +247,15 @@ object ConsumerScalaTest {
protected def receive = { protected def receive = {
case "fail" { throw new Exception("test") } case "fail" { throw new Exception("test") }
case "succeed" reply("ok") case "succeed" channel ! "ok"
} }
override def preRestart(reason: scala.Throwable, msg: Option[Any]) { override def preRestart(reason: scala.Throwable, msg: Option[Any]) {
tryReply("pr") channel.tryTell("pr")
} }
override def postStop { override def postStop {
tryReply("ps") channel.tryTell("ps")
} }
} }
@ -288,7 +288,7 @@ object ConsumerScalaTest {
} }
private def respondTo(msg: Message) = private def respondTo(msg: Message) =
if (valid) reply("accepted: %s" format msg.body) if (valid) channel ! ("accepted: %s" format msg.body)
else throw new Exception("rejected: %s" format msg.body) else throw new Exception("rejected: %s" format msg.body)
} }

View file

@ -253,18 +253,16 @@ object ProducerFeatureTest {
class TestResponder extends Actor { class TestResponder extends Actor {
protected def receive = { protected def receive = {
case msg: Message msg.body match { case msg: Message msg.body match {
case "fail" reply(Failure(new Exception("failure"), msg.headers)) case "fail" channel ! Failure(new Exception("failure"), msg.headers)
case _ reply(msg.transformBody { body: String "received %s" format body }) case _ channel ! (msg.transformBody { body: String "received %s" format body })
} }
} }
} }
class ReplyingForwardTarget extends Actor { class ReplyingForwardTarget extends Actor {
protected def receive = { protected def receive = {
case msg: Message case msg: Message channel ! msg.addHeader("test" -> "result")
reply(msg.addHeader("test" -> "result")) case msg: Failure channel ! Failure(msg.cause, msg.headers + ("test" -> "failure"))
case msg: Failure
reply(Failure(msg.cause, msg.headers + ("test" -> "failure")))
} }
} }

View file

@ -96,13 +96,13 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
object ActorComponentFeatureTest { object ActorComponentFeatureTest {
class CustomIdActor extends Actor { class CustomIdActor extends Actor {
protected def receive = { protected def receive = {
case msg: Message reply("Received %s" format msg.body) case msg: Message channel ! ("Received %s" format msg.body)
} }
} }
class FailWithMessage extends Actor { class FailWithMessage extends Actor {
protected def receive = { protected def receive = {
case msg: Message reply(Failure(new Exception("test"))) case msg: Message channel ! Failure(new Exception("test"))
} }
} }

View file

@ -47,7 +47,7 @@ import akka.cluster.metrics._
import akka.cluster.zookeeper._ import akka.cluster.zookeeper._
import ChangeListener._ import ChangeListener._
import RemoteProtocol._ import RemoteProtocol._
import RemoteDaemonMessageType._ import RemoteSystemDaemonMessageType._
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
@ -818,7 +818,7 @@ class DefaultClusterNode private[akka] (
EventHandler.debug(this, EventHandler.debug(this,
"Sending command to nodes [%s] for checking out actor [%s]".format(nodes.mkString(", "), actorAddress)) "Sending command to nodes [%s] for checking out actor [%s]".format(nodes.mkString(", "), actorAddress))
val builder = RemoteDaemonMessageProtocol.newBuilder val builder = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(USE) .setMessageType(USE)
.setActorAddress(actorAddress) .setActorAddress(actorAddress)
@ -882,7 +882,7 @@ class DefaultClusterNode private[akka] (
EventHandler.debug(this, EventHandler.debug(this,
"Releasing (checking in) all actors with address [%s] on all nodes in cluster".format(actorAddress)) "Releasing (checking in) all actors with address [%s] on all nodes in cluster".format(actorAddress))
val command = RemoteDaemonMessageProtocol.newBuilder val command = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(RELEASE) .setMessageType(RELEASE)
.setActorAddress(actorAddress) .setActorAddress(actorAddress)
.build .build
@ -1030,7 +1030,7 @@ class DefaultClusterNode private[akka] (
Serialization.serialize(f) match { Serialization.serialize(f) match {
case Left(error) throw error case Left(error) throw error
case Right(bytes) case Right(bytes)
val message = RemoteDaemonMessageProtocol.newBuilder val message = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FUNCTION_FUN0_UNIT) .setMessageType(FUNCTION_FUN0_UNIT)
.setPayload(ByteString.copyFrom(bytes)) .setPayload(ByteString.copyFrom(bytes))
.build .build
@ -1046,7 +1046,7 @@ class DefaultClusterNode private[akka] (
Serialization.serialize(f) match { Serialization.serialize(f) match {
case Left(error) throw error case Left(error) throw error
case Right(bytes) case Right(bytes)
val message = RemoteDaemonMessageProtocol.newBuilder val message = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FUNCTION_FUN0_ANY) .setMessageType(FUNCTION_FUN0_ANY)
.setPayload(ByteString.copyFrom(bytes)) .setPayload(ByteString.copyFrom(bytes))
.build .build
@ -1063,7 +1063,7 @@ class DefaultClusterNode private[akka] (
Serialization.serialize((f, arg)) match { Serialization.serialize((f, arg)) match {
case Left(error) throw error case Left(error) throw error
case Right(bytes) case Right(bytes)
val message = RemoteDaemonMessageProtocol.newBuilder val message = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FUNCTION_FUN1_ARG_UNIT) .setMessageType(FUNCTION_FUN1_ARG_UNIT)
.setPayload(ByteString.copyFrom(bytes)) .setPayload(ByteString.copyFrom(bytes))
.build .build
@ -1080,7 +1080,7 @@ class DefaultClusterNode private[akka] (
Serialization.serialize((f, arg)) match { Serialization.serialize((f, arg)) match {
case Left(error) throw error case Left(error) throw error
case Right(bytes) case Right(bytes)
val message = RemoteDaemonMessageProtocol.newBuilder val message = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FUNCTION_FUN1_ARG_ANY) .setMessageType(FUNCTION_FUN1_ARG_ANY)
.setPayload(ByteString.copyFrom(bytes)) .setPayload(ByteString.copyFrom(bytes))
.build .build
@ -1151,7 +1151,7 @@ class DefaultClusterNode private[akka] (
// Private // Private
// ======================================= // =======================================
private def sendCommandToNode(connection: ActorRef, command: RemoteDaemonMessageProtocol, async: Boolean = true) { private def sendCommandToNode(connection: ActorRef, command: RemoteSystemDaemonMessageProtocol, async: Boolean = true) {
if (async) { if (async) {
connection ! command connection ! command
} else { } else {
@ -1442,7 +1442,7 @@ class DefaultClusterNode private[akka] (
case Left(error) throw error case Left(error) throw error
case Right(bytes) case Right(bytes)
val command = RemoteDaemonMessageProtocol.newBuilder val command = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FAIL_OVER_CONNECTIONS) .setMessageType(FAIL_OVER_CONNECTIONS)
.setPayload(ByteString.copyFrom(bytes)) .setPayload(ByteString.copyFrom(bytes))
.build .build
@ -1713,7 +1713,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
} }
def receive: Receive = { def receive: Receive = {
case message: RemoteDaemonMessageProtocol case message: RemoteSystemDaemonMessageProtocol
EventHandler.debug(this, EventHandler.debug(this,
"Received command [\n%s] to RemoteClusterDaemon on node [%s]".format(message, cluster.nodeAddress.nodeName)) "Received command [\n%s] to RemoteClusterDaemon on node [%s]".format(message, cluster.nodeAddress.nodeName))
@ -1735,7 +1735,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
case unknown EventHandler.warning(this, "Unknown message [%s]".format(unknown)) case unknown EventHandler.warning(this, "Unknown message [%s]".format(unknown))
} }
def handleRelease(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handleRelease(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
if (message.hasActorUuid) { if (message.hasActorUuid) {
cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address
cluster.release(address) cluster.release(address)
@ -1748,7 +1748,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
} }
} }
def handleUse(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handleUse(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
def deserializeMessages(entriesAsBytes: Vector[Array[Byte]]): Vector[AnyRef] = { def deserializeMessages(entriesAsBytes: Vector[Array[Byte]]): Vector[AnyRef] = {
import akka.cluster.RemoteProtocol._ import akka.cluster.RemoteProtocol._
import akka.cluster.MessageSerializer import akka.cluster.MessageSerializer
@ -1855,7 +1855,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
} }
} }
def handle_fun0_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handle_fun0_unit(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
new LocalActorRef( new LocalActorRef(
Props( Props(
self { self {
@ -1863,7 +1863,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
}).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Function0[Unit]]) }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
} }
def handle_fun0_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handle_fun0_any(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
new LocalActorRef( new LocalActorRef(
Props( Props(
self { self {
@ -1871,7 +1871,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
}).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Function0[Any]]) }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
} }
def handle_fun1_arg_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handle_fun1_arg_unit(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
new LocalActorRef( new LocalActorRef(
Props( Props(
self { self {
@ -1879,7 +1879,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
}).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
} }
def handle_fun1_arg_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handle_fun1_arg_any(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
new LocalActorRef( new LocalActorRef(
Props( Props(
self { self {
@ -1887,12 +1887,12 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
}).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
} }
def handleFailover(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handleFailover(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
val (from, to) = payloadFor(message, classOf[(InetSocketAddress, InetSocketAddress)]) val (from, to) = payloadFor(message, classOf[(InetSocketAddress, InetSocketAddress)])
cluster.failOverClusterActorRefConnections(from, to) cluster.failOverClusterActorRefConnections(from, to)
} }
private def payloadFor[T](message: RemoteDaemonMessageProtocol, clazz: Class[T]): T = { private def payloadFor[T](message: RemoteSystemDaemonMessageProtocol, clazz: Class[T]): T = {
Serialization.deserialize(message.getPayload.toByteArray, clazz, None) match { Serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
case Left(error) throw error case Left(error) throw error
case Right(instance) instance.asInstanceOf[T] case Right(instance) instance.asInstanceOf[T]

View file

@ -50,8 +50,7 @@ object Pi extends App {
//#calculatePiFor //#calculatePiFor
def receive = { def receive = {
case Work(start, nrOfElements) case Work(start, nrOfElements) channel ! Result(calculatePiFor(start, nrOfElements)) // perform the work
reply(Result(calculatePiFor(start, nrOfElements))) // perform the work
} }
} }
//#worker //#worker

View file

@ -179,7 +179,7 @@ This is fine when dealing with a known amount of Actors, but can grow unwieldy i
To better explain what happened in the example, ``Future.sequence`` is taking the ``List[Future[Int]]`` and turning it into a ``Future[List[Int]]``. We can then use ``map`` to work with the ``List[Int]`` directly, and we find the sum of the ``List``. To better explain what happened in the example, ``Future.sequence`` is taking the ``List[Future[Int]]`` and turning it into a ``Future[List[Int]]``. We can then use ``map`` to work with the ``List[Int]`` directly, and we find the sum of the ``List``.
The ``traverse`` method is similar to ``sequence``, but it takes a ``T[A]`` and a function ``T => Future[B]`` to return a ``Future[T[B]]``, where ``T`` is again a subclass of Traversable. For example, to use ``traverse`` to sum the first 100 odd numbers: The ``traverse`` method is similar to ``sequence``, but it takes a ``T[A]`` and a function ``A => Future[B]`` to return a ``Future[T[B]]``, where ``T`` is again a subclass of Traversable. For example, to use ``traverse`` to sum the first 100 odd numbers:
.. code-block:: scala .. code-block:: scala

View file

@ -365,7 +365,7 @@ public final class RemoteProtocol {
// @@protoc_insertion_point(enum_scope:LifeCycleType) // @@protoc_insertion_point(enum_scope:LifeCycleType)
} }
public enum RemoteDaemonMessageType public enum RemoteSystemDaemonMessageType
implements com.google.protobuf.ProtocolMessageEnum { implements com.google.protobuf.ProtocolMessageEnum {
STOP(0, 1), STOP(0, 1),
USE(1, 2), USE(1, 2),
@ -375,11 +375,13 @@ public final class RemoteProtocol {
DISCONNECT(5, 6), DISCONNECT(5, 6),
RECONNECT(6, 7), RECONNECT(6, 7),
RESIGN(7, 8), RESIGN(7, 8),
FAIL_OVER_CONNECTIONS(8, 9), GOSSIP(8, 9),
FUNCTION_FUN0_UNIT(9, 10), GOSSIP_ACK(9, 10),
FUNCTION_FUN0_ANY(10, 11), FAIL_OVER_CONNECTIONS(10, 20),
FUNCTION_FUN1_ARG_UNIT(11, 12), FUNCTION_FUN0_UNIT(11, 21),
FUNCTION_FUN1_ARG_ANY(12, 13), FUNCTION_FUN0_ANY(12, 22),
FUNCTION_FUN1_ARG_UNIT(13, 23),
FUNCTION_FUN1_ARG_ANY(14, 24),
; ;
public static final int STOP_VALUE = 1; public static final int STOP_VALUE = 1;
@ -390,16 +392,18 @@ public final class RemoteProtocol {
public static final int DISCONNECT_VALUE = 6; public static final int DISCONNECT_VALUE = 6;
public static final int RECONNECT_VALUE = 7; public static final int RECONNECT_VALUE = 7;
public static final int RESIGN_VALUE = 8; public static final int RESIGN_VALUE = 8;
public static final int FAIL_OVER_CONNECTIONS_VALUE = 9; public static final int GOSSIP_VALUE = 9;
public static final int FUNCTION_FUN0_UNIT_VALUE = 10; public static final int GOSSIP_ACK_VALUE = 10;
public static final int FUNCTION_FUN0_ANY_VALUE = 11; public static final int FAIL_OVER_CONNECTIONS_VALUE = 20;
public static final int FUNCTION_FUN1_ARG_UNIT_VALUE = 12; public static final int FUNCTION_FUN0_UNIT_VALUE = 21;
public static final int FUNCTION_FUN1_ARG_ANY_VALUE = 13; public static final int FUNCTION_FUN0_ANY_VALUE = 22;
public static final int FUNCTION_FUN1_ARG_UNIT_VALUE = 23;
public static final int FUNCTION_FUN1_ARG_ANY_VALUE = 24;
public final int getNumber() { return value; } public final int getNumber() { return value; }
public static RemoteDaemonMessageType valueOf(int value) { public static RemoteSystemDaemonMessageType valueOf(int value) {
switch (value) { switch (value) {
case 1: return STOP; case 1: return STOP;
case 2: return USE; case 2: return USE;
@ -409,24 +413,26 @@ public final class RemoteProtocol {
case 6: return DISCONNECT; case 6: return DISCONNECT;
case 7: return RECONNECT; case 7: return RECONNECT;
case 8: return RESIGN; case 8: return RESIGN;
case 9: return FAIL_OVER_CONNECTIONS; case 9: return GOSSIP;
case 10: return FUNCTION_FUN0_UNIT; case 10: return GOSSIP_ACK;
case 11: return FUNCTION_FUN0_ANY; case 20: return FAIL_OVER_CONNECTIONS;
case 12: return FUNCTION_FUN1_ARG_UNIT; case 21: return FUNCTION_FUN0_UNIT;
case 13: return FUNCTION_FUN1_ARG_ANY; case 22: return FUNCTION_FUN0_ANY;
case 23: return FUNCTION_FUN1_ARG_UNIT;
case 24: return FUNCTION_FUN1_ARG_ANY;
default: return null; default: return null;
} }
} }
public static com.google.protobuf.Internal.EnumLiteMap<RemoteDaemonMessageType> public static com.google.protobuf.Internal.EnumLiteMap<RemoteSystemDaemonMessageType>
internalGetValueMap() { internalGetValueMap() {
return internalValueMap; return internalValueMap;
} }
private static com.google.protobuf.Internal.EnumLiteMap<RemoteDaemonMessageType> private static com.google.protobuf.Internal.EnumLiteMap<RemoteSystemDaemonMessageType>
internalValueMap = internalValueMap =
new com.google.protobuf.Internal.EnumLiteMap<RemoteDaemonMessageType>() { new com.google.protobuf.Internal.EnumLiteMap<RemoteSystemDaemonMessageType>() {
public RemoteDaemonMessageType findValueByNumber(int number) { public RemoteSystemDaemonMessageType findValueByNumber(int number) {
return RemoteDaemonMessageType.valueOf(number); return RemoteSystemDaemonMessageType.valueOf(number);
} }
}; };
@ -443,11 +449,11 @@ public final class RemoteProtocol {
return akka.remote.RemoteProtocol.getDescriptor().getEnumTypes().get(5); return akka.remote.RemoteProtocol.getDescriptor().getEnumTypes().get(5);
} }
private static final RemoteDaemonMessageType[] VALUES = { private static final RemoteSystemDaemonMessageType[] VALUES = {
STOP, USE, RELEASE, MAKE_AVAILABLE, MAKE_UNAVAILABLE, DISCONNECT, RECONNECT, RESIGN, FAIL_OVER_CONNECTIONS, FUNCTION_FUN0_UNIT, FUNCTION_FUN0_ANY, FUNCTION_FUN1_ARG_UNIT, FUNCTION_FUN1_ARG_ANY, STOP, USE, RELEASE, MAKE_AVAILABLE, MAKE_UNAVAILABLE, DISCONNECT, RECONNECT, RESIGN, GOSSIP, GOSSIP_ACK, FAIL_OVER_CONNECTIONS, FUNCTION_FUN0_UNIT, FUNCTION_FUN0_ANY, FUNCTION_FUN1_ARG_UNIT, FUNCTION_FUN1_ARG_ANY,
}; };
public static RemoteDaemonMessageType valueOf( public static RemoteSystemDaemonMessageType valueOf(
com.google.protobuf.Descriptors.EnumValueDescriptor desc) { com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
if (desc.getType() != getDescriptor()) { if (desc.getType() != getDescriptor()) {
throw new java.lang.IllegalArgumentException( throw new java.lang.IllegalArgumentException(
@ -459,12 +465,12 @@ public final class RemoteProtocol {
private final int index; private final int index;
private final int value; private final int value;
private RemoteDaemonMessageType(int index, int value) { private RemoteSystemDaemonMessageType(int index, int value) {
this.index = index; this.index = index;
this.value = value; this.value = value;
} }
// @@protoc_insertion_point(enum_scope:RemoteDaemonMessageType) // @@protoc_insertion_point(enum_scope:RemoteSystemDaemonMessageType)
} }
public interface AkkaRemoteProtocolOrBuilder public interface AkkaRemoteProtocolOrBuilder
@ -696,7 +702,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -1490,7 +1496,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -2862,7 +2868,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -3337,7 +3343,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -4164,7 +4170,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -5530,7 +5536,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -6038,7 +6044,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -6503,7 +6509,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -7050,7 +7056,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -7483,7 +7489,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -7886,7 +7892,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -8289,7 +8295,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -8759,7 +8765,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -8983,12 +8989,12 @@ public final class RemoteProtocol {
// @@protoc_insertion_point(class_scope:ExceptionProtocol) // @@protoc_insertion_point(class_scope:ExceptionProtocol)
} }
public interface RemoteDaemonMessageProtocolOrBuilder public interface RemoteSystemDaemonMessageProtocolOrBuilder
extends com.google.protobuf.MessageOrBuilder { extends com.google.protobuf.MessageOrBuilder {
// required .RemoteDaemonMessageType messageType = 1; // required .RemoteSystemDaemonMessageType messageType = 1;
boolean hasMessageType(); boolean hasMessageType();
akka.remote.RemoteProtocol.RemoteDaemonMessageType getMessageType(); akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType getMessageType();
// optional .UuidProtocol actorUuid = 2; // optional .UuidProtocol actorUuid = 2;
boolean hasActorUuid(); boolean hasActorUuid();
@ -9008,42 +9014,42 @@ public final class RemoteProtocol {
akka.remote.RemoteProtocol.UuidProtocol getReplicateActorFromUuid(); akka.remote.RemoteProtocol.UuidProtocol getReplicateActorFromUuid();
akka.remote.RemoteProtocol.UuidProtocolOrBuilder getReplicateActorFromUuidOrBuilder(); akka.remote.RemoteProtocol.UuidProtocolOrBuilder getReplicateActorFromUuidOrBuilder();
} }
public static final class RemoteDaemonMessageProtocol extends public static final class RemoteSystemDaemonMessageProtocol extends
com.google.protobuf.GeneratedMessage com.google.protobuf.GeneratedMessage
implements RemoteDaemonMessageProtocolOrBuilder { implements RemoteSystemDaemonMessageProtocolOrBuilder {
// Use RemoteDaemonMessageProtocol.newBuilder() to construct. // Use RemoteSystemDaemonMessageProtocol.newBuilder() to construct.
private RemoteDaemonMessageProtocol(Builder builder) { private RemoteSystemDaemonMessageProtocol(Builder builder) {
super(builder); super(builder);
} }
private RemoteDaemonMessageProtocol(boolean noInit) {} private RemoteSystemDaemonMessageProtocol(boolean noInit) {}
private static final RemoteDaemonMessageProtocol defaultInstance; private static final RemoteSystemDaemonMessageProtocol defaultInstance;
public static RemoteDaemonMessageProtocol getDefaultInstance() { public static RemoteSystemDaemonMessageProtocol getDefaultInstance() {
return defaultInstance; return defaultInstance;
} }
public RemoteDaemonMessageProtocol getDefaultInstanceForType() { public RemoteSystemDaemonMessageProtocol getDefaultInstanceForType() {
return defaultInstance; return defaultInstance;
} }
public static final com.google.protobuf.Descriptors.Descriptor public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() { getDescriptor() {
return akka.remote.RemoteProtocol.internal_static_RemoteDaemonMessageProtocol_descriptor; return akka.remote.RemoteProtocol.internal_static_RemoteSystemDaemonMessageProtocol_descriptor;
} }
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() { internalGetFieldAccessorTable() {
return akka.remote.RemoteProtocol.internal_static_RemoteDaemonMessageProtocol_fieldAccessorTable; return akka.remote.RemoteProtocol.internal_static_RemoteSystemDaemonMessageProtocol_fieldAccessorTable;
} }
private int bitField0_; private int bitField0_;
// required .RemoteDaemonMessageType messageType = 1; // required .RemoteSystemDaemonMessageType messageType = 1;
public static final int MESSAGETYPE_FIELD_NUMBER = 1; public static final int MESSAGETYPE_FIELD_NUMBER = 1;
private akka.remote.RemoteProtocol.RemoteDaemonMessageType messageType_; private akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType messageType_;
public boolean hasMessageType() { public boolean hasMessageType() {
return ((bitField0_ & 0x00000001) == 0x00000001); return ((bitField0_ & 0x00000001) == 0x00000001);
} }
public akka.remote.RemoteProtocol.RemoteDaemonMessageType getMessageType() { public akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType getMessageType() {
return messageType_; return messageType_;
} }
@ -9116,7 +9122,7 @@ public final class RemoteProtocol {
} }
private void initFields() { private void initFields() {
messageType_ = akka.remote.RemoteProtocol.RemoteDaemonMessageType.STOP; messageType_ = akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType.STOP;
actorUuid_ = akka.remote.RemoteProtocol.UuidProtocol.getDefaultInstance(); actorUuid_ = akka.remote.RemoteProtocol.UuidProtocol.getDefaultInstance();
actorAddress_ = ""; actorAddress_ = "";
payload_ = com.google.protobuf.ByteString.EMPTY; payload_ = com.google.protobuf.ByteString.EMPTY;
@ -9206,41 +9212,41 @@ public final class RemoteProtocol {
return super.writeReplace(); return super.writeReplace();
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseFrom( public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseFrom(
com.google.protobuf.ByteString data) com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed(); return newBuilder().mergeFrom(data).buildParsed();
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseFrom( public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseFrom(
com.google.protobuf.ByteString data, com.google.protobuf.ByteString data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry) return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed(); .buildParsed();
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseFrom(byte[] data) public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseFrom(byte[] data)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data).buildParsed(); return newBuilder().mergeFrom(data).buildParsed();
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseFrom( public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseFrom(
byte[] data, byte[] data,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
return newBuilder().mergeFrom(data, extensionRegistry) return newBuilder().mergeFrom(data, extensionRegistry)
.buildParsed(); .buildParsed();
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseFrom(java.io.InputStream input) public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseFrom(java.io.InputStream input)
throws java.io.IOException { throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed(); return newBuilder().mergeFrom(input).buildParsed();
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseFrom( public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseFrom(
java.io.InputStream input, java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
return newBuilder().mergeFrom(input, extensionRegistry) return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed(); .buildParsed();
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseDelimitedFrom(java.io.InputStream input) public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseDelimitedFrom(java.io.InputStream input)
throws java.io.IOException { throws java.io.IOException {
Builder builder = newBuilder(); Builder builder = newBuilder();
if (builder.mergeDelimitedFrom(input)) { if (builder.mergeDelimitedFrom(input)) {
@ -9249,7 +9255,7 @@ public final class RemoteProtocol {
return null; return null;
} }
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseDelimitedFrom( public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseDelimitedFrom(
java.io.InputStream input, java.io.InputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
@ -9260,12 +9266,12 @@ public final class RemoteProtocol {
return null; return null;
} }
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseFrom( public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseFrom(
com.google.protobuf.CodedInputStream input) com.google.protobuf.CodedInputStream input)
throws java.io.IOException { throws java.io.IOException {
return newBuilder().mergeFrom(input).buildParsed(); return newBuilder().mergeFrom(input).buildParsed();
} }
public static akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol parseFrom( public static akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol parseFrom(
com.google.protobuf.CodedInputStream input, com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry) com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException { throws java.io.IOException {
@ -9275,7 +9281,7 @@ public final class RemoteProtocol {
public static Builder newBuilder() { return Builder.create(); } public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); } public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol prototype) { public static Builder newBuilder(akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol prototype) {
return newBuilder().mergeFrom(prototype); return newBuilder().mergeFrom(prototype);
} }
public Builder toBuilder() { return newBuilder(this); } public Builder toBuilder() { return newBuilder(this); }
@ -9288,23 +9294,23 @@ public final class RemoteProtocol {
} }
public static final class Builder extends public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder<Builder> com.google.protobuf.GeneratedMessage.Builder<Builder>
implements akka.remote.RemoteProtocol.RemoteDaemonMessageProtocolOrBuilder { implements akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocolOrBuilder {
public static final com.google.protobuf.Descriptors.Descriptor public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() { getDescriptor() {
return akka.remote.RemoteProtocol.internal_static_RemoteDaemonMessageProtocol_descriptor; return akka.remote.RemoteProtocol.internal_static_RemoteSystemDaemonMessageProtocol_descriptor;
} }
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() { internalGetFieldAccessorTable() {
return akka.remote.RemoteProtocol.internal_static_RemoteDaemonMessageProtocol_fieldAccessorTable; return akka.remote.RemoteProtocol.internal_static_RemoteSystemDaemonMessageProtocol_fieldAccessorTable;
} }
// Construct using akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol.newBuilder() // Construct using akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol.newBuilder()
private Builder() { private Builder() {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -9320,7 +9326,7 @@ public final class RemoteProtocol {
public Builder clear() { public Builder clear() {
super.clear(); super.clear();
messageType_ = akka.remote.RemoteProtocol.RemoteDaemonMessageType.STOP; messageType_ = akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType.STOP;
bitField0_ = (bitField0_ & ~0x00000001); bitField0_ = (bitField0_ & ~0x00000001);
if (actorUuidBuilder_ == null) { if (actorUuidBuilder_ == null) {
actorUuid_ = akka.remote.RemoteProtocol.UuidProtocol.getDefaultInstance(); actorUuid_ = akka.remote.RemoteProtocol.UuidProtocol.getDefaultInstance();
@ -9347,24 +9353,24 @@ public final class RemoteProtocol {
public com.google.protobuf.Descriptors.Descriptor public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() { getDescriptorForType() {
return akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol.getDescriptor(); return akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol.getDescriptor();
} }
public akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol getDefaultInstanceForType() { public akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol getDefaultInstanceForType() {
return akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol.getDefaultInstance(); return akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol.getDefaultInstance();
} }
public akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol build() { public akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol build() {
akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol result = buildPartial(); akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol result = buildPartial();
if (!result.isInitialized()) { if (!result.isInitialized()) {
throw newUninitializedMessageException(result); throw newUninitializedMessageException(result);
} }
return result; return result;
} }
private akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol buildParsed() private akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException { throws com.google.protobuf.InvalidProtocolBufferException {
akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol result = buildPartial(); akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol result = buildPartial();
if (!result.isInitialized()) { if (!result.isInitialized()) {
throw newUninitializedMessageException( throw newUninitializedMessageException(
result).asInvalidProtocolBufferException(); result).asInvalidProtocolBufferException();
@ -9372,8 +9378,8 @@ public final class RemoteProtocol {
return result; return result;
} }
public akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol buildPartial() { public akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol buildPartial() {
akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol result = new akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol(this); akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol result = new akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol(this);
int from_bitField0_ = bitField0_; int from_bitField0_ = bitField0_;
int to_bitField0_ = 0; int to_bitField0_ = 0;
if (((from_bitField0_ & 0x00000001) == 0x00000001)) { if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
@ -9410,16 +9416,16 @@ public final class RemoteProtocol {
} }
public Builder mergeFrom(com.google.protobuf.Message other) { public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol) { if (other instanceof akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
return mergeFrom((akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol)other); return mergeFrom((akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol)other);
} else { } else {
super.mergeFrom(other); super.mergeFrom(other);
return this; return this;
} }
} }
public Builder mergeFrom(akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol other) { public Builder mergeFrom(akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol other) {
if (other == akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol.getDefaultInstance()) return this; if (other == akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol.getDefaultInstance()) return this;
if (other.hasMessageType()) { if (other.hasMessageType()) {
setMessageType(other.getMessageType()); setMessageType(other.getMessageType());
} }
@ -9484,7 +9490,7 @@ public final class RemoteProtocol {
} }
case 8: { case 8: {
int rawValue = input.readEnum(); int rawValue = input.readEnum();
akka.remote.RemoteProtocol.RemoteDaemonMessageType value = akka.remote.RemoteProtocol.RemoteDaemonMessageType.valueOf(rawValue); akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType value = akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType.valueOf(rawValue);
if (value == null) { if (value == null) {
unknownFields.mergeVarintField(1, rawValue); unknownFields.mergeVarintField(1, rawValue);
} else { } else {
@ -9527,15 +9533,15 @@ public final class RemoteProtocol {
private int bitField0_; private int bitField0_;
// required .RemoteDaemonMessageType messageType = 1; // required .RemoteSystemDaemonMessageType messageType = 1;
private akka.remote.RemoteProtocol.RemoteDaemonMessageType messageType_ = akka.remote.RemoteProtocol.RemoteDaemonMessageType.STOP; private akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType messageType_ = akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType.STOP;
public boolean hasMessageType() { public boolean hasMessageType() {
return ((bitField0_ & 0x00000001) == 0x00000001); return ((bitField0_ & 0x00000001) == 0x00000001);
} }
public akka.remote.RemoteProtocol.RemoteDaemonMessageType getMessageType() { public akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType getMessageType() {
return messageType_; return messageType_;
} }
public Builder setMessageType(akka.remote.RemoteProtocol.RemoteDaemonMessageType value) { public Builder setMessageType(akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType value) {
if (value == null) { if (value == null) {
throw new NullPointerException(); throw new NullPointerException();
} }
@ -9546,7 +9552,7 @@ public final class RemoteProtocol {
} }
public Builder clearMessageType() { public Builder clearMessageType() {
bitField0_ = (bitField0_ & ~0x00000001); bitField0_ = (bitField0_ & ~0x00000001);
messageType_ = akka.remote.RemoteProtocol.RemoteDaemonMessageType.STOP; messageType_ = akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType.STOP;
onChanged(); onChanged();
return this; return this;
} }
@ -9791,15 +9797,15 @@ public final class RemoteProtocol {
return replicateActorFromUuidBuilder_; return replicateActorFromUuidBuilder_;
} }
// @@protoc_insertion_point(builder_scope:RemoteDaemonMessageProtocol) // @@protoc_insertion_point(builder_scope:RemoteSystemDaemonMessageProtocol)
} }
static { static {
defaultInstance = new RemoteDaemonMessageProtocol(true); defaultInstance = new RemoteSystemDaemonMessageProtocol(true);
defaultInstance.initFields(); defaultInstance.initFields();
} }
// @@protoc_insertion_point(class_scope:RemoteDaemonMessageProtocol) // @@protoc_insertion_point(class_scope:RemoteSystemDaemonMessageProtocol)
} }
public interface DurableMailboxMessageProtocolOrBuilder public interface DurableMailboxMessageProtocolOrBuilder
@ -10117,7 +10123,7 @@ public final class RemoteProtocol {
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { private Builder(BuilderParent parent) {
super(parent); super(parent);
maybeForceBuilderInitialization(); maybeForceBuilderInitialization();
} }
@ -10568,10 +10574,10 @@ public final class RemoteProtocol {
com.google.protobuf.GeneratedMessage.FieldAccessorTable com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_ExceptionProtocol_fieldAccessorTable; internal_static_ExceptionProtocol_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor private static com.google.protobuf.Descriptors.Descriptor
internal_static_RemoteDaemonMessageProtocol_descriptor; internal_static_RemoteSystemDaemonMessageProtocol_descriptor;
private static private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_RemoteDaemonMessageProtocol_fieldAccessorTable; internal_static_RemoteSystemDaemonMessageProtocol_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor private static com.google.protobuf.Descriptors.Descriptor
internal_static_DurableMailboxMessageProtocol_descriptor; internal_static_DurableMailboxMessageProtocol_descriptor;
private static private static
@ -10625,30 +10631,32 @@ public final class RemoteProtocol {
"feCycleType\"1\n\017AddressProtocol\022\020\n\010hostna" + "feCycleType\"1\n\017AddressProtocol\022\020\n\010hostna" +
"me\030\001 \002(\t\022\014\n\004port\030\002 \002(\r\"7\n\021ExceptionProto" + "me\030\001 \002(\t\022\014\n\004port\030\002 \002(\r\"7\n\021ExceptionProto" +
"col\022\021\n\tclassname\030\001 \002(\t\022\017\n\007message\030\002 \002(\t\"" + "col\022\021\n\tclassname\030\001 \002(\t\022\017\n\007message\030\002 \002(\t\"" +
"\304\001\n\033RemoteDaemonMessageProtocol\022-\n\013messa", "\320\001\n!RemoteSystemDaemonMessageProtocol\0223\n",
"geType\030\001 \002(\0162\030.RemoteDaemonMessageType\022 " + "\013messageType\030\001 \002(\0162\036.RemoteSystemDaemonM" +
"\n\tactorUuid\030\002 \001(\0132\r.UuidProtocol\022\024\n\014acto" + "essageType\022 \n\tactorUuid\030\002 \001(\0132\r.UuidProt" +
"rAddress\030\003 \001(\t\022\017\n\007payload\030\005 \001(\014\022-\n\026repli" + "ocol\022\024\n\014actorAddress\030\003 \001(\t\022\017\n\007payload\030\005 " +
"cateActorFromUuid\030\006 \001(\0132\r.UuidProtocol\"\212" + "\001(\014\022-\n\026replicateActorFromUuid\030\006 \001(\0132\r.Uu" +
"\001\n\035DurableMailboxMessageProtocol\022\031\n\021owne" + "idProtocol\"\212\001\n\035DurableMailboxMessageProt" +
"rActorAddress\030\001 \002(\t\022\032\n\022senderActorAddres" + "ocol\022\031\n\021ownerActorAddress\030\001 \002(\t\022\032\n\022sende" +
"s\030\002 \001(\t\022!\n\nfutureUuid\030\003 \001(\0132\r.UuidProtoc" + "rActorAddress\030\002 \001(\t\022!\n\nfutureUuid\030\003 \001(\0132" +
"ol\022\017\n\007message\030\004 \002(\014*(\n\013CommandType\022\013\n\007CO" + "\r.UuidProtocol\022\017\n\007message\030\004 \002(\014*(\n\013Comma" +
"NNECT\020\001\022\014\n\010SHUTDOWN\020\002*K\n\026ReplicationStor" + "ndType\022\013\n\007CONNECT\020\001\022\014\n\010SHUTDOWN\020\002*K\n\026Rep" +
"ageType\022\r\n\tTRANSIENT\020\001\022\023\n\017TRANSACTION_LO", "licationStorageType\022\r\n\tTRANSIENT\020\001\022\023\n\017TR",
"G\020\002\022\r\n\tDATA_GRID\020\003*>\n\027ReplicationStrateg" + "ANSACTION_LOG\020\002\022\r\n\tDATA_GRID\020\003*>\n\027Replic" +
"yType\022\021\n\rWRITE_THROUGH\020\001\022\020\n\014WRITE_BEHIND" + "ationStrategyType\022\021\n\rWRITE_THROUGH\020\001\022\020\n\014" +
"\020\002*]\n\027SerializationSchemeType\022\010\n\004JAVA\020\001\022" + "WRITE_BEHIND\020\002*]\n\027SerializationSchemeTyp" +
"\013\n\007SBINARY\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JSO" + "e\022\010\n\004JAVA\020\001\022\013\n\007SBINARY\020\002\022\016\n\nSCALA_JSON\020\003" +
"N\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tPE" + "\022\r\n\tJAVA_JSON\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCyc" +
"RMANENT\020\001\022\r\n\tTEMPORARY\020\002*\217\002\n\027RemoteDaemo" + "leType\022\r\n\tPERMANENT\020\001\022\r\n\tTEMPORARY\020\002*\261\002\n" +
"nMessageType\022\010\n\004STOP\020\001\022\007\n\003USE\020\002\022\013\n\007RELEA" + "\035RemoteSystemDaemonMessageType\022\010\n\004STOP\020\001" +
"SE\020\003\022\022\n\016MAKE_AVAILABLE\020\004\022\024\n\020MAKE_UNAVAIL" + "\022\007\n\003USE\020\002\022\013\n\007RELEASE\020\003\022\022\n\016MAKE_AVAILABLE" +
"ABLE\020\005\022\016\n\nDISCONNECT\020\006\022\r\n\tRECONNECT\020\007\022\n\n" + "\020\004\022\024\n\020MAKE_UNAVAILABLE\020\005\022\016\n\nDISCONNECT\020\006" +
"\006RESIGN\020\010\022\031\n\025FAIL_OVER_CONNECTIONS\020\t\022\026\n\022", "\022\r\n\tRECONNECT\020\007\022\n\n\006RESIGN\020\010\022\n\n\006GOSSIP\020\t\022",
"FUNCTION_FUN0_UNIT\020\n\022\025\n\021FUNCTION_FUN0_AN" + "\016\n\nGOSSIP_ACK\020\n\022\031\n\025FAIL_OVER_CONNECTIONS" +
"Y\020\013\022\032\n\026FUNCTION_FUN1_ARG_UNIT\020\014\022\031\n\025FUNCT" + "\020\024\022\026\n\022FUNCTION_FUN0_UNIT\020\025\022\025\n\021FUNCTION_F" +
"ION_FUN1_ARG_ANY\020\rB\017\n\013akka.remoteH\001" "UN0_ANY\020\026\022\032\n\026FUNCTION_FUN1_ARG_UNIT\020\027\022\031\n" +
"\025FUNCTION_FUN1_ARG_ANY\020\030B\017\n\013akka.remoteH" +
"\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -10759,14 +10767,14 @@ public final class RemoteProtocol {
new java.lang.String[] { "Classname", "Message", }, new java.lang.String[] { "Classname", "Message", },
akka.remote.RemoteProtocol.ExceptionProtocol.class, akka.remote.RemoteProtocol.ExceptionProtocol.class,
akka.remote.RemoteProtocol.ExceptionProtocol.Builder.class); akka.remote.RemoteProtocol.ExceptionProtocol.Builder.class);
internal_static_RemoteDaemonMessageProtocol_descriptor = internal_static_RemoteSystemDaemonMessageProtocol_descriptor =
getDescriptor().getMessageTypes().get(13); getDescriptor().getMessageTypes().get(13);
internal_static_RemoteDaemonMessageProtocol_fieldAccessorTable = new internal_static_RemoteSystemDaemonMessageProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RemoteDaemonMessageProtocol_descriptor, internal_static_RemoteSystemDaemonMessageProtocol_descriptor,
new java.lang.String[] { "MessageType", "ActorUuid", "ActorAddress", "Payload", "ReplicateActorFromUuid", }, new java.lang.String[] { "MessageType", "ActorUuid", "ActorAddress", "Payload", "ReplicateActorFromUuid", },
akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol.class, akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol.class,
akka.remote.RemoteProtocol.RemoteDaemonMessageProtocol.Builder.class); akka.remote.RemoteProtocol.RemoteSystemDaemonMessageProtocol.Builder.class);
internal_static_DurableMailboxMessageProtocol_descriptor = internal_static_DurableMailboxMessageProtocol_descriptor =
getDescriptor().getMessageTypes().get(14); getDescriptor().getMessageTypes().get(14);
internal_static_DurableMailboxMessageProtocol_fieldAccessorTable = new internal_static_DurableMailboxMessageProtocol_fieldAccessorTable = new

View file

@ -156,16 +156,6 @@ enum LifeCycleType {
TEMPORARY = 2; TEMPORARY = 2;
} }
/*
enum DispatcherType {
GLOBAL_EVENT_EXECUTOR_BASED = 1;
GLOBAL_REACTOR_SINGLE_THREAD_BASED = 2;
GLOBAL_REACTOR_THREAD_POOL_BASED = 3;
EVENT_EXECUTOR_BASED = 4;
THREAD_BASED = 5;
}
*/
/** /**
* Defines the life-cycle of a supervised Actor. * Defines the life-cycle of a supervised Actor.
*/ */
@ -190,10 +180,10 @@ message ExceptionProtocol {
} }
/** /**
* Defines the remote daemon message. * Defines the remote system daemon message.
*/ */
message RemoteDaemonMessageProtocol { message RemoteSystemDaemonMessageProtocol {
required RemoteDaemonMessageType messageType = 1; required RemoteSystemDaemonMessageType messageType = 1;
optional UuidProtocol actorUuid = 2; optional UuidProtocol actorUuid = 2;
optional string actorAddress = 3; optional string actorAddress = 3;
optional bytes payload = 5; optional bytes payload = 5;
@ -201,9 +191,9 @@ message RemoteDaemonMessageProtocol {
} }
/** /**
* Defines the remote daemon message type. * Defines the remote system daemon message type.
*/ */
enum RemoteDaemonMessageType { enum RemoteSystemDaemonMessageType {
STOP = 1; STOP = 1;
USE = 2; USE = 2;
RELEASE = 3; RELEASE = 3;
@ -212,11 +202,12 @@ enum RemoteDaemonMessageType {
DISCONNECT = 6; DISCONNECT = 6;
RECONNECT = 7; RECONNECT = 7;
RESIGN = 8; RESIGN = 8;
FAIL_OVER_CONNECTIONS = 9; GOSSIP = 9;
FUNCTION_FUN0_UNIT = 10; FAIL_OVER_CONNECTIONS = 20;
FUNCTION_FUN0_ANY = 11; FUNCTION_FUN0_UNIT = 21;
FUNCTION_FUN1_ARG_UNIT = 12; FUNCTION_FUN0_ANY = 22;
FUNCTION_FUN1_ARG_ANY = 13; FUNCTION_FUN1_ARG_UNIT = 23;
FUNCTION_FUN1_ARG_ANY = 24;
} }
/** /**

View file

@ -28,7 +28,7 @@ import scala.annotation.tailrec
* Default threshold is 8 (taken from Cassandra defaults), but can be configured in the Akka config. * Default threshold is 8 (taken from Cassandra defaults), but can be configured in the Akka config.
*/ */
class AccrualFailureDetector( class AccrualFailureDetector(
val threshold: Int = 8, // FIXME make these configurable val threshold: Int = 8,
val maxSampleSize: Int = 1000) extends FailureDetector { val maxSampleSize: Int = 1000) extends FailureDetector {
final val PhiFactor = 1.0 / math.log(10.0) final val PhiFactor = 1.0 / math.log(10.0)
@ -139,7 +139,7 @@ class AccrualFailureDetector(
def phi(connection: InetSocketAddress): Double = { def phi(connection: InetSocketAddress): Double = {
val oldState = state.get val oldState = state.get
val oldTimestamp = oldState.timestamps.get(connection) val oldTimestamp = oldState.timestamps.get(connection)
if (oldTimestamp.isEmpty) Double.MaxValue // treat unmanaged connections, e.g. with zero heartbeats, as dead connections if (oldTimestamp.isEmpty) 0.0D // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
else { else {
PhiFactor * (newTimestamp - oldTimestamp.get) / oldState.failureStats.get(connection).getOrElse(FailureStats()).mean PhiFactor * (newTimestamp - oldTimestamp.get) / oldState.failureStats.get(connection).getOrElse(FailureStats()).mean
} }

View file

@ -4,22 +4,24 @@
package akka.remote package akka.remote
import akka.{ AkkaException, AkkaApplication }
import akka.actor._ import akka.actor._
import akka.routing._
import akka.actor.Actor._ import akka.actor.Actor._
import akka.actor.Status._ import akka.actor.Status._
import akka.routing._
import akka.dispatch._ import akka.dispatch._
import akka.util.duration._ import akka.util.duration._
import akka.config.ConfigurationException import akka.config.ConfigurationException
import akka.AkkaException
import RemoteProtocol._
import RemoteDaemonMessageType._
import akka.serialization.{ Serialization, Serializer, ActorSerialization, Compression }
import Compression.LZF
import java.net.InetSocketAddress
import com.google.protobuf.ByteString
import akka.AkkaApplication
import akka.event.{ DeathWatch, EventHandler } import akka.event.{ DeathWatch, EventHandler }
import akka.serialization.{ Serialization, Serializer, ActorSerialization, Compression }
import akka.serialization.Compression.LZF
import akka.remote.RemoteProtocol._
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
import java.net.InetSocketAddress
import java.util.concurrent.ConcurrentHashMap
import com.google.protobuf.ByteString
/** /**
* Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it. * Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it.
@ -39,10 +41,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
private val actors = new ConcurrentHashMap[String, AnyRef] private val actors = new ConcurrentHashMap[String, AnyRef]
private val remoteDaemonConnectionManager = new RemoteConnectionManager( private val remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote)
app,
remote = remote,
failureDetector = new BannagePeriodFailureDetector(60 seconds)) // FIXME make timeout configurable
def defaultDispatcher = app.dispatcher def defaultDispatcher = app.dispatcher
def defaultTimeout = app.AkkaConfig.ActorTimeout def defaultTimeout = app.AkkaConfig.ActorTimeout
@ -58,12 +57,13 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
app.deployer.lookupDeploymentFor(address) match { app.deployer.lookupDeploymentFor(address) match {
case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, DeploymentConfig.RemoteScope(remoteAddresses))) case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, DeploymentConfig.RemoteScope(remoteAddresses)))
val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match { // FIXME move to AccrualFailureDetector as soon as we have the Gossiper up and running and remove the option to select impl in the akka.conf file since we only have one
case FailureDetectorType.NoOp new NoOpFailureDetector // val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match {
case FailureDetectorType.RemoveConnectionOnFirstFailure new RemoveConnectionOnFirstFailureFailureDetector // case FailureDetectorType.NoOp new NoOpFailureDetector
case FailureDetectorType.BannagePeriod(timeToBan) new BannagePeriodFailureDetector(timeToBan) // case FailureDetectorType.RemoveConnectionOnFirstFailure new RemoveConnectionOnFirstFailureFailureDetector
case FailureDetectorType.Custom(implClass) FailureDetector.createCustomFailureDetector(implClass) // case FailureDetectorType.BannagePeriod(timeToBan) new BannagePeriodFailureDetector(timeToBan)
} // case FailureDetectorType.Custom(implClass) FailureDetector.createCustomFailureDetector(implClass)
// }
val thisHostname = remote.address.getHostName val thisHostname = remote.address.getHostName
val thisPort = remote.address.getPort val thisPort = remote.address.getPort
@ -112,7 +112,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
conns + (inetAddr -> RemoteActorRef(remote.server, inetAddr, address, None)) conns + (inetAddr -> RemoteActorRef(remote.server, inetAddr, address, None))
} }
val connectionManager = new RemoteConnectionManager(app, remote, connections, failureDetector) val connectionManager = new RemoteConnectionManager(app, remote, connections)
connections.keys foreach { useActorOnNode(_, address, props.creator) } connections.keys foreach { useActorOnNode(_, address, props.creator) }
@ -177,7 +177,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
else bytes else bytes
} }
val command = RemoteDaemonMessageProtocol.newBuilder val command = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(USE) .setMessageType(USE)
.setActorAddress(actorAddress) .setActorAddress(actorAddress)
.setPayload(ByteString.copyFrom(actorFactoryBytes)) .setPayload(ByteString.copyFrom(actorFactoryBytes))
@ -193,29 +193,25 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
sendCommandToRemoteNode(connection, command, withACK = true) // ensure we get an ACK on the USE command sendCommandToRemoteNode(connection, command, withACK = true) // ensure we get an ACK on the USE command
} }
private def sendCommandToRemoteNode( private def sendCommandToRemoteNode(connection: ActorRef, command: RemoteSystemDaemonMessageProtocol, withACK: Boolean) {
connection: ActorRef,
command: RemoteDaemonMessageProtocol,
withACK: Boolean) {
if (withACK) { if (withACK) {
try { try {
(connection ? (command, remote.remoteDaemonAckTimeout)).as[Status] match { (connection ? (command, remote.remoteSystemDaemonAckTimeout)).as[Status] match {
case Some(Success(receiver)) case Some(Success(receiver))
app.eventHandler.debug(this, "Remote command sent to [%s] successfully received".format(receiver)) app.eventHandler.debug(this, "Remote system command sent to [%s] successfully received".format(receiver))
case Some(Failure(cause)) case Some(Failure(cause))
app.eventHandler.error(cause, this, cause.toString) app.eventHandler.error(cause, this, cause.toString)
throw cause throw cause
case None case None
val error = new RemoteException("Remote command to [%s] timed out".format(connection.address)) val error = new RemoteException("Remote system command to [%s] timed out".format(connection.address))
app.eventHandler.error(error, this, error.toString) app.eventHandler.error(error, this, error.toString)
throw error throw error
} }
} catch { } catch {
case e: Exception case e: Exception
app.eventHandler.error(e, this, "Could not send remote command to [%s] due to: %s".format(connection.address, e.toString)) app.eventHandler.error(e, this, "Could not send remote system command to [%s] due to: %s".format(connection.address, e.toString))
throw e throw e
} }
} else { } else {

View file

@ -22,15 +22,17 @@ import java.util.concurrent.atomic.AtomicReference
class RemoteConnectionManager( class RemoteConnectionManager(
app: AkkaApplication, app: AkkaApplication,
remote: Remote, remote: Remote,
initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef], initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef])
failureDetector: FailureDetector = new NoOpFailureDetector)
extends ConnectionManager { extends ConnectionManager {
// FIXME is this VersionedIterable really needed? It is not used I think. Complicates API. See 'def connections' etc.
case class State(version: Long, connections: Map[InetSocketAddress, ActorRef]) case class State(version: Long, connections: Map[InetSocketAddress, ActorRef])
extends VersionedIterable[ActorRef] { extends VersionedIterable[ActorRef] {
def iterable: Iterable[ActorRef] = connections.values def iterable: Iterable[ActorRef] = connections.values
} }
val failureDetector = remote.failureDetector
private val state: AtomicReference[State] = new AtomicReference[State](newState()) private val state: AtomicReference[State] = new AtomicReference[State](newState())
// register all initial connections - e.g listen to events from them // register all initial connections - e.g listen to events from them
@ -48,10 +50,13 @@ class RemoteConnectionManager(
def version: Long = state.get.version def version: Long = state.get.version
// FIXME should not return State value but a Seq with connections
def connections = filterAvailableConnections(state.get) def connections = filterAvailableConnections(state.get)
def size: Int = connections.connections.size def size: Int = connections.connections.size
def connectionFor(address: InetSocketAddress): Option[ActorRef] = connections.connections.get(address)
def isEmpty: Boolean = connections.connections.isEmpty def isEmpty: Boolean = connections.connections.isEmpty
def shutdown() { def shutdown() {

View file

@ -14,44 +14,54 @@ import akka.util.duration._
import akka.util.Helpers._ import akka.util.Helpers._
import akka.actor.DeploymentConfig._ import akka.actor.DeploymentConfig._
import akka.serialization.{ Serialization, Serializer, ActorSerialization, Compression } import akka.serialization.{ Serialization, Serializer, ActorSerialization, Compression }
import Compression.LZF import akka.serialization.Compression.LZF
import RemoteProtocol._ import akka.remote.RemoteProtocol._
import RemoteDaemonMessageType._ import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
import java.net.InetSocketAddress import java.net.InetSocketAddress
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
// FIXME renamed file from RemoteDaemon.scala to Remote.scala
/** /**
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class Remote(val app: AkkaApplication) extends RemoteService { class Remote(val app: AkkaApplication) extends RemoteService {
import app._
import app.config import app.config
import app.AkkaConfig.DefaultTimeUnit import app.AkkaConfig._
// TODO move to AkkaConfig?
val shouldCompressData = config.getBool("akka.remote.use-compression", false) val shouldCompressData = config.getBool("akka.remote.use-compression", false)
val remoteDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt val remoteSystemDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt
val hostname = app.hostname val hostname = app.hostname
val port = app.port val port = app.port
val remoteDaemonServiceName = "akka-remote-daemon".intern val failureDetector = new AccrualFailureDetector(FailureDetectorThreshold, FailureDetectorMaxSampleSize)
// val gossiper = new Gossiper(this)
val remoteDaemonServiceName = "akka-system-remote-daemon".intern
// FIXME configure computeGridDispatcher to what? // FIXME configure computeGridDispatcher to what?
val computeGridDispatcher = app.dispatcherFactory.newDispatcher("akka:compute-grid").build val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build
private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props( private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props(
OneForOneStrategy(List(classOf[Exception]), None, None))) // is infinite restart what we want? OneForOneStrategy(List(classOf[Exception]), None, None))) // is infinite restart what we want?
// FIXME check that this supervision is okay
private[remote] lazy val remoteDaemon = private[remote] lazy val remoteDaemon =
new LocalActorRef( new LocalActorRef(
app, app,
Props(new RemoteDaemon(this)).withDispatcher(app.dispatcherFactory.newPinnedDispatcher("Remote")), Props(new RemoteSystemDaemon(this))
app.guardian, .withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
remoteDaemonSupervisor,
remoteDaemonServiceName, remoteDaemonServiceName,
true) systemService = true)
private[remote] lazy val remoteClientLifeCycleHandler = app.actorOf(Props(new Actor { private[remote] lazy val remoteClientLifeCycleHandler = app.actorOf(Props(new Actor {
def receive = { def receive = {
@ -59,7 +69,7 @@ class Remote(val app: AkkaApplication) extends RemoteService {
case RemoteClientDisconnected(client, address) client.shutdownClientModule() case RemoteClientDisconnected(client, address) client.shutdownClientModule()
case _ //ignore other case _ //ignore other
} }
}), "akka.cluster.RemoteClientLifeCycleListener") }), "akka.remote.RemoteClientLifeCycleListener")
lazy val eventStream = new NetworkEventStream(app) lazy val eventStream = new NetworkEventStream(app)
@ -70,7 +80,7 @@ class Remote(val app: AkkaApplication) extends RemoteService {
remote.addListener(eventStream.channel) remote.addListener(eventStream.channel)
remote.addListener(remoteClientLifeCycleHandler) remote.addListener(remoteClientLifeCycleHandler)
// TODO actually register this provider in app in remote mode // TODO actually register this provider in app in remote mode
//app.provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider) //provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider)
remote remote
} }
@ -78,7 +88,7 @@ class Remote(val app: AkkaApplication) extends RemoteService {
def start() { def start() {
val triggerLazyServerVal = address.toString val triggerLazyServerVal = address.toString
app.eventHandler.info(this, "Starting remote server on [%s]".format(triggerLazyServerVal)) eventHandler.info(this, "Starting remote server on [%s]".format(triggerLazyServerVal))
} }
def uuidProtocolToUuid(uuid: UuidProtocol): UUID = new UUID(uuid.getHigh, uuid.getLow) def uuidProtocolToUuid(uuid: UuidProtocol): UUID = new UUID(uuid.getHigh, uuid.getLow)
@ -91,24 +101,25 @@ class Remote(val app: AkkaApplication) extends RemoteService {
} }
/** /**
* Internal "daemon" actor for cluster internal communication. * Internal system "daemon" actor for remote internal communication.
* *
* It acts as the brain of the cluster that responds to cluster events (messages) and undertakes action. * It acts as the brain of the remote that responds to system remote events (messages) and undertakes action.
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class RemoteDaemon(val remote: Remote) extends Actor { class RemoteSystemDaemon(remote: Remote) extends Actor {
import remote._ import remote._
import remote.app._
override def preRestart(reason: Throwable, msg: Option[Any]) { override def preRestart(reason: Throwable, msg: Option[Any]) {
app.eventHandler.debug(this, "RemoteDaemon failed due to [%s] restarting...".format(reason)) eventHandler.debug(this, "RemoteSystemDaemon failed due to [%s] - restarting...".format(reason))
} }
def receive: Actor.Receive = { def receive: Actor.Receive = {
case message: RemoteDaemonMessageProtocol case message: RemoteSystemDaemonMessageProtocol
app.eventHandler.debug(this, eventHandler.debug(this,
"Received command [\n%s] to RemoteDaemon on [%s]".format(message, app.nodename)) "Received command [\n%s] to RemoteSystemDaemon on [%s]".format(message, nodename))
message.getMessageType match { message.getMessageType match {
case USE handleUse(message) case USE handleUse(message)
@ -118,6 +129,7 @@ class RemoteDaemon(val remote: Remote) extends Actor {
// case RECONNECT cluster.reconnect() // case RECONNECT cluster.reconnect()
// case RESIGN cluster.resign() // case RESIGN cluster.resign()
// case FAIL_OVER_CONNECTIONS handleFailover(message) // case FAIL_OVER_CONNECTIONS handleFailover(message)
case GOSSIP handleGossip(message)
case FUNCTION_FUN0_UNIT handle_fun0_unit(message) case FUNCTION_FUN0_UNIT handle_fun0_unit(message)
case FUNCTION_FUN0_ANY handle_fun0_any(message) case FUNCTION_FUN0_ANY handle_fun0_any(message)
case FUNCTION_FUN1_ARG_UNIT handle_fun1_arg_unit(message) case FUNCTION_FUN1_ARG_UNIT handle_fun1_arg_unit(message)
@ -125,10 +137,10 @@ class RemoteDaemon(val remote: Remote) extends Actor {
//TODO: should we not deal with unrecognized message types? //TODO: should we not deal with unrecognized message types?
} }
case unknown app.eventHandler.warning(this, "Unknown message [%s]".format(unknown)) case unknown eventHandler.warning(this, "Unknown message to RemoteSystemDaemon [%s]".format(unknown))
} }
def handleUse(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handleUse(message: RemoteSystemDaemonMessageProtocol) {
try { try {
if (message.hasActorAddress) { if (message.hasActorAddress) {
@ -137,7 +149,7 @@ class RemoteDaemon(val remote: Remote) extends Actor {
else message.getPayload.toByteArray else message.getPayload.toByteArray
val actorFactory = val actorFactory =
app.serialization.deserialize(actorFactoryBytes, classOf[() Actor], None) match { serialization.deserialize(actorFactoryBytes, classOf[() Actor], None) match {
case Left(error) throw error case Left(error) throw error
case Right(instance) instance.asInstanceOf[() Actor] case Right(instance) instance.asInstanceOf[() Actor]
} }
@ -145,37 +157,43 @@ class RemoteDaemon(val remote: Remote) extends Actor {
val actorAddress = message.getActorAddress val actorAddress = message.getActorAddress
val newActorRef = app.actorOf(Props(creator = actorFactory), actorAddress) val newActorRef = app.actorOf(Props(creator = actorFactory), actorAddress)
remote.server.register(actorAddress, newActorRef) server.register(actorAddress, newActorRef)
} else { } else {
app.eventHandler.error(this, "Actor 'address' is not defined, ignoring remote daemon command [%s]".format(message)) eventHandler.error(this, "Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [%s]".format(message))
} }
reply(Success(address.toString)) channel ! Success(address.toString)
} catch { } catch {
case error: Throwable case error: Throwable
reply(Failure(error)) channel ! Failure(error)
throw error throw error
} }
} }
def handleRelease(message: RemoteProtocol.RemoteDaemonMessageProtocol) { // FIXME implement handleRelease
// FIXME implement handleRelease without Cluster def handleRelease(message: RemoteSystemDaemonMessageProtocol) {
}
// if (message.hasActorUuid) { def handleGossip(message: RemoteSystemDaemonMessageProtocol) {
// cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address // try {
// cluster.release(address) // val gossip = serialization.deserialize(message.getPayload.toByteArray, classOf[Gossip], None) match {
// case Left(error) throw error
// case Right(instance) instance.asInstanceOf[Gossip]
// } // }
// } else if (message.hasActorAddress) {
// cluster release message.getActorAddress // gossiper tell gossip
// } else {
// EventHandler.warning(this, // channel ! Success(address.toString)
// "None of 'uuid' or 'actorAddress'' is specified, ignoring remote cluster daemon command [%s]".format(message)) // } catch {
// case error: Throwable
// channel ! Failure(error)
// throw error
// } // }
} }
// FIXME: handle real remote supervision // FIXME: handle real remote supervision
def handle_fun0_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app, new LocalActorRef(app,
Props( Props(
context { context {
@ -184,16 +202,16 @@ class RemoteDaemon(val remote: Remote) extends Actor {
} }
// FIXME: handle real remote supervision // FIXME: handle real remote supervision
def handle_fun0_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handle_fun0_any(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app, new LocalActorRef(app,
Props( Props(
context { context {
case f: Function0[_] try { reply(f()) } finally { context.self.stop() } case f: Function0[_] try { channel ! f() } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Function0[Any]]) }).copy(dispatcher = computeGridDispatcher), app.guardian, Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
} }
// FIXME: handle real remote supervision // FIXME: handle real remote supervision
def handle_fun1_arg_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handle_fun1_arg_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app, new LocalActorRef(app,
Props( Props(
context { context {
@ -202,21 +220,21 @@ class RemoteDaemon(val remote: Remote) extends Actor {
} }
// FIXME: handle real remote supervision // FIXME: handle real remote supervision
def handle_fun1_arg_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handle_fun1_arg_any(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app, new LocalActorRef(app,
Props( Props(
context { context {
case (fun: Function[_, _], param: Any) try { reply(fun.asInstanceOf[Any Any](param)) } finally { context.self.stop() } case (fun: Function[_, _], param: Any) try { channel ! fun.asInstanceOf[Any Any](param) } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) }).copy(dispatcher = computeGridDispatcher), app.guardian, Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
} }
def handleFailover(message: RemoteProtocol.RemoteDaemonMessageProtocol) { def handleFailover(message: RemoteSystemDaemonMessageProtocol) {
// val (from, to) = payloadFor(message, classOf[(InetSocketremoteDaemonServiceName, InetSocketremoteDaemonServiceName)]) // val (from, to) = payloadFor(message, classOf[(InetSocketremoteDaemonServiceName, InetSocketremoteDaemonServiceName)])
// cluster.failOverClusterActorRefConnections(from, to) // cluster.failOverClusterActorRefConnections(from, to)
} }
private def payloadFor[T](message: RemoteDaemonMessageProtocol, clazz: Class[T]): T = { private def payloadFor[T](message: RemoteSystemDaemonMessageProtocol, clazz: Class[T]): T = {
app.serialization.deserialize(message.getPayload.toByteArray, clazz, None) match { serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
case Left(error) throw error case Left(error) throw error
case Right(instance) instance.asInstanceOf[T] case Right(instance) instance.asInstanceOf[T]
} }

View file

@ -9,8 +9,11 @@ import akka.AkkaException
class VectorClockException(message: String) extends AkkaException(message) class VectorClockException(message: String) extends AkkaException(message)
/** /**
* Representation of a Vector-based clock (counting clock). * Representation of a Vector-based clock (counting clock), inspired by Lamport logical clocks.
* For details see Wikipedia: [http://en.wikipedia.org/wiki/Vector_clock]. *
* Reference:
* Leslie Lamport (1978). "Time, clocks, and the ordering of events in a distributed system". Communications of the ACM 21 (7): 558-565.
* Friedemann Mattern (1988). "Virtual Time and Global States of Distributed Systems". Workshop on Parallel and Distributed Algorithms: pp. 215-226
*/ */
case class VectorClock( case class VectorClock(
versions: Vector[VectorClock.Entry] = Vector.empty[VectorClock.Entry], versions: Vector[VectorClock.Entry] = Vector.empty[VectorClock.Entry],
@ -19,17 +22,17 @@ case class VectorClock(
def compare(other: VectorClock): Ordering = VectorClock.compare(this, other) def compare(other: VectorClock): Ordering = VectorClock.compare(this, other)
def incrementVersionForNode(nodeId: Int, timestamp: Long): VectorClock = { def increment(fingerprint: Int, timestamp: Long): VectorClock = {
val newVersions = val newVersions =
if (versions exists (entry entry.nodeId == nodeId)) { if (versions exists (entry entry.fingerprint == fingerprint)) {
// update existing node entry // update existing node entry
versions map { entry versions map { entry
if (entry.nodeId == nodeId) entry.increment() if (entry.fingerprint == fingerprint) entry.increment()
else entry else entry
} }
} else { } else {
// create and append a new node entry // create and append a new node entry
versions :+ Entry(nodeId = nodeId) versions :+ Entry(fingerprint = fingerprint)
} }
if (newVersions.size > MaxNrOfVersions) throw new VectorClockException("Max number of versions reached") if (newVersions.size > MaxNrOfVersions) throw new VectorClockException("Max number of versions reached")
copy(versions = newVersions, timestamp = timestamp) copy(versions = newVersions, timestamp = timestamp)
@ -37,6 +40,7 @@ case class VectorClock(
def maxVersion: Long = versions.foldLeft(1L)((max, entry) math.max(max, entry.version)) def maxVersion: Long = versions.foldLeft(1L)((max, entry) math.max(max, entry.version))
// FIXME implement VectorClock.merge
def merge(other: VectorClock): VectorClock = { def merge(other: VectorClock): VectorClock = {
sys.error("Not implemented") sys.error("Not implemented")
} }
@ -58,13 +62,13 @@ object VectorClock {
sealed trait Ordering sealed trait Ordering
case object Before extends Ordering case object Before extends Ordering
case object After extends Ordering case object After extends Ordering
case object Concurrently extends Ordering case object Concurrent extends Ordering
/** /**
* Versioned entry in a vector clock. * Versioned entry in a vector clock.
*/ */
case class Entry(nodeId: Int, version: Long = 1) { case class Entry(fingerprint: Int, version: Long = 1L) {
def increment(): Entry = copy(version = version + 1) def increment(): Entry = copy(version = version + 1L)
} }
/** /**
@ -88,12 +92,12 @@ object VectorClock {
while (p1 < v1.versions.size && p2 < v2.versions.size) { while (p1 < v1.versions.size && p2 < v2.versions.size) {
val ver1 = v1.versions(p1) val ver1 = v1.versions(p1)
val ver2 = v2.versions(p2) val ver2 = v2.versions(p2)
if (ver1.nodeId == ver2.nodeId) { if (ver1.fingerprint == ver2.fingerprint) {
if (ver1.version > ver2.version) v1Bigger = true if (ver1.version > ver2.version) v1Bigger = true
else if (ver2.version > ver1.version) v2Bigger = true else if (ver2.version > ver1.version) v2Bigger = true
p1 += 1 p1 += 1
p2 += 1 p2 += 1
} else if (ver1.nodeId > ver2.nodeId) { } else if (ver1.fingerprint > ver2.fingerprint) {
v2Bigger = true // Since ver1 is bigger that means it is missing a version that ver2 has v2Bigger = true // Since ver1 is bigger that means it is missing a version that ver2 has
p2 += 1 p2 += 1
} else { } else {
@ -108,6 +112,6 @@ object VectorClock {
if (!v1Bigger && !v2Bigger) Before // This is the case where they are equal, return BEFORE arbitrarily if (!v1Bigger && !v2Bigger) Before // This is the case where they are equal, return BEFORE arbitrarily
else if (v1Bigger && !v2Bigger) After // This is the case where v1 is a successor clock to v2 else if (v1Bigger && !v2Bigger) After // This is the case where v1 is a successor clock to v2
else if (!v1Bigger && v2Bigger) Before // This is the case where v2 is a successor clock to v1 else if (!v1Bigger && v2Bigger) Before // This is the case where v2 is a successor clock to v1
else Concurrently // This is the case where both clocks are parallel to one another else Concurrent // This is the case where both clocks are parallel to one another
} }
} }

View file

@ -101,7 +101,7 @@ class ActorSerialization(val app: AkkaApplication, remote: RemoteSupport) {
l map { m l map { m
remoteActorSerialization.createRemoteMessageProtocolBuilder( remoteActorSerialization.createRemoteMessageProtocolBuilder(
Option(m.receiver.self), localRef,
Left(actorRef.uuid), Left(actorRef.uuid),
actorRef.address, actorRef.address,
app.AkkaConfig.ActorTimeoutMillis, app.AkkaConfig.ActorTimeoutMillis,

View file

@ -10,9 +10,7 @@ object DirectRoutedRemoteActorMultiJvmSpec {
class SomeActor extends Actor with Serializable { class SomeActor extends Actor with Serializable {
def receive = { def receive = {
case "identify" { case "identify" channel ! app.nodename
reply(app.nodename)
}
} }
} }
} }

View file

@ -8,9 +8,7 @@ object NewRemoteActorMultiJvmSpec {
class SomeActor extends Actor with Serializable { class SomeActor extends Actor with Serializable {
def receive = { def receive = {
case "identify" { case "identify" channel ! app.nodename
reply(app.nodename)
}
} }
} }
} }

View file

@ -9,7 +9,7 @@ object RandomRoutedRemoteActorMultiJvmSpec {
val NrOfNodes = 4 val NrOfNodes = 4
class SomeActor extends Actor with Serializable { class SomeActor extends Actor with Serializable {
def receive = { def receive = {
case "hit" reply(app.nodename) case "hit" channel ! app.nodename
case "end" self.stop() case "end" self.stop()
} }
} }

View file

@ -9,7 +9,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec {
val NrOfNodes = 4 val NrOfNodes = 4
class SomeActor extends Actor with Serializable { class SomeActor extends Actor with Serializable {
def receive = { def receive = {
case "hit" reply(app.nodename) case "hit" channel ! app.nodename
case "end" self.stop() case "end" self.stop()
} }
} }

View file

@ -24,7 +24,8 @@ class AccrualFailureDetectorSpec extends WordSpec with MustMatchers {
fd.isAvailable(conn) must be(true) fd.isAvailable(conn) must be(true)
} }
"mark node as dead after explicit removal of connection" in { // FIXME how should we deal with explicit removal of connection? - if triggered as failure then we have a problem in boostrap - see line 142 in AccrualFailureDetector
"mark node as dead after explicit removal of connection" ignore {
val fd = new AccrualFailureDetector val fd = new AccrualFailureDetector
val conn = new InetSocketAddress("localhost", 2552) val conn = new InetSocketAddress("localhost", 2552)

View file

@ -18,19 +18,19 @@ class VectorClockSpec extends WordSpec with MustMatchers {
"be able to add Entry if non-existing" in { "be able to add Entry if non-existing" in {
val clock1 = VectorClock() val clock1 = VectorClock()
clock1.versions must be(Vector()) clock1.versions must be(Vector())
val clock2 = clock1.incrementVersionForNode(1, System.currentTimeMillis) val clock2 = clock1.increment(1, System.currentTimeMillis)
val clock3 = clock2.incrementVersionForNode(2, System.currentTimeMillis) val clock3 = clock2.increment(2, System.currentTimeMillis)
clock3.versions must be(Vector(Entry(1, 1), Entry(2, 1))) clock3.versions must be(Vector(Entry(1, 1), Entry(2, 1)))
} }
"be able to increment version of existing Entry" in { "be able to increment version of existing Entry" in {
val clock1 = VectorClock() val clock1 = VectorClock()
val clock2 = clock1.incrementVersionForNode(1, System.currentTimeMillis) val clock2 = clock1.increment(1, System.currentTimeMillis)
val clock3 = clock2.incrementVersionForNode(2, System.currentTimeMillis) val clock3 = clock2.increment(2, System.currentTimeMillis)
val clock4 = clock3.incrementVersionForNode(1, System.currentTimeMillis) val clock4 = clock3.increment(1, System.currentTimeMillis)
val clock5 = clock4.incrementVersionForNode(2, System.currentTimeMillis) val clock5 = clock4.increment(2, System.currentTimeMillis)
val clock6 = clock5.incrementVersionForNode(2, System.currentTimeMillis) val clock6 = clock5.increment(2, System.currentTimeMillis)
clock6.versions must be(Vector(Entry(1, 2), Entry(2, 3))) clock6.versions must be(Vector(Entry(1, 2), Entry(2, 3)))
} }
@ -39,86 +39,86 @@ class VectorClockSpec extends WordSpec with MustMatchers {
val clock1 = VectorClock() val clock1 = VectorClock()
val clock2 = VectorClock() val clock2 = VectorClock()
clock1.compare(clock2) must not be (Concurrently) clock1.compare(clock2) must not be (Concurrent)
} }
"A clock should not happen before an identical clock" in { "A clock should not happen before an identical clock" in {
val clock1_1 = VectorClock() val clock1_1 = VectorClock()
val clock2_1 = clock1_1.incrementVersionForNode(1, System.currentTimeMillis) val clock2_1 = clock1_1.increment(1, System.currentTimeMillis)
val clock3_1 = clock2_1.incrementVersionForNode(2, System.currentTimeMillis) val clock3_1 = clock2_1.increment(2, System.currentTimeMillis)
val clock4_1 = clock3_1.incrementVersionForNode(1, System.currentTimeMillis) val clock4_1 = clock3_1.increment(1, System.currentTimeMillis)
val clock1_2 = VectorClock() val clock1_2 = VectorClock()
val clock2_2 = clock1_2.incrementVersionForNode(1, System.currentTimeMillis) val clock2_2 = clock1_2.increment(1, System.currentTimeMillis)
val clock3_2 = clock2_2.incrementVersionForNode(2, System.currentTimeMillis) val clock3_2 = clock2_2.increment(2, System.currentTimeMillis)
val clock4_2 = clock3_2.incrementVersionForNode(1, System.currentTimeMillis) val clock4_2 = clock3_2.increment(1, System.currentTimeMillis)
clock4_1.compare(clock4_2) must not be (Concurrently) clock4_1.compare(clock4_2) must not be (Concurrent)
} }
"A clock should happen before an identical clock with a single additional event" in { "A clock should happen before an identical clock with a single additional event" in {
val clock1_1 = VectorClock() val clock1_1 = VectorClock()
val clock2_1 = clock1_1.incrementVersionForNode(1, System.currentTimeMillis) val clock2_1 = clock1_1.increment(1, System.currentTimeMillis)
val clock3_1 = clock2_1.incrementVersionForNode(2, System.currentTimeMillis) val clock3_1 = clock2_1.increment(2, System.currentTimeMillis)
val clock4_1 = clock3_1.incrementVersionForNode(1, System.currentTimeMillis) val clock4_1 = clock3_1.increment(1, System.currentTimeMillis)
val clock1_2 = VectorClock() val clock1_2 = VectorClock()
val clock2_2 = clock1_2.incrementVersionForNode(1, System.currentTimeMillis) val clock2_2 = clock1_2.increment(1, System.currentTimeMillis)
val clock3_2 = clock2_2.incrementVersionForNode(2, System.currentTimeMillis) val clock3_2 = clock2_2.increment(2, System.currentTimeMillis)
val clock4_2 = clock3_2.incrementVersionForNode(1, System.currentTimeMillis) val clock4_2 = clock3_2.increment(1, System.currentTimeMillis)
val clock5_2 = clock4_2.incrementVersionForNode(3, System.currentTimeMillis) val clock5_2 = clock4_2.increment(3, System.currentTimeMillis)
clock4_1.compare(clock5_2) must be(Before) clock4_1.compare(clock5_2) must be(Before)
} }
"Two clocks with different events should be concurrent: 1" in { "Two clocks with different events should be concurrent: 1" in {
var clock1_1 = VectorClock() var clock1_1 = VectorClock()
val clock2_1 = clock1_1.incrementVersionForNode(1, System.currentTimeMillis) val clock2_1 = clock1_1.increment(1, System.currentTimeMillis)
val clock1_2 = VectorClock() val clock1_2 = VectorClock()
val clock2_2 = clock1_2.incrementVersionForNode(2, System.currentTimeMillis) val clock2_2 = clock1_2.increment(2, System.currentTimeMillis)
clock2_1.compare(clock2_2) must be(Concurrently) clock2_1.compare(clock2_2) must be(Concurrent)
} }
"Two clocks with different events should be concurrent: 2" in { "Two clocks with different events should be concurrent: 2" in {
val clock1_3 = VectorClock() val clock1_3 = VectorClock()
val clock2_3 = clock1_3.incrementVersionForNode(1, System.currentTimeMillis) val clock2_3 = clock1_3.increment(1, System.currentTimeMillis)
val clock3_3 = clock2_3.incrementVersionForNode(2, System.currentTimeMillis) val clock3_3 = clock2_3.increment(2, System.currentTimeMillis)
val clock4_3 = clock3_3.incrementVersionForNode(1, System.currentTimeMillis) val clock4_3 = clock3_3.increment(1, System.currentTimeMillis)
val clock1_4 = VectorClock() val clock1_4 = VectorClock()
val clock2_4 = clock1_4.incrementVersionForNode(1, System.currentTimeMillis) val clock2_4 = clock1_4.increment(1, System.currentTimeMillis)
val clock3_4 = clock2_4.incrementVersionForNode(1, System.currentTimeMillis) val clock3_4 = clock2_4.increment(1, System.currentTimeMillis)
val clock4_4 = clock3_4.incrementVersionForNode(3, System.currentTimeMillis) val clock4_4 = clock3_4.increment(3, System.currentTimeMillis)
clock4_3.compare(clock4_4) must be(Concurrently) clock4_3.compare(clock4_4) must be(Concurrent)
} }
".." in { ".." in {
val clock1_1 = VectorClock() val clock1_1 = VectorClock()
val clock2_1 = clock1_1.incrementVersionForNode(2, System.currentTimeMillis) val clock2_1 = clock1_1.increment(2, System.currentTimeMillis)
val clock3_1 = clock2_1.incrementVersionForNode(2, System.currentTimeMillis) val clock3_1 = clock2_1.increment(2, System.currentTimeMillis)
val clock1_2 = VectorClock() val clock1_2 = VectorClock()
val clock2_2 = clock1_2.incrementVersionForNode(1, System.currentTimeMillis) val clock2_2 = clock1_2.increment(1, System.currentTimeMillis)
val clock3_2 = clock2_2.incrementVersionForNode(2, System.currentTimeMillis) val clock3_2 = clock2_2.increment(2, System.currentTimeMillis)
val clock4_2 = clock3_2.incrementVersionForNode(2, System.currentTimeMillis) val clock4_2 = clock3_2.increment(2, System.currentTimeMillis)
val clock5_2 = clock4_2.incrementVersionForNode(3, System.currentTimeMillis) val clock5_2 = clock4_2.increment(3, System.currentTimeMillis)
clock3_1.compare(clock5_2) must be(Before) clock3_1.compare(clock5_2) must be(Before)
} }
"..." in { "..." in {
val clock1_1 = VectorClock() val clock1_1 = VectorClock()
val clock2_1 = clock1_1.incrementVersionForNode(1, System.currentTimeMillis) val clock2_1 = clock1_1.increment(1, System.currentTimeMillis)
val clock3_1 = clock2_1.incrementVersionForNode(2, System.currentTimeMillis) val clock3_1 = clock2_1.increment(2, System.currentTimeMillis)
val clock4_1 = clock3_1.incrementVersionForNode(2, System.currentTimeMillis) val clock4_1 = clock3_1.increment(2, System.currentTimeMillis)
val clock5_1 = clock4_1.incrementVersionForNode(3, System.currentTimeMillis) val clock5_1 = clock4_1.increment(3, System.currentTimeMillis)
val clock1_2 = VectorClock() val clock1_2 = VectorClock()
val clock2_2 = clock1_2.incrementVersionForNode(2, System.currentTimeMillis) val clock2_2 = clock1_2.increment(2, System.currentTimeMillis)
val clock3_2 = clock2_2.incrementVersionForNode(2, System.currentTimeMillis) val clock3_2 = clock2_2.increment(2, System.currentTimeMillis)
clock5_1.compare(clock3_2) must be(After) clock5_1.compare(clock3_2) must be(After)
} }

View file

@ -128,7 +128,7 @@ class MyJavaSerializableActor extends Actor with scala.Serializable {
def receive = { def receive = {
case "hello" case "hello"
count = count + 1 count = count + 1
reply("world " + count) channel ! "world " + count
} }
} }
@ -136,7 +136,7 @@ class MyStatelessActorWithMessagesInMailbox extends Actor with scala.Serializabl
def receive = { def receive = {
case "hello" case "hello"
Thread.sleep(500) Thread.sleep(500)
case "hello-reply" reply("world") case "hello-reply" channel ! "world"
} }
} }
@ -144,7 +144,7 @@ class MyActorWithProtobufMessagesInMailbox extends Actor with scala.Serializable
def receive = { def receive = {
case m: Message case m: Message
Thread.sleep(500) Thread.sleep(500)
case "hello-reply" reply("world") case "hello-reply" channel ! "world"
} }
} }
@ -152,6 +152,6 @@ class PersonActorWithMessagesInMailbox extends Actor with scala.Serializable {
def receive = { def receive = {
case p: Person case p: Person
Thread.sleep(500) Thread.sleep(500)
case "hello-reply" reply("hello") case "hello-reply" channel ! "hello"
} }
} }

View file

@ -15,6 +15,6 @@ public class UntypedConsumer1 extends UntypedConsumerActor {
public void onReceive(Object message) { public void onReceive(Object message) {
Message msg = (Message)message; Message msg = (Message)message;
String body = msg.getBodyAs(String.class); String body = msg.getBodyAs(String.class);
tryReply(String.format("received %s", body)); channel.tryTell(String.format("received %s", body));
} }
} }

View file

@ -12,7 +12,7 @@ class RemoteActor1 extends Actor with Consumer {
def endpointUri = "jetty:http://localhost:6644/camel/remote-actor-1" def endpointUri = "jetty:http://localhost:6644/camel/remote-actor-1"
protected def receive = { protected def receive = {
case msg: Message reply(Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote1"))) case msg: Message channel ! Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote1"))
} }
} }
@ -23,7 +23,7 @@ class RemoteActor2 extends Actor with Consumer {
def endpointUri = "jetty:http://localhost:6644/camel/remote-actor-2" def endpointUri = "jetty:http://localhost:6644/camel/remote-actor-2"
protected def receive = { protected def receive = {
case msg: Message reply(Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote2"))) case msg: Message channel ! Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote2"))
} }
} }
@ -44,7 +44,7 @@ class Consumer2 extends Actor with Consumer {
def endpointUri = "jetty:http://0.0.0.0:8877/camel/default" def endpointUri = "jetty:http://0.0.0.0:8877/camel/default"
def receive = { def receive = {
case msg: Message reply("Hello %s" format msg.bodyAs[String]) case msg: Message channel ! ("Hello %s" format msg.bodyAs[String])
} }
} }
@ -62,10 +62,10 @@ class Consumer4 extends Actor with Consumer {
def receive = { def receive = {
case msg: Message msg.bodyAs[String] match { case msg: Message msg.bodyAs[String] match {
case "stop" { case "stop" {
reply("Consumer4 stopped") channel ! "Consumer4 stopped"
self.stop self.stop
} }
case body reply(body) case body channel ! body
} }
} }
} }
@ -76,7 +76,7 @@ class Consumer5 extends Actor with Consumer {
def receive = { def receive = {
case _ { case _ {
Actor.actorOf[Consumer4] Actor.actorOf[Consumer4]
reply("Consumer4 started") channel ! "Consumer4 started"
} }
} }
} }
@ -106,7 +106,7 @@ class PublisherBridge(uri: String, publisher: ActorRef) extends Actor with Consu
protected def receive = { protected def receive = {
case msg: Message { case msg: Message {
publisher ! msg.bodyAs[String] publisher ! msg.bodyAs[String]
reply("message published") channel ! "message published"
} }
} }
} }
@ -135,8 +135,8 @@ class HttpProducer(transformer: ActorRef) extends Actor with Producer {
class HttpTransformer extends Actor { class HttpTransformer extends Actor {
protected def receive = { protected def receive = {
case msg: Message reply(msg.transformBody { body: String body replaceAll ("Akka ", "AKKA ") }) case msg: Message channel ! (msg.transformBody { body: String body replaceAll ("Akka ", "AKKA ") })
case msg: Failure reply(msg) case msg: Failure channel ! msg
} }
} }
@ -150,11 +150,11 @@ class FileConsumer extends Actor with Consumer {
case msg: Message { case msg: Message {
if (counter == 2) { if (counter == 2) {
println("received %s" format msg.bodyAs[String]) println("received %s" format msg.bodyAs[String])
reply(Ack) channel ! Ack
} else { } else {
println("rejected %s" format msg.bodyAs[String]) println("rejected %s" format msg.bodyAs[String])
counter += 1 counter += 1
reply(Failure(new Exception("message number %s not accepted" format counter))) channel ! Failure(new Exception("message number %s not accepted" format counter))
} }
} }
} }

View file

@ -15,7 +15,7 @@ public class SampleRemoteUntypedConsumer extends UntypedConsumerActor {
Message msg = (Message)message; Message msg = (Message)message;
String body = msg.getBodyAs(String.class); String body = msg.getBodyAs(String.class);
String header = msg.getHeaderAs("test", String.class); String header = msg.getHeaderAs("test", String.class);
tryReply(String.format("%s %s", body, header)); channel.tryTell(String.format("%s %s", body, header));
} }
} }

View file

@ -66,7 +66,7 @@ object HttpConcurrencyTestStress {
var correlationIds = Set[Any]() var correlationIds = Set[Any]()
override protected def receive = { override protected def receive = {
case "getCorrelationIdCount" reply(correlationIds.size) case "getCorrelationIdCount" channel ! correlationIds.size
case msg super.receive(msg) case msg super.receive(msg)
} }
@ -93,7 +93,7 @@ object HttpConcurrencyTestStress {
class HttpServerWorker extends Actor { class HttpServerWorker extends Actor {
protected def receive = { protected def receive = {
case msg reply(msg) case msg channel ! msg
} }
} }
} }

View file

@ -94,8 +94,8 @@ object RemoteConsumerTest {
def endpointUri = "direct:remote-consumer" def endpointUri = "direct:remote-consumer"
protected def receive = { protected def receive = {
case "init" reply("done") case "init" channel ! "done"
case m: Message reply("remote actor: %s" format m.body) case m: Message channel ! ("remote actor: %s" format m.body)
} }
} }
} }

View file

@ -1,11 +0,0 @@
sbtPlugin := true
organization := "se.scalablesolutions.akka"
name := "akka-sbt-plugin"
version := "2.0-SNAPSHOT"
publishMavenStyle := true

View file

@ -11,7 +11,7 @@ import sbt.classpath.ClasspathUtilities
import sbt.Project.Initialize import sbt.Project.Initialize
import sbt.CommandSupport._ import sbt.CommandSupport._
import java.io.File import java.io.File
import scala.collection.mutable.{ Set => MutableSet } import scala.collection.mutable.{ Set MutableSet }
object AkkaKernelPlugin extends Plugin { object AkkaKernelPlugin extends Plugin {
@ -58,8 +58,7 @@ object AkkaKernelPlugin extends Plugin {
dist <<= (dist in Dist).identity, distNeedsPackageBin) dist <<= (dist in Dist).identity, distNeedsPackageBin)
private def distTask: Initialize[Task[File]] = private def distTask: Initialize[Task[File]] =
(distConfig, sourceDirectory, crossTarget, dependencyClasspath, projectDependencies, allDependencies, buildStructure, state) map { (distConfig, sourceDirectory, crossTarget, dependencyClasspath, projectDependencies, allDependencies, buildStructure, state) map { (conf, src, tgt, cp, projDeps, allDeps, buildStruct, st)
(conf, src, tgt, cp, projDeps, allDeps, buildStruct, st)
if (isKernelProject(allDeps)) { if (isKernelProject(allDeps)) {
val log = logger(st) val log = logger(st)
@ -78,7 +77,7 @@ object AkkaKernelPlugin extends Plugin {
copyFiles(libFiles(cp, conf.libFilter), distLibPath) copyFiles(libFiles(cp, conf.libFilter), distLibPath)
copyFiles(conf.additionalLibs, distLibPath) copyFiles(conf.additionalLibs, distLibPath)
for (subTarget <- subProjectDependencies.map(_.target)) { for (subTarget subProjectDependencies.map(_.target)) {
copyJars(subTarget, distLibPath) copyJars(subTarget, distLibPath)
} }
log.info("Distribution created.") log.info("Distribution created.")
@ -97,7 +96,7 @@ object AkkaKernelPlugin extends Plugin {
} }
def isKernelProject(dependencies: Seq[ModuleID]): Boolean = { def isKernelProject(dependencies: Seq[ModuleID]): Boolean = {
dependencies.exists(moduleId => moduleId.organization == "se.scalablesolutions.akka" && moduleId.name == "akka-kernel") dependencies.exists(moduleId moduleId.organization == "se.scalablesolutions.akka" && moduleId.name == "akka-kernel")
} }
private def defaultConfigSourceDirs = (sourceDirectory, unmanagedResourceDirectories) map { (src, resources) private def defaultConfigSourceDirs = (sourceDirectory, unmanagedResourceDirectories) map { (src, resources)
@ -180,25 +179,25 @@ object AkkaKernelPlugin extends Plugin {
val buildUnit = buildStruct.units(buildStruct.root) val buildUnit = buildStruct.units(buildStruct.root)
val uri = buildStruct.root val uri = buildStruct.root
val allProjects = buildUnit.defined.map { val allProjects = buildUnit.defined.map {
case (id, proj) => (ProjectRef(uri, id) -> proj) case (id, proj) (ProjectRef(uri, id) -> proj)
} }
val projDepsNames = projDeps.map(_.name) val projDepsNames = projDeps.map(_.name)
def include(project: ResolvedProject): Boolean = projDepsNames.exists(_ == project.id) def include(project: ResolvedProject): Boolean = projDepsNames.exists(_ == project.id)
val subProjects: Seq[SubProjectInfo] = allProjects.collect { val subProjects: Seq[SubProjectInfo] = allProjects.collect {
case (projRef, project) if include(project) => projectInfo(projRef, project, buildStruct, state, allProjects) case (projRef, project) if include(project) projectInfo(projRef, project, buildStruct, state, allProjects)
}.toList }.toList
val allSubProjects = subProjects.map(_.recursiveSubProjects).flatten.toSet val allSubProjects = subProjects.map(_.recursiveSubProjects).flatten.toSet
allSubProjects allSubProjects
} }
private def projectInfo(projectRef: ProjectRef, project: ResolvedProject, buildStruct: BuildStructure, state: State, private def projectInfo(projectRef: ProjectRef, project: ResolvedProject, buildStruct: BuildStructure, state: State,
allProjects: Map[ProjectRef, ResolvedProject]): SubProjectInfo = { allProjects: Map[ProjectRef, ResolvedProject]): SubProjectInfo = {
def optionalSetting[A](key: ScopedSetting[A]) = key in projectRef get buildStruct.data def optionalSetting[A](key: ScopedSetting[A]) = key in projectRef get buildStruct.data
def setting[A](key: ScopedSetting[A], errorMessage: => String) = { def setting[A](key: ScopedSetting[A], errorMessage: String) = {
optionalSetting(key) getOrElse { optionalSetting(key) getOrElse {
logger(state).error(errorMessage); logger(state).error(errorMessage);
throw new IllegalArgumentException() throw new IllegalArgumentException()
@ -210,14 +209,14 @@ object AkkaKernelPlugin extends Plugin {
} }
val projDeps: Seq[ModuleID] = evaluateTask(Keys.projectDependencies) match { val projDeps: Seq[ModuleID] = evaluateTask(Keys.projectDependencies) match {
case Some(Value(moduleIds)) => moduleIds case Some(Value(moduleIds)) moduleIds
case _ => Seq.empty case _ Seq.empty
} }
val projDepsNames = projDeps.map(_.name) val projDepsNames = projDeps.map(_.name)
def include(project: ResolvedProject): Boolean = projDepsNames.exists(_ == project.id) def include(project: ResolvedProject): Boolean = projDepsNames.exists(_ == project.id)
val subProjects = allProjects.collect { val subProjects = allProjects.collect {
case (projRef, proj) if include(proj) => projectInfo(projRef, proj, buildStruct, state, allProjects) case (projRef, proj) if include(proj) projectInfo(projRef, proj, buildStruct, state, allProjects)
}.toList }.toList
val target = setting(Keys.crossTarget, "Missing crossTarget directory") val target = setting(Keys.crossTarget, "Missing crossTarget directory")
@ -228,8 +227,8 @@ object AkkaKernelPlugin extends Plugin {
def recursiveSubProjects: Set[SubProjectInfo] = { def recursiveSubProjects: Set[SubProjectInfo] = {
val flatSubProjects = for { val flatSubProjects = for {
x <- subProjects x subProjects
y <- x.recursiveSubProjects y x.recursiveSubProjects
} yield y } yield y
flatSubProjects.toSet + this flatSubProjects.toSet + this

View file

@ -288,8 +288,8 @@ class AgentUpdater[T](agent: Agent[T]) extends Actor {
def receive = { def receive = {
case update: Update[_] case update: Update[_]
tryReply(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T T] }) channel.tryTell(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T T] })
case Get reply(agent.get) case Get channel ! agent.get
case _ () case _ ()
} }
} }
@ -302,7 +302,7 @@ class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor {
def receive = { def receive = {
case update: Update[_] try { case update: Update[_] try {
tryReply(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T T] }) channel.tryTell(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T T] })
} finally { } finally {
agent.resume() agent.resume()
self.stop() self.stop()

View file

@ -32,7 +32,7 @@ public class UntypedCoordinatedCounter extends UntypedActor {
} else if (incoming instanceof String) { } else if (incoming instanceof String) {
String message = (String) incoming; String message = (String) incoming;
if (message.equals("GetCount")) { if (message.equals("GetCount")) {
reply(count.get()); getChannel().tell(count.get());
} }
} }
} }

View file

@ -26,7 +26,7 @@ public class UntypedCounter extends UntypedTransactor {
@Override public boolean normally(Object message) { @Override public boolean normally(Object message) {
if ("GetCount".equals(message)) { if ("GetCount".equals(message)) {
reply(count.get()); getChannel().tell(count.get());
return true; return true;
} else return false; } else return false;
} }

View file

@ -57,7 +57,7 @@ public class UntypedCoordinatedCounter extends UntypedActor {
} else if (incoming instanceof String) { } else if (incoming instanceof String) {
String message = (String) incoming; String message = (String) incoming;
if (message.equals("GetCount")) { if (message.equals("GetCount")) {
reply(count.get()); getChannel().tell(count.get());
} }
} }
} }

View file

@ -70,7 +70,7 @@ public class UntypedCounter extends UntypedTransactor {
@Override public boolean normally(Object message) { @Override public boolean normally(Object message) {
if ("GetCount".equals(message)) { if ("GetCount".equals(message)) {
reply(count.get()); getChannel().tell(count.get());
return true; return true;
} else return false; } else return false;
} }

View file

@ -34,7 +34,7 @@ object CoordinatedIncrement {
} }
} }
case GetCount reply(count.get) case GetCount channel ! count.get
} }
} }

View file

@ -56,7 +56,7 @@ object FickleFriends {
} }
} }
case GetCount reply(count.get) case GetCount channel ! count.get
} }
} }
@ -93,7 +93,7 @@ object FickleFriends {
} }
} }
case GetCount reply(count.get) case GetCount channel ! count.get
} }
} }
} }

View file

@ -49,7 +49,7 @@ object TransactorIncrement {
} }
override def normally = { override def normally = {
case GetCount reply(count.get) case GetCount channel ! count.get
} }
} }

View file

@ -150,20 +150,20 @@ class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling
} }
} }
protected[akka] override def dispatch(handle: Envelope) { protected[akka] override def dispatch(receiver: ActorCell, handle: Envelope) {
val mbox = getMailbox(handle.receiver) val mbox = getMailbox(receiver)
val queue = mbox.queue val queue = mbox.queue
val execute = mbox.suspendSwitch.fold { val execute = mbox.suspendSwitch.fold {
queue.push(handle) queue.push(handle)
if (warnings && handle.channel.isInstanceOf[Promise[_]]) { if (warnings && handle.channel.isInstanceOf[Promise[_]]) {
app.eventHandler.warning(this, "suspendSwitch, creating Future could deadlock; target: %s" format handle.receiver) app.eventHandler.warning(this, "suspendSwitch, creating Future could deadlock; target: %s" format receiver)
} }
false false
} { } {
queue.push(handle) queue.push(handle)
if (queue.isActive) { if (queue.isActive) {
if (warnings && handle.channel.isInstanceOf[Promise[_]]) { if (warnings && handle.channel.isInstanceOf[Promise[_]]) {
app.eventHandler.warning(this, "blocked on this thread, creating Future could deadlock; target: %s" format handle.receiver) app.eventHandler.warning(this, "blocked on this thread, creating Future could deadlock; target: %s" format receiver)
} }
false false
} else { } else {
@ -200,10 +200,10 @@ class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling
} }
if (handle ne null) { if (handle ne null) {
try { try {
handle.invoke mbox.actor.invoke(handle)
if (warnings) handle.channel match { if (warnings) handle.channel match {
case f: ActorPromise if !f.isCompleted case f: ActorPromise if !f.isCompleted
app.eventHandler.warning(this, "calling %s with message %s did not reply as expected, might deadlock" format (handle.receiver, handle.message)) app.eventHandler.warning(this, "calling %s with message %s did not reply as expected, might deadlock" format (mbox.actor, handle.message))
case _ case _
} }
true true

View file

@ -224,14 +224,6 @@ class TestKit(_app: AkkaApplication) {
*/ */
def within[T](max: Duration)(f: T): T = within(0 seconds, max)(f) def within[T](max: Duration)(f: T): T = within(0 seconds, max)(f)
/**
* Send reply to the last dequeued message. Will throw
* IllegalActorStateException if no message has been dequeued, yet. Dequeuing
* means reception of the message as part of an expect... or receive... call,
* not reception by the testActor.
*/
def reply(msg: AnyRef) { lastMessage.channel.!(msg)(testActor) }
/** /**
* Same as `expectMsg(remaining, obj)`, but correctly treating the timeFactor. * Same as `expectMsg(remaining, obj)`, but correctly treating the timeFactor.
*/ */
@ -271,6 +263,27 @@ class TestKit(_app: AkkaApplication) {
f(o) f(o)
} }
/**
* Hybrid of expectMsgPF and receiveWhile: receive messages while the
* partial function matches and returns false. Use it to ignore certain
* messages while waiting for a specific message.
*
* @return the last received messsage, i.e. the first one for which the
* partial function returned true
*/
def fishForMessage(max: Duration = Duration.MinusInf, hint: String = "")(f: PartialFunction[Any, Boolean]): Any = {
val _max = if (max eq Duration.MinusInf) remaining else max.dilated
val end = now + _max
@tailrec
def recv: Any = {
val o = receiveOne(end - now)
assert(o ne null, "timeout during fishForMessage, hint: " + hint)
assert(f.isDefinedAt(o), "fishForMessage(" + hint + ") found unexpected message " + o)
if (f(o)) o else recv
}
recv
}
/** /**
* Same as `expectMsgType[T](remaining)`, but correctly treating the timeFactor. * Same as `expectMsgType[T](remaining)`, but correctly treating the timeFactor.
*/ */
@ -462,12 +475,13 @@ class TestKit(_app: AkkaApplication) {
* assert(series == (1 to 7).toList) * assert(series == (1 to 7).toList)
* </pre> * </pre>
*/ */
def receiveWhile[T](max: Duration = Duration.MinusInf, idle: Duration = Duration.Inf)(f: PartialFunction[AnyRef, T]): Seq[T] = { def receiveWhile[T](max: Duration = Duration.MinusInf, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[AnyRef, T]): Seq[T] = {
val stop = now + (if (max == Duration.MinusInf) remaining else max.dilated) val stop = now + (if (max == Duration.MinusInf) remaining else max.dilated)
var msg: Message = NullMessage var msg: Message = NullMessage
@tailrec @tailrec
def doit(acc: List[T]): List[T] = { def doit(acc: List[T], count: Int): List[T] = {
if (count >= messages) return acc.reverse
receiveOne((stop - now) min idle) receiveOne((stop - now) min idle)
lastMessage match { lastMessage match {
case NullMessage case NullMessage
@ -475,7 +489,7 @@ class TestKit(_app: AkkaApplication) {
acc.reverse acc.reverse
case RealMessage(o, _) if (f isDefinedAt o) case RealMessage(o, _) if (f isDefinedAt o)
msg = lastMessage msg = lastMessage
doit(f(o) :: acc) doit(f(o) :: acc, count + 1)
case RealMessage(o, _) case RealMessage(o, _)
queue.offerFirst(lastMessage) queue.offerFirst(lastMessage)
lastMessage = msg lastMessage = msg
@ -483,7 +497,7 @@ class TestKit(_app: AkkaApplication) {
} }
} }
val ret = doit(Nil) val ret = doit(Nil, 0)
lastWasNoMsg = true lastWasNoMsg = true
ret ret
} }

View file

@ -48,14 +48,14 @@ object TestActorRefSpec {
val worker = TestActorRef(Props[WorkerActor]) val worker = TestActorRef(Props[WorkerActor])
worker ! channel worker ! channel
case "workDone" replyTo ! "complexReply" case "workDone" replyTo ! "complexReply"
case "simpleRequest" reply("simpleReply") case "simpleRequest" channel ! "simpleReply"
} }
} }
class WorkerActor() extends TActor { class WorkerActor() extends TActor {
def receiveT = { def receiveT = {
case "work" { case "work" {
reply("workDone") channel ! "workDone"
self.stop() self.stop()
} }
case replyTo: UntypedChannel { case replyTo: UntypedChannel {
@ -109,7 +109,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
"used with TestActorRef" in { "used with TestActorRef" in {
val a = TestActorRef(Props(new Actor { val a = TestActorRef(Props(new Actor {
val nested = TestActorRef(Props(self { case _ })) val nested = TestActorRef(Props(self { case _ }))
def receive = { case _ reply(nested) } def receive = { case _ channel ! nested }
})) }))
a must not be (null) a must not be (null)
val nested = (a ? "any").as[ActorRef].get val nested = (a ? "any").as[ActorRef].get
@ -120,7 +120,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
"used with ActorRef" in { "used with ActorRef" in {
val a = TestActorRef(Props(new Actor { val a = TestActorRef(Props(new Actor {
val nested = context.actorOf(Props(self { case _ })) val nested = context.actorOf(Props(self { case _ }))
def receive = { case _ reply(nested) } def receive = { case _ channel ! nested }
})) }))
a must not be (null) a must not be (null)
val nested = (a ? "any").as[ActorRef].get val nested = (a ? "any").as[ActorRef].get

View file

@ -16,7 +16,7 @@ class TestProbeSpec extends AkkaSpec {
val tk = TestProbe() val tk = TestProbe()
val future = tk.ref ? "hello" val future = tk.ref ? "hello"
tk.expectMsg(0 millis, "hello") // TestActor runs on CallingThreadDispatcher tk.expectMsg(0 millis, "hello") // TestActor runs on CallingThreadDispatcher
tk.reply("world") tk.lastMessage.channel ! "world"
future must be('completed) future must be('completed)
future.get must equal("world") future.get must equal("world")
} }
@ -26,7 +26,7 @@ class TestProbeSpec extends AkkaSpec {
val tk2 = TestProbe() val tk2 = TestProbe()
tk1.ref.!("hello")(tk2.ref) tk1.ref.!("hello")(tk2.ref)
tk1.expectMsg(0 millis, "hello") tk1.expectMsg(0 millis, "hello")
tk1.reply("world") tk1.lastMessage.channel ! "world"
tk2.expectMsg(0 millis, "world") tk2.expectMsg(0 millis, "world")
} }
@ -35,7 +35,7 @@ class TestProbeSpec extends AkkaSpec {
val probe2 = TestProbe() val probe2 = TestProbe()
probe1.send(probe2.ref, "hello") probe1.send(probe2.ref, "hello")
probe2.expectMsg(0 millis, "hello") probe2.expectMsg(0 millis, "hello")
probe2.reply("world") probe2.lastMessage.channel ! "world"
probe1.expectMsg(0 millis, "world") probe1.expectMsg(0 millis, "world")
} }

View file

@ -83,7 +83,7 @@ public class Pi {
double result = calculatePiFor(work.getStart(), work.getNrOfElements()); double result = calculatePiFor(work.getStart(), work.getNrOfElements());
// reply with the result // reply with the result
reply(new Result(result)); getChannel().tell(new Result(result));
} else throw new IllegalArgumentException("Unknown message [" + message + "]"); } else throw new IllegalArgumentException("Unknown message [" + message + "]");
} }

View file

@ -43,7 +43,7 @@ object Pi extends App {
def receive = { def receive = {
case Work(start, nrOfElements) case Work(start, nrOfElements)
reply(Result(calculatePiFor(start, nrOfElements))) // perform the work channel ! Result(calculatePiFor(start, nrOfElements)) // perform the work
} }
} }

View file

@ -80,7 +80,7 @@ public class Pi {
public void onReceive(Object message) { public void onReceive(Object message) {
if (message instanceof Work) { if (message instanceof Work) {
Work work = (Work) message; Work work = (Work) message;
reply(new Result(calculatePiFor(work.getArg(), work.getNrOfElements()))); // perform the work getChannel().tell(new Result(calculatePiFor(work.getArg(), work.getNrOfElements()))); // perform the work
} else throw new IllegalArgumentException("Unknown message [" + message + "]"); } else throw new IllegalArgumentException("Unknown message [" + message + "]");
} }
} }

View file

@ -41,7 +41,7 @@ object Pi extends App {
def receive = { def receive = {
case Work(arg, nrOfElements) case Work(arg, nrOfElements)
reply(Result(calculatePiFor(arg, nrOfElements))) // perform the work channel ! Result(calculatePiFor(arg, nrOfElements)) // perform the work
} }
} }

View file

@ -25,7 +25,7 @@ object AkkaBuild extends Build {
Unidoc.unidocExclude := Seq(samples.id, tutorials.id), Unidoc.unidocExclude := Seq(samples.id, tutorials.id),
rstdocDirectory <<= baseDirectory / "akka-docs" rstdocDirectory <<= baseDirectory / "akka-docs"
), ),
aggregate = Seq(actor, testkit, actorTests, stm, http, remote, slf4j, samples, tutorials, docs) aggregate = Seq(actor, testkit, actorTests, stm, http, remote, slf4j, akkaSbtPlugin, samples, tutorials, docs)
//aggregate = Seq(actor, testkit, actorTests, stm, http, slf4j, cluster, mailboxes, camel, camelTyped, samples, tutorials) //aggregate = Seq(actor, testkit, actorTests, stm, http, slf4j, cluster, mailboxes, camel, camelTyped, samples, tutorials)
) )
@ -228,6 +228,14 @@ object AkkaBuild extends Build {
// ) // )
// ) // )
lazy val akkaSbtPlugin = Project(
id = "akka-sbt-plugin",
base = file("akka-sbt-plugin"),
settings = defaultSettings ++ Seq(
sbtPlugin := true
)
)
lazy val samples = Project( lazy val samples = Project(
id = "akka-samples", id = "akka-samples",
base = file("akka-samples"), base = file("akka-samples"),