From 37f21c7d9b315ff53b8b3e33adc0079455935626 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bone=CC=81r?= Date: Mon, 5 Dec 2011 14:02:12 +0100 Subject: [PATCH 1/6] Fixed string concatenation error --- akka-actor/src/main/scala/akka/actor/Actor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index d39b0a2270..0770448c3b 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -200,7 +200,7 @@ trait Actor { "\n\tYou cannot create an instance of " + getClass.getName + " explicitly using the constructor (new)." + "\n\tYou have to use one of the factory methods to create a new actor. Either use:" + "\n\t\t'val actor = context.actorOf[MyActor]' (to create a supervised child actor from within an actor), or" + - "\n\t\t'val actor = system.actorOf(new MyActor(..))' (to create a top level actor from the ActorSystem), or") + "\n\t\t'val actor = system.actorOf(new MyActor(..))' (to create a top level actor from the ActorSystem), or" + "\n\t\t'val actor = context.actorOf[MyActor]' (to create a supervised child actor from within an actor), or" + "\n\t\t'val actor = system.actorOf(new MyActor(..))' (to create a top level actor from the ActorSystem)") From 05da3f332145ce4e8222c398264b3784d849b614 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bone=CC=81r?= Date: Tue, 6 Dec 2011 15:27:10 +0100 Subject: [PATCH 2/6] Minor formatting, docs and logging edits. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Jonas Bonér --- .../test/scala/akka/actor/ActorRefSpec.scala | 2 +- .../src/main/scala/akka/actor/Actor.scala | 13 ++++++++----- .../src/main/scala/akka/actor/ActorRef.scala | 18 ++++++++++++------ .../main/scala/akka/actor/ActorSystem.scala | 12 +++++------- .../src/main/scala/akka/event/Logging.scala | 2 +- 5 files changed, 27 insertions(+), 20 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 613b3c7b36..8f0e7727ad 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -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.currentSystem.withValue(system) { ... }" + " Use 'akka.serialization.Serialization.currentSystem.withValue(system) { ... }'" } "must throw exception on deserialize if not present in actor hierarchy (and remoting is not enabled)" in { diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 0770448c3b..ec7fe39092 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -78,7 +78,8 @@ class ActorKilledException private[akka] (message: String, cause: Throwable) } case class ActorInitializationException private[akka] (actor: ActorRef, message: String, cause: Throwable = null) - extends AkkaException(message, cause) with NoStackTrace { + extends AkkaException(message, cause) + with NoStackTrace { def this(msg: String) = this(null, msg, null); } @@ -94,11 +95,13 @@ class InvalidMessageException private[akka] (message: String, cause: Throwable = } case class DeathPactException private[akka] (dead: ActorRef) - extends AkkaException("monitored actor " + dead + " terminated") + extends AkkaException("Monitored actor [" + dead + "] terminated") with NoStackTrace // must not pass InterruptedException to other threads -case class ActorInterruptedException private[akka] (cause: Throwable) extends AkkaException(cause.getMessage, cause) with NoStackTrace +case class ActorInterruptedException private[akka] (cause: Throwable) + extends AkkaException(cause.getMessage, cause) + with NoStackTrace /** * This message is thrown by default when an Actors behavior doesn't match a message @@ -109,7 +112,7 @@ case class UnhandledMessageException(msg: Any, ref: ActorRef = null) extends Exc // constructor with 'null' ActorRef needed to work with client instantiation of remote exception override def getMessage = - if (ref ne null) "Actor %s does not handle [%s]".format(ref, msg) + if (ref ne null) "Actor [%s] does not handle [%s]".format(ref, msg) else "Actor does not handle [%s]".format(msg) override def fillInStackTrace() = this //Don't waste cycles generating stack trace @@ -162,7 +165,7 @@ object Actor { object emptyBehavior extends Receive { def isDefinedAt(x: Any) = false - def apply(x: Any) = throw new UnsupportedOperationException("empty behavior apply()") + def apply(x: Any) = throw new UnsupportedOperationException("Empty behavior apply()") } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index c3865c001b..370528afff 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -162,7 +162,8 @@ class LocalActorRef private[akka] ( /** * Is the actor terminated? - * If this method returns true, it will never return false again, but if it returns false, you cannot be sure if it's alive still (race condition) + * If this method returns true, it will never return false again, but if it + * returns false, you cannot be sure if it's alive still (race condition) */ override def isTerminated: Boolean = actorCell.isTerminated @@ -271,7 +272,7 @@ case class SerializedActorRef(hostname: String, port: Int, path: String) { def readResolve(): AnyRef = currentSystem.value match { case null ⇒ throw new IllegalStateException( "Trying to deserialize a serialized ActorRef without an ActorSystem in scope." + - " Use akka.serialization.Serialization.currentSystem.withValue(system) { ... }") + " Use 'akka.serialization.Serialization.currentSystem.withValue(system) { ... }'") case someSystem ⇒ someSystem.provider.deserialize(this) match { case Some(actor) ⇒ actor case None ⇒ throw new IllegalStateException("Could not deserialize ActorRef") @@ -287,7 +288,7 @@ trait MinimalActorRef extends ActorRef with ScalaActorRef with RefInternals { private[akka] val uuid: Uuid = newUuid() def name: String = uuid.toString - //FIXME REMOVE THIS, ticket #1416 + //FIXME REMOVE THIS, ticket #1416 //FIXME REMOVE THIS, ticket #1415 def suspend(): Unit = () def resume(): Unit = () @@ -299,7 +300,7 @@ trait MinimalActorRef extends ActorRef with ScalaActorRef with RefInternals { def !(message: Any)(implicit sender: ActorRef = null): Unit = () def ?(message: Any)(implicit timeout: Timeout): Future[Any] = - throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName)) + throw new UnsupportedOperationException("Not supported for [%s]".format(getClass.getName)) protected[akka] def sendSystemMessage(message: SystemMessage): Unit = () protected[akka] def restart(cause: Throwable): Unit = () @@ -328,7 +329,7 @@ class DeadLetterActorRef(val eventStream: EventStream) extends MinimalActorRef { private[akka] def init(dispatcher: MessageDispatcher, rootPath: ActorPath) { _path = rootPath / "nul" - brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(dispatcher) + brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef - promises are always broken")))(dispatcher) } override val name: String = "dead-letter" @@ -353,7 +354,12 @@ class DeadLetterActorRef(val eventStream: EventStream) extends MinimalActorRef { private def writeReplace(): AnyRef = DeadLetterActorRef.serialized } -abstract class AskActorRef(val path: ActorPath, provider: ActorRefProvider, deathWatch: DeathWatch, timeout: Timeout, val dispatcher: MessageDispatcher) 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) override def name = path.name diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index fef236d5fd..ea3f94288b 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -30,17 +30,17 @@ object ActorSystem { val Version = "2.0-SNAPSHOT" - val envHome = System.getenv("AKKA_HOME") match { + val EnvHome = System.getenv("AKKA_HOME") match { case null | "" | "." ⇒ None case value ⇒ Some(value) } - val systemHome = System.getProperty("akka.home") match { + val SystemHome = System.getProperty("akka.home") match { case null | "" ⇒ None case value ⇒ Some(value) } - val GlobalHome = systemHome orElse envHome + val GlobalHome = SystemHome orElse EnvHome def create(name: String, config: Config): ActorSystem = apply(name, config) def apply(name: String, config: Config): ActorSystem = new ActorSystemImpl(name, config).start() @@ -49,6 +49,7 @@ object ActorSystem { * Uses the standard default Config from ConfigFactory.load(), since none is provided. */ def create(name: String): ActorSystem = apply(name) + /** * Uses the standard default Config from ConfigFactory.load(), since none is provided. */ @@ -107,7 +108,6 @@ object ActorSystem { "] does not match the provided config version [" + ConfigVersion + "]") override def toString: String = config.root.render - } // TODO move to migration kit @@ -151,9 +151,7 @@ object ActorSystem { } catch { case _ ⇒ None } private def emptyConfig = ConfigFactory.systemProperties - } - } /** @@ -380,7 +378,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor protected def createScheduler(): Scheduler = { val threadFactory = new MonitorableThreadFactory("DefaultScheduler") val hwt = new HashedWheelTimer(log, threadFactory, settings.SchedulerTickDuration, settings.SchedulerTicksPerWheel) - // note that dispatcher is by-name parameter in DefaultScheduler constructor, + // note that dispatcher is by-name parameter in DefaultScheduler constructor, // because dispatcher is not initialized when the scheduler is created def safeDispatcher = { if (dispatcher eq null) { diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 1bff28ea44..07e0149906 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -22,7 +22,7 @@ object LoggingBus { /** * This trait brings log level handling to the EventStream: it reads the log - * levels for the initial logging (StandardOutLogger) and the loggers&level + * levels for the initial logging (StandardOutLogger) and the loggers & level * for after-init logging, possibly keeping the StandardOutLogger enabled if * it is part of the configured loggers. All configured loggers are treated as * system services and managed by this trait, i.e. subscribed/unsubscribed in From 831b3272ad4829db8ac4087a7934e21507cbee96 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 6 Dec 2011 20:33:25 +0100 Subject: [PATCH 3/6] add min/max bounds on absolute number of threads for dispatcher The reason for the previously failing test case was that Jenkins has only two (logical) cores and the test config sets the core-pool-size-factor to 2, meaning four threads. This is not enough to have four Futures waiting (as per the test) and one actor which actually does the work (the guardian in this case). Hence, make it so that core-pool-size-min gives the absolute minimum and set the default of that to eight. While doing so I cleaned up the MessageDispatcherConfigurator to not use HOF since now configuration items are not optional anymore, yielding a flow which is much more readily understandable. --- akka-actor/src/main/resources/reference.conf | 5 +++ .../akka/dispatch/AbstractDispatcher.scala | 32 +++++++++++-------- 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 63f6cc21d9..8d96834092 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -105,8 +105,13 @@ akka { # A FQCN to a class inheriting MessageDispatcherConfigurator with a no-arg visible constructor name = "DefaultDispatcher" # Optional, will be a generated UUID if omitted keep-alive-time = 60s # Keep alive time for threads + core-pool-size-min = 8 # minimum number of threads to cap factor-based core number to core-pool-size-factor = 8.0 # No of core threads ... ceil(available processors * factor) + core-pool-size-max = 4096 # maximum number of threads to cap factor-based number to + # Hint: max-pool-size is only used for bounded task queues + max-pool-size-min = 8 # minimum number of threads to cap factor-based max number to max-pool-size-factor = 8.0 # Max no of threads ... ceil(available processors * factor) + max-pool-size-max = 4096 # maximum number of threads to cap factor-based max number to task-queue-size = -1 # Specifies the bounded capacity of the task queue (< 1 == unbounded) task-queue-type = "linked" # Specifies which type of task queue will be used, can be "array" or "linked" (default) allow-core-timeout = on # Allow core threads to time out diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 56ef4dd336..9ee916f8f8 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -310,22 +310,26 @@ abstract class MessageDispatcherConfigurator() { settings: Settings, createDispatcher: ⇒ (ThreadPoolConfig) ⇒ MessageDispatcher): ThreadPoolConfigDispatcherBuilder = { import ThreadPoolConfigDispatcherBuilder.conf_? + import scala.math.{ min, max } //Apply the following options to the config if they are present in the config - ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig()).configure( - conf_?(Some(config getMilliseconds "keep-alive-time"))(time ⇒ _.setKeepAliveTime(Duration(time, TimeUnit.MILLISECONDS))), - conf_?(Some(config getDouble "core-pool-size-factor"))(factor ⇒ _.setCorePoolSizeFromFactor(factor)), - conf_?(Some(config getDouble "max-pool-size-factor"))(factor ⇒ _.setMaxPoolSizeFromFactor(factor)), - conf_?(Some(config getBoolean "allow-core-timeout"))(allow ⇒ _.setAllowCoreThreadTimeout(allow)), - conf_?(Some(config getInt "task-queue-size") flatMap { - case size if size > 0 ⇒ - Some(config getString "task-queue-type") map { - case "array" ⇒ ThreadPoolConfig.arrayBlockingQueue(size, false) //TODO config fairness? - case "" | "linked" ⇒ ThreadPoolConfig.linkedBlockingQueue(size) - case x ⇒ throw new IllegalArgumentException("[%s] is not a valid task-queue-type [array|linked]!" format x) - } - case _ ⇒ None - })(queueFactory ⇒ _.setQueueFactory(queueFactory))) + ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig()) + .setKeepAliveTime(Duration(config getMilliseconds "keep-alive-time", TimeUnit.MILLISECONDS)) + .setAllowCoreThreadTimeout(config getBoolean "allow-core-timeout") + .setCorePoolSize(min(max(ThreadPoolConfig.scaledPoolSize(config getDouble "core-pool-size-factor"), + config getInt "core-pool-size-min"), config getInt "core-pool-size-max")) + .setMaxPoolSize(min(max(ThreadPoolConfig.scaledPoolSize(config getDouble "max-pool-size-factor"), + config getInt "max-pool-size-min"), config getInt "max-pool-size-max")) + .configure( + conf_?(Some(config getInt "task-queue-size") flatMap { + case size if size > 0 ⇒ + Some(config getString "task-queue-type") map { + case "array" ⇒ ThreadPoolConfig.arrayBlockingQueue(size, false) //TODO config fairness? + case "" | "linked" ⇒ ThreadPoolConfig.linkedBlockingQueue(size) + case x ⇒ throw new IllegalArgumentException("[%s] is not a valid task-queue-type [array|linked]!" format x) + } + case _ ⇒ None + })(queueFactory ⇒ _.setQueueFactory(queueFactory))) } } From a1d8f306698a2f61d57f696e9f69ca1b5a6fdc28 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 6 Dec 2011 21:34:29 +0100 Subject: [PATCH 4/6] fix bug in creating anonymous actors - when no name was given, it resulted in a simple call to the provider - which is completely the wrong thing to do in case of ActorSystem - so, introduce CreateRandomNameChild op for guardians and use that --- .../src/main/scala/akka/actor/ActorCell.scala | 14 +++++++++---- .../scala/akka/actor/ActorRefProvider.scala | 21 ++++++++++++------- .../main/scala/akka/actor/ActorSystem.scala | 11 +++++++--- 3 files changed, 31 insertions(+), 15 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 6660fb38c4..a960d8c8eb 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -101,14 +101,20 @@ private[akka] class ActorCell( var childrenRefs: TreeMap[String, ChildRestartStats] = emptyChildrenRefs + private def _actorOf(props: Props, name: String): ActorRef = { + val actor = provider.actorOf(systemImpl, props, guardian, name, false) + childrenRefs = childrenRefs.updated(name, ChildRestartStats(actor)) + actor + } + + def actorOf(props: Props): ActorRef = _actorOf(props, randomName()) + def actorOf(props: Props, name: String): ActorRef = { if (name == null || name == "" || name.charAt(0) == '$') throw new InvalidActorNameException("actor name must not be null, empty or start with $") if (childrenRefs contains name) throw new InvalidActorNameException("actor name " + name + " is not unique!") - val actor = provider.actorOf(systemImpl, props, guardian, name, false) - childrenRefs = childrenRefs.updated(name, ChildRestartStats(actor)) - actor + _actorOf(props, name) } var currentMessage: Envelope = null @@ -123,7 +129,7 @@ private[akka] class ActorCell( var nextNameSequence: Long = 0 //Not thread safe, so should only be used inside the actor that inhabits this ActorCell - override protected def randomName(): String = { + protected def randomName(): String = { val n = nextNameSequence + 1 nextNameSequence = n Helpers.base64(n) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 91c3115c32..5690b9b9b6 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -137,8 +137,6 @@ trait ActorRefFactory { protected def lookupRoot: InternalActorRef - protected def randomName(): String - /** * Create new actor as child of this context and give it an automatically * generated name (currently similar to base64-encoded integer count, @@ -146,7 +144,7 @@ trait ActorRefFactory { * * See [[akka.actor.Props]] for details on how to obtain a `Props` object. */ - def actorOf(props: Props): ActorRef = provider.actorOf(systemImpl, props, guardian, randomName(), false) + def actorOf(props: Props): ActorRef /** * Create new actor as child of this context with the given name, which must @@ -298,6 +296,11 @@ class ActorRefProviderException(message: String) extends AkkaException(message) */ private[akka] case class CreateChild(props: Props, name: String) +/** + * Internal Akka use only, used in implementation of system.actorOf. + */ +private[akka] case class CreateRandomNameChild(props: Props) + /** * Local ActorRef provider. */ @@ -366,9 +369,10 @@ class LocalActorRefProvider( private class Guardian extends Actor { def receive = { - case Terminated(_) ⇒ context.self.stop() - case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) - case m ⇒ deadLetters ! DeadLetter(m, sender, self) + case Terminated(_) ⇒ context.self.stop() + case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) + case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e }) + case m ⇒ deadLetters ! DeadLetter(m, sender, self) } } @@ -377,8 +381,9 @@ class LocalActorRefProvider( case Terminated(_) ⇒ eventStream.stopDefaultLoggers() context.self.stop() - case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) - case m ⇒ deadLetters ! DeadLetter(m, sender, self) + case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) + case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e }) + case m ⇒ deadLetters ! DeadLetter(m, sender, self) } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 6119abf15d..6da1ee96ad 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -315,6 +315,14 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor } } + def actorOf(props: Props): ActorRef = { + implicit val timeout = settings.CreationTimeout + (guardian ? CreateRandomNameChild(props)).get match { + case ref: ActorRef ⇒ ref + case ex: Exception ⇒ throw ex + } + } + import settings._ // this provides basic logging (to stdout) until .start() is called below @@ -369,9 +377,6 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor def nodename: String = provider.nodename def clustername: String = provider.clustername - private final val nextName = new AtomicLong - override protected def randomName(): String = Helpers.base64(nextName.incrementAndGet()) - def /(actorName: String): ActorPath = guardian.path / actorName def /(path: Iterable[String]): ActorPath = guardian.path / path From c4ed57100ee6f9d03c0a10dca8aef18d12911a18 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 6 Dec 2011 22:50:19 +0100 Subject: [PATCH 5/6] make Jenkins wait for Davy Jones MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - AkkaSpecSpec failed with nothing in the locker, but only on Jenkins - cause: Davy Jones doesn’t get a chance to run because nobody actually waits for his processing - thus, give him permission to leave this world and have him confirm that --- akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index 66e86a476a..5ba6e94c8c 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -128,6 +128,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers { implicit val davyJones = otherSystem.actorOf(Props(new Actor { def receive = { case m: DeadLetter ⇒ locker :+= m + case "Die!" ⇒ sender ! "finally gone"; self.stop() } }), "davyJones") @@ -148,6 +149,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers { system.registerOnTermination(latch.countDown()) system.stop() latch.await(2 seconds) + (davyJones ? "Die!").get must be === "finally gone" // this will typically also contain log messages which were sent after the logger shutdown locker must contain(DeadLetter(42, davyJones, probe.ref)) From d8ede288d78e03e9c86da96e63add8087e7ce0cb Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 6 Dec 2011 23:09:29 +0100 Subject: [PATCH 6/6] improve ScalaDoc of actorOf methods (esp. their blocking on ActorSystem) --- .../src/main/scala/akka/actor/ActorCell.scala | 25 +++++++++++++ .../scala/akka/actor/ActorRefProvider.scala | 35 ++++++++++++++++++- .../main/scala/akka/actor/ActorSystem.scala | 25 +++++++++++++ 3 files changed, 84 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index a960d8c8eb..eab3351e20 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -15,6 +15,31 @@ import akka.util.{ Duration, Helpers } * The actor context - the view of the actor cell from the actor. * Exposes contextual information for the actor and the current message. * TODO: everything here for current compatibility - could be limited more + * + * There are several possibilities for creating actors (see [[akka.actor.Props]] + * for details on `props`): + * + * {{{ + * // Java or Scala + * context.actorOf(props, "name") + * context.actorOf(props) + * + * // Scala + * context.actorOf[MyActor]("name") + * context.actorOf[MyActor] + * context.actorOf(new MyActor(...)) + * + * // Java + * context.actorOf(classOf[MyActor]); + * context.actorOf(new Creator() { + * public MyActor create() { ... } + * }); + * context.actorOf(new Creator() { + * public MyActor create() { ... } + * }, "name"); + * }}} + * + * Where no name is given explicitly, one will be automatically generated. */ trait ActorContext extends ActorRefFactory { diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 5690b9b9b6..299f637ceb 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -120,7 +120,8 @@ trait ActorRefProvider { } /** - * Interface implemented by ActorSystem and AkkaContext, the only two places from which you can get fresh actors. + * Interface implemented by ActorSystem and AkkaContext, the only two places + * from which you can get fresh actors. */ trait ActorRefFactory { @@ -143,6 +144,10 @@ trait ActorRefFactory { * reversed and with “$” prepended, may change in the future). * * See [[akka.actor.Props]] for details on how to obtain a `Props` object. + * + * When invoked on ActorSystem, this method sends a message to the guardian + * actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in + * the `reference.conf`. */ def actorOf(props: Props): ActorRef @@ -152,6 +157,10 @@ trait ActorRefFactory { * and `InvalidActorNameException` is thrown. * * See [[akka.actor.Props]] for details on how to obtain a `Props` object. + * + * When invoked on ActorSystem, this method sends a message to the guardian + * actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in + * the `reference.conf`. */ def actorOf(props: Props, name: String): ActorRef @@ -160,6 +169,10 @@ trait ActorRefFactory { * generated name (currently similar to base64-encoded integer count, * reversed and with “$” prepended, may change in the future). The type must have * a no-arg constructor which will be invoked using reflection. + * + * When invoked on ActorSystem, this method sends a message to the guardian + * actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in + * the `reference.conf`. */ def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]])) @@ -168,6 +181,10 @@ trait ActorRefFactory { * not be null, empty or start with “$”. If the given name is already in use, * and `InvalidActorNameException` is thrown. The type must have * a no-arg constructor which will be invoked using reflection. + * + * When invoked on ActorSystem, this method sends a message to the guardian + * actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in + * the `reference.conf`. */ def actorOf[T <: Actor](name: String)(implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]), name) @@ -177,6 +194,10 @@ trait ActorRefFactory { * generated name (currently similar to base64-encoded integer count, * reversed and with “$” prepended, may change in the future). The class must have * a no-arg constructor which will be invoked using reflection. + * + * When invoked on ActorSystem, this method sends a message to the guardian + * actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in + * the `reference.conf`. */ def actorOf[T <: Actor](clazz: Class[T]): ActorRef = actorOf(Props(clazz)) @@ -186,6 +207,10 @@ trait ActorRefFactory { * reversed and with “$” prepended, may change in the future). Use this * method to pass constructor arguments to the [[akka.actor.Actor]] while using * only default [[akka.actor.Props]]; otherwise refer to `actorOf(Props)`. + * + * When invoked on ActorSystem, this method sends a message to the guardian + * actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in + * the `reference.conf`. */ def actorOf(factory: ⇒ Actor): ActorRef = actorOf(Props(() ⇒ factory)) @@ -195,6 +220,10 @@ trait ActorRefFactory { * count, reversed and with “$” prepended, may change in the future). * * Identical to `actorOf(Props(() => creator.create()))`. + * + * When invoked on ActorSystem, this method sends a message to the guardian + * actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in + * the `reference.conf`. */ def actorOf(creator: UntypedActorFactory): ActorRef = actorOf(Props(() ⇒ creator.create())) @@ -204,6 +233,10 @@ trait ActorRefFactory { * and `InvalidActorNameException` is thrown. * * Identical to `actorOf(Props(() => creator.create()), name)`. + * + * When invoked on ActorSystem, this method sends a message to the guardian + * actor and blocks waiting for a reply, see `akka.actor.creation-timeout` in + * the `reference.conf`. */ def actorOf(creator: UntypedActorFactory, name: String): ActorRef = actorOf(Props(() ⇒ creator.create()), name) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 6da1ee96ad..16edec52fa 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -158,6 +158,31 @@ object ActorSystem { * An actor system is a hierarchical group of actors which share common * configuration, e.g. dispatchers, deployments, remote capabilities and * addresses. It is also the entry point for creating or looking up actors. + * + * There are several possibilities for creating actors (see [[akka.actor.Props]] + * for details on `props`): + * + * {{{ + * // Java or Scala + * system.actorOf(props, "name") + * system.actorOf(props) + * + * // Scala + * system.actorOf[MyActor]("name") + * system.actorOf[MyActor] + * system.actorOf(new MyActor(...)) + * + * // Java + * system.actorOf(classOf[MyActor]); + * system.actorOf(new Creator() { + * public MyActor create() { ... } + * }); + * system.actorOf(new Creator() { + * public MyActor create() { ... } + * }, "name"); + * }}} + * + * Where no name is given explicitly, one will be automatically generated. */ abstract class ActorSystem extends ActorRefFactory { import ActorSystem._