From c3521a799d6d8510a2147fdd84c0667ac90fe823 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 14 Nov 2011 10:46:54 +0100 Subject: [PATCH 01/14] add comment in source for 85e37ea8efddac31c4b58028e5e73589abce82d8 --- akka-actor/src/main/scala/akka/actor/ActorRef.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 26a3ef7eed..00b8dcf386 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -173,6 +173,15 @@ class LocalActorRef private[akka] ( def address: String = app.address + path.toString + /* + * actorCell.start() publishes actorCell & this to the dispatcher, which + * means that messages may be processed theoretically before the constructor + * ends. The JMM guarantees visibility for final fields only after the end + * of the constructor, so publish the actorCell safely by making it a + * @volatile var which is NOT TO BE WRITTEN TO. The alternative would be to + * move start() outside of the constructor, which would basically require + * us to use purely factory methods for creating LocalActorRefs. + */ @volatile private var actorCell = new ActorCell(app, this, _props, _supervisor, _receiveTimeout, _hotswap) actorCell.start() From fc4598d71169a3689e5dc85c100a91e52352b764 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 14 Nov 2011 14:21:53 +0100 Subject: [PATCH 02/14] start clean-up of ActorSystem structure vs. initialization - move guardians, dead letters and deathWatch info provider - move scheduler into ActorSystem - keep forwarding methods in the first step to let the tests run --- .../src/main/scala/akka/actor/ActorRef.scala | 16 ++- .../scala/akka/actor/ActorRefProvider.scala | 103 +++++++++++++----- .../main/scala/akka/actor/ActorSystem.scala | 82 +++++--------- .../scala/akka/util/ReflectiveAccess.scala | 26 ----- .../akka/remote/RemoteActorRefProvider.scala | 22 ++-- 5 files changed, 123 insertions(+), 126 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 00b8dcf386..e6d470111e 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -12,6 +12,7 @@ import akka.serialization.Serialization import java.net.InetSocketAddress import akka.remote.RemoteAddress import java.util.concurrent.TimeUnit +import akka.event.EventStream /** * ActorRef is an immutable and serializable handle to an Actor. @@ -384,25 +385,22 @@ object DeadLetterActorRef { val serialized = new SerializedDeadLetterActorRef } -class DeadLetterActorRef(val app: ActorSystem) extends MinimalActorRef { - val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher) +class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath, val dispatcher: MessageDispatcher) extends MinimalActorRef { + val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(dispatcher) override val name: String = "dead-letter" - // FIXME (actor path): put this under the sys guardian supervisor - val path: ActorPath = app.root / "sys" / name - - def address: String = app.address + path.toString + def address: String = path.toString override def isShutdown(): Boolean = true override def tell(msg: Any, sender: ActorRef): Unit = msg match { - case d: DeadLetter ⇒ app.eventStream.publish(d) - case _ ⇒ app.eventStream.publish(DeadLetter(msg, sender, this)) + case d: DeadLetter ⇒ eventStream.publish(d) + case _ ⇒ eventStream.publish(DeadLetter(msg, sender, this)) } override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = { - app.eventStream.publish(DeadLetter(message, app.provider.dummyAskSender, this)) + eventStream.publish(DeadLetter(message, this, this)) brokenPromise } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 0bc355cf69..ae26fda28d 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -5,17 +5,13 @@ package akka.actor import java.util.concurrent.atomic.AtomicLong -import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.{ TimeUnit, Executors } - +import java.util.concurrent.{ ConcurrentHashMap, TimeUnit } import scala.annotation.tailrec - import org.jboss.netty.akka.util.{ TimerTask, HashedWheelTimer } - import akka.actor.Timeout.intToTimeout import akka.config.ConfigurationException -import akka.dispatch.{ SystemMessage, Supervise, Promise, MessageDispatcher, Future, DefaultPromise } -import akka.event.{ Logging, DeathWatch, ActorClassification } +import akka.dispatch.{ SystemMessage, Supervise, Promise, MessageDispatcher, Future, DefaultPromise, Dispatcher, Mailbox, Envelope } +import akka.event.{ Logging, DeathWatch, ActorClassification, EventStream } import akka.routing.{ ScatterGatherFirstCompletedRouter, Routing, RouterType, Router, RoutedProps, RoutedActorRef, RoundRobinRouter, RandomRouter, LocalConnectionManager, DirectRouter } import akka.util.Helpers import akka.AkkaException @@ -29,6 +25,16 @@ trait ActorRefProvider { def actorFor(path: Iterable[String]): Option[ActorRef] + def guardian: ActorRef + + def systemGuardian: ActorRef + + def deathWatch: DeathWatch + + def deadLetters: ActorRef + + def deadLetterMailbox: Mailbox + /** * What deployer will be used to resolve deployment configuration? */ @@ -54,8 +60,6 @@ trait ActorRefProvider { private[akka] def terminationFuture: Future[ActorSystem.ExitStatus] - private[akka] def dummyAskSender: ActorRef - private[akka] def tempPath: String } @@ -114,18 +118,43 @@ class ActorRefProviderException(message: String) extends AkkaException(message) /** * Local ActorRef provider. */ -class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider { +class LocalActorRefProvider( + private val app: ActorSystem, + val root: ActorPath, + val eventStream: EventStream, + val dispatcher: MessageDispatcher, + val scheduler: Scheduler) extends ActorRefProvider { - val log = Logging(app.eventStream, this) + val log = Logging(eventStream, this) private[akka] val deployer: Deployer = new Deployer(app) val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(app.dispatcher) - private[akka] val scheduler: Scheduler = { //TODO FIXME Make this configurable - val s = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512)) - terminationFuture.onComplete(_ ⇒ s.stop()) - s + /* + * generate name for temporary actor refs + */ + private val tempNumber = new AtomicLong + def tempPath = { + val l = tempNumber.getAndIncrement() + "$_" + Helpers.base64(l) + } + + // 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) + private val actors = new ConcurrentHashMap[String, AnyRef] + + val deadLetters = new DeadLetterActorRef(eventStream, root / "nul", dispatcher) + val deadLetterMailbox = new Mailbox(null) { + becomeClosed() + override def dispatcher = null //MessageDispatcher.this + override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } + override def dequeue() = null + override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) } + override def systemDrain(): SystemMessage = null + override def hasMessages = false + override def hasSystemMessages = false + override def numberOfMessages = 0 } /** @@ -163,9 +192,39 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider { } } - // 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) - private val actors = new ConcurrentHashMap[String, AnyRef] + private class Guardian extends Actor { + def receive = { + case Terminated(_) ⇒ context.self.stop() + } + } + private class SystemGuardian extends Actor { + def receive = { + case Terminated(_) ⇒ + eventStream.stopDefaultLoggers() + context.self.stop() + } + } + private val guardianFaultHandlingStrategy = { + import akka.actor.FaultHandlingStrategy._ + OneForOneStrategy { + case _: ActorKilledException ⇒ Stop + case _: ActorInitializationException ⇒ Stop + case _: Exception ⇒ Restart + } + } + 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) + + val deathWatch = createDeathWatch() + + // 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) @@ -286,14 +345,6 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider { a.result } } - - private[akka] val dummyAskSender = new DeadLetterActorRef(app) - - private val tempNumber = new AtomicLong - def tempPath = { - val l = tempNumber.getAndIncrement() - "$_" + Helpers.base64(l) - } } class LocalDeathWatch extends DeathWatch with ActorClassification { diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 1ceb357c76..b9cd41bace 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -6,14 +6,17 @@ package akka.actor import akka.config._ import akka.actor._ import akka.event._ +import akka.dispatch._ import akka.util.duration._ import java.net.InetAddress import com.eaio.uuid.UUID -import akka.dispatch.{ Dispatchers, Future, Mailbox, Envelope, SystemMessage } import akka.util.Duration import akka.util.ReflectiveAccess import akka.serialization.Serialization import akka.remote.RemoteAddress +import org.jboss.netty.akka.util.HashedWheelTimer +import java.util.concurrent.Executors +import java.util.concurrent.TimeUnit object ActorSystem { @@ -162,10 +165,7 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher - def scheduler = provider.scheduler - - // TODO think about memory consistency effects when doing funky stuff inside constructor - val reflective = new ReflectiveAccess(this) + val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512)) /** * The root actor path for this application. @@ -173,60 +173,29 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF val root: ActorPath = new RootActorPath(this) // TODO think about memory consistency effects when doing funky stuff inside constructor - val provider: ActorRefProvider = reflective.createProvider + 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[ActorPath], classOf[EventStream], classOf[MessageDispatcher], classOf[Scheduler]) + val args: Array[AnyRef] = Array(this, root, eventStream, dispatcher, scheduler) + + ReflectiveAccess.createInstance[ActorRefProvider](providerClass, params, args) match { + case Left(e) ⇒ throw e + case Right(p) ⇒ p + } + } def terminationFuture: Future[ExitStatus] = provider.terminationFuture + def guardian: ActorRef = provider.guardian + def systemGuardian: ActorRef = provider.systemGuardian + def deathWatch: DeathWatch = provider.deathWatch + def deadLetters: ActorRef = provider.deadLetters + def deadLetterMailbox: Mailbox = provider.deadLetterMailbox - private class Guardian extends Actor { - def receive = { - case Terminated(_) ⇒ context.self.stop() - } - } - private class SystemGuardian extends Actor { - def receive = { - case Terminated(_) ⇒ - eventStream.stopDefaultLoggers() - context.self.stop() - } - } - private val guardianFaultHandlingStrategy = { - import akka.actor.FaultHandlingStrategy._ - OneForOneStrategy { - case _: ActorKilledException ⇒ Stop - case _: ActorInitializationException ⇒ Stop - case _: Exception ⇒ Restart - } - } - private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy) - - private val rootGuardian: ActorRef = - provider.actorOf(guardianProps, provider.theOneWhoWalksTheBubblesOfSpaceTime, root, true) - - protected[akka] val guardian: ActorRef = - provider.actorOf(guardianProps, rootGuardian, "app", true) - - protected[akka] val systemGuardian: ActorRef = - provider.actorOf(guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true) - - // TODO think about memory consistency effects when doing funky stuff inside constructor - val deadLetters = new DeadLetterActorRef(this) - val deadLetterMailbox = new Mailbox(null) { - becomeClosed() - override def dispatcher = null //MessageDispatcher.this - override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } - override def dequeue() = null - override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) } - override def systemDrain(): SystemMessage = null - override def hasMessages = false - override def hasSystemMessages = false - override def numberOfMessages = 0 - } - - val deathWatch = provider.createDeathWatch() - - // chain death watchers so that killing guardian stops the application - deathWatch.subscribe(systemGuardian, guardian) - deathWatch.subscribe(rootGuardian, systemGuardian) + 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(this) @@ -251,5 +220,4 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF guardian.stop() } - terminationFuture.onComplete(_ ⇒ dispatcher.shutdown()) } diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala index 285cdaeeb9..d331190de6 100644 --- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala @@ -112,29 +112,3 @@ object ReflectiveAccess { } -/** - * Helper class for reflective access to different modules in order to allow optional loading of modules. - * - * @author Jonas Bonér - */ -class ReflectiveAccess(val app: ActorSystem) { - - import ReflectiveAccess._ - - def providerClass: Class[_] = { - getClassFor(app.AkkaConfig.ProviderClass) match { - case Left(e) ⇒ throw e - case Right(b) ⇒ b - } - } - - def createProvider: ActorRefProvider = { - val params: Array[Class[_]] = Array(classOf[ActorSystem]) - val args: Array[AnyRef] = Array(app) - - createInstance[ActorRefProvider](providerClass, params, args) match { - case Right(p) ⇒ p - case Left(e) ⇒ throw e - } - } -} diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 1eeeb2ccf5..5f0223176e 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -17,26 +17,36 @@ import akka.serialization.{ Serialization, Serializer, Compression } import akka.serialization.Compression.LZF import akka.remote.RemoteProtocol._ import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._ - import java.net.InetSocketAddress import java.util.concurrent.ConcurrentHashMap - import com.google.protobuf.ByteString import java.util.concurrent.atomic.AtomicBoolean +import akka.event.EventStream /** * Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it. * * @author Jonas Bonér */ -class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider { +class RemoteActorRefProvider( + val app: ActorSystem, + val root: ActorPath, + val eventStream: EventStream, + val dispatcher: MessageDispatcher, + val scheduler: Scheduler) extends ActorRefProvider { val log = Logging(app, this) import java.util.concurrent.ConcurrentHashMap import akka.dispatch.Promise - val local = new LocalActorRefProvider(app) + val local = new LocalActorRefProvider(app, root, eventStream, dispatcher, scheduler) + def deadLetterMailbox = local.deadLetterMailbox + def deadLetters = local.deadLetters + def deathWatch = local.deathWatch + def guardian = local.guardian + def systemGuardian = local.systemGuardian + val remote = new Remote(app) private val actors = new ConcurrentHashMap[String, AnyRef] @@ -51,8 +61,6 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider { def defaultDispatcher = app.dispatcher def defaultTimeout = app.AkkaConfig.ActorTimeout - def scheduler: Scheduler = local.scheduler - private[akka] def actorOf(props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = actorOf(props, supervisor, supervisor.path / name, systemService) @@ -242,8 +250,6 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider { private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = local.ask(message, recipient, within) - private[akka] def dummyAskSender = local.dummyAskSender - private[akka] def tempPath = local.tempPath } From 79daccdce1f5e34bd1526613cb023472eb3bc24a Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 10 Nov 2011 19:17:52 +0100 Subject: [PATCH 03/14] move AkkaConfig into ActorSystem companion object as normal class to make it easier to pass around. also remove MistSettings --- .../main/scala/akka/actor/ActorSystem.scala | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index b9cd41bace..f0fd0f5854 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -15,13 +15,10 @@ import akka.util.ReflectiveAccess import akka.serialization.Serialization import akka.remote.RemoteAddress import org.jboss.netty.akka.util.HashedWheelTimer -import java.util.concurrent.Executors -import java.util.concurrent.TimeUnit +import java.util.concurrent.{ Executors, TimeUnit } object ActorSystem { - type AkkaConfig = a.AkkaConfig.type forSome { val a: ActorSystem } - val Version = "2.0-SNAPSHOT" val envHome = System.getenv("AKKA_HOME") match { @@ -74,16 +71,7 @@ object ActorSystem { case object Stopped extends ExitStatus case class Failed(cause: Throwable) extends ExitStatus -} - -class ActorSystem(val name: String, val config: Configuration) extends ActorRefFactory with TypedActorFactory { - - def this(name: String) = this(name, ActorSystem.defaultConfig) - def this() = this("default") - - import ActorSystem._ - - object AkkaConfig { + class AkkaConfig(val config: Configuration) { import config._ val ConfigVersion = getString("akka.version", Version) @@ -129,16 +117,28 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF val FailureDetectorThreshold: Int = getInt("akka.remote.failure-detector.threshold", 8) val FailureDetectorMaxSampleSize: Int = getInt("akka.remote.failure-detector.max-sample-size", 1000) + + if (ConfigVersion != Version) + throw new ConfigurationException("Akka JAR version [" + Version + + "] does not match the provided config version [" + ConfigVersion + "]") + } +} + +class ActorSystem(val name: String, val config: Configuration) extends ActorRefFactory with TypedActorFactory { + + import ActorSystem._ + + def this(name: String) = this(name, ActorSystem.defaultConfig) + def this() = this("default") + + val AkkaConfig = new AkkaConfig(config) + private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(props, systemGuardian, address, true) import AkkaConfig._ - if (ConfigVersion != Version) - throw new ConfigurationException("Akka JAR version [" + Version + - "] does not match the provided config version [" + ConfigVersion + "]") - val startTime = System.currentTimeMillis def uptime = (System.currentTimeMillis - startTime) / 1000 From f2bf27b4de1bfbe1b4e2a46e53f54126147c802d Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 14 Nov 2011 16:03:26 +0100 Subject: [PATCH 04/14] remove app argument from Dispatchers (cascading into all dispatchers, mailboxes, other stuff; had to move deadLetter stuff to ActorSystem again and split its initialization due to cyclic dependency) --- .../scala/akka/actor/LoggingReceiveSpec.scala | 2 +- .../akka/actor/dispatch/ActorModelSpec.scala | 8 +- .../CallingThreadDispatcherModelSpec.scala | 2 +- .../src/main/scala/akka/actor/ActorRef.scala | 11 ++- .../scala/akka/actor/ActorRefProvider.scala | 17 ----- .../main/scala/akka/actor/ActorSystem.scala | 32 +++++--- .../akka/dispatch/AbstractDispatcher.scala | 37 ++++----- .../akka/dispatch/BalancingDispatcher.scala | 10 ++- .../main/scala/akka/dispatch/Dispatcher.scala | 12 ++- .../scala/akka/dispatch/Dispatchers.scala | 76 ++++++++++--------- .../src/main/scala/akka/dispatch/Future.scala | 22 +++--- .../akka/dispatch/PinnedDispatcher.scala | 15 +++- .../akka/dispatch/ThreadPoolBuilder.scala | 11 +-- .../akka/remote/RemoteActorRefProvider.scala | 2 - .../src/main/scala/akka/agent/Agent.scala | 4 +- .../testkit/CallingThreadDispatcher.scala | 12 ++- .../scala/akka/testkit/TestActorRef.scala | 15 +++- .../main/scala/akka/testkit/TestFSMRef.scala | 17 ++++- .../src/main/scala/akka/testkit/TestKit.scala | 3 +- .../scala/akka/testkit/TestActorRefSpec.scala | 2 +- 20 files changed, 180 insertions(+), 130 deletions(-) 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 10e9235c89..b1f23e60e9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala @@ -150,7 +150,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd expectNoMsg(Duration.Zero) assert(supervisorSet == Set(1, 2), supervisorSet + " was not Set(1, 2)") - val actor = new TestActorRef[TestLogActor](app, Props[TestLogActor], supervisor, "none") + val actor = TestActorRef[TestLogActor](Props[TestLogActor], supervisor, "none") val set = receiveWhile(messages = 2) { case Logging.Debug(`supervisor`, msg: String) if msg startsWith "now supervising" ⇒ 1 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 8d1230a3df..6f50112362 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 @@ -421,10 +421,10 @@ class DispatcherModelSpec extends ActorModelSpec { import ActorModelSpec._ def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒ - new Dispatcher(app, "foo", app.AkkaConfig.DispatcherThroughput, + 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)).build.asInstanceOf[MessageDispatcherInterceptor] + ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor] def dispatcherType = "Dispatcher" @@ -458,10 +458,10 @@ class BalancingDispatcherModelSpec extends ActorModelSpec { import ActorModelSpec._ def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒ - new BalancingDispatcher(app, "foo", 1, // TODO check why 1 here? (came from old test) + 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)).build.asInstanceOf[MessageDispatcherInterceptor] + ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor] def dispatcherType = "Balancing Dispatcher" 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 bd3816417e..e82d6d9dcb 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, "test") with MessageDispatcherInterceptor + def newInterceptedDispatcher = new CallingThreadDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "test") with MessageDispatcherInterceptor def dispatcherType = "Calling Thread Dispatcher" } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index e6d470111e..20feee1e02 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -385,8 +385,13 @@ object DeadLetterActorRef { val serialized = new SerializedDeadLetterActorRef } -class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath, val dispatcher: MessageDispatcher) extends MinimalActorRef { - val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(dispatcher) +class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) extends MinimalActorRef { + @volatile + var brokenPromise: Future[Any] = _ + + private[akka] def init(dispatcher: MessageDispatcher) { + brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(dispatcher) + } override val name: String = "dead-letter" @@ -401,6 +406,8 @@ class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath, val override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = { eventStream.publish(DeadLetter(message, this, this)) + // leave this in: guard with good visibility against really stupid/weird errors + assert(brokenPromise != null) brokenPromise } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index ae26fda28d..95db57ce78 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -31,10 +31,6 @@ trait ActorRefProvider { def deathWatch: DeathWatch - def deadLetters: ActorRef - - def deadLetterMailbox: Mailbox - /** * What deployer will be used to resolve deployment configuration? */ @@ -144,19 +140,6 @@ class LocalActorRefProvider( // currently still used for tmp actors (e.g. ask actor refs) private val actors = new ConcurrentHashMap[String, AnyRef] - val deadLetters = new DeadLetterActorRef(eventStream, root / "nul", dispatcher) - val deadLetterMailbox = new Mailbox(null) { - becomeClosed() - override def dispatcher = null //MessageDispatcher.this - override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } - override def dequeue() = null - override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) } - override def systemDrain(): SystemMessage = null - override def hasMessages = false - override def hasSystemMessages = false - override def numberOfMessages = 0 - } - /** * Top-level anchor for the supervision hierarchy of this actor system. Will * receive only Supervise/ChildTerminated system messages or Failure message. diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index f0fd0f5854..43dfa99837 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -158,20 +158,34 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF // this provides basic logging (to stdout) until .start() is called below val eventStream = new EventStream(DebugEventStream) eventStream.startStdoutLogger(AkkaConfig) - val log = new BusLogging(eventStream, this) - - // TODO correctly pull its config from the config - val dispatcherFactory = new Dispatchers(this) - - implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher - - val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512)) + val log = new BusLogging(eventStream, this) // “this” used only for .getClass in tagging messages /** * The root actor path for this application. */ val root: ActorPath = new RootActorPath(this) + val deadLetters = new DeadLetterActorRef(eventStream, root / "nul") + val deadLetterMailbox = new Mailbox(null) { + becomeClosed() + override def dispatcher = null //MessageDispatcher.this + override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } + override def dequeue() = null + override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) } + override def systemDrain(): SystemMessage = null + override def hasMessages = false + override def hasSystemMessages = false + override def numberOfMessages = 0 + } + + val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512)) + + // TODO correctly pull its config from the config + val dispatcherFactory = new Dispatchers(AkkaConfig, eventStream, deadLetterMailbox, scheduler) + implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher + + deadLetters.init(dispatcher) + // TODO think about memory consistency effects when doing funky stuff inside constructor val provider: ActorRefProvider = { val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match { @@ -191,8 +205,6 @@ 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 deadLetters: ActorRef = provider.deadLetters - def deadLetterMailbox: Mailbox = provider.deadLetterMailbox terminationFuture.onComplete(_ ⇒ scheduler.stop()) terminationFuture.onComplete(_ ⇒ dispatcher.shutdown()) diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index d7945f3409..4df4eeac39 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -13,6 +13,8 @@ import java.util.concurrent.ThreadPoolExecutor.{ AbortPolicy, CallerRunsPolicy, import akka.actor._ import akka.actor.ActorSystem import scala.annotation.tailrec +import akka.event.EventStream +import akka.actor.ActorSystem.AkkaConfig /** * @author Jonas Bonér @@ -61,12 +63,12 @@ case class Supervise(child: ActorRef) extends SystemMessage // sent to superviso case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsMonitoring case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsMonitoring -final case class TaskInvocation(app: ActorSystem, function: () ⇒ Unit, cleanup: () ⇒ Unit) extends Runnable { +final case class TaskInvocation(eventStream: EventStream, function: () ⇒ Unit, cleanup: () ⇒ Unit) extends Runnable { def run() { try { function() } catch { - case e ⇒ app.eventStream.publish(Error(e, this, e.getMessage)) + case e ⇒ eventStream.publish(Error(e, this, e.getMessage)) } finally { cleanup() } @@ -84,7 +86,11 @@ object MessageDispatcher { /** * @author Jonas Bonér */ -abstract class MessageDispatcher(val app: ActorSystem) extends Serializable { +abstract class MessageDispatcher( + val deadLetterMailbox: Mailbox, + val eventStream: EventStream, + val scheduler: Scheduler) extends Serializable { + import MessageDispatcher._ protected val _tasks = new AtomicLong(0L) @@ -99,11 +105,6 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable { */ protected[akka] def createMailbox(actor: ActorCell): Mailbox - /** - * a blackhole mailbox for the purpose of replacing the real one upon actor termination - */ - import app.deadLetterMailbox - /** * Name of this dispatcher. */ @@ -133,7 +134,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable { shutdownSchedule match { case UNSCHEDULED ⇒ shutdownSchedule = SCHEDULED - app.scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) + scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) case SCHEDULED ⇒ shutdownSchedule = RESCHEDULED case RESCHEDULED ⇒ //Already marked for reschedule @@ -154,7 +155,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable { _tasks.getAndIncrement() try { startIfUnstarted() - executeTask(TaskInvocation(app, block, taskCleanup)) + executeTask(TaskInvocation(eventStream, block, taskCleanup)) } catch { case e ⇒ _tasks.decrementAndGet @@ -170,7 +171,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable { shutdownSchedule match { case UNSCHEDULED ⇒ shutdownSchedule = SCHEDULED - app.scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) + scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) case SCHEDULED ⇒ shutdownSchedule = RESCHEDULED case RESCHEDULED ⇒ //Already marked for reschedule @@ -234,7 +235,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable { shutdownSchedule match { case RESCHEDULED ⇒ shutdownSchedule = SCHEDULED - app.scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) + scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) case SCHEDULED ⇒ if (_tasks.get == 0) { active switchOff { @@ -329,19 +330,19 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable { /** * Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig */ -abstract class MessageDispatcherConfigurator(val app: ActorSystem) { +abstract class MessageDispatcherConfigurator(val AkkaConfig: AkkaConfig, val eventStream: EventStream) { /** * Returns an instance of MessageDispatcher given a Configuration */ def configure(config: Configuration): MessageDispatcher def mailboxType(config: Configuration): MailboxType = { - val capacity = config.getInt("mailbox-capacity", app.AkkaConfig.MailboxCapacity) + val capacity = config.getInt("mailbox-capacity", AkkaConfig.MailboxCapacity) if (capacity < 1) UnboundedMailbox() else { val duration = Duration( - config.getInt("mailbox-push-timeout-time", app.AkkaConfig.MailboxPushTimeout.toMillis.toInt), - app.AkkaConfig.DefaultTimeUnit) + config.getInt("mailbox-push-timeout-time", AkkaConfig.MailboxPushTimeout.toMillis.toInt), + AkkaConfig.DefaultTimeUnit) BoundedMailbox(capacity, duration) } } @@ -350,8 +351,8 @@ abstract class MessageDispatcherConfigurator(val app: ActorSystem) { import ThreadPoolConfigDispatcherBuilder.conf_? //Apply the following options to the config if they are present in the config - ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(app)).configure( - conf_?(config getInt "keep-alive-time")(time ⇒ _.setKeepAliveTime(Duration(time, app.AkkaConfig.DefaultTimeUnit))), + ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(eventStream)).configure( + conf_?(config getInt "keep-alive-time")(time ⇒ _.setKeepAliveTime(Duration(time, AkkaConfig.DefaultTimeUnit))), conf_?(config getDouble "core-pool-size-factor")(factor ⇒ _.setCorePoolSizeFromFactor(factor)), conf_?(config getDouble "max-pool-size-factor")(factor ⇒ _.setMaxPoolSizeFromFactor(factor)), conf_?(config getInt "executor-bounds")(bounds ⇒ _.setExecutorBounds(bounds)), diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index b64b7ee513..7f614be5a8 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -10,6 +10,8 @@ import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, Concur import java.util.{ Comparator, Queue } import annotation.tailrec import akka.actor.ActorSystem +import akka.event.EventStream +import akka.actor.Scheduler /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -28,16 +30,16 @@ import akka.actor.ActorSystem * @author Viktor Klang */ class BalancingDispatcher( - _app: ActorSystem, + _deadLetterMailbox: Mailbox, + _eventStream: EventStream, + _scheduler: Scheduler, _name: String, throughput: Int, throughputDeadlineTime: Int, mailboxType: MailboxType, config: ThreadPoolConfig, _timeoutMs: Long) - extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) { - - import app.deadLetterMailbox + extends Dispatcher(_deadLetterMailbox, _eventStream, _scheduler, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) { private val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator) diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index bf12a5cafc..8fa84eafb6 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -9,6 +9,8 @@ import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue } import akka.actor.{ ActorCell, ActorKilledException } import akka.actor.ActorSystem +import akka.event.EventStream +import akka.actor.Scheduler /** * Default settings are: @@ -64,14 +66,16 @@ import akka.actor.ActorSystem * Larger values (or zero or negative) increase throughput, smaller values increase fairness */ class Dispatcher( - _app: ActorSystem, + _deadLetterMailbox: Mailbox, + _eventStream: EventStream, + _scheduler: Scheduler, val name: String, val throughput: Int, val throughputDeadlineTime: Int, val mailboxType: MailboxType, executorServiceFactoryProvider: ExecutorServiceFactoryProvider, val timeoutMs: Long) - extends MessageDispatcher(_app) { + extends MessageDispatcher(_deadLetterMailbox, _eventStream, _scheduler) { protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name) protected[akka] val executorService = new AtomicReference[ExecutorService](new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService)) @@ -93,7 +97,7 @@ class Dispatcher( executorService.get() execute invocation } catch { case e: RejectedExecutionException ⇒ - app.eventStream.publish(Warning(this, e.toString)) + eventStream.publish(Warning(this, e.toString)) throw e } } @@ -120,7 +124,7 @@ class Dispatcher( } catch { case e: RejectedExecutionException ⇒ try { - app.eventStream.publish(Warning(this, e.toString)) + eventStream.publish(Warning(this, e.toString)) } finally { mbox.setAsIdle() } diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index 2797652f77..654c3b338e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -10,6 +10,9 @@ import akka.util.{ Duration, ReflectiveAccess } import akka.config.Configuration import java.util.concurrent.TimeUnit import akka.actor.ActorSystem +import akka.event.EventStream +import akka.actor.Scheduler +import akka.actor.ActorSystem.AkkaConfig /** * Scala API. Dispatcher factory. @@ -43,15 +46,20 @@ import akka.actor.ActorSystem * * @author Jonas Bonér */ -class Dispatchers(val app: ActorSystem) { - val ThroughputDeadlineTimeMillis = app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt +class Dispatchers( + val AkkaConfig: ActorSystem.AkkaConfig, + val eventStream: EventStream, + val deadLetterMailbox: Mailbox, + val scheduler: Scheduler) { + + val ThroughputDeadlineTimeMillis = AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt val MailboxType: MailboxType = - if (app.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox() - else BoundedMailbox(app.AkkaConfig.MailboxCapacity, app.AkkaConfig.MailboxPushTimeout) - val DispatcherShutdownMillis = app.AkkaConfig.DispatcherDefaultShutdown.toMillis + if (AkkaConfig.MailboxCapacity < 1) UnboundedMailbox() + else BoundedMailbox(AkkaConfig.MailboxCapacity, AkkaConfig.MailboxPushTimeout) + val DispatcherShutdownMillis = AkkaConfig.DispatcherDefaultShutdown.toMillis lazy val defaultGlobalDispatcher = - app.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build + AkkaConfig.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build /** * Creates an thread based dispatcher serving a single actor through the same single thread. @@ -60,8 +68,8 @@ class Dispatchers(val app: ActorSystem) { * E.g. each actor consumes its own thread. */ def newPinnedDispatcher(actor: LocalActorRef) = actor match { - case null ⇒ new PinnedDispatcher(app, null, "anon", MailboxType, DispatcherShutdownMillis) - case some ⇒ new PinnedDispatcher(app, some.underlying, some.address, MailboxType, DispatcherShutdownMillis) + case null ⇒ new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, null, "anon", MailboxType, DispatcherShutdownMillis) + case some ⇒ new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, some.underlying, some.address, MailboxType, DispatcherShutdownMillis) } /** @@ -71,8 +79,8 @@ class Dispatchers(val app: ActorSystem) { * E.g. each actor consumes its own thread. */ def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match { - case null ⇒ new PinnedDispatcher(app, null, "anon", mailboxType, DispatcherShutdownMillis) - case some ⇒ new PinnedDispatcher(app, some.underlying, some.address, mailboxType, DispatcherShutdownMillis) + case null ⇒ new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, null, "anon", mailboxType, DispatcherShutdownMillis) + case some ⇒ new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, some.underlying, some.address, mailboxType, DispatcherShutdownMillis) } /** @@ -81,7 +89,7 @@ class Dispatchers(val app: ActorSystem) { * E.g. each actor consumes its own thread. */ def newPinnedDispatcher(name: String, mailboxType: MailboxType) = - new PinnedDispatcher(app, null, name, mailboxType, DispatcherShutdownMillis) + new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, null, name, mailboxType, DispatcherShutdownMillis) /** * Creates an thread based dispatcher serving a single actor through the same single thread. @@ -89,7 +97,7 @@ class Dispatchers(val app: ActorSystem) { * E.g. each actor consumes its own thread. */ def newPinnedDispatcher(name: String) = - new PinnedDispatcher(app, null, name, MailboxType, DispatcherShutdownMillis) + new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, null, name, MailboxType, DispatcherShutdownMillis) /** * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. @@ -97,8 +105,8 @@ class Dispatchers(val app: ActorSystem) { * Has a fluent builder interface for configuring its semantics. */ def newDispatcher(name: String) = - ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(app, name, app.AkkaConfig.DispatcherThroughput, - ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) + ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, AkkaConfig.DispatcherThroughput, + ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream)) /** * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. @@ -107,7 +115,7 @@ class Dispatchers(val app: ActorSystem) { */ def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new Dispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) + new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream)) /** * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. @@ -116,7 +124,7 @@ class Dispatchers(val app: ActorSystem) { */ def newDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new Dispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) + new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream)) /** * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. @@ -124,8 +132,8 @@ class Dispatchers(val app: ActorSystem) { * Has a fluent builder interface for configuring its semantics. */ def newBalancingDispatcher(name: String) = - ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(app, name, app.AkkaConfig.DispatcherThroughput, - ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) + ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, AkkaConfig.DispatcherThroughput, + ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream)) /** * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. @@ -134,7 +142,7 @@ class Dispatchers(val app: ActorSystem) { */ def newBalancingDispatcher(name: String, throughput: Int) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) + new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream)) /** * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. @@ -143,7 +151,7 @@ class Dispatchers(val app: ActorSystem) { */ def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) + new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream)) /** * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. @@ -152,13 +160,13 @@ class Dispatchers(val app: ActorSystem) { */ def newBalancingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new BalancingDispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) + new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream)) /** * Utility function that tries to load the specified dispatcher config from the akka.conf * or else use the supplied default dispatcher */ def fromConfig(key: String, default: ⇒ MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher = - app.config getSection key flatMap from getOrElse default + AkkaConfig.config getSection key flatMap from getOrElse default /* * Creates of obtains a dispatcher from a ConfigMap according to the format below @@ -185,8 +193,8 @@ class Dispatchers(val app: ActorSystem) { */ def from(cfg: Configuration): Option[MessageDispatcher] = { cfg.getString("type") flatMap { - case "Dispatcher" ⇒ Some(new DispatcherConfigurator(app)) - case "BalancingDispatcher" ⇒ Some(new BalancingDispatcherConfigurator(app)) + case "Dispatcher" ⇒ Some(new DispatcherConfigurator(AkkaConfig, deadLetterMailbox, eventStream, scheduler)) + case "BalancingDispatcher" ⇒ Some(new BalancingDispatcherConfigurator(AkkaConfig, deadLetterMailbox, eventStream, scheduler)) case "GlobalDispatcher" ⇒ None //TODO FIXME remove this case fqn ⇒ ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match { @@ -206,26 +214,26 @@ class Dispatchers(val app: ActorSystem) { } } -class DispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) { +class DispatcherConfigurator(AkkaConfig: AkkaConfig, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(AkkaConfig, eventStream) { def configure(config: Configuration): MessageDispatcher = { - configureThreadPool(config, threadPoolConfig ⇒ new Dispatcher(app, + configureThreadPool(config, threadPoolConfig ⇒ new Dispatcher(deadLetterMailbox, eventStream, scheduler, config.getString("name", newUuid.toString), - config.getInt("throughput", app.AkkaConfig.DispatcherThroughput), - config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), + config.getInt("throughput", AkkaConfig.DispatcherThroughput), + config.getInt("throughput-deadline-time", AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), mailboxType(config), threadPoolConfig, - app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build + AkkaConfig.DispatcherDefaultShutdown.toMillis)).build } } -class BalancingDispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) { +class BalancingDispatcherConfigurator(AkkaConfig: AkkaConfig, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(AkkaConfig, eventStream) { def configure(config: Configuration): MessageDispatcher = { - configureThreadPool(config, threadPoolConfig ⇒ new BalancingDispatcher(app, + configureThreadPool(config, threadPoolConfig ⇒ new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, config.getString("name", newUuid.toString), - config.getInt("throughput", app.AkkaConfig.DispatcherThroughput), - config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), + config.getInt("throughput", AkkaConfig.DispatcherThroughput), + config.getInt("throughput-deadline-time", AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), mailboxType(config), threadPoolConfig, - app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build + AkkaConfig.DispatcherDefaultShutdown.toMillis)).build } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 990f3832f1..d62543af4c 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -262,7 +262,7 @@ object Future { result completeWithResult currentValue } catch { case e: Exception ⇒ - dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) + dispatcher.eventStream.publish(Error(e, this, e.getMessage)) result completeWithException e } finally { results.clear @@ -631,7 +631,7 @@ sealed trait Future[+T] extends japi.Future[T] { Right(f(res)) } catch { case e: Exception ⇒ - dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) + dispatcher.eventStream.publish(Error(e, this, e.getMessage)) Left(e) }) } @@ -683,7 +683,7 @@ sealed trait Future[+T] extends japi.Future[T] { future.completeWith(f(r)) } catch { case e: Exception ⇒ - dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) + dispatcher.eventStream.publish(Error(e, this, e.getMessage)) future complete Left(e) } } @@ -716,7 +716,7 @@ sealed trait Future[+T] extends japi.Future[T] { if (p(res)) r else Left(new MatchError(res)) } catch { case e: Exception ⇒ - dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) + dispatcher.eventStream.publish(Error(e, this, e.getMessage)) Left(e) }) } @@ -811,7 +811,7 @@ trait Promise[T] extends Future[T] { fr completeWith cont(f) } catch { case e: Exception ⇒ - dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) + dispatcher.eventStream.publish(Error(e, this, e.getMessage)) fr completeWithException e } } @@ -825,7 +825,7 @@ trait Promise[T] extends Future[T] { fr completeWith cont(f) } catch { case e: Exception ⇒ - dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) + dispatcher.eventStream.publish(Error(e, this, e.getMessage)) fr completeWithException e } } @@ -979,12 +979,12 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi val runnable = new Runnable { def run() { if (!isCompleted) { - if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) + if (!isExpired) dispatcher.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) else func(DefaultPromise.this) } } } - val timeoutFuture = dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS) + val timeoutFuture = dispatcher.scheduler.scheduleOnce(runnable, timeLeft(), NANOS) onComplete(_ ⇒ timeoutFuture.cancel()) false } else true @@ -1006,18 +1006,18 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi val runnable = new Runnable { def run() { if (!isCompleted) { - if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) + if (!isExpired) dispatcher.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) else promise complete (try { Right(fallback) } catch { case e ⇒ Left(e) }) } } } - dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS) + dispatcher.scheduler.scheduleOnce(runnable, timeLeft(), NANOS) promise } } else this private def notifyCompleted(func: Future[T] ⇒ Unit) { - try { func(this) } catch { case e ⇒ dispatcher.app.eventStream.publish(Error(e, this, "Future onComplete-callback raised an exception")) } //TODO catch, everything? Really? + try { func(this) } catch { case e ⇒ dispatcher.eventStream.publish(Error(e, this, "Future onComplete-callback raised an exception")) } //TODO catch, everything? Really? } @inline diff --git a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala index f40fe953d0..2bf36dc375 100644 --- a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala @@ -7,14 +7,23 @@ package akka.dispatch import java.util.concurrent.atomic.AtomicReference import akka.actor.ActorCell import akka.actor.ActorSystem +import akka.event.EventStream +import akka.actor.Scheduler /** * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. * * @author Jonas Bonér */ -class PinnedDispatcher(_app: ActorSystem, _actor: ActorCell, _name: String, _mailboxType: MailboxType, _timeoutMs: Long) - extends Dispatcher(_app, _name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread(_app), _timeoutMs) { +class PinnedDispatcher( + _deadLetterMailbox: Mailbox, + _eventStream: EventStream, + _scheduler: Scheduler, + _actor: ActorCell, + _name: String, + _mailboxType: MailboxType, + _timeoutMs: Long) + extends Dispatcher(_deadLetterMailbox, _eventStream, _scheduler, _name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread(_eventStream), _timeoutMs) { @volatile protected[akka] var owner: ActorCell = _actor @@ -34,6 +43,6 @@ class PinnedDispatcher(_app: ActorSystem, _actor: ActorCell, _name: String, _mai } object PinnedDispatcher { - def oneThread(app: ActorSystem): ThreadPoolConfig = ThreadPoolConfig(app, allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1) + def oneThread(eventStream: EventStream): ThreadPoolConfig = ThreadPoolConfig(eventStream, allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1) } diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index b1a9547ccf..867f913d75 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -11,6 +11,7 @@ import ThreadPoolExecutor.CallerRunsPolicy import akka.util.Duration import akka.event.Logging.{ Warning, Error } import akka.actor.ActorSystem +import akka.event.EventStream object ThreadPoolConfig { type Bounds = Int @@ -68,7 +69,7 @@ trait ExecutorServiceFactoryProvider { /** * A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher */ -case class ThreadPoolConfig(app: ActorSystem, +case class ThreadPoolConfig(eventStream: EventStream, allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout, corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize, maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize, @@ -86,7 +87,7 @@ case class ThreadPoolConfig(app: ActorSystem, case Right(bounds) ⇒ val service = new ThreadPoolExecutor(corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory) service.allowCoreThreadTimeOut(allowCorePoolTimeout) - new BoundedExecutorDecorator(app, service, bounds) + new BoundedExecutorDecorator(eventStream, service, bounds) } } } @@ -210,7 +211,7 @@ class MonitorableThread(runnable: Runnable, name: String) /** * @author Jonas Bonér */ -class BoundedExecutorDecorator(val app: ActorSystem, val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate { +class BoundedExecutorDecorator(val eventStream: EventStream, val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate { protected val semaphore = new Semaphore(bound) override def execute(command: Runnable) = { @@ -227,10 +228,10 @@ class BoundedExecutorDecorator(val app: ActorSystem, val executor: ExecutorServi }) } catch { case e: RejectedExecutionException ⇒ - app.eventStream.publish(Warning(this, e.toString)) + eventStream.publish(Warning(this, e.toString)) semaphore.release case e: Throwable ⇒ - app.eventStream.publish(Error(e, this, e.getMessage)) + eventStream.publish(Error(e, this, e.getMessage)) throw e } } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 5f0223176e..31d7b682ef 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -41,8 +41,6 @@ class RemoteActorRefProvider( import akka.dispatch.Promise val local = new LocalActorRefProvider(app, root, eventStream, dispatcher, scheduler) - def deadLetterMailbox = local.deadLetterMailbox - def deadLetters = local.deadLetters def deathWatch = local.deathWatch def guardian = local.guardian def systemGuardian = local.systemGuardian diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala index 29bda7cea6..2cbef7b719 100644 --- a/akka-stm/src/main/scala/akka/agent/Agent.scala +++ b/akka-stm/src/main/scala/akka/agent/Agent.scala @@ -151,7 +151,7 @@ class Agent[T](initialValue: T, app: ActorSystem) { def sendOff(f: T ⇒ T): Unit = { send((value: T) ⇒ { suspend() - val pinnedDispatcher = new PinnedDispatcher(app, null, "agent-send-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) + val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-send-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) threadBased ! Update(f) value @@ -169,7 +169,7 @@ class Agent[T](initialValue: T, app: ActorSystem) { val result = new DefaultPromise[T](timeout)(app.dispatcher) send((value: T) ⇒ { suspend() - val pinnedDispatcher = new PinnedDispatcher(app, null, "agent-alter-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) + val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-alter-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) result completeWith threadBased.?(Update(f), timeout).asInstanceOf[Future[T]] value diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 58ad446728..ac2e9a2bfa 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -12,6 +12,8 @@ import java.lang.ref.WeakReference import scala.annotation.tailrec import akka.actor.{ ActorCell, ActorRef, ActorSystem } import akka.dispatch._ +import akka.actor.Scheduler +import akka.event.EventStream /* * Locking rules: @@ -103,7 +105,11 @@ private[testkit] object CallingThreadDispatcher { * @author Roland Kuhn * @since 1.1 */ -class CallingThreadDispatcher(_app: ActorSystem, val name: String = "calling-thread") extends MessageDispatcher(_app) { +class CallingThreadDispatcher( + _deadLetterMailbox: Mailbox, + _eventStream: EventStream, + _scheduler: Scheduler, + val name: String = "calling-thread") extends MessageDispatcher(_deadLetterMailbox, _eventStream, _scheduler) { import CallingThreadDispatcher._ protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(this, actor) @@ -213,12 +219,12 @@ class CallingThreadDispatcher(_app: ActorSystem, val name: String = "calling-thr true } catch { case ie: InterruptedException ⇒ - app.eventStream.publish(Error(this, ie)) + eventStream.publish(Error(this, ie)) Thread.currentThread().interrupt() intex = ie true case e ⇒ - app.eventStream.publish(Error(this, e)) + eventStream.publish(Error(this, e)) queue.leave false } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 8843330c03..23fdddea2d 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -10,6 +10,8 @@ import com.eaio.uuid.UUID import akka.actor.Props._ import akka.actor.ActorSystem import java.util.concurrent.atomic.AtomicLong +import akka.dispatch.Mailbox +import akka.event.EventStream /** * This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it @@ -19,8 +21,15 @@ import java.util.concurrent.atomic.AtomicLong * @author Roland Kuhn * @since 1.1 */ -class TestActorRef[T <: Actor](_app: ActorSystem, _props: Props, _supervisor: ActorRef, name: String) - extends LocalActorRef(_app, _props.withDispatcher(new CallingThreadDispatcher(_app)), _supervisor, _supervisor.path / name, false) { +class TestActorRef[T <: Actor]( + _app: ActorSystem, + _deadLetterMailbox: Mailbox, + _eventStream: EventStream, + _scheduler: Scheduler, + _props: Props, + _supervisor: ActorRef, + name: String) + extends LocalActorRef(_app, _props.withDispatcher(new CallingThreadDispatcher(_deadLetterMailbox, _eventStream, _scheduler)), _supervisor, _supervisor.path / name, false) { /** * Directly inject messages into actor receive behavior. Any exceptions * thrown will be available to you, while still being able to use @@ -57,7 +66,7 @@ object TestActorRef { 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, supervisor: ActorRef, name: String)(implicit app: ActorSystem): TestActorRef[T] = { - new TestActorRef(app, props, supervisor, name) + new TestActorRef(app, app.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 87b6aa6765..3c0564331a 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -8,6 +8,8 @@ import akka.actor._ import akka.util._ import com.eaio.uuid.UUID import akka.actor.ActorSystem +import akka.dispatch.Mailbox +import akka.event.EventStream /** * This is a specialised form of the TestActorRef with support for querying and @@ -34,8 +36,15 @@ import akka.actor.ActorSystem * @author Roland Kuhn * @since 1.2 */ -class TestFSMRef[S, D, T <: Actor](app: ActorSystem, props: Props, supervisor: ActorRef, name: String)(implicit ev: T <:< FSM[S, D]) - extends TestActorRef(app, props, supervisor, name) { +class TestFSMRef[S, D, T <: Actor]( + app: ActorSystem, + _deadLetterMailbox: Mailbox, + _eventStream: EventStream, + _scheduler: Scheduler, + props: Props, + supervisor: ActorRef, + name: String)(implicit ev: T <:< FSM[S, D]) + extends TestActorRef(app, _deadLetterMailbox, _eventStream, _scheduler, props, supervisor, name) { private def fsm: T = underlyingActor @@ -81,8 +90,8 @@ class TestFSMRef[S, D, T <: Actor](app: ActorSystem, props: Props, supervisor: A 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, Props(creator = () ⇒ factory), app.guardian, TestActorRef.randomName) + new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () ⇒ factory), app.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, Props(creator = () ⇒ factory), app.guardian, name) + new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () ⇒ factory), app.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 0815942e81..fa7c20e000 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -91,7 +91,8 @@ 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)), + val testActor: ActorRef = app.systemActorOf(Props(new TestActor(queue)) + .copy(dispatcher = new CallingThreadDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler)), "testActor" + TestKit.testActorId.incrementAndGet) private var end: Duration = Duration.Undefined diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 7672074b08..4d7fb0283d 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -171,7 +171,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach { val boss = TestActorRef(Props(new TActor { - val ref = new TestActorRef(app, Props(new TActor { + val ref = new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(new TActor { def receiveT = { case _ ⇒ } override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 } override def postRestart(reason: Throwable) { counter -= 1 } From 1cdc8752c23f8aeec911c971f4d1809c103320fa Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 14 Nov 2011 17:41:23 +0100 Subject: [PATCH 05/14] remove app argument from eventStream start methods --- .../src/test/scala/akka/event/EventStreamSpec.scala | 8 ++++---- akka-actor/src/main/scala/akka/actor/ActorSystem.scala | 4 ++-- akka-actor/src/main/scala/akka/event/EventStream.scala | 7 ++++--- akka-actor/src/main/scala/akka/event/Logging.scala | 9 +++++---- 4 files changed, 15 insertions(+), 13 deletions(-) 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 22ecb6ac73..71a52799a8 100644 --- a/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala @@ -40,7 +40,7 @@ class EventStreamSpec extends AkkaSpec(Configuration( "manage subscriptions" in { val bus = new EventStream(true) - bus.start(app) + bus.start(app.provider) bus.subscribe(testActor, classOf[M]) bus.publish(M(42)) within(1 second) { @@ -53,8 +53,8 @@ class EventStreamSpec extends AkkaSpec(Configuration( "manage log levels" in { val bus = new EventStream(false) - bus.start(app) - bus.startDefaultLoggers(app, app.AkkaConfig) + bus.start(app.provider) + bus.startDefaultLoggers(app.provider, app.AkkaConfig) bus.publish(SetTarget(testActor)) expectMsg("OK") within(2 seconds) { @@ -75,7 +75,7 @@ class EventStreamSpec extends AkkaSpec(Configuration( val b2 = new B2 val c = new C val bus = new EventStream(false) - bus.start(app) + bus.start(app.provider) within(2 seconds) { bus.subscribe(testActor, classOf[B2]) === true bus.publish(c) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 43dfa99837..65f6c6cf64 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -210,8 +210,8 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF terminationFuture.onComplete(_ ⇒ dispatcher.shutdown()) // this starts the reaper actor and the user-configured logging subscribers, which are also actors - eventStream.start(this) - eventStream.startDefaultLoggers(this, AkkaConfig) + eventStream.start(provider) + eventStream.startDefaultLoggers(provider, AkkaConfig) // TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor val deployer = new Deployer(this) diff --git a/akka-actor/src/main/scala/akka/event/EventStream.scala b/akka-actor/src/main/scala/akka/event/EventStream.scala index d64fc0a6ce..921ecb6b90 100644 --- a/akka-actor/src/main/scala/akka/event/EventStream.scala +++ b/akka-actor/src/main/scala/akka/event/EventStream.scala @@ -7,6 +7,7 @@ import akka.actor.{ ActorRef, Actor, Props } import akka.actor.ActorSystem import akka.actor.Terminated import akka.util.Subclassification +import akka.actor.ActorRefProvider class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClassification { @@ -41,13 +42,13 @@ class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClas super.unsubscribe(subscriber) } - def start(app: ActorSystem) { - reaper = app.systemActorOf(Props(new Actor { + def start(provider: ActorRefProvider) { + reaper = provider.actorOf(Props(new Actor { def receive = { case ref: ActorRef ⇒ watch(ref) case Terminated(ref) ⇒ unsubscribe(ref) } - }), "MainBusReaper") + }), provider.systemGuardian, "MainBusReaper", true) 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 966b7a534e..052b43062f 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -13,6 +13,7 @@ import akka.util.duration._ import akka.actor.Timeout import akka.dispatch.FutureTimeoutException import java.util.concurrent.atomic.AtomicInteger +import akka.actor.ActorRefProvider /** * This trait brings log level handling to the EventStream: it reads the log @@ -78,7 +79,7 @@ trait LoggingBus extends ActorEventBus { publish(Info(this, "StandardOutLogger started")) } - private[akka] def startDefaultLoggers(app: ActorSystem, config: AkkaConfig) { + 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)) ErrorLevel @@ -94,7 +95,7 @@ trait LoggingBus extends ActorEventBus { } yield { try { ReflectiveAccess.getClassFor[Actor](loggerName) match { - case Right(actorClass) ⇒ addLogger(app, actorClass, level) + case Right(actorClass) ⇒ addLogger(provider, actorClass, level) case Left(exception) ⇒ throw exception } } catch { @@ -137,9 +138,9 @@ trait LoggingBus extends ActorEventBus { publish(Info(this, "all default loggers stopped")) } - private def addLogger(app: ActorSystem, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = { + private def addLogger(provider: ActorRefProvider, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = { val name = "log" + loggerId.incrementAndGet + "-" + simpleName(clazz) - val actor = app.systemActorOf(Props(clazz), name) + val actor = provider.actorOf(Props(clazz), provider.systemGuardian, name, true) implicit val timeout = Timeout(3 seconds) val response = try actor ? InitializeLogger(this) get catch { case _: FutureTimeoutException ⇒ From 3c61e593f2ba8d2a013179d083173f12a6228e4f Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 14 Nov 2011 18:18:08 +0100 Subject: [PATCH 06/14] remove app argument from Deployer --- .../main/scala/akka/actor/ActorRefProvider.scala | 11 ++++++++++- .../src/main/scala/akka/actor/ActorSystem.scala | 12 ++---------- akka-actor/src/main/scala/akka/actor/Deployer.scala | 13 ++++++------- .../main/scala/akka/actor/DeploymentConfig.scala | 6 +++--- .../main/scala/akka/cluster/ClusterInterface.scala | 2 +- akka-remote/src/main/scala/akka/remote/Remote.scala | 4 +--- .../scala/akka/remote/RemoteActorRefProvider.scala | 6 ++++-- .../DirectRoutedRemoteActorMultiJvmSpec.scala | 2 +- .../NewRemoteActorMultiJvmSpec.scala | 2 +- .../RandomRoutedRemoteActorMultiJvmSpec.scala | 2 +- .../RoundRobinRoutedRemoteActorMultiJvmSpec.scala | 2 +- ...ScatterGatherRoutedRemoteActorMultiJvmSpec.scala | 2 +- 12 files changed, 32 insertions(+), 32 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 95db57ce78..43043bfa91 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -15,6 +15,7 @@ import akka.event.{ Logging, DeathWatch, ActorClassification, EventStream } import akka.routing.{ ScatterGatherFirstCompletedRouter, Routing, RouterType, Router, RoutedProps, RoutedActorRef, RoundRobinRouter, RandomRouter, LocalConnectionManager, DirectRouter } import akka.util.Helpers import akka.AkkaException +import com.eaio.uuid.UUID /** * Interface for all ActorRef providers to implement. @@ -31,6 +32,8 @@ trait ActorRefProvider { def deathWatch: DeathWatch + def nodename: String + /** * What deployer will be used to resolve deployment configuration? */ @@ -116,6 +119,7 @@ class ActorRefProviderException(message: String) extends AkkaException(message) */ class LocalActorRefProvider( private val app: ActorSystem, + val AkkaConfig: ActorSystem.AkkaConfig, val root: ActorPath, val eventStream: EventStream, val dispatcher: MessageDispatcher, @@ -123,7 +127,12 @@ class LocalActorRefProvider( val log = Logging(eventStream, this) - private[akka] val deployer: Deployer = new Deployer(app) + val nodename: String = System.getProperty("akka.cluster.nodename") match { + case null | "" ⇒ new UUID().toString + case value ⇒ value + } + + private[akka] val deployer: Deployer = new Deployer(AkkaConfig, eventStream, nodename) val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(app.dispatcher) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 65f6c6cf64..a07a2e74e2 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -142,11 +142,6 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF val startTime = System.currentTimeMillis def uptime = (System.currentTimeMillis - startTime) / 1000 - val nodename: String = System.getProperty("akka.cluster.nodename") match { - case null | "" ⇒ new UUID().toString - case value ⇒ value - } - val address = RemoteAddress(System.getProperty("akka.remote.hostname") match { case null | "" ⇒ InetAddress.getLocalHost.getHostAddress case value ⇒ value @@ -192,8 +187,8 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF case Left(e) ⇒ throw e case Right(b) ⇒ b } - val params: Array[Class[_]] = Array(classOf[ActorSystem], classOf[ActorPath], classOf[EventStream], classOf[MessageDispatcher], classOf[Scheduler]) - val args: Array[AnyRef] = Array(this, root, eventStream, dispatcher, scheduler) + 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) ReflectiveAccess.createInstance[ActorRefProvider](providerClass, params, args) match { case Left(e) ⇒ throw e @@ -213,9 +208,6 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF eventStream.start(provider) eventStream.startDefaultLoggers(provider, AkkaConfig) - // TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor - val deployer = new Deployer(this) - // TODO think about memory consistency effects when doing funky stuff inside constructor val typedActor = new TypedActor(this) diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 3dc309f207..68b2d98d61 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -5,9 +5,7 @@ package akka.actor import collection.immutable.Seq - import java.util.concurrent.ConcurrentHashMap - import akka.event.Logging import akka.actor.DeploymentConfig._ import akka.AkkaException @@ -15,6 +13,7 @@ import akka.config.{ Configuration, ConfigurationException } import akka.util.Duration import java.net.InetSocketAddress import akka.remote.RemoteAddress +import akka.event.EventStream trait ActorDeployer { private[akka] def init(deployments: Seq[Deploy]): Unit @@ -34,10 +33,10 @@ trait ActorDeployer { * * @author Jonas Bonér */ -class Deployer(val app: ActorSystem) extends ActorDeployer { +class Deployer(val AkkaConfig: ActorSystem.AkkaConfig, val eventStream: EventStream, val nodename: String) extends ActorDeployer { - val deploymentConfig = new DeploymentConfig(app) - val log = Logging(app.eventStream, this) + val deploymentConfig = new DeploymentConfig(nodename) + val log = Logging(eventStream, this) val instance: ActorDeployer = { val deployer = new LocalDeployer() @@ -86,7 +85,7 @@ class Deployer(val app: ActorSystem) extends ActorDeployer { private[akka] def pathsInConfig: List[String] = { val deploymentPath = "akka.actor.deployment" - app.config.getSection(deploymentPath) match { + AkkaConfig.config.getSection(deploymentPath) match { case None ⇒ Nil case Some(pathConfig) ⇒ pathConfig.map.keySet @@ -98,7 +97,7 @@ class Deployer(val app: ActorSystem) extends ActorDeployer { /** * Lookup deployment in 'akka.conf' configuration file. */ - private[akka] def lookupInConfig(path: String, configuration: Configuration = app.config): Option[Deploy] = { + private[akka] def lookupInConfig(path: String, configuration: Configuration = AkkaConfig.config): Option[Deploy] = { import akka.util.ReflectiveAccess.{ createInstance, emptyArguments, emptyParams, getClassFor } // -------------------------------- diff --git a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala index 0184ad9fef..d86bf7e142 100644 --- a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala +++ b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala @@ -217,13 +217,13 @@ object DeploymentConfig { * * @author Jonas Bonér */ -class DeploymentConfig(val app: ActorSystem) { +class DeploymentConfig(val nodename: String) { import DeploymentConfig._ - case class ClusterScope(preferredNodes: Iterable[Home] = Vector(Node(app.nodename)), replication: ReplicationScheme = Transient) extends Scope + case class ClusterScope(preferredNodes: Iterable[Home] = Vector(Node(nodename)), replication: ReplicationScheme = Transient) extends Scope - def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home ⇒ nodeNameFor(home) == app.nodename) + def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home ⇒ nodeNameFor(home) == nodename) def replicationSchemeFor(deployment: Deploy): Option[ReplicationScheme] = deployment match { case Deploy(_, _, _, _, ClusterScope(_, replicationScheme)) ⇒ Some(replicationScheme) diff --git a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala index 37c75716d5..1f26ec29e3 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.nodename) + def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.AkkaConfig.ClusterName, app.provider.nodename) def unapply(other: Any) = other match { case address: NodeAddress ⇒ Some((address.clusterName, address.nodeName)) diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index ef29a8de00..01e461d4da 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -27,7 +27,7 @@ import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher } * * @author Jonas Bonér */ -class Remote(val app: ActorSystem) { +class Remote(val app: ActorSystem, val nodename: String) { val log = Logging(app, this) @@ -35,8 +35,6 @@ class Remote(val app: ActorSystem) { import app.config import app.AkkaConfig._ - val nodename = app.nodename - // TODO move to AkkaConfig? val shouldCompressData = config.getBool("akka.remote.use-compression", false) val remoteSystemDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 31d7b682ef..531730a51b 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -30,6 +30,7 @@ import akka.event.EventStream */ class RemoteActorRefProvider( val app: ActorSystem, + val AkkaConfig: ActorSystem.AkkaConfig, val root: ActorPath, val eventStream: EventStream, val dispatcher: MessageDispatcher, @@ -40,12 +41,13 @@ class RemoteActorRefProvider( import java.util.concurrent.ConcurrentHashMap import akka.dispatch.Promise - val local = new LocalActorRefProvider(app, root, eventStream, dispatcher, scheduler) + val local = new LocalActorRefProvider(app, AkkaConfig, root, eventStream, dispatcher, scheduler) def deathWatch = local.deathWatch def guardian = local.guardian def systemGuardian = local.systemGuardian + def nodename = local.nodename - val remote = new Remote(app) + val remote = new Remote(app, nodename) private val actors = new ConcurrentHashMap[String, AnyRef] 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 1577066d67..37a33feb01 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.nodename + case "identify" ⇒ sender ! app.provider.nodename } } } 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 b1e8f793b9..62ab22304b 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.nodename + case "identify" ⇒ sender ! app.provider.nodename } } } 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 380f4d1712..a5b1eea333 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.nodename + case "hit" ⇒ sender ! app.provider.nodename case "end" ⇒ self.stop() } } 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 a076a91786..472dd0afd8 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.nodename + case "hit" ⇒ sender ! app.provider.nodename case "end" ⇒ self.stop() } } 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 a73fc5b908..ac732f3656 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.nodename + case "hit" ⇒ sender ! app.provider.nodename case "end" ⇒ self.stop() } } From 30df7d7bc9d2b356e36fa4ae0dda32f1b6846186 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 14 Nov 2011 18:32:55 +0100 Subject: [PATCH 07/14] remove app argument from TypedActor --- .../src/test/scala/akka/actor/TypedActorSpec.scala | 4 ++-- .../src/main/scala/akka/actor/ActorSystem.scala | 5 ++--- .../src/main/scala/akka/actor/TypedActor.scala | 12 ++++++------ 3 files changed, 10 insertions(+), 11 deletions(-) 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 59ce3d4952..b708f92dcb 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -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, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]()) + val m = TypedActor.MethodCall(app.serialization, classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]()) val baos = new ByteArrayOutputStream(8192 * 4) val out = new ObjectOutputStream(baos) @@ -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, classOf[Foo].getDeclaredMethod("testMethodCallSerialization", Array[Class[_]](classOf[Foo], classOf[String], classOf[Int]): _*), Array[AnyRef](someFoo, null, 1.asInstanceOf[AnyRef])) + 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 baos = new ByteArrayOutputStream(8192 * 4) val out = new ObjectOutputStream(baos) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index a07a2e74e2..b812b936ea 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -208,12 +208,11 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF eventStream.start(provider) eventStream.startDefaultLoggers(provider, AkkaConfig) - // TODO think about memory consistency effects when doing funky stuff inside constructor - val typedActor = new TypedActor(this) - // 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). */ diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index f443be5a2f..7c553d6887 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -16,7 +16,7 @@ object TypedActor { * This class represents a Method call, and has a reference to the Method to be called and the parameters to supply * It's sent to the ActorRef backing the TypedActor and can be serialized and deserialized */ - case class MethodCall(app: ActorSystem, method: Method, parameters: Array[AnyRef]) { + case class MethodCall(ser: Serialization, method: Method, parameters: Array[AnyRef]) { def isOneWay = method.getReturnType == java.lang.Void.TYPE def returnsFuture_? = classOf[Future[_]].isAssignableFrom(method.getReturnType) @@ -40,7 +40,7 @@ object TypedActor { case null ⇒ SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, null, null) case ps if ps.length == 0 ⇒ SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, Array[Serializer.Identifier](), Array[Array[Byte]]()) case ps ⇒ - val serializers: Array[Serializer] = ps map app.serialization.findSerializerFor + val serializers: Array[Serializer] = ps map ser.findSerializerFor val serializedParameters: Array[Array[Byte]] = Array.ofDim[Array[Byte]](serializers.length) for (i ← 0 until serializers.length) serializedParameters(i) = serializers(i) toBinary parameters(i) //Mutable for the sake of sanity @@ -61,7 +61,7 @@ object TypedActor { if (app eq null) throw new IllegalStateException( "Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." + " Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }") - MethodCall(app, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match { + MethodCall(app.serialization, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match { case null ⇒ null case a if a.length == 0 ⇒ Array[AnyRef]() case a ⇒ @@ -264,7 +264,7 @@ trait TypedActorFactory { this: ActorRefFactory ⇒ * * TypedActors needs, just like Actors, to be Stopped when they are no longer needed, use TypedActor.stop(proxy) */ -class TypedActor(val app: ActorSystem) { +class TypedActor(val AkkaConfig: ActorSystem.AkkaConfig, var ser: Serialization) { import TypedActor.MethodCall /** @@ -313,7 +313,7 @@ class TypedActor(val app: ActorSystem) { //Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling val actorVar = new AtomVar[ActorRef](null) val timeout = props.timeout match { - case Props.`defaultTimeout` ⇒ app.AkkaConfig.ActorTimeout + case Props.`defaultTimeout` ⇒ AkkaConfig.ActorTimeout case x ⇒ x } val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar, timeout)).asInstanceOf[T] @@ -365,7 +365,7 @@ class TypedActor(val app: ActorSystem) { case "equals" ⇒ (args.length == 1 && (proxy eq args(0)) || actor == getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean case "hashCode" ⇒ actor.hashCode.asInstanceOf[AnyRef] case _ ⇒ - MethodCall(app, method, args) match { + MethodCall(ser, method, args) match { case m if m.isOneWay ⇒ actor ! m; null //Null return value case m if m.returnsFuture_? ⇒ actor.?(m, timeout) case m if m.returnsJOption_? || m.returnsOption_? ⇒ From 6d85572eccb6de2b871d5ee851f70863b8237ee8 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 14 Nov 2011 20:30:54 +0100 Subject: [PATCH 08/14] - expose ActorRefProvider.AkkaConfig - relax FSMTimingSpec a bit --- .../src/test/scala/akka/actor/FSMTimingSpec.scala | 4 ++-- akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index cb30653d8f..d4d34834a1 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -57,8 +57,8 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { } "receive single-shot timer" in { - within(1.5 seconds) { - within(500 millis, 1 second) { + within(2 seconds) { + within(500 millis, 1.5 seconds) { fsm ! TestSingleTimer expectMsg(Tick) expectMsg(Transition(fsm, TestSingleTimer, Initial)) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 43043bfa91..52773f9c98 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -34,6 +34,8 @@ trait ActorRefProvider { def nodename: String + def AkkaConfig: ActorSystem.AkkaConfig + /** * What deployer will be used to resolve deployment configuration? */ From 648661c5480c4e2aceff78aed48dd86eeeabf566 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 16 Nov 2011 17:18:36 +0100 Subject: [PATCH 09/14] 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) } From 5cc228e7b4a431905e169499b1da85e431643232 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 16 Nov 2011 19:34:37 +0100 Subject: [PATCH 10/14] mark timing tags so they can be omitted on Jenkins --- .../test/scala/akka/actor/FSMTimingSpec.scala | 18 +++++++++--------- .../src/test/scala/akka/testkit/AkkaSpec.scala | 4 +++- .../test/scala/akka/testkit/TestTimeSpec.scala | 4 ++-- 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index d4d34834a1..6e243fc020 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -24,7 +24,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { "A Finite State Machine" must { - "receive StateTimeout" in { + "receive StateTimeout" taggedAs TimingTest in { within(1 second) { within(500 millis, 1 second) { fsm ! TestStateTimeout @@ -34,7 +34,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { } } - "cancel a StateTimeout" in { + "cancel a StateTimeout" taggedAs TimingTest in { within(1 second) { fsm ! TestStateTimeout fsm ! Cancel @@ -44,7 +44,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { } } - "allow StateTimeout override" in { + "allow StateTimeout override" taggedAs TimingTest in { within(500 millis) { fsm ! TestStateTimeoutOverride expectNoMsg @@ -56,9 +56,9 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { } } - "receive single-shot timer" in { + "receive single-shot timer" taggedAs TimingTest in { within(2 seconds) { - within(500 millis, 1.5 seconds) { + within(500 millis, 1 second) { fsm ! TestSingleTimer expectMsg(Tick) expectMsg(Transition(fsm, TestSingleTimer, Initial)) @@ -67,7 +67,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { } } - "correctly cancel a named timer" in { + "correctly cancel a named timer" taggedAs TimingTest in { fsm ! TestCancelTimer within(500 millis) { fsm ! Tick @@ -80,7 +80,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { expectMsg(1 second, Transition(fsm, TestCancelTimer, Initial)) } - "not get confused between named and state timers" in { + "not get confused between named and state timers" taggedAs TimingTest in { fsm ! TestCancelStateTimerInNamedTimerMessage fsm ! Tick expectMsg(500 millis, Tick) @@ -94,7 +94,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { } } - "receive and cancel a repeated timer" in { + "receive and cancel a repeated timer" taggedAs TimingTest in { fsm ! TestRepeatedTimer val seq = receiveWhile(2 seconds) { case Tick ⇒ Tick @@ -105,7 +105,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { } } - "notify unhandled messages" in { + "notify unhandled messages" taggedAs TimingTest in { filterEvents(EventFilter.warning("unhandled event Tick in state TestUnhandled", source = fsm, occurrences = 1), EventFilter.warning("unhandled event Unhandled(test) in state TestUnhandled", source = fsm, occurrences = 1)) { fsm ! TestUnhandled diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index a015bb247d..73f7cf2240 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -4,7 +4,7 @@ package akka.testkit import akka.config.Configuration -import org.scalatest.{ WordSpec, BeforeAndAfterAll } +import org.scalatest.{ WordSpec, BeforeAndAfterAll, Tag } import org.scalatest.matchers.MustMatchers import akka.actor.{ ActorSystem, ActorSystemImpl } import akka.actor.{ Actor, ActorRef, Props } @@ -13,6 +13,8 @@ import akka.event.{ Logging, LoggingAdapter } import akka.util.duration._ import akka.dispatch.FutureTimeoutException +object TimingTest extends Tag("timing") + abstract class AkkaSpec(_application: ActorSystem = ActorSystem()) extends TestKit(_application) with WordSpec with MustMatchers with BeforeAndAfterAll { diff --git a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala index 19844cf225..9a203e3b87 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala @@ -10,14 +10,14 @@ class TestTimeSpec extends AkkaSpec(Configuration("akka.test.timefactor" -> 2.0) "A TestKit" must { - "correctly dilate times" in { + "correctly dilate times" taggedAs TimingTest in { val probe = TestProbe() val now = System.nanoTime intercept[AssertionError] { probe.awaitCond(false, Duration("1 second")) } val diff = System.nanoTime - now val target = (1000000000l * system.AkkaConfig.TestTimeFactor).toLong diff must be > (target - 300000000l) - diff must be < (target + 1000000000l) + diff must be < (target + 300000000l) } } From 2b6d9ca6bd0ddec64a6680603b21563e57a6664e Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 17 Nov 2011 10:54:17 +0100 Subject: [PATCH 11/14] rename ActorSystem.root to rootPath --- .../src/test/scala/akka/actor/ActorRefSpec.scala | 3 ++- akka-actor/src/main/scala/akka/actor/ActorPath.scala | 2 +- .../src/main/scala/akka/actor/ActorRefProvider.scala | 10 +++++----- .../src/main/scala/akka/actor/ActorSystem.scala | 8 ++++---- .../src/main/scala/akka/remote/Gossiper.scala | 2 +- akka-remote/src/main/scala/akka/remote/Remote.scala | 2 +- .../scala/akka/remote/RemoteActorRefProvider.scala | 12 ++++++------ .../scala/akka/remote/netty/NettyRemoteSupport.scala | 6 +++--- 8 files changed, 23 insertions(+), 22 deletions(-) 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 50527ef4b5..fb8e997eb4 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -284,7 +284,8 @@ class ActorRefSpec extends AkkaSpec { val baos = new ByteArrayOutputStream(8192 * 32) val out = new ObjectOutputStream(baos) - val serialized = SerializedActorRef(system.root.remoteAddress.hostname, system.root.remoteAddress.port, "/this/path/does/not/exist") + val addr = system.rootPath.remoteAddress + val serialized = SerializedActorRef(addr.hostname, addr.port, "/this/path/does/not/exist") out.writeObject(serialized) diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index 446e8f43c5..2dfa8b122a 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -21,7 +21,7 @@ object ActorPath { * Create an actor path from an iterable. */ def apply(app: ActorSystem, path: Iterable[String]): ActorPath = - path.foldLeft(app.root)(_ / _) + path.foldLeft(app.rootPath)(_ / _) /** * Split a string path into an iterable. diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index d78ffcab0c..09aef9b570 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -127,7 +127,7 @@ class ActorRefProviderException(message: String) extends AkkaException(message) */ class LocalActorRefProvider( val AkkaConfig: ActorSystem.AkkaConfig, - val root: ActorPath, + val rootPath: ActorPath, val eventStream: EventStream, val dispatcher: MessageDispatcher, val scheduler: Scheduler) extends ActorRefProvider { @@ -151,7 +151,7 @@ class LocalActorRefProvider( val l = tempNumber.getAndIncrement() "$_" + Helpers.base64(l) } - private val tempNode = root / "tmp" + private val tempNode = rootPath / "tmp" def tempPath = tempNode / tempName // FIXME (actor path): this could become a cache for the new tree traversal actorFor @@ -169,7 +169,7 @@ class LocalActorRefProvider( val name = "bubble-walker" // FIXME (actor path): move the root path to the new root guardian - val path = root / name + val path = rootPath / name val address = path.toString @@ -232,7 +232,7 @@ class LocalActorRefProvider( val deathWatch = createDeathWatch() def init(app: ActorSystemImpl) { - rootGuardian = actorOf(app, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, root, true) + rootGuardian = actorOf(app, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, 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 @@ -344,7 +344,7 @@ class LocalActorRefProvider( } private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(ActorPath.split(actor.path)) - private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(root.remoteAddress, actor.path.toString) + private[akka] def serialize(actor: ActorRef): SerializedActorRef = new SerializedActorRef(rootPath.remoteAddress, actor.path.toString) private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index d0a344564f..70664c9f46 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -140,7 +140,7 @@ abstract class ActorSystem extends ActorRefFactory with TypedActorFactory { * Construct a path below the application guardian. */ def /(name: String): ActorPath - def root: ActorPath + def rootPath: ActorPath val startTime = System.currentTimeMillis def uptime = (System.currentTimeMillis - startTime) / 1000 @@ -193,9 +193,9 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst /** * The root actor path for this application. */ - val root: ActorPath = new RootActorPath(address) + val rootPath: ActorPath = new RootActorPath(address) - val deadLetters = new DeadLetterActorRef(eventStream, root / "nul") + val deadLetters = new DeadLetterActorRef(eventStream, rootPath / "nul") val deadLetterMailbox = new Mailbox(null) { becomeClosed() override def dispatcher = null //MessageDispatcher.this @@ -223,7 +223,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst } val arguments = List( classOf[AkkaConfig] -> AkkaConfig, - classOf[ActorPath] -> root, + classOf[ActorPath] -> rootPath, classOf[EventStream] -> eventStream, classOf[MessageDispatcher] -> dispatcher, classOf[Scheduler] -> scheduler) diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index 9873d7960e..415056893c 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.root.remoteAddress + private val address = app.rootPath.remoteAddress private val nodeFingerprint = address.## private val random = SecureRandom.getInstance("SHA1PRNG") diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index dd005a2e15..6d1461b43d 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -81,7 +81,7 @@ class Remote(val app: ActorSystemImpl, val nodename: String) { } def start(): Unit = { - val serverAddress = server.app.root.remoteAddress //Force init of server + val serverAddress = server.app.rootPath.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) } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index f2d334cf3a..cdb12d883c 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -30,7 +30,7 @@ import akka.event.EventStream */ class RemoteActorRefProvider( val AkkaConfig: ActorSystem.AkkaConfig, - val root: ActorPath, + val rootPath: ActorPath, val eventStream: EventStream, val dispatcher: MessageDispatcher, val scheduler: Scheduler) extends ActorRefProvider { @@ -40,7 +40,7 @@ class RemoteActorRefProvider( import java.util.concurrent.ConcurrentHashMap import akka.dispatch.Promise - val local = new LocalActorRefProvider(AkkaConfig, root, eventStream, dispatcher, scheduler) + val local = new LocalActorRefProvider(AkkaConfig, rootPath, eventStream, dispatcher, scheduler) def deathWatch = local.deathWatch def guardian = local.guardian def systemGuardian = local.systemGuardian @@ -194,11 +194,11 @@ class RemoteActorRefProvider( private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = { val remoteAddress = RemoteAddress(actor.hostname, actor.port) - if (optimizeLocalScoped_? && remoteAddress == root.remoteAddress) { + if (optimizeLocalScoped_? && remoteAddress == rootPath.remoteAddress) { local.actorFor(ActorPath.split(actor.path)) } else { - 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 + log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", rootPath.remoteAddress, actor.path, remoteAddress) + Some(RemoteActorRef(remote.app.provider, remote.server, remoteAddress, rootPath / ActorPath.split(actor.path), None)) //Should it be None here } } @@ -206,7 +206,7 @@ class RemoteActorRefProvider( * Using (checking out) actor on a specific node. */ def useActorOnNode(app: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) { - log.debug("[{}] Instantiating Actor [{}] on node [{}]", root, actorPath, remoteAddress) + log.debug("[{}] Instantiating Actor [{}] on node [{}]", rootPath, actorPath, remoteAddress) val actorFactoryBytes = app.serialization.serialize(actorFactory) match { 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 51e2a33b8c..996e38b44d 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,7 @@ class ActiveRemoteClient private[akka] ( def sendSecureCookie(connection: ChannelFuture) { val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT) if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get) - val addr = remoteSupport.app.root.remoteAddress + val addr = remoteSupport.app.rootPath.remoteAddress handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(addr.hostname).setPort(addr.port).build) connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build)) } @@ -429,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.root.remoteAddress + case None ⇒ "Non-running NettyRemoteServer@" + app.rootPath.remoteAddress } private val _isRunning = new Switch(false) @@ -461,7 +461,7 @@ class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Optio val log = Logging(remoteSupport.app, this) import remoteSupport.serverSettings._ - val address = remoteSupport.app.root.remoteAddress + val address = remoteSupport.app.rootPath.remoteAddress val name = "NettyRemoteServer@" + address From c31695bef485dbbce4c223b7e98798c5137704a7 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 17 Nov 2011 11:51:14 +0100 Subject: [PATCH 12/14] rename AkkaConfig to Settings --- .../java/akka/dispatch/JavaFutureTests.java | 2 +- .../scala/akka/actor/ActorTimeoutSpec.scala | 4 +- .../test/scala/akka/actor/ClusterSpec.scala | 2 +- .../akka/actor/dispatch/ActorModelSpec.scala | 2 +- .../test/scala/akka/config/ConfigSpec.scala | 2 +- .../dispatch/PriorityDispatcherSpec.scala | 2 +- .../akka/performance/workbench/Report.scala | 6 +-- .../src/main/scala/akka/actor/Actor.scala | 6 +-- .../src/main/scala/akka/actor/ActorCell.scala | 18 ++++----- .../scala/akka/actor/ActorRefProvider.scala | 10 ++--- .../main/scala/akka/actor/ActorSystem.scala | 18 ++++----- .../actor/BootableActorLoaderService.scala | 6 +-- .../src/main/scala/akka/actor/Deployer.scala | 6 +-- .../src/main/scala/akka/actor/FSM.scala | 2 +- .../main/scala/akka/actor/TypedActor.scala | 6 +-- .../scala/akka/cluster/ClusterInterface.scala | 2 +- .../akka/dispatch/AbstractDispatcher.scala | 12 +++--- .../scala/akka/dispatch/Dispatchers.scala | 40 +++++++++---------- .../src/main/scala/akka/event/Logging.scala | 10 ++--- .../akka/serialization/Serialization.scala | 4 +- .../src/main/scala/akka/util/Duration.scala | 2 +- .../akka/remote/AccrualFailureDetector.scala | 4 +- .../src/main/scala/akka/remote/Remote.scala | 4 +- .../akka/remote/RemoteActorRefProvider.scala | 6 +-- .../remote/netty/NettyRemoteSupport.scala | 4 +- ...onfiggyPropertyPlaceholderConfigurer.scala | 4 +- .../src/test/java/akka/spring/foo/IFoo.java | 2 +- .../test/java/akka/spring/foo/IMyPojo.java | 2 +- .../src/main/scala/akka/agent/Agent.scala | 4 +- .../src/test/scala/config/ConfigSpec.scala | 2 +- .../main/scala/akka/testkit/TestBarrier.scala | 2 +- .../akka/testkit/TestEventListener.scala | 4 +- .../src/main/scala/akka/testkit/TestKit.scala | 4 +- .../main/scala/akka/testkit/TestLatch.scala | 4 +- .../src/main/scala/akka/testkit/package.scala | 4 +- .../scala/akka/testkit/TestTimeSpec.scala | 2 +- project/sbt7/build/AkkaProject.scala | 8 ++-- 37 files changed, 111 insertions(+), 111 deletions(-) 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 7f552c46fc..eb972def86 100644 --- a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java +++ b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java @@ -20,7 +20,7 @@ import scala.Right; public class JavaFutureTests { private final ActorSystem app = ActorSystem.create(); - private final Timeout t = app.AkkaConfig().ActorTimeout(); + private final Timeout t = app.settings().ActorTimeout(); private final FutureFactory ff = new FutureFactory(app.dispatcher(), t); @Test public void mustBeAbleToMapAFuture() { 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 5d8333a6cd..0057cdda60 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 = system.AkkaConfig.ActorTimeout.duration - val testTimeout = if (system.AkkaConfig.ActorTimeout.duration < 400.millis) 500 millis else 100 millis + val defaultTimeout = system.settings.ActorTimeout.duration + val testTimeout = if (system.settings.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 09afd0b527..a96fc68435 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 = system.AkkaConfig.config + val config = system.settings.config import config._ //akka.cluster 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 a1b991add1..aa6e4dc7c9 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 @@ -421,7 +421,7 @@ class DispatcherModelSpec extends ActorModelSpec { import ActorModelSpec._ def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒ - new Dispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", system.AkkaConfig.DispatcherThroughput, + new Dispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, "foo", system.settings.DispatcherThroughput, system.dispatcherFactory.ThroughputDeadlineTimeMillis, system.dispatcherFactory.MailboxType, config, system.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, ThreadPoolConfig(system.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor] 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 ffdc34a903..d33fb9730f 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 = system.AkkaConfig.config + val config = system.settings.config import config._ getList("akka.boot") must equal(Nil) 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 7ed96a6dc8..d73009377a 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -18,7 +18,7 @@ class PriorityDispatcherSpec extends AkkaSpec { testOrdering(BoundedPriorityMailbox(PriorityGenerator({ case i: Int ⇒ i //Reverse order case 'Result ⇒ Int.MaxValue - }: Any ⇒ Int), 1000, system.AkkaConfig.MailboxPushTimeout)) + }: Any ⇒ Int), 1000, system.settings.MailboxPushTimeout)) } } 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 05500932dd..f52e2d8d60 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 @@ -221,11 +221,11 @@ class Report( sb.append("Args:\n ").append(args) sb.append("\n") - sb.append("Akka version: ").append(app.AkkaConfig.ConfigVersion) + sb.append("Akka version: ").append(app.settings.ConfigVersion) sb.append("\n") sb.append("Akka config:") - for (key ← app.AkkaConfig.config.keys) { - sb.append("\n ").append(key).append("=").append(app.AkkaConfig.config(key)) + for (key ← app.settings.config.keys) { + sb.append("\n ").append(key).append("=").append(app.settings.config(key)) } sb.toString diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index b9aba3ac1a..28334980b4 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -150,7 +150,7 @@ object Timeout { implicit def durationToTimeout(duration: Duration) = new Timeout(duration) implicit def intToTimeout(timeout: Int) = new Timeout(timeout) implicit def longToTimeout(timeout: Long) = new Timeout(timeout) - implicit def defaultTimeout(implicit app: ActorSystem) = app.AkkaConfig.ActorTimeout + implicit def defaultTimeout(implicit app: ActorSystem) = app.settings.ActorTimeout } trait ActorLogging { this: Actor ⇒ @@ -234,7 +234,7 @@ trait Actor { /** * The default timeout, based on the config setting 'akka.actor.timeout' */ - implicit def defaultTimeout = system.AkkaConfig.ActorTimeout + implicit def defaultTimeout = system.settings.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 (system.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.settings.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 fd7110a8de..052cf47cb0 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -174,7 +174,7 @@ private[akka] class ActorCell( actor = created created.preStart() checkReceiveTimeout - if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "started (" + actor + ")")) + if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "started (" + actor + ")")) } catch { case e ⇒ try { @@ -188,7 +188,7 @@ private[akka] class ActorCell( def recreate(cause: Throwable): Unit = try { val failedActor = actor - if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "restarting")) + if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "restarting")) val freshActor = newActor() if (failedActor ne null) { val c = currentMessage //One read only plz @@ -202,7 +202,7 @@ private[akka] class ActorCell( } actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call freshActor.postRestart(cause) - if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "restarted")) + if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "restarted")) dispatcher.resume(this) //FIXME should this be moved down? @@ -228,7 +228,7 @@ private[akka] class ActorCell( val c = children if (c.isEmpty) doTerminate() else { - if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopping")) + if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "stopping")) for (child ← c) child.stop() stopping = true } @@ -239,7 +239,7 @@ private[akka] class ActorCell( if (!stats.contains(child)) { childrenRefs = childrenRefs.updated(child.name, child) childrenStats = childrenStats.updated(child, ChildRestartStats()) - if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now supervising " + child)) + if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "now supervising " + child)) } else app.eventStream.publish(Warning(self, "Already supervising " + child)) } @@ -255,10 +255,10 @@ private[akka] class ActorCell( case Recreate(cause) ⇒ recreate(cause) case Link(subject) ⇒ app.deathWatch.subscribe(self, subject) - if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "now monitoring " + subject)) + if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "now monitoring " + subject)) case Unlink(subject) ⇒ app.deathWatch.unsubscribe(self, subject) - if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped monitoring " + subject)) + if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped monitoring " + subject)) case Suspend() ⇒ suspend() case Resume() ⇒ resume() case Terminate() ⇒ terminate() @@ -332,7 +332,7 @@ private[akka] class ActorCell( } def autoReceiveMessage(msg: Envelope) { - if (app.AkkaConfig.DebugAutoReceive) app.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg)) + if (app.settings.DebugAutoReceive) app.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg)) if (stopping) msg.message match { case ChildTerminated ⇒ handleChildTerminated(sender) @@ -359,7 +359,7 @@ private[akka] class ActorCell( try { parent.tell(ChildTerminated, self) app.deathWatch.publish(Terminated(self)) - if (app.AkkaConfig.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped")) + if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped")) } finally { currentMessage = null clearActorContext() diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 09aef9b570..5b45421afe 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -34,7 +34,7 @@ trait ActorRefProvider { def nodename: String - def AkkaConfig: ActorSystem.AkkaConfig + def settings: ActorSystem.Settings def init(app: ActorSystemImpl) @@ -126,7 +126,7 @@ class ActorRefProviderException(message: String) extends AkkaException(message) * Local ActorRef provider. */ class LocalActorRefProvider( - val AkkaConfig: ActorSystem.AkkaConfig, + val settings: ActorSystem.Settings, val rootPath: ActorPath, val eventStream: EventStream, val dispatcher: MessageDispatcher, @@ -139,7 +139,7 @@ class LocalActorRefProvider( case value ⇒ value } - private[akka] val deployer: Deployer = new Deployer(AkkaConfig, eventStream, nodename) + private[akka] val deployer: Deployer = new Deployer(settings, eventStream, nodename) val terminationFuture = new DefaultPromise[ActorSystem.ExitStatus](Timeout.never)(dispatcher) @@ -290,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) dispatcher else props.dispatcher, AkkaConfig.ActorTimeout) + if (props.dispatcher == Props.defaultDispatcher) dispatcher else props.dispatcher, settings.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") @@ -350,7 +350,7 @@ class LocalActorRefProvider( private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = { import akka.dispatch.{ Future, Promise, DefaultPromise } - (if (within == null) AkkaConfig.ActorTimeout else within) match { + (if (within == null) settings.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(tempPath, this, deathWatch, t, dispatcher) { def whenDone() = actors.remove(this) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 70664c9f46..3ad7ddf5f5 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -74,7 +74,7 @@ object ActorSystem { case object Stopped extends ExitStatus case class Failed(cause: Throwable) extends ExitStatus - class AkkaConfig(val config: Configuration) { + class Settings(val config: Configuration) { import config._ val ConfigVersion = getString("akka.version", Version) @@ -133,7 +133,7 @@ abstract class ActorSystem extends ActorRefFactory with TypedActorFactory { import ActorSystem._ def name: String - def AkkaConfig: AkkaConfig + def settings: Settings def nodename: String /** @@ -169,25 +169,25 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst import ActorSystem._ - val AkkaConfig = new AkkaConfig(config) + val settings = new Settings(config) protected def app = this private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(this, props, systemGuardian, address, true) - import AkkaConfig._ + import settings._ val address = RemoteAddress(System.getProperty("akka.remote.hostname") match { case null | "" ⇒ InetAddress.getLocalHost.getHostAddress case value ⇒ value }, System.getProperty("akka.remote.port") match { - case null | "" ⇒ AkkaConfig.RemoteServerPort + case null | "" ⇒ settings.RemoteServerPort case value ⇒ value.toInt }) // this provides basic logging (to stdout) until .start() is called below val eventStream = new EventStream(DebugEventStream) - eventStream.startStdoutLogger(AkkaConfig) + eventStream.startStdoutLogger(settings) val log = new BusLogging(eventStream, this) // “this” used only for .getClass in tagging messages /** @@ -211,7 +211,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512)) // TODO correctly pull its config from the config - val dispatcherFactory = new Dispatchers(AkkaConfig, eventStream, deadLetterMailbox, scheduler) + val dispatcherFactory = new Dispatchers(settings, eventStream, deadLetterMailbox, scheduler) implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher deadLetters.init(dispatcher) @@ -222,7 +222,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst case Right(b) ⇒ b } val arguments = List( - classOf[AkkaConfig] -> AkkaConfig, + classOf[Settings] -> settings, classOf[ActorPath] -> rootPath, classOf[EventStream] -> eventStream, classOf[MessageDispatcher] -> dispatcher, @@ -256,7 +256,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst def start(): this.type = { _serialization = new Serialization(this) - _typedActor = new TypedActor(AkkaConfig, _serialization) + _typedActor = new TypedActor(settings, _serialization) provider.init(this) // this starts the reaper actor and the user-configured logging subscribers, which are also actors eventStream.start(this) diff --git a/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala index 0876671d6e..3dd15e02e5 100644 --- a/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala +++ b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala @@ -16,12 +16,12 @@ trait BootableActorLoaderService extends Bootable { def app: ActorSystem - val BOOT_CLASSES = app.AkkaConfig.BootClasses + val BOOT_CLASSES = app.settings.BootClasses lazy val applicationLoader = createApplicationClassLoader() protected def createApplicationClassLoader(): Option[ClassLoader] = Some({ - if (app.AkkaConfig.Home.isDefined) { - val DEPLOY = app.AkkaConfig.Home.get + "/deploy" + if (app.settings.Home.isDefined) { + val DEPLOY = app.settings.Home.get + "/deploy" val DEPLOY_DIR = new File(DEPLOY) if (!DEPLOY_DIR.exists) { System.exit(-1) diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 68b2d98d61..4b5d64bde6 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -33,7 +33,7 @@ trait ActorDeployer { * * @author Jonas Bonér */ -class Deployer(val AkkaConfig: ActorSystem.AkkaConfig, val eventStream: EventStream, val nodename: String) extends ActorDeployer { +class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream, val nodename: String) extends ActorDeployer { val deploymentConfig = new DeploymentConfig(nodename) val log = Logging(eventStream, this) @@ -85,7 +85,7 @@ class Deployer(val AkkaConfig: ActorSystem.AkkaConfig, val eventStream: EventStr private[akka] def pathsInConfig: List[String] = { val deploymentPath = "akka.actor.deployment" - AkkaConfig.config.getSection(deploymentPath) match { + settings.config.getSection(deploymentPath) match { case None ⇒ Nil case Some(pathConfig) ⇒ pathConfig.map.keySet @@ -97,7 +97,7 @@ class Deployer(val AkkaConfig: ActorSystem.AkkaConfig, val eventStream: EventStr /** * Lookup deployment in 'akka.conf' configuration file. */ - private[akka] def lookupInConfig(path: String, configuration: Configuration = AkkaConfig.config): Option[Deploy] = { + private[akka] def lookupInConfig(path: String, configuration: Configuration = settings.config): Option[Deploy] = { import akka.util.ReflectiveAccess.{ createInstance, emptyArguments, emptyParams, getClassFor } // -------------------------------- diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index 79ca427be4..32c34a801f 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -565,7 +565,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒ def logDepth: Int = 0 - private val debugEvent = system.AkkaConfig.FsmDebugEvent + private val debugEvent = system.settings.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/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index ad4b074cef..51bf2e4639 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -116,7 +116,7 @@ object TypedActor { /** * Returns the default timeout (for a TypedActor) when inside a method call in a TypedActor. */ - implicit def timeout = app.AkkaConfig.ActorTimeout + implicit def timeout = app.settings.ActorTimeout } trait TypedActorFactory { this: ActorRefFactory ⇒ @@ -264,7 +264,7 @@ trait TypedActorFactory { this: ActorRefFactory ⇒ * * TypedActors needs, just like Actors, to be Stopped when they are no longer needed, use TypedActor.stop(proxy) */ -class TypedActor(val AkkaConfig: ActorSystem.AkkaConfig, var ser: Serialization) { +class TypedActor(val settings: ActorSystem.Settings, var ser: Serialization) { import TypedActor.MethodCall /** @@ -313,7 +313,7 @@ class TypedActor(val AkkaConfig: ActorSystem.AkkaConfig, var ser: Serialization) //Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling val actorVar = new AtomVar[ActorRef](null) val timeout = props.timeout match { - case Props.`defaultTimeout` ⇒ AkkaConfig.ActorTimeout + case Props.`defaultTimeout` ⇒ settings.ActorTimeout case x ⇒ x } val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar, timeout)).asInstanceOf[T] diff --git a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala index 37c75716d5..db9f3dc183 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.nodename) + def apply(app: ActorSystem): NodeAddress = new NodeAddress(app.settings.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/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 4df4eeac39..04fd0f94f2 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -14,7 +14,7 @@ import akka.actor._ import akka.actor.ActorSystem import scala.annotation.tailrec import akka.event.EventStream -import akka.actor.ActorSystem.AkkaConfig +import akka.actor.ActorSystem.Settings /** * @author Jonas Bonér @@ -330,19 +330,19 @@ abstract class MessageDispatcher( /** * Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig */ -abstract class MessageDispatcherConfigurator(val AkkaConfig: AkkaConfig, val eventStream: EventStream) { +abstract class MessageDispatcherConfigurator(val settings: Settings, val eventStream: EventStream) { /** * Returns an instance of MessageDispatcher given a Configuration */ def configure(config: Configuration): MessageDispatcher def mailboxType(config: Configuration): MailboxType = { - val capacity = config.getInt("mailbox-capacity", AkkaConfig.MailboxCapacity) + val capacity = config.getInt("mailbox-capacity", settings.MailboxCapacity) if (capacity < 1) UnboundedMailbox() else { val duration = Duration( - config.getInt("mailbox-push-timeout-time", AkkaConfig.MailboxPushTimeout.toMillis.toInt), - AkkaConfig.DefaultTimeUnit) + config.getInt("mailbox-push-timeout-time", settings.MailboxPushTimeout.toMillis.toInt), + settings.DefaultTimeUnit) BoundedMailbox(capacity, duration) } } @@ -352,7 +352,7 @@ abstract class MessageDispatcherConfigurator(val AkkaConfig: AkkaConfig, val eve //Apply the following options to the config if they are present in the config ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(eventStream)).configure( - conf_?(config getInt "keep-alive-time")(time ⇒ _.setKeepAliveTime(Duration(time, AkkaConfig.DefaultTimeUnit))), + conf_?(config getInt "keep-alive-time")(time ⇒ _.setKeepAliveTime(Duration(time, settings.DefaultTimeUnit))), conf_?(config getDouble "core-pool-size-factor")(factor ⇒ _.setCorePoolSizeFromFactor(factor)), conf_?(config getDouble "max-pool-size-factor")(factor ⇒ _.setMaxPoolSizeFromFactor(factor)), conf_?(config getInt "executor-bounds")(bounds ⇒ _.setExecutorBounds(bounds)), diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index 654c3b338e..c94ab9f02c 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -12,7 +12,7 @@ import java.util.concurrent.TimeUnit import akka.actor.ActorSystem import akka.event.EventStream import akka.actor.Scheduler -import akka.actor.ActorSystem.AkkaConfig +import akka.actor.ActorSystem.Settings /** * Scala API. Dispatcher factory. @@ -47,19 +47,19 @@ import akka.actor.ActorSystem.AkkaConfig * @author Jonas Bonér */ class Dispatchers( - val AkkaConfig: ActorSystem.AkkaConfig, + val settings: ActorSystem.Settings, val eventStream: EventStream, val deadLetterMailbox: Mailbox, val scheduler: Scheduler) { - val ThroughputDeadlineTimeMillis = AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt + val ThroughputDeadlineTimeMillis = settings.DispatcherThroughputDeadlineTime.toMillis.toInt val MailboxType: MailboxType = - if (AkkaConfig.MailboxCapacity < 1) UnboundedMailbox() - else BoundedMailbox(AkkaConfig.MailboxCapacity, AkkaConfig.MailboxPushTimeout) - val DispatcherShutdownMillis = AkkaConfig.DispatcherDefaultShutdown.toMillis + if (settings.MailboxCapacity < 1) UnboundedMailbox() + else BoundedMailbox(settings.MailboxCapacity, settings.MailboxPushTimeout) + val DispatcherShutdownMillis = settings.DispatcherDefaultShutdown.toMillis lazy val defaultGlobalDispatcher = - AkkaConfig.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build + settings.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build /** * Creates an thread based dispatcher serving a single actor through the same single thread. @@ -105,7 +105,7 @@ class Dispatchers( * Has a fluent builder interface for configuring its semantics. */ def newDispatcher(name: String) = - ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, AkkaConfig.DispatcherThroughput, + ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, settings.DispatcherThroughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream)) /** @@ -132,7 +132,7 @@ class Dispatchers( * Has a fluent builder interface for configuring its semantics. */ def newBalancingDispatcher(name: String) = - ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, AkkaConfig.DispatcherThroughput, + ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, settings.DispatcherThroughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream)) /** @@ -166,7 +166,7 @@ class Dispatchers( * or else use the supplied default dispatcher */ def fromConfig(key: String, default: ⇒ MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher = - AkkaConfig.config getSection key flatMap from getOrElse default + settings.config getSection key flatMap from getOrElse default /* * Creates of obtains a dispatcher from a ConfigMap according to the format below @@ -193,8 +193,8 @@ class Dispatchers( */ def from(cfg: Configuration): Option[MessageDispatcher] = { cfg.getString("type") flatMap { - case "Dispatcher" ⇒ Some(new DispatcherConfigurator(AkkaConfig, deadLetterMailbox, eventStream, scheduler)) - case "BalancingDispatcher" ⇒ Some(new BalancingDispatcherConfigurator(AkkaConfig, deadLetterMailbox, eventStream, scheduler)) + case "Dispatcher" ⇒ Some(new DispatcherConfigurator(settings, deadLetterMailbox, eventStream, scheduler)) + case "BalancingDispatcher" ⇒ Some(new BalancingDispatcherConfigurator(settings, deadLetterMailbox, eventStream, scheduler)) case "GlobalDispatcher" ⇒ None //TODO FIXME remove this case fqn ⇒ ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match { @@ -214,26 +214,26 @@ class Dispatchers( } } -class DispatcherConfigurator(AkkaConfig: AkkaConfig, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(AkkaConfig, eventStream) { +class DispatcherConfigurator(settings: Settings, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(settings, eventStream) { def configure(config: Configuration): MessageDispatcher = { configureThreadPool(config, threadPoolConfig ⇒ new Dispatcher(deadLetterMailbox, eventStream, scheduler, config.getString("name", newUuid.toString), - config.getInt("throughput", AkkaConfig.DispatcherThroughput), - config.getInt("throughput-deadline-time", AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), + config.getInt("throughput", settings.DispatcherThroughput), + config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt), mailboxType(config), threadPoolConfig, - AkkaConfig.DispatcherDefaultShutdown.toMillis)).build + settings.DispatcherDefaultShutdown.toMillis)).build } } -class BalancingDispatcherConfigurator(AkkaConfig: AkkaConfig, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(AkkaConfig, eventStream) { +class BalancingDispatcherConfigurator(settings: Settings, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(settings, eventStream) { def configure(config: Configuration): MessageDispatcher = { configureThreadPool(config, threadPoolConfig ⇒ new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, config.getString("name", newUuid.toString), - config.getInt("throughput", AkkaConfig.DispatcherThroughput), - config.getInt("throughput-deadline-time", AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), + config.getInt("throughput", settings.DispatcherThroughput), + config.getInt("throughput-deadline-time", settings.DispatcherThroughputDeadlineTime.toMillis.toInt), mailboxType(config), threadPoolConfig, - AkkaConfig.DispatcherDefaultShutdown.toMillis)).build + settings.DispatcherDefaultShutdown.toMillis)).build } } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 09d82e7c09..56b17399e3 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -5,7 +5,7 @@ package akka.event import akka.actor.{ Actor, ActorPath, ActorRef, MinimalActorRef, LocalActorRef, Props, ActorSystem, ActorSystemImpl, simpleName } import akka.AkkaException -import akka.actor.ActorSystem.AkkaConfig +import akka.actor.ActorSystem.Settings import akka.util.ReflectiveAccess import akka.config.ConfigurationException import akka.util.ReentrantGuard @@ -66,7 +66,7 @@ trait LoggingBus extends ActorEventBus { _logLevel = level } - private[akka] def startStdoutLogger(config: AkkaConfig) { + private[akka] def startStdoutLogger(config: Settings) { val level = levelFor(config.StdoutLogLevel) getOrElse { StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + config.StdoutLogLevel)) ErrorLevel @@ -80,12 +80,12 @@ trait LoggingBus extends ActorEventBus { } 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)) + val level = levelFor(app.settings.LogLevel) getOrElse { + StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + app.settings.LogLevel)) ErrorLevel } try { - val defaultLoggers = app.AkkaConfig.EventHandlers match { + val defaultLoggers = app.settings.EventHandlers match { case Nil ⇒ "akka.event.Logging$DefaultLogger" :: Nil case loggers ⇒ loggers } diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 768be6dc9d..899e6790c7 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -70,7 +70,7 @@ class Serialization(val app: ActorSystemImpl) { * But "default" can be overridden in config */ val serializers: Map[String, Serializer] = - app.AkkaConfig.config.getSection("akka.actor.serializers") + app.settings.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: ActorSystemImpl) { /** * 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.AkkaConfig.config.getSection("akka.actor.serialization-bindings") map { + val bindings: Map[String, String] = app.settings.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? diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index fbb27526f1..b9a1e86f33 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -278,7 +278,7 @@ abstract class Duration extends Serializable { def /(other: Duration): Double def unary_- : Duration def finite_? : Boolean - def dilated(implicit app: ActorSystem): Duration = this * app.AkkaConfig.TestTimeFactor + def dilated(implicit app: ActorSystem): Duration = this * app.settings.TestTimeFactor def min(other: Duration): Duration = if (this < other) this else other def max(other: Duration): Duration = if (this > other) this else other def sleep(): Unit = Thread.sleep(toMillis) diff --git a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala index 316bbd5c95..8782a71720 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.AkkaConfig.config.getInt("akka.remote.failure-detector.theshold", 8), - app.AkkaConfig.config.getInt("akka.remote.failure-detector.max-sample-size", 1000)) + app.settings.config.getInt("akka.remote.failure-detector.theshold", 8), + app.settings.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/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index 6d1461b43d..7c40ad428b 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -31,10 +31,10 @@ class Remote(val app: ActorSystemImpl, val nodename: String) { val log = Logging(app, this) import app._ - val AC = AkkaConfig + val AC = settings import AC._ - // TODO move to AkkaConfig? + // TODO move to settings? val shouldCompressData = config.getBool("akka.remote.use-compression", false) val remoteSystemDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index cdb12d883c..0141b88445 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -29,7 +29,7 @@ import akka.event.EventStream * @author Jonas Bonér */ class RemoteActorRefProvider( - val AkkaConfig: ActorSystem.AkkaConfig, + val settings: ActorSystem.Settings, val rootPath: ActorPath, val eventStream: EventStream, val dispatcher: MessageDispatcher, @@ -40,7 +40,7 @@ class RemoteActorRefProvider( import java.util.concurrent.ConcurrentHashMap import akka.dispatch.Promise - val local = new LocalActorRefProvider(AkkaConfig, rootPath, eventStream, dispatcher, scheduler) + val local = new LocalActorRefProvider(settings, rootPath, eventStream, dispatcher, scheduler) def deathWatch = local.deathWatch def guardian = local.guardian def systemGuardian = local.systemGuardian @@ -67,7 +67,7 @@ class RemoteActorRefProvider( private[akka] def deployer: Deployer = local.deployer def defaultDispatcher = dispatcher - def defaultTimeout = AkkaConfig.ActorTimeout + def defaultTimeout = settings.ActorTimeout private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = actorOf(app, props, supervisor, supervisor.path / name, systemService) 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 996e38b44d..3632be6e8b 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -351,8 +351,8 @@ class ActiveRemoteClientHandler( */ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps { - val serverSettings = new RemoteServerSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit) - val clientSettings = new RemoteClientSettings(app.AkkaConfig.config, app.AkkaConfig.DefaultTimeUnit) + val serverSettings = new RemoteServerSettings(app.settings.config, app.settings.DefaultTimeUnit) + val clientSettings = new RemoteClientSettings(app.settings.config, app.settings.DefaultTimeUnit) private val remoteClients = new HashMap[RemoteAddress, RemoteClient] private val clientsLock = new ReadWriteGuard diff --git a/akka-spring/src/main/scala/akka/spring/ConfiggyPropertyPlaceholderConfigurer.scala b/akka-spring/src/main/scala/akka/spring/ConfiggyPropertyPlaceholderConfigurer.scala index c346d94263..1d78b0c14b 100644 --- a/akka-spring/src/main/scala/akka/spring/ConfiggyPropertyPlaceholderConfigurer.scala +++ b/akka-spring/src/main/scala/akka/spring/ConfiggyPropertyPlaceholderConfigurer.scala @@ -19,14 +19,14 @@ class ConfiggyPropertyPlaceholderConfigurer extends PropertyPlaceholderConfigure */ override def setLocation(configgyResource: Resource) { if (configgyResource eq null) throw new IllegalArgumentException("Property 'config' must be set") - val properties = loadAkkaConfig(configgyResource) + val properties = loadSettings(configgyResource) setProperties(properties) } /** * Load the akka.conf and transform to properties. */ - private def loadAkkaConfig(configgyResource: Resource): Properties = { + private def loadSettings(configgyResource: Resource): Properties = { val config = Configuration.fromFile(configgyResource.getFile.getPath) val properties = new Properties() config.map.foreach { case (k, v) ⇒ properties.put(k, v.asInstanceOf[AnyRef]); println("(k,v)=" + k + ", " + v) } diff --git a/akka-spring/src/test/java/akka/spring/foo/IFoo.java b/akka-spring/src/test/java/akka/spring/foo/IFoo.java index e47809f3af..0e5a294811 100644 --- a/akka-spring/src/test/java/akka/spring/foo/IFoo.java +++ b/akka-spring/src/test/java/akka/spring/foo/IFoo.java @@ -5,7 +5,7 @@ package akka.spring.foo; * User: michaelkober * Date: Aug 11, 2010 * Time: 12:49:58 PM - * To change this template use File | Settings | File Templates. + * To change this template use File | settings | File Templates. */ public interface IFoo { public String foo(); diff --git a/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java b/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java index 825d797cf2..24e673a0c3 100644 --- a/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java +++ b/akka-spring/src/test/java/akka/spring/foo/IMyPojo.java @@ -5,7 +5,7 @@ package akka.spring.foo; * User: michaelkober * Date: Aug 11, 2010 * Time: 12:01:00 PM - * To change this template use File | Settings | File Templates. + * To change this template use File | settings | File Templates. */ public interface IMyPojo { public void oneWay(String message); diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala index 2cbef7b719..6a31354e72 100644 --- a/akka-stm/src/main/scala/akka/agent/Agent.scala +++ b/akka-stm/src/main/scala/akka/agent/Agent.scala @@ -151,7 +151,7 @@ class Agent[T](initialValue: T, app: ActorSystem) { def sendOff(f: T ⇒ T): Unit = { send((value: T) ⇒ { suspend() - val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-send-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) + val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-send-off", UnboundedMailbox(), app.settings.ActorTimeoutMillis) val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) threadBased ! Update(f) value @@ -169,7 +169,7 @@ class Agent[T](initialValue: T, app: ActorSystem) { val result = new DefaultPromise[T](timeout)(app.dispatcher) send((value: T) ⇒ { suspend() - val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-alter-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) + val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-alter-off", UnboundedMailbox(), app.settings.ActorTimeoutMillis) val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) result completeWith threadBased.?(Update(f), timeout).asInstanceOf[Future[T]] value diff --git a/akka-stm/src/test/scala/config/ConfigSpec.scala b/akka-stm/src/test/scala/config/ConfigSpec.scala index b6d963c7ea..ffb28e206e 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").AkkaConfig.config + val config = ActorSystem("ConfigSpec").settings.config import config._ diff --git a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala index 5b37d0573a..2f061f7eda 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala @@ -33,7 +33,7 @@ class TestBarrier(count: Int) { } catch { case e: TimeoutException ⇒ throw new TestBarrierTimeoutException("Timeout of %s and time factor of %s" - format (timeout.toString, app.AkkaConfig.TestTimeFactor)) + format (timeout.toString, app.settings.TestTimeFactor)) } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala index 27f9574b43..7e1e946674 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala @@ -83,9 +83,9 @@ abstract class EventFilter(occurrences: Int) { app.eventStream publish TestEvent.Mute(this) try { val result = code - if (!awaitDone(app.AkkaConfig.TestEventFilterLeeway)) + if (!awaitDone(app.settings.TestEventFilterLeeway)) if (todo > 0) - throw new AssertionError("Timeout (" + app.AkkaConfig.TestEventFilterLeeway + ") waiting for " + todo + " messages on " + this) + throw new AssertionError("Timeout (" + app.settings.TestEventFilterLeeway + ") waiting for " + todo + " messages on " + this) else throw new AssertionError("Received " + (-todo) + " messages too many on " + this) result diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index 98154a58ac..a66ae79778 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -125,9 +125,9 @@ class TestKit(_app: ActorSystem) { /** * Obtain time remaining for execution of the innermost enclosing `within` * block or missing that it returns the properly dilated default for this - * case from AkkaConfig (key "akka.test.single-expect-default"). + * case from settings (key "akka.test.single-expect-default"). */ - def remaining: Duration = if (end == Duration.Undefined) system.AkkaConfig.SingleExpectDefaultTimeout.dilated else end - now + def remaining: Duration = if (end == Duration.Undefined) system.settings.SingleExpectDefaultTimeout.dilated else end - now /** * Query queue status. diff --git a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala index f8a0cf3578..7bf5a59e1b 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala @@ -36,7 +36,7 @@ class TestLatch(count: Int = 1)(implicit app: ActorSystem) { def await(timeout: Duration): Boolean = { val opened = latch.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) if (!opened) throw new TestLatchTimeoutException( - "Timeout of %s with time factor of %s" format (timeout.toString, app.AkkaConfig.TestTimeFactor)) + "Timeout of %s with time factor of %s" format (timeout.toString, app.settings.TestTimeFactor)) opened } @@ -46,7 +46,7 @@ class TestLatch(count: Int = 1)(implicit app: ActorSystem) { def awaitTimeout(timeout: Duration = TestLatch.DefaultTimeout) = { val opened = latch.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) if (opened) throw new TestLatchNoTimeoutException( - "Latch opened before timeout of %s with time factor of %s" format (timeout.toString, app.AkkaConfig.TestTimeFactor)) + "Latch opened before timeout of %s with time factor of %s" format (timeout.toString, app.settings.TestTimeFactor)) opened } diff --git a/akka-testkit/src/main/scala/akka/testkit/package.scala b/akka-testkit/src/main/scala/akka/testkit/package.scala index 8a96b1b839..12096d2781 100644 --- a/akka-testkit/src/main/scala/akka/testkit/package.scala +++ b/akka-testkit/src/main/scala/akka/testkit/package.scala @@ -12,8 +12,8 @@ package object testkit { try { val result = block - val stop = now + app.AkkaConfig.TestEventFilterLeeway.toMillis - val failed = eventFilters filterNot (_.awaitDone(Duration(stop - now, MILLISECONDS))) map ("Timeout (" + app.AkkaConfig.TestEventFilterLeeway + ") waiting for " + _) + val stop = now + app.settings.TestEventFilterLeeway.toMillis + val failed = eventFilters filterNot (_.awaitDone(Duration(stop - now, MILLISECONDS))) map ("Timeout (" + app.settings.TestEventFilterLeeway + ") waiting for " + _) if (failed.nonEmpty) throw new AssertionError("Filter completion error:\n" + failed.mkString("\n")) diff --git a/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestTimeSpec.scala index 9a203e3b87..b799df3751 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 * system.AkkaConfig.TestTimeFactor).toLong + val target = (1000000000l * system.settings.TestTimeFactor).toLong diff must be > (target - 300000000l) diff must be < (target + 300000000l) } diff --git a/project/sbt7/build/AkkaProject.scala b/project/sbt7/build/AkkaProject.scala index 2a38c2ab22..7917edbefd 100644 --- a/project/sbt7/build/AkkaProject.scala +++ b/project/sbt7/build/AkkaProject.scala @@ -525,7 +525,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec class AkkaSbtPluginProject(info: ProjectInfo) extends PluginProject(info) { val srcManagedScala = "src_managed" / "main" / "scala" - lazy val addAkkaConfig = systemOptional[Boolean]("akka.release", false) + lazy val addSettings = systemOptional[Boolean]("akka.release", false) lazy val generateAkkaSbtPlugin = { val cleanSrcManaged = cleanTask(srcManagedScala) named ("clean src_managed") @@ -533,7 +533,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec info.parent match { case Some(project: ParentProject) => xsbt.FileUtilities.write((srcManagedScala / "AkkaProject.scala").asFile, - GenerateAkkaSbtPlugin(project, addAkkaConfig.value)) + GenerateAkkaSbtPlugin(project, addSettings.value)) case _ => } None @@ -550,9 +550,9 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec } object GenerateAkkaSbtPlugin { - def apply(project: ParentProject, addAkkaConfig: Boolean): String = { + def apply(project: ParentProject, addSettings: Boolean): String = { val extraConfigs = { - if (addAkkaConfig) Set(ModuleConfiguration("se.scalablesolutions.akka", Repositories.AkkaRepo)) + if (addSettings) Set(ModuleConfiguration("se.scalablesolutions.akka", Repositories.AkkaRepo)) else Set.empty[ModuleConfiguration] } val akkaModules = project.subProjects.values.map(_.name).flatMap{ From d381b720616b31a64f0fd31504d803623a88a6e2 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 17 Nov 2011 12:36:35 +0100 Subject: [PATCH 13/14] rename app: ActorSystem to system everywhere --- .../src/test/java/akka/actor/JavaAPI.java | 8 +-- .../java/akka/dispatch/JavaFutureTests.java | 22 +++---- .../test/scala/akka/actor/ActorRefSpec.scala | 8 +-- .../test/scala/akka/actor/DeployerSpec.scala | 4 +- .../test/scala/akka/actor/FSMActorSpec.scala | 2 +- .../scala/akka/actor/ForwardActorSpec.scala | 6 +- .../test/scala/akka/actor/SchedulerSpec.scala | 4 +- .../scala/akka/actor/TypedActorSpec.scala | 4 +- .../trading/system/TradingSystem.scala | 10 +-- .../akka/performance/workbench/Report.scala | 10 +-- .../akka/serialization/SerializeSpec.scala | 2 +- .../src/main/scala/akka/actor/Actor.scala | 8 +-- .../src/main/scala/akka/actor/ActorCell.scala | 62 +++++++++--------- .../src/main/scala/akka/actor/ActorPath.scala | 8 +-- .../src/main/scala/akka/actor/ActorRef.scala | 14 ++--- .../scala/akka/actor/ActorRefProvider.scala | 40 ++++++------ .../main/scala/akka/actor/ActorSystem.scala | 2 +- .../actor/BootableActorLoaderService.scala | 12 ++-- .../src/main/scala/akka/actor/FSM.scala | 6 +- akka-actor/src/main/scala/akka/actor/IO.scala | 2 +- .../main/scala/akka/actor/TypedActor.scala | 20 +++--- .../scala/akka/cluster/ClusterInterface.scala | 2 +- .../src/main/scala/akka/config/Importer.scala | 2 +- .../akka/dispatch/AbstractDispatcher.scala | 2 +- .../main/scala/akka/dispatch/Mailbox.scala | 2 +- .../main/scala/akka/event/EventStream.scala | 4 +- .../src/main/scala/akka/event/Logging.scala | 18 +++--- .../scala/akka/remote/RemoteInterface.scala | 4 +- .../src/main/scala/akka/routing/Routing.scala | 4 +- .../akka/serialization/Serialization.scala | 10 +-- .../src/main/scala/akka/util/AkkaLoader.scala | 2 +- .../src/main/scala/akka/util/Duration.scala | 2 +- akka-actor/src/main/scala/akka/util/JMX.scala | 8 +-- .../main/scala/akka/camel/CamelService.scala | 2 +- .../src/main/scala/akka/camel/Consumer.scala | 4 +- .../src/main/scala/akka/camel/Message.scala | 2 +- .../src/main/scala/akka/camel/Producer.scala | 2 +- .../scala/akka/camel/ConsumerScalaTest.scala | 6 +- .../scala/akka/cluster/BookKeeperServer.scala | 4 +- akka-docs/disabled/examples/Pi.scala | 4 +- akka-docs/intro/code/tutorials/first/Pi.scala | 12 ++-- .../main/scala/akka/servlet/Initializer.scala | 4 +- .../akka/remote/AccrualFailureDetector.scala | 6 +- .../remote/BootableRemoteActorService.scala | 14 ++--- .../src/main/scala/akka/remote/Gossiper.scala | 16 ++--- .../scala/akka/remote/MessageSerializer.scala | 8 +-- .../akka/remote/NetworkEventStream.scala | 8 +-- .../src/main/scala/akka/remote/Remote.scala | 63 ++++++++++--------- .../akka/remote/RemoteActorRefProvider.scala | 44 ++++++------- .../akka/remote/RemoteConnectionManager.scala | 6 +- .../remote/netty/NettyRemoteSupport.scala | 18 +++--- .../main/scala/DiningHakkersOnBecome.scala | 6 +- .../src/main/scala/DiningHakkersOnFsm.scala | 6 +- .../src/test/scala/ActorFactoryBeanTest.scala | 4 +- .../scala/CamelServiceSpringFeatureTest.scala | 2 +- .../scala/TypedActorSpringFeatureTest.scala | 2 +- .../scala/UntypedActorSpringFeatureTest.scala | 2 +- .../src/main/scala/akka/agent/Agent.scala | 22 +++---- akka-stm/src/test/scala/agent/AgentSpec.scala | 2 +- .../scala/akka/testkit/TestActorRef.scala | 24 +++---- .../main/scala/akka/testkit/TestBarrier.scala | 6 +- .../akka/testkit/TestEventListener.scala | 10 +-- .../main/scala/akka/testkit/TestFSMRef.scala | 12 ++-- .../src/main/scala/akka/testkit/TestKit.scala | 6 +- .../main/scala/akka/testkit/TestLatch.scala | 8 +-- .../src/main/scala/akka/testkit/package.scala | 14 ++--- .../test/scala/akka/testkit/AkkaSpec.scala | 8 +-- .../java/akka/tutorial/first/java/Pi.java | 8 +-- .../src/main/scala/Pi.scala | 8 +-- 69 files changed, 344 insertions(+), 343 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 710370e8f9..aa460e8d20 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java @@ -9,15 +9,15 @@ import static org.junit.Assert.*; public class JavaAPI { - private ActorSystem app = ActorSystem.create(); + private ActorSystem system = ActorSystem.create(); @Test void mustBeAbleToCreateActorRefFromClass() { - ActorRef ref = app.actorOf(JavaAPITestActor.class); + ActorRef ref = system.actorOf(JavaAPITestActor.class); assertNotNull(ref); } @Test void mustBeAbleToCreateActorRefFromFactory() { - ActorRef ref = app.actorOf(new Props().withCreator(new Creator() { + ActorRef ref = system.actorOf(new Props().withCreator(new Creator() { public Actor create() { return new JavaAPITestActor(); } @@ -26,7 +26,7 @@ public class JavaAPI { } @Test void mustAcceptSingleArgTell() { - ActorRef ref = app.actorOf(JavaAPITestActor.class); + ActorRef ref = system.actorOf(JavaAPITestActor.class); ref.tell("hallo"); ref.tell("hallo", ref); } 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 eb972def86..27367ff162 100644 --- a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java +++ b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java @@ -19,9 +19,9 @@ import scala.Right; public class JavaFutureTests { - private final ActorSystem app = ActorSystem.create(); - private final Timeout t = app.settings().ActorTimeout(); - private final FutureFactory ff = new FutureFactory(app.dispatcher(), t); + private final ActorSystem system = ActorSystem.create(); + private final Timeout t = system.settings().ActorTimeout(); + private final FutureFactory ff = new FutureFactory(system.dispatcher(), t); @Test public void mustBeAbleToMapAFuture() { Future f1 = ff.future(new Callable() { @@ -41,7 +41,7 @@ public class JavaFutureTests { @Test public void mustBeAbleToExecuteAnOnResultCallback() throws Throwable { final CountDownLatch latch = new CountDownLatch(1); - Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher()); + Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher()); Future f = cf; f.onResult(new Procedure() { public void apply(String result) { @@ -57,7 +57,7 @@ public class JavaFutureTests { @Test public void mustBeAbleToExecuteAnOnExceptionCallback() throws Throwable { final CountDownLatch latch = new CountDownLatch(1); - Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher()); + Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher()); Future f = cf; f.onException(new Procedure() { public void apply(Throwable t) { @@ -74,7 +74,7 @@ public class JavaFutureTests { @Test public void mustBeAbleToExecuteAnOnTimeoutCallback() throws Throwable { final CountDownLatch latch = new CountDownLatch(1); - Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher()); + Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher()); Future f = cf; f.onTimeout(new Procedure>() { public void apply(Future future) { @@ -88,7 +88,7 @@ public class JavaFutureTests { @Test public void mustBeAbleToExecuteAnOnCompleteCallback() throws Throwable { final CountDownLatch latch = new CountDownLatch(1); - Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher()); + Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher()); Future f = cf; f.onComplete(new Procedure>() { public void apply(akka.dispatch.Future future) { @@ -103,7 +103,7 @@ public class JavaFutureTests { @Test public void mustBeAbleToForeachAFuture() throws Throwable { final CountDownLatch latch = new CountDownLatch(1); - Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher()); + Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher()); Future f = cf; f.foreach(new Procedure() { public void apply(String future) { @@ -118,13 +118,13 @@ public class JavaFutureTests { @Test public void mustBeAbleToFlatMapAFuture() throws Throwable { final CountDownLatch latch = new CountDownLatch(1); - Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher()); + Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher()); cf.completeWithResult("1000"); Future f = cf; Future r = f.flatMap(new Function>() { public Future apply(String r) { latch.countDown(); - Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher()); + Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher()); cf.completeWithResult(Integer.parseInt(r)); return cf; } @@ -137,7 +137,7 @@ public class JavaFutureTests { @Test public void mustBeAbleToFilterAFuture() throws Throwable { final CountDownLatch latch = new CountDownLatch(1); - Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, app.dispatcherFactory().defaultGlobalDispatcher()); + Promise cf = new akka.dispatch.DefaultPromise(1000, TimeUnit.MILLISECONDS, system.dispatcherFactory().defaultGlobalDispatcher()); Future f = cf; Future r = f.filter(new Function() { public Boolean apply(String r) { 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 fb8e997eb4..05b2f67fdb 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(system.asInstanceOf[ActorSystemImpl]) { + Serialization.system.withValue(system.asInstanceOf[ActorSystemImpl]) { val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) val readA = in.readObject @@ -257,7 +257,7 @@ class ActorRefSpec extends AkkaSpec { } } - "throw an exception on deserialize if no app in scope" in { + "throw an exception on deserialize if no system in scope" in { val a = actorOf[InnerActor] import java.io._ @@ -275,7 +275,7 @@ class ActorRefSpec extends AkkaSpec { (intercept[java.lang.IllegalStateException] { in.readObject }).getMessage must be === "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." + - " Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }" + " Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }" } "must throw exception on deserialize if not present in actor hierarchy (and remoting is not enabled)" in { @@ -292,7 +292,7 @@ class ActorRefSpec extends AkkaSpec { out.flush out.close - Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) { + Serialization.system.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/DeployerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala index bbcc84eb46..5750fbe967 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala @@ -14,12 +14,12 @@ class DeployerSpec extends AkkaSpec { "A Deployer" must { "be able to parse 'akka.actor.deployment._' config elements" in { - val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookupInConfig("/app/service-ping") + val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookupInConfig("/system/service-ping") deployment must be('defined) deployment must equal(Some( Deploy( - "/app/service-ping", + "/system/service-ping", None, RoundRobin, NrOfInstances(3), 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 2302e83438..2409d80734 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -17,7 +17,7 @@ import akka.config.Configuration object FSMActorSpec { - class Latches(implicit app: ActorSystem) { + class Latches(implicit system: ActorSystem) { val unlockedLatch = TestLatch() val lockedLatch = TestLatch() val unhandledLatch = TestLatch() 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 c230ecc347..2951e957dc 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala @@ -12,12 +12,12 @@ import akka.util.Duration object ForwardActorSpec { val ExpectedMessage = "FOO" - def createForwardingChain(app: ActorSystem): ActorRef = { - val replier = app.actorOf(new Actor { + def createForwardingChain(system: ActorSystem): ActorRef = { + val replier = system.actorOf(new Actor { def receive = { case x ⇒ sender ! x } }) - def mkforwarder(forwardTo: ActorRef) = app.actorOf( + def mkforwarder(forwardTo: ActorRef) = system.actorOf( new Actor { def receive = { case x ⇒ forwardTo forward x } }) 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 56a3346b72..b7c0ba6c8a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -68,10 +68,10 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { // val actor = actorOf(new Actor { // def receive = { case Ping ⇒ ticks.countDown } // }) - // val numActors = app.registry.local.actors.length + // val numActors = system.registry.local.actors.length // (1 to 1000).foreach(_ ⇒ collectFuture(Scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.MILLISECONDS))) // assert(ticks.await(10, TimeUnit.SECONDS)) - // assert(app.registry.local.actors.length === numActors) + // assert(system.registry.local.actors.length === numActors) // } /** 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 02c44f2216..bb6d6efacb 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -341,7 +341,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) - Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) { + Serialization.system.withValue(system.asInstanceOf[ActorSystemImpl]) { val mNew = in.readObject().asInstanceOf[TypedActor.MethodCall] mNew.method must be(m.method) @@ -360,7 +360,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) - Serialization.app.withValue(system.asInstanceOf[ActorSystemImpl]) { + Serialization.system.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/performance/trading/system/TradingSystem.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingSystem.scala index 88afa18d3d..21096b3c07 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingSystem.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/system/TradingSystem.scala @@ -34,7 +34,7 @@ trait TradingSystem { case class MatchingEngineInfo(primary: ME, standby: Option[ME], orderbooks: List[Orderbook]) } -class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem { +class AkkaTradingSystem(val system: ActorSystem) extends TradingSystem { type ME = ActorRef type OR = ActorRef @@ -70,8 +70,8 @@ class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem { def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match { - case Some(d) ⇒ app.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d)) - case _ ⇒ app.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks))) + case Some(d) ⇒ system.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d)) + case _ ⇒ system.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks))) } override def createOrderReceivers: List[ActorRef] = { @@ -91,8 +91,8 @@ class AkkaTradingSystem(val app: ActorSystem) extends TradingSystem { } def createOrderReceiver() = orDispatcher match { - case Some(d) ⇒ app.actorOf(Props(new AkkaOrderReceiver()).withDispatcher(d)) - case _ ⇒ app.actorOf(Props(new AkkaOrderReceiver())) + case Some(d) ⇒ system.actorOf(Props(new AkkaOrderReceiver()).withDispatcher(d)) + case _ ⇒ system.actorOf(Props(new AkkaOrderReceiver())) } override def start() { 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 f52e2d8d60..83e8b182bf 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 @@ -10,12 +10,12 @@ import akka.event.Logging import scala.collection.immutable.TreeMap class Report( - app: ActorSystem, + system: ActorSystem, resultRepository: BenchResultRepository, compareResultWith: Option[String] = None) { private def doLog = System.getProperty("benchmark.logResult", "true").toBoolean - val log = Logging(app, this) + val log = Logging(system, this) val dateTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm") val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm") @@ -221,11 +221,11 @@ class Report( sb.append("Args:\n ").append(args) sb.append("\n") - sb.append("Akka version: ").append(app.settings.ConfigVersion) + sb.append("Akka version: ").append(system.settings.ConfigVersion) sb.append("\n") sb.append("Akka config:") - for (key ← app.settings.config.keys) { - sb.append("\n ").append(key).append("=").append(app.settings.config(key)) + for (key ← system.settings.config.keys) { + sb.append("\n ").append(key).append("=").append(system.settings.config(key)) } sb.toString 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 0b8e13ca6c..006374e8e9 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -75,7 +75,7 @@ class SerializeSpec extends AkkaSpec { out.close() val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray)) - Serialization.app.withValue(a.asInstanceOf[ActorSystemImpl]) { + Serialization.system.withValue(a.asInstanceOf[ActorSystemImpl]) { val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef] (deadLetters eq a.deadLetters) must be(true) } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 28334980b4..9ab80e8215 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -150,7 +150,7 @@ object Timeout { implicit def durationToTimeout(duration: Duration) = new Timeout(duration) implicit def intToTimeout(timeout: Int) = new Timeout(timeout) implicit def longToTimeout(timeout: Long) = new Timeout(timeout) - implicit def defaultTimeout(implicit app: ActorSystem) = app.settings.ActorTimeout + implicit def defaultTimeout(implicit system: ActorSystem) = system.settings.ActorTimeout } trait ActorLogging { this: Actor ⇒ @@ -164,17 +164,17 @@ object Actor { /** * This decorator adds invocation logging to a Receive function. */ - class LoggingReceive(source: AnyRef, r: Receive)(implicit app: ActorSystem) extends Receive { + class LoggingReceive(source: AnyRef, r: Receive)(implicit system: ActorSystem) extends Receive { def isDefinedAt(o: Any) = { val handled = r.isDefinedAt(o) - app.eventStream.publish(Debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o)) + system.eventStream.publish(Debug(source, "received " + (if (handled) "handled" else "unhandled") + " message " + o)) handled } def apply(o: Any): Unit = r(o) } object LoggingReceive { - def apply(source: AnyRef, r: Receive)(implicit app: ActorSystem): Receive = r match { + def apply(source: AnyRef, r: Receive)(implicit system: ActorSystem): Receive = r match { case _: LoggingReceive ⇒ r case _ ⇒ new LoggingReceive(source, r) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 052cf47cb0..232dd9a339 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -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: ActorSystemImpl, + val system: ActorSystemImpl, val self: ActorRef with ScalaActorRef, val props: Props, val parent: ActorRef, @@ -72,13 +72,13 @@ private[akka] class ActorCell( import ActorCell._ - final def system = app + def systemImpl = system protected final def guardian = self - protected def typedActor = app.typedActor + protected def typedActor = system.typedActor - final def provider = app.provider + final def provider = system.provider var futureTimeout: Option[Cancellable] = None @@ -93,7 +93,7 @@ private[akka] class ActorCell( var stopping = false @inline - final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher + final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher final def isShutdown: Boolean = mailbox.isClosed @@ -141,12 +141,12 @@ private[akka] class ActorCell( } final def tell(message: Any, sender: ActorRef): Unit = - dispatcher.dispatch(this, Envelope(message, if (sender eq null) app.deadLetters else sender)) + dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender)) final def sender: ActorRef = currentMessage match { - case null ⇒ app.deadLetters + case null ⇒ system.deadLetters case msg if msg.sender ne null ⇒ msg.sender - case _ ⇒ app.deadLetters + case _ ⇒ system.deadLetters } //This method is in charge of setting up the contextStack and create a new instance of the Actor @@ -174,11 +174,11 @@ private[akka] class ActorCell( actor = created created.preStart() checkReceiveTimeout - if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "started (" + actor + ")")) + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "started (" + actor + ")")) } catch { case e ⇒ try { - app.eventStream.publish(Error(e, self, "error while creating actor")) + system.eventStream.publish(Error(e, self, "error while creating actor")) // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) } finally { @@ -188,7 +188,7 @@ private[akka] class ActorCell( def recreate(cause: Throwable): Unit = try { val failedActor = actor - if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "restarting")) + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "restarting")) val freshActor = newActor() if (failedActor ne null) { val c = currentMessage //One read only plz @@ -202,14 +202,14 @@ private[akka] class ActorCell( } actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call freshActor.postRestart(cause) - if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "restarted")) + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "restarted")) dispatcher.resume(this) //FIXME should this be moved down? props.faultHandler.handleSupervisorRestarted(cause, self, children) } catch { case e ⇒ try { - app.eventStream.publish(Error(e, self, "error while creating actor")) + system.eventStream.publish(Error(e, self, "error while creating actor")) // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) } finally { @@ -228,7 +228,7 @@ private[akka] class ActorCell( val c = children if (c.isEmpty) doTerminate() else { - if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "stopping")) + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopping")) for (child ← c) child.stop() stopping = true } @@ -239,8 +239,8 @@ private[akka] class ActorCell( if (!stats.contains(child)) { childrenRefs = childrenRefs.updated(child.name, child) childrenStats = childrenStats.updated(child, ChildRestartStats()) - if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "now supervising " + child)) - } else app.eventStream.publish(Warning(self, "Already supervising " + child)) + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "now supervising " + child)) + } else system.eventStream.publish(Warning(self, "Already supervising " + child)) } try { @@ -254,11 +254,11 @@ private[akka] class ActorCell( case Create() ⇒ create() case Recreate(cause) ⇒ recreate(cause) case Link(subject) ⇒ - app.deathWatch.subscribe(self, subject) - if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "now monitoring " + subject)) + system.deathWatch.subscribe(self, subject) + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "now monitoring " + subject)) case Unlink(subject) ⇒ - app.deathWatch.unsubscribe(self, subject) - if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped monitoring " + subject)) + system.deathWatch.unsubscribe(self, subject) + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopped monitoring " + subject)) case Suspend() ⇒ suspend() case Resume() ⇒ resume() case Terminate() ⇒ terminate() @@ -267,7 +267,7 @@ private[akka] class ActorCell( } } catch { case e ⇒ //Should we really catch everything here? - app.eventStream.publish(Error(e, self, "error while processing " + message)) + system.eventStream.publish(Error(e, self, "error while processing " + message)) //TODO FIXME How should problems here be handled? throw e } @@ -286,7 +286,7 @@ private[akka] class ActorCell( case msg ⇒ if (stopping) { // receiving Terminated in response to stopping children is too common to generate noise - if (!msg.isInstanceOf[Terminated]) app.deadLetterMailbox.enqueue(self, messageHandle) + if (!msg.isInstanceOf[Terminated]) system.deadLetterMailbox.enqueue(self, messageHandle) } else { actor(msg) } @@ -294,7 +294,7 @@ private[akka] class ActorCell( currentMessage = null // reset current message after successful invocation } catch { case e ⇒ - app.eventStream.publish(Error(e, self, e.getMessage)) + system.eventStream.publish(Error(e, self, e.getMessage)) // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) @@ -314,7 +314,7 @@ private[akka] class ActorCell( } } catch { case e ⇒ - app.eventStream.publish(Error(e, self, e.getMessage)) + system.eventStream.publish(Error(e, self, e.getMessage)) throw e } } @@ -332,11 +332,11 @@ private[akka] class ActorCell( } def autoReceiveMessage(msg: Envelope) { - if (app.settings.DebugAutoReceive) app.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg)) + if (system.settings.DebugAutoReceive) system.eventStream.publish(Debug(self, "received AutoReceiveMessage " + msg)) if (stopping) msg.message match { case ChildTerminated ⇒ handleChildTerminated(sender) - case _ ⇒ app.deadLetterMailbox.enqueue(self, msg) + case _ ⇒ system.deadLetterMailbox.enqueue(self, msg) } else msg.message match { case HotSwap(code, discardOld) ⇒ become(code(self), discardOld) @@ -349,7 +349,7 @@ private[akka] class ActorCell( } private def doTerminate() { - app.provider.evict(self.path.toString) + system.provider.evict(self.path.toString) dispatcher.detach(this) try { @@ -358,8 +358,8 @@ private[akka] class ActorCell( } finally { try { parent.tell(ChildTerminated, self) - app.deathWatch.publish(Terminated(self)) - if (app.settings.DebugLifecycle) app.eventStream.publish(Debug(self, "stopped")) + system.deathWatch.publish(Terminated(self)) + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self, "stopped")) } finally { currentMessage = null clearActorContext() @@ -369,7 +369,7 @@ private[akka] class ActorCell( final def handleFailure(child: ActorRef, cause: Throwable): Unit = childrenStats.get(child) match { case Some(stats) ⇒ if (!props.faultHandler.handleFailure(child, cause, stats, childrenStats)) throw cause - case None ⇒ app.eventStream.publish(Warning(self, "dropping Failed(" + cause + ") from unknown child")) + case None ⇒ system.eventStream.publish(Warning(self, "dropping Failed(" + cause + ") from unknown child")) } final def handleChildTerminated(child: ActorRef): Unit = { @@ -387,7 +387,7 @@ private[akka] class ActorCell( val recvtimeout = receiveTimeout if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) { //Only reschedule if desired and there are currently no more messages to be processed - futureTimeout = Some(app.scheduler.scheduleOnce(self, ReceiveTimeout, recvtimeout.get, TimeUnit.MILLISECONDS)) + futureTimeout = Some(system.scheduler.scheduleOnce(self, ReceiveTimeout, recvtimeout.get, TimeUnit.MILLISECONDS)) } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index 2dfa8b122a..b1b7ba1aa3 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -14,14 +14,14 @@ object ActorPath { /** * Create an actor path from a string. */ - def apply(app: ActorSystem, path: String): ActorPath = - apply(app, split(path)) + def apply(system: ActorSystem, path: String): ActorPath = + apply(system, split(path)) /** * Create an actor path from an iterable. */ - def apply(app: ActorSystem, path: Iterable[String]): ActorPath = - path.foldLeft(app.rootPath)(_ / _) + def apply(system: ActorSystem, path: Iterable[String]): ActorPath = + path.foldLeft(system.rootPath)(_ / _) /** * Split a string path into an iterable. diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 09264aa2ae..da58825b36 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -162,7 +162,7 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable * @author Jonas Bonér */ class LocalActorRef private[akka] ( - app: ActorSystemImpl, + system: ActorSystemImpl, _props: Props, _supervisor: ActorRef, val path: ActorPath, @@ -185,7 +185,7 @@ class LocalActorRef private[akka] ( * us to use purely factory methods for creating LocalActorRefs. */ @volatile - private var actorCell = new ActorCell(app, this, _props, _supervisor, _receiveTimeout, _hotswap) + private var actorCell = new ActorCell(system, this, _props, _supervisor, _receiveTimeout, _hotswap) actorCell.start() /** @@ -306,17 +306,17 @@ trait ScalaActorRef { ref: ActorRef ⇒ */ case class SerializedActorRef(hostname: String, port: Int, path: String) { - import akka.serialization.Serialization.app + import akka.serialization.Serialization.system def this(remoteAddress: RemoteAddress, path: String) = this(remoteAddress.hostname, remoteAddress.port, path) def this(remoteAddress: InetSocketAddress, path: String) = this(remoteAddress.getAddress.getHostAddress, remoteAddress.getPort, path) //TODO FIXME REMOVE @throws(classOf[java.io.ObjectStreamException]) def readResolve(): AnyRef = { - if (app.value eq null) throw new IllegalStateException( + if (system.value eq null) throw new IllegalStateException( "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." + - " Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }") - app.value.provider.deserialize(this) match { + " Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }") + system.value.provider.deserialize(this) match { case Some(actor) ⇒ actor case None ⇒ throw new IllegalStateException("Could not deserialize ActorRef") } @@ -380,7 +380,7 @@ case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) object DeadLetterActorRef { class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance? @throws(classOf[java.io.ObjectStreamException]) - private def readResolve(): AnyRef = Serialization.app.value.deadLetters + private def readResolve(): AnyRef = Serialization.system.value.deadLetters } val serialized = new SerializedDeadLetterActorRef diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 5b45421afe..d4cd9ea019 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(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(app, props, supervisor, name, false) + def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String): ActorRef = actorOf(system, props, supervisor, name, false) def actorFor(path: Iterable[String]): Option[ActorRef] @@ -36,7 +36,7 @@ trait ActorRefProvider { def settings: ActorSystem.Settings - def init(app: ActorSystemImpl) + def init(system: ActorSystemImpl) /** * What deployer will be used to resolve deployment configuration? @@ -45,9 +45,9 @@ trait ActorRefProvider { private[akka] def scheduler: Scheduler - private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef + private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef - private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef + private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef private[akka] def evict(path: String): Boolean @@ -71,7 +71,7 @@ trait ActorRefProvider { */ trait ActorRefFactory { - protected def app: ActorSystemImpl + protected def systemImpl: ActorSystemImpl protected def provider: ActorRefProvider @@ -89,7 +89,7 @@ trait ActorRefFactory { Helpers.base64(l) } - def actorOf(props: Props): ActorRef = provider.actorOf(app, props, guardian, randomName, false) + def actorOf(props: Props): ActorRef = provider.actorOf(systemImpl, props, guardian, randomName, false) /* * TODO this will have to go at some point, because creating two actors with @@ -99,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(app, props, guardian, name, false) + provider.actorOf(systemImpl, props, guardian, name, false) } def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]])) @@ -231,16 +231,16 @@ class LocalActorRefProvider( val deathWatch = createDeathWatch() - def init(app: ActorSystemImpl) { - rootGuardian = actorOf(app, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) - guardian = actorOf(app, guardianProps, rootGuardian, "app", true) - systemGuardian = actorOf(app, guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true) + def init(system: ActorSystemImpl) { + rootGuardian = actorOf(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) + guardian = actorOf(system, guardianProps, rootGuardian, "system", true) + systemGuardian = actorOf(system, 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) + // FIXME (actor path): should start at the new root guardian, and not use the tail (just to avoid the expected "system" name for now) def actorFor(path: Iterable[String]): Option[ActorRef] = findInCache(ActorPath.join(path)) orElse findInTree(Some(guardian), path.tail) @tailrec @@ -266,10 +266,10 @@ class LocalActorRefProvider( */ private[akka] def evict(path: String): Boolean = actors.remove(path) ne null - 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(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = + actorOf(system, props, supervisor, supervisor.path / name, systemService) - private[akka] def actorOf(app: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = { + private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = { val name = path.name val newFuture = Promise[ActorRef](5000)(dispatcher) // FIXME is this proper timeout? @@ -280,7 +280,7 @@ class LocalActorRefProvider( // create a local actor case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, DeploymentConfig.LocalScope)) ⇒ - new LocalActorRef(app, props, supervisor, path, systemService) // create a local actor + new LocalActorRef(system, props, supervisor, path, systemService) // create a local actor // create a routed actor ref case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, DeploymentConfig.LocalScope)) ⇒ @@ -299,10 +299,10 @@ class LocalActorRefProvider( val connections: Iterable[ActorRef] = (1 to nrOfInstances.factor) map { i ⇒ val routedPath = path.parent / (path.name + ":" + i) - new LocalActorRef(app, props, supervisor, routedPath, systemService) + new LocalActorRef(system, props, supervisor, routedPath, systemService) } - actorOf(app, RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.toString) + actorOf(system, 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) } @@ -327,7 +327,7 @@ class LocalActorRefProvider( /** * Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration. */ - def actorOf(app: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = { + def actorOf(system: 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 @@ -340,7 +340,7 @@ class LocalActorRefProvider( // val localOnly = props.localOnly // if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props) // else new RoutedActorRef(props, address) - new RoutedActorRef(app, props, supervisor, name) + new RoutedActorRef(system, props, supervisor, name) } private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(ActorPath.split(actor.path)) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 3ad7ddf5f5..9cfb9bb156 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -171,7 +171,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst val settings = new Settings(config) - protected def app = this + protected def systemImpl = this private[akka] def systemActorOf(props: Props, address: String): ActorRef = provider.actorOf(this, props, systemGuardian, address, true) diff --git a/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala index 3dd15e02e5..6e0f99b50d 100644 --- a/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala +++ b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala @@ -14,14 +14,14 @@ import akka.util.Bootable */ trait BootableActorLoaderService extends Bootable { - def app: ActorSystem + def system: ActorSystem - val BOOT_CLASSES = app.settings.BootClasses + val BOOT_CLASSES = system.settings.BootClasses lazy val applicationLoader = createApplicationClassLoader() protected def createApplicationClassLoader(): Option[ClassLoader] = Some({ - if (app.settings.Home.isDefined) { - val DEPLOY = app.settings.Home.get + "/deploy" + if (system.settings.Home.isDefined) { + val DEPLOY = system.settings.Home.get + "/deploy" val DEPLOY_DIR = new File(DEPLOY) if (!DEPLOY_DIR.exists) { System.exit(-1) @@ -59,11 +59,11 @@ trait BootableActorLoaderService extends Bootable { super.onUnload() // FIXME shutdown all actors - // app.registry.local.shutdownAll + // system.registry.local.shutdownAll } } /** * Java API for the default JAX-RS/Mist Initializer */ -class DefaultBootableActorLoaderService(val app: ActorSystem) extends BootableActorLoaderService +class DefaultBootableActorLoaderService(val system: ActorSystem) extends BootableActorLoaderService diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index 32c34a801f..cfd9d511ed 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -28,14 +28,14 @@ object FSM { case object StateTimeout case class TimeoutMarker(generation: Long) - case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit app: ActorSystem) { + case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit system: ActorSystem) { private var ref: Option[Cancellable] = _ def schedule(actor: ActorRef, timeout: Duration) { if (repeat) { - ref = Some(app.scheduler.schedule(actor, this, timeout.length, timeout.length, timeout.unit)) + ref = Some(system.scheduler.schedule(actor, this, timeout.length, timeout.length, timeout.unit)) } else { - ref = Some(app.scheduler.scheduleOnce(actor, this, timeout.length, timeout.unit)) + ref = Some(system.scheduler.scheduleOnce(actor, this, timeout.length, timeout.unit)) } } diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index c71938aed7..65e325bfe8 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -290,7 +290,7 @@ private[akka] object IOWorker { case object Shutdown extends Request } -private[akka] class IOWorker(app: ActorSystem, ioManager: ActorRef, val bufferSize: Int) { +private[akka] class IOWorker(system: ActorSystem, ioManager: ActorRef, val bufferSize: Int) { import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT } import IOWorker._ diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 51bf2e4639..ae809cf69a 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -57,17 +57,17 @@ object TypedActor { //TODO implement writeObject and readObject to serialize //TODO Possible optimization is to special encode the parameter-types to conserve space private def readResolve(): AnyRef = { - val app = akka.serialization.Serialization.app.value - if (app eq null) throw new IllegalStateException( + val system = akka.serialization.Serialization.system.value + if (system eq null) throw new IllegalStateException( "Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." + - " Use akka.serialization.Serialization.app.withValue(akkaApplication) { ... }") - MethodCall(app.serialization, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match { + " Use akka.serialization.Serialization.system.withValue(akkaApplication) { ... }") + MethodCall(system.serialization, ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match { case null ⇒ null case a if a.length == 0 ⇒ Array[AnyRef]() case a ⇒ val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity for (i ← 0 until a.length) { - deserializedParameters(i) = app.serialization.serializerByIdentity(serializerIdentifiers(i)).fromBinary(serializedParameters(i)) + deserializedParameters(i) = system.serialization.serializerByIdentity(serializerIdentifiers(i)).fromBinary(serializedParameters(i)) } deserializedParameters }) @@ -101,22 +101,22 @@ object TypedActor { } /** - * Returns the akka app (for a TypedActor) when inside a method call in a TypedActor. + * Returns the akka system (for a TypedActor) when inside a method call in a TypedActor. */ - def app = appReference.get match { - case null ⇒ throw new IllegalStateException("Calling TypedActor.app outside of a TypedActor implementation method!") + def system = appReference.get match { + case null ⇒ throw new IllegalStateException("Calling TypedActor.system outside of a TypedActor implementation method!") case some ⇒ some } /** * Returns the default dispatcher (for a TypedActor) when inside a method call in a TypedActor. */ - implicit def dispatcher = app.dispatcher + implicit def dispatcher = system.dispatcher /** * Returns the default timeout (for a TypedActor) when inside a method call in a TypedActor. */ - implicit def timeout = app.settings.ActorTimeout + implicit def timeout = system.settings.ActorTimeout } trait TypedActorFactory { this: ActorRefFactory ⇒ diff --git a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala index db9f3dc183..db11ab1ef3 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.settings.ClusterName, app.nodename) + def apply(system: ActorSystem): NodeAddress = new NodeAddress(system.settings.ClusterName, system.nodename) def unapply(other: Any) = other match { case address: NodeAddress ⇒ Some((address.clusterName, address.nodeName)) diff --git a/akka-actor/src/main/scala/akka/config/Importer.scala b/akka-actor/src/main/scala/akka/config/Importer.scala index 6045662f35..2198ae58c5 100644 --- a/akka-actor/src/main/scala/akka/config/Importer.scala +++ b/akka-actor/src/main/scala/akka/config/Importer.scala @@ -54,7 +54,7 @@ class FilesystemImporter(val baseDir: String) extends Importer { /** * An Importer that looks for imported config files in the java resources - * of the system class loader (usually the jar used to launch this app). + * of the system class loader (usually the jar used to launch this system). */ class ResourceImporter(classLoader: ClassLoader) extends Importer { def importFile(filename: String): String = { diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 04fd0f94f2..1bdf9d8981 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -80,7 +80,7 @@ object MessageDispatcher { val SCHEDULED = 1 val RESCHEDULED = 2 - implicit def defaultDispatcher(implicit app: ActorSystem) = app.dispatcher + implicit def defaultDispatcher(implicit system: ActorSystem) = system.dispatcher } /** diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 007f117bc3..a373b69839 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -205,7 +205,7 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag } } catch { case e ⇒ - actor.app.eventStream.publish(Error(e, actor.self, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!")) + actor.system.eventStream.publish(Error(e, actor.self, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!")) throw e } } diff --git a/akka-actor/src/main/scala/akka/event/EventStream.scala b/akka-actor/src/main/scala/akka/event/EventStream.scala index 1351007b51..c2be45d81e 100644 --- a/akka-actor/src/main/scala/akka/event/EventStream.scala +++ b/akka-actor/src/main/scala/akka/event/EventStream.scala @@ -39,8 +39,8 @@ class EventStream(debug: Boolean = false) extends LoggingBus with SubchannelClas super.unsubscribe(subscriber) } - def start(app: ActorSystemImpl) { - reaper = app.systemActorOf(Props(new Actor { + def start(system: ActorSystemImpl) { + reaper = system.systemActorOf(Props(new Actor { def receive = { case ref: ActorRef ⇒ watch(ref) case Terminated(ref) ⇒ unsubscribe(ref) diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 56b17399e3..b221c68176 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -79,13 +79,13 @@ trait LoggingBus extends ActorEventBus { publish(Info(this, "StandardOutLogger started")) } - private[akka] def startDefaultLoggers(app: ActorSystemImpl) { - val level = levelFor(app.settings.LogLevel) getOrElse { - StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + app.settings.LogLevel)) + private[akka] def startDefaultLoggers(system: ActorSystemImpl) { + val level = levelFor(system.settings.LogLevel) getOrElse { + StandardOutLogger.print(Error(new EventHandlerException, this, "unknown akka.stdout-loglevel " + system.settings.LogLevel)) ErrorLevel } try { - val defaultLoggers = app.settings.EventHandlers match { + val defaultLoggers = system.settings.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(app, actorClass, level) + case Right(actorClass) ⇒ addLogger(system, 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(app: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = { + private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = { val name = "log" + loggerId.incrementAndGet + "-" + simpleName(clazz) - val actor = app.systemActorOf(Props(clazz), name) + val actor = system.systemActorOf(Props(clazz), name) implicit val timeout = Timeout(3 seconds) val response = try actor ? InitializeLogger(this) get catch { case _: FutureTimeoutException ⇒ @@ -237,12 +237,12 @@ object Logging { * Obtain LoggingAdapter for the given application and source object. The * source object is used to identify the source of this logging channel. */ - def apply(app: ActorSystem, source: AnyRef): LoggingAdapter = new BusLogging(app.eventStream, source) + def apply(system: ActorSystem, source: AnyRef): LoggingAdapter = new BusLogging(system.eventStream, source) /** * Java API: Obtain LoggingAdapter for the given application and source object. The * source object is used to identify the source of this logging channel. */ - def getLogger(app: ActorSystem, source: AnyRef): LoggingAdapter = apply(app, source) + def getLogger(system: ActorSystem, source: AnyRef): LoggingAdapter = apply(system, source) /** * Obtain LoggingAdapter for the given event bus and source object. The * source object is used to identify the source of this logging channel. diff --git a/akka-actor/src/main/scala/akka/remote/RemoteInterface.scala b/akka-actor/src/main/scala/akka/remote/RemoteInterface.scala index d5837c95b1..c49e0c4b4d 100644 --- a/akka-actor/src/main/scala/akka/remote/RemoteInterface.scala +++ b/akka-actor/src/main/scala/akka/remote/RemoteInterface.scala @@ -128,7 +128,7 @@ case class CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorExcept override def printStackTrace(printWriter: PrintWriter) = cause.printStackTrace(printWriter) } -abstract class RemoteSupport(val app: ActorSystem) { +abstract class RemoteSupport(val system: ActorSystem) { /** * Shuts down the remoting */ @@ -162,7 +162,7 @@ abstract class RemoteSupport(val app: ActorSystem) { recipient: ActorRef, loader: Option[ClassLoader]): Unit - protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = app.eventStream.publish(message) + protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = system.eventStream.publish(message) override def toString = name } diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 0fa194f0c0..f9345d0001 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -155,7 +155,7 @@ object Routing { /** * An Abstract convenience implementation for building an ActorReference that uses a Router. */ -abstract private[akka] class AbstractRoutedActorRef(val app: ActorSystem, val props: RoutedProps) extends UnsupportedActorRef { +abstract private[akka] class AbstractRoutedActorRef(val system: ActorSystem, val props: RoutedProps) extends UnsupportedActorRef { val router = props.routerFactory() override def tell(message: Any, sender: ActorRef) = router.route(message)(sender) @@ -167,7 +167,7 @@ abstract private[akka] class AbstractRoutedActorRef(val app: ActorSystem, val pr * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to * on (or more) of these actors. */ -private[akka] class RoutedActorRef(app: ActorSystem, val routedProps: RoutedProps, val supervisor: ActorRef, override val name: String) extends AbstractRoutedActorRef(app, routedProps) { +private[akka] class RoutedActorRef(system: ActorSystem, val routedProps: RoutedProps, val supervisor: ActorRef, override val name: String) extends AbstractRoutedActorRef(system, routedProps) { val path = supervisor.path / name diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 899e6790c7..9973b11a17 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -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: ActorSystemImpl) { +class Serialization(val system: ActorSystemImpl) { //TODO document me def serialize(o: AnyRef): Either[Exception, Array[Byte]] = @@ -28,7 +28,7 @@ class Serialization(val app: ActorSystemImpl) { clazz: Class[_], classLoader: Option[ClassLoader]): Either[Exception, AnyRef] = try { - Serialization.app.withValue(app) { + Serialization.system.withValue(system) { Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader)) } } catch { case e: Exception ⇒ Left(e) } @@ -70,7 +70,7 @@ class Serialization(val app: ActorSystemImpl) { * But "default" can be overridden in config */ val serializers: Map[String, Serializer] = - app.settings.config.getSection("akka.actor.serializers") + system.settings.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: ActorSystemImpl) { /** * 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.settings.config.getSection("akka.actor.serialization-bindings") map { + val bindings: Map[String, String] = system.settings.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: ActorSystemImpl) { object Serialization { // TODO ensure that these are always set (i.e. withValue()) when doing deserialization - val app = new DynamicVariable[ActorSystemImpl](null) + val system = new DynamicVariable[ActorSystemImpl](null) } diff --git a/akka-actor/src/main/scala/akka/util/AkkaLoader.scala b/akka-actor/src/main/scala/akka/util/AkkaLoader.scala index 1d4f23e03f..f2bf63c137 100644 --- a/akka-actor/src/main/scala/akka/util/AkkaLoader.scala +++ b/akka-actor/src/main/scala/akka/util/AkkaLoader.scala @@ -8,7 +8,7 @@ import akka.actor.ActorSystem /* * This class is responsible for booting up a stack of bundles and then shutting them down */ -class AkkaLoader(app: ActorSystem) { +class AkkaLoader(system: ActorSystem) { private val hasBooted = new Switch(false) @volatile diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index b9a1e86f33..8f8b6c5709 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -278,7 +278,7 @@ abstract class Duration extends Serializable { def /(other: Duration): Double def unary_- : Duration def finite_? : Boolean - def dilated(implicit app: ActorSystem): Duration = this * app.settings.TestTimeFactor + def dilated(implicit system: ActorSystem): Duration = this * system.settings.TestTimeFactor def min(other: Duration): Duration = if (this < other) this else other def max(other: Duration): Duration = if (this > other) this else other def sleep(): Unit = Thread.sleep(toMillis) diff --git a/akka-actor/src/main/scala/akka/util/JMX.scala b/akka-actor/src/main/scala/akka/util/JMX.scala index 1c7465882b..2bf8545210 100644 --- a/akka-actor/src/main/scala/akka/util/JMX.scala +++ b/akka-actor/src/main/scala/akka/util/JMX.scala @@ -18,20 +18,20 @@ object JMX { def nameFor(hostname: String, service: String, bean: String): ObjectName = new ObjectName("akka.%s:type=%s,name=%s".format(hostname, service, bean.replace(":", "_"))) - def register(name: ObjectName, mbean: AnyRef)(implicit app: ActorSystem): Option[ObjectInstance] = try { + def register(name: ObjectName, mbean: AnyRef)(implicit system: ActorSystem): Option[ObjectInstance] = try { Some(mbeanServer.registerMBean(mbean, name)) } catch { case e: InstanceAlreadyExistsException ⇒ Some(mbeanServer.getObjectInstance(name)) case e: Exception ⇒ - app.eventStream.publish(Error(e, this, "Error when registering mbean [%s]".format(mbean))) + system.eventStream.publish(Error(e, this, "Error when registering mbean [%s]".format(mbean))) None } - def unregister(mbean: ObjectName)(implicit app: ActorSystem) = try { + def unregister(mbean: ObjectName)(implicit system: ActorSystem) = try { mbeanServer.unregisterMBean(mbean) } catch { case e: InstanceNotFoundException ⇒ {} - case e: Exception ⇒ app.eventStream.publish(Error(e, this, "Error while unregistering mbean [%s]".format(mbean))) + case e: Exception ⇒ system.eventStream.publish(Error(e, this, "Error while unregistering mbean [%s]".format(mbean))) } } diff --git a/akka-camel/src/main/scala/akka/camel/CamelService.scala b/akka-camel/src/main/scala/akka/camel/CamelService.scala index 643d18fb18..eb3c8e4ae1 100644 --- a/akka-camel/src/main/scala/akka/camel/CamelService.scala +++ b/akka-camel/src/main/scala/akka/camel/CamelService.scala @@ -58,7 +58,7 @@ trait CamelService extends Bootable { * Starts this CamelService. */ def start: CamelService = { - // Only init and start if not already done by app + // Only init and start if not already done by system if (!CamelContextManager.initialized) CamelContextManager.init if (!CamelContextManager.started) CamelContextManager.start diff --git a/akka-camel/src/main/scala/akka/camel/Consumer.scala b/akka-camel/src/main/scala/akka/camel/Consumer.scala index 0518a7c271..86f9d7f519 100644 --- a/akka-camel/src/main/scala/akka/camel/Consumer.scala +++ b/akka-camel/src/main/scala/akka/camel/Consumer.scala @@ -35,7 +35,7 @@ trait Consumer { this: Actor ⇒ /** * Determines whether one-way communications between an endpoint and this consumer actor - * should be auto-acknowledged or app-acknowledged. + * should be auto-acknowledged or system-acknowledged. */ def autoack = true @@ -79,7 +79,7 @@ abstract class UntypedConsumerActor extends UntypedActor with Consumer { /** * Determines whether one-way communications between an endpoint and this consumer actor - * should be auto-acknowledged or app-acknowledged. + * should be auto-acknowledged or system-acknowledged. */ def isAutoack() = super.autoack } diff --git a/akka-camel/src/main/scala/akka/camel/Message.scala b/akka-camel/src/main/scala/akka/camel/Message.scala index e3ed12fec0..e5a553f34e 100644 --- a/akka-camel/src/main/scala/akka/camel/Message.scala +++ b/akka-camel/src/main/scala/akka/camel/Message.scala @@ -208,7 +208,7 @@ object Message { } /** - * Positive acknowledgement message (used for app-acknowledged message receipts). + * Positive acknowledgement message (used for system-acknowledged message receipts). * * @author Martin Krasser */ diff --git a/akka-camel/src/main/scala/akka/camel/Producer.scala b/akka-camel/src/main/scala/akka/camel/Producer.scala index 8c65d71c66..4a9367005a 100644 --- a/akka-camel/src/main/scala/akka/camel/Producer.scala +++ b/akka-camel/src/main/scala/akka/camel/Producer.scala @@ -50,7 +50,7 @@ trait ProducerSupport { this: Actor ⇒ /** * Returns the names of message headers to copy from a request message to a response message. * By default only the Message.MessageExchangeId is copied. Applications may override this to - * define an app-specific set of message headers to copy. + * define an system-specific set of message headers to copy. */ def headersToCopy: Set[String] = headersToCopyDefault diff --git a/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala b/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala index 97eb8b49a3..efe7d6aee1 100644 --- a/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala +++ b/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala @@ -138,15 +138,15 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher "An non auto-acknowledging consumer" when { "started" must { - "must support acknowledgements on app level" in { + "must support acknowledgements on system level" in { var consumer: ActorRef = null service.awaitEndpointActivation(1) { - consumer = actorOf(new TestAckConsumer("direct:app-ack-test")) + consumer = actorOf(new TestAckConsumer("direct:system-ack-test")) } must be(true) - val endpoint = mandatoryContext.getEndpoint("direct:app-ack-test", classOf[DirectEndpoint]) + val endpoint = mandatoryContext.getEndpoint("direct:system-ack-test", classOf[DirectEndpoint]) val producer = endpoint.createProducer.asInstanceOf[AsyncProcessor] val exchange = endpoint.createExchange diff --git a/akka-cluster/src/main/scala/akka/cluster/BookKeeperServer.scala b/akka-cluster/src/main/scala/akka/cluster/BookKeeperServer.scala index 4bafaab001..e546d2d9af 100644 --- a/akka-cluster/src/main/scala/akka/cluster/BookKeeperServer.scala +++ b/akka-cluster/src/main/scala/akka/cluster/BookKeeperServer.scala @@ -9,13 +9,13 @@ import java.io.File /* A simple use of BookKeeper is to implement a write-ahead transaction log. A server maintains an in-memory data structure -(with periodic snapshots for example) and logs changes to that structure before it applies the change. The app +(with periodic snapshots for example) and logs changes to that structure before it applies the change. The system server creates a ledger at startup and store the ledger id and password in a well known place (ZooKeeper maybe). When it needs to make a change, the server adds an entry with the change information to a ledger and apply the change when BookKeeper adds the entry successfully. The server can even use asyncAddEntry to queue up many changes for high change throughput. BooKeeper meticulously logs the changes in order and call the completion functions in order. -When the app server dies, a backup server will come online, get the last snapshot and then it will open the +When the system server dies, a backup server will come online, get the last snapshot and then it will open the ledger of the old server and read all the entries from the time the snapshot was taken. (Since it doesn't know the last entry number it will use MAX_INTEGER). Once all the entries have been processed, it will close the ledger and start a new one for its use. diff --git a/akka-docs/disabled/examples/Pi.scala b/akka-docs/disabled/examples/Pi.scala index d0869426fe..2b0fb45914 100644 --- a/akka-docs/disabled/examples/Pi.scala +++ b/akka-docs/disabled/examples/Pi.scala @@ -10,7 +10,7 @@ import System.{currentTimeMillis => now} import java.util.concurrent.CountDownLatch //#imports -//#app +//#system object Pi extends App { calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000) @@ -127,5 +127,5 @@ object Pi extends App { latch.await() } } -//#app +//#system diff --git a/akka-docs/intro/code/tutorials/first/Pi.scala b/akka-docs/intro/code/tutorials/first/Pi.scala index 6bbf05ee27..6be88d0f32 100644 --- a/akka-docs/intro/code/tutorials/first/Pi.scala +++ b/akka-docs/intro/code/tutorials/first/Pi.scala @@ -11,10 +11,10 @@ // import java.util.concurrent.CountDownLatch // //#imports -// //#app +// //#system // object Pi extends App { -// val app = ActorSystem() +// val system = ActorSystem() // calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000) @@ -66,10 +66,10 @@ // //#create-workers // // create the workers -// val workers = Vector.fill(nrOfWorkers)(app.actorOf[Worker]) +// val workers = Vector.fill(nrOfWorkers)(system.actorOf[Worker]) // // wrap them with a load-balancing router -// val router = app.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi") +// val router = system.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi") // //#create-workers // //#master-receive @@ -119,7 +119,7 @@ // val latch = new CountDownLatch(1) // // create the master -// val master = app.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)) +// val master = system.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)) // // start the calculation // master ! Calculate @@ -128,5 +128,5 @@ // latch.await() // } // } -// //#app +// //#system diff --git a/akka-kernel/src/main/scala/akka/servlet/Initializer.scala b/akka-kernel/src/main/scala/akka/servlet/Initializer.scala index 5eeb74cce2..b91e5ae439 100644 --- a/akka-kernel/src/main/scala/akka/servlet/Initializer.scala +++ b/akka-kernel/src/main/scala/akka/servlet/Initializer.scala @@ -14,13 +14,13 @@ import javax.servlet.{ ServletContextListener, ServletContextEvent } /** * This class can be added to web.xml mappings as a listener to start and postStop Akka. * - * + * * ... * * akka.servlet.Initializer * * ... - * + * */ class Initializer extends ServletContextListener { lazy val loader = new AkkaLoader diff --git a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala index 8782a71720..987146a39b 100644 --- a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala +++ b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala @@ -25,10 +25,10 @@ import akka.actor.ActorSystem */ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 1000) { - def this(app: ActorSystem) { + def this(system: ActorSystem) { this( - app.settings.config.getInt("akka.remote.failure-detector.theshold", 8), - app.settings.config.getInt("akka.remote.failure-detector.max-sample-size", 1000)) + system.settings.config.getInt("akka.remote.failure-detector.theshold", 8), + system.settings.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/BootableRemoteActorService.scala b/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala index 07bc74ad30..e3bd903c07 100644 --- a/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala +++ b/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala @@ -21,26 +21,26 @@ trait BootableRemoteActorService extends Bootable { def settings: RemoteServerSettings protected lazy val remoteServerThread = new Thread(new Runnable() { - def run = app.remote.start(self.applicationLoader.getOrElse(null)) //Use config host/port + def run = system.remote.start(self.applicationLoader.getOrElse(null)) //Use config host/port }, "Akka RemoteModule Service") def startRemoteService() { remoteServerThread.start() } abstract override def onLoad() { - if (app.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) { - app.eventHandler.info(this, "Initializing Remote Actors Service...") + if (system.reflective.ClusterModule.isEnabled && settings.isRemotingEnabled) { + system.eventHandler.info(this, "Initializing Remote Actors Service...") startRemoteService() - app.eventHandler.info(this, "Remote Actors Service initialized") + system.eventHandler.info(this, "Remote Actors Service initialized") } super.onLoad() } abstract override def onUnload() { - app.eventHandler.info(this, "Shutting down Remote Actors Service") + system.eventHandler.info(this, "Shutting down Remote Actors Service") - app.remote.shutdown() + system.remote.shutdown() if (remoteServerThread.isAlive) remoteServerThread.join(1000) - app.eventHandler.info(this, "Remote Actors Service has been shut down") + system.eventHandler.info(this, "Remote Actors Service has been shut down") super.onUnload() } } diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index 415056893c..56a59b2ae2 100644 --- a/akka-remote/src/main/scala/akka/remote/Gossiper.scala +++ b/akka-remote/src/main/scala/akka/remote/Gossiper.scala @@ -101,13 +101,13 @@ class Gossiper(remote: Remote) { currentGossip: Gossip, nodeMembershipChangeListeners: Set[NodeMembershipChangeListener] = Set.empty[NodeMembershipChangeListener]) - private val app = remote.app - private val log = Logging(app, this) + private val system = remote.system + private val log = Logging(system, this) private val failureDetector = remote.failureDetector - private val connectionManager = new RemoteConnectionManager(app, remote, Map.empty[RemoteAddress, ActorRef]) + private val connectionManager = new RemoteConnectionManager(system, remote, Map.empty[RemoteAddress, ActorRef]) private val seeds = Set(address) // FIXME read in list of seeds from config - private val address = app.rootPath.remoteAddress + private val address = system.rootPath.remoteAddress private val nodeFingerprint = address.## private val random = SecureRandom.getInstance("SHA1PRNG") @@ -122,8 +122,8 @@ class Gossiper(remote: Remote) { { // start periodic gossip and cluster scrutinization - default is run them every second with 1/2 second in between - app.scheduler schedule (() ⇒ initateGossip(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit) - app.scheduler schedule (() ⇒ scrutinize(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit) + system.scheduler schedule (() ⇒ initateGossip(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit) + system.scheduler schedule (() ⇒ scrutinize(), initalDelayForGossip.toSeconds, gossipFrequency.toSeconds, timeUnit) } /** @@ -153,7 +153,7 @@ class Gossiper(remote: Remote) { node ← oldAvailableNodes if connectionManager.connectionFor(node).isEmpty } { - val connectionFactory = () ⇒ RemoteActorRef(remote.app.provider, remote.server, gossipingNode, remote.remoteDaemon.path, None) + val connectionFactory = () ⇒ RemoteActorRef(remote.system.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 } @@ -299,7 +299,7 @@ class Gossiper(remote: Remote) { } private def toRemoteMessage(gossip: Gossip): RemoteProtocol.RemoteSystemDaemonMessageProtocol = { - val gossipAsBytes = app.serialization.serialize(gossip) match { + val gossipAsBytes = system.serialization.serialize(gossip) match { case Left(error) ⇒ throw error case Right(bytes) ⇒ bytes } diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 74f4073bde..06c66f0188 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -11,15 +11,15 @@ import akka.actor.ActorSystem object MessageSerializer { - def deserialize(app: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = { + def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = { val clazz = loadManifest(classLoader, messageProtocol) - app.serialization.deserialize(messageProtocol.getMessage.toByteArray, + system.serialization.deserialize(messageProtocol.getMessage.toByteArray, clazz, classLoader).fold(x ⇒ throw x, identity) } - def serialize(app: ActorSystem, message: AnyRef): MessageProtocol = { + def serialize(system: ActorSystem, message: AnyRef): MessageProtocol = { val builder = MessageProtocol.newBuilder - val bytes = app.serialization.serialize(message).fold(x ⇒ throw x, identity) + val bytes = system.serialization.serialize(message).fold(x ⇒ throw x, identity) builder.setMessage(ByteString.copyFrom(bytes)) builder.setMessageManifest(ByteString.copyFromUtf8(message.getClass.getName)) builder.build diff --git a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala index de14fabcc4..3376ad9416 100644 --- a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala +++ b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala @@ -58,14 +58,14 @@ object NetworkEventStream { } } -class NetworkEventStream(app: ActorSystemImpl) { +class NetworkEventStream(system: ActorSystemImpl) { import NetworkEventStream._ // FIXME: check that this supervision is correct - private[akka] val sender = app.provider.actorOf(app, - Props[Channel].copy(dispatcher = app.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")), - app.systemGuardian, "network-event-sender", systemService = true) + private[akka] val sender = system.provider.actorOf(system, + Props[Channel].copy(dispatcher = system.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")), + system.systemGuardian, "network-event-sender", systemService = true) /** * Registers a network event stream listener (asyncronously). diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index 7c40ad428b..c27bd67093 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -26,11 +26,11 @@ import java.util.concurrent.atomic.AtomicLong * * @author Jonas Bonér */ -class Remote(val app: ActorSystemImpl, val nodename: String) { +class Remote(val system: ActorSystemImpl, val nodename: String) { - val log = Logging(app, this) + val log = Logging(system, this) - import app._ + import system._ val AC = settings import AC._ @@ -38,7 +38,7 @@ class Remote(val app: ActorSystemImpl, val nodename: String) { val shouldCompressData = config.getBool("akka.remote.use-compression", false) val remoteSystemDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt - val failureDetector = new AccrualFailureDetector(app) + val failureDetector = new AccrualFailureDetector(system) // val gossiper = new Gossiper(this) @@ -48,17 +48,17 @@ class Remote(val app: ActorSystemImpl, val nodename: String) { val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build // FIXME it is probably better to create another supervisor for handling the children created by handle_* - private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props( + private[remote] lazy val remoteDaemonSupervisor = system.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, + system.provider.actorOf(system, Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)), remoteDaemonSupervisor, remoteDaemonServiceName, systemService = true) - private[remote] lazy val remoteClientLifeCycleHandler = app.actorOf(Props(new Actor { + private[remote] lazy val remoteClientLifeCycleHandler = system.actorOf(Props(new Actor { def receive = { case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address) case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address) @@ -66,22 +66,22 @@ class Remote(val app: ActorSystemImpl, val nodename: String) { } }), "akka.remote.RemoteClientLifeCycleListener") - lazy val eventStream = new NetworkEventStream(app) + lazy val eventStream = new NetworkEventStream(system) lazy val server: RemoteSupport = { - val remote = new akka.remote.netty.NettyRemoteSupport(app) + val remote = new akka.remote.netty.NettyRemoteSupport(system) remote.start() //TODO FIXME Any application loader here? - app.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent]) - app.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent]) + system.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent]) + system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent]) - // TODO actually register this provider in app in remote mode + // TODO actually register this provider in system in remote mode //provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider) remote } def start(): Unit = { - val serverAddress = server.app.rootPath.remoteAddress //Force init of server + val serverAddress = server.system.rootPath.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) } @@ -97,6 +97,7 @@ class Remote(val app: ActorSystemImpl, val nodename: String) { class RemoteSystemDaemon(remote: Remote) extends Actor { import remote._ + import remote.{ system ⇒ systemImpl } override def preRestart(reason: Throwable, msg: Option[Any]) { log.debug("RemoteSystemDaemon failed due to [{}] - restarting...", reason) @@ -133,16 +134,16 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { if (shouldCompressData) LZF.uncompress(message.getPayload.toByteArray) else message.getPayload.toByteArray val actorFactory = - app.serialization.deserialize(actorFactoryBytes, classOf[() ⇒ Actor], None) match { + system.serialization.deserialize(actorFactoryBytes, classOf[() ⇒ Actor], None) match { case Left(error) ⇒ throw error case Right(instance) ⇒ instance.asInstanceOf[() ⇒ Actor] } - val actorPath = ActorPath(remote.app, message.getActorPath) - val parent = app.actorFor(actorPath.parent) + val actorPath = ActorPath(systemImpl, message.getActorPath) + val parent = system.actorFor(actorPath.parent) if (parent.isDefined) { - app.provider.actorOf(app, Props(creator = actorFactory), parent.get, actorPath.name) + systemImpl.provider.actorOf(systemImpl, Props(creator = actorFactory), parent.get, actorPath.name) } else { log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message) } @@ -151,7 +152,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { log.error("Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [{}]", message) } - sender ! Success(app.address) + sender ! Success(systemImpl.address) } catch { case error: Throwable ⇒ //FIXME doesn't seem sensible sender ! Failure(error) @@ -192,7 +193,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { // FIXME: handle real remote supervision def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) { - new LocalActorRef(app, + new LocalActorRef(systemImpl, Props( context ⇒ { case f: Function0[_] ⇒ try { f() } finally { context.self.stop() } @@ -201,7 +202,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { // FIXME: handle real remote supervision def handle_fun0_any(message: RemoteSystemDaemonMessageProtocol) { - new LocalActorRef(app, + new LocalActorRef(systemImpl, Props( context ⇒ { case f: Function0[_] ⇒ try { sender ! f() } finally { context.self.stop() } @@ -210,7 +211,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { // FIXME: handle real remote supervision def handle_fun1_arg_unit(message: RemoteSystemDaemonMessageProtocol) { - new LocalActorRef(app, + new LocalActorRef(systemImpl, Props( context ⇒ { case (fun: Function[_, _], param: Any) ⇒ try { fun.asInstanceOf[Any ⇒ Unit].apply(param) } finally { context.self.stop() } @@ -219,7 +220,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { // FIXME: handle real remote supervision def handle_fun1_arg_any(message: RemoteSystemDaemonMessageProtocol) { - new LocalActorRef(app, + new LocalActorRef(systemImpl, Props( context ⇒ { case (fun: Function[_, _], param: Any) ⇒ try { sender ! fun.asInstanceOf[Any ⇒ Any](param) } finally { context.self.stop() } @@ -232,7 +233,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { } private def payloadFor[T](message: RemoteSystemDaemonMessageProtocol, clazz: Class[T]): T = { - app.serialization.deserialize(message.getPayload.toByteArray, clazz, None) match { + system.serialization.deserialize(message.getPayload.toByteArray, clazz, None) match { case Left(error) ⇒ throw error case Right(instance) ⇒ instance.asInstanceOf[T] } @@ -241,20 +242,20 @@ class RemoteSystemDaemon(remote: Remote) extends Actor { class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) { - val provider = remote.app.asInstanceOf[ActorSystemImpl].provider + val provider = remote.system.asInstanceOf[ActorSystemImpl].provider lazy val sender: ActorRef = if (input.hasSender) provider.deserialize( SerializedActorRef(input.getSender.getHost, input.getSender.getPort, input.getSender.getPath)).getOrElse(throw new IllegalStateException("OHNOES")) else - remote.app.deadLetters + remote.system.deadLetters - lazy val recipient: ActorRef = remote.app.actorFor(input.getRecipient.getPath).getOrElse(remote.app.deadLetters) + lazy val recipient: ActorRef = remote.system.actorFor(input.getRecipient.getPath).getOrElse(remote.system.deadLetters) lazy val payload: Either[Throwable, AnyRef] = if (input.hasException) Left(parseException()) - else Right(MessageSerializer.deserialize(remote.app, input.getMessage, classLoader)) + else Right(MessageSerializer.deserialize(remote.system, input.getMessage, classLoader)) protected def parseException(): Throwable = { val exception = input.getException @@ -267,7 +268,7 @@ class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLo .newInstance(exception.getMessage).asInstanceOf[Throwable] } catch { case problem: Exception ⇒ - remote.app.eventStream.publish(Logging.Error(problem, remote, problem.getMessage)) + remote.system.eventStream.publish(Logging.Error(problem, remote, problem.getMessage)) CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(problem, classname, exception.getMessage) } } @@ -277,7 +278,7 @@ class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLo trait RemoteMarshallingOps { - def app: ActorSystem + def system: ActorSystem def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = { val arp = AkkaRemoteProtocol.newBuilder @@ -295,7 +296,7 @@ trait RemoteMarshallingOps { * Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message. */ def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = { - val rep = app.asInstanceOf[ActorSystemImpl].provider.serialize(actor) + val rep = system.asInstanceOf[ActorSystemImpl].provider.serialize(actor) ActorRefProtocol.newBuilder.setHost(rep.hostname).setPort(rep.port).setPath(rep.path).build } @@ -308,7 +309,7 @@ trait RemoteMarshallingOps { message match { case Right(message) ⇒ - messageBuilder.setMessage(MessageSerializer.serialize(app, message.asInstanceOf[AnyRef])) + messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef])) case Left(exception) ⇒ messageBuilder.setException(ExceptionProtocol.newBuilder .setClassname(exception.getClass.getName) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 0141b88445..7e04283eb2 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -55,10 +55,10 @@ class RemoteActorRefProvider( @volatile private var remoteDaemonConnectionManager: RemoteConnectionManager = _ - def init(app: ActorSystemImpl) { - local.init(app) - remote = new Remote(app, nodename) - remoteDaemonConnectionManager = new RemoteConnectionManager(app, remote) + def init(system: ActorSystemImpl) { + local.init(system) + remote = new Remote(system, nodename) + remoteDaemonConnectionManager = new RemoteConnectionManager(system, remote) } private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef = local.theOneWhoWalksTheBubblesOfSpaceTime @@ -69,11 +69,11 @@ class RemoteActorRefProvider( def defaultDispatcher = dispatcher def defaultTimeout = settings.ActorTimeout - 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(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef = + actorOf(system, props, supervisor, supervisor.path / name, 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) + private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef = + if (systemService) local.actorOf(system, props, supervisor, path, systemService) else { val name = path.name val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout? @@ -92,13 +92,13 @@ class RemoteActorRefProvider( // case FailureDetectorType.Custom(implClass) ⇒ FailureDetector.createCustomFailureDetector(implClass) // } - def isReplicaNode: Boolean = remoteAddresses exists { _ == app.address } + def isReplicaNode: Boolean = remoteAddresses exists { _ == system.address } - //app.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(app.defaultAddress, address, remoteAddresses.mkString, isReplicaNode)) + //system.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(system.defaultAddress, address, remoteAddresses.mkString, isReplicaNode)) if (isReplicaNode) { // we are on one of the replica node for this remote actor - 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 + local.actorOf(system, 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 @@ -135,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.app.provider, remote.server, remoteAddress, path, None)) + conns + (remoteAddress -> RemoteActorRef(remote.system.provider, remote.server, remoteAddress, path, None)) } - val connectionManager = new RemoteConnectionManager(app, remote, connections) + val connectionManager = new RemoteConnectionManager(system, remote, connections) - connections.keys foreach { useActorOnNode(app, _, path.toString, props.creator) } + connections.keys foreach { useActorOnNode(system, _, path.toString, props.creator) } - actorOf(app, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name) + actorOf(system, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name) } - case deploy ⇒ local.actorOf(app, props, supervisor, name, systemService) + case deploy ⇒ local.actorOf(system, props, supervisor, name, systemService) } } catch { case e: Exception ⇒ @@ -153,7 +153,7 @@ class RemoteActorRefProvider( throw e } - // actor foreach app.registry.register // only for ActorRegistry backward compat, will be removed later + // actor foreach system.registry.register // only for ActorRegistry backward compat, will be removed later newFuture completeWithResult actor actors.replace(path.toString, newFuture, actor) @@ -167,9 +167,9 @@ class RemoteActorRefProvider( * Copied from LocalActorRefProvider... */ // FIXME: implement supervision - def actorOf(app: ActorSystem, props: RoutedProps, supervisor: ActorRef, name: String): ActorRef = { + def actorOf(system: 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) + new RoutedActorRef(system, props, supervisor, name) } def actorFor(path: Iterable[String]): Option[ActorRef] = actors.get(ActorPath.join(path)) match { @@ -198,18 +198,18 @@ class RemoteActorRefProvider( local.actorFor(ActorPath.split(actor.path)) } else { log.debug("{}: Creating RemoteActorRef with address [{}] connected to [{}]", rootPath.remoteAddress, actor.path, remoteAddress) - Some(RemoteActorRef(remote.app.provider, remote.server, remoteAddress, rootPath / ActorPath.split(actor.path), None)) //Should it be None here + Some(RemoteActorRef(remote.system.provider, remote.server, remoteAddress, rootPath / ActorPath.split(actor.path), None)) //Should it be None here } } /** * Using (checking out) actor on a specific node. */ - def useActorOnNode(app: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) { + def useActorOnNode(system: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) { log.debug("[{}] Instantiating Actor [{}] on node [{}]", rootPath, actorPath, remoteAddress) val actorFactoryBytes = - app.serialization.serialize(actorFactory) match { + system.serialization.serialize(actorFactory) match { case Left(error) ⇒ throw error case Right(bytes) ⇒ if (remote.shouldCompressData) LZF.compress(bytes) else bytes } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala index 8f1c3d4733..e128742365 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala @@ -20,12 +20,12 @@ import java.util.concurrent.atomic.AtomicReference * @author Jonas Bonér */ class RemoteConnectionManager( - app: ActorSystem, + system: ActorSystem, remote: Remote, initialConnections: Map[RemoteAddress, ActorRef] = Map.empty[RemoteAddress, ActorRef]) extends ConnectionManager { - val log = Logging(app, this) + val log = Logging(system, this) // FIXME is this VersionedIterable really needed? It is not used I think. Complicates API. See 'def connections' etc. case class State(version: Long, connections: Map[RemoteAddress, ActorRef]) @@ -149,5 +149,5 @@ class RemoteConnectionManager( } private[remote] def newConnection(remoteAddress: RemoteAddress, actorPath: ActorPath) = - RemoteActorRef(remote.app.provider, remote.server, remoteAddress, actorPath, None) + RemoteActorRef(remote.system.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 3632be6e8b..205df1138d 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -38,7 +38,7 @@ abstract class RemoteClient private[akka] ( val remoteSupport: NettyRemoteSupport, val remoteAddress: RemoteAddress) { - val log = Logging(remoteSupport.app, this) + val log = Logging(remoteSupport.system, this) val name = simpleName(this) + "@" + remoteAddress @@ -147,7 +147,7 @@ class ActiveRemoteClient private[akka] ( def sendSecureCookie(connection: ChannelFuture) { val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT) if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get) - val addr = remoteSupport.app.rootPath.remoteAddress + val addr = remoteSupport.system.rootPath.remoteAddress handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(addr.hostname).setPort(addr.port).build) connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build)) } @@ -349,10 +349,10 @@ class ActiveRemoteClientHandler( /** * Provides the implementation of the Netty remote support */ -class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with RemoteMarshallingOps { +class NettyRemoteSupport(_system: ActorSystem) extends RemoteSupport(_system) with RemoteMarshallingOps { - val serverSettings = new RemoteServerSettings(app.settings.config, app.settings.DefaultTimeUnit) - val clientSettings = new RemoteClientSettings(app.settings.config, app.settings.DefaultTimeUnit) + val serverSettings = new RemoteServerSettings(system.settings.config, system.settings.DefaultTimeUnit) + val clientSettings = new RemoteClientSettings(system.settings.config, system.settings.DefaultTimeUnit) private val remoteClients = new HashMap[RemoteAddress, RemoteClient] private val clientsLock = new ReadWriteGuard @@ -429,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.rootPath.remoteAddress + case None ⇒ "Non-running NettyRemoteServer@" + system.rootPath.remoteAddress } private val _isRunning = new Switch(false) @@ -458,10 +458,10 @@ class NettyRemoteSupport(_app: ActorSystem) extends RemoteSupport(_app) with Rem } class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Option[ClassLoader]) { - val log = Logging(remoteSupport.app, this) + val log = Logging(remoteSupport.system, this) import remoteSupport.serverSettings._ - val address = remoteSupport.app.rootPath.remoteAddress + val address = remoteSupport.system.rootPath.remoteAddress val name = "NettyRemoteServer@" + address @@ -563,7 +563,7 @@ class RemoteServerHandler( val applicationLoader: Option[ClassLoader], val remoteSupport: NettyRemoteSupport) extends SimpleChannelUpstreamHandler { - val log = Logging(remoteSupport.app, this) + val log = Logging(remoteSupport.system, this) import remoteSupport.serverSettings._ 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 0e7592beb7..1c86015df4 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala @@ -122,14 +122,14 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor { * Alright, here's our test-harness */ object DiningHakkers { - val app = ActorSystem() + val system = ActorSystem() def run { //Create 5 chopsticks - val chopsticks = for (i ← 1 to 5) yield app.actorOf(new Chopstick("Chopstick " + i)) + val chopsticks = for (i ← 1 to 5) yield system.actorOf(new Chopstick("Chopstick " + i)) //Create 5 awesome hakkers and assign them their left and right chopstick val hakkers = for { (name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex - } yield app.actorOf(new Hakker(name, chopsticks(i), chopsticks((i + 1) % 5))) + } yield system.actorOf(new Hakker(name, chopsticks(i), chopsticks((i + 1) % 5))) //Signal all hakkers that they should start thinking, and watch the show hakkers.foreach(_ ! Think) 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 4e1bb86ec0..987f630784 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -163,15 +163,15 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit */ object DiningHakkersOnFsm { - val app = ActorSystem() + val system = ActorSystem() def run = { // Create 5 chopsticks - val chopsticks = for (i ← 1 to 5) yield app.actorOf(new Chopstick("Chopstick " + i)) + val chopsticks = for (i ← 1 to 5) yield system.actorOf(new Chopstick("Chopstick " + i)) // Create 5 awesome fsm hakkers and assign them their left and right chopstick val hakkers = for { (name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex - } yield app.actorOf(new FSMHakker(name, chopsticks(i), chopsticks((i + 1) % 5))) + } yield system.actorOf(new FSMHakker(name, chopsticks(i), chopsticks((i + 1) % 5))) hakkers.foreach(_ ! Think) } diff --git a/akka-spring/src/test/scala/ActorFactoryBeanTest.scala b/akka-spring/src/test/scala/ActorFactoryBeanTest.scala index 58a112de83..ff7dd71f47 100644 --- a/akka-spring/src/test/scala/ActorFactoryBeanTest.scala +++ b/akka-spring/src/test/scala/ActorFactoryBeanTest.scala @@ -65,7 +65,7 @@ class ActorFactoryBeanTest extends Spec with ShouldMatchers with BeforeAndAfterA assert(target.getStringFromVal === entry.value) } - it("should create an app context and verify dependency injection for typed") { + it("should create an system context and verify dependency injection for typed") { var ctx = new ClassPathXmlApplicationContext("appContext.xml"); val ta = ctx.getBean("typedActor").asInstanceOf[PojoInf]; assert(ta.isPreStartInvoked) @@ -75,7 +75,7 @@ class ActorFactoryBeanTest extends Spec with ShouldMatchers with BeforeAndAfterA ctx.close } - it("should create an app context and verify dependency injection for untyped actors") { + it("should create an system context and verify dependency injection for untyped actors") { var ctx = new ClassPathXmlApplicationContext("appContext.xml") val uta = ctx.getBean("untypedActor").asInstanceOf[ActorRef] val ping = uta.actor.asInstanceOf[PingActor] diff --git a/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala b/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala index 967d94dd33..5975ba60e3 100644 --- a/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala @@ -17,7 +17,7 @@ class CamelServiceSpringFeatureTest extends FeatureSpec with BeforeAndAfterEach Actor.registry.shutdownAll } - feature("start CamelService from Spring app context") { + feature("start CamelService from Spring system context") { import CamelContextManager._ scenario("with a custom CamelContext and access a registered typed actor") { val appctx = new ClassPathXmlApplicationContext("/appContextCamelServiceCustom.xml") diff --git a/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala b/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala index 69037bd148..9b9f428d3d 100644 --- a/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala @@ -60,7 +60,7 @@ class TypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers with B myPojo } - feature("parse Spring app context") { + feature("parse Spring system context") { scenario("akka:typed-actor and akka:supervision and akka:dispatcher can be used as top level elements") { val context = new ClassPathResource("/typed-actor-config.xml") diff --git a/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala b/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala index ef111c57ab..6c7a0156e7 100644 --- a/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala +++ b/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala @@ -45,7 +45,7 @@ class UntypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers with pingActor } - feature("parse Spring app context") { + feature("parse Spring system context") { scenario("get a untyped actor") { val myactor = getPingActorFromContext("/untyped-actor-config.xml", "simple-untyped-actor") diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala index 6a31354e72..444b6c3d2d 100644 --- a/akka-stm/src/main/scala/akka/agent/Agent.scala +++ b/akka-stm/src/main/scala/akka/agent/Agent.scala @@ -20,7 +20,7 @@ private[akka] case object Get * Factory method for creating an Agent. */ object Agent { - def apply[T](initialValue: T)(implicit app: ActorSystem) = new Agent(initialValue, app) + def apply[T](initialValue: T)(implicit system: ActorSystem) = new Agent(initialValue, system) } /** @@ -93,9 +93,9 @@ object Agent { * agent4.close * }}} */ -class Agent[T](initialValue: T, app: ActorSystem) { +class Agent[T](initialValue: T, system: ActorSystem) { private[akka] val ref = Ref(initialValue) - private[akka] val updater = app.actorOf(Props(new AgentUpdater(this))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow? + private[akka] val updater = system.actorOf(Props(new AgentUpdater(this))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow? /** * Read the internal state of the agent. @@ -123,7 +123,7 @@ class Agent[T](initialValue: T, app: ActorSystem) { def alter(f: T ⇒ T)(timeout: Timeout): Future[T] = { def dispatch = updater.?(Update(f), timeout).asInstanceOf[Future[T]] if (Stm.activeTransaction) { - val result = new DefaultPromise[T](timeout)(app.dispatcher) + val result = new DefaultPromise[T](timeout)(system.dispatcher) get //Join xa deferred { result completeWith dispatch } //Attach deferred-block to current transaction result @@ -151,8 +151,8 @@ class Agent[T](initialValue: T, app: ActorSystem) { def sendOff(f: T ⇒ T): Unit = { send((value: T) ⇒ { suspend() - val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-send-off", UnboundedMailbox(), app.settings.ActorTimeoutMillis) - val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) + val pinnedDispatcher = new PinnedDispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, null, "agent-send-off", UnboundedMailbox(), system.settings.ActorTimeoutMillis) + val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) threadBased ! Update(f) value }) @@ -166,11 +166,11 @@ class Agent[T](initialValue: T, app: ActorSystem) { * still be executed in order. */ def alterOff(f: T ⇒ T)(timeout: Timeout): Future[T] = { - val result = new DefaultPromise[T](timeout)(app.dispatcher) + val result = new DefaultPromise[T](timeout)(system.dispatcher) send((value: T) ⇒ { suspend() - val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-alter-off", UnboundedMailbox(), app.settings.ActorTimeoutMillis) - val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) + val pinnedDispatcher = new PinnedDispatcher(system.deadLetterMailbox, system.eventStream, system.scheduler, null, "agent-alter-off", UnboundedMailbox(), system.settings.ActorTimeoutMillis) + val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) result completeWith threadBased.?(Update(f), timeout).asInstanceOf[Future[T]] value }) @@ -192,7 +192,7 @@ class Agent[T](initialValue: T, app: ActorSystem) { * Map this agent to a new agent, applying the function to the internal state. * Does not change the value of this agent. */ - def map[B](f: T ⇒ B): Agent[B] = Agent(f(get))(app) + def map[B](f: T ⇒ B): Agent[B] = Agent(f(get))(system) /** * Flatmap this agent to a new agent, applying the function to the internal state. @@ -262,7 +262,7 @@ class Agent[T](initialValue: T, app: ActorSystem) { * Map this agent to a new agent, applying the function to the internal state. * Does not change the value of this agent. */ - def map[B](f: JFunc[T, B]): Agent[B] = Agent(f(get))(app) + def map[B](f: JFunc[T, B]): Agent[B] = Agent(f(get))(system) /** * Java API: diff --git a/akka-stm/src/test/scala/agent/AgentSpec.scala b/akka-stm/src/test/scala/agent/AgentSpec.scala index 6110ff78c5..07b1e347b5 100644 --- a/akka-stm/src/test/scala/agent/AgentSpec.scala +++ b/akka-stm/src/test/scala/agent/AgentSpec.scala @@ -20,7 +20,7 @@ class CountDownFunction[A](num: Int = 1) extends Function1[A, A] { class AgentSpec extends WordSpec with MustMatchers { - implicit val app = ActorSystem("AgentSpec") + implicit val system = ActorSystem("AgentSpec") implicit val timeout = Timeout(5.seconds.dilated) "Agent" should { diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 5905892157..2d1560d4b3 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -22,14 +22,14 @@ import akka.event.EventStream * @since 1.1 */ class TestActorRef[T <: Actor]( - _app: ActorSystemImpl, + _system: ActorSystemImpl, _deadLetterMailbox: Mailbox, _eventStream: EventStream, _scheduler: Scheduler, _props: Props, _supervisor: ActorRef, name: String) - extends LocalActorRef(_app, _props.withDispatcher(new CallingThreadDispatcher(_deadLetterMailbox, _eventStream, _scheduler)), _supervisor, _supervisor.path / name, false) { + extends LocalActorRef(_system, _props.withDispatcher(new CallingThreadDispatcher(_deadLetterMailbox, _eventStream, _scheduler)), _supervisor, _supervisor.path / name, false) { /** * Directly inject messages into actor receive behavior. Any exceptions * thrown will be available to you, while still being able to use @@ -57,23 +57,23 @@ object TestActorRef { "$" + akka.util.Helpers.base64(l) } - def apply[T <: Actor](factory: ⇒ T)(implicit app: ActorSystem): TestActorRef[T] = apply[T](Props(factory), randomName) + def apply[T <: Actor](factory: ⇒ T)(implicit system: ActorSystem): TestActorRef[T] = apply[T](Props(factory), randomName) - def apply[T <: Actor](factory: ⇒ T, name: String)(implicit app: ActorSystem): TestActorRef[T] = apply[T](Props(factory), name) + def apply[T <: Actor](factory: ⇒ T, name: String)(implicit system: ActorSystem): TestActorRef[T] = apply[T](Props(factory), name) - def apply[T <: Actor](props: Props)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, randomName) + def apply[T <: Actor](props: Props)(implicit system: ActorSystem): TestActorRef[T] = apply[T](props, randomName) - 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, name: String)(implicit system: ActorSystem): TestActorRef[T] = + apply[T](props, system.asInstanceOf[ActorSystemImpl].guardian, name) - def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit app: ActorSystem): TestActorRef[T] = { - val impl = app.asInstanceOf[ActorSystemImpl] - new TestActorRef(impl, impl.deadLetterMailbox, app.eventStream, app.scheduler, props, supervisor, name) + def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit system: ActorSystem): TestActorRef[T] = { + val impl = system.asInstanceOf[ActorSystemImpl] + new TestActorRef(impl, impl.deadLetterMailbox, system.eventStream, system.scheduler, props, supervisor, name) } - def apply[T <: Actor](implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](randomName) + def apply[T <: Actor](implicit m: Manifest[T], system: ActorSystem): TestActorRef[T] = apply[T](randomName) - def apply[T <: Actor](name: String)(implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](Props({ + def apply[T <: Actor](name: String)(implicit m: Manifest[T], system: ActorSystem): TestActorRef[T] = apply[T](Props({ import ReflectiveAccess.{ createInstance, noParams, noArgs } createInstance[T](m.erasure, noParams, noArgs) match { case Right(value) ⇒ value diff --git a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala index 2f061f7eda..3317139f85 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala @@ -25,15 +25,15 @@ object TestBarrier { class TestBarrier(count: Int) { private val barrier = new CyclicBarrier(count) - def await()(implicit app: ActorSystem): Unit = await(TestBarrier.DefaultTimeout) + def await()(implicit system: ActorSystem): Unit = await(TestBarrier.DefaultTimeout) - def await(timeout: Duration)(implicit app: ActorSystem) { + def await(timeout: Duration)(implicit system: ActorSystem) { try { barrier.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) } catch { case e: TimeoutException ⇒ throw new TestBarrierTimeoutException("Timeout of %s and time factor of %s" - format (timeout.toString, app.settings.TestTimeFactor)) + format (timeout.toString, system.settings.TestTimeFactor)) } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala index 7e1e946674..13b3587624 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala @@ -79,17 +79,17 @@ abstract class EventFilter(occurrences: Int) { * Apply this filter while executing the given code block. Care is taken to * remove the filter when the block is finished or aborted. */ - def intercept[T](code: ⇒ T)(implicit app: ActorSystem): T = { - app.eventStream publish TestEvent.Mute(this) + def intercept[T](code: ⇒ T)(implicit system: ActorSystem): T = { + system.eventStream publish TestEvent.Mute(this) try { val result = code - if (!awaitDone(app.settings.TestEventFilterLeeway)) + if (!awaitDone(system.settings.TestEventFilterLeeway)) if (todo > 0) - throw new AssertionError("Timeout (" + app.settings.TestEventFilterLeeway + ") waiting for " + todo + " messages on " + this) + throw new AssertionError("Timeout (" + system.settings.TestEventFilterLeeway + ") waiting for " + todo + " messages on " + this) else throw new AssertionError("Received " + (-todo) + " messages too many on " + this) result - } finally app.eventStream publish TestEvent.UnMute(this) + } finally system.eventStream publish TestEvent.UnMute(this) } /* diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index 311968c451..d630986aa1 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -37,14 +37,14 @@ import akka.event.EventStream * @since 1.2 */ class TestFSMRef[S, D, T <: Actor]( - app: ActorSystemImpl, + system: ActorSystemImpl, _deadLetterMailbox: Mailbox, _eventStream: EventStream, _scheduler: Scheduler, props: Props, supervisor: ActorRef, name: String)(implicit ev: T <:< FSM[S, D]) - extends TestActorRef(app, _deadLetterMailbox, _eventStream, _scheduler, props, supervisor, name) { + extends TestActorRef(system, _deadLetterMailbox, _eventStream, _scheduler, props, supervisor, name) { private def fsm: T = underlyingActor @@ -89,13 +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] = { - val impl = app.asInstanceOf[ActorSystemImpl] + def apply[S, D, T <: Actor](factory: ⇒ T)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = { + val impl = system.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] = { - val impl = app.asInstanceOf[ActorSystemImpl] + def apply[S, D, T <: Actor](factory: ⇒ T, name: String)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = { + val impl = system.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 a66ae79778..43a3173fcc 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -76,11 +76,11 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor { * @author Roland Kuhn * @since 1.1 */ -class TestKit(_app: ActorSystem) { +class TestKit(_system: ActorSystem) { import TestActor.{ Message, RealMessage, NullMessage } - implicit val system = _app + implicit val system = _system private val queue = new LinkedBlockingDeque[Message]() private[akka] var lastMessage: Message = NullMessage @@ -598,7 +598,7 @@ class TestProbe(_application: ActorSystem) extends TestKit(_application) { } object TestProbe { - def apply()(implicit app: ActorSystem) = new TestProbe(app) + def apply()(implicit system: ActorSystem) = new TestProbe(system) } trait ImplicitSender { this: TestKit ⇒ diff --git a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala index 7bf5a59e1b..45a9a5bbc0 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala @@ -21,10 +21,10 @@ class TestLatchNoTimeoutException(message: String) extends RuntimeException(mess object TestLatch { val DefaultTimeout = Duration(5, TimeUnit.SECONDS) - def apply(count: Int = 1)(implicit app: ActorSystem) = new TestLatch(count) + def apply(count: Int = 1)(implicit system: ActorSystem) = new TestLatch(count) } -class TestLatch(count: Int = 1)(implicit app: ActorSystem) { +class TestLatch(count: Int = 1)(implicit system: ActorSystem) { private var latch = new CountDownLatch(count) def countDown() = latch.countDown() @@ -36,7 +36,7 @@ class TestLatch(count: Int = 1)(implicit app: ActorSystem) { def await(timeout: Duration): Boolean = { val opened = latch.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) if (!opened) throw new TestLatchTimeoutException( - "Timeout of %s with time factor of %s" format (timeout.toString, app.settings.TestTimeFactor)) + "Timeout of %s with time factor of %s" format (timeout.toString, system.settings.TestTimeFactor)) opened } @@ -46,7 +46,7 @@ class TestLatch(count: Int = 1)(implicit app: ActorSystem) { def awaitTimeout(timeout: Duration = TestLatch.DefaultTimeout) = { val opened = latch.await(timeout.dilated.toNanos, TimeUnit.NANOSECONDS) if (opened) throw new TestLatchNoTimeoutException( - "Latch opened before timeout of %s with time factor of %s" format (timeout.toString, app.settings.TestTimeFactor)) + "Latch opened before timeout of %s with time factor of %s" format (timeout.toString, system.settings.TestTimeFactor)) opened } diff --git a/akka-testkit/src/main/scala/akka/testkit/package.scala b/akka-testkit/src/main/scala/akka/testkit/package.scala index 12096d2781..939a6cb5ad 100644 --- a/akka-testkit/src/main/scala/akka/testkit/package.scala +++ b/akka-testkit/src/main/scala/akka/testkit/package.scala @@ -5,25 +5,25 @@ import akka.util.Duration import java.util.concurrent.TimeUnit.MILLISECONDS package object testkit { - def filterEvents[T](eventFilters: Iterable[EventFilter])(block: ⇒ T)(implicit app: ActorSystem): T = { + def filterEvents[T](eventFilters: Iterable[EventFilter])(block: ⇒ T)(implicit system: ActorSystem): T = { def now = System.currentTimeMillis - app.eventStream.publish(TestEvent.Mute(eventFilters.toSeq)) + system.eventStream.publish(TestEvent.Mute(eventFilters.toSeq)) try { val result = block - val stop = now + app.settings.TestEventFilterLeeway.toMillis - val failed = eventFilters filterNot (_.awaitDone(Duration(stop - now, MILLISECONDS))) map ("Timeout (" + app.settings.TestEventFilterLeeway + ") waiting for " + _) + val stop = now + system.settings.TestEventFilterLeeway.toMillis + val failed = eventFilters filterNot (_.awaitDone(Duration(stop - now, MILLISECONDS))) map ("Timeout (" + system.settings.TestEventFilterLeeway + ") waiting for " + _) if (failed.nonEmpty) throw new AssertionError("Filter completion error:\n" + failed.mkString("\n")) result } finally { - app.eventStream.publish(TestEvent.UnMute(eventFilters.toSeq)) + system.eventStream.publish(TestEvent.UnMute(eventFilters.toSeq)) } } - def filterEvents[T](eventFilters: EventFilter*)(block: ⇒ T)(implicit app: ActorSystem): T = filterEvents(eventFilters.toSeq)(block) + def filterEvents[T](eventFilters: EventFilter*)(block: ⇒ T)(implicit system: ActorSystem): T = filterEvents(eventFilters.toSeq)(block) - def filterException[T <: Throwable](block: ⇒ Unit)(implicit app: ActorSystem, m: Manifest[T]): Unit = EventFilter[T]() intercept (block) + def filterException[T <: Throwable](block: ⇒ Unit)(implicit system: ActorSystem, m: Manifest[T]): Unit = EventFilter[T]() intercept (block) } diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index 73f7cf2240..c365cd43fa 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -56,14 +56,14 @@ class AkkaSpecSpec extends WordSpec with MustMatchers { "An AkkaSpec" must { "terminate all actors" in { import ActorSystem.defaultConfig - val app = ActorSystem("test", defaultConfig ++ Configuration( + val system = ActorSystem("test", defaultConfig ++ Configuration( "akka.actor.debug.lifecycle" -> true, "akka.actor.debug.event-stream" -> true, "akka.loglevel" -> "DEBUG", "akka.stdout-loglevel" -> "DEBUG")) - val spec = new AkkaSpec(app) { - val ref = Seq(testActor, app.actorOf(Props.empty, "name")) + val spec = new AkkaSpec(system) { + val ref = Seq(testActor, system.actorOf(Props.empty, "name")) } spec.ref foreach (_ must not be 'shutdown) - app.stop() + system.stop() spec.awaitCond(spec.ref forall (_.isShutdown), 2 seconds) } } diff --git a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java index f73550f16a..ca8fe597f7 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java +++ b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java @@ -24,7 +24,7 @@ // public class Pi { -// private static final ActorSystem app = new ActorSystem(); +// private static final ActorSystem system = new ActorSystem(); // public static void main(String[] args) throws Exception { // Pi pi = new Pi(); @@ -109,11 +109,11 @@ // LinkedList workers = new LinkedList(); // for (int i = 0; i < nrOfWorkers; i++) { -// ActorRef worker = app.actorOf(Worker.class); +// ActorRef worker = system.actorOf(Worker.class); // workers.add(worker); // } -// router = app.actorOf(new RoutedProps().withRoundRobinRouter().withLocalConnections(workers), "pi"); +// router = system.actorOf(new RoutedProps().withRoundRobinRouter().withLocalConnections(workers), "pi"); // } // // message handler @@ -167,7 +167,7 @@ // final CountDownLatch latch = new CountDownLatch(1); // // create the master -// ActorRef master = app.actorOf(new UntypedActorFactory() { +// ActorRef master = system.actorOf(new UntypedActorFactory() { // public UntypedActor create() { // return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch); // } diff --git a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala index 3ea86ae6d6..836f766e12 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala @@ -12,7 +12,7 @@ // object Pi extends App { -// val app = ActorSystem() +// val system = ActorSystem() // calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000) @@ -56,10 +56,10 @@ // var start: Long = _ // // create the workers -// val workers = Vector.fill(nrOfWorkers)(app.actorOf[Worker]) +// val workers = Vector.fill(nrOfWorkers)(system.actorOf[Worker]) // // wrap them with a load-balancing router -// val router = app.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi") +// val router = system.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi") // // message handler // def receive = { @@ -102,7 +102,7 @@ // val latch = new CountDownLatch(1) // // create the master -// val master = app.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)) +// val master = system.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)) // // start the calculation // master ! Calculate From 4470cf0d257bac460c0b6fd6cc5c32feecd57fdf Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 17 Nov 2011 13:27:33 +0100 Subject: [PATCH 14/14] =?UTF-8?q?incorporate=20Viktor=E2=80=99s=20review?= =?UTF-8?q?=20comments?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../test/scala/akka/actor/DeployerSpec.scala | 4 +- .../scala/akka/actor/ActorRefProvider.scala | 46 ++++++++++++------- .../main/scala/akka/actor/ActorSystem.scala | 2 + .../src/main/scala/akka/remote/Remote.scala | 5 +- 4 files changed, 34 insertions(+), 23 deletions(-) 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 5750fbe967..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,12 +14,12 @@ class DeployerSpec extends AkkaSpec { "A Deployer" must { "be able to parse 'akka.actor.deployment._' config elements" in { - val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookupInConfig("/system/service-ping") + val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookupInConfig("/app/service-ping") deployment must be('defined) deployment must equal(Some( Deploy( - "/system/service-ping", + "/app/service-ping", None, RoundRobin, NrOfInstances(3), diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index d4cd9ea019..1eb4ecef40 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -32,23 +32,32 @@ trait ActorRefProvider { def deathWatch: DeathWatch + // FIXME: remove/replace def nodename: String def settings: ActorSystem.Settings def init(system: ActorSystemImpl) - /** - * What deployer will be used to resolve deployment configuration? - */ private[akka] def deployer: Deployer private[akka] def scheduler: Scheduler + /** + * Create an Actor with the given name below the given supervisor. + */ private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef + /** + * Create an Actor with the given full path below the given supervisor. + * + * FIXME: Remove! this is dangerous! + */ private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, path: ActorPath, systemService: Boolean): ActorRef + /** + * Remove this path from the lookup map. + */ private[akka] def evict(path: String): Boolean private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] @@ -57,13 +66,16 @@ trait ActorRefProvider { private[akka] def createDeathWatch(): DeathWatch + /** + * Create AskActorRef to hook up message send to recipient with Future receiver. + */ private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] - private[akka] def theOneWhoWalksTheBubblesOfSpaceTime: ActorRef - + /** + * This Future is completed upon termination of this ActorRefProvider, which + * is usually initiated by stopping the guardian via ActorSystem.stop(). + */ private[akka] def terminationFuture: Future[ActorSystem.ExitStatus] - - private[akka] def tempName: String } /** @@ -134,6 +146,7 @@ class LocalActorRefProvider( val log = Logging(eventStream, this) + // FIXME remove/replave (clustering shall not leak into akka-actor) val nodename: String = System.getProperty("akka.cluster.nodename") match { case null | "" ⇒ new UUID().toString case value ⇒ value @@ -147,10 +160,7 @@ class LocalActorRefProvider( * generate name for temporary actor refs */ private val tempNumber = new AtomicLong - def tempName = { - val l = tempNumber.getAndIncrement() - "$_" + Helpers.base64(l) - } + def tempName = "$_" + Helpers.base64(tempNumber.getAndIncrement()) private val tempNode = rootPath / "tmp" def tempPath = tempNode / tempName @@ -225,19 +235,21 @@ class LocalActorRefProvider( @volatile private var rootGuardian: ActorRef = _ @volatile - var guardian: ActorRef = _ + private var _guardian: ActorRef = _ @volatile - var systemGuardian: ActorRef = _ + private var _systemGuardian: ActorRef = _ + def guardian = _guardian + def systemGuardian = _systemGuardian val deathWatch = createDeathWatch() def init(system: ActorSystemImpl) { rootGuardian = actorOf(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) - guardian = actorOf(system, guardianProps, rootGuardian, "system", true) - systemGuardian = actorOf(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, "sys", true) + _guardian = actorOf(system, guardianProps, rootGuardian, "app", true) + _systemGuardian = actorOf(system, 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) + 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 "system" name for now) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 9cfb9bb156..7d9074d7bf 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -208,6 +208,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst override def numberOfMessages = 0 } + // FIXME make this configurable val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512)) // TODO correctly pull its config from the config @@ -255,6 +256,7 @@ class ActorSystemImpl(val name: String, config: Configuration) extends ActorSyst def /(actorName: String): ActorPath = guardian.path / actorName def start(): this.type = { + if (_serialization != null) throw new IllegalStateException("cannot initialize ActorSystemImpl twice!") _serialization = new Serialization(this) _typedActor = new TypedActor(settings, _serialization) provider.init(this) diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index c27bd67093..a140f1f252 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -185,10 +185,7 @@ 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 tempName = "$_" + Helpers.base64(tempNumber.getAndIncrement()) def tempPath = remoteDaemon.path / tempName // FIXME: handle real remote supervision