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);
}
@Test void mustAcceptSingleArgTryTell() {
@Test void mustAcceptSingleArgTell() {
ActorRef ref = app.actorOf(JavaAPITestActor.class);
ref.tryTell("hallo");
ref.tryTell("hallo", ref);
ref.tell("hallo");
ref.tell("hallo", ref);
}
}

View file

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

View file

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

View file

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

View file

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

View file

@ -101,7 +101,7 @@ object Chameneos {
}
} else {
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 {
filterException[ActorKilledException] {
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 monitor = actorOf(Props(new Actor {

View file

@ -13,7 +13,7 @@ class DeployerSpec extends AkkaSpec {
"A Deployer" must {
"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 equal(Some(

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -5,6 +5,6 @@ package akka.actor
class Supervisor extends Actor {
def receive = {
case x: Props 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 {
protected def receive = {
case p: Props channel ! context.actorOf(p)
case p: Props sender ! context.actorOf(p)
}
override def postRestart(reason: Throwable) = {
countDown.countDown()

View file

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

View file

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

View file

@ -22,7 +22,7 @@ class SupervisorTreeSpec extends AkkaSpec with ImplicitSender {
within(5 seconds) {
val p = Props(new Actor {
def receive = {
case p: Props channel ! context.actorOf(p)
case p: Props sender ! context.actorOf(p)
}
override def preRestart(cause: Throwable, msg: Option[Any]) { testActor ! self.address }
}).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 supervised = (supervisor ? Props[Supervised]).as[ActorRef].get
supervised.!("test")(Some(testActor))
supervised.!("test")(testActor)
expectMsg("failure1")
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 supervised = (supervisor ? Props[Supervised]).as[ActorRef].get
supervised.!("test")(Some(testActor))
supervised.!("test")(testActor)
expectMsg("failure2")
supervisor.stop()
}
@ -51,11 +51,11 @@ object Ticket669Spec {
}
override def preRestart(reason: scala.Throwable, msg: Option[Any]) {
channel.tryTell("failure1")
sender.tell("failure1")
}
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 {
filterEvents(EventFilter[IllegalStateException]("expected")) {
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 {
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.read() must be(1) //Make sure state is not reset after failure
(intercept[IllegalStateException] {
t.failingJOptionPigdog
}).getMessage must be("expected")
(intercept[IllegalStateException] { t.failingJOptionPigdog }).getMessage must be("expected")
t.read() must be(1) //Make sure state is not reset after failure
(intercept[IllegalStateException] {
t.failingOptionPigdog
}).getMessage must be("expected")
(intercept[IllegalStateException] { t.failingOptionPigdog }).getMessage must be("expected")
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 Wait(time) ack; Thread.sleep(time); busy.switchOff()
case WaitAck(time, l) ack; Thread.sleep(time); l.countDown(); busy.switchOff()
case Reply(msg) ack; channel ! msg; busy.switchOff()
case TryReply(msg) ack; channel.tryTell(msg); busy.switchOff()
case Reply(msg) ack; sender ! msg; busy.switchOff()
case TryReply(msg) ack; sender.tell(msg); busy.switchOff()
case Forward(to, msg) ack; to.forward(msg); busy.switchOff()
case CountDown(latch) ack; latch.countDown(); busy.switchOff()
case Increment(count) ack; count.incrementAndGet(); busy.switchOff()
case CountDownNStop(l) ack; l.countDown(); self.stop(); busy.switchOff()
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
}
}
@ -364,6 +364,7 @@ abstract class ActorModelSpec extends AkkaSpec {
} catch {
case e
System.err.println("Error: " + e.getMessage + " missing count downs == " + cachedMessage.latch.getCount() + " out of " + num)
throw e
}
boss.stop()
}
@ -380,30 +381,17 @@ abstract class ActorModelSpec extends AkkaSpec {
val a = newTestActor(dispatcher)
val f1 = a ? Reply("foo")
val f2 = a ? Reply("bar")
val f3 = try {
a ? Interrupt
} catch {
// CallingThreadDispatcher throws IE directly
case ie: InterruptedException new KeptPromise(Left(ActorInterruptedException(ie)))
}
val f3 = try { a ? Interrupt } catch { case ie: InterruptedException new KeptPromise(Left(ActorInterruptedException(ie))) }
val f4 = a ? Reply("foo2")
val f5 = try {
a ? Interrupt
} catch {
case ie: InterruptedException new KeptPromise(Left(ActorInterruptedException(ie)))
}
val f5 = try { a ? Interrupt } catch { case ie: InterruptedException new KeptPromise(Left(ActorInterruptedException(ie))) }
val f6 = a ? Reply("bar2")
assert(f1.get === "foo")
assert(f2.get === "bar")
assert((intercept[ActorInterruptedException] {
f3.get
}).getMessage === "Ping!")
assert(f4.get === "foo2")
assert((intercept[ActorInterruptedException] {
f5.get
}).getMessage === "Ping!")
assert(intercept[ActorInterruptedException](f3.get).getMessage === "Ping!")
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 f1 = a ? Reply("foo")
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 f5 = a ? new ThrowException(new RemoteException("RemoteException"))
val f5 = a ? ThrowException(new RemoteException("RemoteException"))
val f6 = a ? Reply("bar2")
assert(f1.get === "foo")
assert(f2.get === "bar")
assert((intercept[IndexOutOfBoundsException] {
f3.get
}).getMessage === "IndexOutOfBoundsException")
assert(f4.get === "foo2")
assert((intercept[RemoteException] {
f5.get
}).getMessage === "RemoteException")
assert(f6.get === "bar2")
assert(f3.result === None)
assert(f5.result === None)
}
}
}
@ -446,21 +430,6 @@ class DispatcherModelSpec extends ActorModelSpec {
def dispatcherType = "Dispatcher"
"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 {
implicit val dispatcher = newInterceptedDispatcher
val aStart, aStop, bParallel = new CountDownLatch(1)

View file

@ -9,7 +9,7 @@ import akka.actor.{ Props, Actor }
object DispatcherActorSpec {
class TestActor extends Actor {
def receive = {
case "Hello" channel ! "World"
case "Hello" sender ! "World"
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
}
}
@ -46,20 +46,6 @@ class DispatcherActorSpec extends AkkaSpec {
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 {
val throughputDispatcher = app.dispatcherFactory.
newDispatcher("THROUGHPUT", 101, 0, app.dispatcherFactory.MailboxType).
@ -74,7 +60,7 @@ class DispatcherActorSpec extends AkkaSpec {
val slowOne = actorOf(
Props(context {
case "hogexecutor" context.channel ! "OK"; start.await
case "hogexecutor" context.sender ! "OK"; start.await
case "ping" if (works.get) latch.countDown()
}).withDispatcher(throughputDispatcher))

View file

@ -11,7 +11,7 @@ import org.scalatest.BeforeAndAfterEach
object PinnedActorSpec {
class TestActor extends Actor {
def receive = {
case "Hello" channel ! "World"
case "Hello" sender ! "World"
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)
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)
}
"support reply via channel" in {
"support reply via sender" in {
val actor = app.actorOf(Props(new Actor {
def receive = {
case "do" Future(31) pipeTo context.channel
case "ex" Future(throw new AssertionError) pipeTo context.channel
case "do" Future(31) pipeTo context.sender
case "ex" Future(throw new AssertionError) pipeTo context.sender
}
}))
(actor ? "do").as[Int] must be(Some(31))

View file

@ -6,30 +6,32 @@ import org.scalacheck._
import org.scalacheck.Arbitrary._
import org.scalacheck.Prop._
import org.scalacheck.Gen._
import akka.actor.{ Actor, ActorRef, Timeout }
import akka.actor.{ Actor, ActorRef, Status }
import akka.testkit.{ EventFilter, filterEvents, filterException }
import akka.util.duration._
import org.multiverse.api.latches.StandardLatch
import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.testkit.AkkaSpec
import org.scalatest.junit.JUnitSuite
import java.lang.ArithmeticException
object FutureSpec {
class TestActor extends Actor {
def receive = {
case "Hello" channel ! "World"
case "Failure" throw new RuntimeException("Expected exception; to test fault-tolerance")
case "Hello" sender ! "World"
case "Failure"
sender ! Status.Failure(new RuntimeException("Expected exception; to test fault-tolerance"))
case "NoReply"
}
}
class TestDelayActor(await: StandardLatch) extends Actor {
def receive = {
case "Hello" await.await; channel ! "World"
case "Hello" await.await; sender ! "World"
case "NoReply" await.await
case "Failure"
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 {
behave like futureWithResult { test
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 }
future.await
test(future, "WORLD")
@ -149,7 +151,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
behave like futureWithException[ArithmeticException] { test
filterException[ArithmeticException] {
val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String 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 }
future.await
test(future, "/ by zero")
@ -162,7 +164,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
behave like futureWithException[MatchError] { test
filterException[MatchError] {
val actor1 = actorOf[TestActor]
val actor2 = actorOf(new Actor { def receive = { case s: String 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 }
future.await
test(future, "World (of class java.lang.String)")
@ -179,8 +181,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
filterException[ClassCastException] {
val actor = actorOf(new Actor {
def receive = {
case s: String channel ! s.length
case i: Int channel ! (i * 2).toString
case s: String sender ! s.length
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)
val actor = actorOf(new Actor {
def receive = {
case Req(s: String) channel ! Res(s.length)
case Req(i: Int) channel ! Res((i * 2).toString)
case Req(s: String) sender ! Res(s.length)
case Req(i: Int) sender ! Res((i * 2).toString)
}
})
@ -298,7 +300,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"fold" in {
val actors = (1 to 10).toList map { _
actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); channel.tryTell(add) }
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); sender.tell(add) }
})
}
val timeout = 10000
@ -309,7 +311,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"fold by composing" in {
val actors = (1 to 10).toList map { _
actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); 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] }
@ -323,8 +325,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
def receive = {
case (add: Int, wait: Int)
Thread.sleep(wait)
if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected")
channel.tryTell(add)
if (add == 6) sender ! Status.Failure(new IllegalArgumentException("shouldFoldResultsWithException: expected"))
else sender.tell(add)
}
})
}
@ -356,7 +358,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"shouldReduceResults" in {
val actors = (1 to 10).toList map { _
actorOf(new Actor {
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); channel.tryTell(add) }
def receive = { case (add: Int, wait: Int) Thread.sleep(wait); sender.tell(add) }
})
}
val timeout = 10000
@ -371,8 +373,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
def receive = {
case (add: Int, wait: Int)
Thread.sleep(wait)
if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected")
channel.tryTell(add)
if (add == 6) sender ! Status.Failure(new IllegalArgumentException("shouldFoldResultsWithException: expected"))
else sender.tell(add)
}
})
}
@ -401,7 +403,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
var counter = 1
def receive = {
case 'GetNext
channel ! counter
sender ! counter
counter += 2
}
})
@ -831,6 +833,22 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
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 akka.util._
import akka.util.Duration._
import akka.actor.{ LocalActorRef, Actor, NullChannel }
import akka.actor.{ LocalActorRef, Actor }
import akka.testkit.AkkaSpec
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
@ -80,7 +80,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
result
}
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, NullChannel)
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, app.deadLetters)
def ensureInitialMailboxState(config: MailboxType, q: Mailbox) {
q must not be null

View file

@ -30,7 +30,7 @@ class PriorityDispatcherSpec extends AkkaSpec {
def receive = {
case i: Int acc = i :: acc
case 'Result channel.tryTell(acc)
case 'Result sender.tell(acc)
}
}).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 price: Long
def volume: Long
def nanoTime: Long
def withNanoTime: Order
}
case class Bid(
orderbookSymbol: String,
price: Long,
volume: Long)
volume: Long,
nanoTime: Long = 0L)
extends Order {
def split(newVolume: Long) = {
new Bid(orderbookSymbol, price, newVolume)
}
def withNanoTime: Bid = copy(nanoTime = System.nanoTime)
}
case class Ask(
orderbookSymbol: String,
price: Long,
volume: Long)
volume: Long,
nanoTime: Long = 0L)
extends Order {
def split(newVolume: Long) = {
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 {
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.actor._
@ -11,7 +11,7 @@ trait MatchingEngine {
val orderbooks: List[Orderbook]
val supportedOrderbookSymbols = orderbooks map (_.symbol)
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
def receive = {
case standbyRef: ActorRef
standby = Some(standbyRef)
case order: Order
handleOrder(order)
case standbyRef: ActorRef
standby = Some(standbyRef)
case unknown
log.warning("Received unknown message: " + unknown)
}
@ -32,30 +32,21 @@ class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook])
def handleOrder(order: Order) {
orderbooksMap.get(order.orderbookSymbol) match {
case Some(orderbook)
val pendingStandbyReply: Option[Future[_]] =
for (s standby) yield { s ? order }
standby.foreach(_ forward order)
orderbook.addOrder(order)
orderbook.matchOrders()
// wait for standby reply
pendingStandbyReply.foreach(waitForStandby(_))
done(true)
done(true, order)
case None
log.warning("Orderbook not handled by this MatchingEngine: " + order.orderbookSymbol)
done(false)
}
}
def done(status: Boolean) {
channel ! new Rsp(status)
}
def waitForStandby(pendingStandbyFuture: Future[_]) {
try {
pendingStandbyFuture.await
} catch {
case e: FutureTimeoutException
log.error("Standby timeout: " + e)
def done(status: Boolean, order: Order) {
if (standby.isEmpty) {
sender ! Rsp(order, status)
}
}

View file

@ -1,4 +1,4 @@
package akka.performance.trading.common
package akka.performance.trading.system
import akka.performance.trading.domain._
import akka.actor._
@ -28,9 +28,9 @@ class AkkaOrderReceiver extends Actor with OrderReceiver with ActorLogging {
type ME = ActorRef
def receive = {
case order: Order placeOrder(order)
case routing @ MatchingEngineRouting(mapping)
refreshMatchingEnginePartitions(routing.asInstanceOf[MatchingEngineRouting[ActorRef]])
case order: Order placeOrder(order)
case unknown log.warning("Received unknown message: " + unknown)
}
@ -38,10 +38,10 @@ class AkkaOrderReceiver extends Actor with OrderReceiver with ActorLogging {
val matchingEngine = matchingEngineForOrderbook.get(order.orderbookSymbol)
matchingEngine match {
case Some(m)
m.forward(order)
m forward order
case None
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.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 isBaseline(stats: Stats): Boolean
def saveHtmlReport(content: String, name: String): Unit
def htmlReportUrl(name: String): String
@ -31,10 +33,11 @@ trait 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 baselineStats = MutableMap[Key, 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
protected val maxHistorical = 7
val log = Logging(app, this)
case class Key(name: String, load: Int)
def add(stats: Stats) {
@ -62,13 +63,18 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep
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] = {
val key = Key(name, load)
val historical = historicalStats.getOrElse(key, IndexedSeq.empty)
val baseline = baselineStats.get(key)
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() {
@ -105,7 +111,8 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep
out.writeObject(stats)
} catch {
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 {
if (out ne null) try { out.close() } catch { case ignore: Exception }
}
@ -121,7 +128,6 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep
Some(stats)
} catch {
case e: Throwable
log.error("Failed to load from [{}], due to [{}]", f.getAbsolutePath, e.getMessage)
None
} finally {
if (in ne null) try { in.close() } catch { case ignore: Exception }
@ -144,7 +150,8 @@ class FileBenchResultRepository(val app: AkkaApplication) extends BenchResultRep
writer.flush()
} catch {
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 {
if (writer ne null) try { writer.close() } catch { case ignore: Exception }
}

View file

@ -13,6 +13,72 @@ object GoogleChartBuilder {
val ChartWidth = 750
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.
*/
@ -113,6 +179,11 @@ object GoogleChartBuilder {
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) {
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,8 +7,10 @@ import scala.collection.JavaConversions.asScalaBuffer
import scala.collection.JavaConversions.enumerationAsScalaIterator
import akka.AkkaApplication
import akka.event.Logging
import scala.collection.immutable.TreeMap
class Report(app: AkkaApplication,
class Report(
app: AkkaApplication,
resultRepository: BenchResultRepository,
compareResultWith: Option[String] = None) {
@ -33,9 +35,13 @@ class Report(app: AkkaApplication,
sb.append(resultTable)
sb.append("\n</pre>\n")
sb.append(img(percentilesAndMeanChart(current)))
sb.append(img(latencyAndThroughputChart(current)))
compareWithHistoricalTpsChart(statistics).foreach(url sb.append(img(url)))
if (current.max > 0L) {
sb.append(img(percentilesAndMeanChart(current)))
for (stats statistics) {
compareWithHistoricalPercentiliesAndMeanChart(stats).foreach(url sb.append(img(url)))
}
@ -43,6 +49,7 @@ class Report(app: AkkaApplication,
for (stats statistics) {
comparePercentilesAndMeanChart(stats).foreach(url sb.append(img(url)))
}
}
sb.append("<hr/>\n")
sb.append("<pre>\n")
@ -64,6 +71,11 @@ class Report(app: AkkaApplication,
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 = {
val chartTitle = stats.name + " Percentiles and Mean (microseconds)"
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)
if (withHistorical.size > 1) {
val chartTitle = stats.name + " vs. historical, " + stats.load + " clients" + ", Percentiles and Mean (microseconds)"
val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(withHistorical, chartTitle,
stats legendTimeFormat.format(new Date(stats.timestamp)))
val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(withHistorical, chartTitle, timeLegend)
Some(chartUrl)
} else {
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 = {
val chartTitle = stats.name + " Latency (microseconds) and Throughput (TPS)"
val chartUrl = GoogleChartBuilder.latencyAndThroughputChartUrl(resultRepository.get(stats.name), chartTitle)

View file

@ -7,14 +7,23 @@ case class Stats(
name: String,
load: Int,
timestamp: Long = System.currentTimeMillis,
durationNanos: Long,
n: Long,
min: Long,
max: Long,
mean: Double,
tps: Double,
percentiles: TreeMap[Int, Long]) {
durationNanos: Long = 0L,
n: Long = 0L,
min: Long = 0L,
max: Long = 0L,
mean: Double = 0.0,
tps: Double = 0.0,
percentiles: TreeMap[Int, Long] = Stats.emptyPercentiles) {
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 _
count.incrementAndGet
latch.countDown()
channel.tryTell("success")
sender.tell("success")
}
}))
@ -88,7 +88,7 @@ class ActorPoolSpec extends AkkaSpec {
def receive = {
case req: String {
(10 millis).dilated.sleep
channel.tryTell("Response")
sender.tell("Response")
}
}
}))
@ -111,7 +111,7 @@ class ActorPoolSpec extends AkkaSpec {
val count = new AtomicInteger(0)
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 receive = {
case n: Int
@ -233,7 +233,7 @@ class ActorPoolSpec extends AkkaSpec {
def instance(p: Props): ActorRef = actorOf(p.withCreator(new Actor {
def receive = {
case _
delegates put (self.uuid.toString, "")
delegates put (self.address, "")
latch1.countDown()
}
}))
@ -261,7 +261,7 @@ class ActorPoolSpec extends AkkaSpec {
def instance(p: Props) = actorOf(p.withCreator(new Actor {
def receive = {
case _
delegates put (self.uuid.toString, "")
delegates put (self.address, "")
latch2.countDown()
}
}))
@ -331,7 +331,7 @@ class ActorPoolSpec extends AkkaSpec {
"support typed actors" in {
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 upperBound = 5
def pressureThreshold = 1
@ -340,15 +340,17 @@ class ActorPoolSpec extends AkkaSpec {
def rampupRate = 0.1
def backoffRate = 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
}, 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)
}
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)
app.typedActor.stop(pool)
}
"provide default supervision of pooled actors" in {
@ -358,7 +360,7 @@ class ActorPoolSpec extends AkkaSpec {
val keepDying = new AtomicBoolean(false)
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 upperBound = 5
def rampupRate = 0.1
@ -381,7 +383,7 @@ class ActorPoolSpec extends AkkaSpec {
}).withFaultHandler(faultHandler))
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 upperBound = 5
def rampupRate = 0.1
@ -479,7 +481,7 @@ class ActorPoolSpec extends AkkaSpec {
object BadState
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 upperBound = 5
def rampupRate = 0.1

View file

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

View file

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

View file

@ -7,6 +7,9 @@ package akka.serialization
import akka.serialization.Serialization._
import scala.reflect._
import akka.testkit.AkkaSpec
import akka.AkkaApplication
import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import akka.actor.DeadLetterActorRef
object SerializeSpec {
@BeanInfo
@ -61,5 +64,20 @@ class SerializeSpec extends AkkaSpec {
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 {
import ActorModelSpec._
def newInterceptedDispatcher = new CallingThreadDispatcher(app, "test", true) with MessageDispatcherInterceptor
def newInterceptedDispatcher = new CallingThreadDispatcher(app, "test") with MessageDispatcherInterceptor
def dispatcherType = "Calling Thread Dispatcher"
}

View file

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

View file

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

View file

@ -13,7 +13,6 @@ import com.eaio.uuid.UUID
import akka.dispatch.{ Dispatchers, Future }
import akka.util.Duration
import akka.util.ReflectiveAccess
import java.util.concurrent.TimeUnit
import akka.routing.Routing
import akka.remote.RemoteSupport
import akka.serialization.Serialization
@ -156,17 +155,17 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor
case value value
}
val hostname: String = System.getProperty("akka.remote.hostname") match {
case null | "" InetAddress.getLocalHost.getHostName
val defaultAddress = new InetSocketAddress(System.getProperty("akka.remote.hostname") match {
case null | "" InetAddress.getLocalHost.getHostAddress
case value value
}
val port: Int = System.getProperty("akka.remote.port") match {
}, System.getProperty("akka.remote.port") match {
case null | "" AkkaConfig.RemoteServerPort
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
val mainbus = new MainBus(DebugMainBus)

View file

@ -39,7 +39,7 @@ class AkkaException(message: String = "", cause: Throwable = null) extends Runti
object AkkaException {
val hostname = try {
InetAddress.getLocalHost.getHostName
InetAddress.getLocalHost.getHostAddress
} catch {
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.
* 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
case class Success(status: AnyRef) 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.
*/
@transient
private[akka] val context: ActorContext = {
private[akka] implicit val context: ActorContext = {
val contextStack = ActorCell.contextStack.get
def noContextError = {
def noContextError =
throw new ActorInitializationException(
"\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\t\t'val actor = Actor.actorOf[MyActor]', or" +
"\n\t\t'val actor = Actor.actorOf(new MyActor(..))'")
}
if (contextStack.isEmpty) noContextError
val context = contextStack.head
@ -286,14 +286,6 @@ trait Actor {
@inline
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
* When specified, the receive method should be able to handle a 'ReceiveTimeout' message.
@ -329,7 +321,7 @@ trait Actor {
* def receive = {
* case Ping =&gt;
* println("got a 'Ping' message")
* channel ! "pong"
* sender ! "pong"
*
* case OneWay =&gt;
* println("got a 'OneWay' message")
@ -405,13 +397,13 @@ trait Actor {
/**
* 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
/**
* 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
@ -430,10 +422,7 @@ trait Actor {
case f: Failed context.handleFailure(f)
case ct: ChildTerminated context.handleChildTerminated(ct.child)
case Kill throw new ActorKilledException("Kill")
case PoisonPill
val ch = channel
self.stop()
ch.sendException(new ActorKilledException("PoisonPill"))
case PoisonPill self.stop()
}
}

View file

@ -10,7 +10,6 @@ import scala.annotation.tailrec
import scala.collection.immutable.{ Stack, TreeMap }
import scala.collection.JavaConverters
import java.util.concurrent.{ ScheduledFuture, TimeUnit }
import java.util.{ Collection JCollection, Collections JCollections }
import akka.AkkaApplication
import akka.event.Logging.{ Debug, Warning, Error }
@ -39,8 +38,6 @@ private[akka] trait ActorContext extends ActorRefFactory with TypedActorFactory
def sender: ActorRef
def channel: UntypedChannel
def children: Iterable[ActorRef]
def dispatcher: MessageDispatcher
@ -51,6 +48,7 @@ private[akka] trait ActorContext extends ActorRefFactory with TypedActorFactory
def app: AkkaApplication
def parent: ActorRef
}
private[akka] object ActorCell {
@ -67,17 +65,17 @@ private[akka] class ActorCell(
val app: AkkaApplication,
val self: ActorRef with ScalaActorRef,
val props: Props,
val supervisor: ActorRef,
val parent: ActorRef,
var receiveTimeout: Option[Long],
var hotswap: Stack[PartialFunction[Any, Unit]]) extends ActorContext {
import ActorCell._
protected def guardian = self
protected final def guardian = self
protected def typedActor = app.typedActor
def provider = app.provider
final def provider = app.provider
var futureTimeout: Option[ScheduledFuture[AnyRef]] = None
@ -85,77 +83,58 @@ private[akka] class ActorCell(
var currentMessage: Envelope = null
var actor: Actor = _ //FIXME We can most probably make this just a regular reference to Actor
def uuid: Uuid = self.uuid
var actor: Actor = _
@inline
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
var mailbox: Mailbox = _
def hasMessages: Boolean = mailbox.hasMessages
def start(): Unit = {
final def start(): Unit = {
mailbox = dispatcher.createMailbox(this)
// ➡➡➡ 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)
}
// ➡➡➡ 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 ⬅⬅⬅
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 ⬅⬅⬅
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 ⬅⬅⬅
dispatcher.systemDispatch(this, Link(subject))
subject
}
def stopsMonitoring(subject: ActorRef): ActorRef = {
final def stopsMonitoring(subject: ActorRef): ActorRef = {
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
dispatcher.systemDispatch(this, Unlink(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(
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = {
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 {
final def sender: ActorRef = currentMessage match {
case null app.deadLetters
case msg if msg.channel.isInstanceOf[ActorRef] msg.channel.asInstanceOf[ActorRef]
case msg if msg.sender ne null msg.sender
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
protected def newActor(): Actor = {
val stackBefore = contextStack.get
@ -174,7 +153,7 @@ private[akka] class ActorCell(
}
}
def systemInvoke(message: SystemMessage) {
final def systemInvoke(message: SystemMessage) {
def create(): Unit = try {
val created = newActor()
@ -189,7 +168,7 @@ private[akka] class ActorCell(
// prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this)
} 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
dispatcher.suspend(this)
} 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 {
try {
supervisor ! ChildTerminated(self)
parent ! ChildTerminated(self)
app.deathWatch.publish(Terminated(self))
} finally {
currentMessage = null
@ -292,7 +271,7 @@ private[akka] class ActorCell(
}
}
def invoke(messageHandle: Envelope) {
final def invoke(messageHandle: Envelope) {
try {
val isClosed = mailbox.isClosed //Fence plus volatile read
if (!isClosed) {
@ -313,14 +292,12 @@ private[akka] class ActorCell(
// make sure that InterruptedException does not leave this thread
if (e.isInstanceOf[InterruptedException]) {
val ex = ActorInterruptedException(e)
channel.sendException(ex)
props.faultHandler.handleSupervisorFailing(self, children)
supervisor ! Failed(self, ex)
parent ! Failed(self, ex)
throw e //Re-throw InterruptedExceptions as expected
} else {
channel.sendException(e)
props.faultHandler.handleSupervisorFailing(self, children)
supervisor ! Failed(self, e)
parent ! Failed(self, e)
}
} finally {
checkReceiveTimeout // Reschedule receive timeout
@ -330,27 +307,24 @@ private[akka] class ActorCell(
app.mainbus.publish(Error(e, self, e.getMessage))
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 None app.mainbus.publish(Warning(self, "dropping " + fail + " from unknown child"))
}
def handleChildTerminated(child: ActorRef): Unit = {
final def handleChildTerminated(child: ActorRef): Unit = {
_children -= child
props.faultHandler.handleChildTerminated(child, children)
}
// ➡➡➡ 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()
val recvtimeout = receiveTimeout
if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) {
@ -359,16 +333,16 @@ private[akka] class ActorCell(
}
}
def cancelReceiveTimeout() {
final def cancelReceiveTimeout() {
if (futureTimeout.isDefined) {
futureTimeout.get.cancel(true)
futureTimeout = None
}
}
def clearActorContext(): Unit = setActorContext(null)
final def clearActorContext(): Unit = setActorContext(null)
def setActorContext(newContext: ActorContext) {
final def setActorContext(newContext: ActorContext) {
@tailrec
def lookupAndSetSelfFields(clazz: Class[_], actor: Actor, newContext: ActorContext): Boolean = {
val success = try {
@ -392,12 +366,4 @@ private[akka] class ActorCell(
if (a ne null)
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 akka.AkkaApplication
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.
@ -41,7 +44,7 @@ import akka.event.ActorEventBus
*
* @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
// 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
private[akka] def uuid: Uuid //TODO FIXME REMOVE THIS
/**
* Comparison only takes address into account.
*/
def compareTo(other: ActorRef) = this.address compareTo other.address
/**
* Akka Java API. <p/>
* @see ask(message: AnyRef, sender: ActorRef): Future[_]
* Uses the specified timeout (milliseconds)
* Sends the specified message to the sender, i.e. fire-and-forget semantics.<p/>
* <pre>
* 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/>
@ -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
* implement request/response message exchanges.
* <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.
*/
def ask(message: AnyRef, timeout: Long, sender: ActorRef): Future[AnyRef] =
?(message, Timeout(timeout))(sender).asInstanceOf[Future[AnyRef]]
def ask(message: AnyRef, timeout: Long): Future[AnyRef] = ?(message, Timeout(timeout)).asInstanceOf[Future[AnyRef]]
/**
* Akka Java API. <p/>
* Forwards the message specified to this actor and preserves the original sender of the message
* Forwards the message and passes the original sender actor as the sender.
* <p/>
* Works with '!' and '?'/'ask'.
*/
def forward(message: AnyRef, sender: ActorRef) {
if (sender eq null) throw new IllegalArgumentException("The 'sender' argument to 'forward' can't be null")
else forward(message)(ForwardableChannel(sender))
}
def forward(message: Any)(implicit context: ActorContext) = postMessageToMailbox(message, context.sender)
/**
* 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
* 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
@ -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
* 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
@ -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>
*/
class LocalActorRef private[akka] (
app: AkkaApplication,
_app: AkkaApplication,
props: Props,
_supervisor: ActorRef,
_givenAddress: String,
@ -156,7 +166,7 @@ class LocalActorRef private[akka] (
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()
/**
@ -192,7 +202,7 @@ class LocalActorRef private[akka] (
* This means that this actor will get a Terminated()-message when the provided actor
* 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)
@ -201,7 +211,7 @@ class LocalActorRef private[akka] (
* This means that this actor will not get a Terminated()-message when the provided actor
* 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)
@ -222,28 +232,14 @@ class LocalActorRef private[akka] (
protected[akka] def sendSystemMessage(message: SystemMessage) { underlying.dispatcher.systemDispatch(underlying, message) }
protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit =
actorCell.postMessageToMailbox(message, channel)
protected[akka] def postMessageToMailbox(message: Any, sender: ActorRef): Unit = actorCell.postMessageToMailbox(message, sender)
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = {
actorCell.postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, channel)
}
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = actorCell.provider.ask(message, this, timeout)
protected[akka] def handleFailure(fail: Failed): Unit = actorCell.handleFailure(fail)
protected[akka] def restart(cause: Throwable): Unit = actorCell.restart(cause)
// ========= PRIVATE FUNCTIONS =========
protected[akka] override def restart(cause: Throwable): Unit = actorCell.restart(cause)
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = {
// 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)
}
private def writeReplace(): AnyRef = actorCell.provider.serialize(this)
}
/**
@ -251,7 +247,7 @@ class LocalActorRef private[akka] (
* There are implicit conversions in ../actor/Implicits.scala
* from ActorRef -> ScalaActorRef and back
*/
trait ScalaActorRef extends ReplyChannel[Any] { ref: ActorRef
trait ScalaActorRef { ref: ActorRef
protected[akka] def sendSystemMessage(message: SystemMessage): Unit
@ -269,28 +265,16 @@ trait ScalaActorRef extends ReplyChannel[Any] { ref: ActorRef ⇒
* </pre>
* <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.
*/
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)
/**
* 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 postMessageToMailbox(message: Any, sender: ActorRef): 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
*/
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
def this(address: String, inet: InetSocketAddress) = this(address, inet.getAddress.getHostAddress, inet.getPort)
@throws(classOf[java.io.ObjectStreamException])
def readResolve(): AnyRef = {
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 {
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
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))
def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
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 {
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 startsMonitoring(actorRef: ActorRef): ActorRef = actorRef
@ -368,52 +346,70 @@ trait MinimalActorRef extends ActorRef with ScalaActorRef {
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
private def unsupported = throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName))
def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
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)
override val address: String = "akka:internal:DeadLetterActorRef"
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(
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = { app.mainbus.publish(DeadLetter(message, channel)); brokenPromise }
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = {
app.mainbus.publish(DeadLetter(message, this))
brokenPromise
}
abstract class AskActorRef(promise: Promise[Any], app: AkkaApplication) extends MinimalActorRef {
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = DeadLetterActorRef.serialized
}
promise onComplete { _ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
promise onTimeout { _ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
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)
{
val callback: Future[Any] Unit = { _ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
result onComplete callback
result onTimeout callback
}
protected def whenDone(): Unit
override protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = message match {
case akka.actor.Status.Success(r) promise.completeWithResult(r)
case akka.actor.Status.Failure(f) promise.completeWithException(f)
case other promise.completeWithResult(other)
protected[akka] override def postMessageToMailbox(message: Any, sender: ActorRef): Unit = message match {
case Status.Success(r) result.completeWithResult(r)
case Status.Failure(f) result.completeWithException(f)
case other result.completeWithResult(other)
}
override protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = {
postMessageToMailbox(message, channel)
promise
protected[akka] override def sendSystemMessage(message: SystemMessage): Unit = message match {
case _: Terminate stop()
case _
}
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]
/**
* 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 evict(address: String): Boolean
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef]
private[akka] def serialize(actor: ActorRef): SerializedActorRef
private[akka] def createDeathWatch(): DeathWatch
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 {
private[akka] val deployer: Deployer = new Deployer(app)
val terminationFuture = new DefaultPromise[AkkaApplication.ExitStatus](Timeout.never)(app.dispatcher)
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.
*/
private[akka] val theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = new UnsupportedActorRef {
@volatile
var stopped = false
override def address = app.name + ":BubbleWalker"
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 {
case Failed(child, ex) child.stop()
case ChildTerminated(child) terminationFuture.completeWithResult(AkkaApplication.Stopped)
@ -144,7 +160,7 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
actors.putIfAbsent(address, newFuture) match {
case null
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
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
case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, _, DeploymentConfig.LocalScope))
val routerFactory: () Router = DeploymentConfig.routerTypeFor(routerType) match {
case RouterType.Direct () new DirectRouter
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)
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 {
case e: Exception
@ -206,10 +223,11 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
// val localOnly = props.localOnly
// if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props)
// 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 serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(actor.address, app.defaultAddress)
private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch
@ -217,12 +235,11 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
import akka.dispatch.{ Future, Promise, DefaultPromise }
(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 other
val result = new DefaultPromise[Any](other)(app.dispatcher)
val a = new AskActorRef(result, app) { def whenDone() = actors.remove(this) }
case t
val a = new AskActorRef(app)(timeout = t) { def whenDone() = actors.remove(this) }
assert(actors.putIfAbsent(a.address, a) eq null) //If this fails, we're in deep trouble
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.config.{ Configuration, ConfigurationException }
import akka.util.Duration
import java.net.InetSocketAddress
trait ActorDeployer {
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 log = Logging(app.mainbus, this)
// val defaultAddress = Node(Config.nodename)
lazy val instance: ActorDeployer = {
val deployer = if (app.reflective.ClusterModule.isEnabled) app.reflective.ClusterModule.clusterDeployer else LocalDeployer
val instance: ActorDeployer = {
val deployer = new LocalDeployer()
deployer.init(deploymentsInConfig)
deployer
}
@ -73,25 +72,8 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer {
}
}
private[akka] def lookupDeploymentFor(address: String): Option[Deploy] = {
val deployment_? = 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 lookupDeploymentFor(address: String): Option[Deploy] =
instance.lookupDeploymentFor(address)
private[akka] def deploymentsInConfig: List[Deploy] = {
for {
@ -249,7 +231,8 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer {
case e: Exception 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>
*/
object LocalDeployer extends ActorDeployer {
class LocalDeployer extends ActorDeployer {
private val deployments = new ConcurrentHashMap[String, Deploy]
private[akka] def init(deployments: Seq[Deploy]) {
deployments foreach (deploy(_)) // deploy
}
private[akka] def init(deployments: Seq[Deploy]): Unit = deployments foreach deploy // deploy
private[akka] def shutdown() {
deployments.clear() //TODO do something else/more?
}
private[akka] def shutdown(): Unit = deployments.clear() //TODO do something else/more?
private[akka] def deploy(deployment: Deploy) {
deployments.putIfAbsent(deployment.address, deployment)
}
private[akka] def deploy(deployment: Deploy): Unit = deployments.putIfAbsent(deployment.address, deployment)
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
}
generation += 1
processMsg(value, channel)
processMsg(value, sender)
}
}
@ -504,7 +504,7 @@ trait FSM[S, D] extends ListenerManagement {
nextState.stopReason match {
case None makeTransition(nextState)
case _
nextState.replies.reverse foreach { r channel ! r }
nextState.replies.reverse foreach { r sender ! r }
terminate(nextState)
self.stop()
}
@ -514,7 +514,7 @@ trait FSM[S, D] extends ListenerManagement {
if (!stateFunctions.contains(nextState.stateName)) {
terminate(stay withStopReason Failure("Next state %s does not exist".format(nextState.stateName)))
} else {
nextState.replies.reverse foreach { r channel ! r }
nextState.replies.reverse foreach { r sender ! r }
if (currentState.stateName != nextState.stateName) {
handleTransition(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 {
case s: String s
case Timer(name, _, _, _) "timer " + name
case c: UntypedChannel c.toString
case a: ActorRef a.toString
case _ "unknown"
}
log.debug("processing " + event + " from " + srcstr)

View file

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

View file

@ -317,7 +317,7 @@ class TypedActor(val app: AkkaApplication) {
case Props.`defaultTimeout` app.AkkaConfig.ActorTimeout
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
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
@ -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] 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
def receive = {
case m: MethodCall
@ -339,13 +334,23 @@ class TypedActor(val app: AkkaApplication) {
TypedActor.appReference set app
try {
if (m.isOneWay) m(me)
else if (m.returnsFuture_?) {
channel match {
case p: ActorPromise p completeWith m(me).asInstanceOf[Future[Any]]
case _ throw new IllegalStateException("Future-returning TypedActor didn't use ?/ask so cannot reply")
else {
val s = sender
try {
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 {
TypedActor.selfReference 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 invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = method.getName match {
@ -363,17 +368,15 @@ class TypedActor(val app: AkkaApplication) {
case _
MethodCall(app, method, args) match {
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_?
val f = actor ? m
try { f.await } catch { case _: FutureTimeoutException }
f.value match {
val f = actor.?(m, timeout)
(try { f.await.value } catch { case _: FutureTimeoutException None }) match {
case None | Some(Right(null)) if (m.returnsJOption_?) JOption.none[Any] else None
case Some(Right(joption: AnyRef)) joption
case Some(Left(ex)) throw ex
}
case m
(actor ? m).get.asInstanceOf[AnyRef]
case m (actor.?(m, timeout)).get.asInstanceOf[AnyRef]
}
}
}

View file

@ -21,7 +21,7 @@ import akka.dispatch.{ MessageDispatcher, Promise }
*
* if (msg.equals("UseReply")) {
* // 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()) {
* // 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.
* Is defined if the message was sent from another Actor, else None.
*/
def getSender: ActorRef = sender
/**
* Abstraction for unification of sender and senderFuture for later reply
*/
def getChannel: UntypedChannel = channel
def getSender(): ActorRef = sender
/**
* Gets the current receive timeout

View file

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

View file

@ -17,7 +17,7 @@ import scala.annotation.tailrec
/**
* @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")
}
@ -107,7 +107,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable
object DeadLetterMailbox extends Mailbox(null) {
becomeClosed()
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 systemEnqueue(handle: SystemMessage): Unit = ()
override def systemDrain(): SystemMessage = null

View file

@ -37,7 +37,7 @@ class BalancingDispatcher(
_timeoutMs: Long)
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 {
case u: UnboundedMailbox new QueueBasedMessageQueue with UnboundedMessageQueueSemantics {

View file

@ -65,7 +65,7 @@ import akka.AkkaApplication
*/
class Dispatcher(
_app: AkkaApplication,
_name: String,
val name: String,
val throughput: Int,
val throughputDeadlineTime: Int,
val mailboxType: MailboxType,
@ -73,8 +73,6 @@ class Dispatcher(
val timeoutMs: Long)
extends MessageDispatcher(_app) {
val name = "akka:event-driven:dispatcher:" + _name
protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name)
protected[akka] val executorService = new AtomicReference[ExecutorService](new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService))

View file

@ -61,7 +61,7 @@ class Dispatchers(val app: AkkaApplication) {
*/
def newPinnedDispatcher(actor: LocalActorRef) = actor match {
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 {
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.event.Logging.Error
import akka.actor.{ UntypedChannel, Timeout, ExceptionChannel }
import akka.actor.Timeout
import scala.Option
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
/**
* 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]]]() {
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 {
case c: ClassCastException
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)
*/
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
private[akka] object FState {
def apply[T](): FState[T] = EmptyPending.asInstanceOf[FState[T]]
private[dispatch] object DefaultPromise {
def EmptyPending[T](): FState[T] = emptyPendingValue.asInstanceOf[FState[T]]
/**
* Represents the internal state of the DefaultCompletableFuture
@ -824,7 +854,7 @@ private[akka] object FState {
case object Expired extends FState[Nothing] {
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] {
self
import FState.{ FState, Success, Failure, Pending, Expired }
import DefaultPromise.{ FState, Success, Failure, Pending, Expired }
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 =
if (timeout.duration.isFinite && atMost.isFinite)
atMost.toNanos min timeLeft()
@ -997,31 +1029,6 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
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
* a Future-composition but you already have a value to contribute.

View file

@ -22,13 +22,13 @@ trait EventBus {
/**
* 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
/**
* 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

View file

@ -3,7 +3,7 @@
*/
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.AkkaApplication.AkkaConfig
import akka.util.ReflectiveAccess
@ -341,7 +341,7 @@ object Logging {
*/
class StandardOutLogger extends MinimalActorRef with StdOutLogger {
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 StandardOutLoggerName = StandardOutLogger.getClass.getName

View file

@ -21,78 +21,8 @@ import java.lang.reflect.InvocationTargetException
class RemoteException(message: String) extends AkkaException(message)
trait RemoteService {
def server: RemoteSupport
def address: InetSocketAddress
}
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
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() {
this.shutdownClientModule()
this.shutdownServerModule()
clear
}
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
*/
trait RemoteServerModule extends RemoteModule { this: RemoteSupport
protected val guard = new ReentrantGuard
/**
* Signals whether the server is up and running or not
*/
@ -222,39 +139,6 @@ trait RemoteServerModule extends RemoteModule { this: RemoteSupport ⇒
*/
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
*/
@ -264,74 +148,9 @@ trait RemoteServerModule extends RemoteModule { this: RemoteSupport ⇒
* Shuts the server down
*/
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
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.
*/
@ -349,13 +168,9 @@ trait RemoteClientModule extends RemoteModule { self: RemoteSupport ⇒
/** 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[T](message: Any,
protected[akka] def send(message: Any,
senderOption: Option[ActorRef],
senderFuture: Option[Promise[T]],
remoteAddress: InetSocketAddress,
isOneWay: Boolean,
actorRef: ActorRef,
loader: Option[ClassLoader]): Option[Promise[T]]
recipient: ActorRef,
loader: Option[ClassLoader]): Unit
}

View file

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

View file

@ -95,6 +95,10 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒
val defaultProps: Props = Props.default.withDispatcher(this.context.dispatcher)
override def preStart() {
resizeIfAppropriate()
}
override def postStop() {
_delegates foreach evict
_delegates = Vector.empty
@ -103,7 +107,7 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒
protected def _route(): Actor.Receive = {
// for testing...
case Stat
channel.tryTell(Stats(_delegates length))
sender ! Stats(_delegates length)
case Terminated(victim)
_delegates = _delegates filterNot { victim == }
case msg
@ -112,7 +116,7 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒
select(_delegates) foreach { _ forward msg }
}
private def resizeIfAppropriate() {
protected def resizeIfAppropriate() {
val requestedCapacity = capacity(_delegates)
val newDelegates = requestedCapacity match {
case qty if qty > 0
@ -285,16 +289,16 @@ trait MailboxPressureCapacitor {
/**
* 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
* delegates are tied up actively processing a message, as long as the
* messages have somebody waiting on the result. "One way" messages with
* no reply would not be counted.
* delegates are tied up actively processing a message
*/
trait ActiveFuturesPressureCapacitor {
trait ActiveActorsPressureCapacitor {
def pressure(delegates: Seq[ActorRef]): Int =
delegates count {
case a: LocalActorRef a.underlying.channel.isInstanceOf[Promise[_]]
case a: LocalActorRef
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.
*
* FIXME: this is extremely vague currently since there are so many ways to define least amount of ram.
*/
object LeastRAM extends RouterType

View file

@ -41,7 +41,7 @@ trait Router {
*
* @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
@ -49,7 +49,7 @@ trait Router {
*
* @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.
*/
abstract private[akka] class AbstractRoutedActorRef(val props: RoutedProps) extends UnsupportedActorRef {
private[akka] override val uuid: Uuid = newUuid
abstract private[akka] class AbstractRoutedActorRef(val app: AkkaApplication, val props: RoutedProps) extends UnsupportedActorRef {
val router = props.routerFactory()
override def postMessageToMailbox(message: Any, channel: UntypedChannel) = {
val sender = channel match {
case ref: ActorRef Some(ref)
case _ None
}
router.route(message)(sender)
}
override def postMessageToMailbox(message: Any, sender: ActorRef) = router.route(message)(sender)
override def postMessageToMailboxAndCreateFutureResultWithTimeout(
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)
}
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = router.route(message, timeout)
}
/**
* 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.
*/
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
private var running: Boolean = true
@ -130,7 +115,7 @@ private[akka] class RoutedActorRef(val routedProps: RoutedProps, override val ad
synchronized {
if (running) {
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
* 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 {
@ -153,8 +136,9 @@ trait BasicRouter extends Router {
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)
//it is a broadcast message, we are going to send to message to all connections.
connectionManager.connections.iterable foreach { connection
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)
throw new RoutingException("Broadcasting using '?'/'ask' is for the time being is not supported. Use ScatterGatherRouter.")
case _
@ -189,8 +173,7 @@ trait BasicRouter extends Router {
next match {
case Some(connection)
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)(sender).asInstanceOf[Future[T]]
connection.?(message, timeout).asInstanceOf[Future[T]] //FIXME this does not preserve the original sender, shouldn't it?
} catch {
case e: Exception
connectionManager.remove(connection)
@ -203,10 +186,7 @@ trait BasicRouter extends Router {
protected def next: Option[ActorRef]
private def throwNoConnectionsError = {
val error = new RoutingException("No replica connections for router")
throw error
}
private def throwNoConnectionsError = throw new RoutingException("No replica connections for router")
}
/**
@ -257,15 +237,17 @@ class DirectRouter extends BasicRouter {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RandomRouter extends BasicRouter {
import java.security.SecureRandom
private val state = new AtomicReference[RandomRouterState]
//FIXME: threadlocal random?
private val random = new java.util.Random(System.nanoTime)
private val random = new ThreadLocal[SecureRandom] {
override def initialValue = SecureRandom.getInstance("SHA1PRNG")
}
def next: Option[ActorRef] = currentState.array match {
case a if a.isEmpty None
case a Some(a(random.nextInt(a.length)))
case a Some(a(random.get.nextInt(a.length)))
}
@tailrec
@ -358,11 +340,11 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
*/
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
try {
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 {
case e: Exception
connectionManager.remove(actor)
@ -375,9 +357,9 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
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 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
* @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]] = {
val set = container get key

View file

@ -5,15 +5,12 @@
package akka.util
import akka.dispatch.Envelope
import akka.config.ModuleNotAvailableException
import akka.actor._
import DeploymentConfig.ReplicationScheme
import akka.config.ModuleNotAvailableException
import akka.event.Logging.Debug
import akka.cluster.ClusterNode
import akka.remote.{ RemoteSupport, RemoteService }
import akka.routing.{ RoutedProps, Router }
import java.net.InetSocketAddress
import akka.AkkaApplication
object ReflectiveAccess {
@ -140,110 +137,4 @@ class ReflectiveAccess(val app: AkkaApplication) {
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 method: Method
val uuid = actorRef.uuid.toString
val methodName = method.getName
val methodUuid = "%s_%s" format (uuid, methodName)

View file

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

View file

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

View file

@ -97,9 +97,9 @@ trait ProducerSupport { this: Actor ⇒
val exchange = createExchange(pattern).fromRequestMessage(cmsg)
processor.process(exchange, new AsyncCallback {
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.
val replyChannel = channel
val replyChannel = sender
def done(doneSync: Boolean) {
(doneSync, exchange.isFailed) match {
@ -159,7 +159,7 @@ trait ProducerSupport { this: Actor ⇒
* actor).
*/
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.CamelMessageConversion.toExchangeAdapter
import scala.reflect.BeanProperty
import akka.dispatch.{ FutureTimeoutException, Promise, MessageDispatcher }
import akka.dispatch._
/**
* @author Martin Krasser
@ -274,9 +274,7 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
def resume(): Unit = ()
def stop() {
running = false
}
def stop() { running = false }
/**
* 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 sender ignored
*/
protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel) = if(running) {
protected[akka] def postMessageToMailbox(message: Any, sender: ActorRef) = if(running) {
message match {
case Ack { /* no response message to set */ }
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 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
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;
String body = msg.getBodyAs(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;
String body = msg.getBodyAs(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 = {
case SetExpectedMessageCount(num) {
latch = new CountDownLatch(num)
channel ! latch
sender ! latch
}
case msg latch.countDown
}
@ -44,7 +44,7 @@ object CamelTestSupport {
trait Respond { this: Actor
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
@ -54,8 +54,8 @@ object CamelTestSupport {
val messages = Buffer[Any]()
def retain: Handler = {
case GetRetainedMessage channel ! messages.last
case GetRetainedMessages(p) channel ! messages.filter(p).toList
case GetRetainedMessage sender ! messages.last
case GetRetainedMessages(p) sender ! messages.filter(p).toList
case msg {
messages += msg
msg

View file

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

View file

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

View file

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

View file

@ -300,7 +300,7 @@ class DefaultClusterNode private[akka] (
val remote = new akka.cluster.netty.NettyRemoteSupport
remote.start(hostname, port)
remote.register(RemoteClusterDaemon.Address, remoteDaemon)
remote.addListener(RemoteFailureDetector.channel)
remote.addListener(RemoteFailureDetector.sender)
remote.addListener(remoteClientLifeCycleHandler)
remote
}
@ -427,7 +427,7 @@ class DefaultClusterNode private[akka] (
remoteService.shutdown() // shutdown server
RemoteFailureDetector.channel.stop()
RemoteFailureDetector.sender.stop()
remoteClientLifeCycleHandler.stop()
remoteDaemon.stop()
@ -1275,7 +1275,7 @@ class DefaultClusterNode private[akka] (
* Update the list of connections to other nodes in the cluster.
* Tail recursive, using lockless optimimistic concurrency.
*
* @returns a Map with the remote socket addresses to of disconnected node connections
* @return a Map with the remote socket addresses to of disconnected node connections
*/
@tailrec
final private[cluster] def connectToAllNewlyArrivedMembershipNodesInCluster(

View file

@ -11,12 +11,11 @@ Intro
Akka Cluster provides a fault-tolerant, elastic, decentralized peer-to-peer
cluster with no single point of failure (SPOF) or single point of bottleneck
(SPOB). It implemented as a Dynamo-style system using gossip protocols,
automatic failure detection, automatic partitioning, handoff, and cluster
rebalancing. But with some differences due to the fact that it is not just
managing passive data, but actors, e.g. active, sometimes stateful, components
that have requirements on message ordering, the number of active instances in
the cluster, etc.
(SPOB). It implements a Dynamo-style system using gossip protocols, automatic
failure detection, automatic partitioning, handoff, and cluster rebalancing. But
with some differences due to the fact that it is not just managing passive data,
but actors - active, sometimes stateful, components that also have requirements
on message ordering, the number of active instances in the cluster, etc.
Terms
@ -32,8 +31,12 @@ These terms are used throughout the documentation.
A set of nodes. Contains distributed Akka applications.
**partition**
An actor (possibly a subtree of actors) in the Akka application that
is distributed within the cluster.
An actor or subtree of actors in the Akka application that is distributed
within the cluster.
**partition point**
The actor at the head of a partition. The point around which a partition is
formed.
**partition path**
Also referred to as the actor address. Has the format `actor1/actor2/actor3`
@ -46,8 +49,8 @@ These terms are used throughout the documentation.
``N-value`` of the partition.
**partition table**
A mapping from partition path to base node and its ``N-value`` (i.e. its
instance count).
A mapping from partition path to base node and its ``N-value`` (instance
count).
Membership
@ -64,10 +67,11 @@ Gossip
------
The cluster membership used in Akka is based on Amazon's `Dynamo`_ system and
particularly the approach taken Basho's' `Riak`_ distributed database. Cluster
membership is communicated using a `Gossip Protocol`_, where the current state
of the cluster is gossiped randomly through the cluster. Joining a cluster is
initiated by specifying a set of ``seed`` nodes with which to begin gossiping.
particularly the approach taken in Basho's' `Riak`_ distributed database.
Cluster membership is communicated using a `Gossip Protocol`_, where the current
state of the cluster is gossiped randomly through the cluster. Joining a cluster
is initiated by specifying a set of ``seed`` nodes with which to begin
gossiping.
.. _Gossip Protocol: http://en.wikipedia.org/wiki/Gossip_protocol
.. _Dynamo: http://www.allthingsdistributed.com/files/amazon-dynamo-sosp2007.pdf
@ -92,6 +96,7 @@ the `pruning algorithm`_ in Riak.
.. _Vector Clocks: http://en.wikipedia.org/wiki/Vector_clock
.. _pruning algorithm: http://wiki.basho.com/Vector-Clocks.html#Vector-Clock-Pruning
Gossip convergence
^^^^^^^^^^^^^^^^^^
@ -113,7 +118,7 @@ unreachable from the rest of the cluster. For this we are using an
implementation of `The Phi Accrual Failure Detector`_ by Hayashibara et al.
An accrual failure detector decouple monitoring and interpretation. That makes
them applicable to a wider area ofQ scenarios and more adequate to build generic
them applicable to a wider area of scenarios and more adequate to build generic
failure detection services. The idea is that it is keeping a history of failure
statistics, calculated from heartbeats received from the gossip protocol, and is
trying to do educated guesses by taking multiple factors, and how they
@ -132,6 +137,7 @@ order to account for network issues that sometimes occur on such platforms.
.. _The Phi Accrual Failure Detector: http://ddg.jaist.ac.jp/pub/HDY+04.pdf
Leader
^^^^^^
@ -345,28 +351,50 @@ Partitioning
============
Each partition (an actor or actor subtree) in the actor system is assigned to a
base node. The mapping from partition path (actor address on the format "a/b/c")
to base node is stored in the partition table and is maintained as part of the
cluster state through the gossip protocol. The partition table is only updated
by the leader node. If the partition has a configured instance count, referred
to as the ``N-value``, greater than one, then the location of the other
instances can be found deterministically by counting from the base node. (The
``N-value`` is larger than 1 when a actor is configured to be routed.) The first
instance will be found on the base node, and the other instances on the next N-1
nodes, given the nodes in sorted order.
base node. The actor at the head of the partition is referred to as the
partition point. The mapping from partition path (actor address of the format
"a/b/c") to base node is stored in the partition table and is maintained as part
of the cluster state through the gossip protocol. The partition table is only
updated by the leader node. Currently the only possible partition points are
*routed* actors.
TODO: discuss how different N values within the tree work (especially subtrees
with a greater or lesser N value). A simple implementation would only allow the
highest-up-the-tree, non-singular (greater than one) value to be used for any
subtree.
Routed actors can have an instance count greater than one. The instance count is
also referred to as the ``N-value``. If the ``N-value`` is greater than one then
the first instance will be found on the base node, and the other instances on
the next N-1 nodes, given the nodes in sorted order.
Note that in the first implementation there may be a restriction such that only
top-level partitions are possible (the highest possible partition points are
used and sub-partitioning is not allowed). Still to be explored in more detail.
The cluster leader determines the current instance count for a partition based
on two axes: fault-tolerance and scaling.
Fault-tolerance determines a minimum number of instances for a routed actor
(allowing N-1 nodes to crash while still maintaining at least one running actor
instance). The user can specify a function from current number of nodes to the
number of acceptable node failures: n: Int => f: Int where f < n.
Scaling reflects the number of instances needed to maintain good throughput and
is influenced by metrics from the system, particularly a history of mailbox
size, CPU load, and GC percentages. It may also be possible to accept scaling
hints from the user that indicate expected load.
The balancing of partitions is determined in a simple way (at least for the
first implementation) where the overlap of partitions is minimized. Partitions
are spread over the cluster ring in a circular fashion, with each base node
in the first available space.
For example, given a cluster with ten nodes and three partitions having N-values
of 4, 3, and 5; partition 1 would have base node 1 and instances on nodes
1-4; partition 2 would have base node 5 and instances on nodes 5-7; partition 3
would have base node 8 and instances on nodes 8-10 and 1-2. The only overlap is
on nodes 1 and 2.
When rebalancing is required the leader will schedule handoffs, gossiping a set
of pending changes, and when each change is complete the leader will update the
partition table.
TODO: look further into how actors will be distributed and also avoiding
unnecessary migrations just to create a more balanced cluster.
Handoff
-------
@ -433,7 +461,7 @@ Update transition
The second transition begins when the migration is marked as complete and ends
when all nodes have the updated partition table (when all nodes will use ``N2``
as the host for ``A``), e.g. we have convergence, and is referred to as the
as the host for ``A``, i.e. we have convergence) and is referred to as the
*update transition*.
Once the update transition begins ``N1`` can forward any messages it receives
@ -530,5 +558,44 @@ have a dependency on message ordering from any given source.
and state is transfered during the migration initialization, then options 2b
and 3b would be required.
Support for stateful singleton actors will come in future releases of Akka, most
likely Akka 2.2.
Stateful Actor Replication
==========================
Support for stateful singleton actors will come in future releases of Akka, and
is scheduled for Akka 2.2. Having a Dynamo base for the clustering already we
should use the same infrastructure to provide stateful actor clustering and
datastore as well. The stateful actor clustering should be layered on top of the
distributed datastore. See the next section for a rough outline on how the
distributed datastore could be implemented.
Implementing a Dynamo-style distributed database on top of Akka Cluster
-----------------------------------------------------------------------
The missing pieces to implement a full Dynamo-style eventually consistent data
storage on top of the Akka Cluster as described in this document are:
- Configuration of ``READ`` and ``WRITE`` consistency levels according to the ``N/R/W`` numbers
defined in the Dynamo paper.
- R = read replica count
- W = write replica count
- N = replication factor
- Q = QUORUM = N / 2 + 1
- W + R > N = full consistency
- Define a versioned data message wrapper: ``Versioned[T](hash: Long, version: VectorClock, data: T)``.
- Define a single system data broker actor on each node that uses a ``Consistent
Hashing Router`` and that have instances on all other nodes in the node ring.
- For ``WRITE``:
1. Wrap data in a ``Versioned Message``
2. Send a ``Versioned Message`` with the data is sent to a number of nodes matching the ``W-value``.
- For ``READ``:
1. Read in the ``Versioned Message`` with the data from as many replicas as you need for the consistency level required by the ``R-value``.
2. Do comparison on the versions (using `Vector Clocks`_)
3. If the versions differ then do `Read Repair`_ to update the inconsistent nodes.
4. Return the latest versioned data.
.. _Read Repair: http://wiki.apache.org/cassandra/ReadRepair

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