Merge remote branch 'origin/master' into logging

BalancingDispatcherModelSpec is currently broken, leaves count==1 in
wavesOfActors test, committed anyway in order to check out where it
broke before this merge
This commit is contained in:
Roland 2011-11-09 14:56:05 +01:00
commit a747ef7856
190 changed files with 3076 additions and 9153 deletions

View file

@ -25,9 +25,9 @@ public class JavaAPI {
assertNotNull(ref); assertNotNull(ref);
} }
@Test void mustAcceptSingleArgTryTell() { @Test void mustAcceptSingleArgTell() {
ActorRef ref = app.actorOf(JavaAPITestActor.class); ActorRef ref = app.actorOf(JavaAPITestActor.class);
ref.tryTell("hallo"); ref.tell("hallo");
ref.tryTell("hallo", ref); ref.tell("hallo", ref);
} }
} }

View file

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

View file

@ -18,7 +18,7 @@ import scala.Some;
import scala.Right; import scala.Right;
public class JavaFutureTests { public class JavaFutureTests {
private final AkkaApplication app = new AkkaApplication(); private final AkkaApplication app = new AkkaApplication();
private final Timeout t = app.AkkaConfig().ActorTimeout(); private final Timeout t = app.AkkaConfig().ActorTimeout();
private final FutureFactory ff = new FutureFactory(app.dispatcher(), t); private final FutureFactory ff = new FutureFactory(app.dispatcher(), t);

View file

@ -14,9 +14,9 @@ object ActorFireForgetRequestReplySpec {
class ReplyActor extends Actor { class ReplyActor extends Actor {
def receive = { def receive = {
case "Send" case "Send"
channel ! "Reply" sender ! "Reply"
case "SendImplicit" case "SendImplicit"
channel ! "ReplyImplicit" sender ! "ReplyImplicit"
} }
} }

View file

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

View file

@ -17,22 +17,22 @@ import java.util.concurrent.{ CountDownLatch, TimeUnit }
object ActorRefSpec { object ActorRefSpec {
case class ReplyTo(channel: Channel[Any]) case class ReplyTo(sender: ActorRef)
class ReplyActor extends Actor { class ReplyActor extends Actor {
var replyTo: Channel[Any] = null var replyTo: ActorRef = null
def receive = { def receive = {
case "complexRequest" { case "complexRequest" {
replyTo = channel replyTo = sender
val worker = context.actorOf(Props[WorkerActor]) val worker = context.actorOf(Props[WorkerActor])
worker ! "work" worker ! "work"
} }
case "complexRequest2" case "complexRequest2"
val worker = context.actorOf(Props[WorkerActor]) val worker = context.actorOf(Props[WorkerActor])
worker ! ReplyTo(channel) worker ! ReplyTo(sender)
case "workDone" replyTo ! "complexReply" case "workDone" replyTo ! "complexReply"
case "simpleRequest" channel ! "simpleReply" case "simpleRequest" sender ! "simpleReply"
} }
} }
@ -40,7 +40,7 @@ object ActorRefSpec {
def receive = { def receive = {
case "work" { case "work" {
work work
channel ! "workDone" sender ! "workDone"
self.stop() self.stop()
} }
case ReplyTo(replyTo) { case ReplyTo(replyTo) {
@ -71,7 +71,7 @@ object ActorRefSpec {
class OuterActor(val inner: ActorRef) extends Actor { class OuterActor(val inner: ActorRef) extends Actor {
def receive = { def receive = {
case "self" channel ! self case "self" sender ! self
case x inner forward x case x inner forward x
} }
} }
@ -80,7 +80,7 @@ object ActorRefSpec {
val fail = new InnerActor val fail = new InnerActor
def receive = { def receive = {
case "self" channel ! self case "self" sender ! self
case x inner forward x case x inner forward x
} }
} }
@ -91,8 +91,8 @@ object ActorRefSpec {
class InnerActor extends Actor { class InnerActor extends Actor {
def receive = { def receive = {
case "innerself" channel ! self case "innerself" sender ! self
case other channel ! other case other sender ! other
} }
} }
@ -100,8 +100,8 @@ object ActorRefSpec {
val fail = new InnerActor val fail = new InnerActor
def receive = { def receive = {
case "innerself" channel ! self case "innerself" sender ! self
case other channel ! other case other sender ! other
} }
} }
@ -289,11 +289,7 @@ class ActorRefSpec extends AkkaSpec {
val inetAddress = app.defaultAddress val inetAddress = app.defaultAddress
val expectedSerializedRepresentation = SerializedActorRef( val expectedSerializedRepresentation = new SerializedActorRef(a.address, inetAddress)
a.uuid,
a.address,
inetAddress.getAddress.getHostAddress,
inetAddress.getPort)
import java.io._ import java.io._
@ -319,7 +315,7 @@ class ActorRefSpec extends AkkaSpec {
"support nested actorOfs" in { "support nested actorOfs" in {
val a = actorOf(new Actor { val a = actorOf(new Actor {
val nested = actorOf(new Actor { def receive = { case _ } }) val nested = actorOf(new Actor { def receive = { case _ } })
def receive = { case _ channel ! nested } def receive = { case _ sender ! nested }
}) })
val nested = (a ? "any").as[ActorRef].get val nested = (a ? "any").as[ActorRef].get
@ -339,7 +335,7 @@ class ActorRefSpec extends AkkaSpec {
(a ? "msg").as[String] must be === Some("msg") (a ? "msg").as[String] must be === Some("msg")
} }
"support reply via channel" in { "support reply via sender" in {
val latch = new TestLatch(4) val latch = new TestLatch(4)
val serverRef = actorOf(Props[ReplyActor]) val serverRef = actorOf(Props[ReplyActor])
val clientRef = actorOf(Props(new SenderActor(serverRef, latch))) val clientRef = actorOf(Props(new SenderActor(serverRef, latch)))
@ -368,23 +364,19 @@ class ActorRefSpec extends AkkaSpec {
val timeout = Timeout(20000) val timeout = Timeout(20000)
val ref = actorOf(Props(new Actor { val ref = actorOf(Props(new Actor {
def receive = { def receive = {
case 5 channel.tryTell("five") case 5 sender.tell("five")
case null channel.tryTell("null") case null sender.tell("null")
} }
})) }))
val ffive = (ref ? (5, timeout)).mapTo[String] val ffive = (ref ? (5, timeout)).mapTo[String]
val fnull = (ref ? (null, timeout)).mapTo[String] val fnull = (ref ? (null, timeout)).mapTo[String]
ref ! PoisonPill
intercept[ActorKilledException] {
(ref ? PoisonPill).get
fail("shouldn't get here")
}
ffive.get must be("five") ffive.get must be("five")
fnull.get must be("null") fnull.get must be("null")
awaitCond(ref.isShutdown, 100 millis) awaitCond(ref.isShutdown, 2000 millis)
} }
"restart when Kill:ed" in { "restart when Kill:ed" in {

View file

@ -101,7 +101,7 @@ object Chameneos {
} }
} else { } else {
waitingChameneo.foreach(_ ! Exit) waitingChameneo.foreach(_ ! Exit)
channel ! Exit sender ! Exit
} }
} }
} }

View file

@ -1,49 +0,0 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
import akka.dispatch._
import akka.testkit.TestActorRef
import akka.testkit.AkkaSpec
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ChannelSpec extends AkkaSpec {
"A Channel" must {
"be contravariant" in {
val ap = new ActorPromise(1000)
val p: Promise[Any] = ap
val c: Channel[Any] = ap
val cs: Channel[String] = c
}
"find implicit sender actors" in {
var s: (String, UntypedChannel) = null
val ch = new Channel[String] {
def !(msg: String)(implicit sender: UntypedChannel) = { s = (msg, sender) }
}
val a = TestActorRef(new Actor {
def receive = {
case str: String ch ! str
}
})
a ! "hallo"
s must be(("hallo", a))
{
implicit val actor = a
ch tryTell "buh"
}
s must be(("buh", a))
ch.!("world")(a)
s must be(("world", a))
ch.tryTell("bippy")(a)
s must be(("bippy", a))
}
}
}

View file

@ -82,7 +82,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
"notify with a Terminated message once when an Actor is stopped but not when restarted" in { "notify with a Terminated message once when an Actor is stopped but not when restarted" in {
filterException[ActorKilledException] { filterException[ActorKilledException] {
val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(2)))) val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(2))))
val terminalProps = Props(context { case x context.channel ! x }) val terminalProps = Props(context { case x context.sender ! x })
val terminal = (supervisor ? terminalProps).as[ActorRef].get val terminal = (supervisor ? terminalProps).as[ActorRef].get
val monitor = actorOf(Props(new Actor { val monitor = actorOf(Props(new Actor {

View file

@ -13,7 +13,7 @@ class DeployerSpec extends AkkaSpec {
"A Deployer" must { "A Deployer" must {
"be able to parse 'akka.actor.deployment._' config elements" in { "be able to parse 'akka.actor.deployment._' config elements" in {
val deployment = app.deployer.lookupInConfig("service-ping") val deployment = app.provider.deployer.lookupInConfig("service-ping")
deployment must be('defined) deployment must be('defined)
deployment must equal(Some( deployment must equal(Some(

View file

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

View file

@ -48,7 +48,7 @@ object IOActorSpec {
def receiveIO = { def receiveIO = {
case length: Int case length: Int
val bytes = socket.read(length) val bytes = socket.read(length)
channel ! bytes sender ! bytes
} }
} }
} }
@ -108,9 +108,9 @@ object IOActorSpec {
case msg: NewClient createWorker forward msg case msg: NewClient createWorker forward msg
case ('set, key: String, value: ByteString) case ('set, key: String, value: ByteString)
kvs += (key -> value) kvs += (key -> value)
channel.tryTell(())(self) sender.tell((), self)
case ('get, key: String) channel.tryTell(kvs.get(key))(self) case ('get, key: String) sender.tell(kvs.get(key), self)
case 'getall channel.tryTell(kvs)(self) case 'getall sender.tell(kvs, self)
} }
} }
@ -123,7 +123,7 @@ object IOActorSpec {
socket = connect(ioManager, host, port) socket = connect(ioManager, host, port)
} }
def reply(msg: Any) = channel.tryTell(msg)(self) def reply(msg: Any) = sender.tell(msg, self)
def receiveIO = { def receiveIO = {
case ('set, key: String, value: ByteString) case ('set, key: String, value: ByteString)

View file

@ -6,63 +6,21 @@ package akka.actor
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.Future
import java.util.concurrent.{ TimeUnit, CountDownLatch }
object LocalActorRefProviderSpec {
class NewActor extends Actor {
def receive = {
case _ {}
}
}
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class LocalActorRefProviderSpec extends AkkaSpec { class LocalActorRefProviderSpec extends AkkaSpec {
import akka.actor.LocalActorRefProviderSpec._
"An LocalActorRefProvider" must { "An LocalActorRefProvider" must {
"only create one instance of an actor with a specific address in a concurrent environment" in { "only create one instance of an actor with a specific address in a concurrent environment" in {
val provider = app.provider val provider = app.provider
for (i 0 until 100) { // 100 concurrent runs provider.isInstanceOf[LocalActorRefProvider] must be(true)
val latch = new CountDownLatch(4)
var a1: Option[ActorRef] = None
var a2: Option[ActorRef] = None
var a3: Option[ActorRef] = None
var a4: Option[ActorRef] = None
(0 until 100) foreach { i // 100 concurrent runs
val address = "new-actor" + i val address = "new-actor" + i
implicit val timeout = Timeout(5 seconds)
spawn { ((1 to 4) map { _ Future { provider.actorOf(Props(c { case _ }), app.guardian, address) } }).map(_.get).distinct.size must be(1)
a1 = Some(provider.actorOf(Props(creator = () new NewActor), app.guardian, address))
latch.countDown()
}
spawn {
a2 = Some(provider.actorOf(Props(creator = () new NewActor), app.guardian, address))
latch.countDown()
}
spawn {
a3 = Some(provider.actorOf(Props(creator = () new NewActor), app.guardian, address))
latch.countDown()
}
spawn {
a4 = Some(provider.actorOf(Props(creator = () new NewActor), app.guardian, address))
latch.countDown()
}
latch.await(5, TimeUnit.SECONDS) must be === true
a1.isDefined must be(true)
a2.isDefined must be(true)
a3.isDefined must be(true)
a4.isDefined must be(true)
(a1 == a2) must be(true)
(a1 == a3) must be(true)
(a1 == a4) must be(true)
} }
} }
} }

View file

@ -71,7 +71,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
app.mainbus.subscribe(testActor, classOf[Logging.Error]) app.mainbus.subscribe(testActor, classOf[Logging.Error])
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = loggable(this) { def receive = loggable(this) {
case _ channel ! "x" case _ sender ! "x"
} }
}) })
actor ! "buh" actor ! "buh"
@ -99,7 +99,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
app.mainbus.subscribe(testActor, classOf[Logging.Debug]) app.mainbus.subscribe(testActor, classOf[Logging.Debug])
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = loggable(this)(loggable(this) { def receive = loggable(this)(loggable(this) {
case _ channel ! "x" case _ sender ! "x"
}) })
}) })
actor ! "buh" actor ! "buh"

View file

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

View file

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

View file

@ -13,7 +13,7 @@ object SupervisorHierarchySpec {
class CountDownActor(countDown: CountDownLatch) extends Actor { class CountDownActor(countDown: CountDownLatch) extends Actor {
protected def receive = { protected def receive = {
case p: Props channel ! context.actorOf(p) case p: Props sender ! context.actorOf(p)
} }
override def postRestart(reason: Throwable) = { override def postRestart(reason: Throwable) = {
countDown.countDown() countDown.countDown()

View file

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

View file

@ -17,6 +17,8 @@ import akka.testkit.AkkaSpec
object SupervisorSpec { object SupervisorSpec {
val Timeout = 5 seconds val Timeout = 5 seconds
case object DieReply
// ===================================================== // =====================================================
// Message logs // Message logs
// ===================================================== // =====================================================
@ -25,37 +27,38 @@ object SupervisorSpec {
val PongMessage = "pong" val PongMessage = "pong"
val ExceptionMessage = "Expected exception; to test fault-tolerance" val ExceptionMessage = "Expected exception; to test fault-tolerance"
var messageLog = new LinkedBlockingQueue[String]
def messageLogPoll = messageLog.poll(Timeout.length, Timeout.unit)
// ===================================================== // =====================================================
// Actors // Actors
// ===================================================== // =====================================================
class PingPongActor extends Actor { class PingPongActor(sendTo: ActorRef) extends Actor {
def receive = { def receive = {
case Ping case Ping
messageLog.put(PingMessage) sendTo ! PingMessage
channel.tryTell(PongMessage) if (sender != sendTo)
sender ! PongMessage
case Die case Die
throw new RuntimeException(ExceptionMessage) throw new RuntimeException(ExceptionMessage)
case DieReply
val e = new RuntimeException(ExceptionMessage)
sender ! Status.Failure(e)
throw e
} }
override def postRestart(reason: Throwable) { override def postRestart(reason: Throwable) {
messageLog.put(reason.getMessage) sendTo ! reason.getMessage
} }
} }
class Master extends Actor { class Master(sendTo: ActorRef) extends Actor {
val temp = watch(context.actorOf(Props(new PingPongActor(sendTo))))
val temp = context.actorOf(Props[PingPongActor]) var s: ActorRef = _
self startsMonitoring temp
var s: UntypedChannel = _
def receive = { def receive = {
case Die temp ! Die; s = context.channel case Die temp forward Die
case Terminated(`temp`) s ! "terminated" case Terminated(`temp`) sendTo ! "terminated"
case Status.Failure(_) /*Ignore*/
} }
} }
} }
@ -75,45 +78,45 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
def temporaryActorAllForOne = { def temporaryActorAllForOne = {
val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), Some(0)))) val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), Some(0))))
val temporaryActor = child(supervisor, Props[PingPongActor]) val temporaryActor = child(supervisor, Props(new PingPongActor(testActor)))
(temporaryActor, supervisor) (temporaryActor, supervisor)
} }
def singleActorAllForOne = { def singleActorAllForOne = {
val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))
val pingpong = child(supervisor, Props[PingPongActor]) val pingpong = child(supervisor, Props(new PingPongActor(testActor)))
(pingpong, supervisor) (pingpong, supervisor)
} }
def singleActorOneForOne = { def singleActorOneForOne = {
val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))
val pingpong = child(supervisor, Props[PingPongActor]) val pingpong = child(supervisor, Props(new PingPongActor(testActor)))
(pingpong, supervisor) (pingpong, supervisor)
} }
def multipleActorsAllForOne = { def multipleActorsAllForOne = {
val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))
val pingpong1, pingpong2, pingpong3 = child(supervisor, Props[PingPongActor]) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor)))
(pingpong1, pingpong2, pingpong3, supervisor) (pingpong1, pingpong2, pingpong3, supervisor)
} }
def multipleActorsOneForOne = { def multipleActorsOneForOne = {
val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))
val pingpong1, pingpong2, pingpong3 = child(supervisor, Props[PingPongActor]) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor)))
(pingpong1, pingpong2, pingpong3, supervisor) (pingpong1, pingpong2, pingpong3, supervisor)
} }
def nestedSupervisorsAllForOne = { def nestedSupervisorsAllForOne = {
val topSupervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val topSupervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))
val pingpong1 = child(topSupervisor, Props[PingPongActor]) val pingpong1 = child(topSupervisor, Props(new PingPongActor(testActor)))
val middleSupervisor = child(topSupervisor, Props[Supervisor].withFaultHandler(AllForOneStrategy(Nil, 3, TimeoutMillis))) val middleSupervisor = child(topSupervisor, Props[Supervisor].withFaultHandler(AllForOneStrategy(Nil, 3, TimeoutMillis)))
val pingpong2, pingpong3 = child(middleSupervisor, Props[PingPongActor]) val pingpong2, pingpong3 = child(middleSupervisor, Props(new PingPongActor(testActor)))
(pingpong1, pingpong2, pingpong3, topSupervisor) (pingpong1, pingpong2, pingpong3, topSupervisor)
} }
@ -123,49 +126,46 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
} }
override def beforeEach() = { override def beforeEach() = {
messageLog.clear
} }
def ping(pingPongActor: ActorRef) = { def ping(pingPongActor: ActorRef) = {
(pingPongActor.?(Ping, TimeoutMillis)).as[String].getOrElse("nil") must be === PongMessage (pingPongActor.?(Ping, TimeoutMillis)).as[String] must be === Some(PongMessage)
messageLogPoll must be === PingMessage expectMsg(Timeout, PingMessage)
} }
def kill(pingPongActor: ActorRef) = { def kill(pingPongActor: ActorRef) = {
intercept[RuntimeException] { (pingPongActor ? (Die, TimeoutMillis)).as[Any] } val result = (pingPongActor ? (DieReply, TimeoutMillis))
messageLogPoll must be === ExceptionMessage expectMsg(Timeout, ExceptionMessage)
intercept[RuntimeException] { result.get }
} }
"A supervisor" must { "A supervisor" must {
"not restart programmatically linked temporary actor" in { "not restart child more times than permitted" in {
val master = actorOf(Props[Master].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) val master = actorOf(Props(new Master(testActor)).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0))))
master ! Die master ! Die
expectMsg(3 seconds, "terminated") expectMsg(3 seconds, "terminated")
expectNoMsg(1 second)
1.second.dilated.sleep
messageLogPoll must be(null)
} }
"not restart temporary actor" in { "not restart temporary actor" in {
val (temporaryActor, supervisor) = temporaryActorAllForOne val (temporaryActor, _) = temporaryActorAllForOne
intercept[RuntimeException] { intercept[RuntimeException] { (temporaryActor.?(DieReply, TimeoutMillis)).get }
(temporaryActor.?(Die, TimeoutMillis)).get
}
1.second.dilated.sleep expectNoMsg(1 second)
messageLog.size must be(0)
} }
"start server for nested supervisor hierarchy" in { "start server for nested supervisor hierarchy" in {
val (actor1, actor2, actor3, supervisor) = nestedSupervisorsAllForOne val (actor1, _, _, _) = nestedSupervisorsAllForOne
ping(actor1) ping(actor1)
expectNoMsg(1 second)
} }
"kill single actor OneForOne" in { "kill single actor OneForOne" in {
val (actor, supervisor) = singleActorOneForOne val (actor, _) = singleActorOneForOne
kill(actor) kill(actor)
} }
@ -218,8 +218,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
kill(actor2) kill(actor2)
// and two more exception messages // and two more exception messages
messageLogPoll must be(ExceptionMessage) expectMsg(Timeout, ExceptionMessage)
messageLogPoll must be(ExceptionMessage) expectMsg(Timeout, ExceptionMessage)
} }
"call-kill-call multiple actors AllForOne" in { "call-kill-call multiple actors AllForOne" in {
@ -232,8 +232,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
kill(actor2) kill(actor2)
// and two more exception messages // and two more exception messages
messageLogPoll must be(ExceptionMessage) expectMsg(Timeout, ExceptionMessage)
messageLogPoll must be(ExceptionMessage) expectMsg(Timeout, ExceptionMessage)
ping(actor1) ping(actor1)
ping(actor2) ping(actor2)
@ -241,27 +241,26 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
} }
"one-way kill single actor OneForOne" in { "one-way kill single actor OneForOne" in {
val (actor, supervisor) = singleActorOneForOne val (actor, _) = singleActorOneForOne
actor ! Die actor ! Die
messageLogPoll must be(ExceptionMessage) expectMsg(Timeout, ExceptionMessage)
} }
"one-way call-kill-call single actor OneForOne" in { "one-way call-kill-call single actor OneForOne" in {
val (actor, supervisor) = singleActorOneForOne val (actor, _) = singleActorOneForOne
actor ! Ping actor ! Ping
messageLogPoll must be(PingMessage)
actor ! Die actor ! Die
messageLogPoll must be(ExceptionMessage)
actor ! Ping actor ! Ping
messageLogPoll must be(PingMessage)
expectMsg(Timeout, PingMessage)
expectMsg(Timeout, ExceptionMessage)
expectMsg(Timeout, PingMessage)
} }
"restart killed actors in nested superviser hierarchy" in { "restart killed actors in nested superviser hierarchy" in {
val (actor1, actor2, actor3, supervisor) = nestedSupervisorsAllForOne val (actor1, actor2, actor3, _) = nestedSupervisorsAllForOne
ping(actor1) ping(actor1)
ping(actor2) ping(actor2)
@ -270,8 +269,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
kill(actor2) kill(actor2)
// and two more exception messages // and two more exception messages
messageLogPoll must be(ExceptionMessage) expectMsg(Timeout, ExceptionMessage)
messageLogPoll must be(ExceptionMessage) expectMsg(Timeout, ExceptionMessage)
ping(actor1) ping(actor1)
ping(actor2) ping(actor2)
@ -288,20 +287,20 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!") if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!")
def receive = { def receive = {
case Ping channel.tryTell(PongMessage) case Ping sender ! PongMessage
case Die throw new RuntimeException("Expected") case DieReply
val e = new RuntimeException("Expected")
sender ! Status.Failure(e)
throw e
} }
}) })
val dyingActor = (supervisor ? dyingProps).as[ActorRef].get val dyingActor = (supervisor ? dyingProps).as[ActorRef].get
intercept[RuntimeException] { intercept[RuntimeException] {
(dyingActor.?(Die, TimeoutMillis)).get (dyingActor.?(DieReply, TimeoutMillis)).get
} }
// give time for restart (dyingActor.?(Ping, TimeoutMillis)).as[String] must be === Some(PongMessage)
3.seconds.dilated.sleep
(dyingActor.?(Ping, TimeoutMillis)).as[String].getOrElse("nil") must be === PongMessage
inits.get must be(3) inits.get must be(3)

View file

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

View file

@ -25,7 +25,7 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender
val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 5, 10000))) val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 5, 10000)))
val supervised = (supervisor ? Props[Supervised]).as[ActorRef].get val supervised = (supervisor ? Props[Supervised]).as[ActorRef].get
supervised.!("test")(Some(testActor)) supervised.!("test")(testActor)
expectMsg("failure1") expectMsg("failure1")
supervisor.stop() supervisor.stop()
} }
@ -36,7 +36,7 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender
val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), Some(0), None))) val supervisor = actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), Some(0), None)))
val supervised = (supervisor ? Props[Supervised]).as[ActorRef].get val supervised = (supervisor ? Props[Supervised]).as[ActorRef].get
supervised.!("test")(Some(testActor)) supervised.!("test")(testActor)
expectMsg("failure2") expectMsg("failure2")
supervisor.stop() supervisor.stop()
} }
@ -51,11 +51,11 @@ object Ticket669Spec {
} }
override def preRestart(reason: scala.Throwable, msg: Option[Any]) { override def preRestart(reason: scala.Throwable, msg: Option[Any]) {
channel.tryTell("failure1") sender.tell("failure1")
} }
override def postStop() { override def postStop() {
channel.tryTell("failure2") sender.tell("failure2")
} }
} }
} }

View file

@ -263,7 +263,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to handle exceptions when calling methods" in { "be able to handle exceptions when calling methods" in {
filterEvents(EventFilter[IllegalStateException]("expected")) { filterEvents(EventFilter[IllegalStateException]("expected")) {
val boss = actorOf(Props(context { val boss = actorOf(Props(context {
case p: Props context.channel ! context.typedActorOf(classOf[Foo], classOf[Bar], p) case p: Props context.sender ! context.typedActorOf(classOf[Foo], classOf[Bar], p)
}).withFaultHandler(OneForOneStrategy { }).withFaultHandler(OneForOneStrategy {
case e: IllegalStateException if e.getMessage == "expected" FaultHandlingStrategy.Resume case e: IllegalStateException if e.getMessage == "expected" FaultHandlingStrategy.Resume
})) }))
@ -276,14 +276,10 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
t.failingFuturePigdog.await.exception.get.getMessage must be("expected") t.failingFuturePigdog.await.exception.get.getMessage must be("expected")
t.read() must be(1) //Make sure state is not reset after failure t.read() must be(1) //Make sure state is not reset after failure
(intercept[IllegalStateException] { (intercept[IllegalStateException] { t.failingJOptionPigdog }).getMessage must be("expected")
t.failingJOptionPigdog
}).getMessage must be("expected")
t.read() must be(1) //Make sure state is not reset after failure t.read() must be(1) //Make sure state is not reset after failure
(intercept[IllegalStateException] { (intercept[IllegalStateException] { t.failingOptionPigdog }).getMessage must be("expected")
t.failingOptionPigdog
}).getMessage must be("expected")
t.read() must be(1) //Make sure state is not reset after failure t.read() must be(1) //Make sure state is not reset after failure

View file

@ -72,14 +72,14 @@ object ActorModelSpec {
case Meet(sign, wait) ack; sign.countDown(); wait.await(); busy.switchOff() case Meet(sign, wait) ack; sign.countDown(); wait.await(); busy.switchOff()
case Wait(time) ack; Thread.sleep(time); busy.switchOff() case Wait(time) ack; Thread.sleep(time); busy.switchOff()
case WaitAck(time, l) ack; Thread.sleep(time); l.countDown(); busy.switchOff() case WaitAck(time, l) ack; Thread.sleep(time); l.countDown(); busy.switchOff()
case Reply(msg) ack; channel ! msg; busy.switchOff() case Reply(msg) ack; sender ! msg; busy.switchOff()
case TryReply(msg) ack; channel.tryTell(msg); busy.switchOff() case TryReply(msg) ack; sender.tell(msg); busy.switchOff()
case Forward(to, msg) ack; to.forward(msg); busy.switchOff() case Forward(to, msg) ack; to.forward(msg); busy.switchOff()
case CountDown(latch) ack; latch.countDown(); busy.switchOff() case CountDown(latch) ack; latch.countDown(); busy.switchOff()
case Increment(count) ack; count.incrementAndGet(); busy.switchOff() case Increment(count) ack; count.incrementAndGet(); busy.switchOff()
case CountDownNStop(l) ack; l.countDown(); self.stop(); busy.switchOff() case CountDownNStop(l) ack; l.countDown(); self.stop(); busy.switchOff()
case Restart ack; busy.switchOff(); throw new Exception("Restart requested") case Restart ack; busy.switchOff(); throw new Exception("Restart requested")
case Interrupt ack; busy.switchOff(); throw new InterruptedException("Ping!") case Interrupt ack; sender ! Status.Failure(new ActorInterruptedException(new InterruptedException("Ping!"))); busy.switchOff(); throw new InterruptedException("Ping!")
case ThrowException(e: Throwable) ack; busy.switchOff(); throw e case ThrowException(e: Throwable) ack; busy.switchOff(); throw e
} }
} }
@ -364,6 +364,7 @@ abstract class ActorModelSpec extends AkkaSpec {
} catch { } catch {
case e case e
System.err.println("Error: " + e.getMessage + " missing count downs == " + cachedMessage.latch.getCount() + " out of " + num) System.err.println("Error: " + e.getMessage + " missing count downs == " + cachedMessage.latch.getCount() + " out of " + num)
throw e
} }
boss.stop() boss.stop()
} }
@ -380,30 +381,17 @@ abstract class ActorModelSpec extends AkkaSpec {
val a = newTestActor(dispatcher) val a = newTestActor(dispatcher)
val f1 = a ? Reply("foo") val f1 = a ? Reply("foo")
val f2 = a ? Reply("bar") val f2 = a ? Reply("bar")
val f3 = try { val f3 = try { a ? Interrupt } catch { case ie: InterruptedException new KeptPromise(Left(ActorInterruptedException(ie))) }
a ? Interrupt
} catch {
// CallingThreadDispatcher throws IE directly
case ie: InterruptedException new KeptPromise(Left(ActorInterruptedException(ie)))
}
val f4 = a ? Reply("foo2") val f4 = a ? Reply("foo2")
val f5 = try { val f5 = try { a ? Interrupt } catch { case ie: InterruptedException new KeptPromise(Left(ActorInterruptedException(ie))) }
a ? Interrupt
} catch {
case ie: InterruptedException new KeptPromise(Left(ActorInterruptedException(ie)))
}
val f6 = a ? Reply("bar2") val f6 = a ? Reply("bar2")
assert(f1.get === "foo") assert(f1.get === "foo")
assert(f2.get === "bar") assert(f2.get === "bar")
assert((intercept[ActorInterruptedException] {
f3.get
}).getMessage === "Ping!")
assert(f4.get === "foo2") assert(f4.get === "foo2")
assert((intercept[ActorInterruptedException] { assert(intercept[ActorInterruptedException](f3.get).getMessage === "Ping!")
f5.get
}).getMessage === "Ping!")
assert(f6.get === "bar2") assert(f6.get === "bar2")
assert(intercept[ActorInterruptedException](f5.get).getMessage === "Ping!")
} }
} }
@ -413,21 +401,17 @@ abstract class ActorModelSpec extends AkkaSpec {
val a = newTestActor(dispatcher) val a = newTestActor(dispatcher)
val f1 = a ? Reply("foo") val f1 = a ? Reply("foo")
val f2 = a ? Reply("bar") val f2 = a ? Reply("bar")
val f3 = a ? new ThrowException(new IndexOutOfBoundsException("IndexOutOfBoundsException")) val f3 = a ? ThrowException(new IndexOutOfBoundsException("IndexOutOfBoundsException"))
val f4 = a ? Reply("foo2") val f4 = a ? Reply("foo2")
val f5 = a ? new ThrowException(new RemoteException("RemoteException")) val f5 = a ? ThrowException(new RemoteException("RemoteException"))
val f6 = a ? Reply("bar2") val f6 = a ? Reply("bar2")
assert(f1.get === "foo") assert(f1.get === "foo")
assert(f2.get === "bar") assert(f2.get === "bar")
assert((intercept[IndexOutOfBoundsException] {
f3.get
}).getMessage === "IndexOutOfBoundsException")
assert(f4.get === "foo2") assert(f4.get === "foo2")
assert((intercept[RemoteException] {
f5.get
}).getMessage === "RemoteException")
assert(f6.get === "bar2") assert(f6.get === "bar2")
assert(f3.result === None)
assert(f5.result === None)
} }
} }
} }
@ -446,21 +430,6 @@ class DispatcherModelSpec extends ActorModelSpec {
def dispatcherType = "Dispatcher" def dispatcherType = "Dispatcher"
"A " + dispatcherType must { "A " + dispatcherType must {
"complete all uncompleted sender futures on deregister" in {
implicit val dispatcher = newInterceptedDispatcher
val a = newTestActor(dispatcher).asInstanceOf[LocalActorRef]
a.suspend
val f1: Future[String] = a ? Reply("foo") mapTo manifest[String]
val stopped = a ? PoisonPill
val shouldBeCompleted = for (i 1 to 10) yield a ? Reply(i)
a.resume
assert(f1.get == "foo")
stopped.await
for (each shouldBeCompleted)
assert(each.await.exception.get.isInstanceOf[ActorKilledException])
a.stop()
}
"process messages in parallel" in { "process messages in parallel" in {
implicit val dispatcher = newInterceptedDispatcher implicit val dispatcher = newInterceptedDispatcher
val aStart, aStop, bParallel = new CountDownLatch(1) val aStart, aStop, bParallel = new CountDownLatch(1)

View file

@ -9,7 +9,7 @@ import akka.actor.{ Props, Actor }
object DispatcherActorSpec { object DispatcherActorSpec {
class TestActor extends Actor { class TestActor extends Actor {
def receive = { def receive = {
case "Hello" channel ! "World" case "Hello" sender ! "World"
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance") case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
} }
} }
@ -46,20 +46,6 @@ class DispatcherActorSpec extends AkkaSpec {
actor.stop() actor.stop()
} }
"support ask/exception" in {
filterEvents(EventFilter[RuntimeException]("Expected")) {
val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build))
try {
(actor ? "Failure").get
fail("Should have thrown an exception")
} catch {
case e
assert("Expected exception; to test fault-tolerance" === e.getMessage())
}
actor.stop()
}
}
"respect the throughput setting" in { "respect the throughput setting" in {
val throughputDispatcher = app.dispatcherFactory. val throughputDispatcher = app.dispatcherFactory.
newDispatcher("THROUGHPUT", 101, 0, app.dispatcherFactory.MailboxType). newDispatcher("THROUGHPUT", 101, 0, app.dispatcherFactory.MailboxType).
@ -74,7 +60,7 @@ class DispatcherActorSpec extends AkkaSpec {
val slowOne = actorOf( val slowOne = actorOf(
Props(context { Props(context {
case "hogexecutor" context.channel ! "OK"; start.await case "hogexecutor" context.sender ! "OK"; start.await
case "ping" if (works.get) latch.countDown() case "ping" if (works.get) latch.countDown()
}).withDispatcher(throughputDispatcher)) }).withDispatcher(throughputDispatcher))

View file

@ -11,7 +11,7 @@ import org.scalatest.BeforeAndAfterEach
object PinnedActorSpec { object PinnedActorSpec {
class TestActor extends Actor { class TestActor extends Actor {
def receive = { def receive = {
case "Hello" channel ! "World" case "Hello" sender ! "World"
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance") case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
} }
} }
@ -39,15 +39,5 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach {
assert("World" === result.get) assert("World" === result.get)
actor.stop() actor.stop()
} }
"support ask/exception" in {
val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test")))
filterException[RuntimeException] {
intercept[RuntimeException] {
(actor ? "Failure").get
}.getMessage must be("Expected exception; to test fault-tolerance")
}
actor.stop()
}
} }
} }

View file

@ -18,11 +18,11 @@ class Future2ActorSpec extends AkkaSpec {
expectMsgAllOf(1 second, 42, 42) expectMsgAllOf(1 second, 42, 42)
} }
"support reply via channel" in { "support reply via sender" in {
val actor = app.actorOf(Props(new Actor { val actor = app.actorOf(Props(new Actor {
def receive = { def receive = {
case "do" Future(31) pipeTo context.channel case "do" Future(31) pipeTo context.sender
case "ex" Future(throw new AssertionError) pipeTo context.channel case "ex" Future(throw new AssertionError) pipeTo context.sender
} }
})) }))
(actor ? "do").as[Int] must be(Some(31)) (actor ? "do").as[Int] must be(Some(31))

View file

@ -6,30 +6,32 @@ import org.scalacheck._
import org.scalacheck.Arbitrary._ import org.scalacheck.Arbitrary._
import org.scalacheck.Prop._ import org.scalacheck.Prop._
import org.scalacheck.Gen._ import org.scalacheck.Gen._
import akka.actor.{ Actor, ActorRef, Timeout } import akka.actor.{ Actor, ActorRef, Status }
import akka.testkit.{ EventFilter, filterEvents, filterException } import akka.testkit.{ EventFilter, filterEvents, filterException }
import akka.util.duration._ import akka.util.duration._
import org.multiverse.api.latches.StandardLatch import org.multiverse.api.latches.StandardLatch
import java.util.concurrent.{ TimeUnit, CountDownLatch } import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import java.lang.ArithmeticException
object FutureSpec { object FutureSpec {
class TestActor extends Actor { class TestActor extends Actor {
def receive = { def receive = {
case "Hello" channel ! "World" case "Hello" sender ! "World"
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance") case "Failure"
sender ! Status.Failure(new RuntimeException("Expected exception; to test fault-tolerance"))
case "NoReply" case "NoReply"
} }
} }
class TestDelayActor(await: StandardLatch) extends Actor { class TestDelayActor(await: StandardLatch) extends Actor {
def receive = { def receive = {
case "Hello" await.await; channel ! "World" case "Hello" await.await; sender ! "World"
case "NoReply" await.await case "NoReply" await.await
case "Failure" case "Failure"
await.await await.await
throw new RuntimeException("Expected exception; to test fault-tolerance") sender ! Status.Failure(new RuntimeException("Expected exception; to test fault-tolerance"))
} }
} }
} }
@ -137,7 +139,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"will return a result" must { "will return a result" must {
behave like futureWithResult { test behave like futureWithResult { test
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String channel ! s.toUpperCase } }) val actor2 = actorOf(new Actor { def receive = { case s: String sender ! s.toUpperCase } })
val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s } val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s }
future.await future.await
test(future, "WORLD") test(future, "WORLD")
@ -149,7 +151,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
behave like futureWithException[ArithmeticException] { test behave like futureWithException[ArithmeticException] { test
filterException[ArithmeticException] { filterException[ArithmeticException] {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String channel ! s.length / 0 } }) val actor2 = actorOf(new Actor { def receive = { case s: String sender ! Status.Failure(new ArithmeticException("/ by zero")) } })
val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s } val future = actor1 ? "Hello" flatMap { case s: String actor2 ? s }
future.await future.await
test(future, "/ by zero") test(future, "/ by zero")
@ -162,7 +164,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
behave like futureWithException[MatchError] { test behave like futureWithException[MatchError] { test
filterException[MatchError] { filterException[MatchError] {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String channel ! s.toUpperCase } }) val actor2 = actorOf(new Actor { def receive = { case s: String sender ! s.toUpperCase } })
val future = actor1 ? "Hello" flatMap { case i: Int actor2 ? i } val future = actor1 ? "Hello" flatMap { case i: Int actor2 ? i }
future.await future.await
test(future, "World (of class java.lang.String)") test(future, "World (of class java.lang.String)")
@ -179,8 +181,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
filterException[ClassCastException] { filterException[ClassCastException] {
val actor = actorOf(new Actor { val actor = actorOf(new Actor {
def receive = { def receive = {
case s: String channel ! s.length case s: String sender ! s.length
case i: Int channel ! (i * 2).toString case i: Int sender ! (i * 2).toString
} }
}) })
@ -211,8 +213,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
case class Res[T](res: T) case class Res[T](res: T)
val actor = actorOf(new Actor { val actor = actorOf(new Actor {
def receive = { def receive = {
case Req(s: String) channel ! Res(s.length) case Req(s: String) sender ! Res(s.length)
case Req(i: Int) channel ! Res((i * 2).toString) case Req(i: Int) sender ! Res((i * 2).toString)
} }
}) })
@ -298,7 +300,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"fold" in { "fold" in {
val actors = (1 to 10).toList map { _ val actors = (1 to 10).toList map { _
actorOf(new Actor { actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); channel.tryTell(add) } def receive = { case (add: Int, wait: Int) Thread.sleep(wait); sender.tell(add) }
}) })
} }
val timeout = 10000 val timeout = 10000
@ -309,7 +311,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"fold by composing" in { "fold by composing" in {
val actors = (1 to 10).toList map { _ val actors = (1 to 10).toList map { _
actorOf(new Actor { actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); channel.tryTell(add) } def receive = { case (add: Int, wait: Int) Thread.sleep(wait); sender.tell(add) }
}) })
} }
def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 200), 10000).mapTo[Int] } def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) actor.?((idx, idx * 200), 10000).mapTo[Int] }
@ -323,8 +325,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
def receive = { def receive = {
case (add: Int, wait: Int) case (add: Int, wait: Int)
Thread.sleep(wait) Thread.sleep(wait)
if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected") if (add == 6) sender ! Status.Failure(new IllegalArgumentException("shouldFoldResultsWithException: expected"))
channel.tryTell(add) else sender.tell(add)
} }
}) })
} }
@ -356,7 +358,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"shouldReduceResults" in { "shouldReduceResults" in {
val actors = (1 to 10).toList map { _ val actors = (1 to 10).toList map { _
actorOf(new Actor { actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); channel.tryTell(add) } def receive = { case (add: Int, wait: Int) Thread.sleep(wait); sender.tell(add) }
}) })
} }
val timeout = 10000 val timeout = 10000
@ -371,8 +373,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
def receive = { def receive = {
case (add: Int, wait: Int) case (add: Int, wait: Int)
Thread.sleep(wait) Thread.sleep(wait)
if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected") if (add == 6) sender ! Status.Failure(new IllegalArgumentException("shouldFoldResultsWithException: expected"))
channel.tryTell(add) else sender.tell(add)
} }
}) })
} }
@ -401,7 +403,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
var counter = 1 var counter = 1
def receive = { def receive = {
case 'GetNext case 'GetNext
channel ! counter sender ! counter
counter += 2 counter += 2
} }
}) })
@ -831,6 +833,22 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
f4.await must be('completed) f4.await must be('completed)
} }
"should not deadlock with nested await (ticket 1313)" in {
val simple = Future() map (_ (Future(()) map (_ ())).get)
simple.await must be('completed)
val l1, l2 = new StandardLatch
val complex = Future() map { _
Future.blocking()
val nested = Future()
nested foreach (_ l1.open)
l1.await // make sure nested is completed
nested foreach (_ l2.open)
l2.await
}
assert(complex.await.isCompleted)
}
} }
} }

View file

@ -8,7 +8,7 @@ import java.util.concurrent.{ TimeUnit, CountDownLatch, BlockingQueue }
import java.util.{ Queue } import java.util.{ Queue }
import akka.util._ import akka.util._
import akka.util.Duration._ import akka.util.Duration._
import akka.actor.{ LocalActorRef, Actor, NullChannel } import akka.actor.{ LocalActorRef, Actor }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
@ -80,7 +80,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
result result
} }
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, NullChannel) def createMessageInvocation(msg: Any): Envelope = Envelope(msg, app.deadLetters)
def ensureInitialMailboxState(config: MailboxType, q: Mailbox) { def ensureInitialMailboxState(config: MailboxType, q: Mailbox) {
q must not be null q must not be null

View file

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

View file

@ -0,0 +1,140 @@
package akka.performance.microbench
import akka.performance.workbench.PerformanceSpec
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import org.junit.runner.RunWith
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import java.util.Random
import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistics
// -server -Xms512M -Xmx1024M -XX:+UseConcMarkSweepGC -Dbenchmark=true -Dbenchmark.repeatFactor=500
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TellLatencyPerformanceSpec extends PerformanceSpec {
import TellLatencyPerformanceSpec._
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(8)
.build
val repeat = 200L * repeatFactor
def clientDelayMicros = {
System.getProperty("benchmark.clientDelayMicros", "250").toInt
}
var stat: DescriptiveStatistics = _
override def beforeEach() {
stat = new SynchronizedDescriptiveStatistics
}
"Tell" must {
"warmup" in {
runScenario(2, warmup = true)
}
"warmup more" in {
runScenario(4, warmup = true)
}
"perform with load 1" in {
runScenario(1)
}
"perform with load 2" in {
runScenario(2)
}
"perform with load 4" in {
runScenario(4)
}
"perform with load 6" in {
runScenario(6)
}
"perform with load 8" in {
runScenario(8)
}
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
if (acceptClients(numberOfClients)) {
val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients
val clients = (for (i 0 until numberOfClients) yield {
val destination = app.actorOf[Destination]
val w4 = app.actorOf(new Waypoint(destination))
val w3 = app.actorOf(new Waypoint(w4))
val w2 = app.actorOf(new Waypoint(w3))
val w1 = app.actorOf(new Waypoint(w2))
Props(new Client(w1, latch, repeatsPerClient, clientDelayMicros, stat)).withDispatcher(clientDispatcher)
}).toList.map(app.actorOf(_))
val start = System.nanoTime
clients.foreach(_ ! Run)
val ok = latch.await((5000000 + 500 * repeat) * timeDilation, TimeUnit.MICROSECONDS)
val durationNs = (System.nanoTime - start)
if (!warmup) {
ok must be(true)
logMeasurement(numberOfClients, durationNs, stat)
}
clients.foreach(_ ! PoisonPill)
}
}
}
}
object TellLatencyPerformanceSpec {
val random: Random = new Random(0)
case object Run
case class Msg(nanoTime: Long = System.nanoTime)
class Waypoint(next: ActorRef) extends Actor {
def receive = {
case msg: Msg next forward msg
}
}
class Destination extends Actor {
def receive = {
case msg: Msg sender ! msg
}
}
class Client(
actor: ActorRef,
latch: CountDownLatch,
repeat: Long,
delayMicros: Int,
stat: DescriptiveStatistics) extends Actor {
var sent = 0L
var received = 0L
def receive = {
case Msg(sendTime)
val duration = System.nanoTime - sendTime
stat.addValue(duration)
received += 1
if (sent < repeat) {
PerformanceSpec.shortDelay(delayMicros, received)
actor ! Msg()
sent += 1
} else if (received >= repeat) {
latch.countDown()
}
case Run
// random initial delay to spread requests
val initialDelay = random.nextInt(20)
Thread.sleep(initialDelay)
actor ! Msg()
sent += 1
}
}
}

View file

@ -0,0 +1,132 @@
package akka.performance.microbench
import akka.performance.workbench.PerformanceSpec
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import org.junit.runner.RunWith
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import akka.dispatch.Dispatchers
import akka.dispatch.Dispatcher
import akka.dispatch.Dispatchers
// -server -Xms512M -Xmx1024M -XX:+UseParallelGC -Dbenchmark=true -Dbenchmark.repeatFactor=500
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TellThroughputPerformanceSpec extends PerformanceSpec {
import TellThroughputPerformanceSpec._
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.build
val destinationDispatcher = app.dispatcherFactory.newDispatcher("destination-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.build
val repeat = 30000L * repeatFactor
"Tell" must {
"warmup" in {
runScenario(4, warmup = true)
}
"warmup more" in {
runScenario(4, warmup = true)
}
"perform with load 1" in {
runScenario(1)
}
"perform with load 2" in {
runScenario(2)
}
"perform with load 4" in {
runScenario(4)
}
"perform with load 6" in {
runScenario(6)
}
"perform with load 8" in {
runScenario(8)
}
"perform with load 10" in {
runScenario(10)
}
"perform with load 12" in {
runScenario(12)
}
"perform with load 14" in {
runScenario(14)
}
"perform with load 16" in {
runScenario(16)
}
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
if (acceptClients(numberOfClients)) {
val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients
val destinations = for (i 0 until numberOfClients)
yield app.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
val clients = for (dest destinations)
yield app.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
val start = System.nanoTime
clients.foreach(_ ! Run)
val ok = latch.await((5000000 + 500 * repeat) * timeDilation, TimeUnit.MICROSECONDS)
val durationNs = (System.nanoTime - start)
if (!warmup) {
ok must be(true)
logMeasurement(numberOfClients, durationNs, repeat)
}
clients.foreach(_ ! PoisonPill)
destinations.foreach(_ ! PoisonPill)
}
}
}
}
object TellThroughputPerformanceSpec {
case object Run
case object Msg
class Destination extends Actor {
def receive = {
case Msg sender ! Msg
}
}
class Client(
actor: ActorRef,
latch: CountDownLatch,
repeat: Long) extends Actor {
var sent = 0L
var received = 0L
def receive = {
case Msg
received += 1
if (sent < repeat) {
actor ! Msg
sent += 1
} else if (received >= repeat) {
latch.countDown()
}
case Run
for (i 0L until math.min(1000L, repeat)) {
actor ! Msg
sent += 1
}
}
}
}

View file

@ -1,85 +0,0 @@
package akka.performance.trading.common
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import akka.performance.trading.domain._
import akka.performance.trading.common._
import akka.actor.{ Props, ActorRef, Actor, PoisonPill }
import akka.AkkaApplication
import akka.event.Logging
abstract class AkkaPerformanceTest(val app: AkkaApplication) extends BenchmarkScenarios {
val log = Logging(app, this)
type TS = AkkaTradingSystem
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.setMaxPoolSize(maxClients)
.build
override def createTradingSystem: TS = new AkkaTradingSystem(app)
/**
* Implemented in subclass
*/
def placeOrder(orderReceiver: ActorRef, order: Order): Rsp
override def runScenario(scenario: String, orders: List[Order], repeat: Int, numberOfClients: Int, delayMs: Int) = {
val totalNumberOfRequests = orders.size * repeat
val repeatsPerClient = repeat / numberOfClients
val oddRepeats = repeat - (repeatsPerClient * numberOfClients)
val latch = new CountDownLatch(numberOfClients)
val receivers = tradingSystem.orderReceivers.toIndexedSeq
val start = System.nanoTime
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
Props(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), sampling, delayMs)).withDispatcher(clientDispatcher)
}).toList.map(app.actorOf(_))
clients.foreach(_ ! "run")
val ok = latch.await((5000 + (2 + delayMs) * totalNumberOfRequests) * timeDilation, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
assert(ok)
assert((orders.size / 2) * repeat == TotalTradeCounter.counter.get)
logMeasurement(scenario, numberOfClients, durationNs)
clients.foreach(_ ! PoisonPill)
}
class Client(
orderReceiver: ActorRef,
orders: List[Order],
latch: CountDownLatch,
repeat: Int,
sampling: Int,
delayMs: Int = 0) extends Actor {
def receive = {
case "run"
var n = 0
for (r 1 to repeat; o orders) {
n += 1
val rsp =
if (n % sampling == 0) {
val t0 = System.nanoTime
val rsp = placeOrder(orderReceiver, o)
val duration = System.nanoTime - t0
stat.addValue(duration)
rsp
} else {
placeOrder(orderReceiver, o)
}
if (!rsp.status) {
log.error("Invalid rsp")
}
delay(delayMs)
}
latch.countDown()
}
}
}

View file

@ -1,62 +0,0 @@
package akka.performance.trading.common
import org.junit._
import akka.performance.trading.domain._
trait BenchmarkScenarios extends PerformanceTest {
@Test
def complexScenario1 = complexScenario(1)
@Test
def complexScenario2 = complexScenario(2)
@Test
def complexScenario4 = complexScenario(4)
@Test
def complexScenario6 = complexScenario(6)
@Test
def complexScenario8 = complexScenario(8)
@Test
def complexScenario10 = complexScenario(10)
@Test
def complexScenario20 = complexScenario(20)
@Test
def complexScenario30 = complexScenario(30)
@Test
def complexScenario40 = complexScenario(40)
@Test
def complexScenario60 = complexScenario(60)
@Test
def complexScenario80 = complexScenario(80)
@Test
def complexScenario100 = complexScenario(100)
/*
@Test
def complexScenario200 = complexScenario(200)
@Test
def complexScenario300 = complexScenario(300)
@Test
def complexScenario400 = complexScenario(400)
*/
def complexScenario(numberOfClients: Int) {
Assume.assumeTrue(numberOfClients >= minClients)
Assume.assumeTrue(numberOfClients <= maxClients)
val repeat = 500 * repeatFactor
val prefixes = "A" :: "B" :: "C" :: Nil
val askOrders = for {
s prefixes
i 1 to 5
} yield new Ask(s + i, 100 - i, 1000)
val bidOrders = for {
s prefixes
i 1 to 5
} yield new Bid(s + i, 100 - i, 1000)
val orders = askOrders ::: bidOrders
runScenario("benchmark", orders, repeat, numberOfClients, 0)
}
}

View file

@ -1,159 +0,0 @@
package akka.performance.trading.common
import java.util.Random
import scala.collection.immutable.TreeMap
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistics
import org.junit.After
import org.junit.Before
import org.scalatest.junit.JUnitSuite
import akka.performance.trading.domain.Ask
import akka.performance.trading.domain.Bid
import akka.performance.trading.domain.Order
import akka.performance.trading.domain.TotalTradeCounter
import akka.performance.workbench.BenchResultRepository
import akka.performance.workbench.Report
import akka.performance.workbench.Stats
import akka.AkkaApplication
import akka.actor.simpleName
trait PerformanceTest extends JUnitSuite {
def app: AkkaApplication
// jvm parameters
// -server -Xms512m -Xmx1024m -XX:+UseConcMarkSweepGC
var isWarm = false
def isBenchmark() = System.getProperty("benchmark") == "true"
def minClients() = System.getProperty("benchmark.minClients", "1").toInt;
def maxClients() = System.getProperty("benchmark.maxClients", "40").toInt;
def repeatFactor() = {
val defaultRepeatFactor = if (isBenchmark) "150" else "2"
System.getProperty("benchmark.repeatFactor", defaultRepeatFactor).toInt
}
def warmupRepeatFactor() = {
val defaultRepeatFactor = if (isBenchmark) "200" else "1"
System.getProperty("benchmark.warmupRepeatFactor", defaultRepeatFactor).toInt
}
def randomSeed() = {
System.getProperty("benchmark.randomSeed", "0").toInt
}
def timeDilation() = {
System.getProperty("benchmark.timeDilation", "1").toLong
}
def sampling = {
System.getProperty("benchmark.sampling", "100").toInt
}
var stat: DescriptiveStatistics = _
val resultRepository = BenchResultRepository(app)
lazy val report = new Report(app, resultRepository, compareResultWith)
type TS <: TradingSystem
var tradingSystem: TS = _
val random: Random = new Random(randomSeed)
def createTradingSystem(): TS
def placeOrder(orderReceiver: TS#OR, order: Order): Rsp
def runScenario(scenario: String, orders: List[Order], repeat: Int, numberOfClients: Int, delayMs: Int)
@Before
def setUp() {
stat = new SynchronizedDescriptiveStatistics
tradingSystem = createTradingSystem()
tradingSystem.start()
warmUp()
TotalTradeCounter.reset()
stat = new SynchronizedDescriptiveStatistics
}
@After
def tearDown() {
tradingSystem.shutdown()
stat = null
}
def warmUp() {
val bid = new Bid("A1", 100, 1000)
val ask = new Ask("A1", 100, 1000)
val orderReceiver = tradingSystem.orderReceivers.head
val loopCount = if (isWarm) 1 else 10 * warmupRepeatFactor
for (i 1 to loopCount) {
placeOrder(orderReceiver, bid)
placeOrder(orderReceiver, ask)
}
isWarm = true
}
/**
* To compare two tests with each other you can override this method, in
* the test. For example Some("OneWayPerformanceTest")
*/
def compareResultWith: Option[String] = None
def logMeasurement(scenario: String, numberOfClients: Int, durationNs: Long) {
val name = simpleName(this)
val durationS = durationNs.toDouble / 1000000000.0
val percentiles = TreeMap[Int, Long](
5 -> (stat.getPercentile(5.0) / 1000).toLong,
25 -> (stat.getPercentile(25.0) / 1000).toLong,
50 -> (stat.getPercentile(50.0) / 1000).toLong,
75 -> (stat.getPercentile(75.0) / 1000).toLong,
95 -> (stat.getPercentile(95.0) / 1000).toLong)
val n = stat.getN * sampling
val stats = Stats(
name,
load = numberOfClients,
timestamp = TestStart.startTime,
durationNanos = durationNs,
n = n,
min = (stat.getMin / 1000).toLong,
max = (stat.getMax / 1000).toLong,
mean = (stat.getMean / 1000).toLong,
tps = (n.toDouble / durationS),
percentiles)
resultRepository.add(stats)
report.html(resultRepository.get(name))
}
def delay(delayMs: Int) {
val adjustedDelay =
if (delayMs >= 5) {
val dist = 0.2 * delayMs
(delayMs + random.nextGaussian * dist).intValue
} else {
delayMs
}
if (adjustedDelay > 0) {
Thread.sleep(adjustedDelay)
}
}
}
object TestStart {
val startTime = System.currentTimeMillis
}

View file

@ -1,3 +0,0 @@
package akka.performance.trading.common
case class Rsp(status: Boolean)

View file

@ -1,16 +0,0 @@
package akka.performance.trading.domain
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
trait LatchMessage {
val count: Int
lazy val latch: CountDownLatch = new CountDownLatch(count)
}
object LatchOrder {
def apply(order: Order) = order match {
case bid: Bid new Bid(order.orderbookSymbol, order.price, order.volume) with LatchMessage { val count = 2 }
case ask: Ask new Ask(order.orderbookSymbol, order.price, order.volume) with LatchMessage { val count = 2 }
}
}

View file

@ -4,26 +4,34 @@ trait Order {
def orderbookSymbol: String def orderbookSymbol: String
def price: Long def price: Long
def volume: Long def volume: Long
def nanoTime: Long
def withNanoTime: Order
} }
case class Bid( case class Bid(
orderbookSymbol: String, orderbookSymbol: String,
price: Long, price: Long,
volume: Long) volume: Long,
nanoTime: Long = 0L)
extends Order { extends Order {
def split(newVolume: Long) = { def split(newVolume: Long) = {
new Bid(orderbookSymbol, price, newVolume) new Bid(orderbookSymbol, price, newVolume)
} }
def withNanoTime: Bid = copy(nanoTime = System.nanoTime)
} }
case class Ask( case class Ask(
orderbookSymbol: String, orderbookSymbol: String,
price: Long, price: Long,
volume: Long) volume: Long,
nanoTime: Long = 0L)
extends Order { extends Order {
def split(newVolume: Long) = { def split(newVolume: Long) = {
new Ask(orderbookSymbol, price, newVolume) new Ask(orderbookSymbol, price, newVolume)
} }
def withNanoTime: Ask = copy(nanoTime = System.nanoTime)
} }

View file

@ -8,7 +8,9 @@ abstract trait TradeObserver {
trait SimpleTradeObserver extends TradeObserver { trait SimpleTradeObserver extends TradeObserver {
override def trade(bid: Bid, ask: Ask) { override def trade(bid: Bid, ask: Ask) {
val c = TotalTradeCounter.counter.incrementAndGet if (!Orderbook.useDummyOrderbook) {
TotalTradeCounter.counter.incrementAndGet
}
} }
} }

View file

@ -1,24 +0,0 @@
package akka.performance.trading.oneway
import akka.actor._
import akka.dispatch.MessageDispatcher
import akka.performance.trading.domain.Order
import akka.performance.trading.domain.Orderbook
import akka.performance.trading.common.AkkaMatchingEngine
class OneWayMatchingEngine(meId: String, orderbooks: List[Orderbook]) extends AkkaMatchingEngine(meId, orderbooks) {
override def handleOrder(order: Order) {
orderbooksMap.get(order.orderbookSymbol) match {
case Some(orderbook)
standby.foreach(_ ! order)
orderbook.addOrder(order)
orderbook.matchOrders()
case None
log.warning("Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol)
}
}
}

View file

@ -1,19 +0,0 @@
package akka.performance.trading.oneway
import akka.actor._
import akka.dispatch.MessageDispatcher
import akka.performance.trading.domain._
import akka.performance.trading.common.AkkaOrderReceiver
class OneWayOrderReceiver extends AkkaOrderReceiver {
override def placeOrder(order: Order) = {
val matchingEngine = matchingEngineForOrderbook.get(order.orderbookSymbol)
matchingEngine match {
case Some(m)
m ! order
case None
log.warning("Unknown orderbook: " + order.orderbookSymbol)
}
}
}

View file

@ -1,47 +0,0 @@
package akka.performance.trading.oneway
import java.util.concurrent.TimeUnit
import org.junit.Test
import akka.performance.trading.common.AkkaPerformanceTest
import akka.performance.trading.common.Rsp
import akka.performance.trading.domain._
import akka.actor.{ Props, ActorRef }
import akka.AkkaApplication
class OneWayPerformanceTest extends AkkaPerformanceTest(AkkaApplication()) {
override def createTradingSystem: TS = new OneWayTradingSystem(app) {
override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match {
case Some(d) app.actorOf(Props(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown).withDispatcher(d))
case _ app.actorOf(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown)
}
}
override def placeOrder(orderReceiver: ActorRef, order: Order): Rsp = {
val newOrder = LatchOrder(order)
orderReceiver ! newOrder
val ok = newOrder.latch.await(10, TimeUnit.SECONDS)
new Rsp(ok)
}
// need this so that junit will detect this as a test case
@Test
def dummy {}
override def compareResultWith = Some("RspPerformanceTest")
def createLatchOrder(order: Order) = order match {
case bid: Bid new Bid(order.orderbookSymbol, order.price, order.volume) with LatchMessage { val count = 2 }
case ask: Ask new Ask(order.orderbookSymbol, order.price, order.volume) with LatchMessage { val count = 2 }
}
}
trait LatchMessageCountDown extends OneWayMatchingEngine {
override def handleOrder(order: Order) {
super.handleOrder(order)
order.asInstanceOf[LatchMessage].latch.countDown
}
}

View file

@ -1,20 +0,0 @@
package akka.performance.trading.oneway
import akka.performance.trading.common.AkkaTradingSystem
import akka.performance.trading.domain.Orderbook
import akka.actor.{ Props, ActorRef }
import akka.AkkaApplication
class OneWayTradingSystem(_app: AkkaApplication) extends AkkaTradingSystem(_app) {
override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match {
case Some(d) app.actorOf(Props(new OneWayMatchingEngine(meId, orderbooks)).withDispatcher(d))
case _ app.actorOf(Props(new OneWayMatchingEngine(meId, orderbooks)))
}
override def createOrderReceiver() = orDispatcher match {
case Some(d) app.actorOf(Props[OneWayOrderReceiver].withDispatcher(d))
case _ app.actorOf(Props[OneWayOrderReceiver])
}
}

View file

@ -1,25 +0,0 @@
package akka.performance.trading.response
import org.junit.Test
import akka.actor.ActorRef
import akka.performance.trading.common.AkkaPerformanceTest
import akka.performance.trading.domain.Order
import akka.performance.trading.common.Rsp
import akka.AkkaApplication
class RspPerformanceTest extends AkkaPerformanceTest(AkkaApplication()) {
implicit def appl = app
override def placeOrder(orderReceiver: ActorRef, order: Order): Rsp = {
(orderReceiver ? order).get.asInstanceOf[Rsp]
}
// need this so that junit will detect this as a test case
@Test
def dummy {}
override def compareResultWith = Some("OneWayPerformanceTest")
}

View file

@ -1,4 +1,4 @@
package akka.performance.trading.common package akka.performance.trading.system
import akka.performance.trading.domain._ import akka.performance.trading.domain._
import akka.actor._ import akka.actor._
@ -11,7 +11,7 @@ trait MatchingEngine {
val orderbooks: List[Orderbook] val orderbooks: List[Orderbook]
val supportedOrderbookSymbols = orderbooks map (_.symbol) val supportedOrderbookSymbols = orderbooks map (_.symbol)
protected val orderbooksMap: Map[String, Orderbook] = protected val orderbooksMap: Map[String, Orderbook] =
Map() ++ (orderbooks map (o (o.symbol, o))) orderbooks.map(o (o.symbol, o)).toMap
} }
@ -21,10 +21,10 @@ class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook])
var standby: Option[ActorRef] = None var standby: Option[ActorRef] = None
def receive = { def receive = {
case standbyRef: ActorRef
standby = Some(standbyRef)
case order: Order case order: Order
handleOrder(order) handleOrder(order)
case standbyRef: ActorRef
standby = Some(standbyRef)
case unknown case unknown
log.warning("Received unknown message: " + unknown) log.warning("Received unknown message: " + unknown)
} }
@ -32,30 +32,21 @@ class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook])
def handleOrder(order: Order) { def handleOrder(order: Order) {
orderbooksMap.get(order.orderbookSymbol) match { orderbooksMap.get(order.orderbookSymbol) match {
case Some(orderbook) case Some(orderbook)
val pendingStandbyReply: Option[Future[_]] = standby.foreach(_ forward order)
for (s standby) yield { s ? order }
orderbook.addOrder(order) orderbook.addOrder(order)
orderbook.matchOrders() orderbook.matchOrders()
// wait for standby reply
pendingStandbyReply.foreach(waitForStandby(_)) done(true, order)
done(true)
case None case None
log.warning("Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol) log.warning("Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol)
done(false)
} }
} }
def done(status: Boolean) { def done(status: Boolean, order: Order) {
channel ! new Rsp(status) if (standby.isEmpty) {
} sender ! Rsp(order, status)
def waitForStandby(pendingStandbyFuture: Future[_]) {
try {
pendingStandbyFuture.await
} catch {
case e: FutureTimeoutException
log.error("Standby timeout: " + e)
} }
} }

View file

@ -1,4 +1,4 @@
package akka.performance.trading.common package akka.performance.trading.system
import akka.performance.trading.domain._ import akka.performance.trading.domain._
import akka.actor._ import akka.actor._
@ -28,20 +28,20 @@ class AkkaOrderReceiver extends Actor with OrderReceiver with ActorLogging {
type ME = ActorRef type ME = ActorRef
def receive = { def receive = {
case order: Order placeOrder(order)
case routing @ MatchingEngineRouting(mapping) case routing @ MatchingEngineRouting(mapping)
refreshMatchingEnginePartitions(routing.asInstanceOf[MatchingEngineRouting[ActorRef]]) refreshMatchingEnginePartitions(routing.asInstanceOf[MatchingEngineRouting[ActorRef]])
case order: Order placeOrder(order) case unknown log.warning("Received unknown message: " + unknown)
case unknown log.warning("Received unknown message: " + unknown)
} }
def placeOrder(order: Order) = { def placeOrder(order: Order) = {
val matchingEngine = matchingEngineForOrderbook.get(order.orderbookSymbol) val matchingEngine = matchingEngineForOrderbook.get(order.orderbookSymbol)
matchingEngine match { matchingEngine match {
case Some(m) case Some(m)
m.forward(order) m forward order
case None case None
log.warning("Unknown orderbook: " + order.orderbookSymbol) log.warning("Unknown orderbook: " + order.orderbookSymbol)
channel ! new Rsp(false) sender ! Rsp(order, false)
} }
} }
} }

View file

@ -0,0 +1,5 @@
package akka.performance.trading.system
import akka.performance.trading.domain.Order
case class Rsp(order: Order, status: Boolean)

View file

@ -0,0 +1,169 @@
package akka.performance.trading.system
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import java.util.Random
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistics
import org.junit.runner.RunWith
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import akka.performance.trading.domain.Ask
import akka.performance.trading.domain.Bid
import akka.performance.trading.domain.Order
import akka.performance.trading.domain.TotalTradeCounter
import akka.performance.workbench.PerformanceSpec
import akka.performance.trading.domain.Orderbook
// -server -Xms512M -Xmx1024M -XX:+UseConcMarkSweepGC -Dbenchmark=true -Dbenchmark.repeatFactor=500 -Dbenchmark.useDummyOrderbook=true
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TradingLatencyPerformanceSpec extends PerformanceSpec {
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.build
var tradingSystem: AkkaTradingSystem = _
var stat: DescriptiveStatistics = _
val random: Random = new Random(0)
def clientDelayMicros = {
System.getProperty("benchmark.clientDelayMicros", "250").toInt
}
override def beforeEach() {
super.beforeEach()
stat = new SynchronizedDescriptiveStatistics
tradingSystem = new AkkaTradingSystem(app)
tradingSystem.start()
TotalTradeCounter.reset()
stat = new SynchronizedDescriptiveStatistics
}
override def afterEach() {
super.afterEach()
tradingSystem.shutdown()
stat = null
}
getClass.getSimpleName must {
"warmup" in {
runScenario(4, warmup = true)
}
"warmup more" in {
runScenario(4, warmup = true)
}
"perform with load 1" in {
runScenario(1)
}
"perform with load 2" in {
runScenario(2)
}
"perform with load 4" in {
runScenario(4)
}
"perform with load 6" in {
runScenario(6)
}
"perform with load 8" in {
runScenario(8)
}
}
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
if (acceptClients(numberOfClients)) {
val repeat = 4L * repeatFactor
val prefixes = "A" :: "B" :: "C" :: "D" :: Nil
val askOrders = for {
s prefixes
i 1 to 3
} yield Ask(s + i, 100 - i, 1000)
val bidOrders = for {
s prefixes
i 1 to 3
} yield Bid(s + i, 100 - i, 1000)
val orders = askOrders.zip(bidOrders).map(x Seq(x._1, x._2)).flatten
val ordersPerClient = repeat * orders.size / numberOfClients
val totalNumberOfOrders = ordersPerClient * numberOfClients
val latch = new CountDownLatch(numberOfClients)
val receivers = tradingSystem.orderReceivers.toIndexedSeq
val start = System.nanoTime
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelayMicros)).withDispatcher(clientDispatcher)
app.actorOf(props)
})
clients.foreach(_ ! "run")
val ok = latch.await((5000000L + (clientDelayMicros + 500) * totalNumberOfOrders) * timeDilation, TimeUnit.MICROSECONDS)
val durationNs = (System.nanoTime - start)
if (!warmup) {
ok must be(true)
if (!Orderbook.useDummyOrderbook) {
TotalTradeCounter.counter.get must be(totalNumberOfOrders / 2)
}
logMeasurement(numberOfClients, durationNs, stat)
}
clients.foreach(_ ! PoisonPill)
}
}
class Client(
orderReceiver: ActorRef,
orders: List[Order],
latch: CountDownLatch,
repeat: Long,
delayMicros: Int = 0) extends Actor {
var orderIterator = orders.toIterator
def nextOrder(): Order = {
if (!orderIterator.hasNext) {
orderIterator = orders.toIterator
}
orderIterator.next()
}
var sent = 0L
var received = 0L
def receive = {
case Rsp(order, status)
if (!status) {
log.error("Invalid rsp")
}
val duration = System.nanoTime - order.nanoTime
stat.addValue(duration)
received += 1
if (sent < repeat) {
PerformanceSpec.shortDelay(delayMicros, received)
placeOrder()
sent += 1
} else if (received >= repeat) {
latch.countDown()
}
case "run"
// random initial delay to spread requests
val initialDelay = random.nextInt(20)
Thread.sleep(initialDelay)
placeOrder()
sent += 1
}
def placeOrder() {
orderReceiver ! nextOrder().withNanoTime
}
}
}

View file

@ -1,4 +1,4 @@
package akka.performance.trading.common package akka.performance.trading.system
import akka.performance.trading.domain.Orderbook import akka.performance.trading.domain.Orderbook
import akka.performance.trading.domain.OrderbookRepository import akka.performance.trading.domain.OrderbookRepository

View file

@ -0,0 +1,157 @@
package akka.performance.trading.system
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import java.util.Random
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistics
import org.junit.runner.RunWith
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import akka.performance.trading.domain.Ask
import akka.performance.trading.domain.Bid
import akka.performance.trading.domain.Order
import akka.performance.trading.domain.TotalTradeCounter
import akka.performance.workbench.PerformanceSpec
import akka.performance.trading.domain.Orderbook
// -server -Xms512M -Xmx1024M -XX:+UseParallelGC -Dbenchmark=true -Dbenchmark.repeatFactor=500 -Dbenchmark.useDummyOrderbook=true
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TradingThroughputPerformanceSpec extends PerformanceSpec {
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients)
.build
var tradingSystem: AkkaTradingSystem = _
override def beforeEach() {
super.beforeEach()
tradingSystem = new AkkaTradingSystem(app)
tradingSystem.start()
TotalTradeCounter.reset()
}
override def afterEach() {
super.afterEach()
tradingSystem.shutdown()
}
getClass.getSimpleName must {
"warmup" in {
runScenario(4, warmup = true)
}
"warmup more" in {
runScenario(4, warmup = true)
}
"perform with load 1" in {
runScenario(1)
}
"perform with load 2" in {
runScenario(2)
}
"perform with load 4" in {
runScenario(4)
}
"perform with load 6" in {
runScenario(6)
}
"perform with load 8" in {
runScenario(8)
}
"perform with load 10" in {
runScenario(10)
}
}
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
if (acceptClients(numberOfClients)) {
val repeat = 400L * repeatFactor
val prefixes = "A" :: "B" :: "C" :: "D" :: "E" :: "F" :: Nil
val askOrders = for {
s prefixes
i 1 to 4
} yield Ask(s + i, 100 - i, 1000)
val bidOrders = for {
s prefixes
i 1 to 4
} yield Bid(s + i, 100 - i, 1000)
val orders = askOrders.zip(bidOrders).map(x Seq(x._1, x._2)).flatten
val ordersPerClient = repeat * orders.size / numberOfClients
val totalNumberOfOrders = ordersPerClient * numberOfClients
val latch = new CountDownLatch(numberOfClients)
val receivers = tradingSystem.orderReceivers.toIndexedSeq
val start = System.nanoTime
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
val props = Props(new Client(receiver, orders, latch, ordersPerClient)).withDispatcher(clientDispatcher)
app.actorOf(props)
})
clients.foreach(_ ! "run")
val ok = latch.await((5000000L + 500 * totalNumberOfOrders) * timeDilation, TimeUnit.MICROSECONDS)
val durationNs = (System.nanoTime - start)
if (!warmup) {
ok must be(true)
if (!Orderbook.useDummyOrderbook) {
TotalTradeCounter.counter.get must be(totalNumberOfOrders / 2)
}
logMeasurement(numberOfClients, durationNs, totalNumberOfOrders)
}
clients.foreach(_ ! PoisonPill)
}
}
class Client(
orderReceiver: ActorRef,
orders: List[Order],
latch: CountDownLatch,
repeat: Long) extends Actor {
var orderIterator = orders.toIterator
def nextOrder(): Order = {
if (!orderIterator.hasNext) {
orderIterator = orders.toIterator
}
orderIterator.next()
}
var sent = 0L
var received = 0L
def receive = {
case Rsp(order, status)
if (!status) {
log.error("Invalid rsp")
}
received += 1
if (sent < repeat) {
placeOrder()
sent += 1
} else if (received >= repeat) {
latch.countDown()
}
case "run"
for (i 0L until math.min(1000L, repeat)) {
placeOrder()
sent += 1
}
}
def placeOrder() {
orderReceiver ! nextOrder()
}
}
}

View file

@ -24,6 +24,8 @@ trait BenchResultRepository {
def getWithHistorical(name: String, load: Int): Seq[Stats] def getWithHistorical(name: String, load: Int): Seq[Stats]
def isBaseline(stats: Stats): Boolean
def saveHtmlReport(content: String, name: String): Unit def saveHtmlReport(content: String, name: String): Unit
def htmlReportUrl(name: String): String def htmlReportUrl(name: String): String
@ -31,10 +33,11 @@ trait BenchResultRepository {
} }
object BenchResultRepository { object BenchResultRepository {
def apply(app: AkkaApplication): BenchResultRepository = new FileBenchResultRepository(app) private val repository = new FileBenchResultRepository
def apply(): BenchResultRepository = repository
} }
class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRepository { class FileBenchResultRepository extends BenchResultRepository {
private val statsByName = MutableMap[String, Seq[Stats]]() private val statsByName = MutableMap[String, Seq[Stats]]()
private val baselineStats = MutableMap[Key, Stats]() private val baselineStats = MutableMap[Key, Stats]()
private val historicalStats = MutableMap[Key, Seq[Stats]]() private val historicalStats = MutableMap[Key, Seq[Stats]]()
@ -44,8 +47,6 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep
private def htmlDirExists: Boolean = new File(htmlDir).exists private def htmlDirExists: Boolean = new File(htmlDir).exists
protected val maxHistorical = 7 protected val maxHistorical = 7
val log = Logging(app, this)
case class Key(name: String, load: Int) case class Key(name: String, load: Int)
def add(stats: Stats) { def add(stats: Stats) {
@ -62,13 +63,18 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep
get(name).find(_.load == load) get(name).find(_.load == load)
} }
def isBaseline(stats: Stats): Boolean = {
baselineStats.get(Key(stats.name, stats.load)) == Some(stats)
}
def getWithHistorical(name: String, load: Int): Seq[Stats] = { def getWithHistorical(name: String, load: Int): Seq[Stats] = {
val key = Key(name, load) val key = Key(name, load)
val historical = historicalStats.getOrElse(key, IndexedSeq.empty) val historical = historicalStats.getOrElse(key, IndexedSeq.empty)
val baseline = baselineStats.get(key) val baseline = baselineStats.get(key)
val current = get(name, load) val current = get(name, load)
(IndexedSeq.empty ++ historical ++ baseline ++ current).takeRight(maxHistorical) val limited = (IndexedSeq.empty ++ historical ++ baseline ++ current).takeRight(maxHistorical)
limited.sortBy(_.timestamp)
} }
private def loadFiles() { private def loadFiles() {
@ -105,7 +111,8 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep
out.writeObject(stats) out.writeObject(stats)
} catch { } catch {
case e: Exception case e: Exception
log.error("Failed to save [{}] to [{}], due to [{}]", stats, f.getAbsolutePath, e.getMessage) val errMsg = "Failed to save [%s] to [%s], due to [%s]".format(stats, f.getAbsolutePath, e.getMessage)
throw new RuntimeException(errMsg)
} finally { } finally {
if (out ne null) try { out.close() } catch { case ignore: Exception } if (out ne null) try { out.close() } catch { case ignore: Exception }
} }
@ -121,7 +128,6 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep
Some(stats) Some(stats)
} catch { } catch {
case e: Throwable case e: Throwable
log.error("Failed to load from [{}], due to [{}]", f.getAbsolutePath, e.getMessage)
None None
} finally { } finally {
if (in ne null) try { in.close() } catch { case ignore: Exception } if (in ne null) try { in.close() } catch { case ignore: Exception }
@ -144,7 +150,8 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep
writer.flush() writer.flush()
} catch { } catch {
case e: Exception case e: Exception
log.error("Failed to save report to [{}], due to [{}]", f.getAbsolutePath, e.getMessage) val errMsg = "Failed to save report to [%s], due to [%s]".format(f.getAbsolutePath, e.getMessage)
throw new RuntimeException(errMsg)
} finally { } finally {
if (writer ne null) try { writer.close() } catch { case ignore: Exception } if (writer ne null) try { writer.close() } catch { case ignore: Exception }
} }

View file

@ -13,6 +13,72 @@ object GoogleChartBuilder {
val ChartWidth = 750 val ChartWidth = 750
val ChartHeight = 400 val ChartHeight = 400
/**
* Builds a bar chart for tps in the statistics.
*/
def tpsChartUrl(statsByTimestamp: TreeMap[Long, Seq[Stats]], title: String, legend: Stats String): String = {
if (statsByTimestamp.isEmpty) return ""
val loads = statsByTimestamp.values.head.map(_.load)
val allStats = statsByTimestamp.values.flatten
val sb = new StringBuilder
sb.append(BaseUrl)
// bar chart
sb.append("cht=bvg")
sb.append("&")
// size
sb.append("chs=").append(ChartWidth).append("x").append(ChartHeight)
sb.append("&")
// title
sb.append("chtt=").append(urlEncode(title))
sb.append("&")
// axis locations
sb.append("chxt=y,x")
sb.append("&")
// labels
sb.append("chxl=1:|")
sb.append(loads.mkString("|"))
sb.append("&")
// label color and font
//sb.append("chxs=2,D65D82,11.5,0,lt,D65D82")
//sb.append("&")
// legend
val legendStats = statsByTimestamp.values.map(_.head).toSeq
appendLegend(legendStats, sb, legend)
sb.append("&")
// bar spacing
sb.append("chbh=a,4,20")
sb.append("&")
// bar colors
barColors(statsByTimestamp.size, sb)
sb.append("&")
// data series
val loadStr = loads.mkString(",")
sb.append("chd=t:")
val maxValue = allStats.map(_.tps).max
val tpsSeries: Iterable[String] =
for (statsSeq statsByTimestamp.values) yield {
statsSeq.map(_.tps).mkString(",")
}
sb.append(tpsSeries.mkString("|"))
// y range
sb.append("&")
sb.append("chxr=0,0,").append(maxValue)
sb.append("&")
sb.append("chds=0,").append(maxValue)
sb.append("&")
// grid lines
appendGridSpacing(maxValue.toLong, sb)
return sb.toString
}
/** /**
* Builds a bar chart for all percentiles and the mean in the statistics. * Builds a bar chart for all percentiles and the mean in the statistics.
*/ */
@ -113,6 +179,11 @@ object GoogleChartBuilder {
sb.append(series.mkString("|")) sb.append(series.mkString("|"))
} }
private def dataSeries(values: Seq[Double], sb: StringBuilder) {
val series = values.map(formatDouble(_))
sb.append(series.mkString("|"))
}
private def appendGridSpacing(maxValue: Long, sb: StringBuilder) { private def appendGridSpacing(maxValue: Long, sb: StringBuilder) {
sb.append("chg=0,10") sb.append("chg=0,10")
} }

View file

@ -0,0 +1,113 @@
package akka.performance.workbench
import scala.collection.immutable.TreeMap
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import org.scalatest.BeforeAndAfterEach
import akka.actor.simpleName
import akka.testkit.AkkaSpec
import akka.AkkaApplication
trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
def app: AkkaApplication
def isBenchmark() = System.getProperty("benchmark") == "true"
def minClients() = System.getProperty("benchmark.minClients", "1").toInt;
def maxClients() = System.getProperty("benchmark.maxClients", "40").toInt;
def repeatFactor() = {
val defaultRepeatFactor = if (isBenchmark) "150" else "2"
System.getProperty("benchmark.repeatFactor", defaultRepeatFactor).toInt
}
def timeDilation() = {
System.getProperty("benchmark.timeDilation", "1").toLong
}
val resultRepository = BenchResultRepository()
lazy val report = new Report(app, resultRepository, compareResultWith)
/**
* To compare two tests with each other you can override this method, in
* the test. For example Some("OneWayPerformanceTest")
*/
def compareResultWith: Option[String] = None
def acceptClients(numberOfClients: Int): Boolean = {
(minClients <= numberOfClients && numberOfClients <= maxClients)
}
def logMeasurement(numberOfClients: Int, durationNs: Long, n: Long) {
val name = simpleName(this)
val durationS = durationNs.toDouble / 1000000000.0
val stats = Stats(
name,
load = numberOfClients,
timestamp = TestStart.startTime,
durationNanos = durationNs,
n = n,
tps = (n.toDouble / durationS))
logMeasurement(stats)
}
def logMeasurement(numberOfClients: Int, durationNs: Long, stat: DescriptiveStatistics) {
val name = simpleName(this)
val durationS = durationNs.toDouble / 1000000000.0
val percentiles = TreeMap[Int, Long](
5 -> (stat.getPercentile(5.0) / 1000).toLong,
25 -> (stat.getPercentile(25.0) / 1000).toLong,
50 -> (stat.getPercentile(50.0) / 1000).toLong,
75 -> (stat.getPercentile(75.0) / 1000).toLong,
95 -> (stat.getPercentile(95.0) / 1000).toLong)
val n = stat.getN
val stats = Stats(
name,
load = numberOfClients,
timestamp = TestStart.startTime,
durationNanos = durationNs,
n = n,
min = (stat.getMin / 1000).toLong,
max = (stat.getMax / 1000).toLong,
mean = (stat.getMean / 1000).toLong,
tps = (n.toDouble / durationS),
percentiles)
logMeasurement(stats)
}
def logMeasurement(stats: Stats) {
try {
resultRepository.add(stats)
report.html(resultRepository.get(stats.name))
} catch {
// don't fail test due to problems saving bench report
case e: Exception log.error(e, e.getMessage)
}
}
}
object PerformanceSpec {
def shortDelay(micros: Int, n: Long) {
if (micros > 0) {
val sampling = 1000 / micros
if (n % sampling == 0) {
Thread.sleep(1)
}
}
}
}
object TestStart {
val startTime = System.currentTimeMillis
}

View file

@ -7,10 +7,12 @@ import scala.collection.JavaConversions.asScalaBuffer
import scala.collection.JavaConversions.enumerationAsScalaIterator import scala.collection.JavaConversions.enumerationAsScalaIterator
import akka.AkkaApplication import akka.AkkaApplication
import akka.event.Logging import akka.event.Logging
import scala.collection.immutable.TreeMap
class Report(app: AkkaApplication, class Report(
resultRepository: BenchResultRepository, app: AkkaApplication,
compareResultWith: Option[String] = None) { resultRepository: BenchResultRepository,
compareResultWith: Option[String] = None) {
private def doLog = System.getProperty("benchmark.logResult", "true").toBoolean private def doLog = System.getProperty("benchmark.logResult", "true").toBoolean
val log = Logging(app, this) val log = Logging(app, this)
@ -33,15 +35,20 @@ class Report(app: AkkaApplication,
sb.append(resultTable) sb.append(resultTable)
sb.append("\n</pre>\n") sb.append("\n</pre>\n")
sb.append(img(percentilesAndMeanChart(current)))
sb.append(img(latencyAndThroughputChart(current))) sb.append(img(latencyAndThroughputChart(current)))
for (stats statistics) { compareWithHistoricalTpsChart(statistics).foreach(url sb.append(img(url)))
compareWithHistoricalPercentiliesAndMeanChart(stats).foreach(url sb.append(img(url)))
}
for (stats statistics) { if (current.max > 0L) {
comparePercentilesAndMeanChart(stats).foreach(url sb.append(img(url))) sb.append(img(percentilesAndMeanChart(current)))
for (stats statistics) {
compareWithHistoricalPercentiliesAndMeanChart(stats).foreach(url sb.append(img(url)))
}
for (stats statistics) {
comparePercentilesAndMeanChart(stats).foreach(url sb.append(img(url)))
}
} }
sb.append("<hr/>\n") sb.append("<hr/>\n")
@ -64,6 +71,11 @@ class Report(app: AkkaApplication,
url, GoogleChartBuilder.ChartWidth, GoogleChartBuilder.ChartHeight) + "\n" url, GoogleChartBuilder.ChartWidth, GoogleChartBuilder.ChartHeight) + "\n"
} }
protected def timeLegend(stats: Stats): String = {
val baseline = if (resultRepository.isBaseline(stats)) " *" else ""
legendTimeFormat.format(new Date(stats.timestamp)) + baseline
}
def percentilesAndMeanChart(stats: Stats): String = { def percentilesAndMeanChart(stats: Stats): String = {
val chartTitle = stats.name + " Percentiles and Mean (microseconds)" val chartTitle = stats.name + " Percentiles and Mean (microseconds)"
val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(resultRepository.get(stats.name), chartTitle, _.load + " clients") val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(resultRepository.get(stats.name), chartTitle, _.load + " clients")
@ -85,14 +97,36 @@ class Report(app: AkkaApplication,
val withHistorical = resultRepository.getWithHistorical(stats.name, stats.load) val withHistorical = resultRepository.getWithHistorical(stats.name, stats.load)
if (withHistorical.size > 1) { if (withHistorical.size > 1) {
val chartTitle = stats.name + " vs. historical, " + stats.load + " clients" + ", Percentiles and Mean (microseconds)" val chartTitle = stats.name + " vs. historical, " + stats.load + " clients" + ", Percentiles and Mean (microseconds)"
val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(withHistorical, chartTitle, val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(withHistorical, chartTitle, timeLegend)
stats legendTimeFormat.format(new Date(stats.timestamp)))
Some(chartUrl) Some(chartUrl)
} else { } else {
None None
} }
} }
def compareWithHistoricalTpsChart(statistics: Seq[Stats]): Option[String] = {
if (statistics.isEmpty) {
None
} else {
val histTimestamps = resultRepository.getWithHistorical(statistics.head.name, statistics.head.load).map(_.timestamp)
val statsByTimestamp = TreeMap[Long, Seq[Stats]]() ++
(for (ts histTimestamps) yield {
val seq =
for (stats statistics) yield {
val withHistorical: Seq[Stats] = resultRepository.getWithHistorical(stats.name, stats.load)
val cell = withHistorical.find(_.timestamp == ts)
cell.getOrElse(Stats(stats.name, stats.load, ts))
}
(ts, seq)
})
val chartTitle = statistics.last.name + " vs. historical, Throughput (TPS)"
val chartUrl = GoogleChartBuilder.tpsChartUrl(statsByTimestamp, chartTitle, timeLegend)
Some(chartUrl)
}
}
def latencyAndThroughputChart(stats: Stats): String = { def latencyAndThroughputChart(stats: Stats): String = {
val chartTitle = stats.name + " Latency (microseconds) and Throughput (TPS)" val chartTitle = stats.name + " Latency (microseconds) and Throughput (TPS)"
val chartUrl = GoogleChartBuilder.latencyAndThroughputChartUrl(resultRepository.get(stats.name), chartTitle) val chartUrl = GoogleChartBuilder.latencyAndThroughputChartUrl(resultRepository.get(stats.name), chartTitle)

View file

@ -7,14 +7,23 @@ case class Stats(
name: String, name: String,
load: Int, load: Int,
timestamp: Long = System.currentTimeMillis, timestamp: Long = System.currentTimeMillis,
durationNanos: Long, durationNanos: Long = 0L,
n: Long, n: Long = 0L,
min: Long, min: Long = 0L,
max: Long, max: Long = 0L,
mean: Double, mean: Double = 0.0,
tps: Double, tps: Double = 0.0,
percentiles: TreeMap[Int, Long]) { percentiles: TreeMap[Int, Long] = Stats.emptyPercentiles) {
def median: Long = percentiles(50) def median: Long = percentiles(50)
} }
object Stats {
val emptyPercentiles = TreeMap[Int, Long](
5 -> 0L,
25 -> 0L,
50 -> 0L,
75 -> 0L,
95 -> 0L)
}

View file

@ -41,7 +41,7 @@ class ActorPoolSpec extends AkkaSpec {
case _ case _
count.incrementAndGet count.incrementAndGet
latch.countDown() latch.countDown()
channel.tryTell("success") sender.tell("success")
} }
})) }))
@ -88,7 +88,7 @@ class ActorPoolSpec extends AkkaSpec {
def receive = { def receive = {
case req: String { case req: String {
(10 millis).dilated.sleep (10 millis).dilated.sleep
channel.tryTell("Response") sender.tell("Response")
} }
} }
})) }))
@ -111,7 +111,7 @@ class ActorPoolSpec extends AkkaSpec {
val count = new AtomicInteger(0) val count = new AtomicInteger(0)
val pool = actorOf( val pool = actorOf(
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveActorsPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter {
def instance(p: Props) = actorOf(p.withCreator(new Actor { def instance(p: Props) = actorOf(p.withCreator(new Actor {
def receive = { def receive = {
case n: Int case n: Int
@ -233,7 +233,7 @@ class ActorPoolSpec extends AkkaSpec {
def instance(p: Props): ActorRef = actorOf(p.withCreator(new Actor { def instance(p: Props): ActorRef = actorOf(p.withCreator(new Actor {
def receive = { def receive = {
case _ case _
delegates put (self.uuid.toString, "") delegates put (self.address, "")
latch1.countDown() latch1.countDown()
} }
})) }))
@ -261,7 +261,7 @@ class ActorPoolSpec extends AkkaSpec {
def instance(p: Props) = actorOf(p.withCreator(new Actor { def instance(p: Props) = actorOf(p.withCreator(new Actor {
def receive = { def receive = {
case _ case _
delegates put (self.uuid.toString, "") delegates put (self.address, "")
latch2.countDown() latch2.countDown()
} }
})) }))
@ -331,7 +331,7 @@ class ActorPoolSpec extends AkkaSpec {
"support typed actors" in { "support typed actors" in {
import RoutingSpec._ import RoutingSpec._
def createPool = new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup { val pool = app.createProxy[Foo](new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
def lowerBound = 1 def lowerBound = 1
def upperBound = 5 def upperBound = 5
def pressureThreshold = 1 def pressureThreshold = 1
@ -340,15 +340,17 @@ class ActorPoolSpec extends AkkaSpec {
def rampupRate = 0.1 def rampupRate = 0.1
def backoffRate = 0.50 def backoffRate = 0.50
def backoffThreshold = 0.50 def backoffThreshold = 0.50
def instance(p: Props) = app.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](p)) def instance(p: Props) = app.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds)))
def receive = _route def receive = _route
}, Props().withTimeout(10 seconds).withFaultHandler(faultHandler))
val results = for (i 1 to 20) yield (i, pool.sq(i, 10))
for ((i, r) results) {
val value = r.get
value must equal(i * i)
} }
app.typedActor.stop(pool)
val pool = app.createProxy[Foo](createPool, Props().withFaultHandler(faultHandler))
val results = for (i 1 to 100) yield (i, pool.sq(i, 100))
for ((i, r) results) r.get must equal(i * i)
} }
"provide default supervision of pooled actors" in { "provide default supervision of pooled actors" in {
@ -358,7 +360,7 @@ class ActorPoolSpec extends AkkaSpec {
val keepDying = new AtomicBoolean(false) val keepDying = new AtomicBoolean(false)
val pool1, pool2 = actorOf( val pool1, pool2 = actorOf(
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveActorsPressureCapacitor with SmallestMailboxSelector with BasicFilter {
def lowerBound = 2 def lowerBound = 2
def upperBound = 5 def upperBound = 5
def rampupRate = 0.1 def rampupRate = 0.1
@ -381,7 +383,7 @@ class ActorPoolSpec extends AkkaSpec {
}).withFaultHandler(faultHandler)) }).withFaultHandler(faultHandler))
val pool3 = actorOf( val pool3 = actorOf(
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with RoundRobinSelector with BasicFilter { Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveActorsPressureCapacitor with RoundRobinSelector with BasicFilter {
def lowerBound = 2 def lowerBound = 2
def upperBound = 5 def upperBound = 5
def rampupRate = 0.1 def rampupRate = 0.1
@ -479,7 +481,7 @@ class ActorPoolSpec extends AkkaSpec {
object BadState object BadState
val pool1 = actorOf( val pool1 = actorOf(
Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveActorsPressureCapacitor with SmallestMailboxSelector with BasicFilter {
def lowerBound = 2 def lowerBound = 2
def upperBound = 5 def upperBound = 5
def rampupRate = 0.1 def rampupRate = 0.1

View file

@ -16,7 +16,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
"be able to shut down its instance" in { "be able to shut down its instance" in {
val address = "round-robin-0" val address = "round-robin-0"
app.deployer.deploy( app.provider.deployer.deploy(
Deploy( Deploy(
address, address,
None, None,
@ -52,7 +52,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
"deliver messages in a round robin fashion" in { "deliver messages in a round robin fashion" in {
val address = "round-robin-1" val address = "round-robin-1"
app.deployer.deploy( app.provider.deployer.deploy(
Deploy( Deploy(
address, address,
None, None,
@ -74,7 +74,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val actor = app.actorOf(Props(new Actor { val actor = app.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement() lazy val id = counter.getAndIncrement()
def receive = { def receive = {
case "hit" channel ! id case "hit" sender ! id
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
} }
}), address) }), address)
@ -97,7 +97,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
"deliver a broadcast message using the !" in { "deliver a broadcast message using the !" in {
val address = "round-robin-2" val address = "round-robin-2"
app.deployer.deploy( app.provider.deployer.deploy(
Deploy( Deploy(
address, address,
None, None,
@ -132,7 +132,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
"be able to shut down its instance" in { "be able to shut down its instance" in {
val address = "random-0" val address = "random-0"
app.deployer.deploy( app.provider.deployer.deploy(
Deploy( Deploy(
address, address,
None, None,
@ -166,7 +166,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
"deliver messages in a random fashion" in { "deliver messages in a random fashion" in {
val address = "random-1" val address = "random-1"
app.deployer.deploy( app.provider.deployer.deploy(
Deploy( Deploy(
address, address,
None, None,
@ -188,7 +188,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val actor = app.actorOf(Props(new Actor { val actor = app.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement() lazy val id = counter.getAndIncrement()
def receive = { def receive = {
case "hit" channel ! id case "hit" sender ! id
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
} }
}), address) }), address)
@ -211,7 +211,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
"deliver a broadcast message using the !" in { "deliver a broadcast message using the !" in {
val address = "random-2" val address = "random-2"
app.deployer.deploy( app.provider.deployer.deploy(
Deploy( Deploy(
address, address,
None, None,

View file

@ -194,15 +194,9 @@ class RoutingSpec extends AkkaSpec {
} }
}) })
val props = RoutedProps().withRoundRobinRouter.withLocalConnections(List(connection1)) val actor = app.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(List(connection1)), "foo")
val actor = app.actorOf(props, "foo")
try { intercept[RoutingException] { actor ? Broadcast(1) }
actor ? Broadcast(1)
fail()
} catch {
case e: RoutingException
}
actor ! "end" actor ! "end"
doneLatch.await(5, TimeUnit.SECONDS) must be(true) doneLatch.await(5, TimeUnit.SECONDS) must be(true)
@ -459,7 +453,7 @@ class RoutingSpec extends AkkaSpec {
case Stop(None) self.stop() case Stop(None) self.stop()
case Stop(Some(_id)) if (_id == id) self.stop() case Stop(Some(_id)) if (_id == id) self.stop()
case _id: Int if (_id == id) case _id: Int if (_id == id)
case _ Thread sleep 100 * id; channel.tryTell(id) case _ Thread sleep 100 * id; sender.tell(id)
} }
override def postStop = { override def postStop = {

View file

@ -7,6 +7,9 @@ package akka.serialization
import akka.serialization.Serialization._ import akka.serialization.Serialization._
import scala.reflect._ import scala.reflect._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.AkkaApplication
import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import akka.actor.DeadLetterActorRef
object SerializeSpec { object SerializeSpec {
@BeanInfo @BeanInfo
@ -61,5 +64,20 @@ class SerializeSpec extends AkkaSpec {
case Right(p) assert(p === r) case Right(p) assert(p === r)
} }
} }
"serialize DeadLetterActorRef" in {
val outbuf = new ByteArrayOutputStream()
val out = new ObjectOutputStream(outbuf)
val a = new AkkaApplication()
out.writeObject(a.deadLetters)
out.flush()
out.close()
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
Serialization.app.withValue(a) {
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
(deadLetters eq a.deadLetters) must be(true)
}
}
} }
} }

View file

@ -11,7 +11,7 @@ import org.junit.{ After, Test }
class CallingThreadDispatcherModelSpec extends ActorModelSpec { class CallingThreadDispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._ import ActorModelSpec._
def newInterceptedDispatcher = new CallingThreadDispatcher(app, "test", true) with MessageDispatcherInterceptor def newInterceptedDispatcher = new CallingThreadDispatcher(app, "test") with MessageDispatcherInterceptor
def dispatcherType = "Calling Thread Dispatcher" def dispatcherType = "Calling Thread Dispatcher"
} }

View file

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

View file

@ -7,7 +7,7 @@ package akka.dispatch;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
abstract class AbstractPromise { abstract class AbstractPromise {
private volatile Object _ref = FState.apply(); private volatile Object _ref = DefaultPromise.EmptyPending();
protected final static AtomicReferenceFieldUpdater<AbstractPromise, Object> updater = protected final static AtomicReferenceFieldUpdater<AbstractPromise, Object> updater =
AtomicReferenceFieldUpdater.newUpdater(AbstractPromise.class, Object.class, "_ref"); AtomicReferenceFieldUpdater.newUpdater(AbstractPromise.class, Object.class, "_ref");
} }

View file

@ -74,7 +74,7 @@ public final class UUIDGen {
* The last time value. Used to remove duplicate UUIDs. * The last time value. Used to remove duplicate UUIDs.
*/ */
private final static AtomicLong lastTime = new AtomicLong(Long.MIN_VALUE); private final static AtomicLong lastTime = new AtomicLong(Long.MIN_VALUE);
/** /**
* The cached MAC address. * The cached MAC address.
*/ */
@ -233,11 +233,11 @@ public final class UUIDGen {
public static long newTime() { public static long newTime() {
return createTime(System.currentTimeMillis()); return createTime(System.currentTimeMillis());
} }
/** /**
* Creates a new time field from the given timestamp. Note that even identical * Creates a new time field from the given timestamp. Note that even identical
* values of <code>currentTimeMillis</code> will produce different time fields. * values of <code>currentTimeMillis</code> will produce different time fields.
* *
* @param currentTimeMillis the timestamp * @param currentTimeMillis the timestamp
* @return a new time value * @return a new time value
* @see UUID#getTime() * @see UUID#getTime()
@ -275,10 +275,10 @@ public final class UUIDGen {
return time; return time;
} }
/** /**
* Returns the MAC address. Not guaranteed to return anything. * Returns the MAC address. Not guaranteed to return anything.
* *
* @return the MAC address, may be <code>null</code> * @return the MAC address, may be <code>null</code>
*/ */
public static String getMACAddress() { public static String getMACAddress() {

View file

@ -13,7 +13,6 @@ import com.eaio.uuid.UUID
import akka.dispatch.{ Dispatchers, Future } import akka.dispatch.{ Dispatchers, Future }
import akka.util.Duration import akka.util.Duration
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import java.util.concurrent.TimeUnit
import akka.routing.Routing import akka.routing.Routing
import akka.remote.RemoteSupport import akka.remote.RemoteSupport
import akka.serialization.Serialization import akka.serialization.Serialization
@ -156,17 +155,17 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
case value value case value value
} }
val hostname: String = System.getProperty("akka.remote.hostname") match { val defaultAddress = new InetSocketAddress(System.getProperty("akka.remote.hostname") match {
case null | "" InetAddress.getLocalHost.getHostName case null | "" InetAddress.getLocalHost.getHostAddress
case value value case value value
} }, System.getProperty("akka.remote.port") match {
val port: Int = System.getProperty("akka.remote.port") match {
case null | "" AkkaConfig.RemoteServerPort case null | "" AkkaConfig.RemoteServerPort
case value value.toInt case value value.toInt
} })
val defaultAddress = new InetSocketAddress(hostname, AkkaConfig.RemoteServerPort) def hostname: String = defaultAddress.getAddress.getHostAddress
def port: Int = defaultAddress.getPort
// this provides basic logging (to stdout) until .start() is called below // this provides basic logging (to stdout) until .start() is called below
val mainbus = new MainBus(DebugMainBus) val mainbus = new MainBus(DebugMainBus)

View file

@ -39,7 +39,7 @@ class AkkaException(message: String = "", cause: Throwable = null) extends Runti
object AkkaException { object AkkaException {
val hostname = try { val hostname = try {
InetAddress.getLocalHost.getHostName InetAddress.getLocalHost.getHostAddress
} catch { } catch {
case e: UnknownHostException "unknown" case e: UnknownHostException "unknown"
} }

View file

@ -119,8 +119,9 @@ case class UnhandledMessageException(msg: Any, ref: ActorRef = null) extends Exc
/** /**
* Classes for passing status back to the sender. * Classes for passing status back to the sender.
* Used for internal ACKing protocol. But exposed as utility class for user-specific ACKing protocols as well.
*/ */
object Status { //FIXME Why does this exist at all? object Status {
sealed trait Status extends Serializable sealed trait Status extends Serializable
case class Success(status: AnyRef) extends Status case class Success(status: AnyRef) extends Status
case class Failure(cause: Throwable) extends Status case class Failure(cause: Throwable) extends Status
@ -212,16 +213,15 @@ trait Actor {
* Stores the context for this actor, including self, sender, and hotswap. * Stores the context for this actor, including self, sender, and hotswap.
*/ */
@transient @transient
private[akka] val context: ActorContext = { private[akka] implicit val context: ActorContext = {
val contextStack = ActorCell.contextStack.get val contextStack = ActorCell.contextStack.get
def noContextError = { def noContextError =
throw new ActorInitializationException( throw new ActorInitializationException(
"\n\tYou cannot create an instance of " + getClass.getName + " explicitly using the constructor (new)." + "\n\tYou cannot create an instance of " + getClass.getName + " explicitly using the constructor (new)." +
"\n\tYou have to use one of the factory methods to create a new actor. Either use:" + "\n\tYou have to use one of the factory methods to create a new actor. Either use:" +
"\n\t\t'val actor = Actor.actorOf[MyActor]', or" + "\n\t\t'val actor = Actor.actorOf[MyActor]', or" +
"\n\t\t'val actor = Actor.actorOf(new MyActor(..))'") "\n\t\t'val actor = Actor.actorOf(new MyActor(..))'")
}
if (contextStack.isEmpty) noContextError if (contextStack.isEmpty) noContextError
val context = contextStack.head val context = contextStack.head
@ -286,14 +286,6 @@ trait Actor {
@inline @inline
final def sender: ActorRef = context.sender final def sender: ActorRef = context.sender
/**
* Abstraction for unification of sender and senderFuture for later reply
*/
def channel: UntypedChannel = context.channel
// TODO FIXME REMOVE ME just for current compatibility
implicit def forwardable: ForwardableChannel = ForwardableChannel(channel)
/** /**
* Gets the current receive timeout * Gets the current receive timeout
* When specified, the receive method should be able to handle a 'ReceiveTimeout' message. * When specified, the receive method should be able to handle a 'ReceiveTimeout' message.
@ -329,7 +321,7 @@ trait Actor {
* def receive = { * def receive = {
* case Ping =&gt; * case Ping =&gt;
* println("got a 'Ping' message") * println("got a 'Ping' message")
* channel ! "pong" * sender ! "pong"
* *
* case OneWay =&gt; * case OneWay =&gt;
* println("got a 'OneWay' message") * println("got a 'OneWay' message")
@ -405,13 +397,13 @@ trait Actor {
/** /**
* Registers this actor as a Monitor for the provided ActorRef * Registers this actor as a Monitor for the provided ActorRef
* @returns the provided ActorRef * @return the provided ActorRef
*/ */
def watch(subject: ActorRef): ActorRef = self startsMonitoring subject def watch(subject: ActorRef): ActorRef = self startsMonitoring subject
/** /**
* Unregisters this actor as Monitor for the provided ActorRef * Unregisters this actor as Monitor for the provided ActorRef
* @returns the provided ActorRef * @return the provided ActorRef
*/ */
def unwatch(subject: ActorRef): ActorRef = self stopsMonitoring subject def unwatch(subject: ActorRef): ActorRef = self stopsMonitoring subject
@ -430,10 +422,7 @@ trait Actor {
case f: Failed context.handleFailure(f) case f: Failed context.handleFailure(f)
case ct: ChildTerminated context.handleChildTerminated(ct.child) case ct: ChildTerminated context.handleChildTerminated(ct.child)
case Kill throw new ActorKilledException("Kill") case Kill throw new ActorKilledException("Kill")
case PoisonPill case PoisonPill self.stop()
val ch = channel
self.stop()
ch.sendException(new ActorKilledException("PoisonPill"))
} }
} }

View file

@ -10,7 +10,6 @@ import scala.annotation.tailrec
import scala.collection.immutable.{ Stack, TreeMap } import scala.collection.immutable.{ Stack, TreeMap }
import scala.collection.JavaConverters import scala.collection.JavaConverters
import java.util.concurrent.{ ScheduledFuture, TimeUnit } import java.util.concurrent.{ ScheduledFuture, TimeUnit }
import java.util.{ Collection JCollection, Collections JCollections }
import akka.AkkaApplication import akka.AkkaApplication
import akka.event.Logging.{ Debug, Warning, Error } import akka.event.Logging.{ Debug, Warning, Error }
@ -39,8 +38,6 @@ private[akka] trait ActorContext extends ActorRefFactory with TypedActorFactory
def sender: ActorRef def sender: ActorRef
def channel: UntypedChannel
def children: Iterable[ActorRef] def children: Iterable[ActorRef]
def dispatcher: MessageDispatcher def dispatcher: MessageDispatcher
@ -51,6 +48,7 @@ private[akka] trait ActorContext extends ActorRefFactory with TypedActorFactory
def app: AkkaApplication def app: AkkaApplication
def parent: ActorRef
} }
private[akka] object ActorCell { private[akka] object ActorCell {
@ -67,17 +65,17 @@ private[akka] class ActorCell(
val app: AkkaApplication, val app: AkkaApplication,
val self: ActorRef with ScalaActorRef, val self: ActorRef with ScalaActorRef,
val props: Props, val props: Props,
val supervisor: ActorRef, val parent: ActorRef,
var receiveTimeout: Option[Long], var receiveTimeout: Option[Long],
var hotswap: Stack[PartialFunction[Any, Unit]]) extends ActorContext { var hotswap: Stack[PartialFunction[Any, Unit]]) extends ActorContext {
import ActorCell._ import ActorCell._
protected def guardian = self protected final def guardian = self
protected def typedActor = app.typedActor protected def typedActor = app.typedActor
def provider = app.provider final def provider = app.provider
var futureTimeout: Option[ScheduledFuture[AnyRef]] = None var futureTimeout: Option[ScheduledFuture[AnyRef]] = None
@ -85,75 +83,56 @@ private[akka] class ActorCell(
var currentMessage: Envelope = null var currentMessage: Envelope = null
var actor: Actor = _ //FIXME We can most probably make this just a regular reference to Actor var actor: Actor = _
def uuid: Uuid = self.uuid
@inline @inline
final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher
def isShutdown: Boolean = mailbox.isClosed final def isShutdown: Boolean = mailbox.isClosed
@volatile //This must be volatile since it isn't protected by the mailbox status @volatile //This must be volatile since it isn't protected by the mailbox status
var mailbox: Mailbox = _ var mailbox: Mailbox = _
def hasMessages: Boolean = mailbox.hasMessages def hasMessages: Boolean = mailbox.hasMessages
def start(): Unit = { final def start(): Unit = {
mailbox = dispatcher.createMailbox(this) mailbox = dispatcher.createMailbox(this)
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
supervisor.sendSystemMessage(akka.dispatch.Supervise(self)) parent.sendSystemMessage(akka.dispatch.Supervise(self))
dispatcher.attach(this) dispatcher.attach(this)
} }
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
def suspend(): Unit = dispatcher.systemDispatch(this, Suspend()) final def suspend(): Unit = dispatcher.systemDispatch(this, Suspend())
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
def resume(): Unit = dispatcher.systemDispatch(this, Resume()) final def resume(): Unit = dispatcher.systemDispatch(this, Resume())
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
private[akka] def stop(): Unit = dispatcher.systemDispatch(this, Terminate()) private[akka] def stop(): Unit = dispatcher.systemDispatch(this, Terminate())
def startsMonitoring(subject: ActorRef): ActorRef = { final def startsMonitoring(subject: ActorRef): ActorRef = {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Link(subject)) dispatcher.systemDispatch(this, Link(subject))
subject subject
} }
def stopsMonitoring(subject: ActorRef): ActorRef = { final def stopsMonitoring(subject: ActorRef): ActorRef = {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Unlink(subject)) dispatcher.systemDispatch(this, Unlink(subject))
subject subject
} }
def children: Iterable[ActorRef] = _children.keys final def children: Iterable[ActorRef] = _children.keys
def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = dispatcher.dispatch(this, Envelope(message, channel)) final def postMessageToMailbox(message: Any, sender: ActorRef): Unit = dispatcher.dispatch(this, Envelope(message, sender))
def postMessageToMailboxAndCreateFutureResultWithTimeout( final def sender: ActorRef = currentMessage match {
message: Any, case null app.deadLetters
timeout: Timeout, case msg if msg.sender ne null msg.sender
channel: UntypedChannel): Future[Any] = { case _ app.deadLetters
val future = channel match {
case f: ActorPromise f
case _ new ActorPromise(timeout)(dispatcher)
}
dispatcher.dispatch(this, Envelope(message, future))
future
}
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 {
case null NullChannel
case msg msg.channel
} }
//This method is in charge of setting up the contextStack and create a new instance of the Actor //This method is in charge of setting up the contextStack and create a new instance of the Actor
@ -174,7 +153,7 @@ private[akka] class ActorCell(
} }
} }
def systemInvoke(message: SystemMessage) { final def systemInvoke(message: SystemMessage) {
def create(): Unit = try { def create(): Unit = try {
val created = newActor() val created = newActor()
@ -189,7 +168,7 @@ private[akka] class ActorCell(
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
} finally { } finally {
supervisor ! Failed(self, ActorInitializationException(self, "exception during creation", e)) parent ! Failed(self, ActorInitializationException(self, "exception during creation", e))
} }
} }
@ -220,7 +199,7 @@ private[akka] class ActorCell(
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
} finally { } finally {
supervisor ! Failed(self, ActorInitializationException(self, "exception during re-creation", e)) parent ! Failed(self, ActorInitializationException(self, "exception during re-creation", e))
} }
} }
@ -249,7 +228,7 @@ private[akka] class ActorCell(
} }
} finally { } finally {
try { try {
supervisor ! ChildTerminated(self) parent ! ChildTerminated(self)
app.deathWatch.publish(Terminated(self)) app.deathWatch.publish(Terminated(self))
} finally { } finally {
currentMessage = null currentMessage = null
@ -292,7 +271,7 @@ private[akka] class ActorCell(
} }
} }
def invoke(messageHandle: Envelope) { final def invoke(messageHandle: Envelope) {
try { try {
val isClosed = mailbox.isClosed //Fence plus volatile read val isClosed = mailbox.isClosed //Fence plus volatile read
if (!isClosed) { if (!isClosed) {
@ -313,14 +292,12 @@ private[akka] class ActorCell(
// make sure that InterruptedException does not leave this thread // make sure that InterruptedException does not leave this thread
if (e.isInstanceOf[InterruptedException]) { if (e.isInstanceOf[InterruptedException]) {
val ex = ActorInterruptedException(e) val ex = ActorInterruptedException(e)
channel.sendException(ex)
props.faultHandler.handleSupervisorFailing(self, children) props.faultHandler.handleSupervisorFailing(self, children)
supervisor ! Failed(self, ex) parent ! Failed(self, ex)
throw e //Re-throw InterruptedExceptions as expected throw e //Re-throw InterruptedExceptions as expected
} else { } else {
channel.sendException(e)
props.faultHandler.handleSupervisorFailing(self, children) props.faultHandler.handleSupervisorFailing(self, children)
supervisor ! Failed(self, e) parent ! Failed(self, e)
} }
} finally { } finally {
checkReceiveTimeout // Reschedule receive timeout checkReceiveTimeout // Reschedule receive timeout
@ -330,27 +307,24 @@ private[akka] class ActorCell(
app.mainbus.publish(Error(e, self, e.getMessage)) app.mainbus.publish(Error(e, self, e.getMessage))
throw e throw e
} }
} else {
messageHandle.channel sendException new ActorKilledException("Actor has been stopped")
// throwing away message if actor is shut down, no use throwing an exception in receiving actor's thread, isShutdown is enforced on caller side
} }
} }
} }
def handleFailure(fail: Failed): Unit = _children.get(fail.actor) match { final def handleFailure(fail: Failed): Unit = _children.get(fail.actor) match {
case Some(stats) if (!props.faultHandler.handleFailure(fail, stats, _children)) throw fail.cause case Some(stats) if (!props.faultHandler.handleFailure(fail, stats, _children)) throw fail.cause
case None app.mainbus.publish(Warning(self, "dropping " + fail + " from unknown child")) case None app.mainbus.publish(Warning(self, "dropping " + fail + " from unknown child"))
} }
def handleChildTerminated(child: ActorRef): Unit = { final def handleChildTerminated(child: ActorRef): Unit = {
_children -= child _children -= child
props.faultHandler.handleChildTerminated(child, children) props.faultHandler.handleChildTerminated(child, children)
} }
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause)) final def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause))
def checkReceiveTimeout() { final def checkReceiveTimeout() {
cancelReceiveTimeout() cancelReceiveTimeout()
val recvtimeout = receiveTimeout val recvtimeout = receiveTimeout
if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) { if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) {
@ -359,16 +333,16 @@ private[akka] class ActorCell(
} }
} }
def cancelReceiveTimeout() { final def cancelReceiveTimeout() {
if (futureTimeout.isDefined) { if (futureTimeout.isDefined) {
futureTimeout.get.cancel(true) futureTimeout.get.cancel(true)
futureTimeout = None futureTimeout = None
} }
} }
def clearActorContext(): Unit = setActorContext(null) final def clearActorContext(): Unit = setActorContext(null)
def setActorContext(newContext: ActorContext) { final def setActorContext(newContext: ActorContext) {
@tailrec @tailrec
def lookupAndSetSelfFields(clazz: Class[_], actor: Actor, newContext: ActorContext): Boolean = { def lookupAndSetSelfFields(clazz: Class[_], actor: Actor, newContext: ActorContext): Boolean = {
val success = try { val success = try {
@ -392,12 +366,4 @@ private[akka] class ActorCell(
if (a ne null) if (a ne null)
lookupAndSetSelfFields(a.getClass, a, newContext) lookupAndSetSelfFields(a.getClass, a, newContext)
} }
override def hashCode: Int = HashCode.hash(HashCode.SEED, uuid)
override def equals(that: Any): Boolean = {
that.isInstanceOf[ActorCell] && that.asInstanceOf[ActorCell].uuid == uuid
}
override def toString = "ActorCell[%s]".format(uuid)
} }

View file

@ -10,6 +10,9 @@ import scala.collection.immutable.Stack
import java.lang.{ UnsupportedOperationException, IllegalStateException } import java.lang.{ UnsupportedOperationException, IllegalStateException }
import akka.AkkaApplication import akka.AkkaApplication
import akka.event.ActorEventBus import akka.event.ActorEventBus
import akka.serialization.Serialization
import akka.actor.DeadLetterActorRef.SerializedDeadLetterActorRef
import java.net.InetSocketAddress
/** /**
* ActorRef is an immutable and serializable handle to an Actor. * ActorRef is an immutable and serializable handle to an Actor.
@ -41,7 +44,7 @@ import akka.event.ActorEventBus
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
abstract class ActorRef extends UntypedChannel with ReplyChannel[Any] with java.lang.Comparable[ActorRef] with Serializable { abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable {
scalaRef: ScalaActorRef scalaRef: ScalaActorRef
// Only mutable for RemoteServer in order to maintain identity across nodes // Only mutable for RemoteServer in order to maintain identity across nodes
@ -50,19 +53,29 @@ abstract class ActorRef extends UntypedChannel with ReplyChannel[Any] with java.
*/ */
def address: String def address: String
private[akka] def uuid: Uuid //TODO FIXME REMOVE THIS
/** /**
* Comparison only takes address into account. * Comparison only takes address into account.
*/ */
def compareTo(other: ActorRef) = this.address compareTo other.address def compareTo(other: ActorRef) = this.address compareTo other.address
/** /**
* Akka Java API. <p/> * Sends the specified message to the sender, i.e. fire-and-forget semantics.<p/>
* @see ask(message: AnyRef, sender: ActorRef): Future[_] * <pre>
* Uses the specified timeout (milliseconds) * actor.tell(message);
* </pre>
*/ */
def ask(message: AnyRef, timeout: Long): Future[Any] = ask(message, timeout, null) def tell(msg: Any): Unit = this.!(msg)
/**
* Java API. <p/>
* Sends the specified message to the sender, i.e. fire-and-forget
* semantics, including the sender reference if possible (not supported on
* all senders).<p/>
* <pre>
* actor.tell(message, context);
* </pre>
*/
def tell(msg: Any, sender: ActorRef): Unit = this.!(msg)(sender)
/** /**
* Akka Java API. <p/> * Akka Java API. <p/>
@ -72,20 +85,17 @@ abstract class ActorRef extends UntypedChannel with ReplyChannel[Any] with java.
* Use this method with care. In most cases it is better to use 'tell' together with the 'getContext().getSender()' to * Use this method with care. In most cases it is better to use 'tell' together with the 'getContext().getSender()' to
* implement request/response message exchanges. * implement request/response message exchanges.
* <p/> * <p/>
* If you are sending messages using <code>ask</code> then you <b>have to</b> use <code>getContext().channel().tell(...)</code> * If you are sending messages using <code>ask</code> then you <b>have to</b> use <code>getContext().sender().tell(...)</code>
* to send a reply message to the original sender. If not then the sender will block until the timeout expires. * to send a reply message to the original sender. If not then the sender will block until the timeout expires.
*/ */
def ask(message: AnyRef, timeout: Long, sender: ActorRef): Future[AnyRef] = def ask(message: AnyRef, timeout: Long): Future[AnyRef] = ?(message, Timeout(timeout)).asInstanceOf[Future[AnyRef]]
?(message, Timeout(timeout))(sender).asInstanceOf[Future[AnyRef]]
/** /**
* Akka Java API. <p/> * Forwards the message and passes the original sender actor as the sender.
* Forwards the message specified to this actor and preserves the original sender of the message * <p/>
* Works with '!' and '?'/'ask'.
*/ */
def forward(message: AnyRef, sender: ActorRef) { def forward(message: Any)(implicit context: ActorContext) = postMessageToMailbox(message, context.sender)
if (sender eq null) throw new IllegalArgumentException("The 'sender' argument to 'forward' can't be null")
else forward(message)(ForwardableChannel(sender))
}
/** /**
* Suspends the actor. It will not process messages while suspended. * Suspends the actor. It will not process messages while suspended.
@ -112,7 +122,7 @@ abstract class ActorRef extends UntypedChannel with ReplyChannel[Any] with java.
* This means that this actor will get a Terminated()-message when the provided actor * This means that this actor will get a Terminated()-message when the provided actor
* is permanently terminated. * is permanently terminated.
* *
* @returns the same ActorRef that is provided to it, to allow for cleaner invocations * @return the same ActorRef that is provided to it, to allow for cleaner invocations
*/ */
def startsMonitoring(subject: ActorRef): ActorRef //TODO FIXME REMOVE THIS def startsMonitoring(subject: ActorRef): ActorRef //TODO FIXME REMOVE THIS
@ -121,7 +131,7 @@ abstract class ActorRef extends UntypedChannel with ReplyChannel[Any] with java.
* This means that this actor will not get a Terminated()-message when the provided actor * This means that this actor will not get a Terminated()-message when the provided actor
* is permanently terminated. * is permanently terminated.
* *
* @returns the same ActorRef that is provided to it, to allow for cleaner invocations * @return the same ActorRef that is provided to it, to allow for cleaner invocations
*/ */
def stopsMonitoring(subject: ActorRef): ActorRef //TODO FIXME REMOVE THIS def stopsMonitoring(subject: ActorRef): ActorRef //TODO FIXME REMOVE THIS
@ -141,7 +151,7 @@ abstract class ActorRef extends UntypedChannel with ReplyChannel[Any] with java.
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class LocalActorRef private[akka] ( class LocalActorRef private[akka] (
app: AkkaApplication, _app: AkkaApplication,
props: Props, props: Props,
_supervisor: ActorRef, _supervisor: ActorRef,
_givenAddress: String, _givenAddress: String,
@ -156,7 +166,7 @@ class LocalActorRef private[akka] (
case other other case other other
} }
private[this] val actorCell = new ActorCell(app, this, props, _supervisor, receiveTimeout, hotswap) private[this] val actorCell = new ActorCell(_app, this, props, _supervisor, receiveTimeout, hotswap)
actorCell.start() actorCell.start()
/** /**
@ -192,7 +202,7 @@ class LocalActorRef private[akka] (
* This means that this actor will get a Terminated()-message when the provided actor * This means that this actor will get a Terminated()-message when the provided actor
* is permanently terminated. * is permanently terminated.
* *
* @returns the same ActorRef that is provided to it, to allow for cleaner invocations * @return the same ActorRef that is provided to it, to allow for cleaner invocations
*/ */
def startsMonitoring(subject: ActorRef): ActorRef = actorCell.startsMonitoring(subject) def startsMonitoring(subject: ActorRef): ActorRef = actorCell.startsMonitoring(subject)
@ -201,7 +211,7 @@ class LocalActorRef private[akka] (
* This means that this actor will not get a Terminated()-message when the provided actor * This means that this actor will not get a Terminated()-message when the provided actor
* is permanently terminated. * is permanently terminated.
* *
* @returns the same ActorRef that is provided to it, to allow for cleaner invocations * @return the same ActorRef that is provided to it, to allow for cleaner invocations
*/ */
def stopsMonitoring(subject: ActorRef): ActorRef = actorCell.stopsMonitoring(subject) def stopsMonitoring(subject: ActorRef): ActorRef = actorCell.stopsMonitoring(subject)
@ -222,28 +232,14 @@ class LocalActorRef private[akka] (
protected[akka] def sendSystemMessage(message: SystemMessage) { underlying.dispatcher.systemDispatch(underlying, message) } protected[akka] def sendSystemMessage(message: SystemMessage) { underlying.dispatcher.systemDispatch(underlying, message) }
protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = protected[akka] def postMessageToMailbox(message: Any, sender: ActorRef): Unit = actorCell.postMessageToMailbox(message, sender)
actorCell.postMessageToMailbox(message, channel)
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( def ?(message: Any)(implicit timeout: Timeout): Future[Any] = actorCell.provider.ask(message, this, timeout)
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = {
actorCell.postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, channel)
}
protected[akka] def handleFailure(fail: Failed): Unit = actorCell.handleFailure(fail) protected[akka] override def restart(cause: Throwable): Unit = actorCell.restart(cause)
protected[akka] def restart(cause: Throwable): Unit = actorCell.restart(cause)
// ========= PRIVATE FUNCTIONS =========
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = { private def writeReplace(): AnyRef = actorCell.provider.serialize(this)
// TODO: this was used to really send LocalActorRef across the network, which is broken now
val inetaddr = app.defaultAddress
SerializedActorRef(uuid, address, inetaddr.getAddress.getHostAddress, inetaddr.getPort)
}
} }
/** /**
@ -251,7 +247,7 @@ class LocalActorRef private[akka] (
* There are implicit conversions in ../actor/Implicits.scala * There are implicit conversions in ../actor/Implicits.scala
* from ActorRef -> ScalaActorRef and back * from ActorRef -> ScalaActorRef and back
*/ */
trait ScalaActorRef extends ReplyChannel[Any] { ref: ActorRef trait ScalaActorRef { ref: ActorRef
protected[akka] def sendSystemMessage(message: SystemMessage): Unit protected[akka] def sendSystemMessage(message: SystemMessage): Unit
@ -269,28 +265,16 @@ trait ScalaActorRef extends ReplyChannel[Any] { ref: ActorRef ⇒
* </pre> * </pre>
* <p/> * <p/>
*/ */
def !(message: Any)(implicit channel: UntypedChannel): Unit = postMessageToMailbox(message, channel) def !(message: Any)(implicit sender: ActorRef = null): Unit = postMessageToMailbox(message, sender)
/** /**
* Sends a message asynchronously, returning a future which may eventually hold the reply. * Sends a message asynchronously, returning a future which may eventually hold the reply.
*/ */
def ?(message: Any)(implicit channel: UntypedChannel, timeout: Timeout): Future[Any] = postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, channel) def ?(message: Any)(implicit timeout: Timeout): Future[Any]
def ?(message: Any, timeout: Timeout)(implicit channel: UntypedChannel): Future[Any] = ?(message)(channel, timeout) def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout)
/** protected[akka] def postMessageToMailbox(message: Any, sender: ActorRef): Unit
* Forwards the message and passes the original sender actor as the sender.
* <p/>
* Works with '!' and '?'/'ask'.
*/
def forward(message: Any)(implicit forwardable: ForwardableChannel) = postMessageToMailbox(message, forwardable.channel)
protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any]
protected[akka] def restart(cause: Throwable): Unit protected[akka] def restart(cause: Throwable): Unit
} }
@ -299,10 +283,11 @@ trait ScalaActorRef extends ReplyChannel[Any] { ref: ActorRef ⇒
* Memento pattern for serializing ActorRefs transparently * Memento pattern for serializing ActorRefs transparently
*/ */
case class SerializedActorRef(uuid: Uuid, address: String, hostname: String, port: Int) { case class SerializedActorRef(address: String, hostname: String, port: Int) {
import akka.serialization.Serialization.app import akka.serialization.Serialization.app
def this(address: String, inet: InetSocketAddress) = this(address, inet.getAddress.getHostAddress, inet.getPort)
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
def readResolve(): AnyRef = { def readResolve(): AnyRef = {
if (app.value eq null) throw new IllegalStateException( if (app.value eq null) throw new IllegalStateException(
@ -320,31 +305,24 @@ case class SerializedActorRef(uuid: Uuid, address: String, hostname: String, por
*/ */
trait UnsupportedActorRef extends ActorRef with ScalaActorRef { trait UnsupportedActorRef extends ActorRef with ScalaActorRef {
private[akka] def uuid: Uuid = unsupported private[akka] final val uuid: akka.actor.Uuid = newUuid()
def startsMonitoring(actorRef: ActorRef): ActorRef = unsupported def startsMonitoring(actorRef: ActorRef): ActorRef = actorRef
def stopsMonitoring(actorRef: ActorRef): ActorRef = unsupported def stopsMonitoring(actorRef: ActorRef): ActorRef = actorRef
def suspend(): Unit = unsupported def suspend(): Unit = ()
def resume(): Unit = unsupported def resume(): Unit = ()
protected[akka] def restart(cause: Throwable): Unit = unsupported protected[akka] def restart(cause: Throwable): Unit = ()
def stop(): Unit = unsupported protected[akka] def sendSystemMessage(message: SystemMessage): Unit = ()
def address: String = unsupported protected[akka] def postMessageToMailbox(msg: Any, sender: ActorRef): Unit = ()
def isShutdown = false def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName))
protected[akka] def sendSystemMessage(message: SystemMessage) {}
protected[akka] def postMessageToMailbox(msg: Any, channel: UntypedChannel) {}
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(msg: Any, timeout: Timeout, channel: UntypedChannel): Future[Any] = unsupported
private def unsupported = throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName))
} }
/** /**
@ -352,7 +330,7 @@ trait UnsupportedActorRef extends ActorRef with ScalaActorRef {
*/ */
trait MinimalActorRef extends ActorRef with ScalaActorRef { trait MinimalActorRef extends ActorRef with ScalaActorRef {
private[akka] val uuid: Uuid = new com.eaio.uuid.UUID(0L, 0L) //Nil UUID private[akka] val uuid: Uuid = newUuid()
def address = uuid.toString def address = uuid.toString
def startsMonitoring(actorRef: ActorRef): ActorRef = actorRef def startsMonitoring(actorRef: ActorRef): ActorRef = actorRef
@ -368,52 +346,70 @@ trait MinimalActorRef extends ActorRef with ScalaActorRef {
protected[akka] def sendSystemMessage(message: SystemMessage) {} protected[akka] def sendSystemMessage(message: SystemMessage) {}
protected[akka] def postMessageToMailbox(msg: Any, channel: UntypedChannel) {} protected[akka] def postMessageToMailbox(msg: Any, sender: ActorRef) {}
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(msg: Any, timeout: Timeout, channel: UntypedChannel): Future[Any] = unsupported def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName))
private def unsupported = throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName))
} }
case class DeadLetter(message: Any, channel: UntypedChannel) case class DeadLetter(message: Any, sender: ActorRef)
class DeadLetterActorRef(app: AkkaApplication) extends MinimalActorRef { object DeadLetterActorRef {
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
@throws(classOf[java.io.ObjectStreamException])
private def readResolve(): AnyRef = Serialization.app.value.deadLetters
}
val serialized = new SerializedDeadLetterActorRef
}
class DeadLetterActorRef(val app: AkkaApplication) extends MinimalActorRef {
val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher) val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher)
override val address: String = "akka:internal:DeadLetterActorRef" override val address: String = "akka:internal:DeadLetterActorRef"
override def isShutdown(): Boolean = true override def isShutdown(): Boolean = true
protected[akka] override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = app.mainbus.publish(DeadLetter(message, channel)) protected[akka] override def postMessageToMailbox(message: Any, sender: ActorRef): Unit =
app.mainbus.publish(DeadLetter(message, sender))
protected[akka] override def postMessageToMailboxAndCreateFutureResultWithTimeout( override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = {
message: Any, app.mainbus.publish(DeadLetter(message, this))
timeout: Timeout, brokenPromise
channel: UntypedChannel): Future[Any] = { app.mainbus.publish(DeadLetter(message, channel)); brokenPromise } }
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = DeadLetterActorRef.serialized
} }
abstract class AskActorRef(promise: Promise[Any], app: AkkaApplication) extends MinimalActorRef { abstract class AskActorRef(protected val app: AkkaApplication)(timeout: Timeout = app.AkkaConfig.ActorTimeout, dispatcher: MessageDispatcher = app.dispatcher) extends MinimalActorRef {
final val result = new DefaultPromise[Any](timeout)(dispatcher)
promise onComplete { _ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() } {
promise onTimeout { _ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() } val callback: Future[Any] Unit = { _ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
result onComplete callback
result onTimeout callback
}
protected def whenDone(): Unit protected def whenDone(): Unit
override protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = message match { protected[akka] override def postMessageToMailbox(message: Any, sender: ActorRef): Unit = message match {
case akka.actor.Status.Success(r) promise.completeWithResult(r) case Status.Success(r) result.completeWithResult(r)
case akka.actor.Status.Failure(f) promise.completeWithException(f) case Status.Failure(f) result.completeWithException(f)
case other promise.completeWithResult(other) case other result.completeWithResult(other)
} }
override protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( protected[akka] override def sendSystemMessage(message: SystemMessage): Unit = message match {
message: Any, case _: Terminate stop()
timeout: Timeout, case _
channel: UntypedChannel): Future[Any] = {
postMessageToMailbox(message, channel)
promise
} }
override def isShutdown = promise.isCompleted || promise.isExpired override def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
new KeptPromise[Any](Left(new UnsupportedOperationException("Ask/? is not supported for %s".format(getClass.getName))))(dispatcher)
override def stop(): Unit = if (!isShutdown) promise.completeWithException(new ActorKilledException("Stopped")) override def isShutdown = result.isCompleted || result.isExpired
override def stop(): Unit = if (!isShutdown) result.completeWithException(new ActorKilledException("Stopped"))
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = app.provider.serialize(this)
} }

View file

@ -25,12 +25,19 @@ trait ActorRefProvider {
def actorFor(address: String): Option[ActorRef] def actorFor(address: String): Option[ActorRef]
/**
* What deployer will be used to resolve deployment configuration?
*/
private[akka] def deployer: Deployer
private[akka] def actorOf(props: Props, supervisor: ActorRef, address: String, systemService: Boolean): ActorRef private[akka] def actorOf(props: Props, supervisor: ActorRef, address: String, systemService: Boolean): ActorRef
private[akka] def evict(address: String): Boolean private[akka] def evict(address: String): Boolean
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef]
private[akka] def serialize(actor: ActorRef): SerializedActorRef
private[akka] def createDeathWatch(): DeathWatch private[akka] def createDeathWatch(): DeathWatch
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any]
@ -90,6 +97,8 @@ class ActorRefProviderException(message: String) extends AkkaException(message)
*/ */
class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
private[akka] val deployer: Deployer = new Deployer(app)
val terminationFuture = new DefaultPromise[AkkaApplication.ExitStatus](Timeout.never)(app.dispatcher) val terminationFuture = new DefaultPromise[AkkaApplication.ExitStatus](Timeout.never)(app.dispatcher)
val log = Logging(app.mainbus, this) val log = Logging(app.mainbus, this)
@ -98,11 +107,18 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
* receive only Supervise/ChildTerminated system messages or Failure message. * receive only Supervise/ChildTerminated system messages or Failure message.
*/ */
private[akka] val theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = new UnsupportedActorRef { private[akka] val theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = new UnsupportedActorRef {
@volatile
var stopped = false
override def address = app.name + ":BubbleWalker" override def address = app.name + ":BubbleWalker"
override def toString = address override def toString = address
protected[akka] override def postMessageToMailbox(msg: Any, channel: UntypedChannel) { def stop() = stopped = true
def isShutdown = stopped
protected[akka] override def postMessageToMailbox(msg: Any, sender: ActorRef) {
msg match { msg match {
case Failed(child, ex) child.stop() case Failed(child, ex) child.stop()
case ChildTerminated(child) terminationFuture.completeWithResult(AkkaApplication.Stopped) case ChildTerminated(child) terminationFuture.completeWithResult(AkkaApplication.Stopped)
@ -144,7 +160,7 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
actors.putIfAbsent(address, newFuture) match { actors.putIfAbsent(address, newFuture) match {
case null case null
val actor: ActorRef = try { val actor: ActorRef = try {
(if (systemService) None else app.deployer.lookupDeploymentFor(address)) match { // see if the deployment already exists, if so use it, if not create actor (if (systemService) None else deployer.lookupDeployment(address)) match { // see if the deployment already exists, if so use it, if not create actor
// create a local actor // create a local actor
case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, _, DeploymentConfig.LocalScope)) case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, _, DeploymentConfig.LocalScope))
@ -152,6 +168,7 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
// create a routed actor ref // create a routed actor ref
case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, _, DeploymentConfig.LocalScope)) case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, _, DeploymentConfig.LocalScope))
val routerFactory: () Router = DeploymentConfig.routerTypeFor(routerType) match { val routerFactory: () Router = DeploymentConfig.routerTypeFor(routerType) match {
case RouterType.Direct () new DirectRouter case RouterType.Direct () new DirectRouter
case RouterType.Random () new RandomRouter case RouterType.Random () new RandomRouter
@ -169,7 +186,7 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, address) actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, address)
case _ throw new Exception("Don't know how to create this actor ref! Why?") case unknown throw new Exception("Don't know how to create this actor ref! Why? Got: " + unknown)
} }
} catch { } catch {
case e: Exception case e: Exception
@ -206,10 +223,11 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
// val localOnly = props.localOnly // val localOnly = props.localOnly
// if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props) // if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props)
// else new RoutedActorRef(props, address) // else new RoutedActorRef(props, address)
new RoutedActorRef(props, address) new RoutedActorRef(app, props, address)
} }
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(actor.address) private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(actor.address)
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(actor.address, app.defaultAddress)
private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch
@ -217,12 +235,11 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
import akka.dispatch.{ Future, Promise, DefaultPromise } import akka.dispatch.{ Future, Promise, DefaultPromise }
(if (within == null) app.AkkaConfig.ActorTimeout else within) match { (if (within == null) app.AkkaConfig.ActorTimeout else within) match {
case t if t.duration.length <= 0 new DefaultPromise[Any](0)(app.dispatcher) //Abort early if nonsensical timeout case t if t.duration.length <= 0 new DefaultPromise[Any](0)(app.dispatcher) //Abort early if nonsensical timeout
case other case t
val result = new DefaultPromise[Any](other)(app.dispatcher) val a = new AskActorRef(app)(timeout = t) { def whenDone() = actors.remove(this) }
val a = new AskActorRef(result, app) { def whenDone() = actors.remove(this) }
assert(actors.putIfAbsent(a.address, a) eq null) //If this fails, we're in deep trouble assert(actors.putIfAbsent(a.address, a) eq null) //If this fails, we're in deep trouble
recipient.tell(message, a) recipient.tell(message, a)
result a.result
} }
} }
} }

View file

@ -1,146 +0,0 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.actor
/*
* This package is just used to hide the tryTell method from the Scala parts:
* it will be public to javac's eyes by virtue of §5.2 of the SLS.
*/
package japi {
trait Channel[-T] { self: akka.actor.Channel[T]
private[japi] def tryTell(msg: T): Boolean = {
try {
self.!(msg)(NullChannel)
true
} catch {
case _: Exception false
}
}
}
}
/**
* Abstraction for unification of sender and senderFuture for later reply.
* Can be stored away and used at a later point in time.
*
* The possible reply channel which can be passed into ! and tryTell is always
* untyped, as there is no way to utilize its real static type without
* requiring runtime-costly manifests.
*/
trait Channel[-T] extends japi.Channel[T] {
/**
* Scala API. <p/>
* Sends the specified message to the channel.
*/
def !(msg: T)(implicit sender: UntypedChannel): Unit
/**
* Scala and Java API. <p/>
* Try to send the specified message to the channel, i.e. fire-and-forget
* semantics, including the sender reference if possible (not supported on
* all channels).<p/>
* From Java:
* <pre>
* actor.tryTell(message);
* actor.tryTell(message, context);
* </pre>
* <p/>
* From Scala:
* <pre>
* actor tryTell message
* actor.tryTell(message)(sender)
* </pre>
*/
def tryTell(msg: T)(implicit sender: UntypedChannel): Boolean = {
try {
this.!(msg)(sender)
true
} catch {
case _: Exception false
}
}
/**
* Try to send an exception. Not all channel types support this, one notable
* positive example is Future. Failure to send is silent.
*
* @return whether sending was successful
*/
def sendException(ex: Throwable): Boolean = false
/**
* Sends the specified message to the channel, i.e. fire-and-forget semantics.<p/>
* <pre>
* actor.tell(message);
* </pre>
*/
def tell(msg: T): Unit = this.!(msg)
/**
* Java API. <p/>
* Sends the specified message to the channel, i.e. fire-and-forget
* semantics, including the sender reference if possible (not supported on
* all channels).<p/>
* <pre>
* actor.tell(message, context);
* </pre>
*/
def tell(msg: T, sender: UntypedChannel): Unit = this.!(msg)(sender)
}
/**
* This trait marks a channel that a priori does have sending capability,
* i.e. ! is not guaranteed to fail (e.g. NullChannel would be a
* counter-example).
*/
trait AvailableChannel[-T] extends Channel[T]
/**
* This trait marks a channel which is capable of sending exceptions.
*/
trait ExceptionChannel[-T] extends AvailableChannel[T]
/**
* This trait marks a channel which carries reply information when tell()ing.
*/
trait ReplyChannel[-T] extends AvailableChannel[T]
/**
* All channels used in conjunction with MessageInvocation are untyped by
* design, so make this explicit.
*/
trait UntypedChannel extends Channel[Any]
object UntypedChannel {
implicit def senderOption2Channel(sender: Option[ActorRef]): UntypedChannel =
sender match {
case Some(actor) actor
case None NullChannel
}
implicit final val default: UntypedChannel = NullChannel
}
/**
* Default channel when none available.
*/
case object NullChannel extends UntypedChannel {
def !(msg: Any)(implicit channel: UntypedChannel) {
throw new IllegalActorStateException("""
No sender in scope, can't reply.
You have probably:
1. Sent a message to an Actor from an instance that is NOT an Actor.
2. Invoked a method on an TypedActor from an instance NOT an TypedActor.
You may want to have a look at tryTell for a variant returning a Boolean""")
}
override def tryTell(msg: Any)(implicit channel: UntypedChannel): Boolean = false
}
/**
* Wraps a forwardable channel. Used implicitly by ScalaActorRef.forward
*/
case class ForwardableChannel(val channel: UntypedChannel)

View file

@ -13,6 +13,7 @@ import akka.actor.DeploymentConfig._
import akka.{ AkkaException, AkkaApplication } import akka.{ AkkaException, AkkaApplication }
import akka.config.{ Configuration, ConfigurationException } import akka.config.{ Configuration, ConfigurationException }
import akka.util.Duration import akka.util.Duration
import java.net.InetSocketAddress
trait ActorDeployer { trait ActorDeployer {
private[akka] def init(deployments: Seq[Deploy]): Unit private[akka] def init(deployments: Seq[Deploy]): Unit
@ -36,10 +37,8 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer {
val deploymentConfig = new DeploymentConfig(app) val deploymentConfig = new DeploymentConfig(app)
val log = Logging(app.mainbus, this) val log = Logging(app.mainbus, this)
// val defaultAddress = Node(Config.nodename) val instance: ActorDeployer = {
val deployer = new LocalDeployer()
lazy val instance: ActorDeployer = {
val deployer = if (app.reflective.ClusterModule.isEnabled) app.reflective.ClusterModule.clusterDeployer else LocalDeployer
deployer.init(deploymentsInConfig) deployer.init(deploymentsInConfig)
deployer deployer
} }
@ -73,25 +72,8 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer {
} }
} }
private[akka] def lookupDeploymentFor(address: String): Option[Deploy] = { private[akka] def lookupDeploymentFor(address: String): Option[Deploy] =
val deployment_? = instance.lookupDeploymentFor(address) instance.lookupDeploymentFor(address)
if (deployment_?.isDefined && (deployment_?.get ne null)) deployment_?
else {
val newDeployment = try {
lookupInConfig(address)
} catch {
case e: ConfigurationException
log.error(e, e.getMessage) //TODO FIXME I do not condone log AND rethrow
throw e
}
newDeployment match {
case None | Some(null) None
case Some(d) deploy(d); newDeployment // deploy and cache it
}
}
}
private[akka] def deploymentsInConfig: List[Deploy] = { private[akka] def deploymentsInConfig: List[Deploy] = {
for { for {
@ -249,7 +231,8 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer {
case e: Exception raiseRemoteNodeParsingError() case e: Exception raiseRemoteNodeParsingError()
} }
if (port == 0) raiseRemoteNodeParsingError() if (port == 0) raiseRemoteNodeParsingError()
RemoteAddress(hostname, port) val inet = new InetSocketAddress(hostname, port) //FIXME switch to non-ip-tied
RemoteAddress(Option(inet.getAddress).map(_.getHostAddress).getOrElse(hostname), inet.getPort)
} }
} }
@ -341,20 +324,14 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer {
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
object LocalDeployer extends ActorDeployer { class LocalDeployer extends ActorDeployer {
private val deployments = new ConcurrentHashMap[String, Deploy] private val deployments = new ConcurrentHashMap[String, Deploy]
private[akka] def init(deployments: Seq[Deploy]) { private[akka] def init(deployments: Seq[Deploy]): Unit = deployments foreach deploy // deploy
deployments foreach (deploy(_)) // deploy
}
private[akka] def shutdown() { private[akka] def shutdown(): Unit = deployments.clear() //TODO do something else/more?
deployments.clear() //TODO do something else/more?
}
private[akka] def deploy(deployment: Deploy) { private[akka] def deploy(deployment: Deploy): Unit = deployments.putIfAbsent(deployment.address, deployment)
deployments.putIfAbsent(deployment.address, deployment)
}
private[akka] def lookupDeploymentFor(address: String): Option[Deploy] = Option(deployments.get(address)) private[akka] def lookupDeploymentFor(address: String): Option[Deploy] = Option(deployments.get(address))
} }

View file

@ -480,7 +480,7 @@ trait FSM[S, D] extends ListenerManagement {
timeoutFuture = None timeoutFuture = None
} }
generation += 1 generation += 1
processMsg(value, channel) processMsg(value, sender)
} }
} }
@ -504,7 +504,7 @@ trait FSM[S, D] extends ListenerManagement {
nextState.stopReason match { nextState.stopReason match {
case None makeTransition(nextState) case None makeTransition(nextState)
case _ case _
nextState.replies.reverse foreach { r channel ! r } nextState.replies.reverse foreach { r sender ! r }
terminate(nextState) terminate(nextState)
self.stop() self.stop()
} }
@ -514,7 +514,7 @@ trait FSM[S, D] extends ListenerManagement {
if (!stateFunctions.contains(nextState.stateName)) { if (!stateFunctions.contains(nextState.stateName)) {
terminate(stay withStopReason Failure("Next state %s does not exist".format(nextState.stateName))) terminate(stay withStopReason Failure("Next state %s does not exist".format(nextState.stateName)))
} else { } else {
nextState.replies.reverse foreach { r channel ! r } nextState.replies.reverse foreach { r sender ! r }
if (currentState.stateName != nextState.stateName) { if (currentState.stateName != nextState.stateName) {
handleTransition(currentState.stateName, nextState.stateName) handleTransition(currentState.stateName, nextState.stateName)
notifyListeners(Transition(self, currentState.stateName, nextState.stateName)) notifyListeners(Transition(self, currentState.stateName, nextState.stateName))
@ -601,7 +601,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
val srcstr = source match { val srcstr = source match {
case s: String s case s: String s
case Timer(name, _, _, _) "timer " + name case Timer(name, _, _, _) "timer " + name
case c: UntypedChannel c.toString case a: ActorRef a.toString
case _ "unknown" case _ "unknown"
} }
log.debug("processing " + event + " from " + srcstr) log.debug("processing " + event + " from " + srcstr)

View file

@ -21,8 +21,8 @@ case class ChildRestartStats(var maxNrOfRetriesCount: Int = 0, var restartTimeWi
private def retriesInWindowOkay(retries: Int, window: Int): Boolean = { private def retriesInWindowOkay(retries: Int, window: Int): Boolean = {
/* /*
* Simple window algorithm: window is kept open for a certain time * Simple window algorithm: window is kept open for a certain time
* after a restart and if enough restarts happen during this time, it * after a restart and if enough restarts happen during this time, it
* denies. Otherwise window closes and the scheme starts over. * denies. Otherwise window closes and the scheme starts over.
*/ */
val retriesDone = maxNrOfRetriesCount + 1 val retriesDone = maxNrOfRetriesCount + 1
@ -181,7 +181,7 @@ case class AllForOneStrategy(decider: FaultHandlingStrategy.Decider,
if (withinTimeRange < 0) None else Some(withinTimeRange)) if (withinTimeRange < 0) None else Some(withinTimeRange))
/* /*
* this is a performance optimization to avoid re-allocating the pairs upon * this is a performance optimization to avoid re-allocating the pairs upon
* every call to requestRestartPermission, assuming that strategies are shared * every call to requestRestartPermission, assuming that strategies are shared
* across actors and thus this field does not take up much space * across actors and thus this field does not take up much space
*/ */
@ -238,7 +238,7 @@ case class OneForOneStrategy(decider: FaultHandlingStrategy.Decider,
if (withinTimeRange < 0) None else Some(withinTimeRange)) if (withinTimeRange < 0) None else Some(withinTimeRange))
/* /*
* this is a performance optimization to avoid re-allocating the pairs upon * this is a performance optimization to avoid re-allocating the pairs upon
* every call to requestRestartPermission, assuming that strategies are shared * every call to requestRestartPermission, assuming that strategies are shared
* across actors and thus this field does not take up much space * across actors and thus this field does not take up much space
*/ */

View file

@ -28,7 +28,7 @@ object Props {
case _ Escalate case _ Escalate
} }
final val defaultFaultHandler: FaultHandlingStrategy = OneForOneStrategy(defaultDecider, None, None) final val defaultFaultHandler: FaultHandlingStrategy = OneForOneStrategy(defaultDecider, None, None)
final val defaultSupervisor: Option[ActorRef] = None
final val noHotSwap: Stack[Actor.Receive] = Stack.empty final val noHotSwap: Stack[Actor.Receive] = Stack.empty
final val randomAddress: String = "" final val randomAddress: String = ""

View file

@ -317,7 +317,7 @@ class TypedActor(val app: AkkaApplication) {
case Props.`defaultTimeout` app.AkkaConfig.ActorTimeout case Props.`defaultTimeout` app.AkkaConfig.ActorTimeout
case x x case x x
} }
val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)(timeout)).asInstanceOf[T] val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar, timeout)).asInstanceOf[T]
proxyVar.set(proxy) // Chicken and egg situation we needed to solve, set the proxy so that we can set the self-reference inside each receive proxyVar.set(proxy) // Chicken and egg situation we needed to solve, set the proxy so that we can set the self-reference inside each receive
val ref = supervisor.actorOf(props, address) val ref = supervisor.actorOf(props, address)
actorVar.set(ref) //Make sure the InvocationHandler gets ahold of the actor reference, this is not a problem since the proxy hasn't escaped this method yet actorVar.set(ref) //Make sure the InvocationHandler gets ahold of the actor reference, this is not a problem since the proxy hasn't escaped this method yet
@ -327,11 +327,6 @@ class TypedActor(val app: AkkaApplication) {
private[akka] def extractInterfaces(clazz: Class[_]): Array[Class[_]] = if (clazz.isInterface) Array[Class[_]](clazz) else clazz.getInterfaces private[akka] def extractInterfaces(clazz: Class[_]): Array[Class[_]] = if (clazz.isInterface) Array[Class[_]](clazz) else clazz.getInterfaces
private[akka] class TypedActor[R <: AnyRef, T <: R](val proxyVar: AtomVar[R], createInstance: T) extends Actor { private[akka] class TypedActor[R <: AnyRef, T <: R](val proxyVar: AtomVar[R], createInstance: T) extends Actor {
// FIXME TypedActor register/unregister on postStop/preStart
// override def preStart = app.registry.registerTypedActor(self, proxyVar.get) //Make sure actor registry knows about this actor
// override def postStop = app.registry.unregisterTypedActor(self, proxyVar.get)
val me = createInstance val me = createInstance
def receive = { def receive = {
case m: MethodCall case m: MethodCall
@ -339,13 +334,23 @@ class TypedActor(val app: AkkaApplication) {
TypedActor.appReference set app TypedActor.appReference set app
try { try {
if (m.isOneWay) m(me) if (m.isOneWay) m(me)
else if (m.returnsFuture_?) { else {
channel match { val s = sender
case p: ActorPromise p completeWith m(me).asInstanceOf[Future[Any]] try {
case _ throw new IllegalStateException("Future-returning TypedActor didn't use ?/ask so cannot reply") if (m.returnsFuture_?) {
m(me).asInstanceOf[Future[Any]] onComplete {
_.value.get match {
case Left(f) s ! Status.Failure(f)
case Right(r) s ! r
}
}
} else {
s ! m(me)
}
} catch {
case e: Exception s ! Status.Failure(e)
} }
} else channel ! m(me) }
} finally { } finally {
TypedActor.selfReference set null TypedActor.selfReference set null
TypedActor.appReference set null TypedActor.appReference set null
@ -353,7 +358,7 @@ class TypedActor(val app: AkkaApplication) {
} }
} }
private[akka] class TypedActorInvocationHandler(actorVar: AtomVar[ActorRef])(implicit timeout: Timeout) extends InvocationHandler { private[akka] class TypedActorInvocationHandler(actorVar: AtomVar[ActorRef], timeout: Timeout) extends InvocationHandler {
def actor = actorVar.get def actor = actorVar.get
def invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = method.getName match { def invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = method.getName match {
@ -363,17 +368,15 @@ class TypedActor(val app: AkkaApplication) {
case _ case _
MethodCall(app, method, args) match { MethodCall(app, method, args) match {
case m if m.isOneWay actor ! m; null //Null return value case m if m.isOneWay actor ! m; null //Null return value
case m if m.returnsFuture_? actor ? m case m if m.returnsFuture_? actor.?(m, timeout)
case m if m.returnsJOption_? || m.returnsOption_? case m if m.returnsJOption_? || m.returnsOption_?
val f = actor ? m val f = actor.?(m, timeout)
try { f.await } catch { case _: FutureTimeoutException } (try { f.await.value } catch { case _: FutureTimeoutException None }) match {
f.value match {
case None | Some(Right(null)) if (m.returnsJOption_?) JOption.none[Any] else None case None | Some(Right(null)) if (m.returnsJOption_?) JOption.none[Any] else None
case Some(Right(joption: AnyRef)) joption case Some(Right(joption: AnyRef)) joption
case Some(Left(ex)) throw ex case Some(Left(ex)) throw ex
} }
case m case m (actor.?(m, timeout)).get.asInstanceOf[AnyRef]
(actor ? m).get.asInstanceOf[AnyRef]
} }
} }
} }

View file

@ -21,7 +21,7 @@ import akka.dispatch.{ MessageDispatcher, Promise }
* *
* if (msg.equals("UseReply")) { * if (msg.equals("UseReply")) {
* // Reply to original sender of message using the 'reply' method * // Reply to original sender of message using the 'reply' method
* getContext().getChannel().tell(msg + ":" + getSelf().getAddress()); * getContext().getSender().tell(msg + ":" + getSelf().getAddress());
* *
* } else if (msg.equals("UseSender") && getSender().isDefined()) { * } else if (msg.equals("UseSender") && getSender().isDefined()) {
* // Reply to original sender of message using the sender reference * // Reply to original sender of message using the sender reference
@ -67,12 +67,7 @@ abstract class UntypedActor extends Actor {
* The reference sender Actor of the last received message. * The reference sender Actor of the last received message.
* Is defined if the message was sent from another Actor, else None. * Is defined if the message was sent from another Actor, else None.
*/ */
def getSender: ActorRef = sender def getSender(): ActorRef = sender
/**
* Abstraction for unification of sender and senderFuture for later reply
*/
def getChannel: UntypedChannel = channel
/** /**
* Gets the current receive timeout * Gets the current receive timeout

View file

@ -23,12 +23,9 @@ package object actor {
} }
implicit def future2actor[T](f: akka.dispatch.Future[T]) = new { implicit def future2actor[T](f: akka.dispatch.Future[T]) = new {
def pipeTo(channel: Channel[T]): this.type = { def pipeTo(actor: ActorRef): this.type = {
if (f.isCompleted) { def send(f: akka.dispatch.Future[T]) { f.value.get.fold(f actor ! Status.Failure(f), r actor ! r) }
f.value.get.fold(channel.sendException(_), channel.tryTell(_)) if (f.isCompleted) send(f) else f onComplete send
} else {
f onComplete { _.value.get.fold(channel.sendException(_), channel.tryTell(_)) }
}
this this
} }
} }

View file

@ -17,7 +17,7 @@ import scala.annotation.tailrec
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
final case class Envelope(val message: Any, val channel: UntypedChannel) { final case class Envelope(val message: Any, val sender: ActorRef) {
if (message.isInstanceOf[AnyRef] && (message.asInstanceOf[AnyRef] eq null)) throw new InvalidMessageException("Message is null") if (message.isInstanceOf[AnyRef] && (message.asInstanceOf[AnyRef] eq null)) throw new InvalidMessageException("Message is null")
} }
@ -107,7 +107,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable
object DeadLetterMailbox extends Mailbox(null) { object DeadLetterMailbox extends Mailbox(null) {
becomeClosed() becomeClosed()
override def dispatcher = null //MessageDispatcher.this override def dispatcher = null //MessageDispatcher.this
override def enqueue(envelope: Envelope) { envelope.channel sendException new ActorKilledException("Actor has been stopped") } override def enqueue(envelope: Envelope) = ()
override def dequeue() = null override def dequeue() = null
override def systemEnqueue(handle: SystemMessage): Unit = () override def systemEnqueue(handle: SystemMessage): Unit = ()
override def systemDrain(): SystemMessage = null override def systemDrain(): SystemMessage = null

View file

@ -37,7 +37,7 @@ class BalancingDispatcher(
_timeoutMs: Long) _timeoutMs: Long)
extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) { extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
private val buddies = new ConcurrentSkipListSet[ActorCell](new Comparator[ActorCell] { def compare(a: ActorCell, b: ActorCell) = a.uuid.compareTo(b.uuid) }) //new ConcurrentLinkedQueue[ActorCell]() private val buddies = new ConcurrentSkipListSet[ActorCell](new Comparator[ActorCell] { def compare(a: ActorCell, b: ActorCell) = System.identityHashCode(a) - System.identityHashCode(b) }) //new ConcurrentLinkedQueue[ActorCell]()
protected val messageQueue: MessageQueue = mailboxType match { protected val messageQueue: MessageQueue = mailboxType match {
case u: UnboundedMailbox new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { case u: UnboundedMailbox new QueueBasedMessageQueue with UnboundedMessageQueueSemantics {

View file

@ -65,7 +65,7 @@ import akka.AkkaApplication
*/ */
class Dispatcher( class Dispatcher(
_app: AkkaApplication, _app: AkkaApplication,
_name: String, val name: String,
val throughput: Int, val throughput: Int,
val throughputDeadlineTime: Int, val throughputDeadlineTime: Int,
val mailboxType: MailboxType, val mailboxType: MailboxType,
@ -73,8 +73,6 @@ class Dispatcher(
val timeoutMs: Long) val timeoutMs: Long)
extends MessageDispatcher(_app) { extends MessageDispatcher(_app) {
val name = "akka:event-driven:dispatcher:" + _name
protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name) protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name)
protected[akka] val executorService = new AtomicReference[ExecutorService](new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService)) protected[akka] val executorService = new AtomicReference[ExecutorService](new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService))
@ -153,4 +151,4 @@ abstract class PriorityGenerator extends java.util.Comparator[Envelope] {
final def compare(thisMessage: Envelope, thatMessage: Envelope): Int = final def compare(thisMessage: Envelope, thatMessage: Envelope): Int =
gen(thisMessage.message) - gen(thatMessage.message) gen(thisMessage.message) - gen(thatMessage.message)
} }

View file

@ -61,7 +61,7 @@ class Dispatchers(val app: AkkaApplication) {
*/ */
def newPinnedDispatcher(actor: LocalActorRef) = actor match { def newPinnedDispatcher(actor: LocalActorRef) = actor match {
case null new PinnedDispatcher(app, null, "anon", MailboxType, DispatcherShutdownMillis) case null new PinnedDispatcher(app, null, "anon", MailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.underlying.uuid.toString, MailboxType, DispatcherShutdownMillis) case some new PinnedDispatcher(app, some.underlying, some.address, MailboxType, DispatcherShutdownMillis)
} }
/** /**
@ -72,7 +72,7 @@ class Dispatchers(val app: AkkaApplication) {
*/ */
def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match { def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match {
case null new PinnedDispatcher(app, null, "anon", mailboxType, DispatcherShutdownMillis) case null new PinnedDispatcher(app, null, "anon", mailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.underlying.uuid.toString, mailboxType, DispatcherShutdownMillis) case some new PinnedDispatcher(app, some.underlying, some.address, mailboxType, DispatcherShutdownMillis)
} }
/** /**

View file

@ -7,7 +7,7 @@ package akka.dispatch
import akka.AkkaException import akka.AkkaException
import akka.event.Logging.Error import akka.event.Logging.Error
import akka.actor.{ UntypedChannel, Timeout, ExceptionChannel } import akka.actor.Timeout
import scala.Option import scala.Option
import akka.japi.{ Procedure, Function JFunc, Option JOption } import akka.japi.{ Procedure, Function JFunc, Option JOption }
@ -359,6 +359,41 @@ object Future {
// TODO make variant of flow(timeout)(body) which does NOT break type inference // TODO make variant of flow(timeout)(body) which does NOT break type inference
/**
* Assures that any Future tasks initiated in the current thread will be
* executed asynchronously, including any tasks currently queued to be
* executed in the current thread. This is needed if the current task may
* block, causing delays in executing the remaining tasks which in some
* cases may cause a deadlock.
*
* Note: Calling 'Future.await' will automatically trigger this method.
*
* For example, in the following block of code the call to 'latch.open'
* might not be executed until after the call to 'latch.await', causing
* a deadlock. By adding 'Future.blocking()' the call to 'latch.open'
* will instead be dispatched separately from the current block, allowing
* it to be run in parallel:
* <pre>
* val latch = new StandardLatch
* val future = Future() map { _
* Future.blocking()
* val nested = Future()
* nested foreach (_ latch.open)
* latch.await
* }
* </pre>
*/
def blocking()(implicit dispatcher: MessageDispatcher): Unit =
_taskStack.get match {
case Some(taskStack) if taskStack.nonEmpty
val tasks = taskStack.elems
taskStack.clear()
_taskStack set None
dispatchTask(() _taskStack.get.get.elems = tasks, true)
case Some(_) _taskStack set None
case _ // already None
}
private val _taskStack = new ThreadLocal[Option[Stack[() Unit]]]() { private val _taskStack = new ThreadLocal[Option[Stack[() Unit]]]() {
override def initialValue = None override def initialValue = None
} }
@ -439,7 +474,7 @@ sealed trait Future[+T] extends japi.Future[T] {
try { Some(BoxedType(m.erasure).cast(v).asInstanceOf[A]) } catch { try { Some(BoxedType(m.erasure).cast(v).asInstanceOf[A]) } catch {
case c: ClassCastException case c: ClassCastException
if (v.asInstanceOf[AnyRef] eq null) throw new ClassCastException("null cannot be cast to " + m.erasure) if (v.asInstanceOf[AnyRef] eq null) throw new ClassCastException("null cannot be cast to " + m.erasure)
else throw new ClassCastException("" + v + " of class " + v.asInstanceOf[AnyRef].getClass + " cannot be cast to " + m.erasure) else throw new ClassCastException("'" + v + "' of class " + v.asInstanceOf[AnyRef].getClass + " cannot be cast to " + m.erasure)
} }
} }
} }
@ -733,11 +768,6 @@ object Promise {
* Creates a non-completed, new, Promise with the default timeout (akka.actor.timeout in conf) * Creates a non-completed, new, Promise with the default timeout (akka.actor.timeout in conf)
*/ */
def apply[A]()(implicit dispatcher: MessageDispatcher, timeout: Timeout): Promise[A] = apply(timeout) def apply[A]()(implicit dispatcher: MessageDispatcher, timeout: Timeout): Promise[A] = apply(timeout)
/**
* Construct a completable channel
*/
def channel(timeout: Long)(implicit dispatcher: MessageDispatcher): ActorPromise = new ActorPromise(timeout)
} }
/** /**
@ -804,8 +834,8 @@ trait Promise[T] extends Future[T] {
} }
//Companion object to FState, just to provide a cheap, immutable default entry //Companion object to FState, just to provide a cheap, immutable default entry
private[akka] object FState { private[dispatch] object DefaultPromise {
def apply[T](): FState[T] = EmptyPending.asInstanceOf[FState[T]] def EmptyPending[T](): FState[T] = emptyPendingValue.asInstanceOf[FState[T]]
/** /**
* Represents the internal state of the DefaultCompletableFuture * Represents the internal state of the DefaultCompletableFuture
@ -824,7 +854,7 @@ private[akka] object FState {
case object Expired extends FState[Nothing] { case object Expired extends FState[Nothing] {
def value: Option[Either[Throwable, Nothing]] = None def value: Option[Either[Throwable, Nothing]] = None
} }
val EmptyPending = Pending[Nothing](Nil) private val emptyPendingValue = Pending[Nothing](Nil)
} }
/** /**
@ -833,7 +863,7 @@ private[akka] object FState {
class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDispatcher) extends AbstractPromise with Promise[T] { class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDispatcher) extends AbstractPromise with Promise[T] {
self self
import FState.{ FState, Success, Failure, Pending, Expired } import DefaultPromise.{ FState, Success, Failure, Pending, Expired }
def this()(implicit dispatcher: MessageDispatcher, timeout: Timeout) = this(timeout) def this()(implicit dispatcher: MessageDispatcher, timeout: Timeout) = this(timeout)
@ -857,7 +887,9 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
} }
} }
def await(atMost: Duration): this.type = { def await(atMost: Duration): this.type = if (value.isDefined) this else {
Future.blocking()
val waitNanos = val waitNanos =
if (timeout.duration.isFinite && atMost.isFinite) if (timeout.duration.isFinite && atMost.isFinite)
atMost.toNanos min timeLeft() atMost.toNanos min timeLeft()
@ -997,31 +1029,6 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
private def timeLeftNoinline(): Long = timeLeft() private def timeLeftNoinline(): Long = timeLeft()
} }
class ActorPromise(timeout: Timeout)(implicit dispatcher: MessageDispatcher) extends DefaultPromise[Any](timeout)(dispatcher) with UntypedChannel with ExceptionChannel[Any] {
def !(message: Any)(implicit channel: UntypedChannel) = completeWithResult(message)
override def sendException(ex: Throwable) = {
completeWithException(ex)
value == Some(Left(ex))
}
def channel: UntypedChannel = this
}
object ActorPromise {
def apply(f: Promise[Any])(timeout: Timeout = f.timeout): ActorPromise =
new ActorPromise(timeout)(f.dispatcher) {
completeWith(f)
override def !(message: Any)(implicit channel: UntypedChannel) = f completeWithResult message
override def sendException(ex: Throwable) = {
f completeWithException ex
f.value == Some(Left(ex))
}
}
}
/** /**
* An already completed Future is seeded with it's result at creation, is useful for when you are participating in * An already completed Future is seeded with it's result at creation, is useful for when you are participating in
* a Future-composition but you already have a value to contribute. * a Future-composition but you already have a value to contribute.

View file

@ -16,4 +16,4 @@ trait DeathWatch extends ActorEventBus with ActorClassifier {
type Event = Terminated type Event = Terminated
protected final def classify(event: Event): Classifier = event.actor protected final def classify(event: Event): Classifier = event.actor
} }

View file

@ -22,13 +22,13 @@ trait EventBus {
/** /**
* Attempts to register the subscriber to the specified Classifier * Attempts to register the subscriber to the specified Classifier
* @returns true if successful and false if not (because it was already subscribed to that Classifier, or otherwise) * @return true if successful and false if not (because it was already subscribed to that Classifier, or otherwise)
*/ */
def subscribe(subscriber: Subscriber, to: Classifier): Boolean def subscribe(subscriber: Subscriber, to: Classifier): Boolean
/** /**
* Attempts to deregister the subscriber from the specified Classifier * Attempts to deregister the subscriber from the specified Classifier
* @returns true if successful and false if not (because it wasn't subscribed to that Classifier, or otherwise) * @return true if successful and false if not (because it wasn't subscribed to that Classifier, or otherwise)
*/ */
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean
@ -263,4 +263,4 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒
def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber) def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber)
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = dissociate(from, subscriber) def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = dissociate(from, subscriber)
def unsubscribe(subscriber: Subscriber): Unit = dissociate(subscriber) def unsubscribe(subscriber: Subscriber): Unit = dissociate(subscriber)
} }

View file

@ -35,4 +35,4 @@ abstract class ScanningEventBus[E, S, C] extends EventBus with ScanningClassific
abstract class ActorEventBus[E] extends akka.event.ActorEventBus with ActorClassification with ActorClassifier { abstract class ActorEventBus[E] extends akka.event.ActorEventBus with ActorClassification with ActorClassifier {
} }

View file

@ -3,7 +3,7 @@
*/ */
package akka.event package akka.event
import akka.actor.{ Actor, ActorRef, MinimalActorRef, LocalActorRef, Props, UntypedChannel } import akka.actor.{ Actor, ActorRef, MinimalActorRef, LocalActorRef, Props }
import akka.{ AkkaException, AkkaApplication } import akka.{ AkkaException, AkkaApplication }
import akka.AkkaApplication.AkkaConfig import akka.AkkaApplication.AkkaConfig
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
@ -341,7 +341,7 @@ object Logging {
*/ */
class StandardOutLogger extends MinimalActorRef with StdOutLogger { class StandardOutLogger extends MinimalActorRef with StdOutLogger {
override val toString = "StandardOutLogger" override val toString = "StandardOutLogger"
override def postMessageToMailbox(obj: Any, channel: UntypedChannel) { print(obj) } override def postMessageToMailbox(obj: Any, sender: ActorRef) { print(obj) }
} }
val StandardOutLogger = new StandardOutLogger val StandardOutLogger = new StandardOutLogger
val StandardOutLoggerName = StandardOutLogger.getClass.getName val StandardOutLoggerName = StandardOutLogger.getClass.getName
@ -394,7 +394,7 @@ object Logging {
trait LoggingAdapter { trait LoggingAdapter {
/* /*
* implement these as precisely as needed/possible: always returning true * implement these as precisely as needed/possible: always returning true
* just makes the notify... methods be called every time. * just makes the notify... methods be called every time.
*/ */
def isErrorEnabled: Boolean def isErrorEnabled: Boolean
@ -403,7 +403,7 @@ trait LoggingAdapter {
def isDebugEnabled: Boolean def isDebugEnabled: Boolean
/* /*
* These actually implement the passing on of the messages to be logged. * These actually implement the passing on of the messages to be logged.
* Will not be called if is...Enabled returned false. * Will not be called if is...Enabled returned false.
*/ */
protected def notifyError(message: String) protected def notifyError(message: String)
@ -481,4 +481,4 @@ class BusLogging(val bus: LoggingBus, val loggingInstance: AnyRef) extends Loggi
protected def notifyDebug(message: String) { bus.publish(Debug(loggingInstance, message)) } protected def notifyDebug(message: String) { bus.publish(Debug(loggingInstance, message)) }
} }

View file

@ -21,78 +21,8 @@ import java.lang.reflect.InvocationTargetException
class RemoteException(message: String) extends AkkaException(message) class RemoteException(message: String) extends AkkaException(message)
trait RemoteService {
def server: RemoteSupport
def address: InetSocketAddress
}
trait RemoteModule { trait RemoteModule {
val UUID_PREFIX = "uuid:".intern
def optimizeLocalScoped_?(): Boolean //Apply optimizations for remote operations in local scope
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit
private[akka] def actors: ConcurrentHashMap[String, ActorRef] // FIXME need to invalidate this cache on replication
private[akka] def actorsByUuid: ConcurrentHashMap[String, ActorRef] // FIXME remove actorsByUuid map?
private[akka] def actorsFactories: ConcurrentHashMap[String, () ActorRef] // FIXME what to do wit actorsFactories map?
private[akka] def findActorByAddress(address: String): ActorRef = actors.get(address)
private[akka] def findActorByUuid(uuid: String): ActorRef = actorsByUuid.get(uuid)
private[akka] def findActorFactory(address: String): () ActorRef = actorsFactories.get(address)
private[akka] def findActorByAddressOrUuid(address: String, uuid: String): ActorRef = {
var actorRefOrNull = if (address.startsWith(UUID_PREFIX)) findActorByUuid(address.substring(UUID_PREFIX.length))
else findActorByAddress(address)
if (actorRefOrNull eq null) actorRefOrNull = findActorByUuid(uuid)
actorRefOrNull
}
/*
private[akka] def findActorByAddress(address: String): ActorRef = {
val cachedActorRef = actors.get(address)
if (cachedActorRef ne null) cachedActorRef
else {
val actorRef =
Deployer.lookupDeploymentFor(address) match {
case Some(Deploy(_, router, _, Cluster(home, _, _)))
if (DeploymentConfig.isHomeNode(home)) { // on home node
Actor.registry.actorFor(address) match { // try to look up in actor registry
case Some(actorRef) // in registry -> DONE
actorRef
case None // not in registry -> check out as 'ref' from cluster (which puts it in actor registry for next time around)
Actor.cluster.ref(address, DeploymentConfig.routerTypeFor(router))
}
} else throw new IllegalActorStateException("Trying to look up remote actor on non-home node. FIXME: fix this behavior")
case Some(Deploy(_, _, _, Local))
Actor.registry.actorFor(address).getOrElse(throw new IllegalActorStateException("Could not lookup locally deployed actor in actor registry"))
case _
actors.get(address) // FIXME do we need to fall back to local here? If it is not clustered then it should not be a remote actor in the first place. Throw exception.
}
actors.put(address, actorRef) // cache it for next time around
actorRef
}
}
private[akka] def findActorByUuid(uuid: String): ActorRef = actorsByUuid.get(uuid)
private[akka] def findActorFactory(address: String): () ActorRef = actorsFactories.get(address)
private[akka] def findActorByAddressOrUuid(address: String, uuid: String): ActorRef = {
// find by address
var actorRefOrNull =
if (address.startsWith(UUID_PREFIX)) findActorByUuid(address.substring(UUID_PREFIX.length)) // FIXME remove lookup by UUID? probably
else findActorByAddress(address)
// find by uuid
if (actorRefOrNull eq null) actorRefOrNull = findActorByUuid(uuid)
actorRefOrNull
}
*/
} }
/** /**
@ -190,28 +120,15 @@ abstract class RemoteSupport(val app: AkkaApplication) extends RemoteServerModul
def shutdown() { def shutdown() {
this.shutdownClientModule() this.shutdownClientModule()
this.shutdownServerModule() this.shutdownServerModule()
clear
} }
protected[akka] override def notifyListeners(message: RemoteLifeCycleEvent): Unit = app.mainbus.publish(message) protected[akka] override def notifyListeners(message: RemoteLifeCycleEvent): Unit = app.mainbus.publish(message)
private[akka] val actors = new ConcurrentHashMap[String, ActorRef]
private[akka] val actorsByUuid = new ConcurrentHashMap[String, ActorRef]
private[akka] val actorsFactories = new ConcurrentHashMap[String, () ActorRef]
def clear {
actors.clear
actorsByUuid.clear
actorsFactories.clear
}
} }
/** /**
* This is the interface for the RemoteServer functionality, it's used in Actor.remote * This is the interface for the RemoteServer functionality, it's used in Actor.remote
*/ */
trait RemoteServerModule extends RemoteModule { this: RemoteSupport trait RemoteServerModule extends RemoteModule { this: RemoteSupport
protected val guard = new ReentrantGuard
/** /**
* Signals whether the server is up and running or not * Signals whether the server is up and running or not
*/ */
@ -222,39 +139,6 @@ trait RemoteServerModule extends RemoteModule { this: RemoteSupport ⇒
*/ */
def name: String def name: String
/**
* Gets the address of the server instance
*/
def address: InetSocketAddress
/**
* Starts the server up
*/
def start(): RemoteServerModule =
start(app.defaultAddress.getAddress.getHostAddress,
app.defaultAddress.getPort,
None)
/**
* Starts the server up
*/
def start(loader: ClassLoader): RemoteServerModule =
start(app.defaultAddress.getAddress.getHostAddress,
app.defaultAddress.getPort,
Option(loader))
/**
* Starts the server up
*/
def start(host: String, port: Int): RemoteServerModule =
start(host, port, None)
/**
* Starts the server up
*/
def start(host: String, port: Int, loader: ClassLoader): RemoteServerModule =
start(host, port, Option(loader))
/** /**
* Starts the server up * Starts the server up
*/ */
@ -264,74 +148,9 @@ trait RemoteServerModule extends RemoteModule { this: RemoteSupport ⇒
* Shuts the server down * Shuts the server down
*/ */
def shutdownServerModule(): Unit def shutdownServerModule(): Unit
/**
* Register Remote Actor by the Actor's 'id' field. It starts the Actor if it is not started already.
*/
def register(actorRef: ActorRef): Unit = register(actorRef.address, actorRef)
/**
* Register Remote Actor by the Actor's uuid field. It starts the Actor if it is not started already.
*/
def registerByUuid(actorRef: ActorRef): Unit
/**
* Register Remote Actor by a specific 'id' passed as argument. The actor is registered by UUID rather than ID
* when prefixing the handle with the uuid: protocol.
* <p/>
* NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
*/
def register(address: String, actorRef: ActorRef): Unit
/**
* Register Remote Session Actor by a specific 'id' passed as argument.
* <p/>
* NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
*/
def registerPerSession(address: String, factory: ActorRef): Unit
/**
* Register Remote Session Actor by a specific 'id' passed as argument.
* <p/>
* NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
* Java API
*/
def registerPerSession(address: String, factory: Creator[ActorRef]): Unit = registerPerSession(address, factory.create)
/**
* Unregister Remote Actor that is registered using its 'id' field (not custom ID).
*/
def unregister(actorRef: ActorRef): Unit
/**
* Unregister Remote Actor by specific 'id'.
* <p/>
* NOTE: You need to call this method if you have registered an actor by a custom ID.
*/
def unregister(address: String): Unit
/**
* Unregister Remote Actor by specific 'id'.
* <p/>
* NOTE: You need to call this method if you have registered an actor by a custom ID.
*/
def unregisterPerSession(address: String): Unit
} }
trait RemoteClientModule extends RemoteModule { self: RemoteSupport trait RemoteClientModule extends RemoteModule { self: RemoteSupport
def actorFor(address: String, hostname: String, port: Int): ActorRef =
actorFor(address, app.AkkaConfig.ActorTimeoutMillis, hostname, port, None)
def actorFor(address: String, hostname: String, port: Int, loader: ClassLoader): ActorRef =
actorFor(address, app.AkkaConfig.ActorTimeoutMillis, hostname, port, Some(loader))
def actorFor(address: String, timeout: Long, hostname: String, port: Int): ActorRef =
actorFor(address, timeout, hostname, port, None)
def actorFor(address: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): ActorRef =
actorFor(address, timeout, hostname, port, Some(loader))
/** /**
* Clean-up all open connections. * Clean-up all open connections.
*/ */
@ -349,13 +168,9 @@ trait RemoteClientModule extends RemoteModule { self: RemoteSupport ⇒
/** Methods that needs to be implemented by a transport **/ /** Methods that needs to be implemented by a transport **/
protected[akka] def actorFor(address: String, timeout: Long, hostname: String, port: Int, loader: Option[ClassLoader]): ActorRef protected[akka] def send(message: Any,
senderOption: Option[ActorRef],
protected[akka] def send[T](message: Any, remoteAddress: InetSocketAddress,
senderOption: Option[ActorRef], recipient: ActorRef,
senderFuture: Option[Promise[T]], loader: Option[ClassLoader]): Unit
remoteAddress: InetSocketAddress,
isOneWay: Boolean,
actorRef: ActorRef,
loader: Option[ClassLoader]): Option[Promise[T]]
} }

View file

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

View file

@ -95,6 +95,10 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒
val defaultProps: Props = Props.default.withDispatcher(this.context.dispatcher) val defaultProps: Props = Props.default.withDispatcher(this.context.dispatcher)
override def preStart() {
resizeIfAppropriate()
}
override def postStop() { override def postStop() {
_delegates foreach evict _delegates foreach evict
_delegates = Vector.empty _delegates = Vector.empty
@ -103,7 +107,7 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒
protected def _route(): Actor.Receive = { protected def _route(): Actor.Receive = {
// for testing... // for testing...
case Stat case Stat
channel.tryTell(Stats(_delegates length)) sender ! Stats(_delegates length)
case Terminated(victim) case Terminated(victim)
_delegates = _delegates filterNot { victim == } _delegates = _delegates filterNot { victim == }
case msg case msg
@ -112,7 +116,7 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒
select(_delegates) foreach { _ forward msg } select(_delegates) foreach { _ forward msg }
} }
private def resizeIfAppropriate() { protected def resizeIfAppropriate() {
val requestedCapacity = capacity(_delegates) val requestedCapacity = capacity(_delegates)
val newDelegates = requestedCapacity match { val newDelegates = requestedCapacity match {
case qty if qty > 0 case qty if qty > 0
@ -285,17 +289,17 @@ trait MailboxPressureCapacitor {
/** /**
* Implements pressure() to return the number of actors currently processing a * Implements pressure() to return the number of actors currently processing a
* message whose reply will be sent to a [[akka.dispatch.Future]]. * message.
* In other words, this capacitor counts how many * In other words, this capacitor counts how many
* delegates are tied up actively processing a message, as long as the * delegates are tied up actively processing a message
* messages have somebody waiting on the result. "One way" messages with
* no reply would not be counted.
*/ */
trait ActiveFuturesPressureCapacitor { trait ActiveActorsPressureCapacitor {
def pressure(delegates: Seq[ActorRef]): Int = def pressure(delegates: Seq[ActorRef]): Int =
delegates count { delegates count {
case a: LocalActorRef a.underlying.channel.isInstanceOf[Promise[_]] case a: LocalActorRef
case _ false val cell = a.underlying
cell.mailbox.isScheduled && cell.currentMessage != null
case _ false
} }
} }

View file

@ -59,8 +59,6 @@ object RouterType {
/** /**
* A RouterType that select the connection based on the least amount of ram used. * A RouterType that select the connection based on the least amount of ram used.
*
* FIXME: this is extremely vague currently since there are so many ways to define least amount of ram.
*/ */
object LeastRAM extends RouterType object LeastRAM extends RouterType

View file

@ -41,7 +41,7 @@ trait Router {
* *
* @throws RoutingException if something goes wrong while routing the message * @throws RoutingException if something goes wrong while routing the message
*/ */
def route(message: Any)(implicit sender: Option[ActorRef]) def route(message: Any)(implicit sender: ActorRef)
/** /**
* Routes the message using a timeout to one of the connections and returns a Future to synchronize on the * Routes the message using a timeout to one of the connections and returns a Future to synchronize on the
@ -49,7 +49,7 @@ trait Router {
* *
* @throws RoutingExceptionif something goes wrong while routing the message. * @throws RoutingExceptionif something goes wrong while routing the message.
*/ */
def route[T](message: Any, timeout: Timeout)(implicit sender: Option[ActorRef]): Future[T] def route[T](message: Any, timeout: Timeout): Future[T]
} }
/** /**
@ -92,34 +92,19 @@ object Routing {
/** /**
* An Abstract convenience implementation for building an ActorReference that uses a Router. * An Abstract convenience implementation for building an ActorReference that uses a Router.
*/ */
abstract private[akka] class AbstractRoutedActorRef(val props: RoutedProps) extends UnsupportedActorRef { abstract private[akka] class AbstractRoutedActorRef(val app: AkkaApplication, val props: RoutedProps) extends UnsupportedActorRef {
private[akka] override val uuid: Uuid = newUuid
val router = props.routerFactory() val router = props.routerFactory()
override def postMessageToMailbox(message: Any, channel: UntypedChannel) = { override def postMessageToMailbox(message: Any, sender: ActorRef) = router.route(message)(sender)
val sender = channel match {
case ref: ActorRef Some(ref)
case _ None
}
router.route(message)(sender)
}
override def postMessageToMailboxAndCreateFutureResultWithTimeout( override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = router.route(message, timeout)
message: Any, timeout: Timeout, channel: UntypedChannel): Future[Any] = {
val sender = channel match {
case ref: ActorRef Some(ref)
case _ None
}
router.route[Any](message, timeout)(sender)
}
} }
/** /**
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to
* on (or more) of these actors. * on (or more) of these actors.
*/ */
private[akka] class RoutedActorRef(val routedProps: RoutedProps, override val address: String) extends AbstractRoutedActorRef(routedProps) { private[akka] class RoutedActorRef(app: AkkaApplication, val routedProps: RoutedProps, override val address: String) extends AbstractRoutedActorRef(app, routedProps) {
@volatile @volatile
private var running: Boolean = true private var running: Boolean = true
@ -130,7 +115,7 @@ private[akka] class RoutedActorRef(val routedProps: RoutedProps, override val ad
synchronized { synchronized {
if (running) { if (running) {
running = false running = false
router.route(Routing.Broadcast(PoisonPill))(Some(this)) router.route(Routing.Broadcast(PoisonPill))(this)
} }
} }
} }
@ -141,8 +126,6 @@ private[akka] class RoutedActorRef(val routedProps: RoutedProps, override val ad
/** /**
* An Abstract Router implementation that already provides the basic infrastructure so that a concrete * An Abstract Router implementation that already provides the basic infrastructure so that a concrete
* Router only needs to implement the next method. * Router only needs to implement the next method.
*
* FIXME: this is also the location where message buffering should be done in case of failure.
*/ */
trait BasicRouter extends Router { trait BasicRouter extends Router {
@ -153,8 +136,9 @@ trait BasicRouter extends Router {
this.connectionManager = connectionManager this.connectionManager = connectionManager
} }
def route(message: Any)(implicit sender: Option[ActorRef]) = message match { def route(message: Any)(implicit sender: ActorRef) = message match {
case Routing.Broadcast(message) case Routing.Broadcast(message)
//it is a broadcast message, we are going to send to message to all connections. //it is a broadcast message, we are going to send to message to all connections.
connectionManager.connections.iterable foreach { connection connectionManager.connections.iterable foreach { connection
try { try {
@ -181,7 +165,7 @@ trait BasicRouter extends Router {
} }
} }
def route[T](message: Any, timeout: Timeout)(implicit sender: Option[ActorRef]): Future[T] = message match { def route[T](message: Any, timeout: Timeout): Future[T] = message match {
case Routing.Broadcast(message) case Routing.Broadcast(message)
throw new RoutingException("Broadcasting using '?'/'ask' is for the time being is not supported. Use ScatterGatherRouter.") throw new RoutingException("Broadcasting using '?'/'ask' is for the time being is not supported. Use ScatterGatherRouter.")
case _ case _
@ -189,8 +173,7 @@ trait BasicRouter extends Router {
next match { next match {
case Some(connection) case Some(connection)
try { try {
// FIXME is this not wrong? it will not pass on and use the original Future but create a new one. Should reuse 'channel: UntypedChannel' in the AbstractRoutedActorRef connection.?(message, timeout).asInstanceOf[Future[T]] //FIXME this does not preserve the original sender, shouldn't it?
connection.?(message, timeout)(sender).asInstanceOf[Future[T]]
} catch { } catch {
case e: Exception case e: Exception
connectionManager.remove(connection) connectionManager.remove(connection)
@ -203,10 +186,7 @@ trait BasicRouter extends Router {
protected def next: Option[ActorRef] protected def next: Option[ActorRef]
private def throwNoConnectionsError = { private def throwNoConnectionsError = throw new RoutingException("No replica connections for router")
val error = new RoutingException("No replica connections for router")
throw error
}
} }
/** /**
@ -257,15 +237,17 @@ class DirectRouter extends BasicRouter {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class RandomRouter extends BasicRouter { class RandomRouter extends BasicRouter {
import java.security.SecureRandom
private val state = new AtomicReference[RandomRouterState] private val state = new AtomicReference[RandomRouterState]
//FIXME: threadlocal random? private val random = new ThreadLocal[SecureRandom] {
private val random = new java.util.Random(System.nanoTime) override def initialValue = SecureRandom.getInstance("SHA1PRNG")
}
def next: Option[ActorRef] = currentState.array match { def next: Option[ActorRef] = currentState.array match {
case a if a.isEmpty None case a if a.isEmpty None
case a Some(a(random.nextInt(a.length))) case a Some(a(random.get.nextInt(a.length)))
} }
@tailrec @tailrec
@ -358,11 +340,11 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
*/ */
protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G] protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G]
private def scatterGather[S, G >: S](message: Any, timeout: Timeout)(implicit sender: Option[ActorRef]): Future[G] = { private def scatterGather[S, G >: S](message: Any, timeout: Timeout): Future[G] = {
val responses = connectionManager.connections.iterable.flatMap { actor val responses = connectionManager.connections.iterable.flatMap { actor
try { try {
if (actor.isShutdown) throw ActorInitializationException(actor, "For compatability - check death first", new Exception) // for stack trace if (actor.isShutdown) throw ActorInitializationException(actor, "For compatability - check death first", new Exception) // for stack trace
Some(actor.?(message, timeout)(sender).asInstanceOf[Future[S]]) Some(actor.?(message, timeout).asInstanceOf[Future[S]])
} catch { } catch {
case e: Exception case e: Exception
connectionManager.remove(actor) connectionManager.remove(actor)
@ -375,9 +357,9 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
else gather(responses) else gather(responses)
} }
override def route[T](message: Any, timeout: Timeout)(implicit sender: Option[ActorRef]): Future[T] = message match { override def route[T](message: Any, timeout: Timeout): Future[T] = message match {
case Routing.Broadcast(message) scatterGather(message, timeout) case Routing.Broadcast(message) scatterGather(message, timeout)
case message super.route(message, timeout)(sender) case message super.route(message, timeout)
} }
} }

View file

@ -138,7 +138,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) {
/** /**
* Disassociates all the values for the specified key * Disassociates all the values for the specified key
* @returns None if the key wasn't associated at all, or Some(scala.Iterable[V]) if it was associated * @return None if the key wasn't associated at all, or Some(scala.Iterable[V]) if it was associated
*/ */
def remove(key: K): Option[Iterable[V]] = { def remove(key: K): Option[Iterable[V]] = {
val set = container get key val set = container get key

View file

@ -5,15 +5,12 @@
package akka.util package akka.util
import akka.dispatch.Envelope import akka.dispatch.Envelope
import akka.config.ModuleNotAvailableException
import akka.actor._ import akka.actor._
import DeploymentConfig.ReplicationScheme import DeploymentConfig.ReplicationScheme
import akka.config.ModuleNotAvailableException import akka.config.ModuleNotAvailableException
import akka.event.Logging.Debug import akka.event.Logging.Debug
import akka.cluster.ClusterNode import akka.cluster.ClusterNode
import akka.remote.{ RemoteSupport, RemoteService }
import akka.routing.{ RoutedProps, Router } import akka.routing.{ RoutedProps, Router }
import java.net.InetSocketAddress
import akka.AkkaApplication import akka.AkkaApplication
object ReflectiveAccess { object ReflectiveAccess {
@ -140,110 +137,4 @@ class ReflectiveAccess(val app: AkkaApplication) {
case Left(e) throw e case Left(e) throw e
} }
} }
/**
* Reflective access to the Cluster module.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object ClusterModule {
lazy val isEnabled = app.AkkaConfig.ClusterEnabled //&& clusterInstance.isDefined
lazy val clusterRefClass: Class[_] = getClassFor("akka.cluster.ClusterActorRef") match {
case Left(e) throw e
case Right(b) b
}
def newClusteredActorRef(props: RoutedProps): ActorRef = {
val params: Array[Class[_]] = Array(classOf[RoutedProps])
val args: Array[AnyRef] = Array(props)
createInstance(clusterRefClass, params, args) match {
case Left(e) throw e
case Right(b) b.asInstanceOf[ActorRef]
}
}
def ensureEnabled() {
if (!isEnabled) {
val e = new ModuleNotAvailableException(
"Can't load the cluster module, make sure it is enabled in the config ('akka.enabled-modules = [\"cluster\"])' and that akka-cluster.jar is on the classpath")
app.mainbus.publish(Debug(this, e.toString))
throw e
}
}
lazy val clusterInstance: Option[Cluster] = getObjectFor("akka.cluster.Cluster$") match {
case Right(value) Some(value)
case Left(exception)
app.mainbus.publish(Debug(this, exception.toString))
None
}
lazy val clusterDeployerInstance: Option[ActorDeployer] = getObjectFor("akka.cluster.ClusterDeployer$") match {
case Right(value) Some(value)
case Left(exception)
app.mainbus.publish(Debug(this, exception.toString))
None
}
lazy val transactionLogInstance: Option[TransactionLogObject] = getObjectFor("akka.cluster.TransactionLog$") match {
case Right(value) Some(value)
case Left(exception)
app.mainbus.publish(Debug(this, exception.toString))
None
}
lazy val node: ClusterNode = {
ensureEnabled()
clusterInstance.get.node
}
lazy val clusterDeployer: ActorDeployer = {
ensureEnabled()
clusterDeployerInstance.get
}
lazy val transactionLog: TransactionLogObject = {
ensureEnabled()
transactionLogInstance.get
}
type Cluster = {
def node: ClusterNode
}
type Mailbox = {
def enqueue(message: Envelope)
def dequeue: Envelope
}
type TransactionLogObject = {
def newLogFor(
id: String,
isAsync: Boolean,
replicationScheme: ReplicationScheme): TransactionLog
def logFor(
id: String,
isAsync: Boolean,
replicationScheme: ReplicationScheme): TransactionLog
def shutdown()
}
type TransactionLog = {
def recordEntry(messageHandle: Envelope, actorRef: LocalActorRef)
def recordEntry(entry: Array[Byte])
def recordSnapshot(snapshot: Array[Byte])
def entries: Vector[Array[Byte]]
def entriesFromLatestSnapshot: Tuple2[Array[Byte], Vector[Array[Byte]]]
def entriesInRange(from: Long, to: Long): Vector[Array[Byte]]
def latestSnapshotAndSubsequentEntries: (Option[Array[Byte]], Vector[Array[Byte]])
def latestEntryId: Long
def latestSnapshotId: Long
def delete()
def close()
}
}
} }

View file

@ -88,7 +88,6 @@ private[camel] trait ConsumerMethodEvent extends ConsumerEvent {
val typedActor: AnyRef val typedActor: AnyRef
val method: Method val method: Method
val uuid = actorRef.uuid.toString
val methodName = method.getName val methodName = method.getName
val methodUuid = "%s_%s" format (uuid, methodName) val methodUuid = "%s_%s" format (uuid, methodName)

View file

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

View file

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

View file

@ -97,9 +97,9 @@ trait ProducerSupport { this: Actor ⇒
val exchange = createExchange(pattern).fromRequestMessage(cmsg) val exchange = createExchange(pattern).fromRequestMessage(cmsg)
processor.process(exchange, new AsyncCallback { processor.process(exchange, new AsyncCallback {
val producer = self val producer = self
// Need copies of channel reference here since the callback could be done // Need copies of sender reference here since the callback could be done
// later by another thread. // later by another thread.
val replyChannel = channel val replyChannel = sender
def done(doneSync: Boolean) { def done(doneSync: Boolean) {
(doneSync, exchange.isFailed) match { (doneSync, exchange.isFailed) match {
@ -159,7 +159,7 @@ trait ProducerSupport { this: Actor ⇒
* actor). * actor).
*/ */
protected def receiveAfterProduce: Receive = { protected def receiveAfterProduce: Receive = {
case msg if (!oneway) channel ! msg case msg if (!oneway) sender ! msg
} }
/** /**

View file

@ -16,7 +16,7 @@ import akka.actor._
import akka.camel.{ Ack, Failure, Message } import akka.camel.{ Ack, Failure, Message }
import akka.camel.CamelMessageConversion.toExchangeAdapter import akka.camel.CamelMessageConversion.toExchangeAdapter
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
import akka.dispatch.{ FutureTimeoutException, Promise, MessageDispatcher } import akka.dispatch._
/** /**
* @author Martin Krasser * @author Martin Krasser
@ -274,9 +274,7 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
def resume(): Unit = () def resume(): Unit = ()
def stop() { def stop() { running = false }
running = false
}
/** /**
* Populates the initial <code>exchange</code> with the reply <code>message</code> and uses the * Populates the initial <code>exchange</code> with the reply <code>message</code> and uses the
@ -286,7 +284,7 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
* @param message reply message * @param message reply message
* @param sender ignored * @param sender ignored
*/ */
protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel) = if(running) { protected[akka] def postMessageToMailbox(message: Any, sender: ActorRef) = if(running) {
message match { message match {
case Ack { /* no response message to set */ } case Ack { /* no response message to set */ }
case msg: Failure exchange.fromFailureMessage(msg) case msg: Failure exchange.fromFailureMessage(msg)
@ -298,7 +296,8 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
def startsMonitoring(actorRef: ActorRef): ActorRef = unsupported def startsMonitoring(actorRef: ActorRef): ActorRef = unsupported
def stopsMonitoring(actorRef: ActorRef): ActorRef = unsupported def stopsMonitoring(actorRef: ActorRef): ActorRef = unsupported
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(message: Any, timeout: Timeout, channel: UntypedChannel) = unsupported def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
new KeptPromise[Any](Left(new UnsupportedOperationException("Ask/? is not supported for %s".format(getClass.getName))))
def restart(reason: Throwable): Unit = unsupported def restart(reason: Throwable): Unit = unsupported
private def unsupported = throw new UnsupportedOperationException("Not supported for %s" format classOf[AsyncCallbackAdapter].getName) private def unsupported = throw new UnsupportedOperationException("Not supported for %s" format classOf[AsyncCallbackAdapter].getName)

View file

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

View file

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

View file

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

Some files were not shown because too many files have changed in this diff Show more