From 2ce47d6bb5e01b3862d3d9992a5b8f0e6237c903 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 9 Feb 2012 11:56:43 +0100 Subject: [PATCH 01/39] Bye-bye ReflectiveAccess, introducing PropertyMaster, see #1750 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - PropertyMaster is the only place in Akka which calls ClassLoader.getClass (apart from kernel, which might be special) - all PropertyMaster methods (there are only three) take a ClassManifest of what is to be constructed, and they verify that the obtained object is actually compatible with the required type Other stuff: - noticed that I had forgotten to change to ExtendedActorSystem when constructing Extensions by ExtensionKey (damn you, reflection!) - moved Serializer.currentSystem into JavaSerializer, because that’s the only one needing it (it’s only used in readResolve() methods) - Serializers are constructed now with one-arg constructor taking ExtendedActorSystem (if that exists, otherwise no-arg as before), to allow JavaSerializer to do its magic; possibly necessary for others as well - Removed all Option[ClassLoader] signatures - made it so that the ActorSystem will try context class loader, then the class loader which loaded the class actually calling into ActorSystem.apply, then the loader which loaded ActorSystemImpl - for the second of the above I added a (reflectively accessed hopefully safe) facility for getting caller Class[_] objects by using sun.reflect.Reflection; this is optional an defaults to None, e.g. on Android, which means that getting the caller’s classloader is done on a best effort basis (there’s nothing we can do because a StackTrace does not contain actual Class[_] objects). - refactored DurableMailbox to contain the owner val and use that instead of declaring that in all subclasses --- .../test/java/akka/actor/JavaExtension.java | 4 +- .../test/scala/akka/actor/ActorRefSpec.scala | 17 ++- .../scala/akka/actor/TypedActorSpec.scala | 6 +- .../akka/serialization/SerializeSpec.scala | 13 +- .../src/main/scala/akka/actor/ActorCell.scala | 2 +- .../src/main/scala/akka/actor/ActorRef.scala | 4 +- .../scala/akka/actor/ActorRefProvider.scala | 4 +- .../main/scala/akka/actor/ActorSystem.scala | 45 +++++-- .../src/main/scala/akka/actor/Deployer.scala | 5 +- .../src/main/scala/akka/actor/Extension.scala | 10 +- .../scala/akka/actor/PropertyMaster.scala | 102 ++++++++++++++ .../main/scala/akka/actor/TypedActor.scala | 8 +- .../akka/dispatch/AbstractDispatcher.scala | 11 +- .../scala/akka/dispatch/Dispatchers.scala | 24 ++-- .../src/main/scala/akka/event/Logging.scala | 5 +- .../scala/akka/serialization/Format.scala | 5 +- .../akka/serialization/Serialization.scala | 46 ++----- .../scala/akka/serialization/Serializer.scala | 79 +++++++---- .../src/main/scala/akka/util/Reflect.scala | 18 +++ .../scala/akka/util/ReflectiveAccess.scala | 126 ------------------ .../SerializationDocTestBase.java | 7 +- .../serialization/SerializationDocSpec.scala | 9 +- .../actor/mailbox/BeanstalkBasedMailbox.scala | 2 +- .../actor/mailbox/FiledBasedMailbox.scala | 2 +- .../akka/actor/mailbox/DurableMailbox.scala | 14 +- .../actor/mailbox/BSONSerialization.scala | 2 +- .../actor/mailbox/MongoBasedMailbox.scala | 2 +- .../actor/mailbox/RedisBasedMailbox.scala | 2 +- .../actor/mailbox/ZooKeeperBasedMailbox.scala | 2 +- .../scala/akka/remote/MessageSerializer.scala | 27 ++-- .../akka/remote/RemoteActorRefProvider.scala | 7 +- .../scala/akka/remote/RemoteDeployer.scala | 2 +- .../scala/akka/remote/RemoteTransport.scala | 2 +- .../serialization/ProtobufSerializer.scala | 3 +- .../scala/akka/testkit/TestActorRef.scala | 5 +- 35 files changed, 322 insertions(+), 300 deletions(-) create mode 100644 akka-actor/src/main/scala/akka/actor/PropertyMaster.scala create mode 100644 akka-actor/src/main/scala/akka/util/Reflect.scala delete mode 100644 akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java index 28b87bb5db..f8f22ec8c5 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java @@ -40,9 +40,9 @@ public class JavaExtension { static final ExtensionKey key = new ExtensionKey(OtherExtension.class) { }; - public final ActorSystemImpl system; + public final ExtendedActorSystem system; - public OtherExtension(ActorSystemImpl i) { + public OtherExtension(ExtendedActorSystem i) { system = i; } } 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 a2c3c7da5a..e8c667bc7e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -11,11 +11,10 @@ import akka.testkit._ import akka.util.Timeout import akka.util.duration._ import java.lang.IllegalStateException -import akka.util.ReflectiveAccess -import akka.serialization.Serialization import java.util.concurrent.{ CountDownLatch, TimeUnit } import akka.dispatch.{ Await, DefaultPromise, Promise, Future } import akka.pattern.ask +import akka.serialization.JavaSerializer object ActorRefSpec { @@ -240,6 +239,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { "be serializable using Java Serialization on local node" in { val a = system.actorOf(Props[InnerActor]) + val esys = system.asInstanceOf[ExtendedActorSystem] import java.io._ @@ -251,14 +251,21 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { out.flush out.close - Serialization.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) { - val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) + val bytes = baos.toByteArray + + JavaSerializer.currentSystem.withValue(esys) { + val in = new ObjectInputStream(new ByteArrayInputStream(bytes)) val readA = in.readObject a.isInstanceOf[LocalActorRef] must be === true readA.isInstanceOf[LocalActorRef] must be === true (readA eq a) must be === true } + + val ser = new JavaSerializer(esys) + val readA = ser.fromBinary(bytes, None) + readA.isInstanceOf[LocalActorRef] must be === true + (readA eq a) must be === true } "throw an exception on deserialize if no system in scope" in { @@ -297,7 +304,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { out.flush out.close - Serialization.currentSystem.withValue(sysImpl) { + JavaSerializer.currentSystem.withValue(sysImpl) { val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) in.readObject must be === new EmptyLocalActorRef(sysImpl.provider, system.actorFor("/").path / "non-existing", system.eventStream) } 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 b83fe78338..ee15e380c2 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -8,7 +8,6 @@ import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import akka.util.Duration import akka.util.Timeout import akka.util.duration._ -import akka.serialization.Serialization import java.util.concurrent.atomic.AtomicReference import annotation.tailrec import akka.testkit.{ EventFilter, filterEvents, AkkaSpec } @@ -19,6 +18,7 @@ import akka.japi.{ Creator, Option ⇒ JOption } import akka.testkit.DefaultTimeout import akka.dispatch.{ Await, Dispatchers, Future, Promise } import akka.pattern.ask +import akka.serialization.JavaSerializer object TypedActorSpec { @@ -367,7 +367,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) "be able to serialize and deserialize invocations" in { import java.io._ - Serialization.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) { + JavaSerializer.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) { val m = TypedActor.MethodCall(classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]()) val baos = new ByteArrayOutputStream(8192 * 4) val out = new ObjectOutputStream(baos) @@ -386,7 +386,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) "be able to serialize and deserialize invocations' parameters" in { import java.io._ val someFoo: Foo = new Bar - Serialization.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) { + JavaSerializer.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) { val m = TypedActor.MethodCall(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-tests/src/test/scala/akka/serialization/SerializeSpec.scala b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala index df2170905e..4ddf774064 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -78,7 +78,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.serializationConf) { case Left(exception) ⇒ fail(exception) case Right(bytes) ⇒ bytes } - deserialize(b.asInstanceOf[Array[Byte]], classOf[Address], None) match { + deserialize(b.asInstanceOf[Array[Byte]], classOf[Address]) match { case Left(exception) ⇒ fail(exception) case Right(add) ⇒ assert(add === addr) } @@ -90,7 +90,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.serializationConf) { case Left(exception) ⇒ fail(exception) case Right(bytes) ⇒ bytes } - deserialize(b.asInstanceOf[Array[Byte]], classOf[Person], None) match { + deserialize(b.asInstanceOf[Array[Byte]], classOf[Person]) match { case Left(exception) ⇒ fail(exception) case Right(p) ⇒ assert(p === person) } @@ -103,7 +103,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.serializationConf) { case Left(exception) ⇒ fail(exception) case Right(bytes) ⇒ bytes } - deserialize(b.asInstanceOf[Array[Byte]], classOf[Record], None) match { + deserialize(b.asInstanceOf[Array[Byte]], classOf[Record]) match { case Left(exception) ⇒ fail(exception) case Right(p) ⇒ assert(p === r) } @@ -135,7 +135,7 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.serializationConf) { out.close() val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray)) - Serialization.currentSystem.withValue(a.asInstanceOf[ActorSystemImpl]) { + JavaSerializer.currentSystem.withValue(a.asInstanceOf[ActorSystemImpl]) { val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef] (deadLetters eq a.deadLetters) must be(true) } @@ -248,8 +248,5 @@ class TestSerializer extends Serializer { Array.empty[Byte] } - def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]] = None, - classLoader: Option[ClassLoader] = None): AnyRef = { - null - } + def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index f86f64fa99..a854258453 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -220,7 +220,7 @@ private[akka] class ActorCell( val ser = SerializationExtension(system) ser.serialize(props.creator) match { case Left(t) ⇒ throw t - case Right(bytes) ⇒ ser.deserialize(bytes, props.creator.getClass, None) match { + case Right(bytes) ⇒ ser.deserialize(bytes, props.creator.getClass) match { case Left(t) ⇒ throw t case _ ⇒ //All good } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 38e8ab679f..90154a829b 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -334,7 +334,7 @@ private[akka] class LocalActorRef private[akka] ( * Memento pattern for serializing ActorRefs transparently */ case class SerializedActorRef private (path: String) { - import akka.serialization.Serialization.currentSystem + import akka.serialization.JavaSerializer.currentSystem @throws(classOf[java.io.ObjectStreamException]) def readResolve(): AnyRef = currentSystem.value match { @@ -399,7 +399,7 @@ case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) private[akka] object DeadLetterActorRef { class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance? @throws(classOf[java.io.ObjectStreamException]) - private def readResolve(): AnyRef = Serialization.currentSystem.value.deadLetters + private def readResolve(): AnyRef = akka.serialization.JavaSerializer.currentSystem.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 688c036380..98408dca65 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -318,12 +318,12 @@ class LocalActorRefProvider( settings: ActorSystem.Settings, eventStream: EventStream, scheduler: Scheduler, - classloader: ClassLoader) = + propertyMaster: PropertyMaster) = this(_systemName, settings, eventStream, scheduler, - new Deployer(settings, classloader)) + new Deployer(settings, propertyMaster)) val rootPath: ActorPath = RootActorPath(Address("akka", _systemName)) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index b20cdf5f1c..05334f07d6 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -324,13 +324,13 @@ abstract class ExtendedActorSystem extends ActorSystem { def deathWatch: DeathWatch /** - * ClassLoader which is used for reflective accesses internally. This is set - * to the context class loader, if one is set, or the class loader which + * ClassLoader wrapper which is used for reflective accesses internally. This is set + * to use the context class loader, if one is set, or the class loader which * loaded the ActorSystem implementation. The context class loader is also * set on all threads created by the ActorSystem, if one was set during * creation. */ - def internalClassLoader: ClassLoader + def propertyMaster: PropertyMaster } class ActorSystemImpl(val name: String, applicationConfig: Config) extends ExtendedActorSystem { @@ -356,6 +356,34 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten final val threadFactory: MonitorableThreadFactory = MonitorableThreadFactory(name, settings.Daemonicity, Option(Thread.currentThread.getContextClassLoader), uncaughtExceptionHandler) + /** + * This is an extension point: by overriding this method, subclasses can + * control all reflection activities of an actor system. + */ + protected def createPropertyMaster(): PropertyMaster = new DefaultPropertyMaster(findClassLoader) + + protected def findClassLoader: ClassLoader = + Option(Thread.currentThread.getContextClassLoader) orElse + (Reflect.getCallerClass map findCaller) getOrElse + getClass.getClassLoader + + private def findCaller(get: Int ⇒ Class[_]): ClassLoader = { + val frames = Iterator.from(2).map(get) + frames dropWhile { c ⇒ + c != null && + (c.getName.startsWith("akka.actor.ActorSystem") || + c.getName.startsWith("scala.Option") || + c.getName.startsWith("scala.collection.Iterator") || + c.getName.startsWith("akka.util.Reflect")) + } next () match { + case null ⇒ getClass.getClassLoader + case c ⇒ c.getClassLoader + } + } + + private val _pm: PropertyMaster = createPropertyMaster() + def propertyMaster: PropertyMaster = _pm + def logConfiguration(): Unit = log.info(settings.toString) protected def systemImpl = this @@ -406,17 +434,15 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten val scheduler: Scheduler = createScheduler() - val internalClassLoader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader - val provider: ActorRefProvider = { val arguments = Seq( classOf[String] -> name, classOf[Settings] -> settings, classOf[EventStream] -> eventStream, classOf[Scheduler] -> scheduler, - classOf[ClassLoader] -> internalClassLoader) + classOf[PropertyMaster] -> propertyMaster) - ReflectiveAccess.createInstance[ActorRefProvider](ProviderClass, arguments, internalClassLoader) match { + propertyMaster.getInstanceFor[ActorRefProvider](ProviderClass, arguments) match { case Left(e) ⇒ throw e case Right(p) ⇒ p } @@ -438,7 +464,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten def locker: Locker = provider.locker val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites( - threadFactory, eventStream, deadLetterMailbox, scheduler, internalClassLoader)) + threadFactory, eventStream, deadLetterMailbox, scheduler, propertyMaster)) val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher @@ -557,8 +583,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten private def loadExtensions() { import scala.collection.JavaConversions._ settings.config.getStringList("akka.extensions") foreach { fqcn ⇒ - import ReflectiveAccess.{ getObjectFor, createInstance, noParams, noArgs } - getObjectFor[AnyRef](fqcn, internalClassLoader).fold(_ ⇒ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match { + propertyMaster.getObjectFor[AnyRef](fqcn).fold(_ ⇒ propertyMaster.getInstanceFor[AnyRef](fqcn, Seq()), Right(_)) match { case Right(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup()); case Right(p: ExtensionId[_]) ⇒ registerExtension(p); case Right(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index d561c74413..a26ce419d5 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -8,7 +8,6 @@ import akka.util.Duration import com.typesafe.config._ import akka.routing._ import java.util.concurrent.{ TimeUnit, ConcurrentHashMap } -import akka.util.ReflectiveAccess /** * This class represents deployment configuration for a given actor path. It is @@ -83,7 +82,7 @@ case object NoScopeGiven extends Scope { * * @author Jonas Bonér */ -class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader) { +class Deployer(val settings: ActorSystem.Settings, val propertyMaster: PropertyMaster) { import scala.collection.JavaConverters._ @@ -125,7 +124,7 @@ class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader) case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer) case fqn ⇒ val args = Seq(classOf[Config] -> deployment) - ReflectiveAccess.createInstance[RouterConfig](fqn, args, classloader) match { + propertyMaster.getInstanceFor[RouterConfig](fqn, args) match { case Right(router) ⇒ router case Left(exception) ⇒ throw new IllegalArgumentException( diff --git a/akka-actor/src/main/scala/akka/actor/Extension.scala b/akka-actor/src/main/scala/akka/actor/Extension.scala index 547257e850..0b0088c78c 100644 --- a/akka-actor/src/main/scala/akka/actor/Extension.scala +++ b/akka-actor/src/main/scala/akka/actor/Extension.scala @@ -3,8 +3,6 @@ */ package akka.actor -import akka.util.ReflectiveAccess - /** * The basic ActorSystem covers all that is needed for locally running actors, * using futures and so on. In addition, more features can hook into it and @@ -73,12 +71,12 @@ trait ExtensionIdProvider { /** * This is a one-stop-shop if all you want is an extension which is - * constructed with the ActorSystemImpl as its only constructor argument: + * constructed with the ExtendedActorSystem as its only constructor argument: * * {{{ * object MyExt extends ExtensionKey[Ext] * - * class Ext(system: ActorSystemImpl) extends MyExt { + * class Ext(system: ExtendedActorSystem) extends MyExt { * ... * } * }}} @@ -89,7 +87,7 @@ trait ExtensionIdProvider { * public class MyExt extends Extension { * static final ExtensionKey key = new ExtensionKey(MyExt.class); * - * public MyExt(ActorSystemImpl system) { + * public MyExt(ExtendedActorSystem system) { * ... * } * }}} @@ -99,7 +97,7 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassManifest[T]) extend override def lookup(): ExtensionId[T] = this def createExtension(system: ExtendedActorSystem): T = - ReflectiveAccess.createInstance[T](m.erasure, Array[Class[_]](classOf[ActorSystemImpl]), Array[AnyRef](system)) match { + PropertyMaster.getInstanceFor[T](m.erasure, Seq(classOf[ExtendedActorSystem] -> system)) match { case Left(ex) ⇒ throw ex case Right(r) ⇒ r } diff --git a/akka-actor/src/main/scala/akka/actor/PropertyMaster.scala b/akka-actor/src/main/scala/akka/actor/PropertyMaster.scala new file mode 100644 index 0000000000..3f3a493b8f --- /dev/null +++ b/akka-actor/src/main/scala/akka/actor/PropertyMaster.scala @@ -0,0 +1,102 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.actor + +import akka.util.NonFatal +import java.lang.reflect.InvocationTargetException + +/** + * The property master is responsible for acquiring all props needed for a + * performance; in Akka this is the class which is used for reflectively + * loading all configurable parts of an actor system. + */ +trait PropertyMaster { + + def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] + + def getInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] + + def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T] + + /** + * This is needed e.g. by the JavaSerializer to build the ObjectInputStream. + */ + def classLoader: ClassLoader + +} + +object PropertyMaster { + + def getInstanceFor[T: ClassManifest](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = { + val types = args.map(_._1).toArray + val values = args.map(_._2).toArray + withErrorHandling { + val constructor = clazz.getDeclaredConstructor(types: _*) + constructor.setAccessible(true) + val obj = constructor.newInstance(values: _*).asInstanceOf[T] + val t = classManifest[T].erasure + if (t.isInstance(obj)) Right(obj) else Left(new ClassCastException(clazz + " is not a subtype of " + t)) + } + } + + /** + * Caught exception is returned as Left(exception). + * Unwraps `InvocationTargetException` if its getTargetException is an `Exception`. + * Other `Throwable`, such as `Error` is thrown. + */ + @inline + final def withErrorHandling[T](body: ⇒ Either[Throwable, T]): Either[Throwable, T] = + try body catch { + case e: InvocationTargetException ⇒ + e.getTargetException match { + case NonFatal(t) ⇒ Left(t) + case t ⇒ throw t + } + case NonFatal(e) ⇒ Left(e) + } + +} + +class DefaultPropertyMaster(val classLoader: ClassLoader) extends PropertyMaster { + + import PropertyMaster.withErrorHandling + + override def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] = + try { + val c = classLoader.loadClass(fqcn).asInstanceOf[Class[_ <: T]] + val t = classManifest[T].erasure + if (t.isAssignableFrom(c)) Right(c) else Left(new ClassCastException(t + " is not assignable from " + c)) + } catch { + case NonFatal(e) ⇒ Left(e) + } + + override def getInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = + getClassFor(fqcn).fold(Left(_), { c ⇒ + val types = args.map(_._1).toArray + val values = args.map(_._2).toArray + withErrorHandling { + val constructor = c.getDeclaredConstructor(types: _*) + constructor.setAccessible(true) + val obj = constructor.newInstance(values: _*) + val t = classManifest[T].erasure + if (t.isInstance(obj)) Right(obj) else Left(new ClassCastException(fqcn + " is not a subtype of " + t)) + } + }) + + override def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T] = { + getClassFor(fqcn).fold(Left(_), { c ⇒ + withErrorHandling { + val module = c.getDeclaredField("MODULE$") + module.setAccessible(true) + val t = classManifest[T].erasure + module.get(null) match { + case null ⇒ Left(new NullPointerException) + case x if !t.isInstance(x) ⇒ Left(new ClassCastException(fqcn + " is not a subtype of " + t)) + case x ⇒ Right(x.asInstanceOf[T]) + } + } + }) + } + +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 41eba86bbd..cc1e02bb3e 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -129,7 +129,8 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi val serializedParameters = Array.ofDim[(Int, Class[_], Array[Byte])](ps.length) for (i ← 0 until ps.length) { val p = ps(i) - val s = SerializationExtension(Serialization.currentSystem.value).findSerializerFor(p) + val system = akka.serialization.JavaSerializer.currentSystem.value + val s = SerializationExtension(system).findSerializerFor(p) val m = if (s.includeManifest) p.getClass else null serializedParameters(i) = (s.identifier, m, s toBinary parameters(i)) //Mutable for the sake of sanity } @@ -146,7 +147,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi //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 system = akka.serialization.Serialization.currentSystem.value + val system = akka.serialization.JavaSerializer.currentSystem.value if (system eq null) throw new IllegalStateException( "Trying to deserialize a SerializedMethodCall without an ActorSystem in scope." + " Use akka.serialization.Serialization.currentSystem.withValue(system) { ... }") @@ -158,7 +159,8 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity for (i ← 0 until a.length) { val (sId, manifest, bytes) = a(i) - deserializedParameters(i) = serialization.serializerByIdentity(sId).fromBinary(bytes, Option(manifest)) + deserializedParameters(i) = + serialization.serializerByIdentity(sId).fromBinary(bytes, Option(manifest)) } deserializedParameters diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 8a5bcfa385..db7774eeb7 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -12,7 +12,6 @@ import akka.actor.ActorSystem import scala.annotation.tailrec import akka.event.EventStream import com.typesafe.config.Config -import akka.util.ReflectiveAccess import akka.serialization.SerializationExtension import akka.jsr166y.ForkJoinPool import akka.util.NonFatal @@ -26,7 +25,7 @@ final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorS val ser = SerializationExtension(system) ser.serialize(msg) match { //Verify serializability case Left(t) ⇒ throw t - case Right(bytes) ⇒ ser.deserialize(bytes, msg.getClass, None) match { //Verify deserializability + case Right(bytes) ⇒ ser.deserialize(bytes, msg.getClass) match { //Verify deserializability case Left(t) ⇒ throw t case _ ⇒ //All good } @@ -369,7 +368,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit } case fqcn ⇒ val args = Seq(classOf[Config] -> config) - ReflectiveAccess.createInstance[MailboxType](fqcn, args, prerequisites.classloader) match { + prerequisites.propertyMaster.getInstanceFor[MailboxType](fqcn, args) match { case Right(instance) ⇒ instance case Left(exception) ⇒ throw new IllegalArgumentException( @@ -385,8 +384,10 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit case null | "" | "fork-join-executor" ⇒ new ForkJoinExecutorConfigurator(config.getConfig("fork-join-executor"), prerequisites) case "thread-pool-executor" ⇒ new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites) case fqcn ⇒ - val constructorSignature = Array[Class[_]](classOf[Config], classOf[DispatcherPrerequisites]) - ReflectiveAccess.createInstance[ExecutorServiceConfigurator](fqcn, constructorSignature, Array[AnyRef](config, prerequisites), prerequisites.classloader) match { + val args = Seq( + classOf[Config] -> config, + classOf[DispatcherPrerequisites] -> prerequisites) + prerequisites.propertyMaster.getInstanceFor[ExecutorServiceConfigurator](fqcn, args) match { case Right(instance) ⇒ instance case Left(exception) ⇒ throw new IllegalArgumentException( ("""Cannot instantiate ExecutorServiceConfigurator ("executor = [%s]"), defined in [%s], diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index d71604fd1a..8529c8f733 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -4,22 +4,24 @@ package akka.dispatch -import akka.actor.newUuid -import akka.util.{ Duration, ReflectiveAccess } -import akka.actor.ActorSystem -import akka.event.EventStream -import akka.actor.Scheduler -import com.typesafe.config.Config -import com.typesafe.config.ConfigFactory +import java.util.concurrent.{ ConcurrentHashMap, TimeUnit, ThreadFactory } + +import scala.collection.JavaConverters.mapAsJavaMapConverter + +import com.typesafe.config.{ ConfigFactory, Config } + +import Dispatchers.DefaultDispatcherId +import akka.actor.{ Scheduler, PropertyMaster, ActorSystem } import akka.event.Logging.Warning -import java.util.concurrent.{ ThreadFactory, TimeUnit, ConcurrentHashMap } +import akka.event.EventStream +import akka.util.Duration trait DispatcherPrerequisites { def threadFactory: ThreadFactory def eventStream: EventStream def deadLetterMailbox: Mailbox def scheduler: Scheduler - def classloader: ClassLoader + def propertyMaster: PropertyMaster } case class DefaultDispatcherPrerequisites( @@ -27,7 +29,7 @@ case class DefaultDispatcherPrerequisites( val eventStream: EventStream, val deadLetterMailbox: Mailbox, val scheduler: Scheduler, - val classloader: ClassLoader) extends DispatcherPrerequisites + val propertyMaster: PropertyMaster) extends DispatcherPrerequisites object Dispatchers { /** @@ -137,7 +139,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites) case fqn ⇒ val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites) - ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, args, prerequisites.classloader) match { + prerequisites.propertyMaster.getInstanceFor[MessageDispatcherConfigurator](fqn, args) match { case Right(configurator) ⇒ configurator case Left(exception) ⇒ throw new IllegalArgumentException( diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index a6622b7875..c8dcedc3d5 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -6,7 +6,6 @@ package akka.event import akka.actor._ import akka.AkkaException import akka.actor.ActorSystem.Settings -import akka.util.ReflectiveAccess import akka.config.ConfigurationException import akka.util.ReentrantGuard import akka.util.duration._ @@ -101,7 +100,7 @@ trait LoggingBus extends ActorEventBus { if loggerName != StandardOutLoggerName } yield { try { - ReflectiveAccess.getClassFor[Actor](loggerName, system.internalClassLoader) match { + system.propertyMaster.getClassFor[Actor](loggerName) match { case Right(actorClass) ⇒ addLogger(system, actorClass, level, logName) case Left(exception) ⇒ throw exception } @@ -350,7 +349,7 @@ object Logging { object Extension extends ExtensionKey[LogExt] - class LogExt(system: ActorSystemImpl) extends Extension { + class LogExt(system: ExtendedActorSystem) extends Extension { private val loggerId = new AtomicInteger def id() = loggerId.incrementAndGet() } diff --git a/akka-actor/src/main/scala/akka/serialization/Format.scala b/akka-actor/src/main/scala/akka/serialization/Format.scala index 41a8eed658..7ce9d8a2e6 100644 --- a/akka-actor/src/main/scala/akka/serialization/Format.scala +++ b/akka-actor/src/main/scala/akka/serialization/Format.scala @@ -61,6 +61,7 @@ trait ToBinary[T <: Actor] { * Type class definition for Actor Serialization. * Client needs to implement Format[] for the respective actor. */ +// FIXME RK: should this go? It’s not used anywhere, looks like cluster residue. trait Format[T <: Actor] extends FromBinary[T] with ToBinary[T] /** @@ -97,7 +98,7 @@ trait StatelessActorFormat[T <: Actor] extends Format[T] with scala.Serializable trait SerializerBasedActorFormat[T <: Actor] extends Format[T] with scala.Serializable { val serializer: Serializer - def fromBinary(bytes: Array[Byte], act: T) = serializer.fromBinary(bytes, Some(act.getClass)).asInstanceOf[T] + def fromBinary(bytes: Array[Byte], act: T): T = serializer.fromBinary(bytes, Some(act.getClass)).asInstanceOf[T] - def toBinary(ac: T) = serializer.toBinary(ac) + def toBinary(ac: T): Array[Byte] = serializer.toBinary(ac) } diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index e89adde8fb..1751f49082 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -5,30 +5,16 @@ package akka.serialization import akka.AkkaException -import akka.util.ReflectiveAccess import scala.util.DynamicVariable import com.typesafe.config.Config import akka.config.ConfigurationException -import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem, Address } +import akka.actor.{ Extension, ExtendedActorSystem, Address } import java.util.concurrent.ConcurrentHashMap import akka.event.Logging case class NoSerializerFoundException(m: String) extends AkkaException(m) object Serialization { - /** - * This holds a reference to the current ActorSystem (the surrounding context) - * during serialization and deserialization. - * - * If you are using Serializers yourself, outside of SerializationExtension, - * you'll need to surround the serialization/deserialization with: - * - * currentSystem.withValue(system) { - * ...code... - * } - */ - val currentSystem = new DynamicVariable[ActorSystem](null) - /** * This holds a reference to the current transport address to be inserted * into local actor refs during serialization. @@ -74,7 +60,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * to either an Array of Bytes or an Exception if one was thrown. */ def serialize(o: AnyRef): Either[Exception, Array[Byte]] = - try { Right(findSerializerFor(o).toBinary(o)) } catch { case e: Exception ⇒ Left(e) } + try Right(findSerializerFor(o).toBinary(o)) + catch { case e: Exception ⇒ Left(e) } /** * Deserializes the given array of bytes using the specified serializer id, @@ -83,26 +70,18 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { */ def deserialize(bytes: Array[Byte], serializerId: Int, - clazz: Option[Class[_]], - classLoader: ClassLoader): Either[Exception, AnyRef] = - try { - currentSystem.withValue(system) { - Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz, Some(classLoader))) - } - } catch { case e: Exception ⇒ Left(e) } + clazz: Option[Class[_]]): Either[Exception, AnyRef] = + try Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz)) + catch { case e: Exception ⇒ Left(e) } /** * Deserializes the given array of bytes using the specified type to look up what Serializer should be used. * You can specify an optional ClassLoader to load the object into. * Returns either the resulting object or an Exception if one was thrown. */ - def deserialize( - bytes: Array[Byte], - clazz: Class[_], - classLoader: Option[ClassLoader]): Either[Exception, AnyRef] = - try { - currentSystem.withValue(system) { Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader)) } - } catch { case e: Exception ⇒ Left(e) } + def deserialize(bytes: Array[Byte], clazz: Class[_]): Either[Exception, AnyRef] = + try Right(serializerFor(clazz).fromBinary(bytes, Some(clazz))) + catch { case e: Exception ⇒ Left(e) } /** * Returns the Serializer configured for the given object, returns the NullSerializer if it's null, @@ -149,8 +128,11 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { /** * Tries to load the specified Serializer by the FQN */ - def serializerOf(serializerFQN: String): Either[Exception, Serializer] = - ReflectiveAccess.createInstance(serializerFQN, ReflectiveAccess.noParams, ReflectiveAccess.noArgs) + def serializerOf(serializerFQN: String): Either[Throwable, Serializer] = { + val pm = system.propertyMaster + pm.getInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)) + .fold(_ ⇒ pm.getInstanceFor[Serializer](serializerFQN, Seq()), Right(_)) + } /** * A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer) diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index ffe7f50de9..0acd6e4f8c 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -6,11 +6,29 @@ package akka.serialization import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream } import akka.util.ClassLoaderObjectInputStream +import akka.actor.PropertyMaster +import akka.actor.ExtendedActorSystem +import scala.util.DynamicVariable /** - * A Serializer represents a bimap between an object and an array of bytes representing that object + * A Serializer represents a bimap between an object and an array of bytes representing that object. + * + * Serializers are loaded using reflection during [[akka.actor.ActorSystem]] + * start-up, where two constructors are tried in order: + * + *
    + *
  • taking exactly one argument of type [[akka.actor.ExtendedActorSystem]]; + * this should be the preferred one because all reflective loading of classes + * during deserialization should use ExtendedActorSystem.propertyMaster (see + * [[akka.actor.PropertyMaster]]), and
  • + *
  • without arguments, which is only an option if the serializer does not + * load classes using reflection.
  • + *
+ * + * Be sure to always use the PropertyManager for loading classes! */ trait Serializer extends scala.Serializable { + /** * Completely unique value to identify this implementation of Serializer, used to optimize network traffic * Values from 0 to 16 is reserved for Akka internal usage @@ -28,42 +46,52 @@ trait Serializer extends scala.Serializable { def includeManifest: Boolean /** - * Deserializes the given Array of Bytes into an AnyRef + * Produces an object from an array of bytes, with an optional type-hint; + * the class should be loaded using ActorSystem.propertyMaster. */ - def fromBinary(bytes: Array[Byte]): AnyRef = fromBinary(bytes, None, None) - - /** - * Deserializes the given Array of Bytes into an AnyRef with an optional type hint - */ - def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = fromBinary(bytes, manifest, None) - - /** - * Produces an object from an array of bytes, with an optional type-hint and a classloader to load the class into - */ - def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]], classLoader: Option[ClassLoader]): AnyRef + def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef } /** - * Java API for creating a Serializer + * Java API for creating a Serializer: make sure to include a constructor which + * takes exactly one argument of type [[akka.actor.ExtendedActorSystem]], because + * that is the preferred constructor which will be invoked when reflectively instantiating + * the JSerializer (also possible with empty constructor). */ abstract class JSerializer extends Serializer { - def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]] = None, classLoader: Option[ClassLoader] = None): AnyRef = - fromBinary(bytes, manifest.orNull, classLoader.orNull) + def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = + fromBinary(bytes, manifest.orNull) /** * This method should be overridden, * manifest and classLoader may be null. */ - def fromBinary(bytes: Array[Byte], manifest: Class[_], classLoader: ClassLoader): AnyRef + def fromBinary(bytes: Array[Byte], manifest: Class[_]): AnyRef } -object JavaSerializer extends JavaSerializer object NullSerializer extends NullSerializer +object JavaSerializer { + + /** + * This holds a reference to the current ActorSystem (the surrounding context) + * during serialization and deserialization. + * + * If you are using Serializers yourself, outside of SerializationExtension, + * you'll need to surround the serialization/deserialization with: + * + * currentSystem.withValue(system) { + * ...code... + * } + */ + val currentSystem = new DynamicVariable[ExtendedActorSystem](null) + +} + /** * This Serializer uses standard Java Serialization */ -class JavaSerializer extends Serializer { +class JavaSerializer(val system: ExtendedActorSystem) extends Serializer { def includeManifest: Boolean = false @@ -77,12 +105,11 @@ class JavaSerializer extends Serializer { bos.toByteArray } - def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]] = None, - classLoader: Option[ClassLoader] = None): AnyRef = { - val in = - if (classLoader.isDefined) new ClassLoaderObjectInputStream(classLoader.get, new ByteArrayInputStream(bytes)) else - new ObjectInputStream(new ByteArrayInputStream(bytes)) - val obj = in.readObject + def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = { + val in = new ClassLoaderObjectInputStream(system.propertyMaster.classLoader, new ByteArrayInputStream(bytes)) + val obj = JavaSerializer.currentSystem.withValue(system) { + in.readObject + } in.close() obj } @@ -96,5 +123,5 @@ class NullSerializer extends Serializer { def includeManifest: Boolean = false def identifier = 0 def toBinary(o: AnyRef) = nullAsBytes - def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]] = None, classLoader: Option[ClassLoader] = None): AnyRef = null + def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null } diff --git a/akka-actor/src/main/scala/akka/util/Reflect.scala b/akka-actor/src/main/scala/akka/util/Reflect.scala new file mode 100644 index 0000000000..d2ec6ddfbd --- /dev/null +++ b/akka-actor/src/main/scala/akka/util/Reflect.scala @@ -0,0 +1,18 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.util + +object Reflect { + + val getCallerClass: Option[Int ⇒ Class[_]] = { + try { + val c = Class.forName("sun.reflect.Reflection"); + val m = c.getMethod("getCallerClass", Array(classOf[Int]): _*) + Some((i: Int) ⇒ m.invoke(null, Array[AnyRef](i.asInstanceOf[Integer]): _*).asInstanceOf[Class[_]]) + } catch { + case NonFatal(e) ⇒ None + } + } + +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala deleted file mode 100644 index 19ad3bc995..0000000000 --- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala +++ /dev/null @@ -1,126 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.util - -import java.lang.reflect.InvocationTargetException - -object ReflectiveAccess { - - val loader = getClass.getClassLoader - val noParams: Array[Class[_]] = Array() - val noArgs: Array[AnyRef] = Array() - - def createInstance[T](clazz: Class[_], - params: Array[Class[_]], - args: Array[AnyRef]): Either[Exception, T] = withErrorHandling { - assert(clazz ne null) - assert(params ne null) - assert(args ne null) - val ctor = clazz.getDeclaredConstructor(params: _*) - ctor.setAccessible(true) - Right(ctor.newInstance(args: _*).asInstanceOf[T]) - } - - def createInstance[T](fqn: String, - params: Array[Class[_]], - args: Array[AnyRef], - classloader: ClassLoader = loader): Either[Exception, T] = withErrorHandling { - assert(params ne null) - assert(args ne null) - getClassFor(fqn, classloader) match { - case Right(value) ⇒ - val ctor = value.getDeclaredConstructor(params: _*) - ctor.setAccessible(true) - Right(ctor.newInstance(args: _*).asInstanceOf[T]) - case Left(exception) ⇒ Left(exception) //We could just cast this to Either[Exception, T] but it's ugly - } - } - - def createInstance[T](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Exception, T] = - createInstance(clazz, args.map(_._1).toArray, args.map(_._2).toArray) - - def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)], classloader: ClassLoader): Either[Exception, T] = - createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, classloader) - - def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Exception, T] = - createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, loader) - - //Obtains a reference to fqn.MODULE$ - def getObjectFor[T](fqn: String, classloader: ClassLoader = loader): Either[Exception, T] = try { - getClassFor(fqn, classloader) match { - case Right(value) ⇒ - val instance = value.getDeclaredField("MODULE$") - instance.setAccessible(true) - val obj = instance.get(null) - if (obj eq null) Left(new NullPointerException) else Right(obj.asInstanceOf[T]) - case Left(exception) ⇒ Left(exception) //We could just cast this to Either[Exception, T] but it's ugly - } - } catch { - case e: Exception ⇒ - Left(e) - } - - def getClassFor[T](fqn: String, classloader: ClassLoader = loader): Either[Exception, Class[T]] = try { - assert(fqn ne null) - - // First, use the specified CL - val first = try { - Right(classloader.loadClass(fqn).asInstanceOf[Class[T]]) - } catch { - case c: ClassNotFoundException ⇒ Left(c) - } - - if (first.isRight) first - else { - // Second option is to use the ContextClassLoader - val second = try { - Right(Thread.currentThread.getContextClassLoader.loadClass(fqn).asInstanceOf[Class[T]]) - } catch { - case c: ClassNotFoundException ⇒ Left(c) - } - - if (second.isRight) second - else { - val third = try { - if (classloader ne loader) Right(loader.loadClass(fqn).asInstanceOf[Class[T]]) else Left(null) //Horrid - } catch { - case c: ClassNotFoundException ⇒ Left(c) - } - - if (third.isRight) third - else { - try { - Right(Class.forName(fqn).asInstanceOf[Class[T]]) // Last option is Class.forName - } catch { - case c: ClassNotFoundException ⇒ Left(c) - } - } - } - } - } catch { - case e: Exception ⇒ Left(e) - } - - /** - * Caught exception is returned as Left(exception). - * Unwraps `InvocationTargetException` if its getTargetException is an `Exception`. - * Other `Throwable`, such as `Error` is thrown. - */ - @inline - private final def withErrorHandling[T](body: ⇒ Either[Exception, T]): Either[Exception, T] = { - try { - body - } catch { - case e: InvocationTargetException ⇒ e.getTargetException match { - case t: Exception ⇒ Left(t) - case t ⇒ throw t - } - case e: Exception ⇒ - Left(e) - } - } - -} - diff --git a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java index b68f8f2e79..22a9f571d2 100644 --- a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java +++ b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java @@ -3,6 +3,7 @@ */ package akka.docs.serialization; +import akka.japi.Option; import akka.serialization.JSerializer; import akka.serialization.Serialization; import akka.serialization.SerializationExtension; @@ -13,6 +14,7 @@ import static org.junit.Assert.*; import akka.serialization.*; import akka.actor.ActorSystem; +import akka.actor.PropertyMaster; import com.typesafe.config.*; //#imports @@ -45,8 +47,7 @@ public class SerializationDocTestBase { // using the type hint (if any, see "includeManifest" above) // into the optionally provided classLoader. @Override public Object fromBinary(byte[] bytes, - Class clazz, - ClassLoader classLoader) { + Class clazz) { // Put your code that deserializes here //#... return null; @@ -128,7 +129,7 @@ public class SerializationDocTestBase { // Turn it back into an object, // the nulls are for the class manifest and for the classloader - String back = (String)serializer.fromBinary(bytes); + String back = (String)serializer.fromBinary(bytes, Option.>none().asScala()); // Voilá! assertEquals(original, back); diff --git a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala b/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala index 7f1553f75c..ae87e0d5ab 100644 --- a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala @@ -6,7 +6,7 @@ package akka.docs.serialization import org.scalatest.matchers.MustMatchers import akka.testkit._ //#imports -import akka.actor.ActorSystem +import akka.actor.{ ActorSystem, PropertyMaster } import akka.serialization._ import com.typesafe.config.ConfigFactory @@ -35,8 +35,7 @@ class MyOwnSerializer extends Serializer { // using the type hint (if any, see "includeManifest" above) // into the optionally provided classLoader. def fromBinary(bytes: Array[Byte], - clazz: Option[Class[_]], - classLoader: Option[ClassLoader] = None): AnyRef = { + clazz: Option[Class[_]]): AnyRef = { // Put your code that deserializes here //#... null @@ -147,9 +146,7 @@ class SerializationDocSpec extends AkkaSpec { val bytes = serializer.toBinary(original) // Turn it back into an object - val back = serializer.fromBinary(bytes, - manifest = None, - classLoader = None) + val back = serializer.fromBinary(bytes, manifest = None) // Voilá! back must equal(original) diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala index cd2ca2d0f0..489d97d176 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala @@ -22,7 +22,7 @@ class BeanstalkBasedMailboxType(config: Config) extends MailboxType { /** * @author Jonas Bonér */ -class BeanstalkBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization { +class BeanstalkBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { private val settings = BeanstalkBasedMailboxExtension(owner.system) private val messageSubmitDelaySeconds = settings.MessageSubmitDelay.toSeconds.toInt diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala index 8da17a4cc9..ccdbdc4145 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala @@ -17,7 +17,7 @@ class FileBasedMailboxType(config: Config) extends MailboxType { override def create(owner: ActorContext) = new FileBasedMailbox(owner) } -class FileBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization { +class FileBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { val log = Logging(system, "FileBasedMailbox") diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala index 5e319dafac..69f7fb50c1 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -12,7 +12,7 @@ private[akka] object DurableExecutableMailboxConfig { val Name = "[\\.\\/\\$\\s]".r } -abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue { +abstract class DurableMailbox(val owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue { import DurableExecutableMailboxConfig._ def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem] @@ -22,15 +22,13 @@ abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner) } -trait DurableMessageSerialization { - - def owner: ActorContext +trait DurableMessageSerialization { this: DurableMailbox ⇒ def serialize(durableMessage: Envelope): Array[Byte] = { def serializeActorRef(ref: ActorRef): ActorRefProtocol = ActorRefProtocol.newBuilder.setPath(ref.path.toString).build - val message = MessageSerializer.serialize(owner.system, durableMessage.message.asInstanceOf[AnyRef]) + val message = MessageSerializer.serialize(system, durableMessage.message.asInstanceOf[AnyRef]) val builder = RemoteMessageProtocol.newBuilder .setMessage(message) .setRecipient(serializeActorRef(owner.self)) @@ -41,13 +39,13 @@ trait DurableMessageSerialization { def deserialize(bytes: Array[Byte]): Envelope = { - def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = owner.system.actorFor(refProtocol.getPath) + def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = system.actorFor(refProtocol.getPath) val durableMessage = RemoteMessageProtocol.parseFrom(bytes) - val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage, getClass.getClassLoader) + val message = MessageSerializer.deserialize(system, durableMessage.getMessage) val sender = deserializeActorRef(durableMessage.getSender) - new Envelope(message, sender)(owner.system) + new Envelope(message, sender)(system) } } diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala index cc36286e36..5aa314eb55 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala @@ -65,7 +65,7 @@ class BSONSerializableMailbox(system: ExtendedActorSystem) extends SerializableB val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument] system.log.debug("Deserializing a durable message from MongoDB: {}", doc) val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData) - val msg = MessageSerializer.deserialize(system, msgData, system.internalClassLoader) + val msg = MessageSerializer.deserialize(system, msgData) val ownerPath = doc.as[String]("ownerPath") val senderPath = doc.as[String]("senderPath") val sender = system.actorFor(senderPath) diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala index fed643c7d1..d17a1221a8 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala @@ -32,7 +32,7 @@ class MongoBasedMailboxType(config: Config) extends MailboxType { * * @author Brendan W. McAdams */ -class MongoBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) { +class MongoBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) { // this implicit object provides the context for reading/writing things as MongoDurableMessage implicit val mailboxBSONSer = new BSONSerializableMailbox(system) implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate! diff --git a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala index 5d4e09e65e..b6cf3febc6 100644 --- a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala @@ -19,7 +19,7 @@ class RedisBasedMailboxType(config: Config) extends MailboxType { override def create(owner: ActorContext) = new RedisBasedMailbox(owner) } -class RedisBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization { +class RedisBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { private val settings = RedisBasedMailboxExtension(owner.system) diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala index 81e4c378eb..90fd381af1 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala @@ -20,7 +20,7 @@ class ZooKeeperBasedMailboxType(config: Config) extends MailboxType { override def create(owner: ActorContext) = new ZooKeeperBasedMailbox(owner) } -class ZooKeeperBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization { +class ZooKeeperBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { private val settings = ZooKeeperBasedMailboxExtension(owner.system) val queueNode = "/queues" diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 5301f2bdd0..ddb97d02c4 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -6,32 +6,25 @@ package akka.remote import akka.remote.RemoteProtocol._ import com.google.protobuf.ByteString -import akka.actor.ActorSystem +import akka.actor.ExtendedActorSystem import akka.serialization.SerializationExtension -import akka.util.ReflectiveAccess object MessageSerializer { - def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: ClassLoader): AnyRef = { - val clazz = if (messageProtocol.hasMessageManifest) { - Option(ReflectiveAccess.getClassFor[AnyRef]( - messageProtocol.getMessageManifest.toStringUtf8, - classLoader) match { - case Left(e) ⇒ throw e - case Right(r) ⇒ r - }) - } else None - SerializationExtension(system).deserialize( - messageProtocol.getMessage.toByteArray, - messageProtocol.getSerializerId, - clazz, - classLoader) match { + def deserialize(system: ExtendedActorSystem, messageProtocol: MessageProtocol): AnyRef = { + val clazz = + if (messageProtocol.hasMessageManifest) { + system.propertyMaster.getClassFor[AnyRef](messageProtocol.getMessageManifest.toStringUtf8) + .fold(throw _, Some(_)) + } else None + SerializationExtension(system) + .deserialize(messageProtocol.getMessage.toByteArray, messageProtocol.getSerializerId, clazz) match { case Left(e) ⇒ throw e case Right(r) ⇒ r } } - def serialize(system: ActorSystem, message: AnyRef): MessageProtocol = { + def serialize(system: ExtendedActorSystem, message: AnyRef): MessageProtocol = { val s = SerializationExtension(system) val serializer = s.findSerializerFor(message) val builder = MessageProtocol.newBuilder diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 872be5aa41..e62975ea9f 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -12,7 +12,6 @@ import akka.event.EventStream import akka.config.ConfigurationException import java.util.concurrent.{ TimeoutException } import com.typesafe.config.Config -import akka.util.ReflectiveAccess import akka.serialization.Serialization import akka.serialization.SerializationExtension @@ -28,11 +27,11 @@ class RemoteActorRefProvider( val settings: ActorSystem.Settings, val eventStream: EventStream, val scheduler: Scheduler, - val classloader: ClassLoader) extends ActorRefProvider { + val propertyMaster: PropertyMaster) extends ActorRefProvider { val remoteSettings = new RemoteSettings(settings.config, systemName) - val deployer = new RemoteDeployer(settings, classloader) + val deployer = new RemoteDeployer(settings, propertyMaster) private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer) @@ -84,7 +83,7 @@ class RemoteActorRefProvider( classOf[ActorSystemImpl] -> system, classOf[RemoteActorRefProvider] -> this) - ReflectiveAccess.createInstance[RemoteTransport](fqn, args, system.internalClassLoader) match { + system.propertyMaster.getInstanceFor[RemoteTransport](fqn, args) match { case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem) case Right(remote) ⇒ remote } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala index 9686d295ad..ec6ab165a3 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala @@ -12,7 +12,7 @@ case class RemoteScope(node: Address) extends Scope { def withFallback(other: Scope): Scope = this } -class RemoteDeployer(_settings: ActorSystem.Settings, _classloader: ClassLoader) extends Deployer(_settings, _classloader) { +class RemoteDeployer(_settings: ActorSystem.Settings, _pm: PropertyMaster) extends Deployer(_settings, _pm) { override protected def parseConfig(path: String, config: Config): Option[Deploy] = { import scala.collection.JavaConverters._ diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index b40992d12a..256451bc0a 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -219,7 +219,7 @@ class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl) { lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver) - lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, getClass.getClassLoader) + lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage) override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender } diff --git a/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala b/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala index f733e8188f..c970a4b733 100644 --- a/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala +++ b/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala @@ -5,6 +5,7 @@ package akka.serialization import com.google.protobuf.Message +import akka.actor.PropertyMaster /** * This Serializer serializes `com.google.protobuf.Message`s @@ -19,7 +20,7 @@ class ProtobufSerializer extends Serializer { case _ ⇒ throw new IllegalArgumentException("Can't serialize a non-protobuf message using protobuf [" + obj + "]") } - def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]], classLoader: Option[ClassLoader] = None): AnyRef = + def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = clazz match { case None ⇒ throw new IllegalArgumentException("Need a protobuf message class to be able to serialize bytes using protobuf") case Some(c) ⇒ c.getDeclaredMethod("parseFrom", ARRAY_OF_BYTE_ARRAY: _*).invoke(null, bytes).asInstanceOf[Message] diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index d42cfcf165..21b1221b45 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -5,7 +5,7 @@ package akka.testkit import akka.actor._ -import akka.util.{ ReflectiveAccess, Duration } +import akka.util.Duration import java.util.concurrent.atomic.AtomicLong import scala.collection.immutable.Stack import akka.dispatch._ @@ -121,8 +121,7 @@ object TestActorRef { 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], system: ActorSystem): TestActorRef[T] = apply[T](Props({ - import ReflectiveAccess.{ createInstance, noParams, noArgs } - createInstance[T](m.erasure, noParams, noArgs) match { + PropertyMaster.getInstanceFor[T](m.erasure, Seq()) match { case Right(value) ⇒ value case Left(exception) ⇒ throw new ActorInitializationException(null, "Could not instantiate Actor" + From ca3deb4007aacb3ff0d2c027d91a7c102fc623bf Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 9 Feb 2012 16:59:16 +0100 Subject: [PATCH 02/39] now that was a nice journey (related to #1804) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - first, fix quite some data races in RoutedActorRef wrt. the contained ActorCell’s childrenRef field (which is not even @volatile) - then notice that there still are double-deregistrations happening in the dispatcher - coming finally to the conclusion that the Mailbox should not really process all system messages in processAllSystemMessages(): we should really really stop after having closed the mailbox ;-) - added simple test case which stops self twice to keep this fixed --- .../akka/actor/dispatch/ActorModelSpec.scala | 25 +++- .../main/scala/akka/dispatch/Mailbox.scala | 2 +- .../src/main/scala/akka/routing/Routing.scala | 124 +++++++++++++----- 3 files changed, 110 insertions(+), 41 deletions(-) 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 8c949f8776..46bf609c7a 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 @@ -50,6 +50,8 @@ object ActorModelSpec { case object Restart extends ActorModelMessage + case object DoubleStop extends ActorModelMessage + case class ThrowException(e: Throwable) extends ActorModelMessage val Ping = "Ping" @@ -86,6 +88,7 @@ object ActorModelSpec { case Restart ⇒ ack; busy.switchOff(); throw new Exception("Restart requested") case Interrupt ⇒ ack; sender ! Status.Failure(new ActorInterruptedException(new InterruptedException("Ping!"))); busy.switchOff(); throw new InterruptedException("Ping!") case ThrowException(e: Throwable) ⇒ ack; busy.switchOff(); throw e + case DoubleStop ⇒ ack; context.stop(self); context.stop(self); busy.switchOff } } @@ -190,13 +193,13 @@ object ActorModelSpec { } def assertRef(actorRef: ActorRef, dispatcher: MessageDispatcher = null)( - suspensions: Long = statsFor(actorRef).suspensions.get(), - resumes: Long = statsFor(actorRef).resumes.get(), - registers: Long = statsFor(actorRef).registers.get(), - 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 system: ActorSystem) { + suspensions: Long = statsFor(actorRef, dispatcher).suspensions.get(), + resumes: Long = statsFor(actorRef, dispatcher).resumes.get(), + registers: Long = statsFor(actorRef, dispatcher).registers.get(), + unregisters: Long = statsFor(actorRef, dispatcher).unregisters.get(), + msgsReceived: Long = statsFor(actorRef, dispatcher).msgsReceived.get(), + msgsProcessed: Long = statsFor(actorRef, dispatcher).msgsProcessed.get(), + restarts: Long = statsFor(actorRef, dispatcher).restarts.get())(implicit system: ActorSystem) { val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher)) val deadline = System.currentTimeMillis + 1000 try { @@ -426,6 +429,14 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa assert(f5.value.isEmpty) } } + + "not double-deregister" in { + implicit val dispatcher = interceptedDispatcher() + val a = newTestActor(dispatcher.id) + a ! DoubleStop + awaitCond(statsFor(a, dispatcher).registers.get == 1) + awaitCond(statsFor(a, dispatcher).unregisters.get == 1) + } } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 27853b49db..cc15ae2173 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -189,7 +189,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue final def processAllSystemMessages() { var nextMessage = systemDrain() try { - while (nextMessage ne null) { + while ((nextMessage ne null) && !isClosed) { if (debug) println(actor.self + " processing system message " + nextMessage + " with children " + actor.childrenRefs) actor systemInvoke nextMessage nextMessage = nextMessage.next diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index c200be0871..c0bd0df251 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -16,6 +16,7 @@ import scala.collection.JavaConversions.iterableAsScalaIterable import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean } import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock import akka.jsr166y.ThreadLocalRandom @@ -30,14 +31,77 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup _supervisor, _path) { - private val routeeProps = _props.copy(routerConfig = NoRouter) - private val resizeProgress = new AtomicBoolean + /* + * CAUTION: RoutedActorRef is PROBLEMATIC + * ====================================== + * + * We are constructing/assembling the children outside of the scope of the + * Router actor, inserting them in its childrenRef list, which is not at all + * synchronized. This is done exactly once at start-up, all other accesses + * are done from the Router actor. This means that the only thing which is + * really hairy is making sure that the Router does not touch its childrenRefs + * before we are done with them: create a locked latch really early (hence the + * override of newActorCell) and use that to block the Router constructor for + * as long as it takes to setup the RoutedActorRef itself. + */ + private[akka] var routeReady: ReentrantLock = _ + override def newActorCell( + system: ActorSystemImpl, + ref: InternalActorRef, + props: Props, + supervisor: InternalActorRef, + receiveTimeout: Option[Duration]): ActorCell = { + /* + * TODO RK: check that this really sticks, since this is executed before + * the constructor of RoutedActorRef is executed (invoked from + * LocalActorRef); works on HotSpot and JRockit. + */ + routeReady = new ReentrantLock + routeReady.lock() + super.newActorCell(system, ref, props, supervisor, receiveTimeout) + } + + private[akka] val routerConfig = _props.routerConfig + private[akka] val routeeProps = _props.copy(routerConfig = NoRouter) + private[akka] val resizeProgress = new AtomicBoolean private val resizeCounter = new AtomicLong @volatile private var _routees: IndexedSeq[ActorRef] = IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute def routees = _routees + private[akka] var routeeProvider: RouteeProvider = _ + val route = + try { + routeeProvider = routerConfig.createRouteeProvider(actorContext) + val r = routerConfig.createRoute(routeeProps, routeeProvider) + // initial resize, before message send + resize() + r + } finally routeReady.unlock() // unblock Router’s constructor + + if (routerConfig.resizer.isEmpty && _routees.isEmpty) + throw new ActorInitializationException("router " + routerConfig + " did not register routees!") + + _routees match { + case x ⇒ _routees = x // volatile write to publish the route before sending messages + } + + /* + * end of construction + */ + + def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match { + case _: AutoReceivedMessage ⇒ Nil + case Terminated(_) ⇒ Nil + case CurrentRoutees ⇒ + sender ! RouterRoutees(_routees) + Nil + case _ ⇒ + if (route.isDefinedAt(sender, message)) route(sender, message) + else Nil + } + /** * Adds the routees to existing routees. * Adds death watch of the routees so that they are removed when terminated. @@ -61,29 +125,6 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup abandonedRoutees foreach underlying.unwatch } - private val routeeProvider = _props.routerConfig.createRouteeProvider(actorContext) - val route = _props.routerConfig.createRoute(routeeProps, routeeProvider) - // initial resize, before message send - resize() - - def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match { - case _: AutoReceivedMessage ⇒ Nil - case Terminated(_) ⇒ Nil - case CurrentRoutees ⇒ - sender ! RouterRoutees(_routees) - Nil - case _ ⇒ - if (route.isDefinedAt(sender, message)) route(sender, message) - else Nil - } - - if (_props.routerConfig.resizer.isEmpty && _routees.isEmpty) - throw new ActorInitializationException("router " + _props.routerConfig + " did not register routees!") - - _routees match { - case x ⇒ _routees = x // volatile write to publish the route before sending messages - } - override def !(message: Any)(implicit sender: ActorRef = null): Unit = { resize() @@ -101,14 +142,9 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup } def resize() { - for (r ← _props.routerConfig.resizer) { - if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeProgress.compareAndSet(false, true)) { - try { - r.resize(routeeProps, routeeProvider) - } finally { - resizeProgress.set(false) - } - } + for (r ← routerConfig.resizer) { + if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeProgress.compareAndSet(false, true)) + super.!(Router.Resize) } } } @@ -251,8 +287,18 @@ trait Router extends Actor { case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef") } + // make sure that we synchronize properly to get the childrenRefs into our CPU cache + ref.routeReady.lock() + try if (context.children.isEmpty) + throw new ActorInitializationException("RouterConfig did not create any children") + finally ref.routeReady.unlock() + final def receive = ({ + case Router.Resize ⇒ + try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider)) + finally ref.resizeProgress.set(false) + case Terminated(child) ⇒ ref.removeRoutees(IndexedSeq(child)) if (ref.routees.isEmpty) context.stop(self) @@ -264,6 +310,10 @@ trait Router extends Actor { } } +object Router { + case object Resize +} + /** * Used to broadcast a message to all connections in a router; only the * contained message will be forwarded, i.e. the `Broadcast(...)` @@ -795,14 +845,22 @@ trait Resizer { * for the initial resize and continues with 1 for the first message. Make sure to perform * initial resize before first message (messageCounter == 0), because there is no guarantee * that resize will be done when concurrent messages are in play. + * + * CAUTION: this method is invoked from the thread which tries to send a + * message to the pool, i.e. the ActorRef.!() method, hence it may be called + * concurrently. */ def isTimeForResize(messageCounter: Long): Boolean + /** * Decide if the capacity of the router need to be changed. Will be invoked when `isTimeForResize` * returns true and no other resize is in progress. * Create and register more routees with `routeeProvider.registerRoutees(newRoutees) * or remove routees with `routeeProvider.unregisterRoutees(abandonedRoutees)` and * sending [[akka.actor.PoisonPill]] to them. + * + * This method is invoked only in the context of the Router actor in order to safely + * create/stop children. */ def resize(props: Props, routeeProvider: RouteeProvider) } From b193bcee0423e1a1e716a34c5f867bfb1bd1bf03 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 9 Feb 2012 19:26:02 +0100 Subject: [PATCH 03/39] =?UTF-8?q?incorporate=20Viktor=E2=80=99s=20review?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../scala/akka/actor/TypedActorSpec.scala | 4 +- .../main/scala/akka/actor/ActorSystem.scala | 33 +++--- .../scala/akka/actor/PropertyMaster.scala | 44 +++++++- .../scala/akka/serialization/Format.scala | 104 ------------------ .../akka/serialization/Serialization.scala | 16 +-- .../scala/akka/serialization/Serializer.scala | 4 +- .../src/main/scala/akka/util/Reflect.scala | 14 ++- akka-docs/java/serialization.rst | 13 ++- akka-docs/scala/serialization.rst | 13 ++- 9 files changed, 107 insertions(+), 138 deletions(-) delete mode 100644 akka-actor/src/main/scala/akka/serialization/Format.scala 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 ee15e380c2..4e8bc4d7b4 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -367,7 +367,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) "be able to serialize and deserialize invocations" in { import java.io._ - JavaSerializer.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) { + JavaSerializer.currentSystem.withValue(system.asInstanceOf[ExtendedActorSystem]) { val m = TypedActor.MethodCall(classOf[Foo].getDeclaredMethod("pigdog"), Array[AnyRef]()) val baos = new ByteArrayOutputStream(8192 * 4) val out = new ObjectOutputStream(baos) @@ -386,7 +386,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) "be able to serialize and deserialize invocations' parameters" in { import java.io._ val someFoo: Foo = new Bar - JavaSerializer.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) { + JavaSerializer.currentSystem.withValue(system.asInstanceOf[ExtendedActorSystem]) { val m = TypedActor.MethodCall(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 05334f07d6..33e986d025 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -333,7 +333,7 @@ abstract class ExtendedActorSystem extends ActorSystem { def propertyMaster: PropertyMaster } -class ActorSystemImpl(val name: String, applicationConfig: Config) extends ExtendedActorSystem { +class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Config) extends ExtendedActorSystem { if (!name.matches("""^\w+$""")) throw new IllegalArgumentException("invalid ActorSystem name [" + name + "], must contain only word characters (i.e. [a-zA-Z_0-9])") @@ -360,25 +360,26 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten * This is an extension point: by overriding this method, subclasses can * control all reflection activities of an actor system. */ - protected def createPropertyMaster(): PropertyMaster = new DefaultPropertyMaster(findClassLoader) + protected def createPropertyMaster(): PropertyMaster = new ReflectivePropertyMaster(findClassLoader) + + protected def findClassLoader: ClassLoader = { + def findCaller(get: Int ⇒ Class[_]): ClassLoader = { + val frames = Iterator.from(2).map(get) + frames dropWhile { c ⇒ + c != null && + (c.getName.startsWith("akka.actor.ActorSystem") || + c.getName.startsWith("scala.Option") || + c.getName.startsWith("scala.collection.Iterator") || + c.getName.startsWith("akka.util.Reflect")) + } next () match { + case null ⇒ getClass.getClassLoader + case c ⇒ c.getClassLoader + } + } - protected def findClassLoader: ClassLoader = Option(Thread.currentThread.getContextClassLoader) orElse (Reflect.getCallerClass map findCaller) getOrElse getClass.getClassLoader - - private def findCaller(get: Int ⇒ Class[_]): ClassLoader = { - val frames = Iterator.from(2).map(get) - frames dropWhile { c ⇒ - c != null && - (c.getName.startsWith("akka.actor.ActorSystem") || - c.getName.startsWith("scala.Option") || - c.getName.startsWith("scala.collection.Iterator") || - c.getName.startsWith("akka.util.Reflect")) - } next () match { - case null ⇒ getClass.getClassLoader - case c ⇒ c.getClassLoader - } } private val _pm: PropertyMaster = createPropertyMaster() diff --git a/akka-actor/src/main/scala/akka/actor/PropertyMaster.scala b/akka-actor/src/main/scala/akka/actor/PropertyMaster.scala index 3f3a493b8f..05e4a53362 100644 --- a/akka-actor/src/main/scala/akka/actor/PropertyMaster.scala +++ b/akka-actor/src/main/scala/akka/actor/PropertyMaster.scala @@ -8,19 +8,38 @@ import java.lang.reflect.InvocationTargetException /** * The property master is responsible for acquiring all props needed for a - * performance; in Akka this is the class which is used for reflectively - * loading all configurable parts of an actor system. + * performance; in Akka this is the class which is used for + * loading all configurable parts of an actor system (the + * [[akka.actor.ReflectivePropertyMaster]] is the default implementation). + * + * This is an internal facility and users are not expected to encounter it + * unless they are extending Akka in ways which go beyond simple Extensions. */ -trait PropertyMaster { +private[akka] trait PropertyMaster { + /** + * Obtain a `Class[_]` object loaded with the right class loader (i.e. the one + * returned by `classLoader`). + */ def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] + /** + * Obtain an object conforming to the type T, which is expected to be + * instantiated from a class designated by the fully-qualified class name + * given, where the constructor is selected and invoked according to the + * `args` argument. The exact usage of args depends on which type is requested, + * see the relevant requesting code for details. + */ def getInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] + /** + * Obtain the Scala “object” instance for the given fully-qualified class name, if there is one. + */ def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T] /** - * This is needed e.g. by the JavaSerializer to build the ObjectInputStream. + * This is the class loader to be used in those special cases where the + * other factory method are not applicable (e.g. when constructing a ClassLoaderBinaryInputStream). */ def classLoader: ClassLoader @@ -28,6 +47,14 @@ trait PropertyMaster { object PropertyMaster { + /** + * Convenience method which given a `Class[_]` object and a constructor description + * will create a new instance of that class. + * + * {{{ + * val obj = PropertyMaster.getInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name)) + * }}} + */ def getInstanceFor[T: ClassManifest](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = { val types = args.map(_._1).toArray val values = args.map(_._2).toArray @@ -58,7 +85,14 @@ object PropertyMaster { } -class DefaultPropertyMaster(val classLoader: ClassLoader) extends PropertyMaster { +/** + * This is the default [[akka.actor.PropertyMaster]] implementation used by [[akka.actor.ActorSystemImpl]] + * unless overridden. It uses reflection to turn fully-qualified class names into `Class[_]` objects + * and creates instances from there using `getDeclaredConstructor()` and invoking that. The class loader + * to be used for all this is determined by the [[akka.actor.ActorSystemImpl]]’s `findClassLoader` method + * by default. + */ +class ReflectivePropertyMaster(val classLoader: ClassLoader) extends PropertyMaster { import PropertyMaster.withErrorHandling diff --git a/akka-actor/src/main/scala/akka/serialization/Format.scala b/akka-actor/src/main/scala/akka/serialization/Format.scala deleted file mode 100644 index 7ce9d8a2e6..0000000000 --- a/akka-actor/src/main/scala/akka/serialization/Format.scala +++ /dev/null @@ -1,104 +0,0 @@ -package akka.serialization - -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -import akka.actor.Actor - -/** - * trait Serializer extends scala.Serializable { - * @volatile - * var classLoader: Option[ClassLoader] = None - * def deepClone(obj: AnyRef): AnyRef = fromBinary(toBinary(obj), Some(obj.getClass)) - * - * def toBinary(obj: AnyRef): Array[Byte] - * def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef - * } - */ - -/** - * - * object Format { - * implicit object Default extends Serializer { - * import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream } - * //import org.apache.commons.io.input.ClassLoaderObjectInputStream - * - * def toBinary(obj: AnyRef): Array[Byte] = { - * val bos = new ByteArrayOutputStream - * val out = new ObjectOutputStream(bos) - * out.writeObject(obj) - * out.close() - * bos.toByteArray - * } - * - * def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]], classLoader: Option[ClassLoader] = None): AnyRef = { - * val in = - * //if (classLoader.isDefined) new ClassLoaderObjectInputStream(classLoader.get, new ByteArrayInputStream(bytes)) else - * new ObjectInputStream(new ByteArrayInputStream(bytes)) - * val obj = in.readObject - * in.close() - * obj - * } - * - * def identifier: Byte = 111 //Pick a number and hope no one has chosen the same :-) 0 - 16 is reserved for Akka internals - * - * } - * - * val defaultSerializerName = Default.getClass.getName - * } - */ - -trait FromBinary[T <: Actor] { - def fromBinary(bytes: Array[Byte], act: T): T -} - -trait ToBinary[T <: Actor] { - def toBinary(t: T): Array[Byte] -} - -/** - * Type class definition for Actor Serialization. - * Client needs to implement Format[] for the respective actor. - */ -// FIXME RK: should this go? It’s not used anywhere, looks like cluster residue. -trait Format[T <: Actor] extends FromBinary[T] with ToBinary[T] - -/** - * A default implementation for a stateless actor - * - * Create a Format object with the client actor as the implementation of the type class - * - *
- * object BinaryFormatMyStatelessActor  {
- *   implicit object MyStatelessActorFormat extends StatelessActorFormat[MyStatelessActor]
- * }
- * 
- */ -trait StatelessActorFormat[T <: Actor] extends Format[T] with scala.Serializable { - def fromBinary(bytes: Array[Byte], act: T) = act - - def toBinary(ac: T) = Array.empty[Byte] -} - -/** - * A default implementation of the type class for a Format that specifies a serializer - * - * Create a Format object with the client actor as the implementation of the type class and - * a serializer object - * - *
- * object BinaryFormatMyJavaSerializableActor  {
- *   implicit object MyJavaSerializableActorFormat extends SerializerBasedActorFormat[MyJavaSerializableActor]  {
- *     val serializer = Serializers.Java
- *   }
- * }
- * 
- */ -trait SerializerBasedActorFormat[T <: Actor] extends Format[T] with scala.Serializable { - val serializer: Serializer - - def fromBinary(bytes: Array[Byte], act: T): T = serializer.fromBinary(bytes, Some(act.getClass)).asInstanceOf[T] - - def toBinary(ac: T): Array[Byte] = serializer.toBinary(ac) -} diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 1751f49082..9cbb15a676 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -11,6 +11,7 @@ import akka.config.ConfigurationException import akka.actor.{ Extension, ExtendedActorSystem, Address } import java.util.concurrent.ConcurrentHashMap import akka.event.Logging +import akka.util.NonFatal case class NoSerializerFoundException(m: String) extends AkkaException(m) @@ -59,9 +60,9 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * Serializes the given AnyRef/java.lang.Object according to the Serialization configuration * to either an Array of Bytes or an Exception if one was thrown. */ - def serialize(o: AnyRef): Either[Exception, Array[Byte]] = + def serialize(o: AnyRef): Either[Throwable, Array[Byte]] = try Right(findSerializerFor(o).toBinary(o)) - catch { case e: Exception ⇒ Left(e) } + catch { case NonFatal(e) ⇒ Left(e) } /** * Deserializes the given array of bytes using the specified serializer id, @@ -70,18 +71,18 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { */ def deserialize(bytes: Array[Byte], serializerId: Int, - clazz: Option[Class[_]]): Either[Exception, AnyRef] = + clazz: Option[Class[_]]): Either[Throwable, AnyRef] = try Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz)) - catch { case e: Exception ⇒ Left(e) } + catch { case NonFatal(e) ⇒ Left(e) } /** * Deserializes the given array of bytes using the specified type to look up what Serializer should be used. * You can specify an optional ClassLoader to load the object into. * Returns either the resulting object or an Exception if one was thrown. */ - def deserialize(bytes: Array[Byte], clazz: Class[_]): Either[Exception, AnyRef] = + def deserialize(bytes: Array[Byte], clazz: Class[_]): Either[Throwable, AnyRef] = try Right(serializerFor(clazz).fromBinary(bytes, Some(clazz))) - catch { case e: Exception ⇒ Left(e) } + catch { case NonFatal(e) ⇒ Left(e) } /** * Returns the Serializer configured for the given object, returns the NullSerializer if it's null, @@ -126,7 +127,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { } /** - * Tries to load the specified Serializer by the FQN + * Tries to load the specified Serializer by the fully-qualified name; the actual + * loading is performed by the system’s [[akka.actor.PropertyMaster]]. */ def serializerOf(serializerFQN: String): Either[Throwable, Serializer] = { val pm = system.propertyMaster diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index 0acd6e4f8c..66dfd560c2 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -25,7 +25,9 @@ import scala.util.DynamicVariable * load classes using reflection. * * - * Be sure to always use the PropertyManager for loading classes! + * Be sure to always use the PropertyManager for loading classes! This is necessary to + * avoid strange match errors and inequalities which arise from different class loaders loading + * the same class. */ trait Serializer extends scala.Serializable { diff --git a/akka-actor/src/main/scala/akka/util/Reflect.scala b/akka-actor/src/main/scala/akka/util/Reflect.scala index d2ec6ddfbd..25c56a983f 100644 --- a/akka-actor/src/main/scala/akka/util/Reflect.scala +++ b/akka-actor/src/main/scala/akka/util/Reflect.scala @@ -3,13 +3,25 @@ */ package akka.util +/** + * Collection of internal reflection utilities which may or may not be + * available (most services specific to HotSpot, but fails gracefully). + */ object Reflect { + /** + * This optionally holds a function which looks N levels above itself + * on the call stack and returns the `Class[_]` object for the code + * executing in that stack frame. Implemented using + * `sun.reflect.Reflection.getCallerClass` if available, None otherwise. + * + * Hint: when comparing to Thread.currentThread.getStackTrace, add two levels. + */ val getCallerClass: Option[Int ⇒ Class[_]] = { try { val c = Class.forName("sun.reflect.Reflection"); val m = c.getMethod("getCallerClass", Array(classOf[Int]): _*) - Some((i: Int) ⇒ m.invoke(null, Array[AnyRef](i.asInstanceOf[Integer]): _*).asInstanceOf[Class[_]]) + Some((i: Int) ⇒ m.invoke(null, Array[AnyRef](i.asInstanceOf[java.lang.Integer]): _*).asInstanceOf[Class[_]]) } catch { case NonFatal(e) ⇒ None } diff --git a/akka-docs/java/serialization.rst b/akka-docs/java/serialization.rst index 2920538ded..ee654460ad 100644 --- a/akka-docs/java/serialization.rst +++ b/akka-docs/java/serialization.rst @@ -110,4 +110,15 @@ which is done by extending ``akka.serialization.JSerializer``, like this: :exclude: ... Then you only need to fill in the blanks, bind it to a name in your :ref:`configuration` and then -list which classes that should be serialized using it. \ No newline at end of file +list which classes that should be serialized using it. + +A Word About Java Serialization +=============================== + +When using Java serialization without employing the :class:`JavaSerializer` for +the task, you must make sure to supply a valid :class:`ExtendedActorSystem` in +the dynamic variable ``JavaSerializer.currentSystem``. This is used when +reading in the representation of an :class:`ActorRef` for turning the string +representation into a real reference. :class:`DynamicVariable` is a +thread-local variable, so be sure to have it set while deserializing anything +which might contain actor references. diff --git a/akka-docs/scala/serialization.rst b/akka-docs/scala/serialization.rst index 6a0867dea2..164bc1c819 100644 --- a/akka-docs/scala/serialization.rst +++ b/akka-docs/scala/serialization.rst @@ -108,4 +108,15 @@ First you need to create a class definition of your ``Serializer`` like so: :exclude: ... Then you only need to fill in the blanks, bind it to a name in your :ref:`configuration` and then -list which classes that should be serialized using it. \ No newline at end of file +list which classes that should be serialized using it. + +A Word About Java Serialization +=============================== + +When using Java serialization without employing the :class:`JavaSerializer` for +the task, you must make sure to supply a valid :class:`ExtendedActorSystem` in +the dynamic variable ``JavaSerializer.currentSystem``. This is used when +reading in the representation of an :class:`ActorRef` for turning the string +representation into a real reference. :class:`DynamicVariable` is a +thread-local variable, so be sure to have it set while deserializing anything +which might contain actor references. From 4a5f5eef21d9374cb2c635ce54896128c8fd3192 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 9 Feb 2012 21:21:31 +0100 Subject: [PATCH 04/39] Improve docs and api for zeromq. See #1713 * Wrote a comprehensive example for pub-sub * Clarified how publish to topic is done * Several minor, but important, api adjustments for the java api, and some also profit for scala * Added documentation for Java and updated documentation for Scala --- .../code/akka/docs/zeromq/ZeromqDocTest.scala | 8 + .../akka/docs/zeromq/ZeromqDocTestBase.java | 284 ++++++++++++++++++ akka-docs/java/index.rst | 1 + akka-docs/java/zeromq.rst | 98 ++++++ .../code/akka/docs/zeromq/ZeromqDocSpec.scala | 187 ++++++++++++ akka-docs/scala/zeromq.rst | 106 +++---- .../main/scala/akka/zeromq/SocketOption.scala | 29 +- .../akka/zeromq/ZMQMessageDeserializer.scala | 9 +- .../scala/akka/zeromq/ZeroMQExtension.scala | 83 ++--- .../zeromq/ConcurrentSocketActorSpec.scala | 7 +- project/AkkaBuild.scala | 3 +- 11 files changed, 703 insertions(+), 112 deletions(-) create mode 100644 akka-docs/java/code/akka/docs/zeromq/ZeromqDocTest.scala create mode 100644 akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java create mode 100644 akka-docs/java/zeromq.rst create mode 100644 akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala diff --git a/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTest.scala b/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTest.scala new file mode 100644 index 0000000000..a9747959e3 --- /dev/null +++ b/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTest.scala @@ -0,0 +1,8 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.zeromq + +import org.scalatest.junit.JUnitSuite + +class ZeromqDocTest extends ZeromqDocTestBase with JUnitSuite diff --git a/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java b/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java new file mode 100644 index 0000000000..fc40e3af67 --- /dev/null +++ b/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java @@ -0,0 +1,284 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.zeromq; + +//#pub-socket +import akka.zeromq.Bind; +import akka.zeromq.ZeroMQExtension; + +//#pub-socket +//#sub-socket +import akka.zeromq.Connect; +import akka.zeromq.Listener; +import akka.zeromq.Subscribe; + +//#sub-socket +//#unsub-topic-socket +import akka.zeromq.Unsubscribe; + +//#unsub-topic-socket +//#pub-topic +import akka.zeromq.Frame; +import akka.zeromq.ZMQMessage; + +//#pub-topic + +import akka.zeromq.HighWatermark; +import akka.zeromq.SocketOption; +import akka.zeromq.ZeroMQVersion; + +//#health +import akka.actor.ActorRef; +import akka.actor.UntypedActor; +import akka.actor.Props; +import akka.event.Logging; +import akka.event.LoggingAdapter; +import akka.util.Duration; +import akka.serialization.SerializationExtension; +import akka.serialization.Serialization; +import java.io.Serializable; +import java.lang.management.ManagementFactory; +//#health + +import com.typesafe.config.ConfigFactory; + +import java.lang.management.MemoryMXBean; +import java.lang.management.MemoryUsage; +import java.lang.management.OperatingSystemMXBean; +import java.util.concurrent.TimeUnit; +import java.util.Date; +import java.text.SimpleDateFormat; + +import akka.actor.ActorSystem; +import akka.testkit.AkkaSpec; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.Assume; + +import akka.zeromq.SocketType; + +public class ZeromqDocTestBase { + + ActorSystem system; + + @Before + public void setUp() { + system = ActorSystem.create("ZeromqDocTest", + ConfigFactory.parseString("akka.loglevel=INFO").withFallback(AkkaSpec.testConf())); + } + + @After + public void tearDown() { + system.shutdown(); + } + + @Test + public void demonstrateCreateSocket() { + Assume.assumeTrue(checkZeroMQInstallation()); + + //#pub-socket + ActorRef pubSocket = ZeroMQExtension.get(system).newPubSocket(new Bind("tcp://127.0.0.1:1233")); + //#pub-socket + + //#sub-socket + ActorRef listener = system.actorOf(new Props(ListenerActor.class)); + ActorRef subSocket = ZeroMQExtension.get(system).newSubSocket(new Connect("tcp://127.0.0.1:1233"), + new Listener(listener), Subscribe.all()); + //#sub-socket + + //#sub-topic-socket + ActorRef subTopicSocket = ZeroMQExtension.get(system).newSubSocket(new Connect("tcp://127.0.0.1:1233"), + new Listener(listener), new Subscribe("foo.bar")); + //#sub-topic-socket + + //#unsub-topic-socket + subTopicSocket.tell(new Unsubscribe("foo.bar")); + //#unsub-topic-socket + + byte[] payload = new byte[0]; + //#pub-topic + pubSocket.tell(new ZMQMessage(new Frame("foo.bar"), new Frame(payload))); + //#pub-topic + + //#high-watermark + ActorRef highWatermarkSocket = ZeroMQExtension.get(system).newRouterSocket( + new SocketOption[] { new Listener(listener), new Bind("tcp://127.0.0.1:1233"), new HighWatermark(50000) }); + //#high-watermark + } + + @Test + public void demonstratePubSub() throws Exception { + Assume.assumeTrue(checkZeroMQInstallation()); + + //#health2 + + system.actorOf(new Props(HealthProbe.class), "health"); + //#health2 + + //#logger2 + + system.actorOf(new Props(Logger.class), "logger"); + //#logger2 + + //#alerter2 + + system.actorOf(new Props(HeapAlerter.class), "alerter"); + //#alerter2 + + Thread.sleep(3000L); + } + + private boolean checkZeroMQInstallation() { + try { + ZeroMQVersion v = ZeroMQExtension.get(system).version(); + return (v.major() == 2 && v.minor() == 1); + } catch (LinkageError e) { + return false; + } + } + + //#listener-actor + public static class ListenerActor extends UntypedActor { + public void onReceive(Object message) throws Exception { + //... + } + } + + //#listener-actor + + //#health + + public static final Object TICK = "TICK"; + + public static class Heap implements Serializable { + public final long timestamp; + public final long used; + public final long max; + + public Heap(long timestamp, long used, long max) { + this.timestamp = timestamp; + this.used = used; + this.max = max; + } + } + + public static class Load implements Serializable { + public final long timestamp; + public final double loadAverage; + + public Load(long timestamp, double loadAverage) { + this.timestamp = timestamp; + this.loadAverage = loadAverage; + } + } + + public static class HealthProbe extends UntypedActor { + + ActorRef pubSocket = ZeroMQExtension.get(getContext().system()).newPubSocket(new Bind("tcp://127.0.0.1:1237")); + MemoryMXBean memory = ManagementFactory.getMemoryMXBean(); + OperatingSystemMXBean os = ManagementFactory.getOperatingSystemMXBean(); + Serialization ser = SerializationExtension.get(getContext().system()); + + @Override + public void preStart() { + getContext().system().scheduler() + .schedule(Duration.parse("1 second"), Duration.parse("1 second"), getSelf(), TICK); + } + + @Override + public void postRestart(Throwable reason) { + // don't call preStart + } + + @Override + public void onReceive(Object message) { + if (message.equals(TICK)) { + MemoryUsage currentHeap = memory.getHeapMemoryUsage(); + long timestamp = System.currentTimeMillis(); + + // use akka SerializationExtension to convert to bytes + byte[] heapPayload = ser.serializerFor(Heap.class).toBinary( + new Heap(timestamp, currentHeap.getUsed(), currentHeap.getMax())); + // the first frame is the topic, second is the message + pubSocket.tell(new ZMQMessage(new Frame("health.heap"), new Frame(heapPayload))); + + // use akka SerializationExtension to convert to bytes + byte[] loadPayload = ser.serializerFor(Load.class).toBinary(new Load(timestamp, os.getSystemLoadAverage())); + // the first frame is the topic, second is the message + pubSocket.tell(new ZMQMessage(new Frame("health.load"), new Frame(loadPayload))); + } else { + unhandled(message); + } + } + + } + + //#health + + //#logger + public static class Logger extends UntypedActor { + + ActorRef subSocket = ZeroMQExtension.get(getContext().system()).newSubSocket(new Connect("tcp://127.0.0.1:1237"), + new Listener(getSelf()), new Subscribe("health")); + Serialization ser = SerializationExtension.get(getContext().system()); + SimpleDateFormat timestampFormat = new SimpleDateFormat("HH:mm:ss.SSS"); + LoggingAdapter log = Logging.getLogger(getContext().system(), this); + + @Override + public void onReceive(Object message) { + if (message instanceof ZMQMessage) { + ZMQMessage m = (ZMQMessage) message; + // the first frame is the topic, second is the message + if (m.firstFrameAsString().equals("health.heap")) { + Heap heap = (Heap) ser.serializerFor(Heap.class).fromBinary(m.payload(1)); + log.info("Used heap {} bytes, at {}", heap.used, timestampFormat.format(new Date(heap.timestamp))); + } else if (m.firstFrameAsString().equals("health.load")) { + Load load = (Load) ser.serializerFor(Load.class).fromBinary(m.payload(1)); + log.info("Load average {}, at {}", load.loadAverage, timestampFormat.format(new Date(load.timestamp))); + } + } else { + unhandled(message); + } + } + + } + + //#logger + + //#alerter + public static class HeapAlerter extends UntypedActor { + + ActorRef subSocket = ZeroMQExtension.get(getContext().system()).newSubSocket(new Connect("tcp://127.0.0.1:1237"), + new Listener(getSelf()), new Subscribe("health.heap")); + Serialization ser = SerializationExtension.get(getContext().system()); + LoggingAdapter log = Logging.getLogger(getContext().system(), this); + int count = 0; + + @Override + public void onReceive(Object message) { + if (message instanceof ZMQMessage) { + ZMQMessage m = (ZMQMessage) message; + // the first frame is the topic, second is the message + if (m.firstFrameAsString().equals("health.heap")) { + Heap heap = (Heap) ser.serializerFor(Heap.class).fromBinary(m.payload(1)); + if (((double) heap.used / heap.max) > 0.9) { + count += 1; + } else { + count = 0; + } + if (count > 10) { + log.warning("Need more memory, using {} %", (100.0 * heap.used / heap.max)); + } + } + } else { + unhandled(message); + } + } + + } + //#alerter + +} diff --git a/akka-docs/java/index.rst b/akka-docs/java/index.rst index 319dbab302..9ed521e5e7 100644 --- a/akka-docs/java/index.rst +++ b/akka-docs/java/index.rst @@ -23,3 +23,4 @@ Java API transactors fsm extending-akka + zeromq diff --git a/akka-docs/java/zeromq.rst b/akka-docs/java/zeromq.rst new file mode 100644 index 0000000000..e24dd28796 --- /dev/null +++ b/akka-docs/java/zeromq.rst @@ -0,0 +1,98 @@ + +.. _zeromq-java: + +############### + ZeroMQ (Java) +############### + +.. sidebar:: Contents + + .. contents:: :local: + +Akka provides a ZeroMQ module which abstracts a ZeroMQ connection and therefore allows interaction between Akka actors to take place over ZeroMQ connections. The messages can be of a proprietary format or they can be defined using Protobuf. The socket actor is fault-tolerant by default and when you use the newSocket method to create new sockets it will properly reinitialize the socket. + +ZeroMQ is very opinionated when it comes to multi-threading so configuration option `akka.zeromq.socket-dispatcher` always needs to be configured to a PinnedDispatcher, because the actual ZeroMQ socket can only be accessed by the thread that created it. + +The ZeroMQ module for Akka is written against an API introduced in JZMQ, which uses JNI to interact with the native ZeroMQ library. Instead of using JZMQ, the module uses ZeroMQ binding for Scala that uses the native ZeroMQ library through JNA. In other words, the only native library that this module requires is the native ZeroMQ library. +The benefit of the scala library is that you don't need to compile and manage native dependencies at the cost of some runtime performance. The scala-bindings are compatible with the JNI bindings so they are a drop-in replacement, in case you really need to get that extra bit of performance out. + +Connection +========== + +ZeroMQ supports multiple connectivity patterns, each aimed to meet a different set of requirements. Currently, this module supports publisher-subscriber connections and connections based on dealers and routers. For connecting or accepting connections, a socket must be created. +Sockets are always created using the ``akka.zeromq.ZeroMQExtension``, for example: + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#pub-socket + +Above examples will create a ZeroMQ Publisher socket that is Bound to the port 1233 on localhost. + +Similarly you can create a subscription socket, with a listener, that subscribes to all messages from the publisher using: + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#sub-socket + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#listener-actor + +The following sub-sections describe the supported connection patterns and how they can be used in an Akka environment. However, for a comprehensive discussion of connection patterns, please refer to `ZeroMQ -- The Guide `_. + +Publisher-subscriber connection +------------------------------- + +In a publisher-subscriber (pub-sub) connection, the publisher accepts one or more subscribers. Each subscriber shall +subscribe to one or more topics, whereas the publisher publishes messages to a set of topics. Also, a subscriber can +subscribe to all available topics. In an Akka environment, pub-sub connections shall be used when an actor sends messages +to one or more actors that do not interact with the actor that sent the message. + +When you're using zeromq pub/sub you should be aware that it needs multicast - check your cloud - to work properly and that the filtering of events for topics happens client side, so all events are always broadcasted to every subscriber. + +An actor is subscribed to a topic as follows: + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#sub-topic-socket + +It is a prefix match so it is subscribed to all topics starting with ``foo.bar``. Note that if the given string is empty or +``Subscribe.all()`` is used, the actor is subscribed to all topics. + +To unsubscribe from a topic you do the following: + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#unsub-topic-socket + +To publish messages to a topic you must use two Frames with the topic in the first frame. + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#pub-topic + +Pub-Sub in Action +^^^^^^^^^^^^^^^^^ + +The following example illustrates one publisher with two subscribers. + +The publisher monitors current heap usage and system load and periodically publishes ``Heap`` events on the ``"health.heap"`` topic +and ``Load`` events on the ``"health.load"`` topic. + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#health + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#health2 + +Let's add one subscriber that logs the information. It subscribes to all topics starting with ``"health"``, i.e. both ``Heap`` and +``Load`` events. + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#logger + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#logger2 + +Another subscriber keep track of used heap and warns if too much heap is used. It only subscribes to ``Heap`` events. + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#alerter + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#alerter2 + +Router-Dealer connection +------------------------ + +While Pub/Sub is nice the real advantage of zeromq is that it is a "lego-box" for reliable messaging. And because there are so many integrations the multi-language support is fantastic. +When you're using ZeroMQ to integrate many systems you'll probably need to build your own ZeroMQ devices. This is where the router and dealer socket types come in handy. +With those socket types you can build your own reliable pub sub broker that uses TCP/IP and does publisher side filtering of events. + +To create a Router socket that has a high watermark configured, you would do: + +.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#high-watermark + +The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket. diff --git a/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala b/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala new file mode 100644 index 0000000000..796c95b27d --- /dev/null +++ b/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala @@ -0,0 +1,187 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.zeromq + +import akka.actor.Actor +import akka.actor.Props +import akka.util.duration._ +import akka.testkit._ +import akka.zeromq.ZeroMQVersion +import akka.zeromq.ZeroMQExtension +import java.text.SimpleDateFormat +import java.util.Date +import akka.zeromq.SocketType +import akka.zeromq.Bind + +object ZeromqDocSpec { + + //#health + import akka.zeromq._ + import akka.actor.Actor + import akka.actor.Props + import akka.actor.ActorLogging + import akka.serialization.SerializationExtension + import java.lang.management.ManagementFactory + + case object Tick + case class Heap(timestamp: Long, used: Long, max: Long) + case class Load(timestamp: Long, loadAverage: Double) + + class HealthProbe extends Actor { + + val pubSocket = context.system.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1235")) + val memory = ManagementFactory.getMemoryMXBean + val os = ManagementFactory.getOperatingSystemMXBean + val ser = SerializationExtension(context.system) + + context.system.scheduler.schedule(1 second, 1 second, self, Tick) + + def receive: Receive = { + case Tick ⇒ + val currentHeap = memory.getHeapMemoryUsage + val timestamp = System.currentTimeMillis + + // use akka SerializationExtension to convert to bytes + val heapPayload = ser.serialize(Heap(timestamp, currentHeap.getUsed, currentHeap.getMax)).fold(throw _, identity) + // the first frame is the topic, second is the message + pubSocket ! ZMQMessage(Seq(Frame("health.heap"), Frame(heapPayload))) + + // use akka SerializationExtension to convert to bytes + val loadPayload = ser.serialize(Load(timestamp, os.getSystemLoadAverage)).fold(throw _, identity) + // the first frame is the topic, second is the message + pubSocket ! ZMQMessage(Seq(Frame("health.load"), Frame(loadPayload))) + } + } + //#health + + //#logger + class Logger extends Actor with ActorLogging { + + context.system.newSocket(SocketType.Sub, Listener(self), Connect("tcp://127.0.0.1:1235"), Subscribe("health")) + val ser = SerializationExtension(context.system) + val timestampFormat = new SimpleDateFormat("HH:mm:ss.SSS") + + def receive = { + // the first frame is the topic, second is the message + case m: ZMQMessage if m.firstFrameAsString == "health.heap" ⇒ + ser.deserialize(m.payload(1), classOf[Heap], None) match { + case Right(Heap(timestamp, used, max)) ⇒ + log.info("Used heap {} bytes, at {}", used, timestampFormat.format(new Date(timestamp))) + case Left(e) ⇒ throw e + } + + case m: ZMQMessage if m.firstFrameAsString == "health.load" ⇒ + ser.deserialize(m.payload(1), classOf[Load], None) match { + case Right(Load(timestamp, loadAverage)) ⇒ + log.info("Load average {}, at {}", loadAverage, timestampFormat.format(new Date(timestamp))) + case Left(e) ⇒ throw e + } + } + } + //#logger + + //#alerter + class HeapAlerter extends Actor with ActorLogging { + + context.system.newSocket(SocketType.Sub, Listener(self), Connect("tcp://127.0.0.1:1235"), Subscribe("health.heap")) + val ser = SerializationExtension(context.system) + var count = 0 + + def receive = { + // the first frame is the topic, second is the message + case m: ZMQMessage if m.firstFrameAsString == "health.heap" ⇒ + ser.deserialize(m.payload(1), classOf[Heap], None) match { + case Right(Heap(timestamp, used, max)) ⇒ + if ((used.toDouble / max) > 0.9) count += 1 + else count = 0 + if (count > 10) log.warning("Need more memory, using {} %", (100.0 * used / max)) + case Left(e) ⇒ throw e + } + } + } + //#alerter + +} + +class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") { + import ZeromqDocSpec._ + + "demonstrate how to create socket" in { + checkZeroMQInstallation() + + //#pub-socket + import akka.zeromq.ZeroMQExtension + val pubSocket = ZeroMQExtension(system).newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234")) + //#pub-socket + + //#pub-socket2 + import akka.zeromq._ + val pubSocket2 = system.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234")) + //#pub-socket2 + + //#sub-socket + import akka.zeromq._ + val listener = system.actorOf(Props(new Actor { + def receive: Receive = { + case Connecting ⇒ //... + case m: ZMQMessage ⇒ //... + case _ ⇒ //... + } + })) + val subSocket = system.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://127.0.0.1:1234"), SubscribeAll) + //#sub-socket + + //#sub-topic-socket + val subTopicSocket = system.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://127.0.0.1:1234"), Subscribe("foo.bar")) + //#sub-topic-socket + + //#unsub-topic-socket + subTopicSocket ! Unsubscribe("foo.bar") + //#unsub-topic-socket + + val payload = Array.empty[Byte] + //#pub-topic + pubSocket ! ZMQMessage(Seq(Frame("foo.bar"), Frame(payload))) + //#pub-topic + + //#high-watermark + val highWatermarkSocket = system.newSocket( + SocketType.Router, + Listener(listener), + Bind("tcp://127.0.0.1:1234"), + HighWatermark(50000)) + //#high-watermark + } + + "demonstrate pub-sub" in { + checkZeroMQInstallation() + + //#health + + system.actorOf(Props[HealthProbe], name = "health") + //#health + + //#logger + + system.actorOf(Props[Logger], name = "logger") + //#logger + + //#alerter + + system.actorOf(Props[HeapAlerter], name = "alerter") + //#alerter + + Thread.sleep(3000) + + } + + def checkZeroMQInstallation() = try { + ZeroMQExtension(system).version match { + case ZeroMQVersion(2, 1, _) ⇒ Unit + case version ⇒ pending + } + } catch { + case e: LinkageError ⇒ pending + } +} diff --git a/akka-docs/scala/zeromq.rst b/akka-docs/scala/zeromq.rst index e080979910..fa1160ee6d 100644 --- a/akka-docs/scala/zeromq.rst +++ b/akka-docs/scala/zeromq.rst @@ -1,8 +1,10 @@ -.. _zeromq-module: +.. _zeromq-scala: + +################ + ZeroMQ (Scala) +################ -ZeroMQ -====== .. sidebar:: Contents @@ -12,83 +14,76 @@ Akka provides a ZeroMQ module which abstracts a ZeroMQ connection and therefore ZeroMQ is very opinionated when it comes to multi-threading so configuration option `akka.zeromq.socket-dispatcher` always needs to be configured to a PinnedDispatcher, because the actual ZeroMQ socket can only be accessed by the thread that created it. -The ZeroMQ module for Akka is written against an API introduced in JZMQ, which uses JNI to interact with the native ZeroMQ library. Instead of using JZMQ, the module uses ZeroMQ binding for Scala that uses the native ZeroMQ library through JNA. In other words, the only native library that this module requires is the native ZeroMQ library. +The ZeroMQ module for Akka is written against an API introduced in JZMQ, which uses JNI to interact with the native ZeroMQ library. Instead of using JZMQ, the module uses ZeroMQ binding for Scala that uses the native ZeroMQ library through JNA. In other words, the only native library that this module requires is the native ZeroMQ library. The benefit of the scala library is that you don't need to compile and manage native dependencies at the cost of some runtime performance. The scala-bindings are compatible with the JNI bindings so they are a drop-in replacement, in case you really need to get that extra bit of performance out. Connection ----------- +========== -ZeroMQ supports multiple connectivity patterns, each aimed to meet a different set of requirements. Currently, this module supports publisher-subscriber connections and connections based on dealers and routers. For connecting or accepting connections, a socket must be created. Sockets are always created using ``akka.zeromq.ZeroMQ.newSocket``, for example: +ZeroMQ supports multiple connectivity patterns, each aimed to meet a different set of requirements. Currently, this module supports publisher-subscriber connections and connections based on dealers and routers. For connecting or accepting connections, a socket must be created. +Sockets are always created using the ``akka.zeromq.ZeroMQExtension``, for example: -.. code-block:: scala +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-socket - import akka.zeromq._ - val socket = system.zeromq.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234")) +or by importing the ``akka.zeromq._`` package to make newSocket method available on system, via an implicit conversion. -will create a ZeroMQ Publisher socket that is Bound to the port 1234 on localhost. -Importing the akka.zeromq._ package ensures that the implicit zeromq method is available. -Similarly you can create a subscription socket, that subscribes to all messages from the publisher using: +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-socket2 -.. code-block:: scala - val socket = system.zeromq.newSocket(SocketType.Sub, Connect("tcp://127.0.0.1:1234"), SubscribeAll) +Above examples will create a ZeroMQ Publisher socket that is Bound to the port 1234 on localhost. -Also, a socket may be created with a listener that handles received messages as well as notifications: +Similarly you can create a subscription socket, with a listener, that subscribes to all messages from the publisher using: -.. code-block:: scala - - val listener = system.actorOf(Props(new Actor { - def receive: Receive = { - case Connecting => ... - case _ => ... - } - })) - val socket = system.zeromq.newSocket(SocketType.Router, Listener(listener), Connect("tcp://localhost:1234")) +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#sub-socket The following sub-sections describe the supported connection patterns and how they can be used in an Akka environment. However, for a comprehensive discussion of connection patterns, please refer to `ZeroMQ -- The Guide `_. Publisher-subscriber connection -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +------------------------------- -In a publisher-subscriber (pub-sub) connection, the publisher accepts one or more subscribers. Each subscriber shall subscribe to one or more topics, whereas the publisher publishes messages to a set of topics. Also, a subscriber can subscribe to all available topics. +In a publisher-subscriber (pub-sub) connection, the publisher accepts one or more subscribers. Each subscriber shall +subscribe to one or more topics, whereas the publisher publishes messages to a set of topics. Also, a subscriber can +subscribe to all available topics. In an Akka environment, pub-sub connections shall be used when an actor sends messages +to one or more actors that do not interact with the actor that sent the message. When you're using zeromq pub/sub you should be aware that it needs multicast - check your cloud - to work properly and that the filtering of events for topics happens client side, so all events are always broadcasted to every subscriber. An actor is subscribed to a topic as follows: -.. code-block:: scala +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#sub-topic-socket - val socket = system.zeromq.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://localhost:1234"), Subscribe("the-topic")) +It is a prefix match so it is subscribed to all topics starting with ``foo.bar``. Note that if the given string is empty or +``SubscribeAll`` is used, the actor is subscribed to all topics. -Note that if the given string is empty (see below), the actor is subscribed to all topics. To unsubscribe from a topic you do the following: +To unsubscribe from a topic you do the following: -.. code-block:: scala +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#unsub-topic-socket - socket ! Unsubscribe("SomeTopic1") +To publish messages to a topic you must use two Frames with the topic in the first frame. -In an Akka environment, pub-sub connections shall be used when an actor sends messages to one or more actors that do not interact with the actor that sent the message. The following piece of code creates a publisher actor, binds the socket, and sends a message to be published: +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-topic -.. code-block:: scala +Pub-Sub in Action +^^^^^^^^^^^^^^^^^ - import akka.zeromq._ - val socket = system.zeromq.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234")) - socket ! Send("hello".getBytes) +The following example illustrates one publisher with two subscribers. -In the following code, the subscriber is configured to receive messages for all topics: +The publisher monitors current heap usage and system load and periodically publishes ``Heap`` events on the ``"health.heap"`` topic +and ``Load`` events on the ``"health.load"`` topic. -.. code-block:: scala +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#health - import akka.zeromq._ - val listener = system.actorOf(Props(new Actor { - def receive: Receive = { - case Connecting => ... - case _ => ... - } - })) - val socket = system.zeromq.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://127.0.0.1:1234"), SubscribeAll) +Let's add one subscriber that logs the information. It subscribes to all topics starting with ``"health"``, i.e. both ``Heap`` and +``Load`` events. + +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#logger + +Another subscriber keep track of used heap and warns if too much heap is used. It only subscribes to ``Heap`` events. + +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#alerter Router-Dealer connection -^^^^^^^^^^^^^^^^^^^^^^^^ +------------------------ While Pub/Sub is nice the real advantage of zeromq is that it is a "lego-box" for reliable messaging. And because there are so many integrations the multi-language support is fantastic. When you're using ZeroMQ to integrate many systems you'll probably need to build your own ZeroMQ devices. This is where the router and dealer socket types come in handy. @@ -96,19 +91,6 @@ With those socket types you can build your own reliable pub sub broker that uses To create a Router socket that has a high watermark configured, you would do: -.. code-block:: scala - - import akka.zeromq._ - val listener = system.actorOf(Props(new Actor { - def receive: Receive = { - case Connecting => ... - case _ => ... - } - })) - val socket = system.zeromq.newSocket( - SocketType.Router, - Listener(listener), - Bind("tcp://127.0.0.1:1234"), - HWM(50000)) +.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#high-watermark -The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket. \ No newline at end of file +The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket. diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala index 9cded84da1..d3b824bca1 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala @@ -174,9 +174,12 @@ private[zeromq] case object Close extends Request * * @param payload the topic to subscribe to */ -case class Subscribe(payload: Seq[Byte]) extends PubSubOption +case class Subscribe(payload: Seq[Byte]) extends PubSubOption { + def this(topic: String) = this(topic.getBytes("UTF-8")) +} object Subscribe { - def apply(topic: String): Subscribe = new Subscribe(topic.getBytes) + def apply(topic: String): Subscribe = new Subscribe(topic) + val all = Subscribe(Seq.empty) } /** @@ -188,9 +191,11 @@ object Subscribe { * * @param payload */ -case class Unsubscribe(payload: Seq[Byte]) extends PubSubOption +case class Unsubscribe(payload: Seq[Byte]) extends PubSubOption { + def this(topic: String) = this(topic.getBytes("UTF-8")) +} object Unsubscribe { - def apply(topic: String): Unsubscribe = Unsubscribe(topic.getBytes) + def apply(topic: String): Unsubscribe = new Unsubscribe(topic) } /** @@ -204,7 +209,21 @@ case class Send(frames: Seq[Frame]) extends Request * @param frames */ case class ZMQMessage(frames: Seq[Frame]) { - def firstFrameAsString = new String(frames.head.payload.toArray) + + def this(frame: Frame) = this(Seq(frame)) + def this(frame1: Frame, frame2: Frame) = this(Seq(frame1, frame2)) + def this(frameArray: Array[Frame]) = this(frameArray.toSeq) + + /** + * Convert the bytes in the first frame to a String, using specified charset. + */ + def firstFrameAsString(charsetName: String): String = new String(frames.head.payload.toArray, charsetName) + /** + * Convert the bytes in the first frame to a String, using "UTF-8" charset. + */ + def firstFrameAsString: String = firstFrameAsString("UTF-8") + + def payload(frameIndex: Int): Array[Byte] = frames(frameIndex).payload.toArray } object ZMQMessage { def apply(bytes: Array[Byte]): ZMQMessage = ZMQMessage(Seq(Frame(bytes))) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala index 470c98617a..1776f21211 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala @@ -3,11 +3,18 @@ */ package akka.zeromq +object Frame { + def apply(text: String): Frame = new Frame(text) +} + /** * A single message frame of a zeromq message * @param payload */ -case class Frame(payload: Seq[Byte]) +case class Frame(payload: Seq[Byte]) { + def this(bytes: Array[Byte]) = this(bytes.toSeq) + def this(text: String) = this(text.getBytes("UTF-8")) +} /** * Deserializes ZeroMQ messages into an immutable sequence of frames diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 20dbb0724a..7ae178291f 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -22,6 +22,7 @@ case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { * The [[akka.actor.ExtensionId]] and [[akka.actor.ExtensionIdProvider]] for the ZeroMQ module */ object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProvider { + override def get(system: ActorSystem): ZeroMQExtension = super.get(system) def lookup() = this def createExtension(system: ExtendedActorSystem) = new ZeroMQExtension(system) @@ -141,92 +142,94 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { } /** - * Java API helper - * Factory method to create the actor representing the ZeroMQ Publisher socket. + * Java API factory method to create the actor representing the ZeroMQ Publisher socket. * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter * They are matched on type and the first one found wins. * - * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @param socketParameters array of [[akka.zeromq.SocketOption]] to configure the socket * @return the [[akka.actor.ActorRef]] */ - def newPubSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Pub +: socketParameters): _*) + def newPubSocket(socketParameters: Array[SocketOption]): ActorRef = newSocket((SocketType.Pub +: socketParameters): _*) /** - * Java API helper - * Factory method to create the actor representing the ZeroMQ Subscriber socket. - * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter - * They are matched on type and the first one found wins. - * - * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke - * @return the [[akka.actor.ActorRef]] + * Convenience for creating a publisher socket. */ - def newSubSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Sub +: socketParameters): _*) + def newPubSocket(bind: Bind): ActorRef = newSocket(SocketType.Pub, bind) /** - * Java API helper - * Factory method to create the actor representing the ZeroMQ Dealer socket. + * Java API factory method to create the actor representing the ZeroMQ Subscriber socket. * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter * They are matched on type and the first one found wins. * - * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @param socketParameters array of [[akka.zeromq.SocketOption]] to configure the socket * @return the [[akka.actor.ActorRef]] */ - def newDealerSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Dealer +: socketParameters): _*) + def newSubSocket(socketParameters: Array[SocketOption]): ActorRef = newSocket((SocketType.Sub +: socketParameters): _*) /** - * Java API helper - * Factory method to create the actor representing the ZeroMQ Router socket. - * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter - * They are matched on type and the first one found wins. - * - * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke - * @return the [[akka.actor.ActorRef]] + * Convenience for creating a subscriber socket. */ - def newRouterSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Router +: socketParameters): _*) + def newSubSocket(connect: Connect, listener: Listener, subscribe: Subscribe): ActorRef = newSocket(SocketType.Sub, connect, listener, subscribe) /** - * Java API helper - * Factory method to create the actor representing the ZeroMQ Push socket. + * Java API factory method to create the actor representing the ZeroMQ Dealer socket. * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter * They are matched on type and the first one found wins. * - * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @param socketParameters array of [[akka.zeromq.SocketOption]] to configure the socket * @return the [[akka.actor.ActorRef]] */ - def newPushSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Push +: socketParameters): _*) + def newDealerSocket(socketParameters: Array[SocketOption]): ActorRef = newSocket((SocketType.Dealer +: socketParameters): _*) /** - * Java API helper - * Factory method to create the actor representing the ZeroMQ Pull socket. + * Java API factory method to create the actor representing the ZeroMQ Router socket. * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter * They are matched on type and the first one found wins. * - * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @param socketParameters array of [[akka.zeromq.SocketOption]] to configure the socket * @return the [[akka.actor.ActorRef]] */ - def newPullSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Pull +: socketParameters): _*) + def newRouterSocket(socketParameters: Array[SocketOption]): ActorRef = newSocket((SocketType.Router +: socketParameters): _*) /** - * Java API helper - * Factory method to create the actor representing the ZeroMQ Req socket. + * Java API factory method to create the actor representing the ZeroMQ Push socket. * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter * They are matched on type and the first one found wins. * - * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @param socketParameters array of [[akka.zeromq.SocketOption]] to configure the socket * @return the [[akka.actor.ActorRef]] */ - def newReqSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Req +: socketParameters): _*) + def newPushSocket(socketParameters: Array[SocketOption]): ActorRef = newSocket((SocketType.Push +: socketParameters): _*) /** - * Java API helper - * Factory method to create the actor representing the ZeroMQ Rep socket. + * Java API factory method to create the actor representing the ZeroMQ Pull socket. * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter * They are matched on type and the first one found wins. * - * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @param socketParameters array of [[akka.zeromq.SocketOption]] to configure the socket * @return the [[akka.actor.ActorRef]] */ - def newRepSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Rep +: socketParameters): _*) + def newPullSocket(socketParameters: Array[SocketOption]): ActorRef = newSocket((SocketType.Pull +: socketParameters): _*) + + /** + * Java API factory method to create the actor representing the ZeroMQ Req socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters array of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.ActorRef]] + */ + def newReqSocket(socketParameters: Array[SocketOption]): ActorRef = newSocket((SocketType.Req +: socketParameters): _*) + + /** + * Java API factory method to create the actor representing the ZeroMQ Rep socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters array of [[akka.zeromq.SocketOption]] to configure the socke + * @return the [[akka.actor.ActorRef]] + */ + def newRepSocket(socketParameters: Array[SocketOption]): ActorRef = newSocket((SocketType.Rep +: socketParameters): _*) private val zeromqGuardian: ActorRef = { verifyZeroMQVersion diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index 7c498bd653..fe5b85b9dc 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -24,7 +24,8 @@ class ConcurrentSocketActorSpec "ConcurrentSocketActor" should { "support pub-sub connections" in { checkZeroMQInstallation - val (publisherProbe, subscriberProbe) = (TestProbe(), TestProbe()) + val publisherProbe = TestProbe() + val subscriberProbe = TestProbe() val context = Context() val publisher = newPublisher(context, publisherProbe.ref) val subscriber = newSubscriber(context, subscriberProbe.ref) @@ -68,7 +69,7 @@ class ConcurrentSocketActorSpec zmq.newSocket(SocketType.Pub, context, Listener(listener), Bind(endpoint)) } def newSubscriber(context: Context, listener: ActorRef) = { - zmq.newSocket(SocketType.Sub, context, Listener(listener), Connect(endpoint), Subscribe(Seq.empty)) + zmq.newSocket(SocketType.Sub, context, Listener(listener), Connect(endpoint), SubscribeAll) } def newMessageGenerator(actorRef: ActorRef) = { system.actorOf(Props(new MessageGeneratorActor(actorRef))) @@ -110,7 +111,7 @@ class ConcurrentSocketActorSpec protected def receive = { case _ ⇒ val payload = "%s".format(messageNumber) - messageNumber = messageNumber + 1 + messageNumber += 1 actorRef ! ZMQMessage(payload.getBytes) } } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 306aa38e8f..7ad4be2a9f 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -320,7 +320,8 @@ object AkkaBuild extends Build { lazy val docs = Project( id = "akka-docs", base = file("akka-docs"), - dependencies = Seq(actor, testkit % "test->test", remote, cluster, slf4j, agent, transactor, fileMailbox, mongoMailbox, redisMailbox, beanstalkMailbox, zookeeperMailbox), + dependencies = Seq(actor, testkit % "test->test", remote, cluster, slf4j, agent, transactor, + fileMailbox, mongoMailbox, redisMailbox, beanstalkMailbox, zookeeperMailbox, zeroMQ), settings = defaultSettings ++ Seq( unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get }, libraryDependencies ++= Dependencies.docs, From bbe221e8128dde6ddd981569f13c15f9e1f66ca5 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 9 Feb 2012 22:58:28 +0100 Subject: [PATCH 05/39] =?UTF-8?q?incorporate=20Viktor=E2=80=99s=20feedback?= =?UTF-8?q?=20and=20fix=20some=20stuff?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - initial resize should be done directly - must not require children unconditionally in Router constructor - ResizerSpec changed timing due to asynchronous resizing, one test disabled - removed pointless volatile write in RouterActorRef --- .../test/scala/akka/routing/ResizerSpec.scala | 18 +++-- .../src/main/scala/akka/dispatch/Future.scala | 6 +- .../src/main/scala/akka/routing/Routing.scala | 79 ++++++++----------- 3 files changed, 52 insertions(+), 51 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala index 863922491b..1d271a0959 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala @@ -123,7 +123,15 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with current.routees.size must be(2) } - "resize when busy" in { + /* + * TODO RK This test seems invalid to me, because it relies on that no resize() event is lost; + * this currently fails because I made resize() asynchronous (by sending a message to the + * Router), but it could also fail for concurrent send operations, i.e. when one of thread + * fails the resizeInProgress.compareAndSet(false, true) check. + * + * Either the test must be fixed/removed or resize() must be changed to be blocking. + */ + "resize when busy" ignore { val busy = new TestLatch(1) @@ -179,10 +187,10 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(2) def loop(loops: Int, t: Int, latch: TestLatch, count: AtomicInteger) = { - (10 millis).dilated.sleep + (100 millis).dilated.sleep for (m ← 0 until loops) { router.!((t, latch, count)) - (10 millis).dilated.sleep + (100 millis).dilated.sleep } } @@ -198,7 +206,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with // a whole bunch should max it out val count2 = new AtomicInteger val latch2 = TestLatch(10) - loop(10, 200, latch2, count2) + loop(10, 500, latch2, count2) Await.ready(latch2, TestLatch.DefaultTimeout) count2.get must be(10) @@ -238,7 +246,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with // let it cool down for (m ← 0 to 3) { router ! 1 - (200 millis).dilated.sleep + (500 millis).dilated.sleep } Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be < (z) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 97ff17c075..f995642acd 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -20,6 +20,7 @@ import akka.event.Logging.Debug import java.util.concurrent.TimeUnit.NANOSECONDS import java.util.concurrent.{ ExecutionException, Callable, TimeoutException } import java.util.concurrent.atomic.{ AtomicInteger, AtomicReferenceFieldUpdater } +import akka.pattern.AskTimeoutException object Await { @@ -795,8 +796,9 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac def result(atMost: Duration)(implicit permit: CanAwait): T = ready(atMost).value.get match { - case Left(e) ⇒ throw e - case Right(r) ⇒ r + case Left(e: AskTimeoutException) ⇒ throw new AskTimeoutException(e.getMessage, e) + case Left(e) ⇒ throw e + case Right(r) ⇒ r } def value: Option[Either[Throwable, T]] = getState match { diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index c0bd0df251..badfe9bfcc 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -9,16 +9,13 @@ import akka.util.duration._ import akka.config.ConfigurationException import akka.pattern.pipe import akka.pattern.AskSupport - import com.typesafe.config.Config - import scala.collection.JavaConversions.iterableAsScalaIterable - import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean } import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock - import akka.jsr166y.ThreadLocalRandom +import akka.util.Unsafe /** * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to @@ -40,60 +37,57 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup * synchronized. This is done exactly once at start-up, all other accesses * are done from the Router actor. This means that the only thing which is * really hairy is making sure that the Router does not touch its childrenRefs - * before we are done with them: create a locked latch really early (hence the + * before we are done with them: lock the monitor of the actor cell (hence the * override of newActorCell) and use that to block the Router constructor for * as long as it takes to setup the RoutedActorRef itself. */ - private[akka] var routeReady: ReentrantLock = _ override def newActorCell( system: ActorSystemImpl, ref: InternalActorRef, props: Props, supervisor: InternalActorRef, - receiveTimeout: Option[Duration]): ActorCell = { - /* - * TODO RK: check that this really sticks, since this is executed before - * the constructor of RoutedActorRef is executed (invoked from - * LocalActorRef); works on HotSpot and JRockit. - */ - routeReady = new ReentrantLock - routeReady.lock() - super.newActorCell(system, ref, props, supervisor, receiveTimeout) - } + receiveTimeout: Option[Duration]): ActorCell = + { + val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout) + Unsafe.instance.monitorEnter(cell) + cell + } private[akka] val routerConfig = _props.routerConfig private[akka] val routeeProps = _props.copy(routerConfig = NoRouter) - private[akka] val resizeProgress = new AtomicBoolean + private[akka] val resizeInProgress = new AtomicBoolean private val resizeCounter = new AtomicLong @volatile private var _routees: IndexedSeq[ActorRef] = IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute def routees = _routees - private[akka] var routeeProvider: RouteeProvider = _ + @volatile + private var _routeeProvider: RouteeProvider = _ + def routeeProvider = _routeeProvider + val route = try { - routeeProvider = routerConfig.createRouteeProvider(actorContext) + _routeeProvider = routerConfig.createRouteeProvider(actorContext) val r = routerConfig.createRoute(routeeProps, routeeProvider) // initial resize, before message send - resize() + routerConfig.resizer foreach { r ⇒ + if (r.isTimeForResize(resizeCounter.getAndIncrement())) + r.resize(routeeProps, routeeProvider) + } r - } finally routeReady.unlock() // unblock Router’s constructor + } finally Unsafe.instance.monitorExit(actorContext) // unblock Router’s constructor if (routerConfig.resizer.isEmpty && _routees.isEmpty) throw new ActorInitializationException("router " + routerConfig + " did not register routees!") - _routees match { - case x ⇒ _routees = x // volatile write to publish the route before sending messages - } - /* * end of construction */ def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match { - case _: AutoReceivedMessage ⇒ Nil - case Terminated(_) ⇒ Nil + case _: AutoReceivedMessage ⇒ Destination(this, this) :: Nil + case Terminated(_) ⇒ Destination(this, this) :: Nil case CurrentRoutees ⇒ sender ! RouterRoutees(_routees) Nil @@ -108,7 +102,7 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup * Not thread safe, but intended to be called from protected points, such as * `RouterConfig.createRoute` and `Resizer.resize` */ - private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]) { + private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]): Unit = { _routees = _routees ++ newRoutees // subscribe to Terminated messages for all route destinations, to be handled by Router actor newRoutees foreach underlying.watch @@ -120,7 +114,7 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup * Not thread safe, but intended to be called from protected points, such as * `Resizer.resize` */ - private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]) { + private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]): Unit = { _routees = _routees diff abandonedRoutees abandonedRoutees foreach underlying.unwatch } @@ -136,14 +130,14 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup } applyRoute(s, message) match { - case Nil ⇒ super.!(message)(s) - case refs ⇒ refs foreach (p ⇒ p.recipient.!(msg)(p.sender)) + case Destination(_, x) :: Nil if x eq this ⇒ super.!(message)(s) + case refs ⇒ refs foreach (p ⇒ p.recipient.!(msg)(p.sender)) } } - def resize() { + def resize(): Unit = { for (r ← routerConfig.resizer) { - if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeProgress.compareAndSet(false, true)) + if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeInProgress.compareAndSet(false, true)) super.!(Router.Resize) } } @@ -282,22 +276,19 @@ trait CustomRoute { */ trait Router extends Actor { - val ref = self match { - case x: RoutedActorRef ⇒ x - case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef") - } - // make sure that we synchronize properly to get the childrenRefs into our CPU cache - ref.routeReady.lock() - try if (context.children.isEmpty) - throw new ActorInitializationException("RouterConfig did not create any children") - finally ref.routeReady.unlock() + val ref = context.synchronized { + self match { + case x: RoutedActorRef ⇒ x + case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef") + } + } final def receive = ({ case Router.Resize ⇒ try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider)) - finally ref.resizeProgress.set(false) + finally assert(ref.resizeInProgress.getAndSet(false)) case Terminated(child) ⇒ ref.removeRoutees(IndexedSeq(child)) @@ -310,7 +301,7 @@ trait Router extends Actor { } } -object Router { +private object Router { case object Resize } From d910eeae6945b9af35570ee61e22dd46ab589f6f Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 10 Feb 2012 08:20:36 +0100 Subject: [PATCH 06/39] Removing the erronous execution context and added Java API --- .../test/scala/akka/dispatch/FutureSpec.scala | 2 +- .../src/main/scala/akka/dispatch/Future.scala | 36 ++++++++++++++++--- .../akka/docs/future/FutureDocTestBase.java | 1 + .../scala/akka/remote/netty/Settings.scala | 2 +- 4 files changed, 35 insertions(+), 6 deletions(-) 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 e058218f2d..6163123632 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -860,7 +860,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa val l1, l2 = new TestLatch val complex = Future() map { _ ⇒ - Future.blocking(system.dispatcher) + Future.blocking() val nested = Future(()) nested foreach (_ ⇒ l1.open()) Await.ready(l1, TestLatch.DefaultTimeout) // make sure nested is completed diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 97ff17c075..39e9f27d26 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -151,6 +151,26 @@ object Futures { for (r ← fr; b ← fb) yield { r add b; r } } } + + /** + * Signals that the current thread of execution will potentially engage + * in blocking calls after the call to this method, giving the system a + * chance to spawn new threads, reuse old threads or otherwise, to prevent + * starvation and/or unfairness. + * + * Assures that any Future tasks initiated in the current thread will be + * executed asynchronously, including any tasks currently queued to be + * executed in the current thread. This is needed if the current task may + * block, causing delays in executing the remaining tasks which in some + * cases may cause a deadlock. + * + * Usage: Call this method in a callback (map, flatMap etc also count) to a Future, + * if you will be doing blocking in the callback. + * + * Note: Calling 'Await.result(future)' or 'Await.ready(future)' will automatically trigger this method. + * + */ + def blocking(): Unit = Future.blocking() } object Future { @@ -317,17 +337,22 @@ object Future { * } * */ - def blocking(implicit executor: ExecutionContext): Unit = + def blocking(): Unit = _taskStack.get match { case stack if (stack ne null) && stack.nonEmpty ⇒ + val executionContext = _executionContext.get match { + case null ⇒ throw new IllegalStateException("'blocking' needs to be invoked inside a Future callback.") + case some ⇒ some + } val tasks = stack.elems stack.clear() _taskStack.remove() - dispatchTask(() ⇒ _taskStack.get.elems = tasks, true) + dispatchTask(() ⇒ _taskStack.get.elems = tasks, true)(executionContext) case _ ⇒ _taskStack.remove() } private val _taskStack = new ThreadLocal[Stack[() ⇒ Unit]]() + private val _executionContext = new ThreadLocal[ExecutionContext]() /** * Internal API, do not call @@ -339,7 +364,7 @@ object Future { new Runnable { def run = try { - + _executionContext set executor val taskStack = Stack.empty[() ⇒ Unit] taskStack push task _taskStack set taskStack @@ -352,7 +377,10 @@ object Future { case NonFatal(e) ⇒ executor.reportFailure(e) } } - } finally { _taskStack.remove() } + } finally { + _executionContext.remove() + _taskStack.remove() + } }) } diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index b064eb803b..8fc3b29b4e 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -381,6 +381,7 @@ public class FutureDocTestBase { @Test public void useOnSuccessOnFailureAndOnComplete() { { Future future = Futures.successful("foo", system.dispatcher()); + //#onSuccess future.onSuccess(new OnSuccess() { public void onSuccess(String result) { diff --git a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala index 0db6cabf18..daa91a3014 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala @@ -40,7 +40,7 @@ class NettySettings(config: Config, val systemName: String) { case value ⇒ value } - @deprecated("WARNING: This should only be used by professionals.") + @deprecated("WARNING: This should only be used by professionals.", "2.0") val PortSelector = getInt("port") val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS) From f85645e46dbe3be651e1590afce41c77755c0764 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 10 Feb 2012 10:36:35 +0100 Subject: [PATCH 07/39] Removing the old work redistribution --- .../akka/dispatch/BalancingDispatcher.scala | 22 +------------------ 1 file changed, 1 insertion(+), 21 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 8542ac69c8..a6042046ff 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -36,7 +36,6 @@ class BalancingDispatcher( extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) { val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator) - val rebalance = new AtomicBoolean(false) val messageQueue: MessageQueue = mailboxType match { case u: UnboundedMailbox ⇒ new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { @@ -84,30 +83,11 @@ class BalancingDispatcher( protected[akka] override def unregister(actor: ActorCell) = { buddies.remove(actor) super.unregister(actor) - intoTheFray(except = actor) //When someone leaves, he tosses a friend into the fray } - def intoTheFray(except: ActorCell): Unit = - if (rebalance.compareAndSet(false, true)) { - try { - val i = buddies.iterator() - - @tailrec - def throwIn(): Unit = { - val n = if (i.hasNext) i.next() else null - if (n eq null) () - else if ((n ne except) && registerForExecution(n.mailbox, false, false)) () - else throwIn() - } - throwIn() - } finally { - rebalance.set(false) - } - } - override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { messageQueue.enqueue(receiver.self, invocation) registerForExecution(receiver.mailbox, false, false) - intoTheFray(except = receiver) + //Somewhere around here we have to make sure that not only the intended actor is kept busy } } From 5b50ca96f2b3e4fc32f077e730e050f99cddfe79 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 10 Feb 2012 10:40:24 +0100 Subject: [PATCH 08/39] Minor improvement. See #1713 --- .../code/akka/docs/zeromq/ZeromqDocTestBase.java | 4 +++- .../scala/code/akka/docs/zeromq/ZeromqDocSpec.scala | 12 ++++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java b/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java index fc40e3af67..ee8252a6ad 100644 --- a/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java +++ b/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java @@ -128,6 +128,8 @@ public class ZeromqDocTestBase { system.actorOf(new Props(HeapAlerter.class), "alerter"); //#alerter2 + // Let it run for a while to see some output. + // Don't do like this in real tests, this is only doc demonstration. Thread.sleep(3000L); } @@ -190,7 +192,7 @@ public class ZeromqDocTestBase { @Override public void postRestart(Throwable reason) { - // don't call preStart + // don't call preStart, only schedule once } @Override diff --git a/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala b/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala index 796c95b27d..1b3970d70b 100644 --- a/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala @@ -35,7 +35,13 @@ object ZeromqDocSpec { val os = ManagementFactory.getOperatingSystemMXBean val ser = SerializationExtension(context.system) - context.system.scheduler.schedule(1 second, 1 second, self, Tick) + override def preStart() { + context.system.scheduler.schedule(1 second, 1 second, self, Tick) + } + + override def postRestart(reason: Throwable) { + // don't call preStart, only schedule once + } def receive: Receive = { case Tick ⇒ @@ -172,7 +178,9 @@ class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") { system.actorOf(Props[HeapAlerter], name = "alerter") //#alerter - Thread.sleep(3000) + // Let it run for a while to see some output. + // Don't do like this in real tests, this is only doc demonstration. + 3.seconds.sleep() } From dca309c535b2cdec96190b3504e989e2e60174da Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 10 Feb 2012 11:36:23 +0100 Subject: [PATCH 09/39] =?UTF-8?q?incorporate=20Patrik=E2=80=99s=20feedback?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - rename to DynamicAccess - rename to createInstanceFor / createClassFor - fix a few little things --- .../test/java/akka/actor/JavaExtension.java | 4 +-- .../src/main/scala/akka/actor/ActorRef.scala | 4 +-- .../scala/akka/actor/ActorRefProvider.scala | 4 +-- .../main/scala/akka/actor/ActorSystem.scala | 16 +++++----- .../src/main/scala/akka/actor/Deployer.scala | 4 +-- ...opertyMaster.scala => DynamicAccess.scala} | 31 +++++++++---------- .../src/main/scala/akka/actor/Extension.scala | 2 +- .../akka/dispatch/AbstractDispatcher.scala | 4 +-- .../scala/akka/dispatch/Dispatchers.scala | 8 ++--- .../src/main/scala/akka/event/Logging.scala | 2 +- .../akka/serialization/Serialization.scala | 8 ++--- .../scala/akka/serialization/Serializer.scala | 10 +++--- .../SerializationDocTestBase.java | 1 - .../serialization/SerializationDocSpec.scala | 2 +- .../scala/akka/remote/MessageSerializer.scala | 2 +- .../akka/remote/RemoteActorRefProvider.scala | 6 ++-- .../scala/akka/remote/RemoteDeployer.scala | 2 +- .../serialization/ProtobufSerializer.scala | 2 +- .../scala/akka/testkit/TestActorRef.scala | 2 +- 19 files changed, 56 insertions(+), 58 deletions(-) rename akka-actor/src/main/scala/akka/actor/{PropertyMaster.scala => DynamicAccess.scala} (76%) diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java index f8f22ec8c5..616675eb77 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java @@ -42,8 +42,8 @@ public class JavaExtension { public final ExtendedActorSystem system; - public OtherExtension(ExtendedActorSystem i) { - system = i; + public OtherExtension(ExtendedActorSystem system) { + this.system = system; } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 90154a829b..6cdbbb7667 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -8,7 +8,7 @@ import akka.dispatch._ import akka.util._ import scala.collection.immutable.Stack import java.lang.{ UnsupportedOperationException, IllegalStateException } -import akka.serialization.Serialization +import akka.serialization.{ Serialization, JavaSerializer } import akka.event.EventStream import scala.annotation.tailrec import java.util.concurrent.{ ConcurrentHashMap } @@ -399,7 +399,7 @@ case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) private[akka] object DeadLetterActorRef { class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance? @throws(classOf[java.io.ObjectStreamException]) - private def readResolve(): AnyRef = akka.serialization.JavaSerializer.currentSystem.value.deadLetters + private def readResolve(): AnyRef = JavaSerializer.currentSystem.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 98408dca65..a176c6271d 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -318,12 +318,12 @@ class LocalActorRefProvider( settings: ActorSystem.Settings, eventStream: EventStream, scheduler: Scheduler, - propertyMaster: PropertyMaster) = + dynamicAccess: DynamicAccess) = this(_systemName, settings, eventStream, scheduler, - new Deployer(settings, propertyMaster)) + new Deployer(settings, dynamicAccess)) val rootPath: ActorPath = RootActorPath(Address("akka", _systemName)) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 33e986d025..9ba79d4e51 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -330,7 +330,7 @@ abstract class ExtendedActorSystem extends ActorSystem { * set on all threads created by the ActorSystem, if one was set during * creation. */ - def propertyMaster: PropertyMaster + def dynamicAccess: DynamicAccess } class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Config) extends ExtendedActorSystem { @@ -360,7 +360,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf * This is an extension point: by overriding this method, subclasses can * control all reflection activities of an actor system. */ - protected def createPropertyMaster(): PropertyMaster = new ReflectivePropertyMaster(findClassLoader) + protected def createDynamicAccess(): DynamicAccess = new ReflectiveDynamicAccess(findClassLoader) protected def findClassLoader: ClassLoader = { def findCaller(get: Int ⇒ Class[_]): ClassLoader = { @@ -382,8 +382,8 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf getClass.getClassLoader } - private val _pm: PropertyMaster = createPropertyMaster() - def propertyMaster: PropertyMaster = _pm + private val _pm: DynamicAccess = createDynamicAccess() + def dynamicAccess: DynamicAccess = _pm def logConfiguration(): Unit = log.info(settings.toString) @@ -441,9 +441,9 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf classOf[Settings] -> settings, classOf[EventStream] -> eventStream, classOf[Scheduler] -> scheduler, - classOf[PropertyMaster] -> propertyMaster) + classOf[DynamicAccess] -> dynamicAccess) - propertyMaster.getInstanceFor[ActorRefProvider](ProviderClass, arguments) match { + dynamicAccess.createInstanceFor[ActorRefProvider](ProviderClass, arguments) match { case Left(e) ⇒ throw e case Right(p) ⇒ p } @@ -465,7 +465,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf def locker: Locker = provider.locker val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites( - threadFactory, eventStream, deadLetterMailbox, scheduler, propertyMaster)) + threadFactory, eventStream, deadLetterMailbox, scheduler, dynamicAccess)) val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher @@ -584,7 +584,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf private def loadExtensions() { import scala.collection.JavaConversions._ settings.config.getStringList("akka.extensions") foreach { fqcn ⇒ - propertyMaster.getObjectFor[AnyRef](fqcn).fold(_ ⇒ propertyMaster.getInstanceFor[AnyRef](fqcn, Seq()), Right(_)) match { + dynamicAccess.getObjectFor[AnyRef](fqcn).fold(_ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Seq()), Right(_)) match { case Right(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup()); case Right(p: ExtensionId[_]) ⇒ registerExtension(p); case Right(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index a26ce419d5..374812e50d 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -82,7 +82,7 @@ case object NoScopeGiven extends Scope { * * @author Jonas Bonér */ -class Deployer(val settings: ActorSystem.Settings, val propertyMaster: PropertyMaster) { +class Deployer(val settings: ActorSystem.Settings, val dynamicAccess: DynamicAccess) { import scala.collection.JavaConverters._ @@ -124,7 +124,7 @@ class Deployer(val settings: ActorSystem.Settings, val propertyMaster: PropertyM case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer) case fqn ⇒ val args = Seq(classOf[Config] -> deployment) - propertyMaster.getInstanceFor[RouterConfig](fqn, args) match { + dynamicAccess.createInstanceFor[RouterConfig](fqn, args) match { case Right(router) ⇒ router case Left(exception) ⇒ throw new IllegalArgumentException( diff --git a/akka-actor/src/main/scala/akka/actor/PropertyMaster.scala b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala similarity index 76% rename from akka-actor/src/main/scala/akka/actor/PropertyMaster.scala rename to akka-actor/src/main/scala/akka/actor/DynamicAccess.scala index 05e4a53362..4c641e44b2 100644 --- a/akka-actor/src/main/scala/akka/actor/PropertyMaster.scala +++ b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala @@ -7,21 +7,20 @@ import akka.util.NonFatal import java.lang.reflect.InvocationTargetException /** - * The property master is responsible for acquiring all props needed for a - * performance; in Akka this is the class which is used for + * The DynamicAccess implementation is the class which is used for * loading all configurable parts of an actor system (the - * [[akka.actor.ReflectivePropertyMaster]] is the default implementation). + * [[akka.actor.ReflectiveDynamicAccess]] is the default implementation). * * This is an internal facility and users are not expected to encounter it * unless they are extending Akka in ways which go beyond simple Extensions. */ -private[akka] trait PropertyMaster { +trait DynamicAccess { /** * Obtain a `Class[_]` object loaded with the right class loader (i.e. the one * returned by `classLoader`). */ - def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] + def createClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] /** * Obtain an object conforming to the type T, which is expected to be @@ -30,7 +29,7 @@ private[akka] trait PropertyMaster { * `args` argument. The exact usage of args depends on which type is requested, * see the relevant requesting code for details. */ - def getInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] + def createInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] /** * Obtain the Scala “object” instance for the given fully-qualified class name, if there is one. @@ -45,17 +44,17 @@ private[akka] trait PropertyMaster { } -object PropertyMaster { +object DynamicAccess { /** * Convenience method which given a `Class[_]` object and a constructor description * will create a new instance of that class. * * {{{ - * val obj = PropertyMaster.getInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name)) + * val obj = DynamicAccess.createInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name)) * }}} */ - def getInstanceFor[T: ClassManifest](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = { + def createInstanceFor[T: ClassManifest](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = { val types = args.map(_._1).toArray val values = args.map(_._2).toArray withErrorHandling { @@ -86,17 +85,17 @@ object PropertyMaster { } /** - * This is the default [[akka.actor.PropertyMaster]] implementation used by [[akka.actor.ActorSystemImpl]] + * This is the default [[akka.actor.DynamicAccess]] implementation used by [[akka.actor.ActorSystemImpl]] * unless overridden. It uses reflection to turn fully-qualified class names into `Class[_]` objects * and creates instances from there using `getDeclaredConstructor()` and invoking that. The class loader * to be used for all this is determined by the [[akka.actor.ActorSystemImpl]]’s `findClassLoader` method * by default. */ -class ReflectivePropertyMaster(val classLoader: ClassLoader) extends PropertyMaster { +class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAccess { - import PropertyMaster.withErrorHandling + import DynamicAccess.withErrorHandling - override def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] = + override def createClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] = try { val c = classLoader.loadClass(fqcn).asInstanceOf[Class[_ <: T]] val t = classManifest[T].erasure @@ -105,8 +104,8 @@ class ReflectivePropertyMaster(val classLoader: ClassLoader) extends PropertyMas case NonFatal(e) ⇒ Left(e) } - override def getInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = - getClassFor(fqcn).fold(Left(_), { c ⇒ + override def createInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = + createClassFor(fqcn).fold(Left(_), { c ⇒ val types = args.map(_._1).toArray val values = args.map(_._2).toArray withErrorHandling { @@ -119,7 +118,7 @@ class ReflectivePropertyMaster(val classLoader: ClassLoader) extends PropertyMas }) override def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T] = { - getClassFor(fqcn).fold(Left(_), { c ⇒ + createClassFor(fqcn).fold(Left(_), { c ⇒ withErrorHandling { val module = c.getDeclaredField("MODULE$") module.setAccessible(true) diff --git a/akka-actor/src/main/scala/akka/actor/Extension.scala b/akka-actor/src/main/scala/akka/actor/Extension.scala index 0b0088c78c..f5c0e68a5a 100644 --- a/akka-actor/src/main/scala/akka/actor/Extension.scala +++ b/akka-actor/src/main/scala/akka/actor/Extension.scala @@ -97,7 +97,7 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassManifest[T]) extend override def lookup(): ExtensionId[T] = this def createExtension(system: ExtendedActorSystem): T = - PropertyMaster.getInstanceFor[T](m.erasure, Seq(classOf[ExtendedActorSystem] -> system)) match { + DynamicAccess.createInstanceFor[T](m.erasure, Seq(classOf[ExtendedActorSystem] -> system)) match { case Left(ex) ⇒ throw ex case Right(r) ⇒ r } diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index db7774eeb7..f2602e66fc 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -368,7 +368,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit } case fqcn ⇒ val args = Seq(classOf[Config] -> config) - prerequisites.propertyMaster.getInstanceFor[MailboxType](fqcn, args) match { + prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args) match { case Right(instance) ⇒ instance case Left(exception) ⇒ throw new IllegalArgumentException( @@ -387,7 +387,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit val args = Seq( classOf[Config] -> config, classOf[DispatcherPrerequisites] -> prerequisites) - prerequisites.propertyMaster.getInstanceFor[ExecutorServiceConfigurator](fqcn, args) match { + prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args) match { case Right(instance) ⇒ instance case Left(exception) ⇒ throw new IllegalArgumentException( ("""Cannot instantiate ExecutorServiceConfigurator ("executor = [%s]"), defined in [%s], diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index 8529c8f733..8e99e05b06 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -11,7 +11,7 @@ import scala.collection.JavaConverters.mapAsJavaMapConverter import com.typesafe.config.{ ConfigFactory, Config } import Dispatchers.DefaultDispatcherId -import akka.actor.{ Scheduler, PropertyMaster, ActorSystem } +import akka.actor.{ Scheduler, DynamicAccess, ActorSystem } import akka.event.Logging.Warning import akka.event.EventStream import akka.util.Duration @@ -21,7 +21,7 @@ trait DispatcherPrerequisites { def eventStream: EventStream def deadLetterMailbox: Mailbox def scheduler: Scheduler - def propertyMaster: PropertyMaster + def dynamicAccess: DynamicAccess } case class DefaultDispatcherPrerequisites( @@ -29,7 +29,7 @@ case class DefaultDispatcherPrerequisites( val eventStream: EventStream, val deadLetterMailbox: Mailbox, val scheduler: Scheduler, - val propertyMaster: PropertyMaster) extends DispatcherPrerequisites + val dynamicAccess: DynamicAccess) extends DispatcherPrerequisites object Dispatchers { /** @@ -139,7 +139,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites) case fqn ⇒ val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites) - prerequisites.propertyMaster.getInstanceFor[MessageDispatcherConfigurator](fqn, args) match { + prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args) match { case Right(configurator) ⇒ configurator case Left(exception) ⇒ throw new IllegalArgumentException( diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index c8dcedc3d5..0000f0ff80 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -100,7 +100,7 @@ trait LoggingBus extends ActorEventBus { if loggerName != StandardOutLoggerName } yield { try { - system.propertyMaster.getClassFor[Actor](loggerName) match { + system.dynamicAccess.createClassFor[Actor](loggerName) match { case Right(actorClass) ⇒ addLogger(system, actorClass, level, logName) case Left(exception) ⇒ throw exception } diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 9cbb15a676..1ef4e9c150 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -128,12 +128,12 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { /** * Tries to load the specified Serializer by the fully-qualified name; the actual - * loading is performed by the system’s [[akka.actor.PropertyMaster]]. + * loading is performed by the system’s [[akka.actor.DynamicAccess]]. */ def serializerOf(serializerFQN: String): Either[Throwable, Serializer] = { - val pm = system.propertyMaster - pm.getInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)) - .fold(_ ⇒ pm.getInstanceFor[Serializer](serializerFQN, Seq()), Right(_)) + val dynamicAccess = system.dynamicAccess + dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)) + .fold(_ ⇒ dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq()), Right(_)) } /** diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index 66dfd560c2..e8fc19eea7 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -6,7 +6,7 @@ package akka.serialization import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream } import akka.util.ClassLoaderObjectInputStream -import akka.actor.PropertyMaster +import akka.actor.DynamicAccess import akka.actor.ExtendedActorSystem import scala.util.DynamicVariable @@ -19,8 +19,8 @@ import scala.util.DynamicVariable *
    *
  • taking exactly one argument of type [[akka.actor.ExtendedActorSystem]]; * this should be the preferred one because all reflective loading of classes - * during deserialization should use ExtendedActorSystem.propertyMaster (see - * [[akka.actor.PropertyMaster]]), and
  • + * during deserialization should use ExtendedActorSystem.dynamicAccess (see + * [[akka.actor.DynamicAccess]]), and *
  • without arguments, which is only an option if the serializer does not * load classes using reflection.
  • *
@@ -49,7 +49,7 @@ trait Serializer extends scala.Serializable { /** * Produces an object from an array of bytes, with an optional type-hint; - * the class should be loaded using ActorSystem.propertyMaster. + * the class should be loaded using ActorSystem.dynamicAccess. */ def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef } @@ -108,7 +108,7 @@ class JavaSerializer(val system: ExtendedActorSystem) extends Serializer { } def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = { - val in = new ClassLoaderObjectInputStream(system.propertyMaster.classLoader, new ByteArrayInputStream(bytes)) + val in = new ClassLoaderObjectInputStream(system.dynamicAccess.classLoader, new ByteArrayInputStream(bytes)) val obj = JavaSerializer.currentSystem.withValue(system) { in.readObject } diff --git a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java index 22a9f571d2..6b6a8ae632 100644 --- a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java +++ b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java @@ -14,7 +14,6 @@ import static org.junit.Assert.*; import akka.serialization.*; import akka.actor.ActorSystem; -import akka.actor.PropertyMaster; import com.typesafe.config.*; //#imports diff --git a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala b/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala index ae87e0d5ab..bee9cedebd 100644 --- a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala @@ -6,7 +6,7 @@ package akka.docs.serialization import org.scalatest.matchers.MustMatchers import akka.testkit._ //#imports -import akka.actor.{ ActorSystem, PropertyMaster } +import akka.actor.ActorSystem import akka.serialization._ import com.typesafe.config.ConfigFactory diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index ddb97d02c4..398c397de7 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -14,7 +14,7 @@ object MessageSerializer { def deserialize(system: ExtendedActorSystem, messageProtocol: MessageProtocol): AnyRef = { val clazz = if (messageProtocol.hasMessageManifest) { - system.propertyMaster.getClassFor[AnyRef](messageProtocol.getMessageManifest.toStringUtf8) + system.dynamicAccess.createClassFor[AnyRef](messageProtocol.getMessageManifest.toStringUtf8) .fold(throw _, Some(_)) } else None SerializationExtension(system) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index e62975ea9f..89c35115ef 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -27,11 +27,11 @@ class RemoteActorRefProvider( val settings: ActorSystem.Settings, val eventStream: EventStream, val scheduler: Scheduler, - val propertyMaster: PropertyMaster) extends ActorRefProvider { + val dynamicAccess: DynamicAccess) extends ActorRefProvider { val remoteSettings = new RemoteSettings(settings.config, systemName) - val deployer = new RemoteDeployer(settings, propertyMaster) + val deployer = new RemoteDeployer(settings, dynamicAccess) private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer) @@ -83,7 +83,7 @@ class RemoteActorRefProvider( classOf[ActorSystemImpl] -> system, classOf[RemoteActorRefProvider] -> this) - system.propertyMaster.getInstanceFor[RemoteTransport](fqn, args) match { + system.dynamicAccess.createInstanceFor[RemoteTransport](fqn, args) match { case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem) case Right(remote) ⇒ remote } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala index ec6ab165a3..1e1068cd5f 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala @@ -12,7 +12,7 @@ case class RemoteScope(node: Address) extends Scope { def withFallback(other: Scope): Scope = this } -class RemoteDeployer(_settings: ActorSystem.Settings, _pm: PropertyMaster) extends Deployer(_settings, _pm) { +class RemoteDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extends Deployer(_settings, _pm) { override protected def parseConfig(path: String, config: Config): Option[Deploy] = { import scala.collection.JavaConverters._ diff --git a/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala b/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala index c970a4b733..813a22fba4 100644 --- a/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala +++ b/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala @@ -5,7 +5,7 @@ package akka.serialization import com.google.protobuf.Message -import akka.actor.PropertyMaster +import akka.actor.DynamicAccess /** * This Serializer serializes `com.google.protobuf.Message`s diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 21b1221b45..01405d2acf 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -121,7 +121,7 @@ object TestActorRef { 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], system: ActorSystem): TestActorRef[T] = apply[T](Props({ - PropertyMaster.getInstanceFor[T](m.erasure, Seq()) match { + DynamicAccess.createInstanceFor[T](m.erasure, Seq()) match { case Right(value) ⇒ value case Left(exception) ⇒ throw new ActorInitializationException(null, "Could not instantiate Actor" + From 09897459d6dfd5c478ab038567ba7b7b5f53b0b4 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 10 Feb 2012 12:45:22 +0100 Subject: [PATCH 10/39] fix up Java API for serializers (both use and implement) --- .../akka/serialization/Serialization.scala | 6 ++++-- .../scala/akka/serialization/Serializer.scala | 19 ++++++++++++++----- .../SerializationDocTestBase.java | 7 +++---- .../code/akka/docs/zeromq/ZeromqDocSpec.scala | 6 +++--- 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index b57ba190f3..2b30307448 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -7,7 +7,9 @@ package akka.serialization import akka.AkkaException import scala.util.DynamicVariable import com.typesafe.config.Config -import akka.actor.{ Extension, ExtendedActorSystem, Address } +import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess } +import akka.event.Logging +import java.util.concurrent.ConcurrentHashMap import akka.util.NonFatal import scala.collection.mutable.ArrayBuffer import java.io.NotSerializableException @@ -151,7 +153,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { */ private[akka] val bindings: Seq[ClassSerializer] = { val configuredBindings = for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield { - val c = ReflectiveAccess.getClassFor(k, system.internalClassLoader).fold(throw _, identity[Class[_]]) + val c = system.dynamicAccess.createClassFor(k).fold(throw _, identity[Class[_]]) (c, serializers(v)) } sort(configuredBindings) diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index fc2145bff3..2721432460 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -52,6 +52,16 @@ trait Serializer { * the class should be loaded using ActorSystem.dynamicAccess. */ def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef + + /** + * Java API: deserialize without type hint + */ + final def fromBinary(bytes: Array[Byte]): AnyRef = fromBinary(bytes, None) + + /** + * Java API: deserialize with type hint + */ + final def fromBinary(bytes: Array[Byte], clazz: Class[_]): AnyRef = fromBinary(bytes, Option(clazz)) } /** @@ -61,14 +71,13 @@ trait Serializer { * the JSerializer (also possible with empty constructor). */ abstract class JSerializer extends Serializer { - def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = - fromBinary(bytes, manifest.orNull) + final def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = + fromBinaryJava(bytes, manifest.orNull) /** - * This method should be overridden, - * manifest and classLoader may be null. + * This method must be implemented, manifest may be null. */ - def fromBinary(bytes: Array[Byte], manifest: Class[_]): AnyRef + protected def fromBinaryJava(bytes: Array[Byte], manifest: Class[_]): AnyRef } object NullSerializer extends NullSerializer diff --git a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java index d4119b0d98..fa0f5fcce3 100644 --- a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java +++ b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java @@ -44,9 +44,8 @@ public class SerializationDocTestBase { // "fromBinary" deserializes the given array, // using the type hint (if any, see "includeManifest" above) - // into the optionally provided classLoader. - @Override public Object fromBinary(byte[] bytes, - Class clazz) { + @Override public Object fromBinaryJava(byte[] bytes, + Class clazz) { // Put your code that deserializes here //#... return null; @@ -74,7 +73,7 @@ public class SerializationDocTestBase { // Turn it back into an object, // the nulls are for the class manifest and for the classloader - String back = (String)serializer.fromBinary(bytes, Option.>none().asScala()); + String back = (String)serializer.fromBinary(bytes); // Voilá! assertEquals(original, back); diff --git a/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala b/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala index 1b3970d70b..e5e3325dd5 100644 --- a/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala @@ -71,14 +71,14 @@ object ZeromqDocSpec { def receive = { // the first frame is the topic, second is the message case m: ZMQMessage if m.firstFrameAsString == "health.heap" ⇒ - ser.deserialize(m.payload(1), classOf[Heap], None) match { + ser.deserialize(m.payload(1), classOf[Heap]) match { case Right(Heap(timestamp, used, max)) ⇒ log.info("Used heap {} bytes, at {}", used, timestampFormat.format(new Date(timestamp))) case Left(e) ⇒ throw e } case m: ZMQMessage if m.firstFrameAsString == "health.load" ⇒ - ser.deserialize(m.payload(1), classOf[Load], None) match { + ser.deserialize(m.payload(1), classOf[Load]) match { case Right(Load(timestamp, loadAverage)) ⇒ log.info("Load average {}, at {}", loadAverage, timestampFormat.format(new Date(timestamp))) case Left(e) ⇒ throw e @@ -97,7 +97,7 @@ object ZeromqDocSpec { def receive = { // the first frame is the topic, second is the message case m: ZMQMessage if m.firstFrameAsString == "health.heap" ⇒ - ser.deserialize(m.payload(1), classOf[Heap], None) match { + ser.deserialize(m.payload(1), classOf[Heap]) match { case Right(Heap(timestamp, used, max)) ⇒ if ((used.toDouble / max) > 0.9) count += 1 else count = 0 From 3e917e1c57a7ed96514c973ddfd6b45ae445ea60 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 10 Feb 2012 13:38:37 +0100 Subject: [PATCH 11/39] Removed host info from AkkaException. See #1784 --- akka-actor/src/main/scala/akka/AkkaException.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/akka-actor/src/main/scala/akka/AkkaException.scala b/akka-actor/src/main/scala/akka/AkkaException.scala index 2abd10c270..85de2504d3 100644 --- a/akka-actor/src/main/scala/akka/AkkaException.scala +++ b/akka-actor/src/main/scala/akka/AkkaException.scala @@ -5,10 +5,8 @@ package akka import akka.actor.newUuid -import java.net.{ InetAddress, UnknownHostException } object AkkaException { - val hostname = try InetAddress.getLocalHost.getHostAddress catch { case e: UnknownHostException ⇒ "unknown host" } def toStringWithStackTrace(throwable: Throwable): String = throwable match { case null ⇒ "Unknown Throwable: was 'null'" @@ -36,7 +34,7 @@ object AkkaException { */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable { - val uuid = "%s_%s".format(AkkaException.hostname, newUuid) + lazy val uuid = newUuid.toString override lazy val toString = "%s:%s\n[%s]".format(getClass.getName, message, uuid) From 36247b10fe79c8aa6ec878eaf4946c2cd1b65084 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 10 Feb 2012 14:13:40 +0100 Subject: [PATCH 12/39] fix some more comments and make Router dispatcher configurable MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - assert locking balance when using Unsafe.instance.monitorExit - add RouterConfig.routerDispatcher - re-enable “busy” resizer test after switching to BalancingDispatcher - document resizer asynchronicity and how to configure dispatchers --- .../test/scala/akka/routing/ResizerSpec.scala | 15 ++--- .../test/scala/akka/routing/RoutingSpec.scala | 4 ++ .../src/main/scala/akka/dispatch/Future.scala | 2 +- .../src/main/scala/akka/routing/Routing.scala | 59 ++++++++++++++++--- .../jrouting/CustomRouterDocTestBase.java | 18 ++++++ akka-docs/java/routing.rst | 35 +++++++++-- .../akka/docs/routing/RouterDocSpec.scala | 29 +++++++++ akka-docs/scala/routing.rst | 35 +++++++++-- .../akka/routing/RemoteRouterConfig.scala | 2 + 9 files changed, 170 insertions(+), 29 deletions(-) create mode 100644 akka-docs/scala/code/akka/docs/routing/RouterDocSpec.scala diff --git a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala index 1d271a0959..2130afe107 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala @@ -24,6 +24,9 @@ object ResizerSpec { } } } + bal-disp { + type = BalancingDispatcher + } """ class TestActor extends Actor { @@ -123,15 +126,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with current.routees.size must be(2) } - /* - * TODO RK This test seems invalid to me, because it relies on that no resize() event is lost; - * this currently fails because I made resize() asynchronous (by sending a message to the - * Router), but it could also fail for concurrent send operations, i.e. when one of thread - * fails the resizeInProgress.compareAndSet(false, true) check. - * - * Either the test must be fixed/removed or resize() must be changed to be blocking. - */ - "resize when busy" ignore { + "resize when busy" in { val busy = new TestLatch(1) @@ -141,7 +136,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with pressureThreshold = 0, messagesPerResize = 1) - val router = system.actorOf(Props[BusyActor].withRouter(RoundRobinRouter(resizer = Some(resizer)))) + val router = system.actorOf(Props[BusyActor].withRouter(RoundRobinRouter(resizer = Some(resizer))).withDispatcher("bal-disp")) val latch1 = new TestLatch(1) router ! (latch1, busy) 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 9de51bdabf..ad3702d556 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -15,6 +15,7 @@ import com.typesafe.config.ConfigFactory import akka.pattern.ask import java.util.concurrent.ConcurrentHashMap import com.typesafe.config.Config +import akka.dispatch.Dispatchers object RoutingSpec { @@ -51,6 +52,7 @@ object RoutingSpec { case (sender, message) ⇒ Nil } } + def routerDispatcher: String = Dispatchers.DefaultDispatcherId } } @@ -539,6 +541,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with //#crRouter case class VoteCountRouter() extends RouterConfig { + def routerDispatcher: String = Dispatchers.DefaultDispatcherId + //#crRoute def createRoute(routeeProps: Props, routeeProvider: RouteeProvider): Route = { val democratActor = routeeProvider.context.actorOf(Props(new DemocratActor()), "d") diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index f995642acd..8be901fb03 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -796,7 +796,7 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac def result(atMost: Duration)(implicit permit: CanAwait): T = ready(atMost).value.get match { - case Left(e: AskTimeoutException) ⇒ throw new AskTimeoutException(e.getMessage, e) + case Left(e: AskTimeoutException) ⇒ throw new AskTimeoutException(e.getMessage, e) // to get meaningful stack trace case Left(e) ⇒ throw e case Right(r) ⇒ r } diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index badfe9bfcc..f83bca2db9 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -16,6 +16,7 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import akka.jsr166y.ThreadLocalRandom import akka.util.Unsafe +import akka.dispatch.Dispatchers /** * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to @@ -24,7 +25,7 @@ import akka.util.Unsafe private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath) extends LocalActorRef( _system, - _props.copy(creator = () ⇒ _props.routerConfig.createActor()), + _props.copy(creator = () ⇒ _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher), _supervisor, _path) { @@ -76,7 +77,10 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup r.resize(routeeProps, routeeProvider) } r - } finally Unsafe.instance.monitorExit(actorContext) // unblock Router’s constructor + } finally { + assert(Thread.holdsLock(actorContext)) + Unsafe.instance.monitorExit(actorContext) // unblock Router’s constructor + } if (routerConfig.resizer.isEmpty && _routees.isEmpty) throw new ActorInitializationException("router " + routerConfig + " did not register routees!") @@ -169,6 +173,11 @@ trait RouterConfig { def createActor(): Router = new Router {} + /** + * Dispatcher ID to use for running the “head” actor, i.e. the [[akka.routing.Router]]. + */ + def routerDispatcher: String + /** * Overridable merge strategy, by default completely prefers “this” (i.e. no merge). */ @@ -343,6 +352,7 @@ case class Destination(sender: ActorRef, recipient: ActorRef) //TODO add @SerialVersionUID(1L) when SI-4804 is fixed case object NoRouter extends RouterConfig { def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null + def routerDispatcher: String = "" override def withFallback(other: RouterConfig): RouterConfig = other } @@ -352,13 +362,17 @@ case object NoRouter extends RouterConfig { case object FromConfig extends RouterConfig { def createRoute(props: Props, routeeProvider: RouteeProvider): Route = throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)") + def routerDispatcher: String = Dispatchers.DefaultDispatcherId } /** * Java API: Router configuration which has no default, i.e. external configuration is required. + * + * This can be used when the dispatcher to be used for the head Router needs to be configured + * (defaults to default-dispatcher). */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed -case class FromConfig() extends RouterConfig { +case class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig { def createRoute(props: Props, routeeProvider: RouteeProvider): Route = throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)") } @@ -389,7 +403,8 @@ object RoundRobinRouter { * using `actorFor` in [[akka.actor.ActorRefProvider]] */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed -case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None) +case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, + val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig with RoundRobinLike { /** @@ -415,6 +430,11 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = * Java API */ def this(resizer: Resizer) = this(resizer = Some(resizer)) + + /** + * Java API for setting routerDispatcher + */ + def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) } trait RoundRobinLike { this: RouterConfig ⇒ @@ -469,7 +489,8 @@ object RandomRouter { * using `actorFor` in [[akka.actor.ActorRefProvider]] */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed -case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None) +case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, + val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig with RandomLike { /** @@ -495,6 +516,11 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, * Java API */ def this(resizer: Resizer) = this(resizer = Some(resizer)) + + /** + * Java API for setting routerDispatcher + */ + def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) } trait RandomLike { this: RouterConfig ⇒ @@ -555,7 +581,8 @@ object SmallestMailboxRouter { * using `actorFor` in [[akka.actor.ActorRefProvider]] */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed -case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None) +case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, + val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig with SmallestMailboxLike { /** @@ -581,6 +608,11 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin * Java API */ def this(resizer: Resizer) = this(resizer = Some(resizer)) + + /** + * Java API for setting routerDispatcher + */ + def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) } trait SmallestMailboxLike { this: RouterConfig ⇒ @@ -700,7 +732,8 @@ object BroadcastRouter { * using `actorFor` in [[akka.actor.ActorRefProvider]] */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed -case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None) +case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, + val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig with BroadcastLike { /** @@ -727,6 +760,10 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N */ def this(resizer: Resizer) = this(resizer = Some(resizer)) + /** + * Java API for setting routerDispatcher + */ + def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) } trait BroadcastLike { this: RouterConfig ⇒ @@ -773,7 +810,8 @@ object ScatterGatherFirstCompletedRouter { */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: Duration, - override val resizer: Option[Resizer] = None) + override val resizer: Option[Resizer] = None, + val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig with ScatterGatherFirstCompletedLike { if (within <= Duration.Zero) throw new IllegalArgumentException( @@ -802,6 +840,11 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It * Java API */ def this(resizer: Resizer, w: Duration) = this(resizer = Some(resizer), within = w) + + /** + * Java API for setting routerDispatcher + */ + def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) } trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒ diff --git a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java b/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java index 2a2e5c7f22..a20a351f06 100644 --- a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java +++ b/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java @@ -17,6 +17,7 @@ import akka.util.Duration; import akka.util.Timeout; import akka.dispatch.Await; import akka.dispatch.Future; +import akka.dispatch.Dispatchers; import akka.testkit.AkkaSpec; import com.typesafe.config.ConfigFactory; import static akka.pattern.Patterns.ask; @@ -38,6 +39,19 @@ public class CustomRouterDocTestBase { public void tearDown() { system.shutdown(); } + + public static class MyActor extends UntypedActor { + @Override public void onReceive(Object o) {} + } + + @Test + public void demonstrateDispatchers() { + //#dispatchers + final ActorRef router = system.actorOf(new Props(MyActor.class) + .withRouter(new RoundRobinRouter(5).withDispatcher("head")) // “head” router runs on "head" dispatcher + .withDispatcher("workers")); // MyActor “workers” run on "workers" dispatcher + //#dispatchers + } //#crTest @Test @@ -105,6 +119,10 @@ public class CustomRouterDocTestBase { //#crRouter public static class VoteCountRouter extends CustomRouterConfig { + + @Override public String routerDispatcher() { + return Dispatchers.DefaultDispatcherId(); + } //#crRoute @Override diff --git a/akka-docs/java/routing.rst b/akka-docs/java/routing.rst index a422900440..265e31a984 100644 --- a/akka-docs/java/routing.rst +++ b/akka-docs/java/routing.rst @@ -8,11 +8,6 @@ Routing (Java) .. contents:: :local: -Akka-core includes some building blocks to build more complex message flow handlers, they are listed and explained below: - -Router ------- - A Router is an actor that routes incoming messages to outbound actors. The router routes the messages sent to it to its underlying actors called 'routees'. @@ -249,6 +244,16 @@ This is an example of how to programatically create a resizable router: *It is also worth pointing out that if you define the ``router`` in the configuration file then this value will be used instead of any programmatically sent parameters.* +.. note:: + + Resizing is triggered by sending messages to the actor pool, but it is not + completed synchronously; instead a message is sent to the “head” + :class:`Router` to perform the size change. Thus you cannot rely on resizing + to instantaneously create new workers when all others are busy, because the + message just sent will be queued to the mailbox of a busy actor. To remedy + this, configure the pool to use a balancing dispatcher, see `Configuring + Dispatchers`_ for more information. + Custom Router ^^^^^^^^^^^^^ @@ -312,3 +317,23 @@ A router with dynamically resizable number of routees is implemented by providin in ``resizer`` method of the ``RouterConfig``. See ``akka.routing.DefaultResizer`` for inspiration of how to write your own resize strategy. +Configuring Dispatchers +^^^^^^^^^^^^^^^^^^^^^^^ + +The dispatcher for created children of the router will be taken from +:class:`Props` as described in :ref:`dispatchers-java`. For a dynamic pool it +makes sense to configure the :class:`BalancingDispatcher` if the precise +routing is not so important (i.e. no consistent hashing or round-robin is +required); this enables newly created routees to pick up work immediately by +stealing it from their siblings. + +The “head” router, of couse, cannot run on the same balancing dispatcher, +because it does not process the same messages, hence this special actor does +not use the dispatcher configured in :class:`Props`, but takes the +``routerDispatcher`` from the :class:`RouterConfig` instead, which defaults to +the actor system’s default dispatcher. All standard routers allow setting this +property in their constructor or factory method, custom routers have to +implement the method in a suitable way. + +.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java#dispatchers + diff --git a/akka-docs/scala/code/akka/docs/routing/RouterDocSpec.scala b/akka-docs/scala/code/akka/docs/routing/RouterDocSpec.scala new file mode 100644 index 0000000000..229c66f13e --- /dev/null +++ b/akka-docs/scala/code/akka/docs/routing/RouterDocSpec.scala @@ -0,0 +1,29 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.routing + +import RouterDocSpec.MyActor +import akka.actor.{ Props, Actor } +import akka.testkit.AkkaSpec +import akka.routing.RoundRobinRouter + +object RouterDocSpec { + class MyActor extends Actor { + def receive = { + case _ ⇒ + } + } +} + +class RouterDocSpec extends AkkaSpec { + + import RouterDocSpec._ + + //#dispatchers + val router = system.actorOf(Props[MyActor] + .withRouter(RoundRobinRouter(5, routerDispatcher = "router")) // “head” will run on "router" dispatcher + .withDispatcher("workers")) // MyActor workers will run on "workers" dispatcher + //#dispatchers + +} \ No newline at end of file diff --git a/akka-docs/scala/routing.rst b/akka-docs/scala/routing.rst index f67841df2c..161ab88db9 100644 --- a/akka-docs/scala/routing.rst +++ b/akka-docs/scala/routing.rst @@ -8,11 +8,6 @@ Routing (Scala) .. contents:: :local: -Akka-core includes some building blocks to build more complex message flow handlers, they are listed and explained below: - -Router ------- - A Router is an actor that routes incoming messages to outbound actors. The router routes the messages sent to it to its underlying actors called 'routees'. @@ -250,6 +245,16 @@ This is an example of how to programatically create a resizable router: *It is also worth pointing out that if you define the ``router`` in the configuration file then this value will be used instead of any programmatically sent parameters.* +.. note:: + + Resizing is triggered by sending messages to the actor pool, but it is not + completed synchronously; instead a message is sent to the “head” + :class:`Router` to perform the size change. Thus you cannot rely on resizing + to instantaneously create new workers when all others are busy, because the + message just sent will be queued to the mailbox of a busy actor. To remedy + this, configure the pool to use a balancing dispatcher, see `Configuring + Dispatchers`_ for more information. + Custom Router ^^^^^^^^^^^^^ @@ -311,3 +316,23 @@ A router with dynamically resizable number of routees is implemented by providin in ``resizer`` method of the ``RouterConfig``. See ``akka.routing.DefaultResizer`` for inspiration of how to write your own resize strategy. +Configuring Dispatchers +^^^^^^^^^^^^^^^^^^^^^^^ + +The dispatcher for created children of the router will be taken from +:class:`Props` as described in :ref:`dispatchers-scala`. For a dynamic pool it +makes sense to configure the :class:`BalancingDispatcher` if the precise +routing is not so important (i.e. no consistent hashing or round-robin is +required); this enables newly created routees to pick up work immediately by +stealing it from their siblings. + +The “head” router, of couse, cannot run on the same balancing dispatcher, +because it does not process the same messages, hence this special actor does +not use the dispatcher configured in :class:`Props`, but takes the +``routerDispatcher`` from the :class:`RouterConfig` instead, which defaults to +the actor system’s default dispatcher. All standard routers allow setting this +property in their constructor or factory method, custom routers have to +implement the method in a suitable way. + +.. includecode:: code/akka/docs/routing/RouterDocSpec.scala#dispatchers + diff --git a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala index 21f5c400b0..3b1791db8e 100644 --- a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala +++ b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala @@ -30,6 +30,8 @@ case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[String]) exte override def createActor(): Router = local.createActor() + override def routerDispatcher: String = local.routerDispatcher + override def resizer: Option[Resizer] = local.resizer override def withFallback(other: RouterConfig): RouterConfig = other match { From 4b71872aef418639e1033bd7f98edb0e5ea138f4 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 10 Feb 2012 14:36:35 +0100 Subject: [PATCH 13/39] fixing up final misnamers (createClassFor -> getClassFor) --- .../main/scala/akka/actor/DynamicAccess.scala | 64 +++++++++---------- .../src/main/scala/akka/actor/Extension.scala | 2 +- .../src/main/scala/akka/event/Logging.scala | 2 +- .../akka/serialization/Serialization.scala | 10 ++- .../scala/akka/remote/MessageSerializer.scala | 2 +- .../scala/akka/testkit/TestActorRef.scala | 2 +- 6 files changed, 37 insertions(+), 45 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala index 4c641e44b2..8d3ac68852 100644 --- a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala +++ b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala @@ -16,36 +16,6 @@ import java.lang.reflect.InvocationTargetException */ trait DynamicAccess { - /** - * Obtain a `Class[_]` object loaded with the right class loader (i.e. the one - * returned by `classLoader`). - */ - def createClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] - - /** - * Obtain an object conforming to the type T, which is expected to be - * instantiated from a class designated by the fully-qualified class name - * given, where the constructor is selected and invoked according to the - * `args` argument. The exact usage of args depends on which type is requested, - * see the relevant requesting code for details. - */ - def createInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] - - /** - * Obtain the Scala “object” instance for the given fully-qualified class name, if there is one. - */ - def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T] - - /** - * This is the class loader to be used in those special cases where the - * other factory method are not applicable (e.g. when constructing a ClassLoaderBinaryInputStream). - */ - def classLoader: ClassLoader - -} - -object DynamicAccess { - /** * Convenience method which given a `Class[_]` object and a constructor description * will create a new instance of that class. @@ -66,6 +36,32 @@ object DynamicAccess { } } + /** + * Obtain a `Class[_]` object loaded with the right class loader (i.e. the one + * returned by `classLoader`). + */ + def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] + + /** + * Obtain an object conforming to the type T, which is expected to be + * instantiated from a class designated by the fully-qualified class name + * given, where the constructor is selected and invoked according to the + * `args` argument. The exact usage of args depends on which type is requested, + * see the relevant requesting code for details. + */ + def createInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] + + /** + * Obtain the Scala “object” instance for the given fully-qualified class name, if there is one. + */ + def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T] + + /** + * This is the class loader to be used in those special cases where the + * other factory method are not applicable (e.g. when constructing a ClassLoaderBinaryInputStream). + */ + def classLoader: ClassLoader + /** * Caught exception is returned as Left(exception). * Unwraps `InvocationTargetException` if its getTargetException is an `Exception`. @@ -93,9 +89,7 @@ object DynamicAccess { */ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAccess { - import DynamicAccess.withErrorHandling - - override def createClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] = + override def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] = try { val c = classLoader.loadClass(fqcn).asInstanceOf[Class[_ <: T]] val t = classManifest[T].erasure @@ -105,7 +99,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces } override def createInstanceFor[T: ClassManifest](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Throwable, T] = - createClassFor(fqcn).fold(Left(_), { c ⇒ + getClassFor(fqcn).fold(Left(_), { c ⇒ val types = args.map(_._1).toArray val values = args.map(_._2).toArray withErrorHandling { @@ -118,7 +112,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces }) override def getObjectFor[T: ClassManifest](fqcn: String): Either[Throwable, T] = { - createClassFor(fqcn).fold(Left(_), { c ⇒ + getClassFor(fqcn).fold(Left(_), { c ⇒ withErrorHandling { val module = c.getDeclaredField("MODULE$") module.setAccessible(true) diff --git a/akka-actor/src/main/scala/akka/actor/Extension.scala b/akka-actor/src/main/scala/akka/actor/Extension.scala index f5c0e68a5a..3adad6fd4b 100644 --- a/akka-actor/src/main/scala/akka/actor/Extension.scala +++ b/akka-actor/src/main/scala/akka/actor/Extension.scala @@ -97,7 +97,7 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassManifest[T]) extend override def lookup(): ExtensionId[T] = this def createExtension(system: ExtendedActorSystem): T = - DynamicAccess.createInstanceFor[T](m.erasure, Seq(classOf[ExtendedActorSystem] -> system)) match { + system.dynamicAccess.createInstanceFor[T](m.erasure, Seq(classOf[ExtendedActorSystem] -> system)) match { case Left(ex) ⇒ throw ex case Right(r) ⇒ r } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 0000f0ff80..a58ea3fdb9 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -100,7 +100,7 @@ trait LoggingBus extends ActorEventBus { if loggerName != StandardOutLoggerName } yield { try { - system.dynamicAccess.createClassFor[Actor](loggerName) match { + system.dynamicAccess.getClassFor[Actor](loggerName) match { case Right(actorClass) ⇒ addLogger(system, actorClass, level, logName) case Left(exception) ⇒ throw exception } diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 2b30307448..ce0f56a238 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -132,11 +132,9 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * Tries to load the specified Serializer by the fully-qualified name; the actual * loading is performed by the system’s [[akka.actor.DynamicAccess]]. */ - def serializerOf(serializerFQN: String): Either[Throwable, Serializer] = { - val dynamicAccess = system.dynamicAccess - dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)) - .fold(_ ⇒ dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq()), Right(_)) - } + def serializerOf(serializerFQN: String): Either[Throwable, Serializer] = + system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)).fold(_ ⇒ + system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq()), Right(_)) /** * A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer) @@ -153,7 +151,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { */ private[akka] val bindings: Seq[ClassSerializer] = { val configuredBindings = for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield { - val c = system.dynamicAccess.createClassFor(k).fold(throw _, identity[Class[_]]) + val c = system.dynamicAccess.getClassFor(k).fold(throw _, identity[Class[_]]) (c, serializers(v)) } sort(configuredBindings) diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 398c397de7..65777d49ca 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -14,7 +14,7 @@ object MessageSerializer { def deserialize(system: ExtendedActorSystem, messageProtocol: MessageProtocol): AnyRef = { val clazz = if (messageProtocol.hasMessageManifest) { - system.dynamicAccess.createClassFor[AnyRef](messageProtocol.getMessageManifest.toStringUtf8) + system.dynamicAccess.getClassFor[AnyRef](messageProtocol.getMessageManifest.toStringUtf8) .fold(throw _, Some(_)) } else None SerializationExtension(system) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 01405d2acf..8769fdda51 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -121,7 +121,7 @@ object TestActorRef { 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], system: ActorSystem): TestActorRef[T] = apply[T](Props({ - DynamicAccess.createInstanceFor[T](m.erasure, Seq()) match { + system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[T](m.erasure, Seq()) match { case Right(value) ⇒ value case Left(exception) ⇒ throw new ActorInitializationException(null, "Could not instantiate Actor" + From b6b438415e4c729419cab93e24c9d2f4a981510e Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 10 Feb 2012 14:55:40 +0100 Subject: [PATCH 14/39] fix up FromConfig router: allow empty constructor for Java --- akka-actor/src/main/scala/akka/routing/Routing.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index f83bca2db9..da2c81d1a7 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -373,6 +373,9 @@ case object FromConfig extends RouterConfig { */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig { + + def this() = this(Dispatchers.DefaultDispatcherId) + def createRoute(props: Props, routeeProvider: RouteeProvider): Route = throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)") } From e017aeef0826fc99722b74551f336d1751f0e884 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 10 Feb 2012 16:02:37 +0100 Subject: [PATCH 15/39] Replace akka.actor.timeout with specfic settings. See #1808 * ActorTimeout (akka.actor.timeout) was used to all sorts of things. * TestKit default-timeout * TypedActor timeout for non void methods * Transactor coordinated-timeout * ZeroMQ new-socket-timeout * And in various tests --- .../scala/akka/actor/GlobalActorSystem.scala | 9 +- .../main/scala/akka/dispatch/OldFuture.scala | 4 +- .../main/scala/akka/migration/package.scala | 2 +- .../java/akka/dispatch/JavaFutureTests.java | 31 +- .../scala/akka/actor/TypedActorSpec.scala | 2 +- .../src/test/scala/akka/pattern/AskSpec.scala | 4 +- akka-actor/src/main/resources/reference.conf | 17 +- .../main/scala/akka/actor/ActorSystem.scala | 1 - .../main/scala/akka/actor/TypedActor.scala | 13 +- akka-docs/general/configuration.rst | 18 ++ .../akka/docs/future/FutureDocTestBase.java | 265 +++++++++--------- .../code/akka/docs/jrouting/ParentActor.java | 6 +- .../code/akka/docs/future/FutureDocSpec.scala | 17 +- .../akka/docs/routing/RouterTypeExample.scala | 3 +- .../akka/remote/RemoteCommunicationSpec.scala | 4 +- .../src/main/resources/reference.conf | 3 + .../scala/akka/testkit/TestActorRef.scala | 2 +- .../src/main/scala/akka/testkit/TestKit.scala | 3 +- .../scala/akka/testkit/TestKitExtension.scala | 2 + .../test/scala/akka/testkit/AkkaSpec.scala | 2 +- .../src/main/resources/reference.conf | 13 + .../scala/akka/transactor/Transactor.scala | 4 +- .../akka/transactor/TransactorExtension.scala | 25 ++ .../akka/transactor/UntypedTransactor.scala | 4 +- akka-zeromq/src/main/resources/reference.conf | 9 +- .../akka/zeromq/ConcurrentSocketActor.scala | 5 +- .../scala/akka/zeromq/ZeroMQExtension.scala | 8 +- 27 files changed, 283 insertions(+), 193 deletions(-) create mode 100644 akka-transactor/src/main/resources/reference.conf create mode 100644 akka-transactor/src/main/scala/akka/transactor/TransactorExtension.scala diff --git a/akka-actor-migration/src/main/scala/akka/actor/GlobalActorSystem.scala b/akka-actor-migration/src/main/scala/akka/actor/GlobalActorSystem.scala index 694dd5d547..e08883c6ed 100644 --- a/akka-actor-migration/src/main/scala/akka/actor/GlobalActorSystem.scala +++ b/akka-actor-migration/src/main/scala/akka/actor/GlobalActorSystem.scala @@ -4,14 +4,21 @@ package akka.actor import java.io.File - import com.typesafe.config.Config import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigParseOptions +import akka.util.Timeout +import akka.util.duration._ @deprecated("use ActorSystem instead", "2.0") object GlobalActorSystem extends ActorSystemImpl("GlobalSystem", OldConfigurationLoader.defaultConfig) { start() + + /** + * Timeout used in `OldFuture.get` and default implicit ask timeout. + * Hard coded since the migration kit is not intended to be used for production anyway. + */ + val AwaitTimeout = Timeout(5 seconds) } /** diff --git a/akka-actor-migration/src/main/scala/akka/dispatch/OldFuture.scala b/akka-actor-migration/src/main/scala/akka/dispatch/OldFuture.scala index 89941cd066..60029e256a 100644 --- a/akka-actor-migration/src/main/scala/akka/dispatch/OldFuture.scala +++ b/akka-actor-migration/src/main/scala/akka/dispatch/OldFuture.scala @@ -18,13 +18,13 @@ class OldFuture[T](future: Future[T]) { @deprecated("use akka.dispatch.Await.result instead", "2.0") def get: T = try { - Await.result(future, GlobalActorSystem.settings.ActorTimeout.duration) + Await.result(future, GlobalActorSystem.AwaitTimeout.duration) } catch { case e: TimeoutException ⇒ throw new FutureTimeoutException(e.getMessage, e) } @deprecated("use akka.dispatch.Await.ready instead", "2.0") - def await: Future[T] = await(GlobalActorSystem.settings.ActorTimeout.duration) + def await: Future[T] = await(GlobalActorSystem.AwaitTimeout.duration) @deprecated("use akka.dispatch.Await.ready instead", "2.0") def await(atMost: Duration) = try { diff --git a/akka-actor-migration/src/main/scala/akka/migration/package.scala b/akka-actor-migration/src/main/scala/akka/migration/package.scala index 6b8cad6fe2..469604e464 100644 --- a/akka-actor-migration/src/main/scala/akka/migration/package.scala +++ b/akka-actor-migration/src/main/scala/akka/migration/package.scala @@ -14,7 +14,7 @@ package object migration { implicit def future2OldFuture[T](future: Future[T]): OldFuture[T] = new OldFuture[T](future) - implicit def askTimeout: Timeout = GlobalActorSystem.settings.ActorTimeout + implicit def askTimeout: Timeout = GlobalActorSystem.AwaitTimeout implicit def defaultDispatcher: MessageDispatcher = GlobalActorSystem.dispatcher 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 f494fd7d81..b03fe3b5fc 100644 --- a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java +++ b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java @@ -5,6 +5,7 @@ import akka.actor.ActorSystem; import akka.japi.*; import akka.util.Duration; +import akka.testkit.TestKitExtension; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -28,7 +29,7 @@ public class JavaFutureTests { @BeforeClass public static void beforeAll() { system = ActorSystem.create("JavaFutureTests", AkkaSpec.testConf()); - t = system.settings().ActorTimeout(); + t = TestKitExtension.get(system).DefaultTimeout(); } @AfterClass @@ -61,10 +62,10 @@ public class JavaFutureTests { Promise cf = Futures.promise(system.dispatcher()); Future f = cf; f.onSuccess(new OnSuccess() { - public void onSuccess(String result) { - if (result.equals("foo")) - latch.countDown(); - } + public void onSuccess(String result) { + if (result.equals("foo")) + latch.countDown(); + } }); cf.success("foo"); @@ -78,10 +79,10 @@ public class JavaFutureTests { Promise cf = Futures.promise(system.dispatcher()); Future f = cf; f.onFailure(new OnFailure() { - public void onFailure(Throwable t) { - if (t instanceof NullPointerException) - latch.countDown(); - } + public void onFailure(Throwable t) { + if (t instanceof NullPointerException) + latch.countDown(); + } }); Throwable exception = new NullPointerException(); @@ -296,8 +297,10 @@ public class JavaFutureTests { Promise p = Futures.promise(system.dispatcher()); Future f = p.future().recover(new Recover() { public Object recover(Throwable t) throws Throwable { - if (t == fail) return "foo"; - else throw t; + if (t == fail) + return "foo"; + else + throw t; } }); Duration d = Duration.create(1, TimeUnit.SECONDS); @@ -311,8 +314,10 @@ public class JavaFutureTests { Promise p = Futures.promise(system.dispatcher()); Future f = p.future().recoverWith(new Recover>() { public Future recover(Throwable t) throws Throwable { - if (t == fail) return Futures.successful("foo", system.dispatcher()).future(); - else throw t; + if (t == fail) + return Futures. successful("foo", system.dispatcher()).future(); + else + throw t; } }); Duration d = Duration.create(1, TimeUnit.SECONDS); 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 4e8bc4d7b4..5a9fab6c63 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -113,7 +113,7 @@ object TypedActorSpec { } def futureComposePigdogFrom(foo: Foo): Future[String] = { - implicit val timeout = TypedActor.context.system.settings.ActorTimeout + implicit val timeout = TypedActor(TypedActor.context.system).DefaultReturnTimeout foo.futurePigdog(500).map(_.toUpperCase) } diff --git a/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala b/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala index ecb9690594..f3c36665e8 100644 --- a/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala @@ -5,8 +5,9 @@ package akka.pattern import akka.testkit.AkkaSpec import akka.util.duration._ +import akka.testkit.DefaultTimeout -class AskSpec extends AkkaSpec { +class AskSpec extends AkkaSpec with DefaultTimeout { "The “ask” pattern" must { @@ -22,7 +23,6 @@ class AskSpec extends AkkaSpec { "return broken promises on EmptyLocalActorRefs" in { val empty = system.actorFor("unknown") - implicit val timeout = system.settings.ActorTimeout val f = empty ? 3.14 f.isCompleted must be(true) f.value.get match { diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index bc52938a7d..62d49d61cd 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -1,6 +1,6 @@ -############################## -# Akka Reference Config File # -############################## +#################################### +# Akka Actor Reference Config File # +#################################### # This the reference config file has all the default settings. # Make your edits/overrides in your application.conf. @@ -50,12 +50,6 @@ akka { # removed from their parents reaper-interval = 5s - # Default timeout for Future based invocations - # - Actor: ask && ? - # - UntypedActor: ask - # - TypedActor: methods with non-void return type - timeout = 5s - # Serializes and deserializes (non-primitive) messages to ensure immutability, # this is only intended for testing. serialize-messages = off @@ -64,6 +58,11 @@ akka { # this is only intended for testing. serialize-creators = off + typed { + # Default timeout for typed actor methods with non-void return type + timeout = 5s + } + deployment { # deployment id pattern - on the format: /parent/child etc. diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index ac77628c2c..76eea494ca 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -70,7 +70,6 @@ object ActorSystem { final val CreationTimeout = Timeout(Duration(getMilliseconds("akka.actor.creation-timeout"), MILLISECONDS)) final val ReaperInterval = Duration(getMilliseconds("akka.actor.reaper-interval"), MILLISECONDS) - final val ActorTimeout = Timeout(Duration(getMilliseconds("akka.actor.timeout"), MILLISECONDS)) final val SerializeAllMessages = getBoolean("akka.actor.serialize-messages") final val SerializeAllCreators = getBoolean("akka.actor.serialize-creators") diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 98329203e9..3d1e0c76f3 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -11,7 +11,9 @@ import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar } import akka.serialization.{ Serialization, SerializationExtension } import akka.dispatch._ import java.util.concurrent.TimeoutException +import java.util.concurrent.TimeUnit.MILLISECONDS import java.lang.IllegalStateException +import akka.util.Duration trait TypedActorFactory { @@ -502,7 +504,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( /** * @return a new TypedProps that will use the specified Timeout for its non-void-returning methods, - * if null is specified, it will use the default ActorTimeout as specified in the configuration. + * if null is specified, it will use the default timeout as specified in the configuration. * * Java API */ @@ -510,7 +512,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( /** * @return a new TypedProps that will use the specified Timeout for its non-void-returning methods, - * if None is specified, it will use the default ActorTimeout as specified in the configuration. + * if None is specified, it will use the default timeout as specified in the configuration. * * Scala API */ @@ -550,6 +552,11 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory val serialization = SerializationExtension(system) val settings = system.settings + /** + * Default timeout for typed actor methods with non-void return type + */ + final val DefaultReturnTimeout = Timeout(Duration(settings.config.getMilliseconds("akka.actor.typed.timeout"), MILLISECONDS)) + /** * Retrieves the underlying ActorRef for the supplied TypedActor proxy, or null if none found */ @@ -575,7 +582,7 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory new TypedActorInvocationHandler( this, actorVar, - if (props.timeout.isDefined) props.timeout.get else this.settings.ActorTimeout)).asInstanceOf[R] + if (props.timeout.isDefined) props.timeout.get else DefaultReturnTimeout)).asInstanceOf[R] proxyVar match { case null ⇒ diff --git a/akka-docs/general/configuration.rst b/akka-docs/general/configuration.rst index 3e2bca240d..649a6abc04 100644 --- a/akka-docs/general/configuration.rst +++ b/akka-docs/general/configuration.rst @@ -75,6 +75,24 @@ akka-testkit .. literalinclude:: ../../akka-testkit/src/main/resources/reference.conf :language: none +akka-transactor +~~~~~~~~~~~~~~~ + +.. literalinclude:: ../../akka-transactor/src/main/resources/reference.conf + :language: none + +akka-agent +~~~~~~~~~~ + +.. literalinclude:: ../../akka-agent/src/main/resources/reference.conf + :language: none + +akka-zeromq +~~~~~~~~~~~ + +.. literalinclude:: ../../akka-zeromq/src/main/resources/reference.conf + :language: none + akka-beanstalk-mailbox ~~~~~~~~~~~~~~~~~~~~~~ diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index 8fc3b29b4e..e541c925c1 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -78,7 +78,7 @@ public class FutureDocTestBase { ActorRef actor = system.actorOf(new Props(MyActor.class)); String msg = "hello"; //#ask-blocking - Timeout timeout = system.settings().ActorTimeout(); + Timeout timeout = new Timeout(Duration.parse("5 seconds")); Future future = Patterns.ask(actor, msg, timeout); String result = (String) Await.result(future, timeout.duration()); //#ask-blocking @@ -196,19 +196,17 @@ public class FutureDocTestBase { Iterable> listOfFutureInts = source; // now we have a Future[Iterable[Integer]] - Future> futureListOfInts = - sequence(listOfFutureInts, system.dispatcher()); + Future> futureListOfInts = sequence(listOfFutureInts, system.dispatcher()); // Find the sum of the odd numbers - Future futureSum = futureListOfInts.map( - new Mapper, Long>() { - public Long apply(Iterable ints) { - long sum = 0; - for (Integer i : ints) - sum += i; - return sum; - } - }); + Future futureSum = futureListOfInts.map(new Mapper, Long>() { + public Long apply(Iterable ints) { + long sum = 0; + for (Integer i : ints) + sum += i; + return sum; + } + }); long result = Await.result(futureSum, Duration.create(1, SECONDS)); //#sequence @@ -221,20 +219,18 @@ public class FutureDocTestBase { //Just a sequence of Strings Iterable listStrings = Arrays.asList("a", "b", "c"); - Future> futureResult = traverse(listStrings, - new Function>() { - public Future apply(final String r) { - return future(new Callable() { - public String call() { - return r.toUpperCase(); - } - }, system.dispatcher()); - } - }, system.dispatcher()); + Future> futureResult = traverse(listStrings, new Function>() { + public Future apply(final String r) { + return future(new Callable() { + public String call() { + return r.toUpperCase(); + } + }, system.dispatcher()); + } + }, system.dispatcher()); //Returns the sequence of strings as upper case - Iterable result = - Await.result(futureResult, Duration.create(1, SECONDS)); + Iterable result = Await.result(futureResult, Duration.create(1, SECONDS)); assertEquals(Arrays.asList("A", "B", "C"), result); //#traverse } @@ -250,12 +246,11 @@ public class FutureDocTestBase { Iterable> futures = source; //Start value is the empty string - Future resultFuture = fold("", futures, - new Function2() { - public String apply(String r, String t) { - return r + t; //Just concatenate - } - }, system.dispatcher()); + Future resultFuture = fold("", futures, new Function2() { + public String apply(String r, String t) { + return r + t; //Just concatenate + } + }, system.dispatcher()); String result = Await.result(resultFuture, Duration.create(1, SECONDS)); //#fold @@ -272,12 +267,11 @@ public class FutureDocTestBase { //A sequence of Futures, in this case Strings Iterable> futures = source; - Future resultFuture = reduce(futures, - new Function2() { - public Object apply(Object r, String t) { - return r + t; //Just concatenate - } - }, system.dispatcher()); + Future resultFuture = reduce(futures, new Function2() { + public Object apply(Object r, String t) { + return r + t; //Just concatenate + } + }, system.dispatcher()); Object result = Await.result(resultFuture, Duration.create(1, SECONDS)); //#reduce @@ -285,32 +279,35 @@ public class FutureDocTestBase { assertEquals("ab", result); } - @Test public void useSuccessfulAndFailed() { + @Test + public void useSuccessfulAndFailed() { //#successful Future future = Futures.successful("Yay!", system.dispatcher()); //#successful //#failed - Future otherFuture = - Futures.failed(new IllegalArgumentException("Bang!"), system.dispatcher()); + Future otherFuture = Futures.failed(new IllegalArgumentException("Bang!"), system.dispatcher()); //#failed Object result = Await.result(future, Duration.create(1, SECONDS)); - assertEquals("Yay!",result); + assertEquals("Yay!", result); Throwable result2 = Await.result(otherFuture.failed(), Duration.create(1, SECONDS)); - assertEquals("Bang!",result2.getMessage()); + assertEquals("Bang!", result2.getMessage()); } - @Test public void useFilter() { - //#filter + @Test + public void useFilter() { + //#filter Future future1 = Futures.successful(4, system.dispatcher()); - Future successfulFilter = - future1.filter(new Filter() { - public boolean filter(Integer i) { return i % 2 == 0; } - }); + Future successfulFilter = future1.filter(new Filter() { + public boolean filter(Integer i) { + return i % 2 == 0; + } + }); - Future failedFilter = - future1.filter(new Filter() { - public boolean filter(Integer i) { return i % 2 != 0; } - }); + Future failedFilter = future1.filter(new Filter() { + public boolean filter(Integer i) { + return i % 2 != 0; + } + }); //When filter fails, the returned Future will be failed with a scala.MatchError //#filter } @@ -323,138 +320,140 @@ public class FutureDocTestBase { } - @Test public void useAndThen() { + @Test + public void useAndThen() { //#and-then - Future future1 = Futures.successful("value", system.dispatcher()). - andThen(new OnComplete() { - public void onComplete(Throwable failure, String result) { - if (failure != null) sendToIssueTracker(failure); - } + Future future1 = Futures.successful("value", system.dispatcher()).andThen(new OnComplete() { + public void onComplete(Throwable failure, String result) { + if (failure != null) + sendToIssueTracker(failure); + } }).andThen(new OnComplete() { - public void onComplete(Throwable failure, String result) { - if (result != null) sendToTheInternetz(result); - } + public void onComplete(Throwable failure, String result) { + if (result != null) + sendToTheInternetz(result); + } }); //#and-then } - @Test public void useRecover() { + @Test + public void useRecover() { //#recover Future future = future(new Callable() { public Integer call() { return 1 / 0; } }, system.dispatcher()).recover(new Recover() { - public Integer recover(Throwable problem) throws Throwable { - if (problem instanceof ArithmeticException) return 0; - else throw problem; - } + public Integer recover(Throwable problem) throws Throwable { + if (problem instanceof ArithmeticException) + return 0; + else + throw problem; + } }); int result = Await.result(future, Duration.create(1, SECONDS)); assertEquals(result, 0); //#recover } - @Test public void useTryRecover() { + @Test + public void useTryRecover() { //#try-recover Future future = future(new Callable() { public Integer call() { return 1 / 0; } }, system.dispatcher()).recoverWith(new Recover>() { - public Future recover(Throwable problem) throws Throwable { - if (problem instanceof ArithmeticException) { - return future(new Callable() { - public Integer call() { - return 0; - } - }, system.dispatcher()); + public Future recover(Throwable problem) throws Throwable { + if (problem instanceof ArithmeticException) { + return future(new Callable() { + public Integer call() { + return 0; } - else throw problem; - } + }, system.dispatcher()); + } else + throw problem; + } }); int result = Await.result(future, Duration.create(1, SECONDS)); assertEquals(result, 0); //#try-recover } - @Test public void useOnSuccessOnFailureAndOnComplete() { - { + @Test + public void useOnSuccessOnFailureAndOnComplete() { + { Future future = Futures.successful("foo", system.dispatcher()); //#onSuccess future.onSuccess(new OnSuccess() { - public void onSuccess(String result) { - if ("bar" == result) { - //Do something if it resulted in "bar" - } else { - //Do something if it was some other String - } + public void onSuccess(String result) { + if ("bar" == result) { + //Do something if it resulted in "bar" + } else { + //Do something if it was some other String } + } }); //#onSuccess - } - { - Future future = - Futures.failed(new IllegalStateException("OHNOES"), system.dispatcher()); - //#onFailure - future.onFailure( new OnFailure() { + } + { + Future future = Futures.failed(new IllegalStateException("OHNOES"), system.dispatcher()); + //#onFailure + future.onFailure(new OnFailure() { public void onFailure(Throwable failure) { - if (failure instanceof IllegalStateException) { - //Do something if it was this particular failure - } else { - //Do something if it was some other failure - } + if (failure instanceof IllegalStateException) { + //Do something if it was this particular failure + } else { + //Do something if it was some other failure + } } }); //#onFailure - } - { - Future future = Futures.successful("foo", system.dispatcher()); - //#onComplete - future.onComplete(new OnComplete() { - public void onComplete(Throwable failure, String result) { - if (failure != null) { - //We got a failure, handle it here - } else { - // We got a result, do something with it - } - } - }); - //#onComplete - } + } + { + Future future = Futures.successful("foo", system.dispatcher()); + //#onComplete + future.onComplete(new OnComplete() { + public void onComplete(Throwable failure, String result) { + if (failure != null) { + //We got a failure, handle it here + } else { + // We got a result, do something with it + } + } + }); + //#onComplete + } } - @Test public void useOrAndZip(){ + @Test + public void useOrAndZip() { { - //#zip - Future future1 = Futures.successful("foo", system.dispatcher()); - Future future2 = Futures.successful("bar", system.dispatcher()); - Future future3 = - future1.zip(future2).map(new Mapper, String>() { - public String apply(scala.Tuple2 zipped) { - return zipped._1() + " " + zipped._2(); + //#zip + Future future1 = Futures.successful("foo", system.dispatcher()); + Future future2 = Futures.successful("bar", system.dispatcher()); + Future future3 = future1.zip(future2).map(new Mapper, String>() { + public String apply(scala.Tuple2 zipped) { + return zipped._1() + " " + zipped._2(); } - }); + }); - String result = Await.result(future3, Duration.create(1, SECONDS)); - assertEquals("foo bar", result); - //#zip + String result = Await.result(future3, Duration.create(1, SECONDS)); + assertEquals("foo bar", result); + //#zip } { - //#fallback-to - Future future1 = - Futures.failed(new IllegalStateException("OHNOES1"), system.dispatcher()); - Future future2 = - Futures.failed(new IllegalStateException("OHNOES2"), system.dispatcher()); - Future future3 = - Futures.successful("bar", system.dispatcher()); - Future future4 = - future1.fallbackTo(future2).fallbackTo(future3); // Will have "bar" in this case - String result = Await.result(future4, Duration.create(1, SECONDS)); - assertEquals("bar", result); - //#fallback-to + //#fallback-to + Future future1 = Futures.failed(new IllegalStateException("OHNOES1"), system.dispatcher()); + Future future2 = Futures.failed(new IllegalStateException("OHNOES2"), system.dispatcher()); + Future future3 = Futures.successful("bar", system.dispatcher()); + Future future4 = future1.fallbackTo(future2).fallbackTo(future3); // Will have "bar" in this case + String result = Await.result(future4, Duration.create(1, SECONDS)); + assertEquals("bar", result); + //#fallback-to } } diff --git a/akka-docs/java/code/akka/docs/jrouting/ParentActor.java b/akka-docs/java/code/akka/docs/jrouting/ParentActor.java index 1119559489..cf1e2b9cee 100644 --- a/akka-docs/java/code/akka/docs/jrouting/ParentActor.java +++ b/akka-docs/java/code/akka/docs/jrouting/ParentActor.java @@ -54,9 +54,9 @@ public class ParentActor extends UntypedActor { ActorRef scatterGatherFirstCompletedRouter = getContext().actorOf( new Props(FibonacciActor.class).withRouter(new ScatterGatherFirstCompletedRouter(5, Duration .parse("2 seconds"))), "router"); - Timeout timeout = getContext().system().settings().ActorTimeout(); - Future futureResult = akka.pattern.Patterns.ask( - scatterGatherFirstCompletedRouter, new FibonacciActor.FibonacciNumber(10), timeout); + Timeout timeout = new Timeout(Duration.parse("5 seconds")); + Future futureResult = akka.pattern.Patterns.ask(scatterGatherFirstCompletedRouter, + new FibonacciActor.FibonacciNumber(10), timeout); int result = (Integer) Await.result(futureResult, timeout.duration()); //#scatterGatherFirstCompletedRouter System.out.println(String.format("The result of calculating Fibonacci for 10 is %d", result)); diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala index 098fe873ad..3d1ca946a7 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala @@ -11,6 +11,7 @@ import akka.actor.Props import akka.actor.Status.Failure import akka.dispatch.Future import akka.dispatch.Await +import akka.util.Timeout import akka.util.duration._ import akka.dispatch.Promise import java.lang.IllegalStateException @@ -46,8 +47,10 @@ class FutureDocSpec extends AkkaSpec { //#ask-blocking import akka.dispatch.Await import akka.pattern.ask + import akka.util.Timeout + import akka.util.duration._ - implicit val timeout = system.settings.ActorTimeout + implicit val timeout = Timeout(5 seconds) val future = actor ? msg // enabled by the “ask” import val result = Await.result(future, timeout.duration).asInstanceOf[String] //#ask-blocking @@ -57,7 +60,7 @@ class FutureDocSpec extends AkkaSpec { "demonstrate usage of mapTo" in { val actor = system.actorOf(Props[MyActor]) val msg = "hello" - implicit val timeout = system.settings.ActorTimeout + implicit val timeout = Timeout(5 seconds) //#map-to import akka.dispatch.Future import akka.pattern.ask @@ -164,7 +167,7 @@ class FutureDocSpec extends AkkaSpec { val actor3 = system.actorOf(Props[MyActor]) val msg1 = 1 val msg2 = 2 - implicit val timeout = system.settings.ActorTimeout + implicit val timeout = Timeout(5 seconds) import akka.dispatch.Await import akka.pattern.ask //#composing-wrong @@ -188,7 +191,7 @@ class FutureDocSpec extends AkkaSpec { val actor3 = system.actorOf(Props[MyActor]) val msg1 = 1 val msg2 = 2 - implicit val timeout = system.settings.ActorTimeout + implicit val timeout = Timeout(5 seconds) import akka.dispatch.Await import akka.pattern.ask //#composing @@ -208,7 +211,7 @@ class FutureDocSpec extends AkkaSpec { } "demonstrate usage of sequence with actors" in { - implicit val timeout = system.settings.ActorTimeout + implicit val timeout = Timeout(5 seconds) val oddActor = system.actorOf(Props[OddActor]) //#sequence-ask // oddActor returns odd numbers sequentially from 1 as a List[Future[Int]] @@ -256,7 +259,7 @@ class FutureDocSpec extends AkkaSpec { } "demonstrate usage of recover" in { - implicit val timeout = system.settings.ActorTimeout + implicit val timeout = Timeout(5 seconds) val actor = system.actorOf(Props[MyActor]) val msg1 = -1 //#recover @@ -268,7 +271,7 @@ class FutureDocSpec extends AkkaSpec { } "demonstrate usage of recoverWith" in { - implicit val timeout = system.settings.ActorTimeout + implicit val timeout = Timeout(5 seconds) val actor = system.actorOf(Props[MyActor]) val msg1 = -1 //#try-recover diff --git a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala index 2f0a1e634c..6ec475a874 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala +++ b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala @@ -7,6 +7,7 @@ import akka.routing.{ ScatterGatherFirstCompletedRouter, BroadcastRouter, Random import annotation.tailrec import akka.actor.{ Props, Actor } import akka.util.duration._ +import akka.util.Timeout import akka.dispatch.Await import akka.pattern.ask import akka.routing.SmallestMailboxRouter @@ -80,7 +81,7 @@ class ParentActor extends Actor { val scatterGatherFirstCompletedRouter = context.actorOf( Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter( nrOfInstances = 5, within = 2 seconds)), "router") - implicit val timeout = context.system.settings.ActorTimeout + implicit val timeout = Timeout(5 seconds) val futureResult = scatterGatherFirstCompletedRouter ? FibonacciNumber(10) val result = Await.result(futureResult, timeout.duration) //#scatterGatherFirstCompletedRouter diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index 88d80d6d81..73f8a98030 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -44,7 +44,7 @@ akka { /looker/child/grandchild.remote = "akka://RemoteCommunicationSpec@localhost:12345" } } -""") with ImplicitSender { +""") with ImplicitSender with DefaultTimeout { import RemoteCommunicationSpec._ @@ -59,8 +59,6 @@ akka { val here = system.actorFor("akka://remote_sys@localhost:12346/user/echo") - implicit val timeout = system.settings.ActorTimeout - override def atTermination() { other.shutdown() } diff --git a/akka-testkit/src/main/resources/reference.conf b/akka-testkit/src/main/resources/reference.conf index e4ae685f4d..5e70df9403 100644 --- a/akka-testkit/src/main/resources/reference.conf +++ b/akka-testkit/src/main/resources/reference.conf @@ -18,6 +18,9 @@ akka { # duration to wait in expectMsg and friends outside of within() block by default single-expect-default = 3s + # The timeout that is added as an implicit by DefaultTimeout trait + default-timeout = 5s + calling-thread-dispatcher { type = akka.testkit.CallingThreadDispatcherConfigurator } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 8769fdda51..8a2f61bf76 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -68,7 +68,7 @@ class TestActorRef[T <: Actor]( if (isTerminated) throw new IllegalActorStateException("underlying actor is terminated") underlying.actor.asInstanceOf[T] match { case null ⇒ - val t = underlying.system.settings.ActorTimeout + val t = TestKitExtension(_system).DefaultTimeout Await.result(this.?(InternalGetActor)(t), t.duration).asInstanceOf[T] case ref ⇒ ref } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index 4155ea662d..bdfab36ede 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -11,6 +11,7 @@ import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atom import atomic.AtomicInteger import scala.annotation.tailrec import akka.actor.ActorSystem +import akka.util.Timeout object TestActor { type Ignore = Option[PartialFunction[AnyRef, Boolean]] @@ -644,5 +645,5 @@ trait ImplicitSender { this: TestKit ⇒ } trait DefaultTimeout { this: TestKit ⇒ - implicit val timeout = system.settings.ActorTimeout + implicit val timeout: Timeout = testKitSettings.DefaultTimeout } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala b/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala index ada5a4fd30..71ba8d0eac 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala @@ -5,6 +5,7 @@ package akka.testkit import com.typesafe.config.Config import akka.util.Duration +import akka.util.Timeout import java.util.concurrent.TimeUnit.MILLISECONDS import akka.actor.{ ExtensionId, ActorSystem, Extension, ExtendedActorSystem } @@ -20,4 +21,5 @@ class TestKitSettings(val config: Config) extends Extension { val TestTimeFactor = getDouble("akka.test.timefactor") val SingleExpectDefaultTimeout = Duration(getMilliseconds("akka.test.single-expect-default"), MILLISECONDS) val TestEventFilterLeeway = Duration(getMilliseconds("akka.test.filter-leeway"), MILLISECONDS) + val DefaultTimeout = Timeout(Duration(getMilliseconds("akka.test.default-timeout"), MILLISECONDS)) } \ No newline at end of file diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index 172bdc230f..95ce267320 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -122,7 +122,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers { try { var locker = Seq.empty[DeadLetter] - implicit val timeout = system.settings.ActorTimeout + implicit val timeout = TestKitExtension(system).DefaultTimeout implicit val davyJones = otherSystem.actorOf(Props(new Actor { def receive = { case m: DeadLetter ⇒ locker :+= m diff --git a/akka-transactor/src/main/resources/reference.conf b/akka-transactor/src/main/resources/reference.conf new file mode 100644 index 0000000000..d91a50db33 --- /dev/null +++ b/akka-transactor/src/main/resources/reference.conf @@ -0,0 +1,13 @@ +######################################### +# Akka Transactor Reference Config File # +######################################### + +# This the reference config file has all the default settings. +# Make your edits/overrides in your application.conf. + +akka { + transactor { + # The timeout used for coordinated transactions across actors + coordinated-timeout = 5s + } +} diff --git a/akka-transactor/src/main/scala/akka/transactor/Transactor.scala b/akka-transactor/src/main/scala/akka/transactor/Transactor.scala index 627a5ab249..6e390a6623 100644 --- a/akka-transactor/src/main/scala/akka/transactor/Transactor.scala +++ b/akka-transactor/src/main/scala/akka/transactor/Transactor.scala @@ -93,6 +93,8 @@ case class SendTo(actor: ActorRef, message: Option[Any] = None) * @see [[akka.transactor.Coordinated]] for more information about the underlying mechanism */ trait Transactor extends Actor { + private val settings = TransactorExtension(context.system) + /** * Implement a general pattern for using coordinated transactions. */ @@ -108,7 +110,7 @@ trait Transactor extends Actor { } case message ⇒ { if (normally.isDefinedAt(message)) normally(message) - else receive(Coordinated(message)(context.system.settings.ActorTimeout)) + else receive(Coordinated(message)(settings.CoordinatedTimeout)) } } diff --git a/akka-transactor/src/main/scala/akka/transactor/TransactorExtension.scala b/akka-transactor/src/main/scala/akka/transactor/TransactorExtension.scala new file mode 100644 index 0000000000..96aea8904c --- /dev/null +++ b/akka-transactor/src/main/scala/akka/transactor/TransactorExtension.scala @@ -0,0 +1,25 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.transactor + +import akka.actor.{ ActorSystem, ExtensionId, ExtensionIdProvider, ExtendedActorSystem } +import akka.actor.Extension +import com.typesafe.config.Config +import akka.util.Timeout +import akka.util.Duration +import java.util.concurrent.TimeUnit.MILLISECONDS + +/** + * TransactorExtension is an Akka Extension to hold settings for transactors. + */ +object TransactorExtension extends ExtensionId[TransactorSettings] with ExtensionIdProvider { + override def get(system: ActorSystem): TransactorSettings = super.get(system) + override def lookup = TransactorExtension + override def createExtension(system: ExtendedActorSystem): TransactorSettings = new TransactorSettings(system.settings.config) +} + +class TransactorSettings(val config: Config) extends Extension { + import config._ + val CoordinatedTimeout = Timeout(Duration(getMilliseconds("akka.transactor.coordinated-timeout"), MILLISECONDS)) +} \ No newline at end of file diff --git a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala index 0ffbd5e65c..353695fd73 100644 --- a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala +++ b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala @@ -12,6 +12,8 @@ import java.util.{ Set ⇒ JSet } * An UntypedActor version of transactor for using from Java. */ abstract class UntypedTransactor extends UntypedActor { + private val settings = TransactorExtension(context.system) + /** * Implement a general pattern for using coordinated transactions. */ @@ -29,7 +31,7 @@ abstract class UntypedTransactor extends UntypedActor { } case message ⇒ { val normal = normally(message) - if (!normal) onReceive(Coordinated(message)(context.system.settings.ActorTimeout)) + if (!normal) onReceive(Coordinated(message)(settings.CoordinatedTimeout)) } } } diff --git a/akka-zeromq/src/main/resources/reference.conf b/akka-zeromq/src/main/resources/reference.conf index cfb5756156..b94a442c78 100644 --- a/akka-zeromq/src/main/resources/reference.conf +++ b/akka-zeromq/src/main/resources/reference.conf @@ -1,6 +1,6 @@ -############################## -# Akka Reference Config File # -############################## +##################################### +# Akka ZeroMQ Reference Config File # +##################################### # This the reference config file has all the default settings. # Make your edits/overrides in your application.conf. @@ -12,6 +12,9 @@ akka { # The default timeout for a poll on the actual zeromq socket. poll-timeout = 100ms + # Timeout for creating a new socket + new-socket-timeout = 5s + socket-dispatcher { # A zeromq socket needs to be pinned to the thread that created it. # Changing this value results in weird errors and race conditions within zeromq diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 254a097d80..82a07d7aa3 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -187,12 +187,9 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A fromConfig getOrElse context.system.dispatcher } - private val defaultPollTimeout = - Duration(context.system.settings.config.getMilliseconds("akka.zeromq.poll-timeout"), TimeUnit.MILLISECONDS) - private val pollTimeout = { val fromConfig = params collectFirst { case PollTimeoutDuration(duration) ⇒ duration } - fromConfig getOrElse defaultPollTimeout + fromConfig getOrElse ZeroMQExtension(context.system).DefaultPollTimeout } private def newEventLoop: Option[Promise[PollLifeCycle]] = { diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 7ae178291f..25b35e4644 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -7,6 +7,9 @@ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ import akka.dispatch.{ Await } import akka.pattern.ask +import akka.util.Duration +import java.util.concurrent.TimeUnit +import akka.util.Timeout /** * A Model to represent a version of the zeromq library @@ -43,6 +46,9 @@ object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProv */ class ZeroMQExtension(system: ActorSystem) extends Extension { + val DefaultPollTimeout = Duration(system.settings.config.getMilliseconds("akka.zeromq.poll-timeout"), TimeUnit.MILLISECONDS) + val NewSocketTimeout = Timeout(Duration(system.settings.config.getMilliseconds("akka.zeromq.new-socket-timeout"), TimeUnit.MILLISECONDS)) + /** * The version of the ZeroMQ library * @return a [[akka.zeromq.ZeroMQVersion]] @@ -136,7 +142,7 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { * @return the [[akka.actor.ActorRef]] */ def newSocket(socketParameters: SocketOption*): ActorRef = { - implicit val timeout = system.settings.ActorTimeout + implicit val timeout = NewSocketTimeout val req = (zeromqGuardian ? newSocketProps(socketParameters: _*)).mapTo[ActorRef] Await.result(req, timeout.duration) } From be708012899b9014678b3efc928960b0a5fb07d7 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 10 Feb 2012 17:14:05 +0100 Subject: [PATCH 16/39] DOC: Corrections for the MultiJVM doc. See #1801 --- akka-docs/dev/multi-jvm-testing.rst | 46 ++++++++++++++++++++--------- 1 file changed, 32 insertions(+), 14 deletions(-) diff --git a/akka-docs/dev/multi-jvm-testing.rst b/akka-docs/dev/multi-jvm-testing.rst index 85b8ae5027..18154b6c27 100644 --- a/akka-docs/dev/multi-jvm-testing.rst +++ b/akka-docs/dev/multi-jvm-testing.rst @@ -18,28 +18,46 @@ The multi-JVM testing is an sbt plugin that you can find here: http://github.com/typesafehub/sbt-multi-jvm -You can add it as a plugin by adding the following to your plugins/build.sbt:: +You can add it as a plugin by adding the following to your project/plugins.sbt:: resolvers += Classpaths.typesafeResolver addSbtPlugin("com.typesafe.sbtmultijvm" % "sbt-multi-jvm" % "0.1.9") -You can then add multi-JVM testing to a project by including the ``MultiJvm`` +You can then add multi-JVM testing to ``project/Build.scala`` by including the ``MultiJvm`` settings and config. For example, here is how the akka-remote project adds multi-JVM testing:: - import MultiJvmPlugin.{ MultiJvm, extraOptions } + import sbt._ + import Keys._ + import com.typesafe.sbtmultijvm.MultiJvmPlugin + import com.typesafe.sbtmultijvm.MultiJvmPlugin.{ MultiJvm, extraOptions } - lazy val cluster = Project( - id = "akka-remote", - base = file("akka-remote"), - settings = defaultSettings ++ MultiJvmPlugin.settings ++ Seq( - extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => - (name: String) => (src ** (name + ".conf")).get.headOption.map("-Dconfig.file=" + _.absolutePath).toSeq - }, - test in Test <<= (test in Test) dependsOn (test in MultiJvm) + object AkkaBuild extends Build { + + lazy val remote = Project( + id = "akka-remote", + base = file("akka-remote"), + settings = defaultSettings ++ MultiJvmPlugin.settings ++ Seq( + extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => + (name: String) => (src ** (name + ".conf")).get.headOption.map("-Dconfig.file=" + _.absolutePath).toSeq + }, + test in Test <<= (test in Test) dependsOn (test in MultiJvm) + ) + ) configs (MultiJvm) + + lazy val buildSettings = Defaults.defaultSettings ++ Seq( + organization := "com.typesafe.akka", + version := "2.0-SNAPSHOT", + scalaVersion := "2.9.1", + crossPaths := false ) - ) configs (MultiJvm) + + lazy val defaultSettings = buildSettings ++ Seq( + resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/" + ) + + } You can specify JVM options for the forked JVMs:: @@ -87,8 +105,8 @@ options after the test names and ``--``. For example: Creating application tests ========================== -The tests are discovered, and combined, through a naming convention. A test is -named with the following pattern: +The tests are discovered, and combined, through a naming convention. MultiJvm tests are +located in ``src/multi-jvm/scala`` directory. A test is named with the following pattern: .. code-block:: none From 43913b0490a88235748f631896217b38f11a2ca6 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 10 Feb 2012 20:47:59 +0100 Subject: [PATCH 17/39] change IdentityHashComparator to fall back to a real one MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit System.identityHashCode is not guaranteed to be consistent with equals() (cannot be, just imagine more than 2^32 objects); fix it by checking equals in case 0 would be returned and fall back to a real Comparator in case that’s needed. --- .../scala/akka/dispatch/BalancingDispatcher.scala | 10 +++++++--- akka-actor/src/main/scala/akka/util/Helpers.scala | 14 ++++++++++++-- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index a6042046ff..d8274d810a 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -9,7 +9,8 @@ import akka.actor.{ ActorCell, ActorRef } import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet } import annotation.tailrec import java.util.concurrent.atomic.AtomicBoolean -import akka.util.Duration +import akka.util.{ Duration, Helpers } +import java.util.Comparator /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -35,10 +36,13 @@ class BalancingDispatcher( _shutdownTimeout: Duration) extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) { - val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator) + val buddies = new ConcurrentSkipListSet[ActorCell]( + Helpers.identityHashComparator(new Comparator[ActorCell] { + def compare(l: ActorCell, r: ActorCell) = l.self.path compareTo r.self.path + })) val messageQueue: MessageQueue = mailboxType match { - case u: UnboundedMailbox ⇒ new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { + case _: UnboundedMailbox ⇒ new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final val queue = new ConcurrentLinkedQueue[Envelope] } case BoundedMailbox(cap, timeout) ⇒ new QueueBasedMessageQueue with BoundedMessageQueueSemantics { diff --git a/akka-actor/src/main/scala/akka/util/Helpers.scala b/akka-actor/src/main/scala/akka/util/Helpers.scala index 60e6be8b65..25cb279f2e 100644 --- a/akka-actor/src/main/scala/akka/util/Helpers.scala +++ b/akka-actor/src/main/scala/akka/util/Helpers.scala @@ -21,8 +21,18 @@ object Helpers { if (diff > 0) 1 else if (diff < 0) -1 else 0 } - val IdentityHashComparator = new Comparator[AnyRef] { - def compare(a: AnyRef, b: AnyRef): Int = compareIdentityHash(a, b) + /** + * Create a comparator which will efficiently use `System.identityHashCode`, + * unless that happens to be the same for two non-equals objects, in which + * case the supplied “real” comparator is used; the comparator must be + * consistent with equals, otherwise it would not be an enhancement over + * the identityHashCode. + */ + def identityHashComparator[T <: AnyRef](comp: Comparator[T]): Comparator[T] = new Comparator[T] { + def compare(a: T, b: T): Int = compareIdentityHash(a, b) match { + case 0 if a != b ⇒ comp.compare(a, b) + case x ⇒ x + } } final val base64chars = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789+~" From a2ef3eed7eb0e1764a32c3c018250406c126f8c0 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 10 Feb 2012 20:56:52 +0100 Subject: [PATCH 18/39] scaffolding: make debug printout more useful, add assertions --- .../akka/actor/dispatch/ActorModelSpec.scala | 22 ++++++++++++------- .../akka/dispatch/AbstractDispatcher.scala | 2 ++ .../akka/dispatch/BalancingDispatcher.scala | 4 ++-- .../main/scala/akka/dispatch/Mailbox.scala | 5 ++++- 4 files changed, 22 insertions(+), 11 deletions(-) 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 46bf609c7a..4635fc4749 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 @@ -110,8 +110,9 @@ object ActorModelSpec { val stops = new AtomicLong(0) def getStats(actorRef: ActorRef) = { - stats.putIfAbsent(actorRef, new InterceptorStats) match { - case null ⇒ stats.get(actorRef) + val is = new InterceptorStats + stats.putIfAbsent(actorRef, is) match { + case null ⇒ is case other ⇒ other } } @@ -127,12 +128,12 @@ object ActorModelSpec { } protected[akka] abstract override def register(actor: ActorCell) { - getStats(actor.self).registers.incrementAndGet() + assert(getStats(actor.self).registers.incrementAndGet() == 1) super.register(actor) } protected[akka] abstract override def unregister(actor: ActorCell) { - getStats(actor.self).unregisters.incrementAndGet() + assert(getStats(actor.self).unregisters.incrementAndGet() == 1) super.unregister(actor) } @@ -351,7 +352,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa def flood(num: Int) { val cachedMessage = CountDownNStop(new CountDownLatch(num)) val stopLatch = new CountDownLatch(num) - val waitTime = (30 seconds).dilated.toMillis + val waitTime = (20 seconds).dilated.toMillis val boss = system.actorOf(Props(new Actor { def receive = { case "run" ⇒ for (_ ← 1 to num) (context.watch(context.actorOf(props))) ! cachedMessage @@ -368,13 +369,18 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa val buddies = dispatcher.buddies val mq = dispatcher.messageQueue - System.err.println("Buddies left: ") - buddies.toArray foreach { + System.err.println("Buddies left: " + buddies.size + " stopLatch: " + stopLatch.getCount + " inhab:" + dispatcher.inhab) + buddies.toArray sorted new Ordering[AnyRef] { + def compare(l: AnyRef, r: AnyRef) = (l, r) match { + case (ll: ActorCell, rr: ActorCell) ⇒ ll.self.path.toString.compareTo(rr.self.path.toString) + } + } foreach { case cell: ActorCell ⇒ System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain())) } - System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages + " ") + System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages) + Iterator.continually(mq.dequeue) takeWhile (_ ne null) foreach System.err.println case _ ⇒ } diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 1b31be630c..da0e4fdc6e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -260,6 +260,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext mailBox.cleanUp() } + def inhab = inhabitantsUpdater.get(this) + private val shutdownAction = new Runnable { @tailrec final def run() { diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index d8274d810a..4195d0ec61 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -81,11 +81,11 @@ class BalancingDispatcher( protected[akka] override def register(actor: ActorCell) = { super.register(actor) - buddies.add(actor) + assert(buddies.add(actor)) } protected[akka] override def unregister(actor: ActorCell) = { - buddies.remove(actor) + assert(buddies.remove(actor)) super.unregister(actor) } diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index cc15ae2173..32fce8564e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -190,7 +190,10 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue var nextMessage = systemDrain() try { while ((nextMessage ne null) && !isClosed) { - if (debug) println(actor.self + " processing system message " + nextMessage + " with children " + actor.childrenRefs) + if (debug) println(actor.self + " processing system message " + nextMessage + " with " + + (if (actor.childrenRefs.isEmpty) "no children" + else if (actor.childrenRefs.size > 20) actor.childrenRefs.size + " children" + else actor.childrenRefs.mkString("children:\n ", "\n ", ""))) actor systemInvoke nextMessage nextMessage = nextMessage.next // don’t ever execute normal message when system message present! From 880f09be2215c9e8ecacb6cc5e020af570bd2ac9 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 10 Feb 2012 21:29:11 +0100 Subject: [PATCH 19/39] special start-up sequence for actors on BalancingDispatcher Normally the ActorCell would register the actor with the dispatcher (yeah, I moved it into the logical order, because the other one was specifically done for BD but does not work out) and then dispatch the Create() message. This does not work for BD, because then the actor could potentiall process a message before Create() is enqueued, so override systemDispatch() to drop Create() and insert that during register() (which is called from attach()), making sure to achieve the following order: - enqueue Create() - register with dispatcher - add to buddies - schedule mailbox --- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 4 ++-- .../scala/akka/dispatch/BalancingDispatcher.scala | 11 +++++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index aa718e12c8..f4112e5d37 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -290,10 +290,10 @@ private[akka] class ActorCell( // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ parent.sendSystemMessage(akka.dispatch.Supervise(self)) + dispatcher.attach(this) + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ dispatcher.systemDispatch(this, Create()) - - dispatcher.attach(this) } // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 4195d0ec61..4ef7607016 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -79,9 +79,20 @@ class BalancingDispatcher( } } + protected[akka] override def systemDispatch(receiver: ActorCell, invocation: SystemMessage): Unit = + invocation match { + case Create() ⇒ + case x ⇒ super.systemDispatch(receiver, invocation) + } + protected[akka] override def register(actor: ActorCell) = { + val mbox = actor.mailbox + mbox.systemEnqueue(actor.self, Create()) + // must make sure that Create() is the first message enqueued in this mailbox super.register(actor) assert(buddies.add(actor)) + // must make sure that buddy-add is executed before the actor has had a chance to die + registerForExecution(mbox, false, true) } protected[akka] override def unregister(actor: ActorCell) = { From 5a9ec45d01b4cf4554cad9420d45091542ca7611 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 10 Feb 2012 22:16:32 +0100 Subject: [PATCH 20/39] first stab at balancing algorithm: it passes the tests - add new config item "buddy-wakeup-threshold" which defaults to 5 - if BWT>=0, then check mailbox.numberOfMessages in case the target actor was not scheduled during dispatch and schedule a buddie if that is found >=BWT (BWT is a getfield) - if during unregister() there are messages in the queue, schedule a buddie This way people can tune which behavior they want, knowing full well that numberOfMessages is O(n). --- .../scala/akka/actor/dispatch/ActorModelSpec.scala | 3 ++- .../src/test/scala/akka/config/ConfigSpec.scala | 1 + akka-actor/src/main/resources/reference.conf | 7 +++++++ .../scala/akka/dispatch/BalancingDispatcher.scala | 14 ++++++++++---- .../src/main/scala/akka/dispatch/Dispatchers.scala | 3 ++- 5 files changed, 22 insertions(+), 6 deletions(-) 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 4635fc4749..a735e7298b 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 @@ -546,7 +546,8 @@ object BalancingDispatcherModelSpec { Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS), mailboxType, configureExecutor(), - Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS)) with MessageDispatcherInterceptor + Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS), + config.getInt("buddy-wakeup-threshold")) with MessageDispatcherInterceptor override def dispatcher(): MessageDispatcher = instance } 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 dd5149ad8e..13bb3b4f27 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -53,6 +53,7 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) { c.getMilliseconds("shutdown-timeout") must equal(1 * 1000) c.getInt("throughput") must equal(5) c.getMilliseconds("throughput-deadline-time") must equal(0) + c.getInt("buddy-wakeup-threshold") must equal(5) } //Fork join executor config diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index bc52938a7d..b7e0563339 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -246,6 +246,13 @@ akka { # mailbox is used. The Class of the FQCN must have a constructor with a # com.typesafe.config.Config parameter. mailbox-type = "" + + # For BalancingDispatcher: if during message enqueuing the target actor is + # already busy and at least this number of messages is currently in the queue, + # then wake up another actor from the same dispatcher at random. + # Set to -1 to disable (which will also skip the possibly expensive check; + # obtaining the mailbox size is O(n) for the default mailboxes). + buddy-wakeup-threshold = 5 } debug { diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 4ef7607016..63fafef7d1 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -10,7 +10,7 @@ import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, Concur import annotation.tailrec import java.util.concurrent.atomic.AtomicBoolean import akka.util.{ Duration, Helpers } -import java.util.Comparator +import java.util.{ Comparator, Iterator } /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -33,7 +33,8 @@ class BalancingDispatcher( throughputDeadlineTime: Duration, mailboxType: MailboxType, _executorServiceFactoryProvider: ExecutorServiceFactoryProvider, - _shutdownTimeout: Duration) + _shutdownTimeout: Duration, + buddyWakeupThreshold: Int) extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) { val buddies = new ConcurrentSkipListSet[ActorCell]( @@ -98,11 +99,16 @@ class BalancingDispatcher( protected[akka] override def unregister(actor: ActorCell) = { assert(buddies.remove(actor)) super.unregister(actor) + if (messageQueue.hasMessages) registerOne() } override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { messageQueue.enqueue(receiver.self, invocation) - registerForExecution(receiver.mailbox, false, false) - //Somewhere around here we have to make sure that not only the intended actor is kept busy + if (!registerForExecution(receiver.mailbox, false, false) && + buddyWakeupThreshold >= 0 && + messageQueue.numberOfMessages >= buddyWakeupThreshold) registerOne() } + + @tailrec private def registerOne(i: Iterator[ActorCell] = buddies.iterator): Unit = + if (i.hasNext && !registerForExecution(i.next.mailbox, false, false)) registerOne(i) } diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index 8e99e05b06..b9fd3f784b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -189,7 +189,8 @@ class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherP config.getInt("throughput"), Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS), mailboxType, configureExecutor(), - Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS)) + Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS), + config.getInt("buddy-wakeup-threshold")) /** * Returns the same dispatcher instance for each invocation From 49f101d1862bc550d8b0114e50bafd0045b646db Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 12 Feb 2012 19:20:00 +0100 Subject: [PATCH 21/39] #1813 - Adding clarification regarding JMM and Futures --- akka-docs/general/jmm.rst | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/akka-docs/general/jmm.rst b/akka-docs/general/jmm.rst index 7d806f9ac8..3fe94d89db 100644 --- a/akka-docs/general/jmm.rst +++ b/akka-docs/general/jmm.rst @@ -45,6 +45,18 @@ To prevent visibility and reordering problems on actors, Akka guarantees the fol Both rules only apply for the same actor instance and are not valid if different actors are used. +Futures and the Java Memory Model +--------------------------------- + +The completion of a Future "happens before" the invocation of any callbacks registered to it are executed. + +We recommend not to close over non-final fields (final in Java and val in Scala), and if you *do* choose to close over +non-final fields, they must be marked *volatile* in order for the current value of the field to be visible to the callback. + +If you close over a reference, you must also ensure that the instance that is referred to is thread safe. +We highly recommend staying away from objects that use locking, since it can introduce performance problems and in the worst case, deadlocks. +Such are the perils of synchronized. + STM and the Java Memory Model ----------------------------- Akka's Software Transactional Memory (STM) also provides a "happens before" rule: From 1e879b6d03b9d4fb0e66f9d9da3c4d21311edebd Mon Sep 17 00:00:00 2001 From: Henrik Engstrom Date: Sun, 12 Feb 2012 23:09:47 +0100 Subject: [PATCH 22/39] Added HTTP docs with a sample application. See #1538 --- akka-docs/modules/code/Global.scala | 6 + .../akka/docs/http/PlayMiniApplication.scala | 128 ++++++++++++ akka-docs/modules/http.rst | 192 +++++++++++++++++- project/AkkaBuild.scala | 4 +- 4 files changed, 326 insertions(+), 4 deletions(-) create mode 100644 akka-docs/modules/code/Global.scala create mode 100644 akka-docs/modules/code/akka/docs/http/PlayMiniApplication.scala diff --git a/akka-docs/modules/code/Global.scala b/akka-docs/modules/code/Global.scala new file mode 100644 index 0000000000..021e30d5b1 --- /dev/null +++ b/akka-docs/modules/code/Global.scala @@ -0,0 +1,6 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +//#global +object Global extends com.typesafe.play.mini.Setup(akka.docs.http.PlayMiniApplication) +//#global \ No newline at end of file diff --git a/akka-docs/modules/code/akka/docs/http/PlayMiniApplication.scala b/akka-docs/modules/code/akka/docs/http/PlayMiniApplication.scala new file mode 100644 index 0000000000..15c0de0ac5 --- /dev/null +++ b/akka-docs/modules/code/akka/docs/http/PlayMiniApplication.scala @@ -0,0 +1,128 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.http + +//#imports +import com.typesafe.play.mini.{ POST, GET, Path, Application } +import play.api.mvc.{ Action, AsyncResult } +import play.api.mvc.Results._ +import play.api.libs.concurrent._ +import play.api.data._ +import play.api.data.Forms._ +import akka.pattern.ask +import akka.util.Timeout +import akka.util.duration._ +import akka.actor.{ ActorSystem, Props, Actor } +import scala.collection.mutable.{ Map ⇒ MutableMap } +//#imports + +//#playMiniDefinition +object PlayMiniApplication extends Application { + //#playMiniDefinition + private val system = ActorSystem("sample") + //#regexURI + private final val StatementPattern = """/account/statement/(\w+)""".r + //#regexURI + private lazy val accountActor = system.actorOf(Props[AccountActor]) + implicit val timeout = Timeout(1000 milliseconds) + + //#route + def route = { + //#routeLogic + //#simpleGET + case GET(Path("/ping")) ⇒ Action { + Ok("Pong @ " + System.currentTimeMillis) + } + //#simpleGET + //#regexGET + case GET(Path(StatementPattern(accountId))) ⇒ Action { + AsyncResult { + //#innerRegexGET + (accountActor ask Status(accountId)).mapTo[Int].asPromise.map { r ⇒ + if (r >= 0) Ok("Account total: " + r) + else BadRequest("Unknown account: " + accountId) + } + //#innerRegexGET + } + } + //#regexGET + //#asyncDepositPOST + case POST(Path("/account/deposit")) ⇒ Action { implicit request ⇒ + //#formAsyncDepositPOST + val (accountId, amount) = commonForm.bindFromRequest.get + //#formAsyncDepositPOST + AsyncResult { + (accountActor ask Deposit(accountId, amount)).mapTo[Int].asPromise.map { r ⇒ Ok("Updated account total: " + r) } + } + } + //#asyncDepositPOST + //#asyncWithdrawPOST + case POST(Path("/account/withdraw")) ⇒ Action { implicit request ⇒ + val (accountId, amount) = commonForm.bindFromRequest.get + AsyncResult { + (accountActor ask Withdraw(accountId, amount)).mapTo[Int].asPromise.map { r ⇒ + if (r >= 0) Ok("Updated account total: " + r) + else BadRequest("Unknown account or insufficient funds. Get your act together.") + } + } + } + //#asyncWithdrawPOST + //#routeLogic + } + //#route + + //#form + val commonForm = Form( + tuple( + "accountId" -> nonEmptyText, + "amount" -> number(min = 1))) + //#form +} + +//#cases +case class Status(accountId: String) +case class Deposit(accountId: String, amount: Int) +case class Withdraw(accountId: String, amount: Int) +//#cases + +//#actor +class AccountActor extends Actor { + var accounts = MutableMap[String, Int]() + + //#receive + def receive = { + //#senderBang + case Status(accountId) ⇒ sender ! accounts.getOrElse(accountId, -1) + //#senderBang + case Deposit(accountId, amount) ⇒ sender ! deposit(accountId, amount) + case Withdraw(accountId, amount) ⇒ sender ! withdraw(accountId, amount) + } + //#receive + + private def deposit(accountId: String, amount: Int): Int = { + accounts.get(accountId) match { + case Some(value) ⇒ + val newValue = value + amount + accounts += accountId -> newValue + newValue + case None ⇒ + accounts += accountId -> amount + amount + } + } + + private def withdraw(accountId: String, amount: Int): Int = { + accounts.get(accountId) match { + case Some(value) ⇒ + if (value < amount) -1 + else { + val newValue = value - amount + accounts += accountId -> newValue + newValue + } + case None ⇒ -1 + } + } + //#actor +} \ No newline at end of file diff --git a/akka-docs/modules/http.rst b/akka-docs/modules/http.rst index 8388d65702..97978ed5f5 100644 --- a/akka-docs/modules/http.rst +++ b/akka-docs/modules/http.rst @@ -7,8 +7,194 @@ HTTP .. contents:: :local: -Play! ------ +Play2-mini +---------- +The Akka team recommends the `Play2-mini `_ framework when building RESTful +service applications that integrates with Akka. It provides a REST API on top of `Play2 `_. -Akka will recommend using `Play! Mini `_ +Getting started +--------------- + +First you must make your application aware of play-mini. +In SBT you just have to add the following to your _libraryDependencies_:: + + libraryDependencies += "com.typesafe" %% "play-mini" % "2.0-RC1-SNAPSHOT" + +Sample Application +------------------ + +To illustrate how easy it is to wire a RESTful service with Akka we will use a sample application. +The aim of the application is to show how to use play-mini and Akka in combination. Do not put too much +attention on the actual business logic itself, which is a extremely simple bank application, as building a bank +application is a little more complex than what's shown in the sample... + +The application should support the following URL commands: + - GET /ping - returns a Pong message with the time of the server (used to see if the application is up and running) + - GET /account/statement/{accountId} - returns the account statement + - POST /account/deposit - deposits money to an account (and creates a new one if it's not already existing) + - POST /account/withdraw - withdraws money from an account + +Error messages will be returned in case of any misuse of the application, e.g. withdrawing more money than an +account has etc. + +Getting started +--------------- + +To build a play-mini application you first have to make your object extend com.typesafe.play.mini.Application: + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: playMiniDefinition + +The next step is to implement the mandatory method ``route``: + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: route + :exclude: routeLogic + +It is inside the ``route`` method that all the magic happens. +In the sections below we will show how to set up play-mini to handle both GET and POST HTTP calls. + +Simple GET +---------- + +We start off by creating the simplest method we can - a "ping" method: + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: simpleGET + +As you can see in the section above play-mini uses Scala's wonderful pattern matching. +In the snippet we instruct play-mini to reply to all HTTP GET calls with the URI "/ping". +The ``Action`` returned comes from Play! and you can find more information about it `here `_. + +.. _Advanced-GET: + +Advanced GET +------------ + +Let's try something more advanced, retrieving parameters from the URI and also make an asynchronous call to an actor: + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: regexGET + +The regular expression looks like this: + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: regexURI + +In the snippets above we extract a URI parameter with the help of a simple regular expression and then we pass this +parameter on to the underlying actor system. As you can see ``AsyncResult`` is being used. This means that the call to +the actor will be performed asynchronously, i.e. no blocking. + +The asynchronous call to the actor is being done with a ``ask``, e.g.:: + + (accountActor ask Status(accountId)) + +The actor that receives the message returns the result by using a standard *sender !* +as can be seen here: + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: senderBang + +When the result is returned to the calling code we use some mapping code in Play to convert a Akka future to a Play future. +This is shown in this code: + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: innerRegexGET + +In this snippet we check the result to decide what type of response we want to send to the calling client. + +Using HTTP POST +--------------- + +Okay, in the sections above we have shown you how to use play-mini for HTTP GET calls. Let's move on to when the user +posts values to the application. + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: asyncDepositPOST + +As you can see the structure is almost the same as for the :ref:`Advanced-GET`. The difference is that we make the +``request`` parameter ``implicit`` and also that the following line of code is used to extract parameters from the POST. + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: formAsyncDepositPOST + +The code snippet used to map the call to parameters looks like this: + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + :include: form + +Apart from the mapping of parameters the call to the actor looks is done the same as in :ref:`Advanced-GET`. + +The Complete Code Sample +------------------------ + +Below is the complete application in all its beauty. + +Global.scala (/src/main/scala/Global.scala): + +.. includecode:: code/Global.scala + +PlayMiniApplication.scala (/src/main/scala/akka/docs/http/PlayMiniApplication.scala): + +.. includecode:: code/akka/docs/http/PlayMiniApplication.scala + +Build.scala (/project/Build.scala): + +.. code-block:: scala + + import sbt._ + import Keys._ + + object PlayMiniApplicationBuild extends Build { + lazy val root = Project(id = "play-mini-application", base = file("."), settings = Project.defaultSettings).settings( + libraryDependencies += "com.typesafe" %% "play-mini" % "2.0-RC1-SNAPSHOT", + mainClass in (Compile, run) := Some("play.core.server.NettyServer")) + } + +Running the Application +----------------------- + +Firstly, start up the application by opening a command terminal and type:: + + > sbt + > run + +Now you should see something similar to this in your terminal window:: + + [info] Running play.core.server.NettyServer + Play server process ID is 2523 + [info] play - Application started (Prod) + [info] play - Listening for HTTP on port 9000... + +In this example we will use the awesome `cURL `_ command to interact with the application. +Fire up a command terminal and try the application out:: + + First we check the status of a couple of accounts: + > curl http://localhost:9000/account/statement/TheDudesAccount + Unknown account: TheDudesAccount + > curl http://localhost:9000/account/statement/MrLebowskisAccount + Unknown account: MrLebowskisAccount + + Now deposit some money to the accounts: + > curl -d "accountId=TheDudesAccount&amount=1000" http://localhost:9000/account/deposit + Updated account total: 1000 + > curl -d "accountId=MrLebowskisAccount&amount=500" http://localhost:9000/account/deposit + Updated account total: 500 + + Next thing is to check the status of the account: + > curl http://localhost:9000/account/statement/TheDudesAccount + Account total: 1000 + > curl http://localhost:9000/account/statement/MrLebowskisAccount + Account total: 500 + + Fair enough, let's try to withdraw some cash shall we: + > curl -d "accountId=TheDudesAccount&amount=999" http://localhost:9000/account/withdraw + Updated account total: 1 + > curl -d "accountId=MrLebowskisAccount&amount=999" http://localhost:9000/account/withdraw + Unknown account or insufficient funds. Get your act together. + > curl -d "accountId=MrLebowskisAccount&amount=500" http://localhost:9000/account/withdraw + Updated account total: 0 + +Yeah, it works! +Now we leave it to the astute reader of this document to take advantage of the power of play-mini and Akka. \ No newline at end of file diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 7ad4be2a9f..3d2b17b63b 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -473,7 +473,7 @@ object Dependencies { val tutorials = Seq(Test.scalatest, Test.junit) - val docs = Seq(Test.scalatest, Test.junit) + val docs = Seq(Test.scalatest, Test.junit, playMini) val zeroMQ = Seq(Test.scalatest, Test.junit, protobuf, Dependency.zeroMQ) } @@ -497,6 +497,7 @@ object Dependency { val Slf4j = "1.6.4" val Spring = "3.0.5.RELEASE" val Zookeeper = "3.4.0" + val PlayMini = "2.0-RC1-SNAPSHOT" } // Compile @@ -533,6 +534,7 @@ object Dependency { val zookeeper = "org.apache.hadoop.zookeeper" % "zookeeper" % V.Zookeeper // ApacheV2 val zookeeperLock = "org.apache.hadoop.zookeeper" % "zookeeper-recipes-lock" % V.Zookeeper // ApacheV2 val zeroMQ = "org.zeromq" %% "zeromq-scala-binding" % "0.0.3" // ApacheV2 + val playMini = "com.typesafe" % "play-mini_2.9.1" % V.PlayMini // Provided From 0cc34fdb3c3e680a7b42e43a7bdb0a85d84aec3c Mon Sep 17 00:00:00 2001 From: Henrik Engstrom Date: Mon, 13 Feb 2012 08:42:57 +0100 Subject: [PATCH 23/39] Added resolver to Typesafe snapshot repo used for play-mini. See #1538 --- project/AkkaBuild.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 3d2b17b63b..905e345dec 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -348,6 +348,7 @@ object AkkaBuild extends Build { lazy val defaultSettings = baseSettings ++ formatSettings ++ Seq( resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/", resolvers += "Twitter Public Repo" at "http://maven.twttr.com", // This will be going away with com.mongodb.async's next release + resolvers += "Typesafe Snapshot Repo" at "http://repo.typesafe.com/typesafe/snapshots/", // Used while play-mini is still on RC // compile options scalacOptions ++= Seq("-encoding", "UTF-8", "-deprecation", "-unchecked") ++ ( From 251a7cc7e399778d4e2765742ea7dcb69086e64b Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 12:38:59 +0100 Subject: [PATCH 24/39] clean up BalancingDispatcher: - change from messageQueue.numberOfMessages to maintaining an AtomicLong for performance reasons - add comments/scaladoc where missing - remove some assert()s - fix ResiserSpec to employ buddy-wakeup-threshold --- .../akka/actor/dispatch/ActorModelSpec.scala | 4 +- .../test/scala/akka/routing/ResizerSpec.scala | 1 + .../src/main/scala/akka/actor/ActorCell.scala | 5 ++ .../akka/dispatch/AbstractDispatcher.scala | 2 +- .../akka/dispatch/BalancingDispatcher.scala | 55 +++++++++++++++---- .../main/scala/akka/dispatch/Mailbox.scala | 29 +++++++--- 6 files changed, 74 insertions(+), 22 deletions(-) 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 a735e7298b..15886973b2 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 @@ -352,7 +352,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa def flood(num: Int) { val cachedMessage = CountDownNStop(new CountDownLatch(num)) val stopLatch = new CountDownLatch(num) - val waitTime = (20 seconds).dilated.toMillis + val waitTime = (30 seconds).dilated.toMillis val boss = system.actorOf(Props(new Actor { def receive = { case "run" ⇒ for (_ ← 1 to num) (context.watch(context.actorOf(props))) ! cachedMessage @@ -369,7 +369,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa val buddies = dispatcher.buddies val mq = dispatcher.messageQueue - System.err.println("Buddies left: " + buddies.size + " stopLatch: " + stopLatch.getCount + " inhab:" + dispatcher.inhab) + System.err.println("Buddies left: " + buddies.size + " stopLatch: " + stopLatch.getCount + " inhab:" + dispatcher.inhabitants) buddies.toArray sorted new Ordering[AnyRef] { def compare(l: AnyRef, r: AnyRef) = (l, r) match { case (ll: ActorCell, rr: ActorCell) ⇒ ll.self.path.toString.compareTo(rr.self.path.toString) diff --git a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala index 2130afe107..26b5021c18 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala @@ -26,6 +26,7 @@ object ResizerSpec { } bal-disp { type = BalancingDispatcher + buddy-wakeup-threshold = 1 } """ diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index f4112e5d37..9268406086 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -290,6 +290,11 @@ private[akka] class ActorCell( // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ parent.sendSystemMessage(akka.dispatch.Supervise(self)) + /* + * attach before submitting the mailbox for the first time, because + * otherwise the actor could already be dead before the dispatcher is + * informed of its existence (with reversed attach/detach sequence). + */ dispatcher.attach(this) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index da0e4fdc6e..9d1575c4ec 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -260,7 +260,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext mailBox.cleanUp() } - def inhab = inhabitantsUpdater.get(this) + def inhabitants: Long = inhabitantsUpdater.get(this) private val shutdownAction = new Runnable { @tailrec diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 63fafef7d1..70101578f0 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -11,6 +11,8 @@ import annotation.tailrec import java.util.concurrent.atomic.AtomicBoolean import akka.util.{ Duration, Helpers } import java.util.{ Comparator, Iterator } +import akka.util.Unsafe +import java.util.concurrent.atomic.AtomicLong /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -43,18 +45,46 @@ class BalancingDispatcher( })) val messageQueue: MessageQueue = mailboxType match { - case _: UnboundedMailbox ⇒ new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { - final val queue = new ConcurrentLinkedQueue[Envelope] - } - case BoundedMailbox(cap, timeout) ⇒ new QueueBasedMessageQueue with BoundedMessageQueueSemantics { - final val queue = new LinkedBlockingQueue[Envelope](cap) - final val pushTimeOut = timeout - } + case UnboundedMailbox() ⇒ + new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { + final val queue = new ConcurrentLinkedQueue[Envelope] + + override def enqueue(receiver: ActorRef, handle: Envelope) = { + super.enqueue(receiver, handle) + _pressure.getAndIncrement() + } + + override def dequeue(): Envelope = + super.dequeue() match { + case null ⇒ null + case x ⇒ _pressure.getAndDecrement(); x + } + } + + case BoundedMailbox(cap, timeout) ⇒ + new QueueBasedMessageQueue with BoundedMessageQueueSemantics { + final val queue = new LinkedBlockingQueue[Envelope](cap) + final val pushTimeOut = timeout + + override def enqueue(receiver: ActorRef, handle: Envelope) = { + super.enqueue(receiver, handle) + _pressure.getAndIncrement() + } + + override def dequeue(): Envelope = + super.dequeue() match { + case null ⇒ null + case x ⇒ _pressure.getAndDecrement(); x + } + } + case other ⇒ throw new IllegalArgumentException("Only handles BoundedMailbox and UnboundedMailbox, but you specified [" + other + "]") } protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor) + private val _pressure = new AtomicLong + class SharingMailbox(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue { final def enqueue(receiver: ActorRef, handle: Envelope) = messageQueue.enqueue(receiver, handle) @@ -81,6 +111,11 @@ class BalancingDispatcher( } protected[akka] override def systemDispatch(receiver: ActorCell, invocation: SystemMessage): Unit = + /* + * need to filter out Create() messages here because BalancingDispatcher + * already enqueues this within register(), which is called first by the + * ActorCell. + */ invocation match { case Create() ⇒ case x ⇒ super.systemDispatch(receiver, invocation) @@ -91,13 +126,13 @@ class BalancingDispatcher( mbox.systemEnqueue(actor.self, Create()) // must make sure that Create() is the first message enqueued in this mailbox super.register(actor) - assert(buddies.add(actor)) + buddies.add(actor) // must make sure that buddy-add is executed before the actor has had a chance to die registerForExecution(mbox, false, true) } protected[akka] override def unregister(actor: ActorCell) = { - assert(buddies.remove(actor)) + buddies.remove(actor) super.unregister(actor) if (messageQueue.hasMessages) registerOne() } @@ -106,7 +141,7 @@ class BalancingDispatcher( messageQueue.enqueue(receiver.self, invocation) if (!registerForExecution(receiver.mailbox, false, false) && buddyWakeupThreshold >= 0 && - messageQueue.numberOfMessages >= buddyWakeupThreshold) registerOne() + _pressure.get >= buddyWakeupThreshold) registerOne() } @tailrec private def registerOne(i: Iterator[ActorCell] = buddies.iterator): Unit = diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 32fce8564e..4c50cb5c8d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -239,15 +239,26 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue } trait MessageQueue { - /* - * These method need to be implemented in subclasses; they should not rely on the internal stuff above. + /** + * Try to enqueue the message to this queue, or throw an exception. */ - def enqueue(receiver: ActorRef, handle: Envelope) + def enqueue(receiver: ActorRef, handle: Envelope): Unit // NOTE: receiver is used only in two places, but cannot be removed + /** + * Try to dequeue the next message from this queue, return null failing that. + */ def dequeue(): Envelope + /** + * Should return the current number of messages held in this queue; may + * always return 0 if no other value is available efficiently. Do not use + * this for testing for presence of messages, use `hasMessages` instead. + */ def numberOfMessages: Int + /** + * Indicates whether this queue is non-empty. + */ def hasMessages: Boolean } @@ -295,15 +306,15 @@ trait DefaultSystemMessageQueue { self: Mailbox ⇒ } trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue { - final def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle - final def dequeue(): Envelope = queue.poll() + def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle + def dequeue(): Envelope = queue.poll() } trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue { def pushTimeOut: Duration override def queue: BlockingQueue[Envelope] - final def enqueue(receiver: ActorRef, handle: Envelope) { + def enqueue(receiver: ActorRef, handle: Envelope) { if (pushTimeOut.length > 0) { queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || { throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver) @@ -311,13 +322,13 @@ trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue { } else queue put handle } - final def dequeue(): Envelope = queue.poll() + def dequeue(): Envelope = queue.poll() } trait QueueBasedMessageQueue extends MessageQueue { def queue: Queue[Envelope] - final def numberOfMessages = queue.size - final def hasMessages = !queue.isEmpty + def numberOfMessages = queue.size + def hasMessages = !queue.isEmpty } /** From bb40c1ae307bdb5147b4de03efb7f239f8d2d424 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 14:43:20 +0100 Subject: [PATCH 25/39] tweak ResizerSpec to work better with async Resize(), see #1814 - previously relied on resize() being invoked before enqueueing to the mailbox, which is not at all guaranteed any longer. --- .../test/scala/akka/routing/ResizerSpec.scala | 52 +++++++++---------- .../src/main/scala/akka/routing/Routing.scala | 3 +- 2 files changed, 26 insertions(+), 29 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala index 26b5021c18..b9765c8e92 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala @@ -11,6 +11,7 @@ import akka.util.duration._ import akka.actor.ActorRef import java.util.concurrent.atomic.AtomicInteger import akka.pattern.ask +import akka.util.Duration object ResizerSpec { @@ -161,53 +162,48 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with // as influenced by the backlog of blocking pooled actors val resizer = DefaultResizer( - lowerBound = 2, - upperBound = 4, + lowerBound = 3, + upperBound = 5, rampupRate = 0.1, + backoffRate = 0.0, pressureThreshold = 1, messagesPerResize = 1, backoffThreshold = 0.0) val router = system.actorOf(Props(new Actor { def receive = { - case (n: Int, latch: TestLatch, count: AtomicInteger) ⇒ - (n millis).dilated.sleep - count.incrementAndGet - latch.countDown() + case d: Duration ⇒ d.dilated.sleep; sender ! "done" + case "echo" ⇒ sender ! "reply" } }).withRouter(RoundRobinRouter(resizer = Some(resizer)))) // first message should create the minimum number of routees - router ! 1 + router ! "echo" + expectMsg("reply") - Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(2) + def routees(r: ActorRef): Int = { + r ! CurrentRoutees + expectMsgType[RouterRoutees].routees.size + } - def loop(loops: Int, t: Int, latch: TestLatch, count: AtomicInteger) = { - (100 millis).dilated.sleep - for (m ← 0 until loops) { - router.!((t, latch, count)) - (100 millis).dilated.sleep - } + routees(router) must be(3) + + def loop(loops: Int, d: Duration) = { + for (m ← 0 until loops) router ! d + for (m ← 0 until loops) expectMsg(d * 2, "done") } // 2 more should go thru without triggering more - val count1 = new AtomicInteger - val latch1 = TestLatch(2) - loop(2, 200, latch1, count1) - Await.ready(latch1, TestLatch.DefaultTimeout) - count1.get must be(2) + loop(2, 200 millis) - Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(2) + routees(router) must be(3) // a whole bunch should max it out - val count2 = new AtomicInteger - val latch2 = TestLatch(10) - loop(10, 500, latch2, count2) - Await.ready(latch2, TestLatch.DefaultTimeout) - count2.get must be(10) - - Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(4) + loop(4, 500 millis) + awaitCond(routees(router) == 4) + loop(10, 500 millis) + awaitCond(routees(router) == 5) } "backoff" in { @@ -240,7 +236,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with (300 millis).dilated.sleep // let it cool down - for (m ← 0 to 3) { + for (m ← 0 to 5) { router ! 1 (500 millis).dilated.sleep } diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index da2c81d1a7..b050a21b53 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -1028,7 +1028,8 @@ case class DefaultResizer( */ def capacity(routees: IndexedSeq[ActorRef]): Int = { val currentSize = routees.size - val delta = filter(pressure(routees), currentSize) + val press = pressure(routees) + val delta = filter(press, currentSize) val proposed = currentSize + delta if (proposed < lowerBound) delta + (lowerBound - proposed) From 7c57a9d60e6f0f9fb014b61d649f3ab802b5fd89 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 15:33:31 +0100 Subject: [PATCH 26/39] final touch to actor start-up sequence split systemDispatch(Create()) into systemEnqueue(Create()) directly after createMailbox and registerForExecution from within Dispatcher.attach() (resp. CallingThreadDispatcher.register() does its own thing) --- .../src/main/scala/akka/actor/ActorCell.scala | 15 ++++++------ .../akka/dispatch/AbstractDispatcher.scala | 11 ++++++--- .../akka/dispatch/BalancingDispatcher.scala | 24 ++++--------------- .../testkit/CallingThreadDispatcher.scala | 13 +++++++++- 4 files changed, 31 insertions(+), 32 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 9268406086..c22529b2c8 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -285,20 +285,19 @@ private[akka] class ActorCell( final def isTerminated: Boolean = mailbox.isClosed final def start(): Unit = { + /* + * Create the mailbox and enqueue the Create() message to ensure that + * this is processed before anything else. + */ mailbox = dispatcher.createMailbox(this) + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ + mailbox.systemEnqueue(self, Create()) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ parent.sendSystemMessage(akka.dispatch.Supervise(self)) - /* - * attach before submitting the mailbox for the first time, because - * otherwise the actor could already be dead before the dispatcher is - * informed of its existence (with reversed attach/detach sequence). - */ + // This call is expected to start off the actor by scheduling its mailbox. dispatcher.attach(this) - - // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ - dispatcher.systemDispatch(this, Create()) } // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 9d1575c4ec..22eadb55d5 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -185,9 +185,14 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext def id: String /** - * Attaches the specified actor instance to this dispatcher + * Attaches the specified actor instance to this dispatcher, which includes + * scheduling it to run for the first time (Create() is expected to have + * been enqueued by the ActorCell upon mailbox creation). */ - final def attach(actor: ActorCell): Unit = register(actor) + final def attach(actor: ActorCell): Unit = { + register(actor) + registerForExecution(actor.mailbox, false, true) + } /** * Detaches the specified actor instance from this dispatcher @@ -243,7 +248,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext () ⇒ if (inhabitantsUpdater.decrementAndGet(this) == 0) ifSensibleToDoSoThenScheduleShutdown() /** - * If you override it, you must call it. But only ever once. See "attach" for only invocation + * If you override it, you must call it. But only ever once. See "attach" for only invocation. */ protected[akka] def register(actor: ActorCell) { inhabitantsUpdater.incrementAndGet(this) diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 70101578f0..61ac773aa0 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -110,40 +110,24 @@ class BalancingDispatcher( } } - protected[akka] override def systemDispatch(receiver: ActorCell, invocation: SystemMessage): Unit = - /* - * need to filter out Create() messages here because BalancingDispatcher - * already enqueues this within register(), which is called first by the - * ActorCell. - */ - invocation match { - case Create() ⇒ - case x ⇒ super.systemDispatch(receiver, invocation) - } - protected[akka] override def register(actor: ActorCell) = { - val mbox = actor.mailbox - mbox.systemEnqueue(actor.self, Create()) - // must make sure that Create() is the first message enqueued in this mailbox super.register(actor) buddies.add(actor) - // must make sure that buddy-add is executed before the actor has had a chance to die - registerForExecution(mbox, false, true) } protected[akka] override def unregister(actor: ActorCell) = { buddies.remove(actor) super.unregister(actor) - if (messageQueue.hasMessages) registerOne() + if (messageQueue.hasMessages) scheduleOne() } override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { messageQueue.enqueue(receiver.self, invocation) if (!registerForExecution(receiver.mailbox, false, false) && buddyWakeupThreshold >= 0 && - _pressure.get >= buddyWakeupThreshold) registerOne() + _pressure.get >= buddyWakeupThreshold) scheduleOne() } - @tailrec private def registerOne(i: Iterator[ActorCell] = buddies.iterator): Unit = - if (i.hasNext && !registerForExecution(i.next.mailbox, false, false)) registerOne(i) + @tailrec private def scheduleOne(i: Iterator[ActorCell] = buddies.iterator): Unit = + if (i.hasNext && !registerForExecution(i.next.mailbox, false, false)) scheduleOne(i) } diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 8282ee58f5..8b2d15a079 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -8,7 +8,7 @@ import java.util.concurrent.locks.ReentrantLock import java.util.LinkedList import scala.annotation.tailrec import com.typesafe.config.Config -import akka.actor.{ ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorRef, ActorCell } +import akka.actor.{ ActorInitializationException, ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorRef, ActorCell } import akka.dispatch.{ TaskInvocation, SystemMessage, Suspend, Resume, MessageDispatcherConfigurator, MessageDispatcher, Mailbox, Envelope, DispatcherPrerequisites, DefaultSystemMessageQueue } import akka.util.duration.intToDurationInt import akka.util.{ Switch, Duration } @@ -132,6 +132,17 @@ class CallingThreadDispatcher( protected[akka] override def shutdownTimeout = 1 second + protected[akka] override def register(actor: ActorCell): Unit = { + super.register(actor) + actor.mailbox match { + case mbox: CallingThreadMailbox ⇒ + val queue = mbox.queue + queue.enter + runQueue(mbox, queue) + case x ⇒ throw new ActorInitializationException("expected CallingThreadMailbox, got " + x.getClass) + } + } + override def suspend(actor: ActorCell) { actor.mailbox match { case m: CallingThreadMailbox ⇒ m.suspendSwitch.switchOn From 669a4ff9ca04c2422416435dc33a909ad7c5cc76 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 17:46:14 +0100 Subject: [PATCH 27/39] make unbalanced Address() constructor private, fix parsing, see #1806 --- .../scala/akka/actor/ActorLookupSpec.scala | 1 + .../src/main/scala/akka/actor/Address.scala | 38 ++++++++++--------- .../remoting/RemoteDeploymentDocSpec.scala | 2 +- .../remote/netty/NettyRemoteSupport.scala | 2 +- .../main/scala/akka/remote/netty/Server.scala | 4 +- .../akka/remote/RemoteDeployerSpec.scala | 2 +- .../scala/akka/remote/RemoteRouterSpec.scala | 2 +- 7 files changed, 27 insertions(+), 24 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala index 626e413ec8..299cc16679 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala @@ -305,6 +305,7 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout { intercept[MalformedURLException] { ActorPath.fromString("://hallo") } intercept[MalformedURLException] { ActorPath.fromString("s://dd@:12") } intercept[MalformedURLException] { ActorPath.fromString("s://dd@h:hd") } + intercept[MalformedURLException] { ActorPath.fromString("a://l:1/b") } } } diff --git a/akka-actor/src/main/scala/akka/actor/Address.scala b/akka-actor/src/main/scala/akka/actor/Address.scala index d182a0e3b4..502ee56fdb 100644 --- a/akka-actor/src/main/scala/akka/actor/Address.scala +++ b/akka-actor/src/main/scala/akka/actor/Address.scala @@ -15,7 +15,7 @@ import java.net.MalformedURLException * for example a remote transport would want to associate additional * information with an address, then this must be done externally. */ -final case class Address(protocol: String, system: String, host: Option[String], port: Option[Int]) { +final case class Address private (protocol: String, system: String, host: Option[String], port: Option[Int]) { def this(protocol: String, system: String) = this(protocol, system, None, None) def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port)) @@ -62,20 +62,25 @@ object RelativeActorPath { * This object serves as extractor for Scala and as address parser for Java. */ object AddressExtractor { - def unapply(addr: String): Option[Address] = { + def unapply(addr: String): Option[Address] = try { val uri = new URI(addr) - if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None - else { - val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost, - if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost), - if (uri.getPort < 0) None else Some(uri.getPort)) - Some(addr) - } + unapply(uri) } catch { case _: URISyntaxException ⇒ None } - } + + def unapply(uri: URI): Option[Address] = + if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None + else if (uri.getUserInfo == null) { // case 1: “akka://system” + if (uri.getPort != -1) None + else Some(Address(uri.getScheme, uri.getHost)) + } else { // case 2: “akka://system@host:port” + if (uri.getHost == null || uri.getPort == -1) None + else Some( + if (uri.getUserInfo == null) Address(uri.getScheme, uri.getHost) + else Address(uri.getScheme, uri.getUserInfo, uri.getHost, uri.getPort)) + } /** * Try to construct an Address from the given String or throw a java.net.MalformedURLException. @@ -92,18 +97,15 @@ object AddressExtractor { } object ActorPathExtractor { - def unapply(addr: String): Option[(Address, Iterable[String])] = { + def unapply(addr: String): Option[(Address, Iterable[String])] = try { val uri = new URI(addr) - if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null) || uri.getPath == null) None - else { - val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost, - if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost), - if (uri.getPort < 0) None else Some(uri.getPort)) - Some((addr, ActorPath.split(uri.getPath).drop(1))) + if (uri.getPath == null) None + else AddressExtractor.unapply(uri) match { + case None ⇒ None + case Some(addr) ⇒ Some((addr, ActorPath.split(uri.getPath).drop(1))) } } catch { case _: URISyntaxException ⇒ None } - } } \ No newline at end of file diff --git a/akka-docs/scala/code/akka/docs/remoting/RemoteDeploymentDocSpec.scala b/akka-docs/scala/code/akka/docs/remoting/RemoteDeploymentDocSpec.scala index 0a8785444f..a8d75512c5 100644 --- a/akka-docs/scala/code/akka/docs/remoting/RemoteDeploymentDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/remoting/RemoteDeploymentDocSpec.scala @@ -28,7 +28,7 @@ class RemoteDeploymentDocSpec extends AkkaSpec(""" import RemoteDeploymentDocSpec._ val other = ActorSystem("remote", system.settings.config) - val address = other.asInstanceOf[ExtendedActorSystem].provider.getExternalAddressFor(Address("akka", "s", Some("host"), Some(1))).get + val address = other.asInstanceOf[ExtendedActorSystem].provider.getExternalAddressFor(Address("akka", "s", "host", 1)).get override def atTermination() { other.shutdown() } 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 a182948dba..8acd33c7fb 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -61,7 +61,7 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor case sa: InetSocketAddress ⇒ sa case x ⇒ throw new RemoteTransportException("unknown local address type " + x.getClass, null) } - _address.compareAndSet(null, Address("akka", remoteSettings.systemName, Some(settings.Hostname), Some(addr.getPort))) + _address.compareAndSet(null, Address("akka", remoteSettings.systemName, settings.Hostname, addr.getPort)) } def address = _address.get diff --git a/akka-remote/src/main/scala/akka/remote/netty/Server.scala b/akka-remote/src/main/scala/akka/remote/netty/Server.scala index 2c51875e9d..0bafb1c712 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -183,7 +183,7 @@ class RemoteServerHandler( instruction.getCommandType match { case CommandType.CONNECT if settings.UsePassiveConnections ⇒ val origin = instruction.getOrigin - val inbound = Address("akka", origin.getSystem, Some(origin.getHostname), Some(origin.getPort)) + val inbound = Address("akka", origin.getSystem, origin.getHostname, origin.getPort) val client = new PassiveRemoteClient(event.getChannel, netty, inbound) netty.bindClient(inbound, client) case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed @@ -203,7 +203,7 @@ class RemoteServerHandler( private def getClientAddress(c: Channel): Option[Address] = c.getRemoteAddress match { - case inet: InetSocketAddress ⇒ Some(Address("akka", "unknown(yet)", Some(inet.getAddress.toString), Some(inet.getPort))) + case inet: InetSocketAddress ⇒ Some(Address("akka", "unknown(yet)", inet.getAddress.toString, inet.getPort)) case _ ⇒ None } } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala index 1b250f1ea9..57d240e8d8 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala @@ -42,7 +42,7 @@ class RemoteDeployerSpec extends AkkaSpec(RemoteDeployerSpec.deployerConf) { service, deployment.get.config, RoundRobinRouter(3), - RemoteScope(Address("akka", "sys", Some("wallace"), Some(2552)))))) + RemoteScope(Address("akka", "sys", "wallace", 2552))))) } } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala index bcabd85098..bf79caf847 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala @@ -160,7 +160,7 @@ akka.actor.deployment { children must have size 2 val parents = children.map(_.parent) parents must have size 1 - parents.head.address must be(Address("akka", "remote_sys", Some("localhost"), Some(12347))) + parents.head.address must be(Address("akka", "remote_sys", "localhost", 12347)) children foreach (_.address.toString must be === "akka://remote_sys@localhost:12347") system.stop(router) } From 607ec4c2cf6b7a4a57665dace3252c0562c35d22 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 13 Feb 2012 18:14:35 +0100 Subject: [PATCH 28/39] Switching approaches to check for max throttle --- .../akka/actor/dispatch/ActorModelSpec.scala | 2 +- akka-actor/src/main/resources/reference.conf | 10 ++--- .../akka/dispatch/AbstractDispatcher.scala | 7 ++- .../akka/dispatch/BalancingDispatcher.scala | 44 +++++-------------- .../main/scala/akka/dispatch/Dispatcher.scala | 7 ++- .../scala/akka/dispatch/Dispatchers.scala | 2 +- .../akka/dispatch/ThreadPoolBuilder.scala | 8 ++-- 7 files changed, 30 insertions(+), 50 deletions(-) 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 15886973b2..deafb9cdc1 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 @@ -547,7 +547,7 @@ object BalancingDispatcherModelSpec { mailboxType, configureExecutor(), Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS), - config.getInt("buddy-wakeup-threshold")) with MessageDispatcherInterceptor + config.getBoolean("attempt-teamwork")) with MessageDispatcherInterceptor override def dispatcher(): MessageDispatcher = instance } diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index b7e0563339..9e0ee70ff1 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -247,12 +247,10 @@ akka { # com.typesafe.config.Config parameter. mailbox-type = "" - # For BalancingDispatcher: if during message enqueuing the target actor is - # already busy and at least this number of messages is currently in the queue, - # then wake up another actor from the same dispatcher at random. - # Set to -1 to disable (which will also skip the possibly expensive check; - # obtaining the mailbox size is O(n) for the default mailboxes). - buddy-wakeup-threshold = 5 + # For BalancingDispatcher: If the balancing dispatcher should attempt to + # schedule idle actors using the same dispatcher when a message comes in, + # and the dispatchers ExecutorService is not fully busy already. + attempt-teamwork = on } debug { diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 22eadb55d5..6046e249af 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -156,7 +156,10 @@ trait ExecutionContext { * log the problem or whatever is appropriate for the implementation. */ def reportFailure(t: Throwable): Unit +} +private[akka] trait LoadMetrics { self: Executor ⇒ + def atFullThrottle(): Boolean } object MessageDispatcher { @@ -447,11 +450,13 @@ object ForkJoinExecutorConfigurator { final class AkkaForkJoinPool(parallelism: Int, threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, unhandledExceptionHandler: Thread.UncaughtExceptionHandler) - extends ForkJoinPool(parallelism, threadFactory, unhandledExceptionHandler, true) { + extends ForkJoinPool(parallelism, threadFactory, unhandledExceptionHandler, true) with LoadMetrics { override def execute(r: Runnable): Unit = r match { case m: Mailbox ⇒ super.execute(new MailboxExecutionTask(m)) case other ⇒ super.execute(other) } + + def atFullThrottle(): Boolean = this.getActiveThreadCount() >= this.getParallelism() } /** diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 61ac773aa0..d2d978341c 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -4,15 +4,11 @@ package akka.dispatch -import util.DynamicVariable import akka.actor.{ ActorCell, ActorRef } -import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet } import annotation.tailrec -import java.util.concurrent.atomic.AtomicBoolean import akka.util.{ Duration, Helpers } import java.util.{ Comparator, Iterator } -import akka.util.Unsafe -import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.{ Executor, LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet } /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -36,7 +32,7 @@ class BalancingDispatcher( mailboxType: MailboxType, _executorServiceFactoryProvider: ExecutorServiceFactoryProvider, _shutdownTimeout: Duration, - buddyWakeupThreshold: Int) + attemptTeamWork: Boolean) extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) { val buddies = new ConcurrentSkipListSet[ActorCell]( @@ -48,34 +44,12 @@ class BalancingDispatcher( case UnboundedMailbox() ⇒ new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final val queue = new ConcurrentLinkedQueue[Envelope] - - override def enqueue(receiver: ActorRef, handle: Envelope) = { - super.enqueue(receiver, handle) - _pressure.getAndIncrement() - } - - override def dequeue(): Envelope = - super.dequeue() match { - case null ⇒ null - case x ⇒ _pressure.getAndDecrement(); x - } } case BoundedMailbox(cap, timeout) ⇒ new QueueBasedMessageQueue with BoundedMessageQueueSemantics { final val queue = new LinkedBlockingQueue[Envelope](cap) final val pushTimeOut = timeout - - override def enqueue(receiver: ActorRef, handle: Envelope) = { - super.enqueue(receiver, handle) - _pressure.getAndIncrement() - } - - override def dequeue(): Envelope = - super.dequeue() match { - case null ⇒ null - case x ⇒ _pressure.getAndDecrement(); x - } } case other ⇒ throw new IllegalArgumentException("Only handles BoundedMailbox and UnboundedMailbox, but you specified [" + other + "]") @@ -83,8 +57,6 @@ class BalancingDispatcher( protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor) - private val _pressure = new AtomicLong - class SharingMailbox(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue { final def enqueue(receiver: ActorRef, handle: Envelope) = messageQueue.enqueue(receiver, handle) @@ -123,11 +95,15 @@ class BalancingDispatcher( override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { messageQueue.enqueue(receiver.self, invocation) - if (!registerForExecution(receiver.mailbox, false, false) && - buddyWakeupThreshold >= 0 && - _pressure.get >= buddyWakeupThreshold) scheduleOne() + if (!registerForExecution(receiver.mailbox, false, false) && doTeamWork) scheduleOne() } + protected def doTeamWork(): Boolean = + attemptTeamWork && (executorService.get().executor match { + case lm: LoadMetrics ⇒ lm.atFullThrottle == false + case other ⇒ true + }) + @tailrec private def scheduleOne(i: Iterator[ActorCell] = buddies.iterator): Unit = if (i.hasNext && !registerForExecution(i.next.mailbox, false, false)) scheduleOne(i) -} +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index a735ea367e..2046f02286 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -32,12 +32,11 @@ class Dispatcher( val shutdownTimeout: Duration) extends MessageDispatcher(_prerequisites) { - protected[akka] val executorServiceFactory: ExecutorServiceFactory = + protected val executorServiceFactory: ExecutorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(id, prerequisites.threadFactory) - protected[akka] val executorService = new AtomicReference[ExecutorService](new ExecutorServiceDelegate { - lazy val executor = executorServiceFactory.createExecutorService - }) + protected val executorService = new AtomicReference[ExecutorServiceDelegate]( + new ExecutorServiceDelegate { lazy val executor = executorServiceFactory.createExecutorService }) protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { val mbox = receiver.mailbox diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index b9fd3f784b..5f4528146d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -190,7 +190,7 @@ class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherP Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS), mailboxType, configureExecutor(), Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS), - config.getInt("buddy-wakeup-threshold")) + config.getBoolean("attempt-teamwork")) /** * Returns the same dispatcher instance for each invocation diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index 1c63831013..b6fd432296 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -81,14 +81,16 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def extends ExecutorServiceFactoryProvider { class ThreadPoolExecutorServiceFactory(val threadFactory: ThreadFactory) extends ExecutorServiceFactory { def createExecutorService: ExecutorService = { - val service = new ThreadPoolExecutor( + val service: ThreadPoolExecutor = new ThreadPoolExecutor( corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory, - rejectionPolicy) + rejectionPolicy) with LoadMetrics { + def atFullThrottle(): Boolean = this.getActiveCount >= this.getPoolSize + } service.allowCoreThreadTimeOut(allowCorePoolTimeout) service } @@ -182,7 +184,7 @@ case class MonitorableThreadFactory(name: String, protected def wire[T <: Thread](t: T): T = { t.setUncaughtExceptionHandler(exceptionHandler) t.setDaemon(daemonic) - contextClassLoader foreach (t.setContextClassLoader(_)) + contextClassLoader foreach t.setContextClassLoader t } } From 14d180c92ad2cf97b2f1d56f28bcea9d84fdc0df Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 20:16:54 +0100 Subject: [PATCH 29/39] =?UTF-8?q?unborking=20compile=20in=20akka-cluster?= =?UTF-8?q?=20(sorry=20=E2=80=A6)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../test/scala/akka/cluster/AccrualFailureDetectorSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala index f611fc9812..4aab105273 100644 --- a/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala @@ -9,7 +9,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec(""" """) { "An AccrualFailureDetector" must { - val conn = Address("akka", "", Some("localhost"), Some(2552)) + val conn = Address("akka", "", "localhost", 2552) "mark node as available after a series of successful heartbeats" in { val fd = new AccrualFailureDetector(system) From 89cf7aa2f003740d0c53e2c30f3b0fcc149a4ae9 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 20:16:54 +0100 Subject: [PATCH 30/39] =?UTF-8?q?unborking=20compile=20in=20akka-cluster?= =?UTF-8?q?=20(sorry=20=E2=80=A6)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../test/scala/akka/cluster/AccrualFailureDetectorSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala index f611fc9812..4aab105273 100644 --- a/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala @@ -9,7 +9,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec(""" """) { "An AccrualFailureDetector" must { - val conn = Address("akka", "", Some("localhost"), Some(2552)) + val conn = Address("akka", "", "localhost", 2552) "mark node as available after a series of successful heartbeats" in { val fd = new AccrualFailureDetector(system) From 11f067abfc8817cad13a6ed407b1b310381bff6d Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 20:42:14 +0100 Subject: [PATCH 31/39] fix DefaultResizer.pressure, make ResizerSpec less flaky MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - also clean up left-over reference to “buddy-wakeup-threshold” --- .../src/test/scala/akka/config/ConfigSpec.scala | 2 +- .../src/test/scala/akka/routing/ResizerSpec.scala | 7 +++---- akka-actor/src/main/scala/akka/routing/Routing.scala | 2 +- 3 files changed, 5 insertions(+), 6 deletions(-) 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 13bb3b4f27..127907412e 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -53,7 +53,7 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) { c.getMilliseconds("shutdown-timeout") must equal(1 * 1000) c.getInt("throughput") must equal(5) c.getMilliseconds("throughput-deadline-time") must equal(0) - c.getInt("buddy-wakeup-threshold") must equal(5) + c.getBoolean("attempt-teamwork") must equal(true) } //Fork join executor config diff --git a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala index b9765c8e92..1f78c64edf 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala @@ -27,7 +27,6 @@ object ResizerSpec { } bal-disp { type = BalancingDispatcher - buddy-wakeup-threshold = 1 } """ @@ -190,7 +189,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with def loop(loops: Int, d: Duration) = { for (m ← 0 until loops) router ! d - for (m ← 0 until loops) expectMsg(d * 2, "done") + for (m ← 0 until loops) expectMsg(d * 3, "done") } // 2 more should go thru without triggering more @@ -199,8 +198,8 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with routees(router) must be(3) // a whole bunch should max it out - loop(4, 500 millis) - awaitCond(routees(router) == 4) + loop(10, 500 millis) + awaitCond(routees(router) > 3) loop(10, 500 millis) awaitCond(routees(router) == 5) diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index b050a21b53..4ff6609255 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -1059,7 +1059,7 @@ case class DefaultResizer( case a: LocalActorRef ⇒ val cell = a.underlying pressureThreshold match { - case 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null + case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.hasMessages case i if i < 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null case threshold ⇒ cell.mailbox.numberOfMessages >= threshold } From c0ec1b994f7d57d5bb5d825c9ff1d0bac38d8488 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 20:48:25 +0100 Subject: [PATCH 32/39] fix AddressExtractor (from URI) --- akka-actor/src/main/scala/akka/actor/Address.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/Address.scala b/akka-actor/src/main/scala/akka/actor/Address.scala index 502ee56fdb..e5335bcac1 100644 --- a/akka-actor/src/main/scala/akka/actor/Address.scala +++ b/akka-actor/src/main/scala/akka/actor/Address.scala @@ -71,7 +71,8 @@ object AddressExtractor { } def unapply(uri: URI): Option[Address] = - if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None + if (uri eq null) None + else if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None else if (uri.getUserInfo == null) { // case 1: “akka://system” if (uri.getPort != -1) None else Some(Address(uri.getScheme, uri.getHost)) From 0f314582b852a84e7a3dda4976a8351ff0245052 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 13 Feb 2012 21:37:30 +0100 Subject: [PATCH 33/39] Sprinkling magic dust and webscale sauce --- .../akka/dispatch/BalancingDispatcher.scala | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index d2d978341c..7ac06da5ad 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -90,20 +90,23 @@ class BalancingDispatcher( protected[akka] override def unregister(actor: ActorCell) = { buddies.remove(actor) super.unregister(actor) - if (messageQueue.hasMessages) scheduleOne() + scheduleOne() } override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { messageQueue.enqueue(receiver.self, invocation) - if (!registerForExecution(receiver.mailbox, false, false) && doTeamWork) scheduleOne() + registerForExecution(receiver.mailbox, false, false) + scheduleOne() } - protected def doTeamWork(): Boolean = - attemptTeamWork && (executorService.get().executor match { - case lm: LoadMetrics ⇒ lm.atFullThrottle == false - case other ⇒ true - }) - @tailrec private def scheduleOne(i: Iterator[ActorCell] = buddies.iterator): Unit = - if (i.hasNext && !registerForExecution(i.next.mailbox, false, false)) scheduleOne(i) + if (attemptTeamWork + && messageQueue.hasMessages + && i.hasNext + && (executorService.get().executor match { + case lm: LoadMetrics ⇒ lm.atFullThrottle == false + case other ⇒ true + }) + && !registerForExecution(i.next.mailbox, false, false)) + scheduleOne(i) } \ No newline at end of file From 498011815980828415eac11ff19e09a41a40d91f Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 13 Feb 2012 22:04:39 +0100 Subject: [PATCH 34/39] Renaming buddies to team and then optimized usage of the teamwork --- .../akka/actor/dispatch/ActorModelSpec.scala | 10 +++--- .../akka/dispatch/BalancingDispatcher.scala | 35 ++++++++++--------- 2 files changed, 22 insertions(+), 23 deletions(-) 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 deafb9cdc1..88358e9f16 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 @@ -366,14 +366,12 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa case e ⇒ dispatcher match { case dispatcher: BalancingDispatcher ⇒ - val buddies = dispatcher.buddies + val team = dispatcher.team val mq = dispatcher.messageQueue - System.err.println("Buddies left: " + buddies.size + " stopLatch: " + stopLatch.getCount + " inhab:" + dispatcher.inhabitants) - buddies.toArray sorted new Ordering[AnyRef] { - def compare(l: AnyRef, r: AnyRef) = (l, r) match { - case (ll: ActorCell, rr: ActorCell) ⇒ ll.self.path.toString.compareTo(rr.self.path.toString) - } + System.err.println("Teammates left: " + team.size + " stopLatch: " + stopLatch.getCount + " inhab:" + dispatcher.inhabitants) + team.toArray sorted new Ordering[AnyRef] { + def compare(l: AnyRef, r: AnyRef) = (l, r) match { case (ll: ActorCell, rr: ActorCell) ⇒ ll.self.path compareTo rr.self.path } } foreach { case cell: ActorCell ⇒ System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain())) diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 7ac06da5ad..46701848c5 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -35,7 +35,7 @@ class BalancingDispatcher( attemptTeamWork: Boolean) extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) { - val buddies = new ConcurrentSkipListSet[ActorCell]( + val team = new ConcurrentSkipListSet[ActorCell]( Helpers.identityHashComparator(new Comparator[ActorCell] { def compare(l: ActorCell, r: ActorCell) = l.self.path compareTo r.self.path })) @@ -82,31 +82,32 @@ class BalancingDispatcher( } } - protected[akka] override def register(actor: ActorCell) = { + protected[akka] override def register(actor: ActorCell): Unit = { super.register(actor) - buddies.add(actor) + team.add(actor) } - protected[akka] override def unregister(actor: ActorCell) = { - buddies.remove(actor) + protected[akka] override def unregister(actor: ActorCell): Unit = { + team.remove(actor) super.unregister(actor) - scheduleOne() + teamWork() } override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { messageQueue.enqueue(receiver.self, invocation) registerForExecution(receiver.mailbox, false, false) - scheduleOne() + teamWork() } - @tailrec private def scheduleOne(i: Iterator[ActorCell] = buddies.iterator): Unit = - if (attemptTeamWork - && messageQueue.hasMessages - && i.hasNext - && (executorService.get().executor match { - case lm: LoadMetrics ⇒ lm.atFullThrottle == false - case other ⇒ true - }) - && !registerForExecution(i.next.mailbox, false, false)) - scheduleOne(i) + protected def teamWork(): Unit = if (attemptTeamWork) { + @tailrec def scheduleOne(i: Iterator[ActorCell] = team.iterator): Unit = + if (messageQueue.hasMessages + && i.hasNext + && (executorService.get().executor match { + case lm: LoadMetrics ⇒ lm.atFullThrottle == false + case other ⇒ true + }) + && !registerForExecution(i.next.mailbox, false, false)) + scheduleOne(i) + } } \ No newline at end of file From 060633331a1f8e58c9f0258a50bf5145ebe9d900 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 13 Feb 2012 22:07:12 +0100 Subject: [PATCH 35/39] Restoring the order of the universe --- .../src/main/scala/akka/dispatch/BalancingDispatcher.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 46701848c5..ec675f7b5b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -109,5 +109,7 @@ class BalancingDispatcher( }) && !registerForExecution(i.next.mailbox, false, false)) scheduleOne(i) + + scheduleOne() } } \ No newline at end of file From 68d0e2612d5df7dce817310812f3a2c9ce61bb6f Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 13 Feb 2012 22:09:57 +0100 Subject: [PATCH 36/39] Performance optimization for BalancingDispatcher --- .../src/main/scala/akka/dispatch/BalancingDispatcher.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index ec675f7b5b..a7f03db5a0 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -95,8 +95,7 @@ class BalancingDispatcher( override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { messageQueue.enqueue(receiver.self, invocation) - registerForExecution(receiver.mailbox, false, false) - teamWork() + if (!registerForExecution(receiver.mailbox, false, false)) teamWork() } protected def teamWork(): Unit = if (attemptTeamWork) { From 8a20a375687ab6b59ee3f3bd76b5ae49769ff9d8 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 13 Feb 2012 22:27:33 +0100 Subject: [PATCH 37/39] Nulling out the actor field when the actor terminates --- .../src/main/scala/akka/actor/ActorCell.scala | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index c22529b2c8..cfcce7ab2c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -388,8 +388,6 @@ private[akka] class ActorCell( failedActor.preRestart(cause, if (c ne null) Some(c.message) else None) } finally { clearActorFields() - currentMessage = null - actor = null } } actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call @@ -559,9 +557,8 @@ private[akka] class ActorCell( if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped")) // FIXME: can actor be null? } finally { - currentMessage = null - clearActorFields() if (a ne null) a.clearBehaviorStack() + clearActorFields() } } } @@ -601,7 +598,11 @@ private[akka] class ActorCell( } } - final def clearActorFields(): Unit = setActorFields(context = null, self = system.deadLetters) + final def clearActorFields(): Unit = { + setActorFields(context = null, self = system.deadLetters) + currentMessage = null + actor = null + } final def setActorFields(context: ActorContext, self: ActorRef) { @tailrec @@ -629,8 +630,5 @@ private[akka] class ActorCell( } } - private def clazz(o: AnyRef): Class[_] = { - if (o eq null) this.getClass - else o.getClass - } + private final def clazz(o: AnyRef): Class[_] = if (o eq null) this.getClass else o.getClass } From c59e1381ab3f7c4a3afb9dd240ef73345ff147fa Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 22:38:18 +0100 Subject: [PATCH 38/39] update to latest config lib --- akka-actor/src/main/java/com/typesafe/config/Config.java | 9 +++++---- .../src/main/java/com/typesafe/config/ConfigFactory.java | 4 ++-- .../com/typesafe/config/impl/AbstractConfigObject.java | 3 +++ .../com/typesafe/config/impl/AbstractConfigValue.java | 2 ++ .../java/com/typesafe/config/impl/ConfigBoolean.java | 2 ++ .../main/java/com/typesafe/config/impl/ConfigDouble.java | 2 ++ .../main/java/com/typesafe/config/impl/ConfigInt.java | 2 ++ .../main/java/com/typesafe/config/impl/ConfigLong.java | 2 ++ .../main/java/com/typesafe/config/impl/ConfigNull.java | 2 ++ .../main/java/com/typesafe/config/impl/ConfigNumber.java | 3 +++ .../main/java/com/typesafe/config/impl/ConfigString.java | 2 ++ .../src/main/java/com/typesafe/config/impl/Path.java | 2 ++ .../main/java/com/typesafe/config/impl/SimpleConfig.java | 2 ++ .../com/typesafe/config/impl/SimpleConfigObject.java | 2 ++ .../com/typesafe/config/impl/SimpleConfigOrigin.java | 3 +++ .../com/typesafe/config/impl/SubstitutionExpression.java | 2 ++ 16 files changed, 38 insertions(+), 6 deletions(-) diff --git a/akka-actor/src/main/java/com/typesafe/config/Config.java b/akka-actor/src/main/java/com/typesafe/config/Config.java index e820d1b369..febeece3e2 100644 --- a/akka-actor/src/main/java/com/typesafe/config/Config.java +++ b/akka-actor/src/main/java/com/typesafe/config/Config.java @@ -378,10 +378,11 @@ public interface Config extends ConfigMergeable { Object getAnyRef(String path); /** - * Gets the value at the given path, unless the value is a null value or - * missing, in which case it throws just like the other getters. Use - * {@code get()} from the {@link java.util.Map Map} interface if you want an - * unprocessed value. + * Gets the value at the given path, unless the value is a + * null value or missing, in which case it throws just like + * the other getters. Use {@code get()} on the {@link + * Config#root()} object (or other object in the tree) if you + * want an unprocessed value. * * @param path * path expression diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java b/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java index bb3f0aa131..1b819ab4c5 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java @@ -161,8 +161,8 @@ public final class ConfigFactory { * used rather than the default * application.{conf,json,properties} classpath resources. * These system properties should not be set in code (after all, you can - * just parse whatever you want manually and then use {@link #load(Config) - * if you don't want to use application.conf}). The properties + * just parse whatever you want manually and then use {@link #load(Config)} + * if you don't want to use application.conf). The properties * are intended for use by the person or script launching the application. * For example someone might have a production.conf that * include application.conf but then change a couple of values. diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java index 1eb31c6397..6f9fd542ff 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java @@ -23,6 +23,9 @@ import com.typesafe.config.ConfigValueType; abstract class AbstractConfigObject extends AbstractConfigValue implements ConfigObject { + + private static final long serialVersionUID = 1L; + final private SimpleConfig config; protected AbstractConfigObject(ConfigOrigin origin) { diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java index 642fb7ae26..7eea5f0a8a 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java @@ -20,6 +20,8 @@ import com.typesafe.config.ConfigValue; */ abstract class AbstractConfigValue implements ConfigValue, MergeableValue, Serializable { + private static final long serialVersionUID = 1L; + final private SimpleConfigOrigin origin; AbstractConfigValue(ConfigOrigin origin) { diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java index a1a7677c78..5a5c0f8ffd 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java @@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType; final class ConfigBoolean extends AbstractConfigValue { + private static final long serialVersionUID = 1L; + final private boolean value; ConfigBoolean(ConfigOrigin origin, boolean value) { diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java index 65cfb5d966..8b99db9640 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java @@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType; final class ConfigDouble extends ConfigNumber { + private static final long serialVersionUID = 1L; + final private double value; ConfigDouble(ConfigOrigin origin, double value, String originalText) { diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java index d33bd21274..8ccb3f872a 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java @@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType; final class ConfigInt extends ConfigNumber { + private static final long serialVersionUID = 1L; + final private int value; ConfigInt(ConfigOrigin origin, int value, String originalText) { diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java index a2e2ab3228..6e1a5073fb 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java @@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType; final class ConfigLong extends ConfigNumber { + private static final long serialVersionUID = 1L; + final private long value; ConfigLong(ConfigOrigin origin, long value, String originalText) { diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java index 91b8586674..1dcd32b5b0 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java @@ -16,6 +16,8 @@ import com.typesafe.config.ConfigValueType; */ final class ConfigNull extends AbstractConfigValue { + private static final long serialVersionUID = 1L; + ConfigNull(ConfigOrigin origin) { super(origin); } diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java index d2f4bb6ac4..8f6996d77b 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java @@ -7,6 +7,9 @@ import com.typesafe.config.ConfigException; import com.typesafe.config.ConfigOrigin; abstract class ConfigNumber extends AbstractConfigValue { + + private static final long serialVersionUID = 1L; + // This is so when we concatenate a number into a string (say it appears in // a sentence) we always have it exactly as the person typed it into the // config file. It's purely cosmetic; equals/hashCode don't consider this diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java index f5293c8fc7..4bdce3b129 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java @@ -8,6 +8,8 @@ import com.typesafe.config.ConfigValueType; final class ConfigString extends AbstractConfigValue { + private static final long serialVersionUID = 1L; + final private String value; ConfigString(ConfigOrigin origin, String value) { diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Path.java b/akka-actor/src/main/java/com/typesafe/config/impl/Path.java index cb0d5a423b..fbbe1e0874 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Path.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/Path.java @@ -11,6 +11,8 @@ import com.typesafe.config.ConfigException; final class Path implements Serializable { + private static final long serialVersionUID = 1L; + final private String first; final private Path remainder; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java index fdcf540816..daa9f4c368 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java @@ -31,6 +31,8 @@ import com.typesafe.config.ConfigValueType; */ final class SimpleConfig implements Config, MergeableValue, Serializable { + private static final long serialVersionUID = 1L; + final private AbstractConfigObject object; SimpleConfig(AbstractConfigObject object) { diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java index c961ae6b34..c4b8cf461a 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java @@ -17,6 +17,8 @@ import com.typesafe.config.ConfigValue; final class SimpleConfigObject extends AbstractConfigObject { + private static final long serialVersionUID = 1L; + // this map should never be modified - assume immutable final private Map value; final private boolean resolved; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java index 962788dcc4..23351c1e8e 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java @@ -19,6 +19,9 @@ import com.typesafe.config.ConfigOrigin; // it would be cleaner to have a class hierarchy for various origin types, // but was hoping this would be enough simpler to be a little messy. eh. final class SimpleConfigOrigin implements ConfigOrigin, Serializable { + + private static final long serialVersionUID = 1L; + final private String description; final private int lineNumber; final private int endLineNumber; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java b/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java index 95e121e8b0..be67073a1d 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java @@ -4,6 +4,8 @@ import java.io.Serializable; final class SubstitutionExpression implements Serializable { + private static final long serialVersionUID = 1L; + final private Path path; final private boolean optional; From c316cd4e0b47f35b9d4e16ca9122980b834b59ff Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 13 Feb 2012 22:53:38 +0100 Subject: [PATCH 39/39] =?UTF-8?q?don=E2=80=99t=20mask=20origin=20NPE=20whe?= =?UTF-8?q?n=20invoking=20actor?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 2 +- 1 file changed, 1 insertion(+), 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 cfcce7ab2c..a5331e9f49 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -495,7 +495,7 @@ private[akka] class ActorCell( dispatcher.reportFailure(new LogEventException(Error(e, self.path.toString, clazz(actor), e.getMessage), e)) // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) - actor.supervisorStrategy.handleSupervisorFailing(self, children) + if (actor ne null) actor.supervisorStrategy.handleSupervisorFailing(self, children) parent.tell(Failed(e), self) } finally { checkReceiveTimeout // Reschedule receive timeout