merge master into parental-supervision, fixing up resulting breakage

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -12,59 +12,70 @@ import java.util.concurrent.atomic._
class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender {
"The Death Watch" must {
def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(2 seconds, actorRef + ": Stopped") {
case Terminated(`actorRef`, ex: ActorKilledException) if ex.getMessage == "Stopped" true
def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(5 seconds, actorRef + ": Stopped or Already terminated when linking") {
case Terminated(`actorRef`) true
}
"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
terminal ! "anything"
testActor ! "ping"
expectMsg("ping")
terminal ! PoisonPill
expectTerminationOf(terminal)
terminal.stop()
}
"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 _ context.self.stop() }))
val terminal = actorOf(Props(context { case _ }))
val monitor1, monitor2, monitor3 =
actorOf(Props(new Actor {
watch(terminal)
def receive = { case t: Terminated testActor ! t }
}))
monitor1 startsMonitoring terminal
monitor2 startsMonitoring terminal
testActor startsMonitoring terminal
terminal ! "anything"
terminal ! PoisonPill
expectTerminationOf(terminal)
expectTerminationOf(terminal)
expectTerminationOf(terminal)
terminal.stop()
monitor1.stop()
monitor2.stop()
monitor3.stop()
}
"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 _ context.self.stop() }))
val terminal = actorOf(Props(context { case _ }))
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 startsMonitoring terminal
testActor startsMonitoring terminal
monitor2 ! "ping"
monitor2 stopsMonitoring terminal
expectMsg("pong") //Needs to be here since startsMonitoring and stopsMonitoring are asynchronous
terminal ! "anything"
terminal ! PoisonPill
expectTerminationOf(terminal)
expectTerminationOf(terminal)
terminal.stop()
monitor1.stop()
monitor2.stop()
monitor3.stop()
}
"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 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
@ -81,24 +95,34 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
terminal ! Kill
expectTerminationOf(terminal)
terminal.isShutdown must be === true
terminal.stop()
supervisor.stop()
}
}
"fail a monitor which does not handle Terminated()" in {
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
brother startsMonitoring failed
testActor startsMonitoring brother
failed ! Kill
expectMsgPF() {
case Terminated(brother, DeathPactException(failed, _: ActorKilledException)) true
val result = receiveWhile(3 seconds, messages = 3) {
case Failed(`failed`, _: ActorKilledException) 1
case Failed(`brother`, DeathPactException(`failed`)) 2
case Terminated(`brother`) 3
}
testActor must not be 'shutdown
result must be(Seq(1, 2, 3))
}
}
}

View file

@ -55,8 +55,12 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
"not fail when listener goes away" in {
val forward = actorOf(new Forwarder(testActor))
val sup = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None)))
val fsm = sup startsMonitoring actorOf(new MyFSM(testActor))
val fsm = 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) {
fsm ! SubscribeTransitionCallBack(forward)
expectMsg(CurrentState(fsm, 0))

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -4,16 +4,13 @@
package akka.actor
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import org.scalatest.BeforeAndAfterEach
import org.scalatest.BeforeAndAfterAll
import akka.testkit.Testing.sleepFor
import akka.util.duration._
import akka.{ Die, Ping }
import akka.actor.Actor._
import akka.testkit.TestEvent._
import akka.testkit.EventFilter
import akka.testkit.{ EventFilter, ImplicitSender }
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.LinkedBlockingQueue
import akka.testkit.AkkaSpec
@ -42,7 +39,7 @@ object SupervisorSpec {
def receive = {
case Ping
messageLog.put(PingMessage)
tryReply(PongMessage)
channel.tryTell(PongMessage)
case Die
throw new RuntimeException(ExceptionMessage)
}
@ -60,13 +57,13 @@ object SupervisorSpec {
def receive = {
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])
class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach {
class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender {
import SupervisorSpec._
@ -150,9 +147,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach {
"not restart programmatically linked temporary actor" in {
val master = actorOf(Props[Master].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0))))
(master.?(Die, TimeoutMillis)).get match {
case r: RuntimeException r === ExceptionMessage
}
master ! Die
expectMsg(3 seconds, "terminated")
sleepFor(1 second)
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!")
def receive = {
case Ping tryReply(PongMessage)
case Ping channel.tryTell(PongMessage)
case Die throw new RuntimeException("Expected")
}
})

View file

@ -22,7 +22,7 @@ class SupervisorTreeSpec extends AkkaSpec with ImplicitSender {
within(5 seconds) {
val p = Props(new Actor {
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 }
}).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 1000))

View file

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

View file

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

View file

@ -9,7 +9,7 @@ import akka.actor.{ Props, Actor }
object DispatcherActorSpec {
class TestActor extends Actor {
def receive = {
case "Hello" reply("World")
case "Hello" channel ! "World"
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 latch = new CountDownLatch(100)
val thereWeAre = new CountDownLatch(1)
val start = new CountDownLatch(1)
val fastOne = actorOf(
Props(context { case "sabotage" works.set(false) }).withDispatcher(throughputDispatcher))
val slowOne = actorOf(
Props(context {
case "hogexecutor" thereWeAre.countDown(); start.await
case "hogexecutor" context.channel ! "OK"; start.await
case "ping" if (works.get) latch.countDown()
}).withDispatcher(throughputDispatcher))
slowOne ! "hogexecutor"
assert((slowOne ? "hogexecutor").get === "OK")
(1 to 100) foreach { _ slowOne ! "ping" }
assert(thereWeAre.await(2, TimeUnit.SECONDS))
fastOne ! "sabotage"
start.countDown()
latch.await(10, TimeUnit.SECONDS)

View file

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

View file

@ -17,20 +17,16 @@ import org.scalatest.junit.JUnitSuite
object FutureSpec {
class TestActor extends Actor {
def receive = {
case "Hello"
reply("World")
case "NoReply" {}
case "Failure"
throw new RuntimeException("Expected exception; to test fault-tolerance")
case "Hello" channel ! "World"
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
case "NoReply"
}
}
class TestDelayActor(await: StandardLatch) extends Actor {
def receive = {
case "Hello"
await.await
reply("World")
case "NoReply" { await.await }
case "Hello" await.await; channel ! "World"
case "NoReply" await.await
case "Failure"
await.await
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 {
behave like futureWithResult { test
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 }
future.await
test(future, "WORLD")
@ -152,7 +148,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
behave like futureWithException[ArithmeticException] { test
filterException[ArithmeticException] {
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 }
future.await
test(future, "/ by zero")
@ -165,7 +161,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
behave like futureWithException[MatchError] { test
filterException[MatchError] {
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 }
future.await
test(future, "World (of class java.lang.String)")
@ -182,8 +178,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
filterException[ClassCastException] {
val actor = actorOf(new Actor {
def receive = {
case s: String reply(s.length)
case i: Int reply((i * 2).toString)
case s: String channel ! s.length
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)
val actor = actorOf(new Actor {
def receive = {
case Req(s: String) reply(Res(s.length))
case Req(i: Int) reply(Res((i * 2).toString))
case Req(s: String) channel ! Res(s.length)
case Req(i: Int) channel ! Res((i * 2).toString)
}
})
@ -301,7 +297,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"fold" in {
val actors = (1 to 10).toList map { _
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
@ -312,7 +308,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"fold by composing" in {
val actors = (1 to 10).toList map { _
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] }
@ -327,7 +323,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
case (add: Int, wait: Int)
Thread.sleep(wait)
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 {
val actors = (1 to 10).toList map { _
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
@ -375,7 +371,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
case (add: Int, wait: Int)
Thread.sleep(wait)
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
def receive = {
case 'GetNext
reply(counter)
channel ! counter
counter += 2
}
})

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -5,6 +5,7 @@ package akka
import akka.config._
import akka.actor._
import event._
import java.net.InetAddress
import com.eaio.uuid.UUID
import akka.dispatch.{ Dispatcher, Dispatchers, Future, DefaultPromise }
@ -16,9 +17,6 @@ import akka.dispatch.UnboundedMailbox
import akka.routing.Routing
import akka.remote.RemoteSupport
import akka.serialization.Serialization
import akka.event.EventHandler
import akka.event.EventHandlerLogging
import akka.event.Logging
import java.net.InetSocketAddress
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 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 {
@ -193,6 +194,9 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
val log: Logging = new EventHandlerLogging(eventHandler, this)
// 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 deathWatch = provider.createDeathWatch()

View file

@ -55,7 +55,7 @@ case class HotSwap(code: ActorRef ⇒ Actor.Receive, discardOld: Boolean = true)
case class Failed(@BeanProperty actor: ActorRef,
@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
@ -63,7 +63,7 @@ case object PoisonPill 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
@ -95,8 +95,8 @@ class InvalidMessageException private[akka] (message: String, cause: Throwable =
def this(msg: String) = this(msg, null);
}
case class DeathPactException private[akka] (dead: ActorRef, cause: Throwable)
extends AkkaException("monitored actor " + dead + " terminated", cause)
case class DeathPactException private[akka] (dead: ActorRef)
extends AkkaException("monitored actor " + dead + " terminated")
with NoStackTrace
// must not pass InterruptedException to other threads
@ -175,6 +175,11 @@ object Actor {
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
private def config = context.app.AkkaConfig
/**
* 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
@ -244,7 +247,7 @@ trait Actor {
* This method does NOT modify the given Receive unless
* 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.
@ -252,7 +255,7 @@ trait Actor {
* Mainly for internal use, functions as the implicit sender references when invoking
* 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.
@ -260,7 +263,7 @@ trait Actor {
* Mainly for internal use, functions as the implicit sender references when invoking
* 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.
@ -276,20 +279,15 @@ trait Actor {
* The reference sender Actor of the last received message.
* Is defined if the message was sent from another Actor, else None.
*/
def sender: Option[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
@inline
final def sender: ActorRef = context.sender
/**
* Abstraction for unification of sender and senderFuture for later reply
*/
def channel: UntypedChannel = context.channel
// just for current compatibility
// TODO FIXME REMOVE ME just for current compatibility
implicit def forwardable: ForwardableChannel = ForwardableChannel(channel)
/**
@ -306,31 +304,6 @@ trait Actor {
*/
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
*/
@ -352,7 +325,7 @@ trait Actor {
* def receive = {
* case Ping =&gt;
* println("got a 'Ping' message")
* reply("pong")
* channel ! "pong"
*
* case OneWay =&gt;
* println("got a 'OneWay' message")
@ -403,7 +376,7 @@ trait Actor {
*/
def unhandled(message: Any) {
message match {
case Terminated(dead, cause) throw new DeathPactException(dead, cause)
case Terminated(dead) throw new DeathPactException(dead)
case _ throw new UnhandledMessageException(message, self)
}
}
@ -426,16 +399,26 @@ trait Actor {
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 ====
// =========================================
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) {
if (config.DebugAutoReceive) app.eventHandler.debug(this, "received AutoReceiveMessage " + msg)
if (app.AkkaConfig.DebugAutoReceive) app.eventHandler.debug(this, "received AutoReceiveMessage " + msg)
msg match {
case HotSwap(code, discardOld) become(code(self), discardOld)

View file

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

View file

@ -9,6 +9,7 @@ import akka.util._
import scala.collection.immutable.Stack
import java.lang.{ UnsupportedOperationException, IllegalStateException }
import akka.AkkaApplication
import akka.event.ActorEventBus
/**
* 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
* implement request/response message exchanges.
* <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.
*/
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))
}
case class DeadLetter(message: Any, channel: UntypedChannel)
class DeadLetterActorRef(app: AkkaApplication) extends UnsupportedActorRef {
val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher)
override val address: String = "akka:internal:DeadLetterActorRef"
@ -375,10 +378,10 @@ class DeadLetterActorRef(app: AkkaApplication) extends UnsupportedActorRef {
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(
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = { app.eventHandler.debug(this, message); brokenPromise }
channel: UntypedChannel): Future[Any] = { app.eventHandler.notify(DeadLetter(message, channel)); brokenPromise }
}

View file

@ -101,10 +101,7 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
protected[akka] override def postMessageToMailbox(msg: Any, channel: UntypedChannel) {
msg match {
case Failed(child, ex) child.stop()
case ChildTerminated(child, ex) ex match {
case a: ActorKilledException if a.getMessage == "Stopped" terminationFuture.completeWithResult(AkkaApplication.Stopped)
case x terminationFuture.completeWithResult(AkkaApplication.Failed(x))
}
case ChildTerminated(child) terminationFuture.completeWithResult(AkkaApplication.Stopped)
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 = {
if (!super.subscribe(subscriber, to)) {
subscriber ! Terminated(subscriber, new ActorKilledException("Already terminated when linking"))
subscriber ! Terminated(to)
false
} else true
}

View file

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

View file

@ -133,7 +133,7 @@ abstract class FaultHandlingStrategy {
/**
* Returns whether it processed the failure or not
*/
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 action = if (decider.isDefinedAt(cause)) decider(cause) else Escalate
action match {

View file

@ -42,7 +42,7 @@ object Props {
*/
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

View file

@ -130,7 +130,15 @@ trait TypedActorFactory { this: ActorRefFactory ⇒
* 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 =
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,
@ -138,10 +146,31 @@ trait TypedActorFactory { this: ActorRefFactory ⇒
* 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 =
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 =
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,
@ -149,44 +178,73 @@ trait TypedActorFactory { this: ActorRefFactory ⇒
* 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 =
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,
* 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 =
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,
* 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]]
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,
* to create TypedActor proxies, use typedActorOf
*/
def createProxy[R <: AnyRef](constructor: Actor, props: Props = Props(), 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)
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, 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,
* to create TypedActor proxies, use typedActorOf
*/
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,
* to create TypedActor proxies, use typedActorOf
*/
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
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]
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 =
createProxy[R](supervisor, extractInterfaces(interface), (ref: AtomVar[R]) new TypedActor[R, T](ref, constructor), props, loader)
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, 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
val actorVar = new AtomVar[ActorRef](null)
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]
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
proxyVar.get
}
@ -286,7 +344,7 @@ class TypedActor(val app: AkkaApplication) {
case p: ActorPromise p completeWith m(me).asInstanceOf[Future[Any]]
case _ throw new IllegalStateException("Future-returning TypedActor didn't use ?/ask so cannot reply")
}
} else reply(m(me))
} else channel ! m(me)
} finally {
TypedActor.selfReference set null

View file

@ -21,17 +21,13 @@ import akka.dispatch.{ MessageDispatcher, Promise }
*
* if (msg.equals("UseReply")) {
* // 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()) {
* // Reply to original sender of message using the sender reference
* // also passing along my own reference (the self)
* 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")) {
* // Send message to the actor itself recursively
* getSelf().tell(msg)
@ -71,13 +67,7 @@ abstract class UntypedActor extends Actor {
* The reference sender Actor of the last received message.
* Is defined if the message was sent from another Actor, else None.
*/
def getSender: Option[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
def getSender: ActorRef = sender
/**
* Abstraction for unification of sender and senderFuture for later reply

View file

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

View file

@ -57,12 +57,7 @@ class BalancingDispatcher(
class SharingMailbox(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue {
final def enqueue(handle: Envelope) = messageQueue.enqueue(handle)
final def dequeue(): Envelope = {
val envelope = messageQueue.dequeue()
if (envelope eq null) null
else if (envelope.receiver eq actor) envelope
else envelope.copy(receiver = actor)
}
final def dequeue(): Envelope = messageQueue.dequeue()
final def numberOfMessages: Int = messageQueue.numberOfMessages
@ -95,6 +90,8 @@ class BalancingDispatcher(
while (messages ne null) {
deadLetterMailbox.systemEnqueue(messages) //Send to dead letter queue
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
}
override protected[akka] def dispatch(invocation: Envelope) = {
val receiver = invocation.receiver
override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
messageQueue enqueue invocation
val buddy = buddies.pollFirst()

View file

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

View file

@ -30,7 +30,7 @@ private[dispatch] object Mailbox {
// secondary status: Scheduled bit may be added to Open/Suspended
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
}
@ -172,8 +172,8 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag
var processedMessages = 0
val deadlineNs = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + TimeUnit.MILLISECONDS.toNanos(dispatcher.throughputDeadlineTime) else 0
do {
if (debug) println(actor + " processing message " + nextMessage.message + " from " + nextMessage.channel)
nextMessage.invoke
if (debug) println(actor + " processing message " + nextMessage)
actor invoke nextMessage
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
} while (nextMessage ne null)
} 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
}
}
@ -202,7 +202,6 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag
// dont ever execute normal message when system message present!
if (nextMessage eq null) nextMessage = systemDrain()
}
if (debug) println(actor + " has finished processing system messages")
} catch {
case e
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
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
/*
* this write is safely published by the compareAndSet contained within

View file

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

View file

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

View file

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

View file

@ -24,7 +24,7 @@ object TypedCamelTestSupport {
def countdown: Handler = {
case SetExpectedMessageCount(num) {
latch = new CountDownLatch(num)
reply(latch)
channel ! latch
}
case msg latch.countDown
}
@ -32,7 +32,7 @@ object TypedCamelTestSupport {
trait Respond { this: Actor
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
@ -42,8 +42,8 @@ object TypedCamelTestSupport {
val messages = Buffer[Any]()
def retain: Handler = {
case GetRetainedMessage reply(messages.last)
case GetRetainedMessages(p) reply(messages.toList.filter(p))
case GetRetainedMessage channel ! messages.last
case GetRetainedMessages(p) channel ! messages.filter(p).toList
case msg {
messages += msg
msg

View file

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

View file

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

View file

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

View file

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

View file

@ -36,7 +36,7 @@ object CamelTestSupport {
def countdown: Handler = {
case SetExpectedMessageCount(num) {
latch = new CountDownLatch(num)
reply(latch)
channel ! latch
}
case msg latch.countDown
}
@ -44,7 +44,7 @@ object CamelTestSupport {
trait Respond { this: Actor
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
@ -54,8 +54,8 @@ object CamelTestSupport {
val messages = Buffer[Any]()
def retain: Handler = {
case GetRetainedMessage reply(messages.last)
case GetRetainedMessages(p) reply(messages.toList.filter(p))
case GetRetainedMessage channel ! messages.last
case GetRetainedMessages(p) channel ! messages.filter(p).toList
case msg {
messages += msg
msg

View file

@ -211,7 +211,7 @@ object ConsumerScalaTest {
class TestConsumer(uri: String) extends Actor with Consumer {
def endpointUri = uri
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
override def autoack = false
protected def receive = {
case msg: Message reply(Ack)
case msg: Message channel ! Ack
}
}
@ -247,15 +247,15 @@ object ConsumerScalaTest {
protected def receive = {
case "fail" { throw new Exception("test") }
case "succeed" reply("ok")
case "succeed" channel ! "ok"
}
override def preRestart(reason: scala.Throwable, msg: Option[Any]) {
tryReply("pr")
channel.tryTell("pr")
}
override def postStop {
tryReply("ps")
channel.tryTell("ps")
}
}
@ -288,7 +288,7 @@ object ConsumerScalaTest {
}
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)
}

View file

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

View file

@ -96,13 +96,13 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
object ActorComponentFeatureTest {
class CustomIdActor extends Actor {
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 {
protected def receive = {
case msg: Message reply(Failure(new Exception("test")))
case msg: Message channel ! Failure(new Exception("test"))
}
}

View file

@ -47,7 +47,7 @@ import akka.cluster.metrics._
import akka.cluster.zookeeper._
import ChangeListener._
import RemoteProtocol._
import RemoteDaemonMessageType._
import RemoteSystemDaemonMessageType._
import com.eaio.uuid.UUID
@ -818,7 +818,7 @@ class DefaultClusterNode private[akka] (
EventHandler.debug(this,
"Sending command to nodes [%s] for checking out actor [%s]".format(nodes.mkString(", "), actorAddress))
val builder = RemoteDaemonMessageProtocol.newBuilder
val builder = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(USE)
.setActorAddress(actorAddress)
@ -882,7 +882,7 @@ class DefaultClusterNode private[akka] (
EventHandler.debug(this,
"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)
.setActorAddress(actorAddress)
.build
@ -1030,7 +1030,7 @@ class DefaultClusterNode private[akka] (
Serialization.serialize(f) match {
case Left(error) throw error
case Right(bytes)
val message = RemoteDaemonMessageProtocol.newBuilder
val message = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FUNCTION_FUN0_UNIT)
.setPayload(ByteString.copyFrom(bytes))
.build
@ -1046,7 +1046,7 @@ class DefaultClusterNode private[akka] (
Serialization.serialize(f) match {
case Left(error) throw error
case Right(bytes)
val message = RemoteDaemonMessageProtocol.newBuilder
val message = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FUNCTION_FUN0_ANY)
.setPayload(ByteString.copyFrom(bytes))
.build
@ -1063,7 +1063,7 @@ class DefaultClusterNode private[akka] (
Serialization.serialize((f, arg)) match {
case Left(error) throw error
case Right(bytes)
val message = RemoteDaemonMessageProtocol.newBuilder
val message = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FUNCTION_FUN1_ARG_UNIT)
.setPayload(ByteString.copyFrom(bytes))
.build
@ -1080,7 +1080,7 @@ class DefaultClusterNode private[akka] (
Serialization.serialize((f, arg)) match {
case Left(error) throw error
case Right(bytes)
val message = RemoteDaemonMessageProtocol.newBuilder
val message = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FUNCTION_FUN1_ARG_ANY)
.setPayload(ByteString.copyFrom(bytes))
.build
@ -1151,7 +1151,7 @@ class DefaultClusterNode private[akka] (
// Private
// =======================================
private def sendCommandToNode(connection: ActorRef, command: RemoteDaemonMessageProtocol, async: Boolean = true) {
private def sendCommandToNode(connection: ActorRef, command: RemoteSystemDaemonMessageProtocol, async: Boolean = true) {
if (async) {
connection ! command
} else {
@ -1442,7 +1442,7 @@ class DefaultClusterNode private[akka] (
case Left(error) throw error
case Right(bytes)
val command = RemoteDaemonMessageProtocol.newBuilder
val command = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(FAIL_OVER_CONNECTIONS)
.setPayload(ByteString.copyFrom(bytes))
.build
@ -1713,7 +1713,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
}
def receive: Receive = {
case message: RemoteDaemonMessageProtocol
case message: RemoteSystemDaemonMessageProtocol
EventHandler.debug(this,
"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))
}
def handleRelease(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
def handleRelease(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
if (message.hasActorUuid) {
cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { 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] = {
import akka.cluster.RemoteProtocol._
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(
Props(
self {
@ -1863,7 +1863,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
}).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(
Props(
self {
@ -1871,7 +1871,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
}).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(
Props(
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]])
}
def handle_fun1_arg_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
def handle_fun1_arg_any(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(
Props(
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]])
}
def handleFailover(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
def handleFailover(message: RemoteProtocol.RemoteSystemDaemonMessageProtocol) {
val (from, to) = payloadFor(message, classOf[(InetSocketAddress, InetSocketAddress)])
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 {
case Left(error) throw error
case Right(instance) instance.asInstanceOf[T]

View file

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

View file

@ -179,7 +179,7 @@ This is fine when dealing with a known amount of Actors, but can grow unwieldy i
To better explain what happened in the example, ``Future.sequence`` is taking the ``List[Future[Int]]`` and turning it into a ``Future[List[Int]]``. We can then use ``map`` to work with the ``List[Int]`` directly, and we find the sum of the ``List``.
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

View file

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

View file

@ -156,16 +156,6 @@ enum LifeCycleType {
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.
*/
@ -190,10 +180,10 @@ message ExceptionProtocol {
}
/**
* Defines the remote daemon message.
* Defines the remote system daemon message.
*/
message RemoteDaemonMessageProtocol {
required RemoteDaemonMessageType messageType = 1;
message RemoteSystemDaemonMessageProtocol {
required RemoteSystemDaemonMessageType messageType = 1;
optional UuidProtocol actorUuid = 2;
optional string actorAddress = 3;
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;
USE = 2;
RELEASE = 3;
@ -212,11 +202,12 @@ enum RemoteDaemonMessageType {
DISCONNECT = 6;
RECONNECT = 7;
RESIGN = 8;
FAIL_OVER_CONNECTIONS = 9;
FUNCTION_FUN0_UNIT = 10;
FUNCTION_FUN0_ANY = 11;
FUNCTION_FUN1_ARG_UNIT = 12;
FUNCTION_FUN1_ARG_ANY = 13;
GOSSIP = 9;
FAIL_OVER_CONNECTIONS = 20;
FUNCTION_FUN0_UNIT = 21;
FUNCTION_FUN0_ANY = 22;
FUNCTION_FUN1_ARG_UNIT = 23;
FUNCTION_FUN1_ARG_ANY = 24;
}
/**

View file

@ -28,7 +28,7 @@ import scala.annotation.tailrec
* Default threshold is 8 (taken from Cassandra defaults), but can be configured in the Akka config.
*/
class AccrualFailureDetector(
val threshold: Int = 8, // FIXME make these configurable
val threshold: Int = 8,
val maxSampleSize: Int = 1000) extends FailureDetector {
final val PhiFactor = 1.0 / math.log(10.0)
@ -139,7 +139,7 @@ class AccrualFailureDetector(
def phi(connection: InetSocketAddress): Double = {
val oldState = state.get
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 {
PhiFactor * (newTimestamp - oldTimestamp.get) / oldState.failureStats.get(connection).getOrElse(FailureStats()).mean
}

View file

@ -4,22 +4,24 @@
package akka.remote
import akka.{ AkkaException, AkkaApplication }
import akka.actor._
import akka.routing._
import akka.actor.Actor._
import akka.actor.Status._
import akka.routing._
import akka.dispatch._
import akka.util.duration._
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.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.
@ -39,10 +41,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
private val actors = new ConcurrentHashMap[String, AnyRef]
private val remoteDaemonConnectionManager = new RemoteConnectionManager(
app,
remote = remote,
failureDetector = new BannagePeriodFailureDetector(60 seconds)) // FIXME make timeout configurable
private val remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote)
def defaultDispatcher = app.dispatcher
def defaultTimeout = app.AkkaConfig.ActorTimeout
@ -58,12 +57,13 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
app.deployer.lookupDeploymentFor(address) match {
case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, DeploymentConfig.RemoteScope(remoteAddresses)))
val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match {
case FailureDetectorType.NoOp new NoOpFailureDetector
case FailureDetectorType.RemoveConnectionOnFirstFailure new RemoveConnectionOnFirstFailureFailureDetector
case FailureDetectorType.BannagePeriod(timeToBan) new BannagePeriodFailureDetector(timeToBan)
case FailureDetectorType.Custom(implClass) FailureDetector.createCustomFailureDetector(implClass)
}
// 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
// val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match {
// case FailureDetectorType.NoOp new NoOpFailureDetector
// case FailureDetectorType.RemoveConnectionOnFirstFailure new RemoveConnectionOnFirstFailureFailureDetector
// case FailureDetectorType.BannagePeriod(timeToBan) new BannagePeriodFailureDetector(timeToBan)
// case FailureDetectorType.Custom(implClass) FailureDetector.createCustomFailureDetector(implClass)
// }
val thisHostname = remote.address.getHostName
val thisPort = remote.address.getPort
@ -112,7 +112,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
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) }
@ -177,7 +177,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider
else bytes
}
val command = RemoteDaemonMessageProtocol.newBuilder
val command = RemoteSystemDaemonMessageProtocol.newBuilder
.setMessageType(USE)
.setActorAddress(actorAddress)
.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
}
private def sendCommandToRemoteNode(
connection: ActorRef,
command: RemoteDaemonMessageProtocol,
withACK: Boolean) {
private def sendCommandToRemoteNode(connection: ActorRef, command: RemoteSystemDaemonMessageProtocol, withACK: Boolean) {
if (withACK) {
try {
(connection ? (command, remote.remoteDaemonAckTimeout)).as[Status] match {
(connection ? (command, remote.remoteSystemDaemonAckTimeout)).as[Status] match {
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))
app.eventHandler.error(cause, this, cause.toString)
throw cause
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)
throw error
}
} catch {
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
}
} else {

View file

@ -22,15 +22,17 @@ import java.util.concurrent.atomic.AtomicReference
class RemoteConnectionManager(
app: AkkaApplication,
remote: Remote,
initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef],
failureDetector: FailureDetector = new NoOpFailureDetector)
initialConnections: Map[InetSocketAddress, ActorRef] = Map.empty[InetSocketAddress, ActorRef])
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])
extends VersionedIterable[ActorRef] {
def iterable: Iterable[ActorRef] = connections.values
}
val failureDetector = remote.failureDetector
private val state: AtomicReference[State] = new AtomicReference[State](newState())
// register all initial connections - e.g listen to events from them
@ -48,10 +50,13 @@ class RemoteConnectionManager(
def version: Long = state.get.version
// FIXME should not return State value but a Seq with connections
def connections = filterAvailableConnections(state.get)
def size: Int = connections.connections.size
def connectionFor(address: InetSocketAddress): Option[ActorRef] = connections.connections.get(address)
def isEmpty: Boolean = connections.connections.isEmpty
def shutdown() {

View file

@ -14,44 +14,54 @@ import akka.util.duration._
import akka.util.Helpers._
import akka.actor.DeploymentConfig._
import akka.serialization.{ Serialization, Serializer, ActorSerialization, Compression }
import Compression.LZF
import RemoteProtocol._
import RemoteDaemonMessageType._
import akka.serialization.Compression.LZF
import akka.remote.RemoteProtocol._
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
import java.net.InetSocketAddress
import com.eaio.uuid.UUID
// FIXME renamed file from RemoteDaemon.scala to Remote.scala
/**
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class Remote(val app: AkkaApplication) extends RemoteService {
import app._
import app.config
import app.AkkaConfig.DefaultTimeUnit
import app.AkkaConfig._
// TODO move to AkkaConfig?
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 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?
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(
OneForOneStrategy(List(classOf[Exception]), None, None))) // is infinite restart what we want?
// FIXME check that this supervision is okay
private[remote] lazy val remoteDaemon =
new LocalActorRef(
app,
Props(new RemoteDaemon(this)).withDispatcher(app.dispatcherFactory.newPinnedDispatcher("Remote")),
app.guardian,
Props(new RemoteSystemDaemon(this))
.withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
remoteDaemonSupervisor,
remoteDaemonServiceName,
true)
systemService = true)
private[remote] lazy val remoteClientLifeCycleHandler = app.actorOf(Props(new Actor {
def receive = {
@ -59,7 +69,7 @@ class Remote(val app: AkkaApplication) extends RemoteService {
case RemoteClientDisconnected(client, address) client.shutdownClientModule()
case _ //ignore other
}
}), "akka.cluster.RemoteClientLifeCycleListener")
}), "akka.remote.RemoteClientLifeCycleListener")
lazy val eventStream = new NetworkEventStream(app)
@ -70,7 +80,7 @@ class Remote(val app: AkkaApplication) extends RemoteService {
remote.addListener(eventStream.channel)
remote.addListener(remoteClientLifeCycleHandler)
// TODO actually register this provider in app in remote mode
//app.provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider)
//provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider)
remote
}
@ -78,7 +88,7 @@ class Remote(val app: AkkaApplication) extends RemoteService {
def start() {
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)
@ -91,24 +101,25 @@ class Remote(val app: AkkaApplication) extends RemoteService {
}
/**
* Internal "daemon" actor for cluster internal communication.
* Internal system "daemon" actor for remote internal communication.
*
* It acts as the brain of the cluster that responds to cluster events (messages) and undertakes action.
* It acts as the brain of the remote that responds to system remote events (messages) and undertakes action.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteDaemon(val remote: Remote) extends Actor {
class RemoteSystemDaemon(remote: Remote) extends Actor {
import remote._
import remote.app._
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 = {
case message: RemoteDaemonMessageProtocol
app.eventHandler.debug(this,
"Received command [\n%s] to RemoteDaemon on [%s]".format(message, app.nodename))
case message: RemoteSystemDaemonMessageProtocol
eventHandler.debug(this,
"Received command [\n%s] to RemoteSystemDaemon on [%s]".format(message, nodename))
message.getMessageType match {
case USE handleUse(message)
@ -118,6 +129,7 @@ class RemoteDaemon(val remote: Remote) extends Actor {
// case RECONNECT cluster.reconnect()
// case RESIGN cluster.resign()
// case FAIL_OVER_CONNECTIONS handleFailover(message)
case GOSSIP handleGossip(message)
case FUNCTION_FUN0_UNIT handle_fun0_unit(message)
case FUNCTION_FUN0_ANY handle_fun0_any(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?
}
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 {
if (message.hasActorAddress) {
@ -137,7 +149,7 @@ class RemoteDaemon(val remote: Remote) extends Actor {
else message.getPayload.toByteArray
val actorFactory =
app.serialization.deserialize(actorFactoryBytes, classOf[() Actor], None) match {
serialization.deserialize(actorFactoryBytes, classOf[() Actor], None) match {
case Left(error) throw error
case Right(instance) instance.asInstanceOf[() Actor]
}
@ -145,37 +157,43 @@ class RemoteDaemon(val remote: Remote) extends Actor {
val actorAddress = message.getActorAddress
val newActorRef = app.actorOf(Props(creator = actorFactory), actorAddress)
remote.server.register(actorAddress, newActorRef)
server.register(actorAddress, newActorRef)
} 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 {
case error: Throwable
reply(Failure(error))
channel ! Failure(error)
throw error
}
}
def handleRelease(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
// FIXME implement handleRelease without Cluster
// FIXME implement handleRelease
def handleRelease(message: RemoteSystemDaemonMessageProtocol) {
}
// if (message.hasActorUuid) {
// cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address
// cluster.release(address)
def handleGossip(message: RemoteSystemDaemonMessageProtocol) {
// try {
// 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
// } else {
// EventHandler.warning(this,
// "None of 'uuid' or 'actorAddress'' is specified, ignoring remote cluster daemon command [%s]".format(message))
// gossiper tell gossip
// channel ! Success(address.toString)
// } catch {
// case error: Throwable
// channel ! Failure(error)
// throw error
// }
}
// FIXME: handle real remote supervision
def handle_fun0_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app,
Props(
context {
@ -184,16 +202,16 @@ class RemoteDaemon(val remote: Remote) extends Actor {
}
// FIXME: handle real remote supervision
def handle_fun0_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
def handle_fun0_any(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app,
Props(
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]])
}
// FIXME: handle real remote supervision
def handle_fun1_arg_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
def handle_fun1_arg_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app,
Props(
context {
@ -202,21 +220,21 @@ class RemoteDaemon(val remote: Remote) extends Actor {
}
// FIXME: handle real remote supervision
def handle_fun1_arg_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
def handle_fun1_arg_any(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app,
Props(
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]])
}
def handleFailover(message: RemoteProtocol.RemoteDaemonMessageProtocol) {
def handleFailover(message: RemoteSystemDaemonMessageProtocol) {
// val (from, to) = payloadFor(message, classOf[(InetSocketremoteDaemonServiceName, InetSocketremoteDaemonServiceName)])
// cluster.failOverClusterActorRefConnections(from, to)
}
private def payloadFor[T](message: RemoteDaemonMessageProtocol, clazz: Class[T]): T = {
app.serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
private def payloadFor[T](message: RemoteSystemDaemonMessageProtocol, clazz: Class[T]): T = {
serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
case Left(error) throw error
case Right(instance) instance.asInstanceOf[T]
}

View file

@ -9,8 +9,11 @@ import akka.AkkaException
class VectorClockException(message: String) extends AkkaException(message)
/**
* Representation of a Vector-based clock (counting clock).
* For details see Wikipedia: [http://en.wikipedia.org/wiki/Vector_clock].
* Representation of a Vector-based clock (counting clock), inspired by Lamport logical clocks.
*
* 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(
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 incrementVersionForNode(nodeId: Int, timestamp: Long): VectorClock = {
def increment(fingerprint: Int, timestamp: Long): VectorClock = {
val newVersions =
if (versions exists (entry entry.nodeId == nodeId)) {
if (versions exists (entry entry.fingerprint == fingerprint)) {
// update existing node entry
versions map { entry
if (entry.nodeId == nodeId) entry.increment()
if (entry.fingerprint == fingerprint) entry.increment()
else entry
}
} else {
// 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")
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))
// FIXME implement VectorClock.merge
def merge(other: VectorClock): VectorClock = {
sys.error("Not implemented")
}
@ -58,13 +62,13 @@ object VectorClock {
sealed trait Ordering
case object Before extends Ordering
case object After extends Ordering
case object Concurrently extends Ordering
case object Concurrent extends Ordering
/**
* Versioned entry in a vector clock.
*/
case class Entry(nodeId: Int, version: Long = 1) {
def increment(): Entry = copy(version = version + 1)
case class Entry(fingerprint: Int, version: Long = 1L) {
def increment(): Entry = copy(version = version + 1L)
}
/**
@ -88,12 +92,12 @@ object VectorClock {
while (p1 < v1.versions.size && p2 < v2.versions.size) {
val ver1 = v1.versions(p1)
val ver2 = v2.versions(p2)
if (ver1.nodeId == ver2.nodeId) {
if (ver1.fingerprint == ver2.fingerprint) {
if (ver1.version > ver2.version) v1Bigger = true
else if (ver2.version > ver1.version) v2Bigger = true
p1 += 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
p2 += 1
} else {
@ -108,6 +112,6 @@ object VectorClock {
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) Before // This is the case where v2 is a successor clock to v1
else Concurrently // This is the case where both clocks are parallel to one another
else Concurrent // This is the case where both clocks are parallel to one another
}
}

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -24,7 +24,8 @@ class AccrualFailureDetectorSpec extends WordSpec with MustMatchers {
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 conn = new InetSocketAddress("localhost", 2552)

View file

@ -18,19 +18,19 @@ class VectorClockSpec extends WordSpec with MustMatchers {
"be able to add Entry if non-existing" in {
val clock1 = VectorClock()
clock1.versions must be(Vector())
val clock2 = clock1.incrementVersionForNode(1, System.currentTimeMillis)
val clock3 = clock2.incrementVersionForNode(2, System.currentTimeMillis)
val clock2 = clock1.increment(1, System.currentTimeMillis)
val clock3 = clock2.increment(2, System.currentTimeMillis)
clock3.versions must be(Vector(Entry(1, 1), Entry(2, 1)))
}
"be able to increment version of existing Entry" in {
val clock1 = VectorClock()
val clock2 = clock1.incrementVersionForNode(1, System.currentTimeMillis)
val clock3 = clock2.incrementVersionForNode(2, System.currentTimeMillis)
val clock4 = clock3.incrementVersionForNode(1, System.currentTimeMillis)
val clock5 = clock4.incrementVersionForNode(2, System.currentTimeMillis)
val clock6 = clock5.incrementVersionForNode(2, System.currentTimeMillis)
val clock2 = clock1.increment(1, System.currentTimeMillis)
val clock3 = clock2.increment(2, System.currentTimeMillis)
val clock4 = clock3.increment(1, System.currentTimeMillis)
val clock5 = clock4.increment(2, System.currentTimeMillis)
val clock6 = clock5.increment(2, System.currentTimeMillis)
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 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 {
val clock1_1 = VectorClock()
val clock2_1 = clock1_1.incrementVersionForNode(1, System.currentTimeMillis)
val clock3_1 = clock2_1.incrementVersionForNode(2, System.currentTimeMillis)
val clock4_1 = clock3_1.incrementVersionForNode(1, System.currentTimeMillis)
val clock2_1 = clock1_1.increment(1, System.currentTimeMillis)
val clock3_1 = clock2_1.increment(2, System.currentTimeMillis)
val clock4_1 = clock3_1.increment(1, System.currentTimeMillis)
val clock1_2 = VectorClock()
val clock2_2 = clock1_2.incrementVersionForNode(1, System.currentTimeMillis)
val clock3_2 = clock2_2.incrementVersionForNode(2, System.currentTimeMillis)
val clock4_2 = clock3_2.incrementVersionForNode(1, System.currentTimeMillis)
val clock2_2 = clock1_2.increment(1, System.currentTimeMillis)
val clock3_2 = clock2_2.increment(2, 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 {
val clock1_1 = VectorClock()
val clock2_1 = clock1_1.incrementVersionForNode(1, System.currentTimeMillis)
val clock3_1 = clock2_1.incrementVersionForNode(2, System.currentTimeMillis)
val clock4_1 = clock3_1.incrementVersionForNode(1, System.currentTimeMillis)
val clock2_1 = clock1_1.increment(1, System.currentTimeMillis)
val clock3_1 = clock2_1.increment(2, System.currentTimeMillis)
val clock4_1 = clock3_1.increment(1, System.currentTimeMillis)
val clock1_2 = VectorClock()
val clock2_2 = clock1_2.incrementVersionForNode(1, System.currentTimeMillis)
val clock3_2 = clock2_2.incrementVersionForNode(2, System.currentTimeMillis)
val clock4_2 = clock3_2.incrementVersionForNode(1, System.currentTimeMillis)
val clock5_2 = clock4_2.incrementVersionForNode(3, System.currentTimeMillis)
val clock2_2 = clock1_2.increment(1, System.currentTimeMillis)
val clock3_2 = clock2_2.increment(2, System.currentTimeMillis)
val clock4_2 = clock3_2.increment(1, System.currentTimeMillis)
val clock5_2 = clock4_2.increment(3, System.currentTimeMillis)
clock4_1.compare(clock5_2) must be(Before)
}
"Two clocks with different events should be concurrent: 1" in {
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 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 {
val clock1_3 = VectorClock()
val clock2_3 = clock1_3.incrementVersionForNode(1, System.currentTimeMillis)
val clock3_3 = clock2_3.incrementVersionForNode(2, System.currentTimeMillis)
val clock4_3 = clock3_3.incrementVersionForNode(1, System.currentTimeMillis)
val clock2_3 = clock1_3.increment(1, System.currentTimeMillis)
val clock3_3 = clock2_3.increment(2, System.currentTimeMillis)
val clock4_3 = clock3_3.increment(1, System.currentTimeMillis)
val clock1_4 = VectorClock()
val clock2_4 = clock1_4.incrementVersionForNode(1, System.currentTimeMillis)
val clock3_4 = clock2_4.incrementVersionForNode(1, System.currentTimeMillis)
val clock4_4 = clock3_4.incrementVersionForNode(3, System.currentTimeMillis)
val clock2_4 = clock1_4.increment(1, System.currentTimeMillis)
val clock3_4 = clock2_4.increment(1, 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 {
val clock1_1 = VectorClock()
val clock2_1 = clock1_1.incrementVersionForNode(2, System.currentTimeMillis)
val clock3_1 = clock2_1.incrementVersionForNode(2, System.currentTimeMillis)
val clock2_1 = clock1_1.increment(2, System.currentTimeMillis)
val clock3_1 = clock2_1.increment(2, System.currentTimeMillis)
val clock1_2 = VectorClock()
val clock2_2 = clock1_2.incrementVersionForNode(1, System.currentTimeMillis)
val clock3_2 = clock2_2.incrementVersionForNode(2, System.currentTimeMillis)
val clock4_2 = clock3_2.incrementVersionForNode(2, System.currentTimeMillis)
val clock5_2 = clock4_2.incrementVersionForNode(3, System.currentTimeMillis)
val clock2_2 = clock1_2.increment(1, System.currentTimeMillis)
val clock3_2 = clock2_2.increment(2, System.currentTimeMillis)
val clock4_2 = clock3_2.increment(2, System.currentTimeMillis)
val clock5_2 = clock4_2.increment(3, System.currentTimeMillis)
clock3_1.compare(clock5_2) must be(Before)
}
"..." in {
val clock1_1 = VectorClock()
val clock2_1 = clock1_1.incrementVersionForNode(1, System.currentTimeMillis)
val clock3_1 = clock2_1.incrementVersionForNode(2, System.currentTimeMillis)
val clock4_1 = clock3_1.incrementVersionForNode(2, System.currentTimeMillis)
val clock5_1 = clock4_1.incrementVersionForNode(3, System.currentTimeMillis)
val clock2_1 = clock1_1.increment(1, System.currentTimeMillis)
val clock3_1 = clock2_1.increment(2, System.currentTimeMillis)
val clock4_1 = clock3_1.increment(2, System.currentTimeMillis)
val clock5_1 = clock4_1.increment(3, System.currentTimeMillis)
val clock1_2 = VectorClock()
val clock2_2 = clock1_2.incrementVersionForNode(2, System.currentTimeMillis)
val clock3_2 = clock2_2.incrementVersionForNode(2, System.currentTimeMillis)
val clock2_2 = clock1_2.increment(2, System.currentTimeMillis)
val clock3_2 = clock2_2.increment(2, System.currentTimeMillis)
clock5_1.compare(clock3_2) must be(After)
}

View file

@ -128,7 +128,7 @@ class MyJavaSerializableActor extends Actor with scala.Serializable {
def receive = {
case "hello"
count = count + 1
reply("world " + count)
channel ! "world " + count
}
}
@ -136,7 +136,7 @@ class MyStatelessActorWithMessagesInMailbox extends Actor with scala.Serializabl
def receive = {
case "hello"
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 = {
case m: Message
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 = {
case p: Person
Thread.sleep(500)
case "hello-reply" reply("hello")
case "hello-reply" channel ! "hello"
}
}

View file

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

View file

@ -12,7 +12,7 @@ class RemoteActor1 extends Actor with Consumer {
def endpointUri = "jetty:http://localhost:6644/camel/remote-actor-1"
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"
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 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 = {
case msg: Message msg.bodyAs[String] match {
case "stop" {
reply("Consumer4 stopped")
channel ! "Consumer4 stopped"
self.stop
}
case body reply(body)
case body channel ! body
}
}
}
@ -76,7 +76,7 @@ class Consumer5 extends Actor with Consumer {
def receive = {
case _ {
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 = {
case msg: Message {
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 {
protected def receive = {
case msg: Message reply(msg.transformBody { body: String body replaceAll ("Akka ", "AKKA ") })
case msg: Failure reply(msg)
case msg: Message channel ! (msg.transformBody { body: String body replaceAll ("Akka ", "AKKA ") })
case msg: Failure channel ! msg
}
}
@ -150,11 +150,11 @@ class FileConsumer extends Actor with Consumer {
case msg: Message {
if (counter == 2) {
println("received %s" format msg.bodyAs[String])
reply(Ack)
channel ! Ack
} else {
println("rejected %s" format msg.bodyAs[String])
counter += 1
reply(Failure(new Exception("message number %s not accepted" format counter)))
channel ! Failure(new Exception("message number %s not accepted" format counter))
}
}
}

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -150,20 +150,20 @@ class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling
}
}
protected[akka] override def dispatch(handle: Envelope) {
val mbox = getMailbox(handle.receiver)
protected[akka] override def dispatch(receiver: ActorCell, handle: Envelope) {
val mbox = getMailbox(receiver)
val queue = mbox.queue
val execute = mbox.suspendSwitch.fold {
queue.push(handle)
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
} {
queue.push(handle)
if (queue.isActive) {
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
} else {
@ -200,10 +200,10 @@ class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling
}
if (handle ne null) {
try {
handle.invoke
mbox.actor.invoke(handle)
if (warnings) handle.channel match {
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 _
}
true

View file

@ -224,14 +224,6 @@ class TestKit(_app: AkkaApplication) {
*/
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.
*/
@ -271,6 +263,27 @@ class TestKit(_app: AkkaApplication) {
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.
*/
@ -462,12 +475,13 @@ class TestKit(_app: AkkaApplication) {
* assert(series == (1 to 7).toList)
* </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)
var msg: Message = NullMessage
@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)
lastMessage match {
case NullMessage
@ -475,7 +489,7 @@ class TestKit(_app: AkkaApplication) {
acc.reverse
case RealMessage(o, _) if (f isDefinedAt o)
msg = lastMessage
doit(f(o) :: acc)
doit(f(o) :: acc, count + 1)
case RealMessage(o, _)
queue.offerFirst(lastMessage)
lastMessage = msg
@ -483,7 +497,7 @@ class TestKit(_app: AkkaApplication) {
}
}
val ret = doit(Nil)
val ret = doit(Nil, 0)
lastWasNoMsg = true
ret
}

View file

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

View file

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

View file

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

View file

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

View file

@ -80,7 +80,7 @@ public class Pi {
public void onReceive(Object message) {
if (message instanceof Work) {
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 + "]");
}
}

View file

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

View file

@ -25,7 +25,7 @@ object AkkaBuild extends Build {
Unidoc.unidocExclude := Seq(samples.id, tutorials.id),
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)
)
@ -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(
id = "akka-samples",
base = file("akka-samples"),