clean up initialization of ActorSystem, fixes #1050

- create ActorSystemImpl trait to make ActorSystem fully abstract
- add Java API for constructing (ActorSystem.create(...))
- only go through factory methods because .start() has become necessary
- rename all user-facing occurrences of “app” to “system” (Actor trait
  and TestKit/AkkaSpec)
- pass ActorSystemImpl to ActorRefs upon creation, which means that
  actorOf() and friends need such an argument, which must be provided to
  the ActorRefProvider by the ActorRefFactory implementation
This commit is contained in:
Roland 2011-11-16 17:18:36 +01:00
parent 6d85572ecc
commit 648661c548
83 changed files with 494 additions and 390 deletions

View file

@ -9,7 +9,7 @@ import static org.junit.Assert.*;
public class JavaAPI { public class JavaAPI {
private ActorSystem app = new ActorSystem(); private ActorSystem app = ActorSystem.create();
@Test void mustBeAbleToCreateActorRefFromClass() { @Test void mustBeAbleToCreateActorRefFromClass() {
ActorRef ref = app.actorOf(JavaAPITestActor.class); ActorRef ref = app.actorOf(JavaAPITestActor.class);

View file

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

View file

@ -247,7 +247,7 @@ class ActorRefSpec extends AkkaSpec {
out.flush out.flush
out.close out.close
Serialization.app.withValue(app) { Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
val readA = in.readObject val readA = in.readObject
@ -284,14 +284,14 @@ class ActorRefSpec extends AkkaSpec {
val baos = new ByteArrayOutputStream(8192 * 32) val baos = new ByteArrayOutputStream(8192 * 32)
val out = new ObjectOutputStream(baos) val out = new ObjectOutputStream(baos)
val serialized = SerializedActorRef(app.address.hostname, app.address.port, "/this/path/does/not/exist") val serialized = SerializedActorRef(system.root.remoteAddress.hostname, system.root.remoteAddress.port, "/this/path/does/not/exist")
out.writeObject(serialized) out.writeObject(serialized)
out.flush out.flush
out.close out.close
Serialization.app.withValue(app) { Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
(intercept[java.lang.IllegalStateException] { (intercept[java.lang.IllegalStateException] {
in.readObject in.readObject

View file

@ -17,8 +17,8 @@ class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll {
} }
}, timeout = t)) }, timeout = t))
val defaultTimeout = app.AkkaConfig.ActorTimeout.duration val defaultTimeout = system.AkkaConfig.ActorTimeout.duration
val testTimeout = if (app.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis val testTimeout = if (system.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis
"An Actor-based Future" must { "An Actor-based Future" must {

View file

@ -9,7 +9,7 @@ class ClusterSpec extends AkkaSpec {
"be able to parse 'akka.actor.cluster._' config elements" in { "be able to parse 'akka.actor.cluster._' config elements" in {
// TODO: make it use its own special config? // TODO: make it use its own special config?
val config = app.config val config = system.AkkaConfig.config
import config._ import config._
//akka.cluster //akka.cluster

View file

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

View file

@ -167,12 +167,12 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
} }
}) })
filterException[Logging.EventHandlerException] { filterException[Logging.EventHandlerException] {
app.eventStream.subscribe(testActor, classOf[Logging.Error]) system.eventStream.subscribe(testActor, classOf[Logging.Error])
fsm ! "go" fsm ! "go"
expectMsgPF(1 second, hint = "Next state 2 does not exist") { expectMsgPF(1 second, hint = "Next state 2 does not exist") {
case Logging.Error(_, `fsm`, "Next state 2 does not exist") true case Logging.Error(_, `fsm`, "Next state 2 does not exist") true
} }
app.eventStream.unsubscribe(testActor) system.eventStream.unsubscribe(testActor)
} }
} }
@ -213,20 +213,20 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true
case StopEvent(r, _, _) testActor ! r case StopEvent(r, _, _) testActor ! r
} }
}) })
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
fsm ! "go" fsm ! "go"
expectMsgPF(1 second, hint = "processing Event(go,null)") { expectMsgPF(1 second, hint = "processing Event(go,null)") {
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[" + app.address + "/sys/testActor") true case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(go,null) from Actor[") true
} }
expectMsg(1 second, Logging.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown")) expectMsg(1 second, Logging.Debug(fsm, "setting timer 't'/1500 milliseconds: Shutdown"))
expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2")) expectMsg(1 second, Logging.Debug(fsm, "transition 1 -> 2"))
fsm ! "stop" fsm ! "stop"
expectMsgPF(1 second, hint = "processing Event(stop,null)") { expectMsgPF(1 second, hint = "processing Event(stop,null)") {
case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[" + app.address + "/sys/testActor") true case Logging.Debug(`fsm`, s: String) if s.startsWith("processing Event(stop,null) from Actor[") true
} }
expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal) expectMsgAllOf(1 second, Logging.Debug(fsm, "canceling timer 't'"), Normal)
expectNoMsg(1 second) expectNoMsg(1 second)
app.eventStream.unsubscribe(testActor) system.eventStream.unsubscribe(testActor)
} }
} }
} }

View file

@ -37,14 +37,14 @@ class ForwardActorSpec extends AkkaSpec {
val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage latch.countDown() } }) val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage latch.countDown() } })
val chain = createForwardingChain(app) val chain = createForwardingChain(system)
chain.tell(ExpectedMessage, replyTo) chain.tell(ExpectedMessage, replyTo)
latch.await(Duration(5, "s")) must be === true latch.await(Duration(5, "s")) must be === true
} }
"forward actor reference when invoking forward on bang bang" in { "forward actor reference when invoking forward on bang bang" in {
val chain = createForwardingChain(app) val chain = createForwardingChain(system)
chain.ask(ExpectedMessage, 5000).get must be === ExpectedMessage chain.ask(ExpectedMessage, 5000).get must be === ExpectedMessage
} }
} }

View file

@ -13,14 +13,15 @@ class LocalActorRefProviderSpec extends AkkaSpec {
"An LocalActorRefProvider" must { "An LocalActorRefProvider" must {
"only create one instance of an actor with a specific address in a concurrent environment" in { "only create one instance of an actor with a specific address in a concurrent environment" in {
val provider = app.provider val impl = system.asInstanceOf[ActorSystemImpl]
val provider = impl.provider
provider.isInstanceOf[LocalActorRefProvider] must be(true) provider.isInstanceOf[LocalActorRefProvider] must be(true)
(0 until 100) foreach { i // 100 concurrent runs (0 until 100) foreach { i // 100 concurrent runs
val address = "new-actor" + i val address = "new-actor" + i
implicit val timeout = Timeout(5 seconds) 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) ((1 to 4) map { _ Future { provider.actorOf(impl, Props(c { case _ }), impl.guardian, address) } }).map(_.get).distinct.size must be(1)
} }
} }
} }

View file

@ -53,7 +53,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"decorate a Receive" in { "decorate a Receive" in {
new TestKit(appLogging) { new TestKit(appLogging) {
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
val r: Actor.Receive = { val r: Actor.Receive = {
case null case null
} }
@ -66,8 +66,8 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"be added on Actor if requested" in { "be added on Actor if requested" in {
new TestKit(appLogging) with ImplicitSender { new TestKit(appLogging) with ImplicitSender {
ignoreMute(this) ignoreMute(this)
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
app.eventStream.subscribe(testActor, classOf[Logging.Error]) system.eventStream.subscribe(testActor, classOf[Logging.Error])
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = loggable(this) { def receive = loggable(this) {
case _ sender ! "x" case _ sender ! "x"
@ -95,7 +95,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"not duplicate logging" in { "not duplicate logging" in {
new TestKit(appLogging) with ImplicitSender { new TestKit(appLogging) with ImplicitSender {
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = loggable(this)(loggable(this) { def receive = loggable(this)(loggable(this) {
case _ sender ! "x" case _ sender ! "x"
@ -115,7 +115,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
"log AutoReceiveMessages if requested" in { "log AutoReceiveMessages if requested" in {
new TestKit(appAuto) { new TestKit(appAuto) {
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
val actor = TestActorRef(new Actor { val actor = TestActorRef(new Actor {
def receive = { def receive = {
case _ case _
@ -137,10 +137,10 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
val s = ref.toString val s = ref.toString
s.contains("MainBusReaper") || s.contains("Supervisor") s.contains("MainBusReaper") || s.contains("Supervisor")
} }
app.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
app.eventStream.subscribe(testActor, classOf[Logging.Error]) system.eventStream.subscribe(testActor, classOf[Logging.Error])
within(3 seconds) { within(3 seconds) {
val lifecycleGuardian = appLifecycle.guardian val lifecycleGuardian = appLifecycle.asInstanceOf[ActorSystemImpl].guardian
val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000))) val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000)))
val supervisorSet = receiveWhile(messages = 2) { val supervisorSet = receiveWhile(messages = 2) {

View file

@ -16,7 +16,7 @@ import akka.testkit.AkkaSpec
class RestartStrategySpec extends AkkaSpec { class RestartStrategySpec extends AkkaSpec {
override def atStartup { override def atStartup {
app.eventStream.publish(Mute(EventFilter[Exception]("Crashing..."))) system.eventStream.publish(Mute(EventFilter[Exception]("Crashing...")))
} }
object Ping object Ping

View file

@ -28,14 +28,14 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
def receive = { case Tick countDownLatch.countDown() } def receive = { case Tick countDownLatch.countDown() }
}) })
// run every 50 millisec // run every 50 millisec
collectCancellable(app.scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS)) collectCancellable(system.scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS))
// after max 1 second it should be executed at least the 3 times already // after max 1 second it should be executed at least the 3 times already
assert(countDownLatch.await(1, TimeUnit.SECONDS)) assert(countDownLatch.await(1, TimeUnit.SECONDS))
val countDownLatch2 = new CountDownLatch(3) val countDownLatch2 = new CountDownLatch(3)
collectCancellable(app.scheduler.schedule(() countDownLatch2.countDown(), 0, 50, TimeUnit.MILLISECONDS)) collectCancellable(system.scheduler.schedule(() countDownLatch2.countDown(), 0, 50, TimeUnit.MILLISECONDS))
// after max 1 second it should be executed at least the 3 times already // after max 1 second it should be executed at least the 3 times already
assert(countDownLatch2.await(2, TimeUnit.SECONDS)) assert(countDownLatch2.await(2, TimeUnit.SECONDS))
@ -49,8 +49,8 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
}) })
// run every 50 millisec // run every 50 millisec
collectCancellable(app.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS)) collectCancellable(system.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS))
collectCancellable(app.scheduler.scheduleOnce(() countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS)) collectCancellable(system.scheduler.scheduleOnce(() countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS))
// after 1 second the wait should fail // after 1 second the wait should fail
assert(countDownLatch.await(2, TimeUnit.SECONDS) == false) assert(countDownLatch.await(2, TimeUnit.SECONDS) == false)
@ -86,7 +86,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
}) })
(1 to 10).foreach { i (1 to 10).foreach { i
val timeout = collectCancellable(app.scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.SECONDS)) val timeout = collectCancellable(system.scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.SECONDS))
timeout.cancel() timeout.cancel()
} }
@ -114,10 +114,10 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach {
}) })
val actor = (supervisor ? props).as[ActorRef].get val actor = (supervisor ? props).as[ActorRef].get
collectCancellable(app.scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS)) collectCancellable(system.scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS))
// appx 2 pings before crash // appx 2 pings before crash
EventFilter[Exception]("CRASH", occurrences = 1) intercept { EventFilter[Exception]("CRASH", occurrences = 1) intercept {
collectCancellable(app.scheduler.scheduleOnce(actor, Crash, 1000, TimeUnit.MILLISECONDS)) collectCancellable(system.scheduler.scheduleOnce(actor, Crash, 1000, TimeUnit.MILLISECONDS))
} }
assert(restartLatch.tryAwait(2, TimeUnit.SECONDS)) assert(restartLatch.tryAwait(2, TimeUnit.SECONDS))

View file

@ -27,13 +27,13 @@ class SupervisorMiscSpec extends AkkaSpec {
} }
}) })
val actor1 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get val actor1 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
val actor2 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get val actor2 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
val actor3 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newDispatcher("test").build)).as[ActorRef].get val actor3 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newDispatcher("test").build)).as[ActorRef].get
val actor4 = (supervisor ? workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get val actor4 = (supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).as[ActorRef].get
actor1 ! Kill actor1 ! Kill
actor2 ! Kill actor2 ! Kill

View file

@ -121,7 +121,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
} }
override def atStartup() { override def atStartup() {
app.eventStream.publish(Mute(EventFilter[RuntimeException](ExceptionMessage))) system.eventStream.publish(Mute(EventFilter[RuntimeException](ExceptionMessage)))
} }
override def beforeEach() = { override def beforeEach() = {

View file

@ -147,18 +147,18 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
newFooBar(Props().withTimeout(Timeout(d))) newFooBar(Props().withTimeout(Timeout(d)))
def newFooBar(props: Props): Foo = def newFooBar(props: Props): Foo =
app.typedActorOf(classOf[Foo], classOf[Bar], props) system.typedActorOf(classOf[Foo], classOf[Bar], props)
def newStacked(props: Props = Props().withTimeout(Timeout(2000))): Stacked = def newStacked(props: Props = Props().withTimeout(Timeout(2000))): Stacked =
app.typedActorOf(classOf[Stacked], classOf[StackedImpl], props) system.typedActorOf(classOf[Stacked], classOf[StackedImpl], props)
def mustStop(typedActor: AnyRef) = app.typedActor.stop(typedActor) must be(true) def mustStop(typedActor: AnyRef) = system.typedActor.stop(typedActor) must be(true)
"TypedActors" must { "TypedActors" must {
"be able to instantiate" in { "be able to instantiate" in {
val t = newFooBar val t = newFooBar
app.typedActor.isTypedActor(t) must be(true) system.typedActor.isTypedActor(t) must be(true)
mustStop(t) mustStop(t)
} }
@ -168,7 +168,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
} }
"not stop non-started ones" in { "not stop non-started ones" in {
app.typedActor.stop(null) must be(false) system.typedActor.stop(null) must be(false)
} }
"throw an IllegalStateExcpetion when TypedActor.self is called in the wrong scope" in { "throw an IllegalStateExcpetion when TypedActor.self is called in the wrong scope" in {
@ -187,7 +187,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to call toString" in { "be able to call toString" in {
val t = newFooBar val t = newFooBar
t.toString must be(app.typedActor.getActorRefFor(t).toString) t.toString must be(system.typedActor.getActorRefFor(t).toString)
mustStop(t) mustStop(t)
} }
@ -200,7 +200,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to call hashCode" in { "be able to call hashCode" in {
val t = newFooBar val t = newFooBar
t.hashCode must be(app.typedActor.getActorRefFor(t).hashCode) t.hashCode must be(system.typedActor.getActorRefFor(t).hashCode)
mustStop(t) mustStop(t)
} }
@ -295,7 +295,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
} }
"be able to support implementation only typed actors" in { "be able to support implementation only typed actors" in {
val t = app.typedActorOf[Foo, Bar](Props()) val t = system.typedActorOf[Foo, Bar](Props())
val f = t.futurePigdog(200) val f = t.futurePigdog(200)
val f2 = t.futurePigdog(0) val f2 = t.futurePigdog(0)
f2.isCompleted must be(false) f2.isCompleted must be(false)
@ -305,7 +305,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
} }
"be able to support implementation only typed actors with complex interfaces" in { "be able to support implementation only typed actors with complex interfaces" in {
val t = app.typedActorOf[Stackable1 with Stackable2, StackedImpl]() val t = system.typedActorOf[Stackable1 with Stackable2, StackedImpl]()
t.stackable1 must be("foo") t.stackable1 must be("foo")
t.stackable2 must be("bar") t.stackable2 must be("bar")
mustStop(t) mustStop(t)
@ -314,7 +314,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to use work-stealing dispatcher" in { "be able to use work-stealing dispatcher" in {
val props = Props( val props = Props(
timeout = Timeout(6600), timeout = Timeout(6600),
dispatcher = app.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher") dispatcher = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(60) .setCorePoolSize(60)
.setMaxPoolSize(60) .setMaxPoolSize(60)
@ -332,7 +332,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to serialize and deserialize invocations" in { "be able to serialize and deserialize invocations" in {
import java.io._ import java.io._
val m = TypedActor.MethodCall(app.serialization, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]()) val m = TypedActor.MethodCall(system.serialization, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]())
val baos = new ByteArrayOutputStream(8192 * 4) val baos = new ByteArrayOutputStream(8192 * 4)
val out = new ObjectOutputStream(baos) val out = new ObjectOutputStream(baos)
@ -341,7 +341,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
Serialization.app.withValue(app) { Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall] val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
mNew.method must be(m.method) mNew.method must be(m.method)
@ -351,7 +351,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
"be able to serialize and deserialize invocations' parameters" in { "be able to serialize and deserialize invocations' parameters" in {
import java.io._ import java.io._
val someFoo: Foo = new Bar val someFoo: Foo = new Bar
val m = TypedActor.MethodCall(app.serialization, classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef])) val m = TypedActor.MethodCall(system.serialization, classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef]))
val baos = new ByteArrayOutputStream(8192 * 4) val baos = new ByteArrayOutputStream(8192 * 4)
val out = new ObjectOutputStream(baos) val out = new ObjectOutputStream(baos)
@ -360,7 +360,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
Serialization.app.withValue(app) { Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) {
val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall] val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall]
mNew.method must be(m.method) mNew.method must be(m.method)

View file

@ -148,14 +148,14 @@ object ActorModelSpec {
def assertDispatcher(dispatcher: MessageDispatcherInterceptor)( def assertDispatcher(dispatcher: MessageDispatcherInterceptor)(
starts: Long = dispatcher.starts.get(), starts: Long = dispatcher.starts.get(),
stops: Long = dispatcher.stops.get())(implicit app: ActorSystem) { stops: Long = dispatcher.stops.get())(implicit system: ActorSystem) {
val deadline = System.currentTimeMillis + dispatcher.timeoutMs * 5 val deadline = System.currentTimeMillis + dispatcher.timeoutMs * 5
try { try {
await(deadline)(starts == dispatcher.starts.get) await(deadline)(starts == dispatcher.starts.get)
await(deadline)(stops == dispatcher.stops.get) await(deadline)(stops == dispatcher.stops.get)
} catch { } catch {
case e case e
app.eventStream.publish(Error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get + system.eventStream.publish(Error(e, dispatcher, "actual: starts=" + dispatcher.starts.get + ",stops=" + dispatcher.stops.get +
" required: starts=" + starts + ",stops=" + stops)) " required: starts=" + starts + ",stops=" + stops))
throw e throw e
} }
@ -181,7 +181,7 @@ object ActorModelSpec {
unregisters: Long = 0, unregisters: Long = 0,
msgsReceived: Long = 0, msgsReceived: Long = 0,
msgsProcessed: Long = 0, msgsProcessed: Long = 0,
restarts: Long = 0)(implicit app: ActorSystem) { restarts: Long = 0)(implicit system: ActorSystem) {
assertRef(actorRef, dispatcher)( assertRef(actorRef, dispatcher)(
suspensions, suspensions,
resumes, resumes,
@ -199,7 +199,7 @@ object ActorModelSpec {
unregisters: Long = statsFor(actorRef).unregisters.get(), unregisters: Long = statsFor(actorRef).unregisters.get(),
msgsReceived: Long = statsFor(actorRef).msgsReceived.get(), msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(), msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
restarts: Long = statsFor(actorRef).restarts.get())(implicit app: ActorSystem) { restarts: Long = statsFor(actorRef).restarts.get())(implicit system: ActorSystem) {
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher)) val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
val deadline = System.currentTimeMillis + 1000 val deadline = System.currentTimeMillis + 1000
try { try {
@ -212,7 +212,7 @@ object ActorModelSpec {
await(deadline)(stats.restarts.get() == restarts) await(deadline)(stats.restarts.get() == restarts)
} catch { } catch {
case e case e
app.eventStream.publish(Error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions + system.eventStream.publish(Error(e, dispatcher, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters + ",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters +
",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts)) ",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts))
throw e throw e
@ -235,7 +235,7 @@ abstract class ActorModelSpec extends AkkaSpec {
import ActorModelSpec._ import ActorModelSpec._
def newTestActor(dispatcher: MessageDispatcher) = app.actorOf(Props[DispatcherActor].withDispatcher(dispatcher)) def newTestActor(dispatcher: MessageDispatcher) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
protected def newInterceptedDispatcher: MessageDispatcherInterceptor protected def newInterceptedDispatcher: MessageDispatcherInterceptor
protected def dispatcherType: String protected def dispatcherType: String
@ -318,7 +318,7 @@ abstract class ActorModelSpec extends AkkaSpec {
try { try {
f f
} catch { } catch {
case e app.eventStream.publish(Error(e, this, "error in spawned thread")) case e system.eventStream.publish(Error(e, this, "error in spawned thread"))
} }
} }
} }
@ -421,10 +421,10 @@ class DispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._ import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", app.AkkaConfig.DispatcherThroughput, new Dispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", system.AkkaConfig.DispatcherThroughput,
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType, system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor] ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Dispatcher" def dispatcherType = "Dispatcher"
@ -458,14 +458,14 @@ class BalancingDispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._ import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", 1, // TODO check why 1 here? (came from old test) new BalancingDispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", 1, // TODO check why 1 here? (came from old test)
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType, system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor] ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Balancing Dispatcher" def dispatcherType = "Balancing Dispatcher"
override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = app.dispatcher override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = system.dispatcher
"A " + dispatcherType must { "A " + dispatcherType must {
"process messages in parallel" in { "process messages in parallel" in {

View file

@ -8,7 +8,7 @@ import akka.testkit.AkkaSpec
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class BalancingDispatcherSpec extends AkkaSpec { class BalancingDispatcherSpec extends AkkaSpec {
def newWorkStealer() = app.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher", 1).build def newWorkStealer() = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher", 1).build
val delayableActorDispatcher, sharedActorDispatcher, parentActorDispatcher = newWorkStealer() val delayableActorDispatcher, sharedActorDispatcher, parentActorDispatcher = newWorkStealer()

View file

@ -33,22 +33,22 @@ class DispatcherActorSpec extends AkkaSpec {
"A Dispatcher and an Actor" must { "A Dispatcher and an Actor" must {
"support tell" in { "support tell" in {
val actor = actorOf(Props[OneWayTestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) val actor = actorOf(Props[OneWayTestActor].withDispatcher(system.dispatcherFactory.newDispatcher("test").build))
val result = actor ! "OneWay" val result = actor ! "OneWay"
assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS)) assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS))
actor.stop() actor.stop()
} }
"support ask/reply" in { "support ask/reply" in {
val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) val actor = actorOf(Props[TestActor].withDispatcher(system.dispatcherFactory.newDispatcher("test").build))
val result = (actor ? "Hello").as[String] val result = (actor ? "Hello").as[String]
assert("World" === result.get) assert("World" === result.get)
actor.stop() actor.stop()
} }
"respect the throughput setting" in { "respect the throughput setting" in {
val throughputDispatcher = app.dispatcherFactory. val throughputDispatcher = system.dispatcherFactory.
newDispatcher("THROUGHPUT", 101, 0, app.dispatcherFactory.MailboxType). newDispatcher("THROUGHPUT", 101, 0, system.dispatcherFactory.MailboxType).
setCorePoolSize(1). setCorePoolSize(1).
build build
@ -76,8 +76,8 @@ class DispatcherActorSpec extends AkkaSpec {
"respect throughput deadline" in { "respect throughput deadline" in {
val deadlineMs = 100 val deadlineMs = 100
val throughputDispatcher = app.dispatcherFactory. val throughputDispatcher = system.dispatcherFactory.
newDispatcher("THROUGHPUT", 2, deadlineMs, app.dispatcherFactory.MailboxType). newDispatcher("THROUGHPUT", 2, deadlineMs, system.dispatcherFactory.MailboxType).
setCorePoolSize(1). setCorePoolSize(1).
build build
val works = new AtomicBoolean(true) val works = new AtomicBoolean(true)

View file

@ -12,7 +12,8 @@ import akka.config.Configuration
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class DispatchersSpec extends AkkaSpec { class DispatchersSpec extends AkkaSpec {
import app.dispatcherFactory._ val df = system.dispatcherFactory
import df._
val tipe = "type" val tipe = "type"
val keepalivems = "keep-alive-time" val keepalivems = "keep-alive-time"

View file

@ -27,14 +27,14 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach {
"support tell" in { "support tell" in {
var oneWay = new CountDownLatch(1) var oneWay = new CountDownLatch(1)
val actor = actorOf(Props(self { case "OneWay" oneWay.countDown() }).withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) val actor = actorOf(Props(self { case "OneWay" oneWay.countDown() }).withDispatcher(system.dispatcherFactory.newPinnedDispatcher("test")))
val result = actor ! "OneWay" val result = actor ! "OneWay"
assert(oneWay.await(1, TimeUnit.SECONDS)) assert(oneWay.await(1, TimeUnit.SECONDS))
actor.stop() actor.stop()
} }
"support ask/reply" in { "support ask/reply" in {
val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) val actor = actorOf(Props[TestActor].withDispatcher(system.dispatcherFactory.newPinnedDispatcher("test")))
val result = (actor ? "Hello").as[String] val result = (actor ? "Hello").as[String]
assert("World" === result.get) assert("World" === result.get)
actor.stop() actor.stop()

View file

@ -13,7 +13,7 @@ class ConfigSpec extends AkkaSpec(ActorSystem("ConfigSpec", Configuration.fromFi
"The default configuration file (i.e. akka-reference.conf)" must { "The default configuration file (i.e. akka-reference.conf)" must {
"contain all configuration properties for akka-actor that are used in code with their correct defaults" in { "contain all configuration properties for akka-actor that are used in code with their correct defaults" in {
val config = app.config val config = system.AkkaConfig.config
import config._ import config._
getList("akka.boot") must equal(Nil) getList("akka.boot") must equal(Nil)

View file

@ -19,7 +19,7 @@ class Future2ActorSpec extends AkkaSpec {
} }
"support reply via sender" in { "support reply via sender" in {
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "do" Future(31) pipeTo context.sender case "do" Future(31) pipeTo context.sender
case "ex" Future(throw new AssertionError) pipeTo context.sender case "ex" Future(throw new AssertionError) pipeTo context.sender

View file

@ -773,7 +773,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll {
"ticket812FutureDispatchCleanup" in { "ticket812FutureDispatchCleanup" in {
filterException[FutureTimeoutException] { filterException[FutureTimeoutException] {
implicit val dispatcher = app.dispatcherFactory.newDispatcher("ticket812FutureDispatchCleanup").build implicit val dispatcher = system.dispatcherFactory.newDispatcher("ticket812FutureDispatchCleanup").build
assert(dispatcher.tasks === 0) assert(dispatcher.tasks === 0)
val future = Future({ Thread.sleep(100); "Done" }, 10) val future = Future({ Thread.sleep(100); "Done" }, 10)
intercept[FutureTimeoutException] { future.await } intercept[FutureTimeoutException] { future.await }

View file

@ -80,7 +80,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
result result
} }
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, app.deadLetters) def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters)
def ensureInitialMailboxState(config: MailboxType, q: Mailbox) { def ensureInitialMailboxState(config: MailboxType, q: Mailbox) {
q must not be null q must not be null

View file

@ -18,12 +18,12 @@ class PriorityDispatcherSpec extends AkkaSpec {
testOrdering(BoundedPriorityMailbox(PriorityGenerator({ testOrdering(BoundedPriorityMailbox(PriorityGenerator({
case i: Int i //Reverse order case i: Int i //Reverse order
case 'Result Int.MaxValue case 'Result Int.MaxValue
}: Any Int), 1000, app.AkkaConfig.MailboxPushTimeout)) }: Any Int), 1000, system.AkkaConfig.MailboxPushTimeout))
} }
} }
def testOrdering(mboxType: MailboxType) { def testOrdering(mboxType: MailboxType) {
val dispatcher = app.dispatcherFactory.newDispatcher("Test", 1, -1, mboxType).build val dispatcher = system.dispatcherFactory.newDispatcher("Test", 1, -1, mboxType).build
val actor = actorOf(Props(new Actor { val actor = actorOf(Props(new Actor {
var acc: List[Int] = Nil var acc: List[Int] = Nil

View file

@ -6,7 +6,7 @@ package akka.event
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.config.Configuration import akka.config.Configuration
import akka.util.duration._ import akka.util.duration._
import akka.actor.{ Actor, ActorRef } import akka.actor.{ Actor, ActorRef, ActorSystemImpl }
object EventStreamSpec { object EventStreamSpec {
case class M(i: Int) case class M(i: Int)
@ -14,7 +14,7 @@ object EventStreamSpec {
case class SetTarget(ref: ActorRef) case class SetTarget(ref: ActorRef)
class MyLog extends Actor { class MyLog extends Actor {
var dst: ActorRef = app.deadLetters var dst: ActorRef = system.deadLetters
def receive = { def receive = {
case Logging.InitializeLogger(bus) bus.subscribe(context.self, classOf[SetTarget]); sender ! Logging.LoggerInitialized case Logging.InitializeLogger(bus) bus.subscribe(context.self, classOf[SetTarget]); sender ! Logging.LoggerInitialized
case SetTarget(ref) dst = ref; dst ! "OK" case SetTarget(ref) dst = ref; dst ! "OK"
@ -36,11 +36,13 @@ class EventStreamSpec extends AkkaSpec(Configuration(
import EventStreamSpec._ import EventStreamSpec._
val impl = system.asInstanceOf[ActorSystemImpl]
"An EventStream" must { "An EventStream" must {
"manage subscriptions" in { "manage subscriptions" in {
val bus = new EventStream(true) val bus = new EventStream(true)
bus.start(app.provider) bus.start(impl)
bus.subscribe(testActor, classOf[M]) bus.subscribe(testActor, classOf[M])
bus.publish(M(42)) bus.publish(M(42))
within(1 second) { within(1 second) {
@ -53,8 +55,8 @@ class EventStreamSpec extends AkkaSpec(Configuration(
"manage log levels" in { "manage log levels" in {
val bus = new EventStream(false) val bus = new EventStream(false)
bus.start(app.provider) bus.start(impl)
bus.startDefaultLoggers(app.provider, app.AkkaConfig) bus.startDefaultLoggers(impl)
bus.publish(SetTarget(testActor)) bus.publish(SetTarget(testActor))
expectMsg("OK") expectMsg("OK")
within(2 seconds) { within(2 seconds) {
@ -75,7 +77,7 @@ class EventStreamSpec extends AkkaSpec(Configuration(
val b2 = new B2 val b2 = new B2
val c = new C val c = new C
val bus = new EventStream(false) val bus = new EventStream(false)
bus.start(app.provider) bus.start(impl)
within(2 seconds) { within(2 seconds) {
bus.subscribe(testActor, classOf[B2]) === true bus.subscribe(testActor, classOf[B2]) === true
bus.publish(c) bus.publish(c)

View file

@ -17,7 +17,7 @@ import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistic
class TellLatencyPerformanceSpec extends PerformanceSpec { class TellLatencyPerformanceSpec extends PerformanceSpec {
import TellLatencyPerformanceSpec._ import TellLatencyPerformanceSpec._
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher") val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(8) .setCorePoolSize(8)
.build .build
@ -62,13 +62,13 @@ class TellLatencyPerformanceSpec extends PerformanceSpec {
val latch = new CountDownLatch(numberOfClients) val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients val repeatsPerClient = repeat / numberOfClients
val clients = (for (i 0 until numberOfClients) yield { val clients = (for (i 0 until numberOfClients) yield {
val destination = app.actorOf[Destination] val destination = system.actorOf[Destination]
val w4 = app.actorOf(new Waypoint(destination)) val w4 = system.actorOf(new Waypoint(destination))
val w3 = app.actorOf(new Waypoint(w4)) val w3 = system.actorOf(new Waypoint(w4))
val w2 = app.actorOf(new Waypoint(w3)) val w2 = system.actorOf(new Waypoint(w3))
val w1 = app.actorOf(new Waypoint(w2)) val w1 = system.actorOf(new Waypoint(w2))
Props(new Client(w1, latch, repeatsPerClient, clientDelayMicros, stat)).withDispatcher(clientDispatcher) Props(new Client(w1, latch, repeatsPerClient, clientDelayMicros, stat)).withDispatcher(clientDispatcher)
}).toList.map(app.actorOf(_)) }).toList.map(system.actorOf(_))
val start = System.nanoTime val start = System.nanoTime
clients.foreach(_ ! Run) clients.foreach(_ ! Run)

View file

@ -18,12 +18,12 @@ import akka.dispatch.Dispatchers
class TellThroughputPerformanceSpec extends PerformanceSpec { class TellThroughputPerformanceSpec extends PerformanceSpec {
import TellThroughputPerformanceSpec._ import TellThroughputPerformanceSpec._
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher") val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients) .setCorePoolSize(maxClients)
.build .build
val destinationDispatcher = app.dispatcherFactory.newDispatcher("destination-dispatcher") val destinationDispatcher = system.dispatcherFactory.newDispatcher("destination-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients) .setCorePoolSize(maxClients)
.build .build
@ -71,9 +71,9 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
val latch = new CountDownLatch(numberOfClients) val latch = new CountDownLatch(numberOfClients)
val repeatsPerClient = repeat / numberOfClients val repeatsPerClient = repeat / numberOfClients
val destinations = for (i 0 until numberOfClients) val destinations = for (i 0 until numberOfClients)
yield app.actorOf(Props(new Destination).withDispatcher(destinationDispatcher)) yield system.actorOf(Props(new Destination).withDispatcher(destinationDispatcher))
val clients = for (dest destinations) val clients = for (dest destinations)
yield app.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher)) yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
val start = System.nanoTime val start = System.nanoTime
clients.foreach(_ ! Run) clients.foreach(_ ! Run)

View file

@ -21,7 +21,7 @@ import akka.performance.trading.domain.Orderbook
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TradingLatencyPerformanceSpec extends PerformanceSpec { class TradingLatencyPerformanceSpec extends PerformanceSpec {
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher") val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients) .setCorePoolSize(maxClients)
.build .build
@ -38,7 +38,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
override def beforeEach() { override def beforeEach() {
super.beforeEach() super.beforeEach()
stat = new SynchronizedDescriptiveStatistics stat = new SynchronizedDescriptiveStatistics
tradingSystem = new AkkaTradingSystem(app) tradingSystem = new AkkaTradingSystem(system)
tradingSystem.start() tradingSystem.start()
TotalTradeCounter.reset() TotalTradeCounter.reset()
stat = new SynchronizedDescriptiveStatistics stat = new SynchronizedDescriptiveStatistics
@ -99,7 +99,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
val clients = (for (i 0 until numberOfClients) yield { val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size) val receiver = receivers(i % receivers.size)
val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelayMicros)).withDispatcher(clientDispatcher) val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelayMicros)).withDispatcher(clientDispatcher)
app.actorOf(props) system.actorOf(props)
}) })
clients.foreach(_ ! "run") clients.foreach(_ ! "run")

View file

@ -21,7 +21,7 @@ import akka.performance.trading.domain.Orderbook
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class TradingThroughputPerformanceSpec extends PerformanceSpec { class TradingThroughputPerformanceSpec extends PerformanceSpec {
val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher") val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
.setCorePoolSize(maxClients) .setCorePoolSize(maxClients)
.build .build
@ -30,7 +30,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
override def beforeEach() { override def beforeEach() {
super.beforeEach() super.beforeEach()
tradingSystem = new AkkaTradingSystem(app) tradingSystem = new AkkaTradingSystem(system)
tradingSystem.start() tradingSystem.start()
TotalTradeCounter.reset() TotalTradeCounter.reset()
} }
@ -92,7 +92,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
val clients = (for (i 0 until numberOfClients) yield { val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size) val receiver = receivers(i % receivers.size)
val props = Props(new Client(receiver, orders, latch, ordersPerClient)).withDispatcher(clientDispatcher) val props = Props(new Client(receiver, orders, latch, ordersPerClient)).withDispatcher(clientDispatcher)
app.actorOf(props) system.actorOf(props)
}) })
clients.foreach(_ ! "run") clients.foreach(_ ! "run")

View file

@ -11,8 +11,6 @@ import akka.actor.ActorSystem
trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach { trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
def app: ActorSystem
def isBenchmark() = System.getProperty("benchmark") == "true" def isBenchmark() = System.getProperty("benchmark") == "true"
def minClients() = System.getProperty("benchmark.minClients", "1").toInt; def minClients() = System.getProperty("benchmark.minClients", "1").toInt;
@ -29,7 +27,7 @@ trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
} }
val resultRepository = BenchResultRepository() val resultRepository = BenchResultRepository()
lazy val report = new Report(app, resultRepository, compareResultWith) lazy val report = new Report(system, resultRepository, compareResultWith)
/** /**
* To compare two tests with each other you can override this method, in * To compare two tests with each other you can override this method, in

View file

@ -224,8 +224,8 @@ class Report(
sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion) sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion)
sb.append("\n") sb.append("\n")
sb.append("Akka config:") sb.append("Akka config:")
for (key app.config.keys) { for (key app.AkkaConfig.config.keys) {
sb.append("\n ").append(key).append("=").append(app.config(key)) sb.append("\n ").append(key).append("=").append(app.AkkaConfig.config(key))
} }
sb.toString sb.toString

View file

@ -329,7 +329,7 @@ class ActorPoolSpec extends AkkaSpec {
} }
"support typed actors" in { "support typed actors" in {
val pool = app.createProxy[Foo](new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup { val pool = system.createProxy[Foo](new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup {
def lowerBound = 1 def lowerBound = 1
def upperBound = 5 def upperBound = 5
def pressureThreshold = 1 def pressureThreshold = 1
@ -338,7 +338,7 @@ class ActorPoolSpec extends AkkaSpec {
def rampupRate = 0.1 def rampupRate = 0.1
def backoffRate = 0.50 def backoffRate = 0.50
def backoffThreshold = 0.50 def backoffThreshold = 0.50
def instance(p: Props) = app.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds))) def instance(p: Props) = system.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds)))
def receive = _route def receive = _route
}, Props().withTimeout(10 seconds).withFaultHandler(faultHandler)) }, Props().withTimeout(10 seconds).withFaultHandler(faultHandler))
@ -348,7 +348,7 @@ class ActorPoolSpec extends AkkaSpec {
val value = r.get val value = r.get
value must equal(i * i) value must equal(i * i)
} }
app.typedActor.stop(pool) system.typedActor.stop(pool)
} }
"provide default supervision of pooled actors" in { "provide default supervision of pooled actors" in {

View file

@ -11,12 +11,14 @@ import akka.routing.Routing.Broadcast
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ConfiguredLocalRoutingSpec extends AkkaSpec { class ConfiguredLocalRoutingSpec extends AkkaSpec {
val deployer = system.asInstanceOf[ActorSystemImpl].provider.deployer
"round robin router" must { "round robin router" must {
"be able to shut down its instance" in { "be able to shut down its instance" in {
val path = app / "round-robin-0" val path = system / "round-robin-0"
app.provider.deployer.deploy( deployer.deploy(
Deploy( Deploy(
path.toString, path.toString,
None, None,
@ -27,7 +29,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val helloLatch = new CountDownLatch(5) val helloLatch = new CountDownLatch(5)
val stopLatch = new CountDownLatch(5) val stopLatch = new CountDownLatch(5)
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "hello" helloLatch.countDown() case "hello" helloLatch.countDown()
} }
@ -49,9 +51,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
} }
"deliver messages in a round robin fashion" in { "deliver messages in a round robin fashion" in {
val path = app / "round-robin-1" val path = system / "round-robin-1"
app.provider.deployer.deploy( deployer.deploy(
Deploy( Deploy(
path.toString, path.toString,
None, None,
@ -69,7 +71,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
replies = replies + (i -> 0) replies = replies + (i -> 0)
} }
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement() lazy val id = counter.getAndIncrement()
def receive = { def receive = {
case "hit" sender ! id case "hit" sender ! id
@ -93,9 +95,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
} }
"deliver a broadcast message using the !" in { "deliver a broadcast message using the !" in {
val path = app / "round-robin-2" val path = system / "round-robin-2"
app.provider.deployer.deploy( deployer.deploy(
Deploy( Deploy(
path.toString, path.toString,
None, None,
@ -106,7 +108,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val helloLatch = new CountDownLatch(5) val helloLatch = new CountDownLatch(5)
val stopLatch = new CountDownLatch(5) val stopLatch = new CountDownLatch(5)
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "hello" helloLatch.countDown() case "hello" helloLatch.countDown()
} }
@ -127,9 +129,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
"random router" must { "random router" must {
"be able to shut down its instance" in { "be able to shut down its instance" in {
val path = app / "random-0" val path = system / "random-0"
app.provider.deployer.deploy( deployer.deploy(
Deploy( Deploy(
path.toString, path.toString,
None, None,
@ -139,7 +141,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val stopLatch = new CountDownLatch(7) val stopLatch = new CountDownLatch(7)
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "hello" {} case "hello" {}
} }
@ -160,9 +162,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
} }
"deliver messages in a random fashion" in { "deliver messages in a random fashion" in {
val path = app / "random-1" val path = system / "random-1"
app.provider.deployer.deploy( deployer.deploy(
Deploy( Deploy(
path.toString, path.toString,
None, None,
@ -180,7 +182,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
replies = replies + (i -> 0) replies = replies + (i -> 0)
} }
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
lazy val id = counter.getAndIncrement() lazy val id = counter.getAndIncrement()
def receive = { def receive = {
case "hit" sender ! id case "hit" sender ! id
@ -204,9 +206,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
} }
"deliver a broadcast message using the !" in { "deliver a broadcast message using the !" in {
val path = app / "random-2" val path = system / "random-2"
app.provider.deployer.deploy( deployer.deploy(
Deploy( Deploy(
path.toString, path.toString,
None, None,
@ -217,7 +219,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec {
val helloLatch = new CountDownLatch(6) val helloLatch = new CountDownLatch(6)
val stopLatch = new CountDownLatch(6) val stopLatch = new CountDownLatch(6)
val actor = app.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "hello" helloLatch.countDown() case "hello" helloLatch.countDown()
} }

View file

@ -22,6 +22,8 @@ object RoutingSpec {
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class RoutingSpec extends AkkaSpec { class RoutingSpec extends AkkaSpec {
val impl = system.asInstanceOf[ActorSystemImpl]
import akka.routing.RoutingSpec._ import akka.routing.RoutingSpec._
"direct router" must { "direct router" must {
@ -29,7 +31,7 @@ class RoutingSpec extends AkkaSpec {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(actor1))) val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false) actor.isShutdown must be(false)
} }
@ -45,7 +47,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1))) val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1)))
val routedActor = new RoutedActorRef(app, props, app.guardian, "foo") val routedActor = new RoutedActorRef(system, props, impl.guardian, "foo")
routedActor ! "hello" routedActor ! "hello"
routedActor ! "end" routedActor ! "end"
@ -66,7 +68,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1))) val props = RoutedProps(routerFactory = () new DirectRouter, connectionManager = new LocalConnectionManager(List(connection1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(1) actor ! Broadcast(1)
actor ! "end" actor ! "end"
@ -83,7 +85,7 @@ class RoutingSpec extends AkkaSpec {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(actor1))) val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false) actor.isShutdown must be(false)
} }
@ -113,7 +115,7 @@ class RoutingSpec extends AkkaSpec {
//create the routed actor. //create the routed actor.
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(connections)) val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(connections))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
//send messages to the actor. //send messages to the actor.
for (i 0 until iterationCount) { for (i 0 until iterationCount) {
@ -152,7 +154,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2))) val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(1) actor ! Broadcast(1)
actor ! Broadcast("end") actor ! Broadcast("end")
@ -175,7 +177,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1))) val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(List(connection1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
intercept[RoutingException] { actor ? Broadcast(1) } intercept[RoutingException] { actor ? Broadcast(1) }
@ -192,7 +194,7 @@ class RoutingSpec extends AkkaSpec {
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(actor1))) val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false) actor.isShutdown must be(false)
} }
@ -216,7 +218,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2))) val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(1) actor ! Broadcast(1)
actor ! Broadcast("end") actor ! Broadcast("end")
@ -239,7 +241,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1))) val props = RoutedProps(routerFactory = () new RandomRouter, connectionManager = new LocalConnectionManager(List(connection1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
try { try {
actor ? Broadcast(1) actor ? Broadcast(1)
@ -262,7 +264,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch))))) val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch)))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(Stop(Some(0))) actor ! Broadcast(Stop(Some(0)))
@ -277,7 +279,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch))))) val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch)))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(Stop()) actor ! Broadcast(Stop())
@ -293,7 +295,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0), newActor(1)))) val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0), newActor(1))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
(actor ? Broadcast("Hi!")).get.asInstanceOf[Int] must be(0) (actor ? Broadcast("Hi!")).get.asInstanceOf[Int] must be(0)
@ -302,14 +304,14 @@ class RoutingSpec extends AkkaSpec {
"return the first response from connections, when some of them failed to reply" in { "return the first response from connections, when some of them failed to reply" in {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0), newActor(1)))) val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0), newActor(1))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
(actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1) (actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1)
} }
"be started when constructed" in { "be started when constructed" in {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0)))) val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(newActor(0))))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor.isShutdown must be(false) actor.isShutdown must be(false)
} }
@ -324,7 +326,7 @@ class RoutingSpec extends AkkaSpec {
for (i 0 until connectionCount) { for (i 0 until connectionCount) {
counters = counters :+ new AtomicInteger() counters = counters :+ new AtomicInteger()
val connection = app.actorOf(new Actor { val connection = system.actorOf(new Actor {
def receive = { def receive = {
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
case msg: Int counters.get(i).get.addAndGet(msg) case msg: Int counters.get(i).get.addAndGet(msg)
@ -335,7 +337,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(connections)) val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(connections))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
for (i 0 until iterationCount) { for (i 0 until iterationCount) {
for (k 0 until connectionCount) { for (k 0 until connectionCount) {
@ -357,7 +359,7 @@ class RoutingSpec extends AkkaSpec {
val doneLatch = new TestLatch(2) val doneLatch = new TestLatch(2)
val counter1 = new AtomicInteger val counter1 = new AtomicInteger
val connection1 = app.actorOf(new Actor { val connection1 = system.actorOf(new Actor {
def receive = { def receive = {
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg) case msg: Int counter1.addAndGet(msg)
@ -365,7 +367,7 @@ class RoutingSpec extends AkkaSpec {
}) })
val counter2 = new AtomicInteger val counter2 = new AtomicInteger
val connection2 = app.actorOf(new Actor { val connection2 = system.actorOf(new Actor {
def receive = { def receive = {
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg) case msg: Int counter2.addAndGet(msg)
@ -374,7 +376,7 @@ class RoutingSpec extends AkkaSpec {
val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2))) val props = RoutedProps(routerFactory = () new ScatterGatherFirstCompletedRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo") val actor = new RoutedActorRef(system, props, impl.guardian, "foo")
actor ! Broadcast(1) actor ! Broadcast(1)
actor ! Broadcast("end") actor ! Broadcast("end")
@ -387,7 +389,7 @@ class RoutingSpec extends AkkaSpec {
case class Stop(id: Option[Int] = None) case class Stop(id: Option[Int] = None)
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = app.actorOf(new Actor { def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = system.actorOf(new Actor {
def receive = { def receive = {
case Stop(None) self.stop() case Stop(None) self.stop()
case Stop(Some(_id)) if (_id == id) self.stop() case Stop(Some(_id)) if (_id == id) self.stop()

View file

@ -7,7 +7,7 @@ package akka.serialization
import akka.serialization.Serialization._ import akka.serialization.Serialization._
import scala.reflect._ import scala.reflect._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.actor.ActorSystem import akka.actor.{ ActorSystem, ActorSystemImpl }
import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream } import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import akka.actor.DeadLetterActorRef import akka.actor.DeadLetterActorRef
@ -24,7 +24,8 @@ object SerializeSpec {
class SerializeSpec extends AkkaSpec { class SerializeSpec extends AkkaSpec {
import SerializeSpec._ import SerializeSpec._
import app.serialization._ val ser = system.serialization
import ser._
"Serialization" must { "Serialization" must {
@ -68,13 +69,13 @@ class SerializeSpec extends AkkaSpec {
"serialize DeadLetterActorRef" in { "serialize DeadLetterActorRef" in {
val outbuf = new ByteArrayOutputStream() val outbuf = new ByteArrayOutputStream()
val out = new ObjectOutputStream(outbuf) val out = new ObjectOutputStream(outbuf)
val a = new ActorSystem() val a = ActorSystem()
out.writeObject(a.deadLetters) out.writeObject(a.deadLetters)
out.flush() out.flush()
out.close() out.close()
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray)) val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
Serialization.app.withValue(a) { Serialization.app.withValue(a.asInstanceOf[ActorSystemImpl]) {
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef] val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
(deadLetters eq a.deadLetters) must be(true) (deadLetters eq a.deadLetters) must be(true)
} }

View file

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

View file

@ -154,7 +154,7 @@ object Timeout {
} }
trait ActorLogging { this: Actor trait ActorLogging { this: Actor
val log = akka.event.Logging(app.eventStream, context.self) val log = akka.event.Logging(system.eventStream, context.self)
} }
object Actor { object Actor {
@ -229,12 +229,12 @@ trait Actor {
context context
} }
implicit def app = context.app implicit def system = context.system
/** /**
* The default timeout, based on the config setting 'akka.actor.timeout' * The default timeout, based on the config setting 'akka.actor.timeout'
*/ */
implicit def defaultTimeout = app.AkkaConfig.ActorTimeout implicit def defaultTimeout = system.AkkaConfig.ActorTimeout
/** /**
* Wrap a Receive partial function in a logging enclosure, which sends a * Wrap a Receive partial function in a logging enclosure, which sends a
@ -250,7 +250,7 @@ trait Actor {
* This method does NOT modify the given Receive unless * This method does NOT modify the given Receive unless
* akka.actor.debug.receive is set within akka.conf. * akka.actor.debug.receive is set within akka.conf.
*/ */
def loggable(self: AnyRef)(r: Receive): Receive = if (app.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait? def loggable(self: AnyRef)(r: Receive): Receive = if (system.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait?
/** /**
* Some[ActorRef] representation of the 'self' ActorRef reference. * Some[ActorRef] representation of the 'self' ActorRef reference.

View file

@ -45,7 +45,7 @@ trait ActorContext extends ActorRefFactory with TypedActorFactory {
def handleChildTerminated(child: ActorRef): Unit def handleChildTerminated(child: ActorRef): Unit
def app: ActorSystem def system: ActorSystem
def parent: ActorRef def parent: ActorRef
} }
@ -63,7 +63,7 @@ private[akka] object ActorCell {
//vars don't need volatile since it's protected with the mailbox status //vars don't need volatile since it's protected with the mailbox status
//Make sure that they are not read/written outside of a message processing (systemInvoke/invoke) //Make sure that they are not read/written outside of a message processing (systemInvoke/invoke)
private[akka] class ActorCell( private[akka] class ActorCell(
val app: ActorSystem, val app: ActorSystemImpl,
val self: ActorRef with ScalaActorRef, val self: ActorRef with ScalaActorRef,
val props: Props, val props: Props,
val parent: ActorRef, val parent: ActorRef,
@ -72,6 +72,8 @@ private[akka] class ActorCell(
import ActorCell._ import ActorCell._
final def system = app
protected final def guardian = self protected final def guardian = self
protected def typedActor = app.typedActor protected def typedActor = app.typedActor

View file

@ -4,6 +4,8 @@
package akka.actor package akka.actor
import akka.remote.RemoteAddress
object ActorPath { object ActorPath {
final val separator = "/" final val separator = "/"
@ -58,9 +60,9 @@ object ActorPath {
*/ */
trait ActorPath { trait ActorPath {
/** /**
* The akka application for this path. * The RemoteAddress for this path.
*/ */
def app: ActorSystem def remoteAddress: RemoteAddress
/** /**
* The name of the actor that this path refers to. * The name of the actor that this path refers to.
@ -78,9 +80,9 @@ trait ActorPath {
def /(child: String): ActorPath def /(child: String): ActorPath
/** /**
* Find the ActorRef for this path. * Recursively create a descendants path by appending all child names.
*/ */
def ref: Option[ActorRef] def /(child: Iterable[String]): ActorPath = (this /: child)(_ / _)
/** /**
* String representation of this path. Different from toString for root path. * String representation of this path. Different from toString for root path.
@ -98,15 +100,13 @@ trait ActorPath {
def isRoot: Boolean def isRoot: Boolean
} }
class RootActorPath(val app: ActorSystem) extends ActorPath { class RootActorPath(val remoteAddress: RemoteAddress) extends ActorPath {
def name: String = "/" def name: String = "/"
def parent: ActorPath = this def parent: ActorPath = this
def /(child: String): ActorPath = new ChildActorPath(app, this, child) def /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child)
def ref: Option[ActorRef] = app.actorFor(path)
def string: String = "" def string: String = ""
@ -117,11 +117,9 @@ class RootActorPath(val app: ActorSystem) extends ActorPath {
override def toString = ActorPath.separator override def toString = ActorPath.separator
} }
class ChildActorPath(val app: ActorSystem, val parent: ActorPath, val name: String) extends ActorPath { class ChildActorPath(val remoteAddress: RemoteAddress, val parent: ActorPath, val name: String) extends ActorPath {
def /(child: String): ActorPath = new ChildActorPath(app, this, child) def /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child)
def ref: Option[ActorRef] = app.actorFor(path)
def string: String = parent.string + ActorPath.separator + name def string: String = parent.string + ActorPath.separator + name

View file

@ -13,6 +13,7 @@ import java.net.InetSocketAddress
import akka.remote.RemoteAddress import akka.remote.RemoteAddress
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.event.EventStream import akka.event.EventStream
import akka.event.DeathWatch
/** /**
* ActorRef is an immutable and serializable handle to an Actor. * ActorRef is an immutable and serializable handle to an Actor.
@ -161,7 +162,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class LocalActorRef private[akka] ( class LocalActorRef private[akka] (
app: ActorSystem, app: ActorSystemImpl,
_props: Props, _props: Props,
_supervisor: ActorRef, _supervisor: ActorRef,
val path: ActorPath, val path: ActorPath,
@ -172,7 +173,7 @@ class LocalActorRef private[akka] (
def name = path.name def name = path.name
def address: String = app.address + path.toString def address: String = path.toString
/* /*
* actorCell.start() publishes actorCell & this to the dispatcher, which * actorCell.start() publishes actorCell & this to the dispatcher, which
@ -415,16 +416,15 @@ class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) exte
private def writeReplace(): AnyRef = DeadLetterActorRef.serialized private def writeReplace(): AnyRef = DeadLetterActorRef.serialized
} }
abstract class AskActorRef(protected val app: ActorSystem)(timeout: Timeout = app.AkkaConfig.ActorTimeout, dispatcher: MessageDispatcher = app.dispatcher) extends MinimalActorRef { abstract class AskActorRef(val path: ActorPath, provider: ActorRefProvider, deathWatch: DeathWatch, timeout: Timeout, val dispatcher: MessageDispatcher) extends MinimalActorRef {
final val result = new DefaultPromise[Any](timeout)(dispatcher) final val result = new DefaultPromise[Any](timeout)(dispatcher)
// FIXME (actor path): put this under the tmp guardian supervisor override def name = path.name
val path: ActorPath = app.root / "tmp" / name
def address: String = app.address + path.toString def address: String = path.toString
{ {
val callback: Future[Any] Unit = { _ app.deathWatch.publish(Terminated(AskActorRef.this)); whenDone() } val callback: Future[Any] Unit = { _ deathWatch.publish(Terminated(AskActorRef.this)); whenDone() }
result onComplete callback result onComplete callback
result onTimeout callback result onTimeout callback
} }
@ -450,5 +450,5 @@ abstract class AskActorRef(protected val app: ActorSystem)(timeout: Timeout = ap
override def stop(): Unit = if (!isShutdown) result.completeWithException(new ActorKilledException("Stopped")) override def stop(): Unit = if (!isShutdown) result.completeWithException(new ActorKilledException("Stopped"))
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = app.provider.serialize(this) private def writeReplace(): AnyRef = provider.serialize(this)
} }

View file

@ -22,7 +22,7 @@ import com.eaio.uuid.UUID
*/ */
trait ActorRefProvider { trait ActorRefProvider {
def actorOf(props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(props, supervisor, name, false) def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(app, props, supervisor, name, false)
def actorFor(path: Iterable[String]): Option[ActorRef] def actorFor(path: Iterable[String]): Option[ActorRef]
@ -36,6 +36,8 @@ trait ActorRefProvider {
def AkkaConfig: ActorSystem.AkkaConfig def AkkaConfig: ActorSystem.AkkaConfig
def init(app: ActorSystemImpl)
/** /**
* What deployer will be used to resolve deployment configuration? * What deployer will be used to resolve deployment configuration?
*/ */
@ -43,9 +45,9 @@ trait ActorRefProvider {
private[akka] def scheduler: Scheduler private[akka] def scheduler: Scheduler
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef
private[akka] def evict(path: String): Boolean private[akka] def evict(path: String): Boolean
@ -61,7 +63,7 @@ trait ActorRefProvider {
private[akka] def terminationFuture: Future[ActorSystem.ExitStatus] private[akka] def terminationFuture: Future[ActorSystem.ExitStatus]
private[akka] def tempPath: String private[akka] def tempName: String
} }
/** /**
@ -69,9 +71,11 @@ trait ActorRefProvider {
*/ */
trait ActorRefFactory { trait ActorRefFactory {
def provider: ActorRefProvider protected def app: ActorSystemImpl
def dispatcher: MessageDispatcher protected def provider: ActorRefProvider
protected def dispatcher: MessageDispatcher
/** /**
* Father of all children created by this interface. * Father of all children created by this interface.
@ -85,7 +89,7 @@ trait ActorRefFactory {
Helpers.base64(l) Helpers.base64(l)
} }
def actorOf(props: Props): ActorRef = provider.actorOf(props, guardian, randomName, false) def actorOf(props: Props): ActorRef = provider.actorOf(app, props, guardian, randomName, false)
/* /*
* TODO this will have to go at some point, because creating two actors with * TODO this will have to go at some point, because creating two actors with
@ -95,7 +99,7 @@ trait ActorRefFactory {
def actorOf(props: Props, name: String): ActorRef = { def actorOf(props: Props, name: String): ActorRef = {
if (name == null || name == "" || name.startsWith("$")) if (name == null || name == "" || name.startsWith("$"))
throw new ActorInitializationException("actor name must not be null, empty or start with $") throw new ActorInitializationException("actor name must not be null, empty or start with $")
provider.actorOf(props, guardian, name, false) provider.actorOf(app, props, guardian, name, false)
} }
def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]])) def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]))
@ -109,6 +113,8 @@ trait ActorRefFactory {
def actorOf(creator: UntypedActorFactory): ActorRef = actorOf(Props(() creator.create())) def actorOf(creator: UntypedActorFactory): ActorRef = actorOf(Props(() creator.create()))
def actorFor(path: ActorPath): Option[ActorRef] = actorFor(path.path)
def actorFor(path: String): Option[ActorRef] = actorFor(ActorPath.split(path)) def actorFor(path: String): Option[ActorRef] = actorFor(ActorPath.split(path))
def actorFor(path: Iterable[String]): Option[ActorRef] = provider.actorFor(path) def actorFor(path: Iterable[String]): Option[ActorRef] = provider.actorFor(path)
@ -120,7 +126,6 @@ class ActorRefProviderException(message: String) extends AkkaException(message)
* Local ActorRef provider. * Local ActorRef provider.
*/ */
class LocalActorRefProvider( class LocalActorRefProvider(
private val app: ActorSystem,
val AkkaConfig: ActorSystem.AkkaConfig, val AkkaConfig: ActorSystem.AkkaConfig,
val root: ActorPath, val root: ActorPath,
val eventStream: EventStream, val eventStream: EventStream,
@ -136,16 +141,18 @@ class LocalActorRefProvider(
private[akka] val deployer: Deployer = new Deployer(AkkaConfig, eventStream, nodename) private[akka] val deployer: Deployer = new Deployer(AkkaConfig, eventStream, nodename)
val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(app.dispatcher) val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(dispatcher)
/* /*
* generate name for temporary actor refs * generate name for temporary actor refs
*/ */
private val tempNumber = new AtomicLong private val tempNumber = new AtomicLong
def tempPath = { def tempName = {
val l = tempNumber.getAndIncrement() val l = tempNumber.getAndIncrement()
"$_" + Helpers.base64(l) "$_" + Helpers.base64(l)
} }
private val tempNode = root / "tmp"
def tempPath = tempNode / tempName
// FIXME (actor path): this could become a cache for the new tree traversal actorFor // FIXME (actor path): this could become a cache for the new tree traversal actorFor
// currently still used for tmp actors (e.g. ask actor refs) // currently still used for tmp actors (e.g. ask actor refs)
@ -159,12 +166,12 @@ class LocalActorRefProvider(
@volatile @volatile
var stopped = false var stopped = false
val name = app.name + "-bubble-walker" val name = "bubble-walker"
// FIXME (actor path): move the root path to the new root guardian // FIXME (actor path): move the root path to the new root guardian
val path = app.root val path = root / name
val address = app.address + path.toString val address = path.toString
override def toString = name override def toString = name
@ -208,20 +215,33 @@ class LocalActorRefProvider(
} }
private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy) private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy)
private val rootGuardian: ActorRef = actorOf(guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, root, true) /*
* The problem is that ActorRefs need a reference to the ActorSystem to
val guardian: ActorRef = actorOf(guardianProps, rootGuardian, "app", true) * provide their service. Hence they cannot be created while the
* constructors of ActorSystem and ActorRefProvider are still running.
val systemGuardian: ActorRef = actorOf(guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true) * The solution is to split out that last part into an init() method,
* but it also requires these references to be @volatile.
*/
@volatile
private var rootGuardian: ActorRef = _
@volatile
var guardian: ActorRef = _
@volatile
var systemGuardian: ActorRef = _
val deathWatch = createDeathWatch() val deathWatch = createDeathWatch()
def init(app: ActorSystemImpl) {
rootGuardian = actorOf(app, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, root, true)
guardian = actorOf(app, guardianProps, rootGuardian, "app", true)
systemGuardian = actorOf(app, guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true)
// chain death watchers so that killing guardian stops the application // chain death watchers so that killing guardian stops the application
deathWatch.subscribe(systemGuardian, guardian) deathWatch.subscribe(systemGuardian, guardian)
deathWatch.subscribe(rootGuardian, systemGuardian) deathWatch.subscribe(rootGuardian, systemGuardian)
}
// FIXME (actor path): should start at the new root guardian, and not use the tail (just to avoid the expected "app" name for now) // FIXME (actor path): should start at the new root guardian, and not use the tail (just to avoid the expected "app" name for now)
def actorFor(path: Iterable[String]): Option[ActorRef] = findInCache(ActorPath.join(path)) orElse findInTree(Some(app.guardian), path.tail) def actorFor(path: Iterable[String]): Option[ActorRef] = findInCache(ActorPath.join(path)) orElse findInTree(Some(guardian), path.tail)
@tailrec @tailrec
private def findInTree(start: Option[ActorRef], path: Iterable[String]): Option[ActorRef] = { private def findInTree(start: Option[ActorRef], path: Iterable[String]): Option[ActorRef] = {
@ -246,12 +266,12 @@ class LocalActorRefProvider(
*/ */
private[akka] def evict(path: String): Boolean = actors.remove(path) ne null private[akka] def evict(path: String): Boolean = actors.remove(path) ne null
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(props, supervisor, supervisor.path / name, systemService) actorOf(app, props, supervisor, supervisor.path / name, systemService)
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = { private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = {
val name = path.name val name = path.name
val newFuture = Promise[ActorRef](5000)(app.dispatcher) // FIXME is this proper timeout? val newFuture = Promise[ActorRef](5000)(dispatcher) // FIXME is this proper timeout?
actors.putIfAbsent(path.toString, newFuture) match { actors.putIfAbsent(path.toString, newFuture) match {
case null case null
@ -270,7 +290,7 @@ class LocalActorRefProvider(
case RouterType.Random () new RandomRouter case RouterType.Random () new RandomRouter
case RouterType.RoundRobin () new RoundRobinRouter case RouterType.RoundRobin () new RoundRobinRouter
case RouterType.ScatterGather () new ScatterGatherFirstCompletedRouter()( case RouterType.ScatterGather () new ScatterGatherFirstCompletedRouter()(
if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher, app.AkkaConfig.ActorTimeout) if (props.dispatcher == Props.defaultDispatcher) dispatcher else props.dispatcher, AkkaConfig.ActorTimeout)
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet") case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet") case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet") case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet")
@ -282,7 +302,7 @@ class LocalActorRefProvider(
new LocalActorRef(app, props, supervisor, routedPath, systemService) new LocalActorRef(app, props, supervisor, routedPath, systemService)
} }
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString) actorOf(app, RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString)
case unknown throw new Exception("Don't know how to create this actor ref! Why? Got: " + unknown) case unknown throw new Exception("Don't know how to create this actor ref! Why? Got: " + unknown)
} }
@ -307,7 +327,7 @@ class LocalActorRefProvider(
/** /**
* Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration. * Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration.
*/ */
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = { def actorOf(app: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
// FIXME: this needs to take supervision into account! // FIXME: this needs to take supervision into account!
//FIXME clustering should be implemented by cluster actor ref provider //FIXME clustering should be implemented by cluster actor ref provider
@ -324,16 +344,16 @@ class LocalActorRefProvider(
} }
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(ActorPath.split(actor.path)) private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(ActorPath.split(actor.path))
private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(app.address, actor.path.toString) private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(root.remoteAddress, actor.path.toString)
private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = { private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = {
import akka.dispatch.{ Future, Promise, DefaultPromise } import akka.dispatch.{ Future, Promise, DefaultPromise }
(if (within == null) app.AkkaConfig.ActorTimeout else within) match { (if (within == null) AkkaConfig.ActorTimeout else within) match {
case t if t.duration.length <= 0 new DefaultPromise[Any](0)(app.dispatcher) //Abort early if nonsensical timeout case t if t.duration.length <= 0 new DefaultPromise[Any](0)(dispatcher) //Abort early if nonsensical timeout
case t case t
val a = new AskActorRef(app)(timeout = t) { def whenDone() = actors.remove(this) } val a = new AskActorRef(tempPath, this, deathWatch, t, dispatcher) { def whenDone() = actors.remove(this) }
assert(actors.putIfAbsent(a.path.toString, a) eq null) //If this fails, we're in deep trouble assert(actors.putIfAbsent(a.path.toString, a) eq null) //If this fails, we're in deep trouble
recipient.tell(message, a) recipient.tell(message, a)
a.result a.result

View file

@ -61,11 +61,14 @@ object ActorSystem {
val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig val defaultConfig = fromProperties orElse fromClasspath orElse fromHome getOrElse emptyConfig
def apply(name: String, config: Configuration) = new ActorSystem(name, config) def create(name: String, config: Configuration): ActorSystem = apply(name, config)
def apply(name: String, config: Configuration): ActorSystem = new ActorSystemImpl(name, config).start()
def apply(name: String): ActorSystem = new ActorSystem(name) def create(name: String): ActorSystem = apply(name)
def apply(name: String): ActorSystem = apply(name, defaultConfig)
def apply(): ActorSystem = new ActorSystem() def create(): ActorSystem = apply()
def apply(): ActorSystem = apply("default")
sealed trait ExitStatus sealed trait ExitStatus
case object Stopped extends ExitStatus case object Stopped extends ExitStatus
@ -126,22 +129,54 @@ object ActorSystem {
} }
class ActorSystem(val name: String, val config: Configuration) extends ActorRefFactory with TypedActorFactory { abstract class ActorSystem extends ActorRefFactory with TypedActorFactory {
import ActorSystem._ import ActorSystem._
def this(name: String) = this(name, ActorSystem.defaultConfig) def name: String
def this() = this("default") def AkkaConfig: AkkaConfig
def nodename: String
val AkkaConfig = new AkkaConfig(config) /**
* Construct a path below the application guardian.
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(props, systemGuardian, address, true) */
def /(name: String): ActorPath
import AkkaConfig._ def root: ActorPath
val startTime = System.currentTimeMillis val startTime = System.currentTimeMillis
def uptime = (System.currentTimeMillis - startTime) / 1000 def uptime = (System.currentTimeMillis - startTime) / 1000
def eventStream: EventStream
def log: LoggingAdapter
def deadLetters: ActorRef
def deadLetterMailbox: Mailbox
// FIXME: Serialization should be an extension
def serialization: Serialization
// FIXME: TypedActor should be an extension
def typedActor: TypedActor
def scheduler: Scheduler
def dispatcherFactory: Dispatchers
def dispatcher: MessageDispatcher
def registerOnTermination(code: Unit)
def registerOnTermination(code: Runnable)
def stop()
}
class ActorSystemImpl(val name: String, config: Configuration) extends ActorSystem {
import ActorSystem._
val AkkaConfig = new AkkaConfig(config)
protected def app = this
private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(this, props, systemGuardian, address, true)
import AkkaConfig._
val address = RemoteAddress(System.getProperty("akka.remote.hostname") match { val address = RemoteAddress(System.getProperty("akka.remote.hostname") match {
case null | "" InetAddress.getLocalHost.getHostAddress case null | "" InetAddress.getLocalHost.getHostAddress
case value value case value value
@ -158,7 +193,7 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
/** /**
* The root actor path for this application. * The root actor path for this application.
*/ */
val root: ActorPath = new RootActorPath(this) val root: ActorPath = new RootActorPath(address)
val deadLetters = new DeadLetterActorRef(eventStream, root / "nul") val deadLetters = new DeadLetterActorRef(eventStream, root / "nul")
val deadLetterMailbox = new Mailbox(null) { val deadLetterMailbox = new Mailbox(null) {
@ -181,16 +216,21 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
deadLetters.init(dispatcher) deadLetters.init(dispatcher)
// TODO think about memory consistency effects when doing funky stuff inside constructor
val provider: ActorRefProvider = { val provider: ActorRefProvider = {
val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match { val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match {
case Left(e) throw e case Left(e) throw e
case Right(b) b case Right(b) b
} }
val params: Array[Class[_]] = Array(classOf[ActorSystem], classOf[AkkaConfig], classOf[ActorPath], classOf[EventStream], classOf[MessageDispatcher], classOf[Scheduler]) val arguments = List(
val args: Array[AnyRef] = Array(this, AkkaConfig, root, eventStream, dispatcher, scheduler) classOf[AkkaConfig] -> AkkaConfig,
classOf[ActorPath] -> root,
classOf[EventStream] -> eventStream,
classOf[MessageDispatcher] -> dispatcher,
classOf[Scheduler] -> scheduler)
val types: Array[Class[_]] = arguments map (_._1) toArray
val values: Array[AnyRef] = arguments map (_._2) toArray
ReflectiveAccess.createInstance[ActorRefProvider](providerClass, params, args) match { ReflectiveAccess.createInstance[ActorRefProvider](providerClass, types, values) match {
case Left(e) throw e case Left(e) throw e
case Right(p) p case Right(p) p
} }
@ -200,24 +240,33 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
def guardian: ActorRef = provider.guardian def guardian: ActorRef = provider.guardian
def systemGuardian: ActorRef = provider.systemGuardian def systemGuardian: ActorRef = provider.systemGuardian
def deathWatch: DeathWatch = provider.deathWatch def deathWatch: DeathWatch = provider.deathWatch
def nodename: String = provider.nodename
terminationFuture.onComplete(_ scheduler.stop()) terminationFuture.onComplete(_ scheduler.stop())
terminationFuture.onComplete(_ dispatcher.shutdown()) terminationFuture.onComplete(_ dispatcher.shutdown())
// this starts the reaper actor and the user-configured logging subscribers, which are also actors @volatile
eventStream.start(provider) private var _serialization: Serialization = _
eventStream.startDefaultLoggers(provider, AkkaConfig) def serialization = _serialization
@volatile
private var _typedActor: TypedActor = _
def typedActor = _typedActor
// TODO think about memory consistency effects when doing funky stuff inside constructor
val serialization = new Serialization(this)
val typedActor = new TypedActor(AkkaConfig, serialization)
/**
* Create an actor path under the application supervisor (/app).
*/
def /(actorName: String): ActorPath = guardian.path / actorName def /(actorName: String): ActorPath = guardian.path / actorName
def start(): this.type = {
_serialization = new Serialization(this)
_typedActor = new TypedActor(AkkaConfig, _serialization)
provider.init(this)
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
eventStream.start(this)
eventStream.startDefaultLoggers(this)
this
}
def registerOnTermination(code: Unit) { terminationFuture onComplete (_ code) }
def registerOnTermination(code: Runnable) { terminationFuture onComplete (_ code.run) }
// TODO shutdown all that other stuff, whatever that may be // TODO shutdown all that other stuff, whatever that may be
def stop() { def stop() {
guardian.stop() guardian.stop()

View file

@ -188,7 +188,7 @@ trait FSM[S, D] extends ListenerManagement {
type Timeout = Option[Duration] type Timeout = Option[Duration]
type TransitionHandler = PartialFunction[(S, S), Unit] type TransitionHandler = PartialFunction[(S, S), Unit]
val log = Logging(app.eventStream, context.self) val log = Logging(system, context.self)
/** /**
* **************************************** * ****************************************
@ -522,7 +522,7 @@ trait FSM[S, D] extends ListenerManagement {
if (timeout.isDefined) { if (timeout.isDefined) {
val t = timeout.get val t = timeout.get
if (t.finite_? && t.length >= 0) { if (t.finite_? && t.length >= 0) {
timeoutFuture = Some(app.scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit)) timeoutFuture = Some(system.scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit))
} }
} }
} }
@ -565,7 +565,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
def logDepth: Int = 0 def logDepth: Int = 0
private val debugEvent = context.app.AkkaConfig.FsmDebugEvent private val debugEvent = system.AkkaConfig.FsmDebugEvent
private val events = new Array[Event](logDepth) private val events = new Array[Event](logDepth)
private val states = new Array[AnyRef](logDepth) private val states = new Array[AnyRef](logDepth)

View file

@ -253,7 +253,7 @@ class IOManager(bufferSize: Int = 8192) extends Actor {
var worker: IOWorker = _ var worker: IOWorker = _
override def preStart { override def preStart {
worker = new IOWorker(app, self, bufferSize) worker = new IOWorker(system, self, bufferSize)
worker.start() worker.start()
} }

View file

@ -330,7 +330,7 @@ class TypedActor(val AkkaConfig: ActorSystem.AkkaConfig, var ser: Serialization)
def receive = { def receive = {
case m: MethodCall case m: MethodCall
TypedActor.selfReference set proxyVar.get TypedActor.selfReference set proxyVar.get
TypedActor.appReference set app TypedActor.appReference set system
try { try {
if (m.isOneWay) m(me) if (m.isOneWay) m(me)
else { else {

View file

@ -8,10 +8,6 @@ package object actor {
implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef] implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef]
implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef] implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef]
// actor path can be used as an actor ref (note: does a lookup in the app using path.ref)
implicit def actorPath2Ref(path: ActorPath): ActorRef = path.ref.getOrElse(path.app.deadLetters)
implicit def actorPath2ScalaRef(path: ActorPath): ScalaActorRef = actorPath2Ref(path).asInstanceOf[ScalaActorRef]
type Uuid = com.eaio.uuid.UUID type Uuid = com.eaio.uuid.UUID
def newUuid(): Uuid = new Uuid() def newUuid(): Uuid = new Uuid()

View file

@ -103,7 +103,7 @@ class NodeAddress(val clusterName: String, val nodeName: String) {
*/ */
object NodeAddress { object NodeAddress {
def apply(clusterName: String, nodeName: String): NodeAddress = new NodeAddress(clusterName, nodeName) def apply(clusterName: String, nodeName: String): NodeAddress = new NodeAddress(clusterName, nodeName)
def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.provider.nodename) def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.nodename)
def unapply(other: Any) = other match { def unapply(other: Any) = other match {
case address: NodeAddress Some((address.clusterName, address.nodeName)) case address: NodeAddress Some((address.clusterName, address.nodeName))

View file

@ -3,11 +3,8 @@
*/ */
package akka.event package akka.event
import akka.actor.{ ActorRef, Actor, Props } import akka.actor.{ ActorRef, Actor, Props, ActorSystemImpl, Terminated }
import akka.actor.ActorSystem
import akka.actor.Terminated
import akka.util.Subclassification import akka.util.Subclassification
import akka.actor.ActorRefProvider
class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClassification { class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClassification {
@ -42,13 +39,13 @@ class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClas
super.unsubscribe(subscriber) super.unsubscribe(subscriber)
} }
def start(provider: ActorRefProvider) { def start(app: ActorSystemImpl) {
reaper = provider.actorOf(Props(new Actor { reaper = app.systemActorOf(Props(new Actor {
def receive = { def receive = {
case ref: ActorRef watch(ref) case ref: ActorRef watch(ref)
case Terminated(ref) unsubscribe(ref) case Terminated(ref) unsubscribe(ref)
} }
}), provider.systemGuardian, "MainBusReaper", true) }), "MainBusReaper")
subscribers foreach (reaper ! _) subscribers foreach (reaper ! _)
} }

View file

@ -3,7 +3,7 @@
*/ */
package akka.event package akka.event
import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, simpleName } import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, ActorSystemImpl, simpleName }
import akka.AkkaException import akka.AkkaException
import akka.actor.ActorSystem.AkkaConfig import akka.actor.ActorSystem.AkkaConfig
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
@ -79,13 +79,13 @@ trait LoggingBus extends ActorEventBus {
publish(Info(this, "StandardOutLogger started")) publish(Info(this, "StandardOutLogger started"))
} }
private[akka] def startDefaultLoggers(provider: ActorRefProvider, config: AkkaConfig) { private[akka] def startDefaultLoggers(app: ActorSystemImpl) {
val level = levelFor(config.LogLevel) getOrElse { val level = levelFor(app.AkkaConfig.LogLevel) getOrElse {
StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel)) StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + app.AkkaConfig.LogLevel))
ErrorLevel ErrorLevel
} }
try { try {
val defaultLoggers = config.EventHandlers match { val defaultLoggers = app.AkkaConfig.EventHandlers match {
case Nil "akka.event.Logging$DefaultLogger" :: Nil case Nil "akka.event.Logging$DefaultLogger" :: Nil
case loggers loggers case loggers loggers
} }
@ -95,7 +95,7 @@ trait LoggingBus extends ActorEventBus {
} yield { } yield {
try { try {
ReflectiveAccess.getClassFor[Actor](loggerName) match { ReflectiveAccess.getClassFor[Actor](loggerName) match {
case Right(actorClass) addLogger(provider, actorClass, level) case Right(actorClass) addLogger(app, actorClass, level)
case Left(exception) throw exception case Left(exception) throw exception
} }
} catch { } catch {
@ -138,9 +138,9 @@ trait LoggingBus extends ActorEventBus {
publish(Info(this, "all default loggers stopped")) publish(Info(this, "all default loggers stopped"))
} }
private def addLogger(provider: ActorRefProvider, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = { private def addLogger(app: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
val name = "log" + loggerId.incrementAndGet + "-" + simpleName(clazz) val name = "log" + loggerId.incrementAndGet + "-" + simpleName(clazz)
val actor = provider.actorOf(Props(clazz), provider.systemGuardian, name, true) val actor = app.systemActorOf(Props(clazz), name)
implicit val timeout = Timeout(3 seconds) implicit val timeout = Timeout(3 seconds)
val response = try actor ? InitializeLogger(this) get catch { val response = try actor ? InitializeLogger(this) get catch {
case _: FutureTimeoutException case _: FutureTimeoutException

View file

@ -6,7 +6,7 @@ package akka.serialization
import akka.AkkaException import akka.AkkaException
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import akka.actor.ActorSystem import akka.actor.{ ActorSystem, ActorSystemImpl }
import scala.util.DynamicVariable import scala.util.DynamicVariable
import akka.remote.RemoteSupport import akka.remote.RemoteSupport
@ -16,7 +16,7 @@ case class NoSerializerFoundException(m: String) extends AkkaException(m)
* Serialization module. Contains methods for serialization and deserialization as well as * Serialization module. Contains methods for serialization and deserialization as well as
* locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file. * locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file.
*/ */
class Serialization(val app: ActorSystem) { class Serialization(val app: ActorSystemImpl) {
//TODO document me //TODO document me
def serialize(o: AnyRef): Either[Exception, Array[Byte]] = def serialize(o: AnyRef): Either[Exception, Array[Byte]] =
@ -70,7 +70,7 @@ class Serialization(val app: ActorSystem) {
* But "default" can be overridden in config * But "default" can be overridden in config
*/ */
val serializers: Map[String, Serializer] = val serializers: Map[String, Serializer] =
app.config.getSection("akka.actor.serializers") app.AkkaConfig.config.getSection("akka.actor.serializers")
.map(_.map) .map(_.map)
.getOrElse(Map()) .getOrElse(Map())
.foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) { .foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) {
@ -81,7 +81,7 @@ class Serialization(val app: ActorSystem) {
/** /**
* bindings is a Map whose keys = FQN of class that is serializable and values = the alias of the serializer to be used * bindings is a Map whose keys = FQN of class that is serializable and values = the alias of the serializer to be used
*/ */
val bindings: Map[String, String] = app.config.getSection("akka.actor.serialization-bindings") map { val bindings: Map[String, String] = app.AkkaConfig.config.getSection("akka.actor.serialization-bindings") map {
_.map.foldLeft(Map[String, String]()) { _.map.foldLeft(Map[String, String]()) {
case (result, (k: String, vs: List[_])) result ++ (vs collect { case v: String (v, k) }) //All keys which are lists, take the Strings from them and Map them case (result, (k: String, vs: List[_])) result ++ (vs collect { case v: String (v, k) }) //All keys which are lists, take the Strings from them and Map them
case (result, _) result //For any other values, just skip them, TODO: print out warnings? case (result, _) result //For any other values, just skip them, TODO: print out warnings?
@ -102,6 +102,6 @@ class Serialization(val app: ActorSystem) {
object Serialization { object Serialization {
// TODO ensure that these are always set (i.e. withValue()) when doing deserialization // TODO ensure that these are always set (i.e. withValue()) when doing deserialization
val app = new DynamicVariable[ActorSystem](null) val app = new DynamicVariable[ActorSystemImpl](null)
} }

View file

@ -14,7 +14,7 @@ import akka.config.Configuration
//#my-actor //#my-actor
class MyActor extends Actor { class MyActor extends Actor {
val log = Logging(app, this) val log = Logging(system, this)
def receive = { def receive = {
case "test" log.info("received test") case "test" log.info("received test")
case _ log.info("received unknown message") case _ log.info("received unknown message")
@ -36,8 +36,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) {
case e: Logging.Info true case e: Logging.Info true
case _ false case _ false
} }
app.eventStream.publish(TestEvent.Mute(filter)) system.eventStream.publish(TestEvent.Mute(filter))
app.eventStream.subscribe(testActor, classOf[Logging.Info]) system.eventStream.subscribe(testActor, classOf[Logging.Info])
myActor ! "test" myActor ! "test"
expectMsgPF(1 second) { case Logging.Info(_, "received test") true } expectMsgPF(1 second) { case Logging.Info(_, "received test") true }
@ -45,8 +45,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) {
myActor ! "unknown" myActor ! "unknown"
expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") true } expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") true }
app.eventStream.unsubscribe(testActor) system.eventStream.unsubscribe(testActor)
app.eventStream.publish(TestEvent.UnMute(filter)) system.eventStream.publish(TestEvent.UnMute(filter))
myActor.stop() myActor.stop()
} }

View file

@ -27,8 +27,8 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
def this(app: ActorSystem) { def this(app: ActorSystem) {
this( this(
app.config.getInt("akka.remote.failure-detector.theshold", 8), app.AkkaConfig.config.getInt("akka.remote.failure-detector.theshold", 8),
app.config.getInt("akka.remote.failure-detector.max-sample-size", 1000)) app.AkkaConfig.config.getInt("akka.remote.failure-detector.max-sample-size", 1000))
} }
private final val PhiFactor = 1.0 / math.log(10.0) private final val PhiFactor = 1.0 / math.log(10.0)

View file

@ -107,7 +107,7 @@ class Gossiper(remote: Remote) {
private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[RemoteAddress, ActorRef]) private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[RemoteAddress, ActorRef])
private val seeds = Set(address) // FIXME read in list of seeds from config private val seeds = Set(address) // FIXME read in list of seeds from config
private val address = app.address private val address = app.root.remoteAddress
private val nodeFingerprint = address.## private val nodeFingerprint = address.##
private val random = SecureRandom.getInstance("SHA1PRNG") private val random = SecureRandom.getInstance("SHA1PRNG")
@ -153,7 +153,7 @@ class Gossiper(remote: Remote) {
node oldAvailableNodes node oldAvailableNodes
if connectionManager.connectionFor(node).isEmpty if connectionManager.connectionFor(node).isEmpty
} { } {
val connectionFactory = () RemoteActorRef(remote.server, gossipingNode, remote.remoteDaemon.path, None) val connectionFactory = () RemoteActorRef(remote.app.provider, remote.server, gossipingNode, remote.remoteDaemon.path, None)
connectionManager.putIfAbsent(node, connectionFactory) // create a new remote connection to the new node connectionManager.putIfAbsent(node, connectionFactory) // create a new remote connection to the new node
oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes
} }

View file

@ -7,7 +7,7 @@ package akka.remote
import scala.collection.mutable import scala.collection.mutable
import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid } import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid }
import akka.actor.Actor._ import akka.actor.Actor._
import akka.actor.ActorSystem import akka.actor.ActorSystemImpl
/** /**
* Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc. * Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc.
@ -58,12 +58,12 @@ object NetworkEventStream {
} }
} }
class NetworkEventStream(val app: ActorSystem) { class NetworkEventStream(app: ActorSystemImpl) {
import NetworkEventStream._ import NetworkEventStream._
// FIXME: check that this supervision is correct // FIXME: check that this supervision is correct
private[akka] val sender = app.provider.actorOf( private[akka] val sender = app.provider.actorOf(app,
Props[Channel].copy(dispatcher = app.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")), Props[Channel].copy(dispatcher = app.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")),
app.systemGuardian, "network-event-sender", systemService = true) app.systemGuardian, "network-event-sender", systemService = true)

View file

@ -15,25 +15,24 @@ import akka.actor.DeploymentConfig._
import akka.serialization.Compression.LZF import akka.serialization.Compression.LZF
import akka.remote.RemoteProtocol._ import akka.remote.RemoteProtocol._
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._ import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
import java.net.InetSocketAddress import java.net.InetSocketAddress
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import akka.serialization.{ JavaSerializer, Serialization, Serializer, Compression } import akka.serialization.{ JavaSerializer, Serialization, Serializer, Compression }
import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher } import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher }
import java.util.concurrent.atomic.AtomicLong
/** /**
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc. * Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class Remote(val app: ActorSystem, val nodename: String) { class Remote(val app: ActorSystemImpl, val nodename: String) {
val log = Logging(app, this) val log = Logging(app, this)
import app._ import app._
import app.config val AC = AkkaConfig
import app.AkkaConfig._ import AC._
// TODO move to AkkaConfig? // TODO move to AkkaConfig?
val shouldCompressData = config.getBool("akka.remote.use-compression", false) val shouldCompressData = config.getBool("akka.remote.use-compression", false)
@ -48,11 +47,12 @@ class Remote(val app: ActorSystem, val nodename: String) {
// FIXME configure computeGridDispatcher to what? // FIXME configure computeGridDispatcher to what?
val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build
// FIXME it is probably better to create another supervisor for handling the children created by handle_*
private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props( private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props(
OneForOneStrategy(List(classOf[Exception]), None, None)), "akka-system-remote-supervisor") // is infinite restart what we want? OneForOneStrategy(List(classOf[Exception]), None, None)), "akka-system-remote-supervisor") // is infinite restart what we want?
private[remote] lazy val remoteDaemon = private[remote] lazy val remoteDaemon =
app.provider.actorOf( app.provider.actorOf(app,
Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)), Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
remoteDaemonSupervisor, remoteDaemonSupervisor,
remoteDaemonServiceName, remoteDaemonServiceName,
@ -81,7 +81,7 @@ class Remote(val app: ActorSystem, val nodename: String) {
} }
def start(): Unit = { def start(): Unit = {
val serverAddress = server.app.address //Force init of server val serverAddress = server.app.root.remoteAddress //Force init of server
val daemonAddress = remoteDaemon.address //Force init of daemon val daemonAddress = remoteDaemon.address //Force init of daemon
log.info("Starting remote server on [{}] and starting remoteDaemon with address [{}]", serverAddress, daemonAddress) log.info("Starting remote server on [{}] and starting remoteDaemon with address [{}]", serverAddress, daemonAddress)
} }
@ -139,10 +139,10 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
} }
val actorPath = ActorPath(remote.app, message.getActorPath) val actorPath = ActorPath(remote.app, message.getActorPath)
val parent = actorPath.parent.ref val parent = app.actorFor(actorPath.parent)
if (parent.isDefined) { if (parent.isDefined) {
app.provider.actorOf(Props(creator = actorFactory), parent.get, actorPath.name) app.provider.actorOf(app, Props(creator = actorFactory), parent.get, actorPath.name)
} else { } else {
log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message) log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message)
} }
@ -180,13 +180,23 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
// } // }
} }
/*
* generate name for temporary actor refs
*/
private val tempNumber = new AtomicLong
def tempName = {
val l = tempNumber.getAndIncrement()
"$_" + Helpers.base64(l)
}
def tempPath = remoteDaemon.path / tempName
// FIXME: handle real remote supervision // FIXME: handle real remote supervision
def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) { def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
new LocalActorRef(app, new LocalActorRef(app,
Props( Props(
context { context {
case f: Function0[_] try { f() } finally { context.self.stop() } case f: Function0[_] try { f() } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) ! payloadFor(message, classOf[Function0[Unit]]) }).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
} }
// FIXME: handle real remote supervision // FIXME: handle real remote supervision
@ -195,7 +205,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
Props( Props(
context { context {
case f: Function0[_] try { sender ! f() } finally { context.self.stop() } case f: Function0[_] try { sender ! f() } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) forward payloadFor(message, classOf[Function0[Any]]) }).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
} }
// FIXME: handle real remote supervision // FIXME: handle real remote supervision
@ -204,7 +214,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
Props( Props(
context { context {
case (fun: Function[_, _], param: Any) try { fun.asInstanceOf[Any Unit].apply(param) } finally { context.self.stop() } case (fun: Function[_, _], param: Any) try { fun.asInstanceOf[Any Unit].apply(param) } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) }).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
} }
// FIXME: handle real remote supervision // FIXME: handle real remote supervision
@ -213,7 +223,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
Props( Props(
context { context {
case (fun: Function[_, _], param: Any) try { sender ! fun.asInstanceOf[Any Any](param) } finally { context.self.stop() } case (fun: Function[_, _], param: Any) try { sender ! fun.asInstanceOf[Any Any](param) } finally { context.self.stop() }
}).copy(dispatcher = computeGridDispatcher), app.guardian, app.guardian.path / app.provider.tempPath, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) }).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
} }
def handleFailover(message: RemoteSystemDaemonMessageProtocol) { def handleFailover(message: RemoteSystemDaemonMessageProtocol) {
@ -230,9 +240,12 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
} }
class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) { class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) {
val provider = remote.app.asInstanceOf[ActorSystemImpl].provider
lazy val sender: ActorRef = lazy val sender: ActorRef =
if (input.hasSender) if (input.hasSender)
remote.app.provider.deserialize( provider.deserialize(
SerializedActorRef(input.getSender.getHost, input.getSender.getPort, input.getSender.getPath)).getOrElse(throw new IllegalStateException("OHNOES")) SerializedActorRef(input.getSender.getHost, input.getSender.getPort, input.getSender.getPath)).getOrElse(throw new IllegalStateException("OHNOES"))
else else
remote.app.deadLetters remote.app.deadLetters
@ -282,7 +295,7 @@ trait RemoteMarshallingOps {
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message. * Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
*/ */
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = { def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = {
val rep = app.provider.serialize(actor) val rep = app.asInstanceOf[ActorSystemImpl].provider.serialize(actor)
ActorRefProtocol.newBuilder.setHost(rep.hostname).setPort(rep.port).setPath(rep.path).build ActorRefProtocol.newBuilder.setHost(rep.hostname).setPort(rep.port).setPath(rep.path).build
} }

View file

@ -29,43 +29,51 @@ import akka.event.EventStream
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class RemoteActorRefProvider( class RemoteActorRefProvider(
val app: ActorSystem,
val AkkaConfig: ActorSystem.AkkaConfig, val AkkaConfig: ActorSystem.AkkaConfig,
val root: ActorPath, val root: ActorPath,
val eventStream: EventStream, val eventStream: EventStream,
val dispatcher: MessageDispatcher, val dispatcher: MessageDispatcher,
val scheduler: Scheduler) extends ActorRefProvider { val scheduler: Scheduler) extends ActorRefProvider {
val log = Logging(app, this) val log = Logging(eventStream, this)
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import akka.dispatch.Promise import akka.dispatch.Promise
val local = new LocalActorRefProvider(app, AkkaConfig, root, eventStream, dispatcher, scheduler) val local = new LocalActorRefProvider(AkkaConfig, root, eventStream, dispatcher, scheduler)
def deathWatch = local.deathWatch def deathWatch = local.deathWatch
def guardian = local.guardian def guardian = local.guardian
def systemGuardian = local.systemGuardian def systemGuardian = local.systemGuardian
def nodename = local.nodename def nodename = local.nodename
def tempName = local.tempName
val remote = new Remote(app, nodename) @volatile
var remote: Remote = _
private val actors = new ConcurrentHashMap[String, AnyRef] private val actors = new ConcurrentHashMap[String, AnyRef]
private val remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote) @volatile
private var remoteDaemonConnectionManager: RemoteConnectionManager = _
def init(app: ActorSystemImpl) {
local.init(app)
remote = new Remote(app, nodename)
remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote)
}
private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = local.theOneWhoWalksTheBubblesOfSpaceTime private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = local.theOneWhoWalksTheBubblesOfSpaceTime
private[akka] def terminationFuture = local.terminationFuture private[akka] def terminationFuture = local.terminationFuture
private[akka] def deployer: Deployer = local.deployer private[akka] def deployer: Deployer = local.deployer
def defaultDispatcher = app.dispatcher def defaultDispatcher = dispatcher
def defaultTimeout = app.AkkaConfig.ActorTimeout def defaultTimeout = AkkaConfig.ActorTimeout
private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
actorOf(props, supervisor, supervisor.path / name, systemService) actorOf(app, props, supervisor, supervisor.path / name, systemService)
private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef =
if (systemService) local.actorOf(props, supervisor, path, systemService) if (systemService) local.actorOf(app, props, supervisor, path, systemService)
else { else {
val name = path.name val name = path.name
val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout? val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout?
@ -90,7 +98,7 @@ class RemoteActorRefProvider(
if (isReplicaNode) { if (isReplicaNode) {
// we are on one of the replica node for this remote actor // we are on one of the replica node for this remote actor
local.actorOf(props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create local.actorOf(app, props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create
} else { } else {
// we are on the single "reference" node uses the remote actors on the replica nodes // we are on the single "reference" node uses the remote actors on the replica nodes
@ -127,17 +135,17 @@ class RemoteActorRefProvider(
val connections = (Map.empty[RemoteAddress, ActorRef] /: remoteAddresses) { (conns, a) val connections = (Map.empty[RemoteAddress, ActorRef] /: remoteAddresses) { (conns, a)
val remoteAddress = RemoteAddress(a.hostname, a.port) val remoteAddress = RemoteAddress(a.hostname, a.port)
conns + (remoteAddress -> RemoteActorRef(remote.server, remoteAddress, path, None)) conns + (remoteAddress -> RemoteActorRef(remote.app.provider, remote.server, remoteAddress, path, None))
} }
val connectionManager = new RemoteConnectionManager(app, remote, connections) val connectionManager = new RemoteConnectionManager(app, remote, connections)
connections.keys foreach { useActorOnNode(_, path.toString, props.creator) } connections.keys foreach { useActorOnNode(app, _, path.toString, props.creator) }
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name) actorOf(app, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
} }
case deploy local.actorOf(props, supervisor, name, systemService) case deploy local.actorOf(app, props, supervisor, name, systemService)
} }
} catch { } catch {
case e: Exception case e: Exception
@ -159,7 +167,7 @@ class RemoteActorRefProvider(
* Copied from LocalActorRefProvider... * Copied from LocalActorRefProvider...
*/ */
// FIXME: implement supervision // FIXME: implement supervision
def actorOf(props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = { def actorOf(app: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = {
if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router") if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router")
new RoutedActorRef(app, props, supervisor, name) new RoutedActorRef(app, props, supervisor, name)
} }
@ -170,6 +178,7 @@ class RemoteActorRefProvider(
case future: Future[_] Some(future.get.asInstanceOf[ActorRef]) case future: Future[_] Some(future.get.asInstanceOf[ActorRef])
} }
// TODO remove me
val optimizeLocal = new AtomicBoolean(true) val optimizeLocal = new AtomicBoolean(true)
def optimizeLocalScoped_?() = optimizeLocal.get def optimizeLocalScoped_?() = optimizeLocal.get
@ -185,19 +194,19 @@ class RemoteActorRefProvider(
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = { private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = {
val remoteAddress = RemoteAddress(actor.hostname, actor.port) val remoteAddress = RemoteAddress(actor.hostname, actor.port)
if (optimizeLocalScoped_? && remoteAddress == app.address) { if (optimizeLocalScoped_? && remoteAddress == root.remoteAddress) {
local.actorFor(ActorPath.split(actor.path)) local.actorFor(ActorPath.split(actor.path))
} else { } else {
log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", app.address, actor.path, remoteAddress) log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", root.remoteAddress, actor.path, remoteAddress)
Some(RemoteActorRef(remote.server, remoteAddress, ActorPath(app, actor.path), None)) //Should it be None here Some(RemoteActorRef(remote.app.provider, remote.server, remoteAddress, root / ActorPath.split(actor.path), None)) //Should it be None here
} }
} }
/** /**
* Using (checking out) actor on a specific node. * Using (checking out) actor on a specific node.
*/ */
def useActorOnNode(remoteAddress: RemoteAddress, actorPath: String, actorFactory: () Actor) { def useActorOnNode(app: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () Actor) {
log.debug("[{}] Instantiating Actor [{}] on node [{}]", app.address, actorPath, remoteAddress) log.debug("[{}] Instantiating Actor [{}] on node [{}]", root, actorPath, remoteAddress)
val actorFactoryBytes = val actorFactoryBytes =
app.serialization.serialize(actorFactory) match { app.serialization.serialize(actorFactory) match {
@ -260,6 +269,7 @@ class RemoteActorRefProvider(
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
private[akka] case class RemoteActorRef private[akka] ( private[akka] case class RemoteActorRef private[akka] (
provider: ActorRefProvider,
remote: RemoteSupport, remote: RemoteSupport,
remoteAddress: RemoteAddress, remoteAddress: RemoteAddress,
path: ActorPath, path: ActorPath,
@ -279,7 +289,7 @@ private[akka] case class RemoteActorRef private[akka] (
def tell(message: Any, sender: ActorRef): Unit = remote.send(message, Option(sender), remoteAddress, this, loader) def tell(message: Any, sender: ActorRef): Unit = remote.send(message, Option(sender), remoteAddress, this, loader)
def ?(message: Any)(implicit timeout: Timeout): Future[Any] = remote.app.provider.ask(message, this, timeout) def ?(message: Any)(implicit timeout: Timeout): Future[Any] = provider.ask(message, this, timeout)
def suspend(): Unit = () def suspend(): Unit = ()
@ -295,7 +305,7 @@ private[akka] case class RemoteActorRef private[akka] (
} }
@throws(classOf[java.io.ObjectStreamException]) @throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = remote.app.provider.serialize(this) private def writeReplace(): AnyRef = provider.serialize(this)
def startsMonitoring(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement def startsMonitoring(actorRef: ActorRef): ActorRef = unsupported //FIXME Implement

View file

@ -149,5 +149,5 @@ class RemoteConnectionManager(
} }
private[remote] def newConnection(remoteAddress: RemoteAddress, actorPath: ActorPath) = private[remote] def newConnection(remoteAddress: RemoteAddress, actorPath: ActorPath) =
RemoteActorRef(remote.server, remoteAddress, actorPath, None) RemoteActorRef(remote.app.provider, remote.server, remoteAddress, actorPath, None)
} }

View file

@ -147,7 +147,8 @@ class ActiveRemoteClient private[akka] (
def sendSecureCookie(connection: ChannelFuture) { def sendSecureCookie(connection: ChannelFuture) {
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT) val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get) if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get)
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(remoteSupport.app.address.hostname).setPort(remoteSupport.app.address.port).build) val addr = remoteSupport.app.root.remoteAddress
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(addr.hostname).setPort(addr.port).build)
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build)) connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
} }
@ -350,8 +351,8 @@ class ActiveRemoteClientHandler(
*/ */
class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps { class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps {
val serverSettings = new RemoteServerSettings(app.config, app.AkkaConfig.DefaultTimeUnit) val serverSettings = new RemoteServerSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit)
val clientSettings = new RemoteClientSettings(app.config, app.AkkaConfig.DefaultTimeUnit) val clientSettings = new RemoteClientSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit)
private val remoteClients = new HashMap[RemoteAddress, RemoteClient] private val remoteClients = new HashMap[RemoteAddress, RemoteClient]
private val clientsLock = new ReadWriteGuard private val clientsLock = new ReadWriteGuard
@ -428,7 +429,7 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
def name = currentServer.get match { def name = currentServer.get match {
case Some(server) server.name case Some(server) server.name
case None "Non-running NettyRemoteServer@" + app.address case None "Non-running NettyRemoteServer@" + app.root.remoteAddress
} }
private val _isRunning = new Switch(false) private val _isRunning = new Switch(false)
@ -459,7 +460,8 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem
class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) { class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) {
val log = Logging(remoteSupport.app, this) val log = Logging(remoteSupport.app, this)
import remoteSupport.serverSettings._ import remoteSupport.serverSettings._
import remoteSupport.app.address
val address = remoteSupport.app.root.remoteAddress
val name = "NettyRemoteServer@" + address val name = "NettyRemoteServer@" + address

View file

@ -5,6 +5,7 @@
package akka.remote package akka.remote
import akka.testkit._ import akka.testkit._
import akka.actor.ActorSystemImpl
abstract class AkkaRemoteSpec extends AkkaSpec with MultiJvmSync { abstract class AkkaRemoteSpec extends AkkaSpec with MultiJvmSync {
@ -12,7 +13,7 @@ abstract class AkkaRemoteSpec extends AkkaSpec with MultiJvmSync {
* Helper function for accessing the underlying remoting. * Helper function for accessing the underlying remoting.
*/ */
def remote: Remote = { def remote: Remote = {
app.provider match { system.asInstanceOf[ActorSystemImpl].provider match {
case r: RemoteActorRefProvider r.remote case r: RemoteActorRefProvider r.remote
case _ throw new Exception("Remoting is not enabled") case _ throw new Exception("Remoting is not enabled")
} }

View file

@ -10,7 +10,7 @@ object DirectRoutedRemoteActorMultiJvmSpec {
class SomeActor extends Actor with Serializable { class SomeActor extends Actor with Serializable {
def receive = { def receive = {
case "identify" sender ! app.provider.nodename case "identify" sender ! system.nodename
} }
} }
} }
@ -47,7 +47,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec {
barrier("start") barrier("start")
val actor = app.actorOf[SomeActor]("service-hello") val actor = system.actorOf[SomeActor]("service-hello")
actor.isInstanceOf[RoutedActorRef] must be(true) actor.isInstanceOf[RoutedActorRef] must be(true)
val result = (actor ? "identify").get val result = (actor ? "identify").get

View file

@ -8,7 +8,7 @@ object NewRemoteActorMultiJvmSpec {
class SomeActor extends Actor with Serializable { class SomeActor extends Actor with Serializable {
def receive = { def receive = {
case "identify" sender ! app.provider.nodename case "identify" sender ! system.nodename
} }
} }
} }
@ -46,7 +46,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec {
barrier("start") barrier("start")
val actor = app.actorOf[SomeActor]("service-hello") val actor = system.actorOf[SomeActor]("service-hello")
val result = (actor ? "identify").get val result = (actor ? "identify").get
result must equal("node1") result must equal("node1")

View file

@ -9,7 +9,7 @@ object RandomRoutedRemoteActorMultiJvmSpec {
val NrOfNodes = 4 val NrOfNodes = 4
class SomeActor extends Actor with Serializable { class SomeActor extends Actor with Serializable {
def receive = { def receive = {
case "hit" sender ! app.provider.nodename case "hit" sender ! system.nodename
case "end" self.stop() case "end" self.stop()
} }
} }
@ -70,7 +70,7 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec {
remote.start() remote.start()
barrier("start") barrier("start")
val actor = app.actorOf[SomeActor]("service-hello") val actor = system.actorOf[SomeActor]("service-hello")
actor.isInstanceOf[RoutedActorRef] must be(true) actor.isInstanceOf[RoutedActorRef] must be(true)
val connectionCount = NrOfNodes - 1 val connectionCount = NrOfNodes - 1

View file

@ -9,7 +9,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec {
val NrOfNodes = 4 val NrOfNodes = 4
class SomeActor extends Actor with Serializable { class SomeActor extends Actor with Serializable {
def receive = { def receive = {
case "hit" sender ! app.provider.nodename case "hit" sender ! system.nodename
case "end" self.stop() case "end" self.stop()
} }
} }
@ -70,7 +70,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec {
remote.start() remote.start()
barrier("start") barrier("start")
val actor = app.actorOf[SomeActor]("service-hello") val actor = system.actorOf[SomeActor]("service-hello")
actor.isInstanceOf[RoutedActorRef] must be(true) actor.isInstanceOf[RoutedActorRef] must be(true)
val connectionCount = NrOfNodes - 1 val connectionCount = NrOfNodes - 1

View file

@ -9,7 +9,7 @@ object ScatterGatherRoutedRemoteActorMultiJvmSpec {
val NrOfNodes = 4 val NrOfNodes = 4
class SomeActor extends Actor with Serializable { class SomeActor extends Actor with Serializable {
def receive = { def receive = {
case "hit" sender ! app.provider.nodename case "hit" sender ! system.nodename
case "end" self.stop() case "end" self.stop()
} }
} }
@ -70,7 +70,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec {
remote.start() remote.start()
barrier("start") barrier("start")
val actor = app.actorOf[SomeActor]("service-hello") val actor = system.actorOf[SomeActor]("service-hello")
actor.isInstanceOf[RoutedActorRef] must be(true) actor.isInstanceOf[RoutedActorRef] must be(true)
actor.asInstanceOf[RoutedActorRef].router.isInstanceOf[ScatterGatherFirstCompletedRouter] must be(true) actor.asInstanceOf[RoutedActorRef].router.isInstanceOf[ScatterGatherFirstCompletedRouter] must be(true)

View file

@ -77,7 +77,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
case Taken(`chopstickToWaitFor`) case Taken(`chopstickToWaitFor`)
println("%s has picked up %s and %s, and starts to eat", name, left.address, right.address) println("%s has picked up %s and %s, and starts to eat", name, left.address, right.address)
become(eating) become(eating)
app.scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS) system.scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS)
case Busy(chopstick) case Busy(chopstick)
become(thinking) become(thinking)
@ -106,7 +106,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
left ! Put(self) left ! Put(self)
right ! Put(self) right ! Put(self)
println("%s puts down his chopsticks and starts to think", name) println("%s puts down his chopsticks and starts to think", name)
app.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) system.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
} }
//All hakkers start in a non-eating state //All hakkers start in a non-eating state
@ -114,7 +114,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor {
case Think case Think
println("%s starts to think", name) println("%s starts to think", name)
become(thinking) become(thinking)
app.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) system.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS)
} }
} }

View file

@ -32,7 +32,7 @@ case class TakenBy(hakker: ActorRef)
class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] { class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] {
// A chopstick begins its existence as available and taken by no one // A chopstick begins its existence as available and taken by no one
startWith(Available, TakenBy(app.deadLetters)) startWith(Available, TakenBy(system.deadLetters))
// When a chopstick is available, it can be taken by a some hakker // When a chopstick is available, it can be taken by a some hakker
when(Available) { when(Available) {
@ -47,7 +47,7 @@ class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] {
case Event(Take, currentState) case Event(Take, currentState)
stay replying Busy(self) stay replying Busy(self)
case Event(Put, TakenBy(hakker)) if sender == hakker case Event(Put, TakenBy(hakker)) if sender == hakker
goto(Available) using TakenBy(app.deadLetters) goto(Available) using TakenBy(system.deadLetters)
} }
// Initialze the chopstick // Initialze the chopstick

View file

@ -9,7 +9,7 @@ public class EitherOrElseExample {
System.out.println("EitherOrElse example"); System.out.println("EitherOrElse example");
System.out.println(); System.out.println();
ActorSystem application = new ActorSystem("UntypedTransactorExample"); ActorSystem application = ActorSystem.create("UntypedTransactorExample");
final Ref<Integer> left = new Ref<Integer>(100); final Ref<Integer> left = new Ref<Integer>(100);
final Ref<Integer> right = new Ref<Integer>(100); final Ref<Integer> right = new Ref<Integer>(100);

View file

@ -10,7 +10,7 @@ public class RetryExample {
System.out.println("Retry example"); System.out.println("Retry example");
System.out.println(); System.out.println();
ActorSystem application = new ActorSystem("RetryExample"); ActorSystem application = ActorSystem.create("RetryExample");
final Ref<Double> account1 = new Ref<Double>(100.0); final Ref<Double> account1 = new Ref<Double>(100.0);
final Ref<Double> account2 = new Ref<Double>(100.0); final Ref<Double> account2 = new Ref<Double>(100.0);

View file

@ -12,7 +12,7 @@ public class UntypedCoordinatedExample {
System.out.println("Untyped transactor example"); System.out.println("Untyped transactor example");
System.out.println(); System.out.println();
ActorSystem application = new ActorSystem("UntypedCoordinatedExample"); ActorSystem application = ActorSystem.create("UntypedCoordinatedExample");
ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class)); ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class));
ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class)); ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class));

View file

@ -11,7 +11,7 @@ public class UntypedTransactorExample {
System.out.println("Untyped transactor example"); System.out.println("Untyped transactor example");
System.out.println(); System.out.println();
ActorSystem application = new ActorSystem("UntypedTransactorExample"); ActorSystem application = ActorSystem.create("UntypedTransactorExample");
ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCounter.class)); ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCounter.class));
ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCounter.class)); ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCounter.class));

View file

@ -30,7 +30,7 @@ import scala.collection.JavaConverters;
import scala.collection.Seq; import scala.collection.Seq;
public class UntypedCoordinatedIncrementTest { public class UntypedCoordinatedIncrementTest {
ActorSystem application = new ActorSystem("UntypedCoordinatedIncrementTest"); ActorSystem application = ActorSystem.create("UntypedCoordinatedIncrementTest");
List<ActorRef> counters; List<ActorRef> counters;
ActorRef failer; ActorRef failer;

View file

@ -27,7 +27,7 @@ import scala.collection.JavaConverters;
import scala.collection.Seq; import scala.collection.Seq;
public class UntypedTransactorTest { public class UntypedTransactorTest {
ActorSystem application = new ActorSystem("UntypedTransactorTest"); ActorSystem application = ActorSystem.create("UntypedTransactorTest");
List<ActorRef> counters; List<ActorRef> counters;
ActorRef failer; ActorRef failer;

View file

@ -16,7 +16,7 @@ class ConfigSpec extends WordSpec with MustMatchers {
"The default configuration file (i.e. akka-reference.conf)" should { "The default configuration file (i.e. akka-reference.conf)" should {
"contain all configuration properties for akka-stm that are used in code with their correct defaults" in { "contain all configuration properties for akka-stm that are used in code with their correct defaults" in {
val config = ActorSystem("ConfigSpec").config val config = ActorSystem("ConfigSpec").AkkaConfig.config
import config._ import config._

View file

@ -61,9 +61,9 @@ class CoordinatedIncrementSpec extends AkkaSpec with BeforeAndAfterAll {
val numCounters = 4 val numCounters = 4
def actorOfs = { def actorOfs = {
def createCounter(i: Int) = app.actorOf(Props(new Counter("counter" + i))) def createCounter(i: Int) = system.actorOf(Props(new Counter("counter" + i)))
val counters = (1 to numCounters) map createCounter val counters = (1 to numCounters) map createCounter
val failer = app.actorOf(Props(new Failer)) val failer = system.actorOf(Props(new Failer))
(counters, failer) (counters, failer)
} }

View file

@ -104,9 +104,9 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll {
val numCounters = 2 val numCounters = 2
def actorOfs = { def actorOfs = {
def createCounter(i: Int) = app.actorOf(Props(new FickleCounter("counter" + i))) def createCounter(i: Int) = system.actorOf(Props(new FickleCounter("counter" + i)))
val counters = (1 to numCounters) map createCounter val counters = (1 to numCounters) map createCounter
val coordinator = app.actorOf(Props(new Coordinator("coordinator"))) val coordinator = system.actorOf(Props(new Coordinator("coordinator")))
(counters, coordinator) (counters, coordinator)
} }
@ -116,7 +116,7 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll {
EventFilter[ExpectedFailureException](), EventFilter[ExpectedFailureException](),
EventFilter[CoordinatedTransactionException](), EventFilter[CoordinatedTransactionException](),
EventFilter[ActorTimeoutException]()) EventFilter[ActorTimeoutException]())
app.eventStream.publish(Mute(ignoreExceptions)) system.eventStream.publish(Mute(ignoreExceptions))
val (counters, coordinator) = actorOfs val (counters, coordinator) = actorOfs
val latch = new CountDownLatch(1) val latch = new CountDownLatch(1)
coordinator ! FriendlyIncrement(counters, latch) coordinator ! FriendlyIncrement(counters, latch)

View file

@ -84,9 +84,9 @@ class TransactorSpec extends AkkaSpec {
val numCounters = 3 val numCounters = 3
def createTransactors = { def createTransactors = {
def createCounter(i: Int) = app.actorOf(Props(new Counter("counter" + i))) def createCounter(i: Int) = system.actorOf(Props(new Counter("counter" + i)))
val counters = (1 to numCounters) map createCounter val counters = (1 to numCounters) map createCounter
val failer = app.actorOf(Props(new Failer)) val failer = system.actorOf(Props(new Failer))
(counters, failer) (counters, failer)
} }
@ -124,7 +124,7 @@ class TransactorSpec extends AkkaSpec {
"Transactor" should { "Transactor" should {
"be usable without overriding normally" in { "be usable without overriding normally" in {
val transactor = app.actorOf(Props(new Setter)) val transactor = system.actorOf(Props(new Setter))
val ref = Ref(0) val ref = Ref(0)
val latch = TestLatch(1) val latch = TestLatch(1)
transactor ! Set(ref, 5, latch) transactor ! Set(ref, 5, latch)

View file

@ -22,7 +22,7 @@ import akka.event.EventStream
* @since 1.1 * @since 1.1
*/ */
class TestActorRef[T <: Actor]( class TestActorRef[T <: Actor](
_app: ActorSystem, _app: ActorSystemImpl,
_deadLetterMailbox: Mailbox, _deadLetterMailbox: Mailbox,
_eventStream: EventStream, _eventStream: EventStream,
_scheduler: Scheduler, _scheduler: Scheduler,
@ -63,10 +63,12 @@ object TestActorRef {
def apply[T <: Actor](props: Props)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, randomName) def apply[T <: Actor](props: Props)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, randomName)
def apply[T <: Actor](props: Props, name: String)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, app.guardian, name) def apply[T <: Actor](props: Props, name: String)(implicit app: ActorSystem): TestActorRef[T] =
apply[T](props, app.asInstanceOf[ActorSystemImpl].guardian, name)
def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit app: ActorSystem): TestActorRef[T] = { def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit app: ActorSystem): TestActorRef[T] = {
new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, props, supervisor, name) val impl = app.asInstanceOf[ActorSystemImpl]
new TestActorRef(impl, impl.deadLetterMailbox, app.eventStream, app.scheduler, props, supervisor, name)
} }
def apply[T <: Actor](implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](randomName) def apply[T <: Actor](implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](randomName)

View file

@ -37,7 +37,7 @@ import akka.event.EventStream
* @since 1.2 * @since 1.2
*/ */
class TestFSMRef[S, D, T <: Actor]( class TestFSMRef[S, D, T <: Actor](
app: ActorSystem, app: ActorSystemImpl,
_deadLetterMailbox: Mailbox, _deadLetterMailbox: Mailbox,
_eventStream: EventStream, _eventStream: EventStream,
_scheduler: Scheduler, _scheduler: Scheduler,
@ -89,9 +89,13 @@ class TestFSMRef[S, D, T <: Actor](
object TestFSMRef { object TestFSMRef {
def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = {
new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () factory), app.guardian, TestActorRef.randomName) val impl = app.asInstanceOf[ActorSystemImpl]
new TestFSMRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(creator = () factory), impl.guardian, TestActorRef.randomName)
}
def apply[S, D, T <: Actor](factory: T, name: String)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = def apply[S, D, T <: Actor](factory: T, name: String)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = {
new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () factory), app.guardian, name) val impl = app.asInstanceOf[ActorSystemImpl]
new TestFSMRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(creator = () factory), impl.guardian, name)
}
} }

View file

@ -80,7 +80,7 @@ class TestKit(_app: ActorSystem) {
import TestActor.{ Message, RealMessage, NullMessage } import TestActor.{ Message, RealMessage, NullMessage }
implicit val app = _app implicit val system = _app
private val queue = new LinkedBlockingDeque[Message]() private val queue = new LinkedBlockingDeque[Message]()
private[akka] var lastMessage: Message = NullMessage private[akka] var lastMessage: Message = NullMessage
@ -91,9 +91,12 @@ class TestKit(_app: ActorSystem) {
* ActorRef of the test actor. Access is provided to enable e.g. * ActorRef of the test actor. Access is provided to enable e.g.
* registration as message target. * registration as message target.
*/ */
val testActor: ActorRef = app.systemActorOf(Props(new TestActor(queue)) val testActor: ActorRef = {
.copy(dispatcher = new CallingThreadDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler)), val impl = system.asInstanceOf[ActorSystemImpl]
impl.systemActorOf(Props(new TestActor(queue))
.copy(dispatcher = new CallingThreadDispatcher(impl.deadLetterMailbox, impl.eventStream, impl.scheduler)),
"testActor" + TestKit.testActorId.incrementAndGet) "testActor" + TestKit.testActorId.incrementAndGet)
}
private var end: Duration = Duration.Undefined private var end: Duration = Duration.Undefined
@ -124,7 +127,7 @@ class TestKit(_app: ActorSystem) {
* block or missing that it returns the properly dilated default for this * block or missing that it returns the properly dilated default for this
* case from AkkaConfig (key "akka.test.single-expect-default"). * case from AkkaConfig (key "akka.test.single-expect-default").
*/ */
def remaining: Duration = if (end == Duration.Undefined) app.AkkaConfig.SingleExpectDefaultTimeout.dilated else end - now def remaining: Duration = if (end == Duration.Undefined) system.AkkaConfig.SingleExpectDefaultTimeout.dilated else end - now
/** /**
* Query queue status. * Query queue status.

View file

@ -6,7 +6,7 @@ package akka.testkit
import akka.config.Configuration import akka.config.Configuration
import org.scalatest.{ WordSpec, BeforeAndAfterAll } import org.scalatest.{ WordSpec, BeforeAndAfterAll }
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.actor.ActorSystem import akka.actor.{ ActorSystem, ActorSystemImpl }
import akka.actor.{ Actor, ActorRef, Props } import akka.actor.{ Actor, ActorRef, Props }
import akka.dispatch.MessageDispatcher import akka.dispatch.MessageDispatcher
import akka.event.{ Logging, LoggingAdapter } import akka.event.{ Logging, LoggingAdapter }
@ -16,16 +16,16 @@ import akka.dispatch.FutureTimeoutException
abstract class AkkaSpec(_application: ActorSystem = ActorSystem()) abstract class AkkaSpec(_application: ActorSystem = ActorSystem())
extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll { extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll {
val log: LoggingAdapter = Logging(app.eventStream, this) val log: LoggingAdapter = Logging(system, this)
final override def beforeAll { final override def beforeAll {
atStartup() atStartup()
} }
final override def afterAll { final override def afterAll {
app.stop() system.stop()
try app.terminationFuture.await(5 seconds) catch { try system.asInstanceOf[ActorSystemImpl].terminationFuture.await(5 seconds) catch {
case _: FutureTimeoutException app.log.warning("failed to stop within 5 seconds") case _: FutureTimeoutException system.log.warning("failed to stop within 5 seconds")
} }
atTermination() atTermination()
} }
@ -34,9 +34,9 @@ abstract class AkkaSpec(_application: ActorSystem = ActorSystem())
protected def atTermination() {} protected def atTermination() {}
def this(config: Configuration) = this(new ActorSystem(getClass.getSimpleName, ActorSystem.defaultConfig ++ config)) def this(config: Configuration) = this(ActorSystem(getClass.getSimpleName, ActorSystem.defaultConfig ++ config))
def actorOf(props: Props): ActorRef = app.actorOf(props) def actorOf(props: Props): ActorRef = system.actorOf(props)
def actorOf[T <: Actor](clazz: Class[T]): ActorRef = actorOf(Props(clazz)) def actorOf[T <: Actor](clazz: Class[T]): ActorRef = actorOf(Props(clazz))

View file

@ -170,8 +170,8 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
counter = 2 counter = 2
val boss = TestActorRef(Props(new TActor { val boss = TestActorRef(Props(new TActor {
val impl = system.asInstanceOf[ActorSystemImpl]
val ref = new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(new TActor { val ref = new TestActorRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(new TActor {
def receiveT = { case _ } def receiveT = { case _ }
override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 } override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 }
override def postRestart(reason: Throwable) { counter -= 1 } override def postRestart(reason: Throwable) { counter -= 1 }

View file

@ -15,7 +15,7 @@ class TestTimeSpec extends AkkaSpec(Configuration("akka.test.timefactor" -> 2.0)
val now = System.nanoTime val now = System.nanoTime
intercept[AssertionError] { probe.awaitCond(false, Duration("1 second")) } intercept[AssertionError] { probe.awaitCond(false, Duration("1 second")) }
val diff = System.nanoTime - now val diff = System.nanoTime - now
val target = (1000000000l * app.AkkaConfig.TestTimeFactor).toLong val target = (1000000000l * system.AkkaConfig.TestTimeFactor).toLong
diff must be > (target - 300000000l) diff must be > (target - 300000000l)
diff must be < (target + 1000000000l) diff must be < (target + 1000000000l)
} }