From 648661c5480c4e2aceff78aed48dd86eeeabf566 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 16 Nov 2011 17:18:36 +0100 Subject: [PATCH] clean up initialization of ActorSystem, fixes #1050 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - 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 --- .../src/test/java/akka/actor/JavaAPI.java | 2 +- .../java/akka/dispatch/JavaFutureTests.java | 2 +- .../test/scala/akka/actor/ActorRefSpec.scala | 6 +- .../scala/akka/actor/ActorTimeoutSpec.scala | 4 +- .../test/scala/akka/actor/ClusterSpec.scala | 2 +- .../test/scala/akka/actor/DeployerSpec.scala | 2 +- .../test/scala/akka/actor/FSMActorSpec.scala | 12 +- .../scala/akka/actor/ForwardActorSpec.scala | 4 +- .../actor/LocalActorRefProviderSpec.scala | 5 +- .../scala/akka/actor/LoggingReceiveSpec.scala | 16 +-- .../akka/actor/RestartStrategySpec.scala | 2 +- .../test/scala/akka/actor/SchedulerSpec.scala | 14 +-- .../scala/akka/actor/SupervisorMiscSpec.scala | 8 +- .../scala/akka/actor/SupervisorSpec.scala | 2 +- .../scala/akka/actor/TypedActorSpec.scala | 28 ++--- .../akka/actor/dispatch/ActorModelSpec.scala | 32 +++--- .../dispatch/BalancingDispatcherSpec.scala | 2 +- .../actor/dispatch/DispatcherActorSpec.scala | 12 +- .../akka/actor/dispatch/DispatchersSpec.scala | 3 +- .../akka/actor/dispatch/PinnedActorSpec.scala | 4 +- .../test/scala/akka/config/ConfigSpec.scala | 2 +- .../scala/akka/dataflow/Future2Actor.scala | 2 +- .../test/scala/akka/dispatch/FutureSpec.scala | 2 +- .../akka/dispatch/MailboxConfigSpec.scala | 2 +- .../dispatch/PriorityDispatcherSpec.scala | 4 +- .../scala/akka/event/EventStreamSpec.scala | 14 ++- .../TellLatencyPerformanceSpec.scala | 14 +-- .../TellThroughputPerformanceSpec.scala | 8 +- .../TradingLatencyPerformanceSpec.scala | 6 +- .../TradingThroughputPerformanceSpec.scala | 6 +- .../workbench/PerformanceSpec.scala | 4 +- .../akka/performance/workbench/Report.scala | 4 +- .../scala/akka/routing/ActorPoolSpec.scala | 6 +- .../routing/ConfiguredLocalRoutingSpec.scala | 38 ++++--- .../test/scala/akka/routing/RoutingSpec.scala | 44 ++++---- .../akka/serialization/SerializeSpec.scala | 9 +- .../CallingThreadDispatcherModelSpec.scala | 2 +- .../src/main/scala/akka/actor/Actor.scala | 8 +- .../src/main/scala/akka/actor/ActorCell.scala | 6 +- .../src/main/scala/akka/actor/ActorPath.scala | 22 ++-- .../src/main/scala/akka/actor/ActorRef.scala | 16 +-- .../scala/akka/actor/ActorRefProvider.scala | 88 +++++++++------ .../main/scala/akka/actor/ActorSystem.scala | 105 +++++++++++++----- .../src/main/scala/akka/actor/FSM.scala | 6 +- akka-actor/src/main/scala/akka/actor/IO.scala | 2 +- .../main/scala/akka/actor/TypedActor.scala | 2 +- .../src/main/scala/akka/actor/package.scala | 4 - .../scala/akka/cluster/ClusterInterface.scala | 2 +- .../main/scala/akka/event/EventStream.scala | 11 +- .../src/main/scala/akka/event/Logging.scala | 16 +-- .../akka/serialization/Serialization.scala | 10 +- akka-docs/scala/code/ActorDocSpec.scala | 10 +- .../akka/remote/AccrualFailureDetector.scala | 4 +- .../src/main/scala/akka/remote/Gossiper.scala | 4 +- .../akka/remote/NetworkEventStream.scala | 6 +- .../src/main/scala/akka/remote/Remote.scala | 43 ++++--- .../akka/remote/RemoteActorRefProvider.scala | 58 ++++++---- .../akka/remote/RemoteConnectionManager.scala | 2 +- .../remote/netty/NettyRemoteSupport.scala | 12 +- .../scala/akka/remote/AkkaRemoteSpec.scala | 3 +- .../DirectRoutedRemoteActorMultiJvmSpec.scala | 4 +- .../NewRemoteActorMultiJvmSpec.scala | 4 +- .../RandomRoutedRemoteActorMultiJvmSpec.scala | 4 +- ...ndRobinRoutedRemoteActorMultiJvmSpec.scala | 4 +- ...rGatherRoutedRemoteActorMultiJvmSpec.scala | 4 +- .../main/scala/DiningHakkersOnBecome.scala | 6 +- .../src/main/scala/DiningHakkersOnFsm.scala | 4 +- .../akka/stm/example/EitherOrElseExample.java | 2 +- .../java/akka/stm/example/RetryExample.java | 2 +- .../example/UntypedCoordinatedExample.java | 2 +- .../example/UntypedTransactorExample.java | 2 +- .../test/UntypedCoordinatedIncrementTest.java | 2 +- .../test/UntypedTransactorTest.java | 2 +- .../src/test/scala/config/ConfigSpec.scala | 2 +- .../transactor/CoordinatedIncrementSpec.scala | 4 +- .../scala/transactor/FickleFriendsSpec.scala | 6 +- .../scala/transactor/TransactorSpec.scala | 6 +- .../scala/akka/testkit/TestActorRef.scala | 8 +- .../main/scala/akka/testkit/TestFSMRef.scala | 14 ++- .../src/main/scala/akka/testkit/TestKit.scala | 13 ++- .../test/scala/akka/testkit/AkkaSpec.scala | 14 +-- .../scala/akka/testkit/TestActorRefSpec.scala | 4 +- .../scala/akka/testkit/TestTimeSpec.scala | 2 +- 83 files changed, 494 insertions(+), 390 deletions(-) diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java b/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java index cfe2a7c63c..710370e8f9 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java @@ -9,7 +9,7 @@ import static org.junit.Assert.*; public class JavaAPI { - private ActorSystem app = new ActorSystem(); + private ActorSystem app = ActorSystem.create(); @Test void mustBeAbleToCreateActorRefFromClass() { ActorRef ref = app.actorOf(JavaAPITestActor.class); diff --git a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java index 7c7bc52876..7f552c46fc 100644 --- a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java +++ b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java @@ -19,7 +19,7 @@ import scala.Right; public class JavaFutureTests { - private final ActorSystem app = new ActorSystem(); + private final ActorSystem app = ActorSystem.create(); private final Timeout t = app.AkkaConfig().ActorTimeout(); private final FutureFactory ff = new FutureFactory(app.dispatcher(), t); diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index 4865654da0..50527ef4b5 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -247,7 +247,7 @@ class ActorRefSpec extends AkkaSpec { out.flush out.close - Serialization.app.withValue(app) { + Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) { val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val readA = in.readObject @@ -284,14 +284,14 @@ class ActorRefSpec extends AkkaSpec { val baos = new ByteArrayOutputStream(8192 * 32) 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.flush out.close - Serialization.app.withValue(app) { + Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) { val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) (intercept[java.lang.IllegalStateException] { in.readObject diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala index 46a345a7c2..5d8333a6cd 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala @@ -17,8 +17,8 @@ class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll { } }, timeout = t)) - val defaultTimeout = app.AkkaConfig.ActorTimeout.duration - val testTimeout = if (app.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis + val defaultTimeout = system.AkkaConfig.ActorTimeout.duration + val testTimeout = if (system.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis "An Actor-based Future" must { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ClusterSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ClusterSpec.scala index 2d2bed7342..09afd0b527 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ClusterSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ClusterSpec.scala @@ -9,7 +9,7 @@ class ClusterSpec extends AkkaSpec { "be able to parse 'akka.actor.cluster._' config elements" in { // TODO: make it use its own special config? - val config = app.config + val config = system.AkkaConfig.config import config._ //akka.cluster diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala index 0604f9e01f..bbcc84eb46 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala @@ -14,7 +14,7 @@ class DeployerSpec extends AkkaSpec { "A Deployer" must { "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 equal(Some( diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index 2ba83a9971..2302e83438 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -167,12 +167,12 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true } }) filterException[Logging.EventHandlerException] { - app.eventStream.subscribe(testActor, classOf[Logging.Error]) + system.eventStream.subscribe(testActor, classOf[Logging.Error]) fsm ! "go" expectMsgPF(1 second, hint = "Next state 2 does not exist") { 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 } }) - app.eventStream.subscribe(testActor, classOf[Logging.Debug]) + system.eventStream.subscribe(testActor, classOf[Logging.Debug]) fsm ! "go" 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, "transition 1 -> 2")) fsm ! "stop" 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) expectNoMsg(1 second) - app.eventStream.unsubscribe(testActor) + system.eventStream.unsubscribe(testActor) } } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala index 9b6a7d1d86..c230ecc347 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala @@ -37,14 +37,14 @@ class ForwardActorSpec extends AkkaSpec { val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage ⇒ latch.countDown() } }) - val chain = createForwardingChain(app) + val chain = createForwardingChain(system) chain.tell(ExpectedMessage, replyTo) latch.await(Duration(5, "s")) must be === true } "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 } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala index 4b93d37d2c..707c425295 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala @@ -13,14 +13,15 @@ class LocalActorRefProviderSpec extends AkkaSpec { "An LocalActorRefProvider" must { "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) (0 until 100) foreach { i ⇒ // 100 concurrent runs val address = "new-actor" + i 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) } } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala index b1f23e60e9..83e923c5f0 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala @@ -53,7 +53,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd "decorate a Receive" in { new TestKit(appLogging) { - app.eventStream.subscribe(testActor, classOf[Logging.Debug]) + system.eventStream.subscribe(testActor, classOf[Logging.Debug]) val r: Actor.Receive = { case null ⇒ } @@ -66,8 +66,8 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd "be added on Actor if requested" in { new TestKit(appLogging) with ImplicitSender { ignoreMute(this) - app.eventStream.subscribe(testActor, classOf[Logging.Debug]) - app.eventStream.subscribe(testActor, classOf[Logging.Error]) + system.eventStream.subscribe(testActor, classOf[Logging.Debug]) + system.eventStream.subscribe(testActor, classOf[Logging.Error]) val actor = TestActorRef(new Actor { def receive = loggable(this) { case _ ⇒ sender ! "x" @@ -95,7 +95,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd "not duplicate logging" in { new TestKit(appLogging) with ImplicitSender { - app.eventStream.subscribe(testActor, classOf[Logging.Debug]) + system.eventStream.subscribe(testActor, classOf[Logging.Debug]) val actor = TestActorRef(new Actor { def receive = loggable(this)(loggable(this) { case _ ⇒ sender ! "x" @@ -115,7 +115,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd "log AutoReceiveMessages if requested" in { new TestKit(appAuto) { - app.eventStream.subscribe(testActor, classOf[Logging.Debug]) + system.eventStream.subscribe(testActor, classOf[Logging.Debug]) val actor = TestActorRef(new Actor { def receive = { case _ ⇒ @@ -137,10 +137,10 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd val s = ref.toString s.contains("MainBusReaper") || s.contains("Supervisor") } - app.eventStream.subscribe(testActor, classOf[Logging.Debug]) - app.eventStream.subscribe(testActor, classOf[Logging.Error]) + system.eventStream.subscribe(testActor, classOf[Logging.Debug]) + system.eventStream.subscribe(testActor, classOf[Logging.Error]) 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 supervisorSet = receiveWhile(messages = 2) { diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index f967e9c5f3..95255c1c8a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -16,7 +16,7 @@ import akka.testkit.AkkaSpec class RestartStrategySpec extends AkkaSpec { override def atStartup { - app.eventStream.publish(Mute(EventFilter[Exception]("Crashing..."))) + system.eventStream.publish(Mute(EventFilter[Exception]("Crashing..."))) } object Ping diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index ca7863f00e..56a3346b72 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -28,14 +28,14 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { def receive = { case Tick ⇒ countDownLatch.countDown() } }) // 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 assert(countDownLatch.await(1, TimeUnit.SECONDS)) 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 assert(countDownLatch2.await(2, TimeUnit.SECONDS)) @@ -49,8 +49,8 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { }) // run every 50 millisec - collectCancellable(app.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS)) - collectCancellable(app.scheduler.scheduleOnce(() ⇒ countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS)) + collectCancellable(system.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS)) + collectCancellable(system.scheduler.scheduleOnce(() ⇒ countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS)) // after 1 second the wait should fail assert(countDownLatch.await(2, TimeUnit.SECONDS) == false) @@ -86,7 +86,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { }) (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() } @@ -114,10 +114,10 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { }) 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 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)) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index 99068ed76e..d8ae9d7444 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -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 actor2 ! Kill diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index adc6e25574..e0f488e3bc 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -121,7 +121,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } override def atStartup() { - app.eventStream.publish(Mute(EventFilter[RuntimeException](ExceptionMessage))) + system.eventStream.publish(Mute(EventFilter[RuntimeException](ExceptionMessage))) } override def beforeEach() = { diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index b708f92dcb..02c44f2216 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -147,18 +147,18 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte newFooBar(Props().withTimeout(Timeout(d))) 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 = - 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 { "be able to instantiate" in { val t = newFooBar - app.typedActor.isTypedActor(t) must be(true) + system.typedActor.isTypedActor(t) must be(true) mustStop(t) } @@ -168,7 +168,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte } "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 { @@ -187,7 +187,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte "be able to call toString" in { val t = newFooBar - t.toString must be(app.typedActor.getActorRefFor(t).toString) + t.toString must be(system.typedActor.getActorRefFor(t).toString) mustStop(t) } @@ -200,7 +200,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte "be able to call hashCode" in { val t = newFooBar - t.hashCode must be(app.typedActor.getActorRefFor(t).hashCode) + t.hashCode must be(system.typedActor.getActorRefFor(t).hashCode) mustStop(t) } @@ -295,7 +295,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte } "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 f2 = t.futurePigdog(0) 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 { - val t = app.typedActorOf[Stackable1 with Stackable2, StackedImpl]() + val t = system.typedActorOf[Stackable1 with Stackable2, StackedImpl]() t.stackable1 must be("foo") t.stackable2 must be("bar") mustStop(t) @@ -314,7 +314,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte "be able to use work-stealing dispatcher" in { val props = Props( timeout = Timeout(6600), - dispatcher = app.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher") + dispatcher = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher") .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .setCorePoolSize(60) .setMaxPoolSize(60) @@ -332,7 +332,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte "be able to serialize and deserialize invocations" in { 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 out = new ObjectOutputStream(baos) @@ -341,7 +341,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte 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] 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 { import java.io._ 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 out = new ObjectOutputStream(baos) @@ -360,7 +360,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte 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] mNew.method must be(m.method) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index 6f50112362..a1b991add1 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -148,14 +148,14 @@ object ActorModelSpec { def assertDispatcher(dispatcher: MessageDispatcherInterceptor)( 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 try { await(deadline)(starts == dispatcher.starts.get) await(deadline)(stops == dispatcher.stops.get) } catch { 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)) throw e } @@ -181,7 +181,7 @@ object ActorModelSpec { unregisters: Long = 0, msgsReceived: Long = 0, msgsProcessed: Long = 0, - restarts: Long = 0)(implicit app: ActorSystem) { + restarts: Long = 0)(implicit system: ActorSystem) { assertRef(actorRef, dispatcher)( suspensions, resumes, @@ -199,7 +199,7 @@ object ActorModelSpec { unregisters: Long = statsFor(actorRef).unregisters.get(), msgsReceived: Long = statsFor(actorRef).msgsReceived.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 deadline = System.currentTimeMillis + 1000 try { @@ -212,7 +212,7 @@ object ActorModelSpec { await(deadline)(stats.restarts.get() == restarts) } catch { 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 + ",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts)) throw e @@ -235,7 +235,7 @@ abstract class ActorModelSpec extends AkkaSpec { 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 dispatcherType: String @@ -318,7 +318,7 @@ abstract class ActorModelSpec extends AkkaSpec { try { f } 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._ def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒ - new Dispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", app.AkkaConfig.DispatcherThroughput, - app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType, - config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, - ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor] + new Dispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", system.AkkaConfig.DispatcherThroughput, + system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType, + config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, + ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor] def dispatcherType = "Dispatcher" @@ -458,14 +458,14 @@ class BalancingDispatcherModelSpec extends ActorModelSpec { import ActorModelSpec._ def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒ - new BalancingDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", 1, // TODO check why 1 here? (came from old test) - app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType, - config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, - ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor] + new BalancingDispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", 1, // TODO check why 1 here? (came from old test) + system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType, + config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, + ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor] def dispatcherType = "Balancing Dispatcher" - override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = app.dispatcher + override def wavesSupervisorDispatcher(dispatcher: MessageDispatcher) = system.dispatcher "A " + dispatcherType must { "process messages in parallel" in { diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala index c30db1d5bc..b6ef8468f7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala @@ -8,7 +8,7 @@ import akka.testkit.AkkaSpec @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) 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() diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala index 2ce2171438..d2bd4e9c2d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala @@ -33,22 +33,22 @@ class DispatcherActorSpec extends AkkaSpec { "A Dispatcher and an Actor" must { "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" assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS)) actor.stop() } "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] assert("World" === result.get) actor.stop() } "respect the throughput setting" in { - val throughputDispatcher = app.dispatcherFactory. - newDispatcher("THROUGHPUT", 101, 0, app.dispatcherFactory.MailboxType). + val throughputDispatcher = system.dispatcherFactory. + newDispatcher("THROUGHPUT", 101, 0, system.dispatcherFactory.MailboxType). setCorePoolSize(1). build @@ -76,8 +76,8 @@ class DispatcherActorSpec extends AkkaSpec { "respect throughput deadline" in { val deadlineMs = 100 - val throughputDispatcher = app.dispatcherFactory. - newDispatcher("THROUGHPUT", 2, deadlineMs, app.dispatcherFactory.MailboxType). + val throughputDispatcher = system.dispatcherFactory. + newDispatcher("THROUGHPUT", 2, deadlineMs, system.dispatcherFactory.MailboxType). setCorePoolSize(1). build val works = new AtomicBoolean(true) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala index 3e8336be51..5e9eed4a5d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala @@ -12,7 +12,8 @@ import akka.config.Configuration @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class DispatchersSpec extends AkkaSpec { - import app.dispatcherFactory._ + val df = system.dispatcherFactory + import df._ val tipe = "type" val keepalivems = "keep-alive-time" diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala index d2b410a7bf..d9feed3209 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala @@ -27,14 +27,14 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach { "support tell" in { 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" assert(oneWay.await(1, TimeUnit.SECONDS)) actor.stop() } "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] assert("World" === result.get) actor.stop() diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala index b143b99e14..ffdc34a903 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -13,7 +13,7 @@ class ConfigSpec extends AkkaSpec(ActorSystem("ConfigSpec", Configuration.fromFi "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 { - val config = app.config + val config = system.AkkaConfig.config import config._ getList("akka.boot") must equal(Nil) diff --git a/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala b/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala index 94e59ebbf1..35924a2b14 100644 --- a/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala +++ b/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala @@ -19,7 +19,7 @@ class Future2ActorSpec extends AkkaSpec { } "support reply via sender" in { - val actor = app.actorOf(Props(new Actor { + val actor = system.actorOf(Props(new Actor { def receive = { case "do" ⇒ Future(31) pipeTo context.sender case "ex" ⇒ Future(throw new AssertionError) pipeTo context.sender diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index 256bb4f9ca..edb959dfa1 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -773,7 +773,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "ticket812FutureDispatchCleanup" in { filterException[FutureTimeoutException] { - implicit val dispatcher = app.dispatcherFactory.newDispatcher("ticket812FutureDispatchCleanup").build + implicit val dispatcher = system.dispatcherFactory.newDispatcher("ticket812FutureDispatchCleanup").build assert(dispatcher.tasks === 0) val future = Future({ Thread.sleep(100); "Done" }, 10) intercept[FutureTimeoutException] { future.await } diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala index c5ee2f1dfb..7ac0061e3c 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala @@ -80,7 +80,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn 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) { q must not be null diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala index ebc42c92d9..7ed96a6dc8 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -18,12 +18,12 @@ class PriorityDispatcherSpec extends AkkaSpec { testOrdering(BoundedPriorityMailbox(PriorityGenerator({ case i: Int ⇒ i //Reverse order case 'Result ⇒ Int.MaxValue - }: Any ⇒ Int), 1000, app.AkkaConfig.MailboxPushTimeout)) + }: Any ⇒ Int), 1000, system.AkkaConfig.MailboxPushTimeout)) } } 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 { var acc: List[Int] = Nil diff --git a/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala b/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala index 71a52799a8..d5371af0b9 100644 --- a/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala @@ -6,7 +6,7 @@ package akka.event import akka.testkit.AkkaSpec import akka.config.Configuration import akka.util.duration._ -import akka.actor.{ Actor, ActorRef } +import akka.actor.{ Actor, ActorRef, ActorSystemImpl } object EventStreamSpec { case class M(i: Int) @@ -14,7 +14,7 @@ object EventStreamSpec { case class SetTarget(ref: ActorRef) class MyLog extends Actor { - var dst: ActorRef = app.deadLetters + var dst: ActorRef = system.deadLetters def receive = { case Logging.InitializeLogger(bus) ⇒ bus.subscribe(context.self, classOf[SetTarget]); sender ! Logging.LoggerInitialized case SetTarget(ref) ⇒ dst = ref; dst ! "OK" @@ -36,11 +36,13 @@ class EventStreamSpec extends AkkaSpec(Configuration( import EventStreamSpec._ + val impl = system.asInstanceOf[ActorSystemImpl] + "An EventStream" must { "manage subscriptions" in { val bus = new EventStream(true) - bus.start(app.provider) + bus.start(impl) bus.subscribe(testActor, classOf[M]) bus.publish(M(42)) within(1 second) { @@ -53,8 +55,8 @@ class EventStreamSpec extends AkkaSpec(Configuration( "manage log levels" in { val bus = new EventStream(false) - bus.start(app.provider) - bus.startDefaultLoggers(app.provider, app.AkkaConfig) + bus.start(impl) + bus.startDefaultLoggers(impl) bus.publish(SetTarget(testActor)) expectMsg("OK") within(2 seconds) { @@ -75,7 +77,7 @@ class EventStreamSpec extends AkkaSpec(Configuration( val b2 = new B2 val c = new C val bus = new EventStream(false) - bus.start(app.provider) + bus.start(impl) within(2 seconds) { bus.subscribe(testActor, classOf[B2]) === true bus.publish(c) diff --git a/akka-actor-tests/src/test/scala/akka/performance/microbench/TellLatencyPerformanceSpec.scala b/akka-actor-tests/src/test/scala/akka/performance/microbench/TellLatencyPerformanceSpec.scala index 14acef4373..7eec58b70c 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/microbench/TellLatencyPerformanceSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/microbench/TellLatencyPerformanceSpec.scala @@ -17,7 +17,7 @@ import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistic class TellLatencyPerformanceSpec extends PerformanceSpec { import TellLatencyPerformanceSpec._ - val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher") + val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher") .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .setCorePoolSize(8) .build @@ -62,13 +62,13 @@ class TellLatencyPerformanceSpec extends PerformanceSpec { val latch = new CountDownLatch(numberOfClients) val repeatsPerClient = repeat / numberOfClients val clients = (for (i ← 0 until numberOfClients) yield { - val destination = app.actorOf[Destination] - val w4 = app.actorOf(new Waypoint(destination)) - val w3 = app.actorOf(new Waypoint(w4)) - val w2 = app.actorOf(new Waypoint(w3)) - val w1 = app.actorOf(new Waypoint(w2)) + val destination = system.actorOf[Destination] + val w4 = system.actorOf(new Waypoint(destination)) + val w3 = system.actorOf(new Waypoint(w4)) + val w2 = system.actorOf(new Waypoint(w3)) + val w1 = system.actorOf(new Waypoint(w2)) Props(new Client(w1, latch, repeatsPerClient, clientDelayMicros, stat)).withDispatcher(clientDispatcher) - }).toList.map(app.actorOf(_)) + }).toList.map(system.actorOf(_)) val start = System.nanoTime clients.foreach(_ ! Run) diff --git a/akka-actor-tests/src/test/scala/akka/performance/microbench/TellThroughputPerformanceSpec.scala b/akka-actor-tests/src/test/scala/akka/performance/microbench/TellThroughputPerformanceSpec.scala index 5f2c3ec74f..1b7c7899c6 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/microbench/TellThroughputPerformanceSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/microbench/TellThroughputPerformanceSpec.scala @@ -18,12 +18,12 @@ import akka.dispatch.Dispatchers class TellThroughputPerformanceSpec extends PerformanceSpec { import TellThroughputPerformanceSpec._ - val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher") + val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher") .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .setCorePoolSize(maxClients) .build - val destinationDispatcher = app.dispatcherFactory.newDispatcher("destination-dispatcher") + val destinationDispatcher = system.dispatcherFactory.newDispatcher("destination-dispatcher") .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .setCorePoolSize(maxClients) .build @@ -71,9 +71,9 @@ class TellThroughputPerformanceSpec extends PerformanceSpec { val latch = new CountDownLatch(numberOfClients) val repeatsPerClient = repeat / 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) - 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 clients.foreach(_ ! Run) diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingLatencyPerformanceSpec.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingLatencyPerformanceSpec.scala index 95249cfe29..06a4bd9fa7 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingLatencyPerformanceSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingLatencyPerformanceSpec.scala @@ -21,7 +21,7 @@ import akka.performance.trading.domain.Orderbook @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class TradingLatencyPerformanceSpec extends PerformanceSpec { - val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher") + val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher") .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .setCorePoolSize(maxClients) .build @@ -38,7 +38,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec { override def beforeEach() { super.beforeEach() stat = new SynchronizedDescriptiveStatistics - tradingSystem = new AkkaTradingSystem(app) + tradingSystem = new AkkaTradingSystem(system) tradingSystem.start() TotalTradeCounter.reset() stat = new SynchronizedDescriptiveStatistics @@ -99,7 +99,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec { val clients = (for (i ← 0 until numberOfClients) yield { val receiver = receivers(i % receivers.size) val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelayMicros)).withDispatcher(clientDispatcher) - app.actorOf(props) + system.actorOf(props) }) clients.foreach(_ ! "run") diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingThroughputPerformanceSpec.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingThroughputPerformanceSpec.scala index 4e5e108e0a..7ec84137a7 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingThroughputPerformanceSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingThroughputPerformanceSpec.scala @@ -21,7 +21,7 @@ import akka.performance.trading.domain.Orderbook @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class TradingThroughputPerformanceSpec extends PerformanceSpec { - val clientDispatcher = app.dispatcherFactory.newDispatcher("client-dispatcher") + val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher") .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .setCorePoolSize(maxClients) .build @@ -30,7 +30,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec { override def beforeEach() { super.beforeEach() - tradingSystem = new AkkaTradingSystem(app) + tradingSystem = new AkkaTradingSystem(system) tradingSystem.start() TotalTradeCounter.reset() } @@ -92,7 +92,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec { val clients = (for (i ← 0 until numberOfClients) yield { val receiver = receivers(i % receivers.size) val props = Props(new Client(receiver, orders, latch, ordersPerClient)).withDispatcher(clientDispatcher) - app.actorOf(props) + system.actorOf(props) }) clients.foreach(_ ! "run") diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/PerformanceSpec.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/PerformanceSpec.scala index 93d7282b14..b17bb5913c 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/workbench/PerformanceSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/PerformanceSpec.scala @@ -11,8 +11,6 @@ import akka.actor.ActorSystem trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach { - def app: ActorSystem - def isBenchmark() = System.getProperty("benchmark") == "true" def minClients() = System.getProperty("benchmark.minClients", "1").toInt; @@ -29,7 +27,7 @@ trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach { } 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 diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala index b42e1b87c6..05500932dd 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala @@ -224,8 +224,8 @@ class Report( sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion) sb.append("\n") sb.append("Akka config:") - for (key ← app.config.keys) { - sb.append("\n ").append(key).append("=").append(app.config(key)) + for (key ← app.AkkaConfig.config.keys) { + sb.append("\n ").append(key).append("=").append(app.AkkaConfig.config(key)) } sb.toString diff --git a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala index 305d70e00a..126a15b447 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala @@ -329,7 +329,7 @@ class ActorPoolSpec extends AkkaSpec { } "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 upperBound = 5 def pressureThreshold = 1 @@ -338,7 +338,7 @@ class ActorPoolSpec extends AkkaSpec { def rampupRate = 0.1 def backoffRate = 0.50 def backoffThreshold = 0.50 - def instance(p: Props) = app.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds))) + def instance(p: Props) = system.typedActor.getActorRefFor(context.typedActorOf[Foo, FooImpl](props = p.withTimeout(10 seconds))) def receive = _route }, Props().withTimeout(10 seconds).withFaultHandler(faultHandler)) @@ -348,7 +348,7 @@ class ActorPoolSpec extends AkkaSpec { val value = r.get value must equal(i * i) } - app.typedActor.stop(pool) + system.typedActor.stop(pool) } "provide default supervision of pooled actors" in { diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala index 424e897fa4..575fcfb7fe 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala @@ -11,12 +11,14 @@ import akka.routing.Routing.Broadcast @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ConfiguredLocalRoutingSpec extends AkkaSpec { + val deployer = system.asInstanceOf[ActorSystemImpl].provider.deployer + "round robin router" must { "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( path.toString, None, @@ -27,7 +29,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val helloLatch = new CountDownLatch(5) val stopLatch = new CountDownLatch(5) - val actor = app.actorOf(Props(new Actor { + val actor = system.actorOf(Props(new Actor { def receive = { case "hello" ⇒ helloLatch.countDown() } @@ -49,9 +51,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { } "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( path.toString, None, @@ -69,7 +71,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { replies = replies + (i -> 0) } - val actor = app.actorOf(Props(new Actor { + val actor = system.actorOf(Props(new Actor { lazy val id = counter.getAndIncrement() def receive = { case "hit" ⇒ sender ! id @@ -93,9 +95,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { } "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( path.toString, None, @@ -106,7 +108,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val helloLatch = new CountDownLatch(5) val stopLatch = new CountDownLatch(5) - val actor = app.actorOf(Props(new Actor { + val actor = system.actorOf(Props(new Actor { def receive = { case "hello" ⇒ helloLatch.countDown() } @@ -127,9 +129,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { "random router" must { "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( path.toString, None, @@ -139,7 +141,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val stopLatch = new CountDownLatch(7) - val actor = app.actorOf(Props(new Actor { + val actor = system.actorOf(Props(new Actor { def receive = { case "hello" ⇒ {} } @@ -160,9 +162,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { } "deliver messages in a random fashion" in { - val path = app / "random-1" + val path = system / "random-1" - app.provider.deployer.deploy( + deployer.deploy( Deploy( path.toString, None, @@ -180,7 +182,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { replies = replies + (i -> 0) } - val actor = app.actorOf(Props(new Actor { + val actor = system.actorOf(Props(new Actor { lazy val id = counter.getAndIncrement() def receive = { case "hit" ⇒ sender ! id @@ -204,9 +206,9 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { } "deliver a broadcast message using the !" in { - val path = app / "random-2" + val path = system / "random-2" - app.provider.deployer.deploy( + deployer.deploy( Deploy( path.toString, None, @@ -217,7 +219,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val helloLatch = new CountDownLatch(6) val stopLatch = new CountDownLatch(6) - val actor = app.actorOf(Props(new Actor { + val actor = system.actorOf(Props(new Actor { def receive = { case "hello" ⇒ helloLatch.countDown() } diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index 8fea2d6f26..186e0abf90 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -22,6 +22,8 @@ object RoutingSpec { @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class RoutingSpec extends AkkaSpec { + val impl = system.asInstanceOf[ActorSystemImpl] + import akka.routing.RoutingSpec._ "direct router" must { @@ -29,7 +31,7 @@ class RoutingSpec extends AkkaSpec { val actor1 = actorOf[TestActor] 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) } @@ -45,7 +47,7 @@ class RoutingSpec extends AkkaSpec { }) 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 ! "end" @@ -66,7 +68,7 @@ class RoutingSpec extends AkkaSpec { }) 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 ! "end" @@ -83,7 +85,7 @@ class RoutingSpec extends AkkaSpec { val actor1 = actorOf[TestActor] 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) } @@ -113,7 +115,7 @@ class RoutingSpec extends AkkaSpec { //create the routed actor. 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. 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 actor = new RoutedActorRef(app, props, app.guardian, "foo") + val actor = new RoutedActorRef(system, props, impl.guardian, "foo") actor ! Broadcast(1) actor ! Broadcast("end") @@ -175,7 +177,7 @@ class RoutingSpec extends AkkaSpec { }) 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) } @@ -192,7 +194,7 @@ class RoutingSpec extends AkkaSpec { val actor1 = actorOf[TestActor] 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) } @@ -216,7 +218,7 @@ class RoutingSpec extends AkkaSpec { }) 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("end") @@ -239,7 +241,7 @@ class RoutingSpec extends AkkaSpec { }) 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 { 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 actor = new RoutedActorRef(app, props, app.guardian, "foo") + val actor = new RoutedActorRef(system, props, impl.guardian, "foo") 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 actor = new RoutedActorRef(app, props, app.guardian, "foo") + val actor = new RoutedActorRef(system, props, impl.guardian, "foo") 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 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) @@ -302,14 +304,14 @@ class RoutingSpec extends AkkaSpec { "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 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) } "be started when constructed" in { 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) } @@ -324,7 +326,7 @@ class RoutingSpec extends AkkaSpec { for (i ← 0 until connectionCount) { counters = counters :+ new AtomicInteger() - val connection = app.actorOf(new Actor { + val connection = system.actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() 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 actor = new RoutedActorRef(app, props, app.guardian, "foo") + val actor = new RoutedActorRef(system, props, impl.guardian, "foo") for (i ← 0 until iterationCount) { for (k ← 0 until connectionCount) { @@ -357,7 +359,7 @@ class RoutingSpec extends AkkaSpec { val doneLatch = new TestLatch(2) val counter1 = new AtomicInteger - val connection1 = app.actorOf(new Actor { + val connection1 = system.actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counter1.addAndGet(msg) @@ -365,7 +367,7 @@ class RoutingSpec extends AkkaSpec { }) val counter2 = new AtomicInteger - val connection2 = app.actorOf(new Actor { + val connection2 = system.actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() 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 actor = new RoutedActorRef(app, props, app.guardian, "foo") + val actor = new RoutedActorRef(system, props, impl.guardian, "foo") actor ! Broadcast(1) actor ! Broadcast("end") @@ -387,7 +389,7 @@ class RoutingSpec extends AkkaSpec { 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 = { case Stop(None) ⇒ self.stop() case Stop(Some(_id)) if (_id == id) ⇒ self.stop() diff --git a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala index 36cc6c03ca..0b8e13ca6c 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -7,7 +7,7 @@ package akka.serialization import akka.serialization.Serialization._ import scala.reflect._ import akka.testkit.AkkaSpec -import akka.actor.ActorSystem +import akka.actor.{ ActorSystem, ActorSystemImpl } import java.io.{ ObjectInputStream, ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream } import akka.actor.DeadLetterActorRef @@ -24,7 +24,8 @@ object SerializeSpec { class SerializeSpec extends AkkaSpec { import SerializeSpec._ - import app.serialization._ + val ser = system.serialization + import ser._ "Serialization" must { @@ -68,13 +69,13 @@ class SerializeSpec extends AkkaSpec { "serialize DeadLetterActorRef" in { val outbuf = new ByteArrayOutputStream() val out = new ObjectOutputStream(outbuf) - val a = new ActorSystem() + val a = ActorSystem() out.writeObject(a.deadLetters) out.flush() out.close() val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray)) - Serialization.app.withValue(a) { + Serialization.app.withValue(a.asInstanceOf[ActorSystemImpl]) { val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef] (deadLetters eq a.deadLetters) must be(true) } diff --git a/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala b/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala index e82d6d9dcb..95c4dc9e8b 100644 --- a/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala @@ -11,7 +11,7 @@ import org.junit.{ After, Test } class CallingThreadDispatcherModelSpec extends 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" } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 6820d12e8a..b9aba3ac1a 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -154,7 +154,7 @@ object Timeout { } trait ActorLogging { this: Actor ⇒ - val log = akka.event.Logging(app.eventStream, context.self) + val log = akka.event.Logging(system.eventStream, context.self) } object Actor { @@ -229,12 +229,12 @@ trait Actor { context } - implicit def app = context.app + implicit def system = context.system /** * 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 @@ -250,7 +250,7 @@ trait Actor { * This method does NOT modify the given Receive unless * akka.actor.debug.receive is set within akka.conf. */ - def loggable(self: AnyRef)(r: Receive): Receive = if (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. diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 4df2834142..fd7110a8de 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -45,7 +45,7 @@ trait ActorContext extends ActorRefFactory with TypedActorFactory { def handleChildTerminated(child: ActorRef): Unit - def app: ActorSystem + def system: ActorSystem def parent: ActorRef } @@ -63,7 +63,7 @@ private[akka] object ActorCell { //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) private[akka] class ActorCell( - val app: ActorSystem, + val app: ActorSystemImpl, val self: ActorRef with ScalaActorRef, val props: Props, val parent: ActorRef, @@ -72,6 +72,8 @@ private[akka] class ActorCell( import ActorCell._ + final def system = app + protected final def guardian = self protected def typedActor = app.typedActor diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index 1b98863b29..446e8f43c5 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -4,6 +4,8 @@ package akka.actor +import akka.remote.RemoteAddress + object ActorPath { final val separator = "/" @@ -58,9 +60,9 @@ object 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. @@ -78,9 +80,9 @@ trait ActorPath { def /(child: String): ActorPath /** - * Find the ActorRef for this path. + * Recursively create a descendant’s 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. @@ -98,15 +100,13 @@ trait ActorPath { def isRoot: Boolean } -class RootActorPath(val app: ActorSystem) extends ActorPath { +class RootActorPath(val remoteAddress: RemoteAddress) extends ActorPath { def name: String = "/" def parent: ActorPath = this - def /(child: String): ActorPath = new ChildActorPath(app, this, child) - - def ref: Option[ActorRef] = app.actorFor(path) + def /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child) def string: String = "" @@ -117,11 +117,9 @@ class RootActorPath(val app: ActorSystem) extends ActorPath { 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 ref: Option[ActorRef] = app.actorFor(path) + def /(child: String): ActorPath = new ChildActorPath(remoteAddress, this, child) def string: String = parent.string + ActorPath.separator + name diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 20feee1e02..09264aa2ae 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -13,6 +13,7 @@ import java.net.InetSocketAddress import akka.remote.RemoteAddress import java.util.concurrent.TimeUnit import akka.event.EventStream +import akka.event.DeathWatch /** * 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 Jonas Bonér */ class LocalActorRef private[akka] ( - app: ActorSystem, + app: ActorSystemImpl, _props: Props, _supervisor: ActorRef, val path: ActorPath, @@ -172,7 +173,7 @@ class LocalActorRef private[akka] ( 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 @@ -415,16 +416,15 @@ class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) exte 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) - // FIXME (actor path): put this under the tmp guardian supervisor - val path: ActorPath = app.root / "tmp" / name + override def name = path.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 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")) @throws(classOf[java.io.ObjectStreamException]) - private def writeReplace(): AnyRef = app.provider.serialize(this) + private def writeReplace(): AnyRef = provider.serialize(this) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 52773f9c98..d78ffcab0c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -22,7 +22,7 @@ import com.eaio.uuid.UUID */ 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] @@ -36,6 +36,8 @@ trait ActorRefProvider { def AkkaConfig: ActorSystem.AkkaConfig + def init(app: ActorSystemImpl) + /** * What deployer will be used to resolve deployment configuration? */ @@ -43,9 +45,9 @@ trait ActorRefProvider { 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 @@ -61,7 +63,7 @@ trait ActorRefProvider { 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 { - 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. @@ -85,7 +89,7 @@ trait ActorRefFactory { 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 @@ -95,7 +99,7 @@ trait ActorRefFactory { def actorOf(props: Props, name: String): ActorRef = { if (name == null || name == "" || name.startsWith("$")) 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]])) @@ -109,6 +113,8 @@ trait ActorRefFactory { 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: Iterable[String]): Option[ActorRef] = provider.actorFor(path) @@ -120,7 +126,6 @@ class ActorRefProviderException(message: String) extends AkkaException(message) * Local ActorRef provider. */ class LocalActorRefProvider( - private val app: ActorSystem, val AkkaConfig: ActorSystem.AkkaConfig, val root: ActorPath, val eventStream: EventStream, @@ -136,16 +141,18 @@ class LocalActorRefProvider( 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 */ private val tempNumber = new AtomicLong - def tempPath = { + def tempName = { val l = tempNumber.getAndIncrement() "$_" + 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 // currently still used for tmp actors (e.g. ask actor refs) @@ -159,12 +166,12 @@ class LocalActorRefProvider( @volatile 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 - val path = app.root + val path = root / name - val address = app.address + path.toString + val address = path.toString override def toString = name @@ -208,20 +215,33 @@ class LocalActorRefProvider( } private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy) - private val rootGuardian: ActorRef = actorOf(guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, root, true) - - val guardian: ActorRef = actorOf(guardianProps, rootGuardian, "app", true) - - val systemGuardian: ActorRef = actorOf(guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true) + /* + * The problem is that ActorRefs need a reference to the ActorSystem to + * provide their service. Hence they cannot be created while the + * constructors of ActorSystem and ActorRefProvider are still running. + * 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() - // chain death watchers so that killing guardian stops the application - deathWatch.subscribe(systemGuardian, guardian) - deathWatch.subscribe(rootGuardian, systemGuardian) + 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 + deathWatch.subscribe(systemGuardian, guardian) + 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) - 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 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 actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = - actorOf(props, supervisor, supervisor.path / name, systemService) + private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = + 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 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 { case null ⇒ @@ -270,7 +290,7 @@ class LocalActorRefProvider( case RouterType.Random ⇒ () ⇒ new RandomRouter case RouterType.RoundRobin ⇒ () ⇒ new RoundRobinRouter 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.LeastRAM ⇒ sys.error("Router LeastRAM 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) } - 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) } @@ -307,7 +327,7 @@ class LocalActorRefProvider( /** * 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 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 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 ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = { import akka.dispatch.{ Future, Promise, DefaultPromise } - (if (within == null) app.AkkaConfig.ActorTimeout else within) match { - case t if t.duration.length <= 0 ⇒ new DefaultPromise[Any](0)(app.dispatcher) //Abort early if nonsensical timeout + (if (within == null) AkkaConfig.ActorTimeout else within) match { + case t if t.duration.length <= 0 ⇒ new DefaultPromise[Any](0)(dispatcher) //Abort early if nonsensical timeout 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 recipient.tell(message, a) a.result diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index b812b936ea..d0a344564f 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -61,11 +61,14 @@ object ActorSystem { 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 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._ - def this(name: String) = this(name, ActorSystem.defaultConfig) - def this() = this("default") + def name: String + def AkkaConfig: AkkaConfig + def nodename: String - val AkkaConfig = new AkkaConfig(config) - - private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(props, systemGuardian, address, true) - - import AkkaConfig._ + /** + * Construct a path below the application guardian. + */ + def /(name: String): ActorPath + def root: ActorPath val startTime = System.currentTimeMillis 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 { case null | "" ⇒ InetAddress.getLocalHost.getHostAddress case value ⇒ value @@ -158,7 +193,7 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF /** * 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 deadLetterMailbox = new Mailbox(null) { @@ -181,16 +216,21 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF deadLetters.init(dispatcher) - // TODO think about memory consistency effects when doing funky stuff inside constructor val provider: ActorRefProvider = { val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match { case Left(e) ⇒ throw e case Right(b) ⇒ b } - val params: Array[Class[_]] = Array(classOf[ActorSystem], classOf[AkkaConfig], classOf[ActorPath], classOf[EventStream], classOf[MessageDispatcher], classOf[Scheduler]) - val args: Array[AnyRef] = Array(this, AkkaConfig, root, eventStream, dispatcher, scheduler) + val arguments = List( + 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 Right(p) ⇒ p } @@ -200,24 +240,33 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF def guardian: ActorRef = provider.guardian def systemGuardian: ActorRef = provider.systemGuardian def deathWatch: DeathWatch = provider.deathWatch + def nodename: String = provider.nodename terminationFuture.onComplete(_ ⇒ scheduler.stop()) terminationFuture.onComplete(_ ⇒ dispatcher.shutdown()) - // this starts the reaper actor and the user-configured logging subscribers, which are also actors - eventStream.start(provider) - eventStream.startDefaultLoggers(provider, AkkaConfig) + @volatile + private var _serialization: Serialization = _ + 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 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 def stop() { guardian.stop() diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index d4b65ba453..79ca427be4 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -188,7 +188,7 @@ trait FSM[S, D] extends ListenerManagement { type Timeout = Option[Duration] 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) { val t = timeout.get 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 - private val debugEvent = context.app.AkkaConfig.FsmDebugEvent + private val debugEvent = system.AkkaConfig.FsmDebugEvent private val events = new Array[Event](logDepth) private val states = new Array[AnyRef](logDepth) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 4f3219a4e2..c71938aed7 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -253,7 +253,7 @@ class IOManager(bufferSize: Int = 8192) extends Actor { var worker: IOWorker = _ override def preStart { - worker = new IOWorker(app, self, bufferSize) + worker = new IOWorker(system, self, bufferSize) worker.start() } diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 7c553d6887..ad4b074cef 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -330,7 +330,7 @@ class TypedActor(val AkkaConfig: ActorSystem.AkkaConfig, var ser: Serialization) def receive = { case m: MethodCall ⇒ TypedActor.selfReference set proxyVar.get - TypedActor.appReference set app + TypedActor.appReference set system try { if (m.isOneWay) m(me) else { diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala index a3ffb5ae97..569c66f03e 100644 --- a/akka-actor/src/main/scala/akka/actor/package.scala +++ b/akka-actor/src/main/scala/akka/actor/package.scala @@ -8,10 +8,6 @@ package object actor { implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef] 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 def newUuid(): Uuid = new Uuid() diff --git a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala index 1f26ec29e3..37c75716d5 100644 --- a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala +++ b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala @@ -103,7 +103,7 @@ class NodeAddress(val clusterName: String, val nodeName: String) { */ object NodeAddress { 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 { case address: NodeAddress ⇒ Some((address.clusterName, address.nodeName)) diff --git a/akka-actor/src/main/scala/akka/event/EventStream.scala b/akka-actor/src/main/scala/akka/event/EventStream.scala index 921ecb6b90..1351007b51 100644 --- a/akka-actor/src/main/scala/akka/event/EventStream.scala +++ b/akka-actor/src/main/scala/akka/event/EventStream.scala @@ -3,11 +3,8 @@ */ package akka.event -import akka.actor.{ ActorRef, Actor, Props } -import akka.actor.ActorSystem -import akka.actor.Terminated +import akka.actor.{ ActorRef, Actor, Props, ActorSystemImpl, Terminated } import akka.util.Subclassification -import akka.actor.ActorRefProvider 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) } - def start(provider: ActorRefProvider) { - reaper = provider.actorOf(Props(new Actor { + def start(app: ActorSystemImpl) { + reaper = app.systemActorOf(Props(new Actor { def receive = { case ref: ActorRef ⇒ watch(ref) case Terminated(ref) ⇒ unsubscribe(ref) } - }), provider.systemGuardian, "MainBusReaper", true) + }), "MainBusReaper") subscribers foreach (reaper ! _) } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 052b43062f..09d82e7c09 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -3,7 +3,7 @@ */ 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.actor.ActorSystem.AkkaConfig import akka.util.ReflectiveAccess @@ -79,13 +79,13 @@ trait LoggingBus extends ActorEventBus { publish(Info(this, "StandardOutLogger started")) } - private[akka] def startDefaultLoggers(provider: ActorRefProvider, config: AkkaConfig) { - val level = levelFor(config.LogLevel) getOrElse { - StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.LogLevel)) + private[akka] def startDefaultLoggers(app: ActorSystemImpl) { + val level = levelFor(app.AkkaConfig.LogLevel) getOrElse { + StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + app.AkkaConfig.LogLevel)) ErrorLevel } try { - val defaultLoggers = config.EventHandlers match { + val defaultLoggers = app.AkkaConfig.EventHandlers match { case Nil ⇒ "akka.event.Logging$DefaultLogger" :: Nil case loggers ⇒ loggers } @@ -95,7 +95,7 @@ trait LoggingBus extends ActorEventBus { } yield { try { ReflectiveAccess.getClassFor[Actor](loggerName) match { - case Right(actorClass) ⇒ addLogger(provider, actorClass, level) + case Right(actorClass) ⇒ addLogger(app, actorClass, level) case Left(exception) ⇒ throw exception } } catch { @@ -138,9 +138,9 @@ trait LoggingBus extends ActorEventBus { 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 actor = provider.actorOf(Props(clazz), provider.systemGuardian, name, true) + val actor = app.systemActorOf(Props(clazz), name) implicit val timeout = Timeout(3 seconds) val response = try actor ? InitializeLogger(this) get catch { case _: FutureTimeoutException ⇒ diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index f57cde814d..768be6dc9d 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -6,7 +6,7 @@ package akka.serialization import akka.AkkaException import akka.util.ReflectiveAccess -import akka.actor.ActorSystem +import akka.actor.{ ActorSystem, ActorSystemImpl } import scala.util.DynamicVariable 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 * 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 def serialize(o: AnyRef): Either[Exception, Array[Byte]] = @@ -70,7 +70,7 @@ class Serialization(val app: ActorSystem) { * But "default" can be overridden in config */ val serializers: Map[String, Serializer] = - app.config.getSection("akka.actor.serializers") + app.AkkaConfig.config.getSection("akka.actor.serializers") .map(_.map) .getOrElse(Map()) .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 */ - 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]()) { 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? @@ -102,6 +102,6 @@ class Serialization(val app: ActorSystem) { object Serialization { // 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) } diff --git a/akka-docs/scala/code/ActorDocSpec.scala b/akka-docs/scala/code/ActorDocSpec.scala index fb2ccdde5b..8ec3daa2f2 100644 --- a/akka-docs/scala/code/ActorDocSpec.scala +++ b/akka-docs/scala/code/ActorDocSpec.scala @@ -14,7 +14,7 @@ import akka.config.Configuration //#my-actor class MyActor extends Actor { - val log = Logging(app, this) + val log = Logging(system, this) def receive = { case "test" ⇒ log.info("received test") case _ ⇒ log.info("received unknown message") @@ -36,8 +36,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) { case e: Logging.Info ⇒ true case _ ⇒ false } - app.eventStream.publish(TestEvent.Mute(filter)) - app.eventStream.subscribe(testActor, classOf[Logging.Info]) + system.eventStream.publish(TestEvent.Mute(filter)) + system.eventStream.subscribe(testActor, classOf[Logging.Info]) myActor ! "test" expectMsgPF(1 second) { case Logging.Info(_, "received test") ⇒ true } @@ -45,8 +45,8 @@ class ActorDocSpec extends AkkaSpec(Configuration("akka.loglevel" -> "INFO")) { myActor ! "unknown" expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") ⇒ true } - app.eventStream.unsubscribe(testActor) - app.eventStream.publish(TestEvent.UnMute(filter)) + system.eventStream.unsubscribe(testActor) + system.eventStream.publish(TestEvent.UnMute(filter)) myActor.stop() } diff --git a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala index 8a6d1fb8fa..316bbd5c95 100644 --- a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala +++ b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala @@ -27,8 +27,8 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10 def this(app: ActorSystem) { this( - app.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.theshold", 8), + app.AkkaConfig.config.getInt("akka.remote.failure-detector.max-sample-size", 1000)) } private final val PhiFactor = 1.0 / math.log(10.0) diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index 475f146e3d..9873d7960e 100644 --- a/akka-remote/src/main/scala/akka/remote/Gossiper.scala +++ b/akka-remote/src/main/scala/akka/remote/Gossiper.scala @@ -107,7 +107,7 @@ class Gossiper(remote: Remote) { 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 address = app.address + private val address = app.root.remoteAddress private val nodeFingerprint = address.## private val random = SecureRandom.getInstance("SHA1PRNG") @@ -153,7 +153,7 @@ class Gossiper(remote: Remote) { node ← oldAvailableNodes 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 oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes } diff --git a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala index 2763080121..de14fabcc4 100644 --- a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala +++ b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala @@ -7,7 +7,7 @@ package akka.remote import scala.collection.mutable import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid } 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. @@ -58,12 +58,12 @@ object NetworkEventStream { } } -class NetworkEventStream(val app: ActorSystem) { +class NetworkEventStream(app: ActorSystemImpl) { import NetworkEventStream._ // 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")), app.systemGuardian, "network-event-sender", systemService = true) diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index 01e461d4da..dd005a2e15 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -15,25 +15,24 @@ import akka.actor.DeploymentConfig._ import akka.serialization.Compression.LZF import akka.remote.RemoteProtocol._ import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._ - import java.net.InetSocketAddress - import com.eaio.uuid.UUID import akka.serialization.{ JavaSerializer, Serialization, Serializer, Compression } 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. * * @author Jonas Bonér */ -class Remote(val app: ActorSystem, val nodename: String) { +class Remote(val app: ActorSystemImpl, val nodename: String) { val log = Logging(app, this) import app._ - import app.config - import app.AkkaConfig._ + val AC = AkkaConfig + import AC._ // TODO move to AkkaConfig? 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? 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( OneForOneStrategy(List(classOf[Exception]), None, None)), "akka-system-remote-supervisor") // is infinite restart what we want? private[remote] lazy val remoteDaemon = - app.provider.actorOf( + app.provider.actorOf(app, Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)), remoteDaemonSupervisor, remoteDaemonServiceName, @@ -81,7 +81,7 @@ class Remote(val app: ActorSystem, val nodename: String) { } 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 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 parent = actorPath.parent.ref + val parent = app.actorFor(actorPath.parent) 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 { 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 def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) { new LocalActorRef(app, Props( context ⇒ { 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 @@ -195,7 +205,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { Props( context ⇒ { 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 @@ -204,7 +214,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { Props( context ⇒ { 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 @@ -213,7 +223,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { Props( context ⇒ { 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) { @@ -230,9 +240,12 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { } class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) { + + val provider = remote.app.asInstanceOf[ActorSystemImpl].provider + lazy val sender: ActorRef = if (input.hasSender) - remote.app.provider.deserialize( + provider.deserialize( SerializedActorRef(input.getSender.getHost, input.getSender.getPort, input.getSender.getPath)).getOrElse(throw new IllegalStateException("OHNOES")) else remote.app.deadLetters @@ -282,7 +295,7 @@ trait RemoteMarshallingOps { * Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message. */ 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 } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 531730a51b..f2d334cf3a 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -29,43 +29,51 @@ import akka.event.EventStream * @author Jonas Bonér */ class RemoteActorRefProvider( - val app: ActorSystem, val AkkaConfig: ActorSystem.AkkaConfig, val root: ActorPath, val eventStream: EventStream, val dispatcher: MessageDispatcher, val scheduler: Scheduler) extends ActorRefProvider { - val log = Logging(app, this) + val log = Logging(eventStream, this) import java.util.concurrent.ConcurrentHashMap 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 guardian = local.guardian def systemGuardian = local.systemGuardian 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 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 terminationFuture = local.terminationFuture private[akka] def deployer: Deployer = local.deployer - def defaultDispatcher = app.dispatcher - def defaultTimeout = app.AkkaConfig.ActorTimeout + def defaultDispatcher = dispatcher + def defaultTimeout = AkkaConfig.ActorTimeout - private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = - actorOf(props, supervisor, supervisor.path / name, systemService) + private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = + actorOf(app, props, supervisor, supervisor.path / name, systemService) - private[akka] def actorOf(props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = - if (systemService) local.actorOf(props, supervisor, path, systemService) + private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = + if (systemService) local.actorOf(app, props, supervisor, path, systemService) else { val name = path.name val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout? @@ -90,7 +98,7 @@ class RemoteActorRefProvider( if (isReplicaNode) { // 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 { // 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 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) - 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 { case e: Exception ⇒ @@ -159,7 +167,7 @@ class RemoteActorRefProvider( * Copied from LocalActorRefProvider... */ // 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") new RoutedActorRef(app, props, supervisor, name) } @@ -170,6 +178,7 @@ class RemoteActorRefProvider( case future: Future[_] ⇒ Some(future.get.asInstanceOf[ActorRef]) } + // TODO remove me val optimizeLocal = new AtomicBoolean(true) def optimizeLocalScoped_?() = optimizeLocal.get @@ -185,19 +194,19 @@ class RemoteActorRefProvider( private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = { val remoteAddress = RemoteAddress(actor.hostname, actor.port) - if (optimizeLocalScoped_? && remoteAddress == app.address) { + if (optimizeLocalScoped_? && remoteAddress == root.remoteAddress) { local.actorFor(ActorPath.split(actor.path)) } else { - log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", app.address, actor.path, remoteAddress) - Some(RemoteActorRef(remote.server, remoteAddress, ActorPath(app, actor.path), None)) //Should it be None here + log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", root.remoteAddress, actor.path, remoteAddress) + 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. */ - def useActorOnNode(remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) { - log.debug("[{}] Instantiating Actor [{}] on node [{}]", app.address, actorPath, remoteAddress) + def useActorOnNode(app: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) { + log.debug("[{}] Instantiating Actor [{}] on node [{}]", root, actorPath, remoteAddress) val actorFactoryBytes = app.serialization.serialize(actorFactory) match { @@ -260,6 +269,7 @@ class RemoteActorRefProvider( * @author Jonas Bonér */ private[akka] case class RemoteActorRef private[akka] ( + provider: ActorRefProvider, remote: RemoteSupport, remoteAddress: RemoteAddress, 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 ?(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 = () @@ -295,7 +305,7 @@ private[akka] case class RemoteActorRef private[akka] ( } @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 diff --git a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala index dab5c579e6..8f1c3d4733 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala @@ -149,5 +149,5 @@ class RemoteConnectionManager( } private[remote] def newConnection(remoteAddress: RemoteAddress, actorPath: ActorPath) = - RemoteActorRef(remote.server, remoteAddress, actorPath, None) + RemoteActorRef(remote.app.provider, remote.server, remoteAddress, actorPath, None) } diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index d02283bb37..51e2a33b8c 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -147,7 +147,8 @@ class ActiveRemoteClient private[akka] ( def sendSecureCookie(connection: ChannelFuture) { val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT) 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)) } @@ -350,8 +351,8 @@ class ActiveRemoteClientHandler( */ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps { - val serverSettings = new RemoteServerSettings(app.config, app.AkkaConfig.DefaultTimeUnit) - val clientSettings = new RemoteClientSettings(app.config, app.AkkaConfig.DefaultTimeUnit) + val serverSettings = new RemoteServerSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit) + val clientSettings = new RemoteClientSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit) private val remoteClients = new HashMap[RemoteAddress, RemoteClient] private val clientsLock = new ReadWriteGuard @@ -428,7 +429,7 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem def name = currentServer.get match { 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) @@ -459,7 +460,8 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) { val log = Logging(remoteSupport.app, this) import remoteSupport.serverSettings._ - import remoteSupport.app.address + + val address = remoteSupport.app.root.remoteAddress val name = "NettyRemoteServer@" + address diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala index 003f324217..f41ea0e855 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala @@ -5,6 +5,7 @@ package akka.remote import akka.testkit._ +import akka.actor.ActorSystemImpl 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. */ def remote: Remote = { - app.provider match { + system.asInstanceOf[ActorSystemImpl].provider match { case r: RemoteActorRefProvider ⇒ r.remote case _ ⇒ throw new Exception("Remoting is not enabled") } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala index 37a33feb01..f1a6745d91 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala @@ -10,7 +10,7 @@ object DirectRoutedRemoteActorMultiJvmSpec { class SomeActor extends Actor with Serializable { def receive = { - case "identify" ⇒ sender ! app.provider.nodename + case "identify" ⇒ sender ! system.nodename } } } @@ -47,7 +47,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec { barrier("start") - val actor = app.actorOf[SomeActor]("service-hello") + val actor = system.actorOf[SomeActor]("service-hello") actor.isInstanceOf[RoutedActorRef] must be(true) val result = (actor ? "identify").get diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala index 62ab22304b..3be4979964 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala @@ -8,7 +8,7 @@ object NewRemoteActorMultiJvmSpec { class SomeActor extends Actor with Serializable { def receive = { - case "identify" ⇒ sender ! app.provider.nodename + case "identify" ⇒ sender ! system.nodename } } } @@ -46,7 +46,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec { barrier("start") - val actor = app.actorOf[SomeActor]("service-hello") + val actor = system.actorOf[SomeActor]("service-hello") val result = (actor ? "identify").get result must equal("node1") diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala index a5b1eea333..a5701cccd4 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala @@ -9,7 +9,7 @@ object RandomRoutedRemoteActorMultiJvmSpec { val NrOfNodes = 4 class SomeActor extends Actor with Serializable { def receive = { - case "hit" ⇒ sender ! app.provider.nodename + case "hit" ⇒ sender ! system.nodename case "end" ⇒ self.stop() } } @@ -70,7 +70,7 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec { remote.start() barrier("start") - val actor = app.actorOf[SomeActor]("service-hello") + val actor = system.actorOf[SomeActor]("service-hello") actor.isInstanceOf[RoutedActorRef] must be(true) val connectionCount = NrOfNodes - 1 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala index 472dd0afd8..413d7814a5 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala @@ -9,7 +9,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec { val NrOfNodes = 4 class SomeActor extends Actor with Serializable { def receive = { - case "hit" ⇒ sender ! app.provider.nodename + case "hit" ⇒ sender ! system.nodename case "end" ⇒ self.stop() } } @@ -70,7 +70,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec { remote.start() barrier("start") - val actor = app.actorOf[SomeActor]("service-hello") + val actor = system.actorOf[SomeActor]("service-hello") actor.isInstanceOf[RoutedActorRef] must be(true) val connectionCount = NrOfNodes - 1 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/scatter_gather_routed/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/scatter_gather_routed/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala index ac732f3656..95c5037e8f 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/scatter_gather_routed/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/scatter_gather_routed/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala @@ -9,7 +9,7 @@ object ScatterGatherRoutedRemoteActorMultiJvmSpec { val NrOfNodes = 4 class SomeActor extends Actor with Serializable { def receive = { - case "hit" ⇒ sender ! app.provider.nodename + case "hit" ⇒ sender ! system.nodename case "end" ⇒ self.stop() } } @@ -70,7 +70,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec { remote.start() barrier("start") - val actor = app.actorOf[SomeActor]("service-hello") + val actor = system.actorOf[SomeActor]("service-hello") actor.isInstanceOf[RoutedActorRef] must be(true) actor.asInstanceOf[RoutedActorRef].router.isInstanceOf[ScatterGatherFirstCompletedRouter] must be(true) diff --git a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala index f322e62144..0e7592beb7 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala @@ -77,7 +77,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor { case Taken(`chopstickToWaitFor`) ⇒ println("%s has picked up %s and %s, and starts to eat", name, left.address, right.address) become(eating) - app.scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS) + system.scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS) case Busy(chopstick) ⇒ become(thinking) @@ -106,7 +106,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor { left ! Put(self) right ! Put(self) 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 @@ -114,7 +114,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor { case Think ⇒ println("%s starts to think", name) become(thinking) - app.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) + system.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) } } diff --git a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala index 97696e0dbe..4e1bb86ec0 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -32,7 +32,7 @@ case class TakenBy(hakker: ActorRef) class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] { // 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(Available) { @@ -47,7 +47,7 @@ class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] { case Event(Take, currentState) ⇒ stay replying Busy(self) case Event(Put, TakenBy(hakker)) if sender == hakker ⇒ - goto(Available) using TakenBy(app.deadLetters) + goto(Available) using TakenBy(system.deadLetters) } // Initialze the chopstick diff --git a/akka-stm/src/test/java/akka/stm/example/EitherOrElseExample.java b/akka-stm/src/test/java/akka/stm/example/EitherOrElseExample.java index 2c76ef90f3..a8f3fd475c 100644 --- a/akka-stm/src/test/java/akka/stm/example/EitherOrElseExample.java +++ b/akka-stm/src/test/java/akka/stm/example/EitherOrElseExample.java @@ -9,7 +9,7 @@ public class EitherOrElseExample { System.out.println("EitherOrElse example"); System.out.println(); - ActorSystem application = new ActorSystem("UntypedTransactorExample"); + ActorSystem application = ActorSystem.create("UntypedTransactorExample"); final Ref left = new Ref(100); final Ref right = new Ref(100); diff --git a/akka-stm/src/test/java/akka/stm/example/RetryExample.java b/akka-stm/src/test/java/akka/stm/example/RetryExample.java index f0c55d35df..f8b5700cd1 100644 --- a/akka-stm/src/test/java/akka/stm/example/RetryExample.java +++ b/akka-stm/src/test/java/akka/stm/example/RetryExample.java @@ -10,7 +10,7 @@ public class RetryExample { System.out.println("Retry example"); System.out.println(); - ActorSystem application = new ActorSystem("RetryExample"); + ActorSystem application = ActorSystem.create("RetryExample"); final Ref account1 = new Ref(100.0); final Ref account2 = new Ref(100.0); diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java index d6874ab6e6..dd8498ef42 100644 --- a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java +++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java @@ -12,7 +12,7 @@ public class UntypedCoordinatedExample { System.out.println("Untyped transactor example"); System.out.println(); - ActorSystem application = new ActorSystem("UntypedCoordinatedExample"); + ActorSystem application = ActorSystem.create("UntypedCoordinatedExample"); ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class)); ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class)); diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java b/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java index 1e3a5caf3c..0e425e4094 100644 --- a/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java +++ b/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java @@ -11,7 +11,7 @@ public class UntypedTransactorExample { System.out.println("Untyped transactor example"); System.out.println(); - ActorSystem application = new ActorSystem("UntypedTransactorExample"); + ActorSystem application = ActorSystem.create("UntypedTransactorExample"); ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCounter.class)); ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCounter.class)); diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java index 7c201ca103..e4d861f894 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java @@ -30,7 +30,7 @@ import scala.collection.JavaConverters; import scala.collection.Seq; public class UntypedCoordinatedIncrementTest { - ActorSystem application = new ActorSystem("UntypedCoordinatedIncrementTest"); + ActorSystem application = ActorSystem.create("UntypedCoordinatedIncrementTest"); List counters; ActorRef failer; diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java index 9c54a38b74..7de09c6697 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java @@ -27,7 +27,7 @@ import scala.collection.JavaConverters; import scala.collection.Seq; public class UntypedTransactorTest { - ActorSystem application = new ActorSystem("UntypedTransactorTest"); + ActorSystem application = ActorSystem.create("UntypedTransactorTest"); List counters; ActorRef failer; diff --git a/akka-stm/src/test/scala/config/ConfigSpec.scala b/akka-stm/src/test/scala/config/ConfigSpec.scala index 895605395a..b6d963c7ea 100644 --- a/akka-stm/src/test/scala/config/ConfigSpec.scala +++ b/akka-stm/src/test/scala/config/ConfigSpec.scala @@ -16,7 +16,7 @@ class ConfigSpec extends WordSpec with MustMatchers { "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 { - val config = ActorSystem("ConfigSpec").config + val config = ActorSystem("ConfigSpec").AkkaConfig.config import config._ diff --git a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala index c6dcb8fd31..e6e015a546 100644 --- a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala +++ b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala @@ -61,9 +61,9 @@ class CoordinatedIncrementSpec extends AkkaSpec with BeforeAndAfterAll { val numCounters = 4 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 failer = app.actorOf(Props(new Failer)) + val failer = system.actorOf(Props(new Failer)) (counters, failer) } diff --git a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala index ed4da08623..885ce0283c 100644 --- a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala +++ b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala @@ -104,9 +104,9 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll { val numCounters = 2 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 coordinator = app.actorOf(Props(new Coordinator("coordinator"))) + val coordinator = system.actorOf(Props(new Coordinator("coordinator"))) (counters, coordinator) } @@ -116,7 +116,7 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll { EventFilter[ExpectedFailureException](), EventFilter[CoordinatedTransactionException](), EventFilter[ActorTimeoutException]()) - app.eventStream.publish(Mute(ignoreExceptions)) + system.eventStream.publish(Mute(ignoreExceptions)) val (counters, coordinator) = actorOfs val latch = new CountDownLatch(1) coordinator ! FriendlyIncrement(counters, latch) diff --git a/akka-stm/src/test/scala/transactor/TransactorSpec.scala b/akka-stm/src/test/scala/transactor/TransactorSpec.scala index 84dc33c431..9dc195e39a 100644 --- a/akka-stm/src/test/scala/transactor/TransactorSpec.scala +++ b/akka-stm/src/test/scala/transactor/TransactorSpec.scala @@ -84,9 +84,9 @@ class TransactorSpec extends AkkaSpec { val numCounters = 3 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 failer = app.actorOf(Props(new Failer)) + val failer = system.actorOf(Props(new Failer)) (counters, failer) } @@ -124,7 +124,7 @@ class TransactorSpec extends AkkaSpec { "Transactor" should { "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 latch = TestLatch(1) transactor ! Set(ref, 5, latch) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 23fdddea2d..5905892157 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -22,7 +22,7 @@ import akka.event.EventStream * @since 1.1 */ class TestActorRef[T <: Actor]( - _app: ActorSystem, + _app: ActorSystemImpl, _deadLetterMailbox: Mailbox, _eventStream: EventStream, _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, 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] = { - 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) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index 3c0564331a..311968c451 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -37,7 +37,7 @@ import akka.event.EventStream * @since 1.2 */ class TestFSMRef[S, D, T <: Actor]( - app: ActorSystem, + app: ActorSystemImpl, _deadLetterMailbox: Mailbox, _eventStream: EventStream, _scheduler: Scheduler, @@ -89,9 +89,13 @@ class TestFSMRef[S, D, T <: Actor]( object TestFSMRef { - 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) + def apply[S, D, T <: Actor](factory: ⇒ T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = { + 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] = - new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () ⇒ factory), app.guardian, name) + def apply[S, D, T <: Actor](factory: ⇒ T, name: String)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = { + val impl = app.asInstanceOf[ActorSystemImpl] + new TestFSMRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(creator = () ⇒ factory), impl.guardian, name) + } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index fa7c20e000..98154a58ac 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -80,7 +80,7 @@ class TestKit(_app: ActorSystem) { import TestActor.{ Message, RealMessage, NullMessage } - implicit val app = _app + implicit val system = _app private val queue = new LinkedBlockingDeque[Message]() 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. * registration as message target. */ - val testActor: ActorRef = app.systemActorOf(Props(new TestActor(queue)) - .copy(dispatcher = new CallingThreadDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler)), - "testActor" + TestKit.testActorId.incrementAndGet) + val testActor: ActorRef = { + 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) + } 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 * 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. diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index 74a9e57d3b..a015bb247d 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -6,7 +6,7 @@ package akka.testkit import akka.config.Configuration import org.scalatest.{ WordSpec, BeforeAndAfterAll } import org.scalatest.matchers.MustMatchers -import akka.actor.ActorSystem +import akka.actor.{ ActorSystem, ActorSystemImpl } import akka.actor.{ Actor, ActorRef, Props } import akka.dispatch.MessageDispatcher import akka.event.{ Logging, LoggingAdapter } @@ -16,16 +16,16 @@ import akka.dispatch.FutureTimeoutException abstract class AkkaSpec(_application: ActorSystem = ActorSystem()) 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 { atStartup() } final override def afterAll { - app.stop() - try app.terminationFuture.await(5 seconds) catch { - case _: FutureTimeoutException ⇒ app.log.warning("failed to stop within 5 seconds") + system.stop() + try system.asInstanceOf[ActorSystemImpl].terminationFuture.await(5 seconds) catch { + case _: FutureTimeoutException ⇒ system.log.warning("failed to stop within 5 seconds") } atTermination() } @@ -34,9 +34,9 @@ abstract class AkkaSpec(_application: ActorSystem = ActorSystem()) 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)) diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 4d7fb0283d..4aa9ba69ff 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -170,8 +170,8 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach { counter = 2 val boss = TestActorRef(Props(new TActor { - - val ref = new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(new TActor { + val impl = system.asInstanceOf[ActorSystemImpl] + val ref = new TestActorRef(impl, impl.deadLetterMailbox, impl.eventStream, impl.scheduler, Props(new TActor { def receiveT = { case _ ⇒ } override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 } override def postRestart(reason: Throwable) { counter -= 1 } diff --git a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala index 3dd2415e20..19844cf225 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala @@ -15,7 +15,7 @@ class TestTimeSpec extends AkkaSpec(Configuration("akka.test.timefactor" -> 2.0) val now = System.nanoTime intercept[AssertionError] { probe.awaitCond(false, Duration("1 second")) } 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 + 1000000000l) }