merge master into parental-supervision, fixing up resulting breakage
This commit is contained in:
commit
d55f02e6c1
95 changed files with 759 additions and 731 deletions
|
|
@ -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!");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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"
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
}))
|
}))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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))
|
||||||
|
|
|
||||||
|
|
@ -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(
|
||||||
|
|
|
||||||
|
|
@ -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 = {
|
||||||
|
|
|
||||||
|
|
@ -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"
|
||||||
|
|
|
||||||
|
|
@ -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))))
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
|
||||||
|
|
@ -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))
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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]
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}))
|
}))
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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 = {
|
||||||
|
|
|
||||||
|
|
@ -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)))
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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 =>
|
* case Ping =>
|
||||||
* println("got a 'Ping' message")
|
* println("got a 'Ping' message")
|
||||||
* reply("pong")
|
* channel ! "pong"
|
||||||
*
|
*
|
||||||
* case OneWay =>
|
* case OneWay =>
|
||||||
* 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)
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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 }
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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))
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -17,12 +17,8 @@ import scala.annotation.tailrec
|
||||||
/**
|
/**
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Boné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
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
// don’t ever execute normal message when system message present!
|
// don’t 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
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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")))
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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"))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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]
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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() {
|
||||||
|
|
|
||||||
|
|
@ -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ér</a>
|
* @author <a href="http://jonasboner.com">Jonas Boné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ér</a>
|
* @author <a href="http://jonasboner.com">Jonas Boné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]
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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,
|
||||||
|
|
|
||||||
|
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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"
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,11 +0,0 @@
|
||||||
|
|
||||||
sbtPlugin := true
|
|
||||||
|
|
||||||
organization := "se.scalablesolutions.akka"
|
|
||||||
|
|
||||||
name := "akka-sbt-plugin"
|
|
||||||
|
|
||||||
version := "2.0-SNAPSHOT"
|
|
||||||
|
|
||||||
publishMavenStyle := true
|
|
||||||
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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()
|
||||||
|
|
|
||||||
|
|
@ -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());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -34,7 +34,7 @@ object CoordinatedIncrement {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
case GetCount ⇒ reply(count.get)
|
case GetCount ⇒ channel ! count.get
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -49,7 +49,7 @@ object TransactorIncrement {
|
||||||
}
|
}
|
||||||
|
|
||||||
override def normally = {
|
override def normally = {
|
||||||
case GetCount ⇒ reply(count.get)
|
case GetCount ⇒ channel ! count.get
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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 + "]");
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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 + "]");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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"),
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue