From a607fbd517e79355ae1f6c6d40e1482a3fc0ac46 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 6 Feb 2012 12:10:37 +0100 Subject: [PATCH 01/34] Adding type parameter to onComplete for better type inference. Making complete throw exception on race. Minor code improvements inside Future --- .../src/main/scala/akka/dispatch/Future.scala | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index c781595aeb..e3821e8334 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -14,12 +14,12 @@ import java.util.{ LinkedList ⇒ JLinkedList } import scala.annotation.tailrec import scala.collection.mutable.Stack import akka.util.{ Duration, BoxedType } -import java.util.concurrent.atomic.{ AtomicReferenceFieldUpdater, AtomicInteger } import akka.dispatch.Await.CanAwait import java.util.concurrent._ import akka.util.NonFatal import akka.event.Logging.LogEventException import akka.event.Logging.Debug +import atomic.{ AtomicInteger, AtomicReferenceFieldUpdater } object Await { @@ -192,7 +192,7 @@ object Future { def firstCompletedOf[T](futures: Traversable[Future[T]])(implicit executor: ExecutionContext): Future[T] = { val futureResult = Promise[T]() - val completeFirst: Either[Throwable, T] ⇒ Unit = futureResult complete _ + val completeFirst: Either[Throwable, T] ⇒ Unit = futureResult tryComplete _ futures.foreach(_ onComplete completeFirst) futureResult @@ -208,12 +208,12 @@ object Future { val ref = new AtomicInteger(futures.size) val search: Either[Throwable, T] ⇒ Unit = v ⇒ try { v match { - case Right(r) ⇒ if (predicate(r)) result success Some(r) + case Right(r) ⇒ if (predicate(r)) result tryComplete Right(Some(r)) case _ ⇒ } } finally { if (ref.decrementAndGet == 0) - result success None + result tryComplete Right(None) } futures.foreach(_ onComplete search) @@ -279,13 +279,13 @@ object Future { * The Delimited Continuations compiler plugin must be enabled in order to use this method. */ def flow[A](body: ⇒ A @cps[Future[Any]])(implicit executor: ExecutionContext): Future[A] = { - val future = Promise[A] + val p = Promise[A] dispatchTask({ () ⇒ - (reify(body) foreachFull (future success, future failure): Future[Any]) onFailure { - case e: Exception ⇒ future failure e + (reify(body) foreachFull (p success, p failure): Future[Any]) onFailure { + case NonFatal(e) ⇒ p tryComplete Left(e) } }, true) - future + p.future } /** @@ -379,7 +379,7 @@ sealed trait Future[+T] extends Await.Awaitable[T] { case Left(t) ⇒ p failure t case Right(r) ⇒ that onSuccess { case r2 ⇒ p success ((r, r2)) } } - that onFailure { case f ⇒ p failure f } + that onFailure { case f ⇒ p tryComplete Left(f) } p.future } @@ -411,7 +411,7 @@ sealed trait Future[+T] extends Await.Awaitable[T] { * callbacks may be registered; there is no guarantee that they will be * executed in a particular order. */ - def onComplete(func: Either[Throwable, T] ⇒ Unit): this.type + def onComplete[U](func: Either[Throwable, T] ⇒ U): this.type /** * When the future is completed with a valid result, apply the provided @@ -483,7 +483,7 @@ sealed trait Future[+T] extends Await.Awaitable[T] { final def recover[A >: T](pf: PartialFunction[Throwable, A]): Future[A] = { val p = Promise[A]() onComplete { - case Left(e) if pf isDefinedAt e ⇒ p.complete(try { Right(pf(e)) } catch { case x: Exception ⇒ Left(x) }) + case Left(e) if pf isDefinedAt e ⇒ p.complete(try { Right(pf(e)) } catch { case NonFatal(x) ⇒ Left(x) }) case otherwise ⇒ p complete otherwise } p.future @@ -699,9 +699,12 @@ trait Promise[T] extends Future[T] { /** * Completes this Promise with the specified result, if not already completed. + * @throws IllegalStateException if already completed, this is to aid in debugging of complete-races, + * use tryComplete to do a conditional complete. * @return this */ - final def complete(value: Either[Throwable, T]): this.type = { tryComplete(value); this } + final def complete(value: Either[Throwable, T]): this.type = + if (tryComplete(value)) this else throw new IllegalStateException("Promise already completed: " + this + " tried to complete with " + value) /** * Completes this Promise with the specified result, if not already completed. @@ -721,7 +724,7 @@ trait Promise[T] extends Future[T] { * @return this. */ final def completeWith(other: Future[T]): this.type = { - other onComplete { complete(_) } + other onComplete { tryComplete(_) } this } @@ -840,7 +843,7 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac } } - def onComplete(func: Either[Throwable, T] ⇒ Unit): this.type = { + def onComplete[U](func: Either[Throwable, T] ⇒ U): this.type = { @tailrec //Returns whether the future has already been completed or not def tryAddCallback(): Either[Throwable, T] = { val cur = getState @@ -858,9 +861,8 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac } } - private final def notifyCompleted(func: Either[Throwable, T] ⇒ Unit, result: Either[Throwable, T]) { - try { func(result) } catch { case NonFatal(e) ⇒ executor.reportFailure(e) } - } + private final def notifyCompleted[U](func: Either[Throwable, T] ⇒ U, result: Either[Throwable, T]): Unit = + try func(result) catch { case NonFatal(e) ⇒ executor reportFailure e } } /** @@ -871,7 +873,7 @@ final class KeptPromise[T](suppliedValue: Either[Throwable, T])(implicit val exe val value = Some(resolve(suppliedValue)) def tryComplete(value: Either[Throwable, T]): Boolean = false - def onComplete(func: Either[Throwable, T] ⇒ Unit): this.type = { + def onComplete[U](func: Either[Throwable, T] ⇒ U): this.type = { val completedAs = value.get Future dispatchTask (() ⇒ func(completedAs)) this From fac0d2e509f792321f27e62fcab5f8687078faaf Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 6 Feb 2012 12:48:03 +0100 Subject: [PATCH 02/34] Improving the imports of Future --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index e3821e8334..ad27c8f8be 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -8,18 +8,18 @@ import akka.event.Logging.Error import scala.Option import akka.japi.{ Function ⇒ JFunc, Option ⇒ JOption } import scala.util.continuations._ -import java.util.concurrent.TimeUnit.NANOSECONDS import java.lang.{ Iterable ⇒ JIterable } import java.util.{ LinkedList ⇒ JLinkedList } import scala.annotation.tailrec import scala.collection.mutable.Stack import akka.util.{ Duration, BoxedType } import akka.dispatch.Await.CanAwait -import java.util.concurrent._ import akka.util.NonFatal import akka.event.Logging.LogEventException import akka.event.Logging.Debug -import atomic.{ AtomicInteger, AtomicReferenceFieldUpdater } +import java.util.concurrent.TimeUnit.NANOSECONDS +import java.util.concurrent.{ ExecutionException, Callable, TimeoutException } +import java.util.concurrent.atomic.{ AtomicInteger, AtomicReferenceFieldUpdater } object Await { From 09c44289d45c9b002f34f13e79cc214a4de43fce Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 6 Feb 2012 15:19:05 +0100 Subject: [PATCH 03/34] Creating our own FJTask and escalate exceptions to UHE --- .../akka/dispatch/AbstractDispatcher.scala | 27 +++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 8a5bcfa385..e60b4fd335 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -14,9 +14,9 @@ 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 import akka.event.Logging.LogEventException +import akka.jsr166y.{ ForkJoinTask, ForkJoinPool } final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorSystem) { if (message.isInstanceOf[AnyRef]) { @@ -424,7 +424,30 @@ class ThreadPoolExecutorConfigurator(config: Config, prerequisites: DispatcherPr threadPoolConfig.createExecutorServiceFactory(name, threadFactory) } +object ForkJoinExecutorConfigurator { + class AkkaForkJoinPool(parallelism: Int, threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory) extends ForkJoinPool(parallelism, threadFactory, MonitorableThreadFactory.doNothing, true) { + override def execute(r: Runnable): Unit = r match { + case m: Mailbox ⇒ super.execute(new MailboxExecutionTask(m)) + case other ⇒ super.execute(other) + } + } + class MailboxExecutionTask(mailbox: Mailbox) extends ForkJoinTask[Unit] { + final override def setRawResult(u: Unit): Unit = () + final override def getRawResult(): Unit = () + final override def exec(): Boolean = try { mailbox.run; true } catch { + case anything ⇒ + val t = Thread.currentThread + t.getUncaughtExceptionHandler match { + case null ⇒ + case some ⇒ some.uncaughtException(t, anything) + } + throw anything + } + } +} + class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceConfigurator(config, prerequisites) { + import ForkJoinExecutorConfigurator._ def validate(t: ThreadFactory): ForkJoinPool.ForkJoinWorkerThreadFactory = prerequisites.threadFactory match { case correct: ForkJoinPool.ForkJoinWorkerThreadFactory ⇒ correct @@ -433,7 +456,7 @@ class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrer class ForkJoinExecutorServiceFactory(val threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, val parallelism: Int) extends ExecutorServiceFactory { - def createExecutorService: ExecutorService = new ForkJoinPool(parallelism, threadFactory, MonitorableThreadFactory.doNothing, true) + def createExecutorService: ExecutorService = new AkkaForkJoinPool(parallelism, threadFactory) } final def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory = new ForkJoinExecutorServiceFactory( From 66c1e2d835bc1c93db397144e4d646931ed6bbd3 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 6 Feb 2012 15:42:28 +0100 Subject: [PATCH 04/34] Sprinkling some finals and adding return types --- .../akka/dispatch/AbstractDispatcher.scala | 17 ++++++++++++++--- .../src/main/scala/akka/dispatch/Mailbox.scala | 8 ++++---- 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index e60b4fd335..0cf6c4a77b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -425,13 +425,24 @@ class ThreadPoolExecutorConfigurator(config: Config, prerequisites: DispatcherPr } object ForkJoinExecutorConfigurator { - class AkkaForkJoinPool(parallelism: Int, threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory) extends ForkJoinPool(parallelism, threadFactory, MonitorableThreadFactory.doNothing, true) { + + /** + * INTERNAL AKKA USAGE ONLY + */ + final class AkkaForkJoinPool(parallelism: Int, + threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, + unhandledExceptionHandler: Thread.UncaughtExceptionHandler) + extends ForkJoinPool(parallelism, threadFactory, unhandledExceptionHandler, true) { override def execute(r: Runnable): Unit = r match { case m: Mailbox ⇒ super.execute(new MailboxExecutionTask(m)) case other ⇒ super.execute(other) } } - class MailboxExecutionTask(mailbox: Mailbox) extends ForkJoinTask[Unit] { + + /** + * INTERNAL AKKA USAGE ONLY + */ + final class MailboxExecutionTask(mailbox: Mailbox) extends ForkJoinTask[Unit] { final override def setRawResult(u: Unit): Unit = () final override def getRawResult(): Unit = () final override def exec(): Boolean = try { mailbox.run; true } catch { @@ -456,7 +467,7 @@ class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrer class ForkJoinExecutorServiceFactory(val threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, val parallelism: Int) extends ExecutorServiceFactory { - def createExecutorService: ExecutorService = new AkkaForkJoinPool(parallelism, threadFactory) + def createExecutorService: ExecutorService = new AkkaForkJoinPool(parallelism, threadFactory, MonitorableThreadFactory.doNothing) } final def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory = new ForkJoinExecutorServiceFactory( diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 3aa6103b22..27853b49db 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -328,7 +328,7 @@ trait MailboxType { * It's a case class for Java (new UnboundedMailbox) */ case class UnboundedMailbox() extends MailboxType { - override def create(receiver: ActorContext) = + final override def create(receiver: ActorContext): Mailbox = new Mailbox(receiver.asInstanceOf[ActorCell]) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { final val queue = new ConcurrentLinkedQueue[Envelope]() } @@ -339,7 +339,7 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative") if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null") - override def create(receiver: ActorContext) = + final override def create(receiver: ActorContext): Mailbox = new Mailbox(receiver.asInstanceOf[ActorCell]) with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue { final val queue = new LinkedBlockingQueue[Envelope](capacity) final val pushTimeOut = BoundedMailbox.this.pushTimeOut @@ -347,7 +347,7 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat } case class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType { - override def create(receiver: ActorContext) = + final override def create(receiver: ActorContext): Mailbox = new Mailbox(receiver.asInstanceOf[ActorCell]) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { final val queue = new PriorityBlockingQueue[Envelope](11, cmp) } @@ -358,7 +358,7 @@ case class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final va if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative") if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null") - override def create(receiver: ActorContext) = + final override def create(receiver: ActorContext): Mailbox = new Mailbox(receiver.asInstanceOf[ActorCell]) with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue { final val queue = new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut From 2acce5b17f688b403d6e358ea9eb03f657beb520 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 6 Feb 2012 16:42:42 +0100 Subject: [PATCH 05/34] Switching to using the internalClassLoader for loading the Serializers --- .../src/main/scala/akka/serialization/Serialization.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index e89adde8fb..05612b9cca 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -150,7 +150,7 @@ 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) + ReflectiveAccess.createInstance(serializerFQN, ReflectiveAccess.noParams, ReflectiveAccess.noArgs, system.internalClassLoader) /** * A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer) From 73fce5223545cf86ea9a0164921c3ed7a5af1e49 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 6 Feb 2012 16:59:09 +0100 Subject: [PATCH 06/34] Fixing returns to return in ScalaDoc --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 8 ++++---- akka-cluster/src/main/scala/akka/cluster/Gossiper.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index ad27c8f8be..97ff17c075 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -53,7 +53,7 @@ object Await { * WARNING: Blocking operation, use with caution. * * @throws [[java.util.concurrent.TimeoutException]] if times out - * @returns The returned value as returned by Awaitable.ready + * @return The returned value as returned by Awaitable.ready */ def ready[T <: Awaitable[_]](awaitable: T, atMost: Duration): T = awaitable.ready(atMost) @@ -62,7 +62,7 @@ object Await { * WARNING: Blocking operation, use with caution. * * @throws [[java.util.concurrent.TimeoutException]] if times out - * @returns The returned value as returned by Awaitable.result + * @return The returned value as returned by Awaitable.result */ def result[T](awaitable: Awaitable[T], atMost: Duration): T = awaitable.result(atMost) } @@ -984,7 +984,7 @@ abstract class Recover[+T] extends japi.RecoverBridge[T] { * This method will be invoked once when/if the Future this recover callback is registered on * becomes completed with a failure. * - * @returns a successful value for the passed in failure + * @return a successful value for the passed in failure * @throws the passed in failure to propagate it. * * Java API @@ -1007,7 +1007,7 @@ abstract class Filter[-T] extends japi.BooleanFunctionBridge[T] { * This method will be invoked once when/if a Future that this callback is registered on * becomes completed with a success. * - * @returns true if the successful value should be propagated to the new Future or not + * @return true if the successful value should be propagated to the new Future or not */ def filter(result: T): Boolean } diff --git a/akka-cluster/src/main/scala/akka/cluster/Gossiper.scala b/akka-cluster/src/main/scala/akka/cluster/Gossiper.scala index 1b9026d082..bb15223842 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Gossiper.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Gossiper.scala @@ -369,7 +369,7 @@ case class Gossiper(remote: RemoteActorRefProvider, system: ActorSystemImpl) { /** * Gossips to a random member in the set of members passed in as argument. * - * @returns 'true' if it gossiped to a "seed" member. + * @return 'true' if it gossiped to a "seed" member. */ private def gossipToRandomNodeOf(members: Set[Member]): Boolean = { val peers = members filter (_.address != address) // filter out myself From 4b56e754bc16db421bd90b35fcdc9e52ec135dfd Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 6 Feb 2012 17:06:08 +0100 Subject: [PATCH 07/34] Commenting out Akka Cluster because it shouldn't be Released, have fun Jonas ;) --- project/AkkaBuild.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index f810da86a6..1f763e582a 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -31,7 +31,7 @@ object AkkaBuild extends Build { Unidoc.unidocExclude := Seq(samples.id, tutorials.id), Dist.distExclude := Seq(actorTests.id, akkaSbtPlugin.id, docs.id) ), - aggregate = Seq(actor, testkit, actorTests, remote, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, actorMigration, samples, tutorials, docs) + aggregate = Seq(actor, testkit, actorTests, remote, /*cluster,*/ slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, actorMigration, samples, tutorials, docs) ) lazy val actor = Project( @@ -86,7 +86,7 @@ object AkkaBuild extends Build { ) ) configs (MultiJvm) - lazy val cluster = Project( + /*lazy val cluster = Project( id = "akka-cluster", base = file("akka-cluster"), dependencies = Seq(remote, remote % "test->test", testkit % "test->test"), @@ -103,7 +103,7 @@ object AkkaBuild extends Build { }, test in Test <<= (test in Test) dependsOn (test in MultiJvm) ) - ) configs (MultiJvm) + ) configs (MultiJvm)*/ lazy val slf4j = Project( id = "akka-slf4j", @@ -320,7 +320,7 @@ 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), settings = defaultSettings ++ Seq( unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get }, libraryDependencies ++= Dependencies.docs, @@ -429,7 +429,7 @@ object Dependencies { Test.zookeeper, Test.log4j // needed for ZkBarrier in multi-jvm tests ) - val cluster = Seq(Test.junit, Test.scalatest) + //val cluster = Seq(Test.junit, Test.scalatest) val slf4j = Seq(slf4jApi) From d7435547ff69ebdc4161c72ef1ecf68b839e36d8 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 6 Feb 2012 17:29:17 +0100 Subject: [PATCH 08/34] Revert "Commenting out Akka Cluster because it shouldn't be Released, have fun Jonas ;)" This reverts commit 4b56e754bc16db421bd90b35fcdc9e52ec135dfd. --- project/AkkaBuild.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 1f763e582a..f810da86a6 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -31,7 +31,7 @@ object AkkaBuild extends Build { Unidoc.unidocExclude := Seq(samples.id, tutorials.id), Dist.distExclude := Seq(actorTests.id, akkaSbtPlugin.id, docs.id) ), - aggregate = Seq(actor, testkit, actorTests, remote, /*cluster,*/ slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, actorMigration, samples, tutorials, docs) + aggregate = Seq(actor, testkit, actorTests, remote, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, actorMigration, samples, tutorials, docs) ) lazy val actor = Project( @@ -86,7 +86,7 @@ object AkkaBuild extends Build { ) ) configs (MultiJvm) - /*lazy val cluster = Project( + lazy val cluster = Project( id = "akka-cluster", base = file("akka-cluster"), dependencies = Seq(remote, remote % "test->test", testkit % "test->test"), @@ -103,7 +103,7 @@ object AkkaBuild extends Build { }, test in Test <<= (test in Test) dependsOn (test in MultiJvm) ) - ) configs (MultiJvm)*/ + ) configs (MultiJvm) lazy val slf4j = Project( id = "akka-slf4j", @@ -320,7 +320,7 @@ 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), settings = defaultSettings ++ Seq( unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get }, libraryDependencies ++= Dependencies.docs, @@ -429,7 +429,7 @@ object Dependencies { Test.zookeeper, Test.log4j // needed for ZkBarrier in multi-jvm tests ) - //val cluster = Seq(Test.junit, Test.scalatest) + val cluster = Seq(Test.junit, Test.scalatest) val slf4j = Seq(slf4jApi) From 1dbce493593b67d52cac3baa06a8211433560204 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 6 Feb 2012 21:12:26 +0100 Subject: [PATCH 09/34] Configure serializer with class as key. See #1789 --- .../akka/serialization/SerializeSpec.scala | 96 +++++++------ akka-actor/src/main/resources/reference.conf | 24 ++-- .../akka/serialization/Serialization.scala | 132 +++++++++--------- .../SerializationDocTestBase.java | 54 ------- akka-docs/java/serialization.rst | 41 ++---- .../serialization/SerializationDocSpec.scala | 32 ++--- akka-docs/scala/serialization.rst | 33 +---- akka-remote/src/main/resources/reference.conf | 15 ++ .../ProtobufSerializerSpec.scala | 25 ++++ 9 files changed, 207 insertions(+), 245 deletions(-) create mode 100644 akka-remote/src/test/scala/akka/serialization/ProtobufSerializerSpec.scala 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..8cf314e0d5 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -5,7 +5,6 @@ package akka.serialization import akka.testkit.AkkaSpec -import com.typesafe.config.ConfigFactory import akka.actor._ import java.io._ import akka.dispatch.Await @@ -17,21 +16,25 @@ import akka.pattern.ask object SerializeSpec { - val serializationConf = ConfigFactory.parseString(""" + val config = """ akka { actor { serializers { - java = "akka.serialization.JavaSerializer" test = "akka.serialization.TestSerializer" } serialization-bindings { - java = ["akka.serialization.SerializeSpec$Person", "akka.serialization.SerializeSpec$Address", "akka.serialization.MyJavaSerializableActor", "akka.serialization.MyStatelessActorWithMessagesInMailbox", "akka.serialization.MyActorWithProtobufMessagesInMailbox"] - test = ["akka.serialization.TestSerializble", "akka.serialization.SerializeSpec$PlainMessage"] + "akka.serialization.SerializeSpec$Person" = java + "akka.serialization.SerializeSpec$Address" = java + "akka.serialization.TestSerializble" = test + "akka.serialization.SerializeSpec$PlainMessage" = test + "akka.serialization.SerializeSpec$A" = java + "akka.serialization.SerializeSpec$B" = test + "akka.serialization.SerializeSpec$D" = test } } } - """) + """ @BeanInfo case class Address(no: String, street: String, city: String, zip: String) { def this() = this("", "", "", "") } @@ -54,10 +57,18 @@ object SerializeSpec { class ExtendedPlainMessage extends PlainMessage + class Both(s: String) extends SimpleMessage(s) with Serializable + + trait A + trait B + class C extends B with A + class D extends A + class E extends D + } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class SerializeSpec extends AkkaSpec(SerializeSpec.serializationConf) { +class SerializeSpec extends AkkaSpec(SerializeSpec.config) { import SerializeSpec._ val ser = SerializationExtension(system) @@ -69,8 +80,8 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.serializationConf) { "Serialization" must { "have correct bindings" in { - ser.bindings(addr.getClass.getName) must be("java") - ser.bindings(classOf[PlainMessage].getName) must be("test") + ser.bindings.find(_._1 == addr.getClass).map(_._2.getClass) must be(Some(classOf[JavaSerializer])) + ser.bindings.find(_._1 == classOf[PlainMessage]).map(_._2.getClass) must be(Some(classOf[TestSerializer])) } "serialize Address" in { @@ -144,58 +155,64 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.serializationConf) { } } - "resove serializer by direct interface" in { - val msg = new SimpleMessage("foo") - ser.serializerFor(msg.getClass).getClass must be(classOf[TestSerializer]) + "resolve serializer by direct interface" in { + ser.serializerFor(classOf[SimpleMessage]).getClass must be(classOf[TestSerializer]) } - "resove serializer by interface implemented by super class" in { - val msg = new ExtendedSimpleMessage("foo", 17) - ser.serializerFor(msg.getClass).getClass must be(classOf[TestSerializer]) + "resolve serializer by interface implemented by super class" in { + ser.serializerFor(classOf[ExtendedSimpleMessage]).getClass must be(classOf[TestSerializer]) } - "resove serializer by indirect interface" in { - val msg = new AnotherMessage - ser.serializerFor(msg.getClass).getClass must be(classOf[TestSerializer]) + "resolve serializer by indirect interface" in { + ser.serializerFor(classOf[AnotherMessage]).getClass must be(classOf[TestSerializer]) } - "resove serializer by indirect interface implemented by super class" in { - val msg = new ExtendedAnotherMessage - ser.serializerFor(msg.getClass).getClass must be(classOf[TestSerializer]) + "resolve serializer by indirect interface implemented by super class" in { + ser.serializerFor(classOf[ExtendedAnotherMessage]).getClass must be(classOf[TestSerializer]) } - "resove serializer for message with binding" in { - val msg = new PlainMessage - ser.serializerFor(msg.getClass).getClass must be(classOf[TestSerializer]) + "resolve serializer for message with binding" in { + ser.serializerFor(classOf[PlainMessage]).getClass must be(classOf[TestSerializer]) } - "resove serializer for message extending class with with binding" in { - val msg = new ExtendedPlainMessage - ser.serializerFor(msg.getClass).getClass must be(classOf[TestSerializer]) + "resolve serializer for message extending class with with binding" in { + ser.serializerFor(classOf[ExtendedPlainMessage]).getClass must be(classOf[TestSerializer]) + } + + "resolve serializer for message with several bindings" in { + ser.serializerFor(classOf[Both]).getClass must be(classOf[TestSerializer]) + } + + "resolve serializer in the order of the bindings" in { + ser.serializerFor(classOf[A]).getClass must be(classOf[JavaSerializer]) + ser.serializerFor(classOf[B]).getClass must be(classOf[TestSerializer]) + ser.serializerFor(classOf[C]).getClass must be(classOf[JavaSerializer]) + } + + "resolve serializer in the order of most specific binding first" in { + ser.serializerFor(classOf[A]).getClass must be(classOf[JavaSerializer]) + ser.serializerFor(classOf[D]).getClass must be(classOf[TestSerializer]) + ser.serializerFor(classOf[E]).getClass must be(classOf[TestSerializer]) + } + + "throw java.io.NotSerializableException when no binding" in { + intercept[java.io.NotSerializableException] { + ser.serializerFor(classOf[Actor]) + } } } } object VerifySerializabilitySpec { - val conf = ConfigFactory.parseString(""" + val conf = """ akka { actor { serialize-messages = on - serialize-creators = on - - serializers { - java = "akka.serialization.JavaSerializer" - default = "akka.serialization.JavaSerializer" - } - - serialization-bindings { - java = ["akka.serialization.SerializeSpec$Address", "akka.serialization.MyJavaSerializableActor", "akka.serialization.MyStatelessActorWithMessagesInMailbox", "akka.serialization.MyActorWithProtobufMessagesInMailbox"] - } } } - """) + """ class FooActor extends Actor { def receive = { @@ -210,6 +227,7 @@ object VerifySerializabilitySpec { } } +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class VerifySerializabilitySpec extends AkkaSpec(VerifySerializabilitySpec.conf) { import VerifySerializabilitySpec._ implicit val timeout = Timeout(5 seconds) diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index cdab8e968e..ebeb7766ba 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -97,7 +97,7 @@ akka { paths = [] } - # Routers with dynamically resizable number of routees; this feature is enabled + # Routers with dynamically resizable number of routees; this feature is enabled # by including (parts of) this section in the deployment resizer { @@ -262,23 +262,19 @@ akka { event-stream = off } - # Entries for pluggable serializers and their bindings. If a binding for a specific - # class is not found, then the default serializer (Java serialization) is used. + # Entries for pluggable serializers and their bindings. serializers { - # java = "akka.serialization.JavaSerializer" + java = "akka.serialization.JavaSerializer" # proto = "akka.serialization.ProtobufSerializer" - - default = "akka.serialization.JavaSerializer" } - # serialization-bindings { - # java = ["akka.serialization.SerializeSpec$Address", - # "akka.serialization.MyJavaSerializableActor", - # "akka.serialization.MyStatelessActorWithMessagesInMailbox", - # "akka.serialization.MyActorWithProtobufMessagesInMailbox"] - # proto = ["com.google.protobuf.Message", - # "akka.actor.ProtobufProtocol$MyMessage"] - # } + # Class to Serializer binding. You only need to specify the name of an interface + # or abstract base class of the messages. In case of ambiguity it is primarily + # using the most specific configured class, and secondly the entry configured first. + serialization-bindings { + "java.io.Serializable" = java + #"com.google.protobuf.Message" = proto + } } # Used to set the behavior of the scheduler. diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 05612b9cca..2d313b05a0 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -8,14 +8,21 @@ 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 java.util.concurrent.ConcurrentHashMap import akka.event.Logging +import scala.collection.mutable.ArrayBuffer +import java.io.NotSerializableException case class NoSerializerFoundException(m: String) extends AkkaException(m) object Serialization { + + /** + * Tuple that represents mapping from Class to Serializer + */ + type ClassSerializer = (Class[_], Serializer) + /** * This holds a reference to the current ActorSystem (the surrounding context) * during serialization and deserialization. @@ -40,28 +47,19 @@ object Serialization { import scala.collection.JavaConverters._ import config._ - val Serializers: Map[String, String] = - getConfig("akka.actor.serializers").root.unwrapped.asScala.toMap.map { case (k, v) ⇒ (k, v.toString) } + val Serializers: Map[String, String] = configToMap(getConfig("akka.actor.serializers")) - val SerializationBindings: Map[String, Seq[String]] = { - val configPath = "akka.actor.serialization-bindings" - hasPath(configPath) match { - case false ⇒ Map() - case true ⇒ - val serializationBindings: Map[String, Seq[String]] = getConfig(configPath).root.unwrapped.asScala.toMap.map { - case (k: String, v: java.util.Collection[_]) ⇒ (k -> v.asScala.toSeq.asInstanceOf[Seq[String]]) - case invalid ⇒ throw new ConfigurationException("Invalid serialization-bindings [%s]".format(invalid)) - } - serializationBindings + val SerializationBindings: Map[String, String] = configToMap(getConfig("akka.actor.serialization-bindings")) + + private def configToMap(cfg: Config): Map[String, String] = + cfg.root.unwrapped.asScala.toMap.map { case (k, v) ⇒ (k, v.toString) } - } - } } } /** * Serialization module. Contains methods for serialization and deserialization as well as - * locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file. + * locating a Serializer for a particular class as defined in the mapping in the configuration. */ class Serialization(val system: ExtendedActorSystem) extends Extension { import Serialization._ @@ -105,8 +103,10 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { } catch { case e: Exception ⇒ Left(e) } /** - * Returns the Serializer configured for the given object, returns the NullSerializer if it's null, - * falls back to the Serializer named "default" + * Returns the Serializer configured for the given object, returns the NullSerializer if it's null. + * + * @throws akka.config.ConfigurationException if no `serialization-bindings` is configured for the + * class of the object */ def findSerializerFor(o: AnyRef): Serializer = o match { case null ⇒ NullSerializer @@ -114,82 +114,86 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { } /** - * Returns the configured Serializer for the given Class, falls back to the Serializer named "default". - * It traverses interfaces and super classes to find any configured Serializer that match - * the class name. + * Returns the configured Serializer for the given Class. The configured Serializer + * is used if the configured class `isAssignableFrom` from the `clazz`, i.e. + * the configured class is a super class or implemented interface. In case of + * ambiguity it is primarily using the most specific configured class, + * and secondly the entry configured first. + * + * @throws java.io.NotSerializableException if no `serialization-bindings` is configured for the class */ def serializerFor(clazz: Class[_]): Serializer = - if (bindings.isEmpty) { - // quick path to default when no bindings are registered - serializers("default") - } else { - - def resolve(c: Class[_]): Option[Serializer] = - serializerMap.get(c.getName) match { - case null ⇒ - val classes = c.getInterfaces ++ Option(c.getSuperclass) - classes.view map resolve collectFirst { case Some(x) ⇒ x } - case x ⇒ Some(x) + serializerMap.get(clazz) match { + case null ⇒ + val ser = bindings.find { case (c, s) ⇒ c.isAssignableFrom(clazz) } match { + case None ⇒ throw new NotSerializableException( + "No configured serialization-bindings for class [%s]" format clazz.getName) + case Some((c, s)) ⇒ s } - - serializerMap.get(clazz.getName) match { - case null ⇒ - val ser = resolve(clazz).getOrElse(serializers("default")) - // memorize the lookups for performance - serializerMap.putIfAbsent(clazz.getName, ser) match { - case null ⇒ - log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName) - ser - case some ⇒ some - } - case ser ⇒ ser - } + // memorize for performance + serializerMap.putIfAbsent(clazz, ser) match { + case null ⇒ + log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName) + ser + case some ⇒ some + } + case ser ⇒ ser } /** - * Tries to load the specified Serializer by the FQN + * Tries to instantiate the specified Serializer by the FQN */ def serializerOf(serializerFQN: String): Either[Exception, Serializer] = ReflectiveAccess.createInstance(serializerFQN, ReflectiveAccess.noParams, ReflectiveAccess.noArgs, system.internalClassLoader) /** * A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer) - * By default always contains the following mapping: "default" -> akka.serialization.JavaSerializer - * But "default" can be overridden in config + * By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer */ - lazy val serializers: Map[String, Serializer] = { - val serializersConf = settings.Serializers - for ((k: String, v: String) ← serializersConf) + private val serializers: Map[String, Serializer] = { + for ((k: String, v: String) ← settings.Serializers) yield k -> serializerOf(v).fold(throw _, identity) } /** - * bindings is a Map whose keys = FQN of class that is serializable and values = the alias of the serializer to be used + * bindings is a Seq of tuple representing the mapping from Class to Serializer. + * It is primarily ordered by the most specific classes first, and secondly in the configured order. */ - lazy val bindings: Map[String, String] = { - settings.SerializationBindings.foldLeft(Map[String, String]()) { - //All keys which are lists, take the Strings from them and Map them - case (result, (k: String, vs: Seq[_])) ⇒ result ++ (vs collect { case v: String ⇒ (v, k) }) - //For any other values, just skip them - case (result, _) ⇒ result + private[akka] val bindings: Seq[ClassSerializer] = { + val configuredBindings = for ((k: String, v: String) ← settings.SerializationBindings) yield { + val c = ReflectiveAccess.getClassFor(k, system.internalClassLoader).fold(throw _, (c: Class[_]) ⇒ c) + (c, serializers(v)) } + sort(configuredBindings) } /** - * serializerMap is a Map whose keys = FQN of class that is serializable and values is the serializer to be used for that class + * Sort so that subtypes always precede their supertypes, but without + * obeying any order between unrelated subtypes (insert sort). */ - private lazy val serializerMap: ConcurrentHashMap[String, Serializer] = { - val serializerMap = new ConcurrentHashMap[String, Serializer] - for ((k, v) ← bindings) { - serializerMap.put(k, serializers(v)) + private def sort(in: Iterable[ClassSerializer]): Seq[ClassSerializer] = + (new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca) ⇒ + buf.indexWhere(_._1 isAssignableFrom ca._1) match { + case -1 ⇒ buf append ca + case x ⇒ buf insert (x, ca) + } + buf } + + /** + * serializerMap is a Map whose keys is the class that is serializable and values is the serializer + * to be used for that class. + */ + private val serializerMap: ConcurrentHashMap[Class[_], Serializer] = { + val serializerMap = new ConcurrentHashMap[Class[_], Serializer] + for ((c, s) ← bindings) serializerMap.put(c, s) serializerMap } /** * Maps from a Serializer Identity (Int) to a Serializer instance (optimization) */ - lazy val serializerByIdentity: Map[Int, Serializer] = + val serializerByIdentity: Map[Int, Serializer] = Map(NullSerializer.identifier -> NullSerializer) ++ serializers map { case (_, v) ⇒ (v.identifier, v) } } diff --git a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java index b68f8f2e79..3db385ca1c 100644 --- a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java +++ b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java @@ -54,61 +54,7 @@ public class SerializationDocTestBase { } } //#my-own-serializer - @Test public void haveExamples() { - /* - //#serialize-messages-config - akka { - actor { - serialize-messages = on - } - } - //#serialize-messages-config - //#serialize-creators-config - akka { - actor { - serialize-creators = on - } - } - //#serialize-creators-config - - - //#serialize-serializers-config - akka { - actor { - serializers { - default = "akka.serialization.JavaSerializer" - - myown = "akka.docs.serialization.MyOwnSerializer" - } - } - } - //#serialize-serializers-config - - //#serialization-bindings-config - akka { - actor { - serializers { - default = "akka.serialization.JavaSerializer" - java = "akka.serialization.JavaSerializer" - proto = "akka.serialization.ProtobufSerializer" - myown = "akka.docs.serialization.MyOwnSerializer" - } - - serialization-bindings { - java = ["java.lang.String", - "app.my.Customer"] - proto = ["com.google.protobuf.Message"] - myown = ["my.own.BusinessObject", - "something.equally.Awesome", - "akka.docs.serialization.MyOwnSerializable" - "java.lang.Boolean"] - } - } - } - //#serialization-bindings-config - */ - } @Test public void demonstrateTheProgrammaticAPI() { //#programmatic diff --git a/akka-docs/java/serialization.rst b/akka-docs/java/serialization.rst index 2920538ded..b0721e82cc 100644 --- a/akka-docs/java/serialization.rst +++ b/akka-docs/java/serialization.rst @@ -25,47 +25,28 @@ For Akka to know which ``Serializer`` to use for what, you need edit your :ref:` in the "akka.actor.serializers"-section you bind names to implementations of the ``akka.serialization.Serializer`` you wish to use, like this: -.. includecode:: code/akka/docs/serialization/SerializationDocTestBase.java#serialize-serializers-config - -.. note:: - - The name ``default`` is special in the sense that the ``Serializer`` - mapped to it will be used as default. +.. includecode:: ../scala/code/akka/docs/serialization/SerializationDocSpec.scala#serialize-serializers-config After you've bound names to different implementations of ``Serializer`` you need to wire which classes should be serialized using which ``Serializer``, this is done in the "akka.actor.serialization-bindings"-section: -.. includecode:: code/akka/docs/serialization/SerializationDocTestBase.java#serialization-bindings-config +.. includecode:: ../scala/code/akka/docs/serialization/SerializationDocSpec.scala#serialization-bindings-config -.. note:: +You only need to specify the name of an interface or abstract base class of the messages. In case of ambiguity, +i.e. the message implements several of the configured classes, it is primarily using the most specific +configured class, and secondly the entry configured first. - You only need to specify the name of an interface or abstract base class if the messages implements - that. E.g. ``com.google.protobuf.Message`` for protobuf serialization. - -Protobuf --------- - -Akka provides a ``Serializer`` for `protobuf `_ messages. -To use that you need to add the following to the configuration:: - - akka { - actor { - serializers { - proto = "akka.serialization.ProtobufSerializer" - } - - serialization-bindings { - proto = ["com.google.protobuf.Message"] - } - } - } +Akka provides serializers for ``java.io.Serializable`` and `protobuf `_ +``com.google.protobuf.Message`` by default, so normally you don't need to add configuration for that, but +it can be done to force a specific serializer in case messages implements both ``java.io.Serializable`` +and ``com.google.protobuf.Message``. Verification ------------ If you want to verify that your messages are serializable you can enable the following config option: -.. includecode:: code/akka/docs/serialization/SerializationDocTestBase.java#serialize-messages-config +.. includecode:: ../scala/code/akka/docs/serialization/SerializationDocSpec.scala#serialize-messages-config .. warning:: @@ -74,7 +55,7 @@ If you want to verify that your messages are serializable you can enable the fol If you want to verify that your ``Props`` are serializable you can enable the following config option: -.. includecode:: code/akka/docs/serialization/SerializationDocTestBase.java#serialize-creators-config +.. includecode:: ../scala/code/akka/docs/serialization/SerializationDocSpec.scala#serialize-creators-config .. warning:: diff --git a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala b/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala index 7f1553f75c..ce40adce3e 100644 --- a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala @@ -45,6 +45,9 @@ class MyOwnSerializer extends Serializer { } //#my-own-serializer +trait MyOwnSerializable +case class Customer(name: String) extends MyOwnSerializable + class SerializationDocSpec extends AkkaSpec { "demonstrate configuration of serialize messages" in { //#serialize-messages-config @@ -82,8 +85,8 @@ class SerializationDocSpec extends AkkaSpec { akka { actor { serializers { - default = "akka.serialization.JavaSerializer" - + java = "akka.serialization.JavaSerializer" + proto = "akka.serialization.ProtobufSerializer" myown = "akka.docs.serialization.MyOwnSerializer" } } @@ -91,8 +94,6 @@ class SerializationDocSpec extends AkkaSpec { """) //#serialize-serializers-config val a = ActorSystem("system", config) - SerializationExtension(a).serializers("default").getClass.getName must equal("akka.serialization.JavaSerializer") - SerializationExtension(a).serializers("myown").getClass.getName must equal("akka.docs.serialization.MyOwnSerializer") a.shutdown() } @@ -102,31 +103,26 @@ class SerializationDocSpec extends AkkaSpec { akka { actor { serializers { - default = "akka.serialization.JavaSerializer" java = "akka.serialization.JavaSerializer" proto = "akka.serialization.ProtobufSerializer" myown = "akka.docs.serialization.MyOwnSerializer" } serialization-bindings { - java = ["java.lang.String", - "app.my.Customer"] - proto = ["com.google.protobuf.Message"] - myown = ["my.own.BusinessObject", - "something.equally.Awesome", - "akka.docs.serialization.MyOwnSerializable" - "java.lang.Boolean"] - } + "java.lang.String" = java + "akka.docs.serialization.Customer" = java + "com.google.protobuf.Message" = proto + "akka.docs.serialization.MyOwnSerializable" = myown + "java.lang.Boolean" = myown + } } } """) //#serialization-bindings-config val a = ActorSystem("system", config) - SerializationExtension(a).serializers("default").getClass.getName must equal("akka.serialization.JavaSerializer") - SerializationExtension(a).serializers("java").getClass.getName must equal("akka.serialization.JavaSerializer") - SerializationExtension(a).serializers("myown").getClass.getName must equal("akka.docs.serialization.MyOwnSerializer") - SerializationExtension(a).serializerFor(classOf[String]).getClass.getName must equal("akka.serialization.JavaSerializer") - SerializationExtension(a).serializerFor(classOf[java.lang.Boolean]).getClass.getName must equal("akka.docs.serialization.MyOwnSerializer") + SerializationExtension(a).serializerFor(classOf[String]).getClass must equal(classOf[JavaSerializer]) + SerializationExtension(a).serializerFor(classOf[Customer]).getClass must equal(classOf[JavaSerializer]) + SerializationExtension(a).serializerFor(classOf[java.lang.Boolean]).getClass must equal(classOf[MyOwnSerializer]) a.shutdown() } diff --git a/akka-docs/scala/serialization.rst b/akka-docs/scala/serialization.rst index 6a0867dea2..9cfaf909b9 100644 --- a/akka-docs/scala/serialization.rst +++ b/akka-docs/scala/serialization.rst @@ -27,38 +27,19 @@ you wish to use, like this: .. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala#serialize-serializers-config -.. note:: - - The name ``default`` is special in the sense that the ``Serializer`` - mapped to it will be used as default. - After you've bound names to different implementations of ``Serializer`` you need to wire which classes should be serialized using which ``Serializer``, this is done in the "akka.actor.serialization-bindings"-section: .. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala#serialization-bindings-config -.. note:: +You only need to specify the name of an interface or abstract base class of the messages. In case of ambiguity, +i.e. the message implements several of the configured classes, it is primarily using the most specific +configured class, and secondly the entry configured first. - You only need to specify the name of an interface or abstract base class if the messages implements - that. E.g. ``com.google.protobuf.Message`` for protobuf serialization. - -Protobuf --------- - -Akka provides a ``Serializer`` for `protobuf `_ messages. -To use that you need to add the following to the configuration:: - - akka { - actor { - serializers { - proto = "akka.serialization.ProtobufSerializer" - } - - serialization-bindings { - proto = ["com.google.protobuf.Message"] - } - } - } +Akka provides serializers for ``java.io.Serializable`` and `protobuf `_ +``com.google.protobuf.Message`` by default, so normally you don't need to add configuration for that, but +it can be done to force a specific serializer in case messages implements both ``java.io.Serializable`` +and ``com.google.protobuf.Message``. Verification ------------ diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 1158d12295..14269d305a 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -9,6 +9,21 @@ akka { actor { + # Entries for pluggable serializers and their bindings. + serializers { + java = "akka.serialization.JavaSerializer" + proto = "akka.serialization.ProtobufSerializer" + } + + + # Class to Serializer binding. You only need to specify the name of an interface + # or abstract base class of the messages. In case of ambiguity it is primarily + # using the most specific configured class, and secondly the entry configured first. + serialization-bindings { + "com.google.protobuf.Message" = proto + "java.io.Serializable" = java + } + deployment { default { diff --git a/akka-remote/src/test/scala/akka/serialization/ProtobufSerializerSpec.scala b/akka-remote/src/test/scala/akka/serialization/ProtobufSerializerSpec.scala new file mode 100644 index 0000000000..474ef485d7 --- /dev/null +++ b/akka-remote/src/test/scala/akka/serialization/ProtobufSerializerSpec.scala @@ -0,0 +1,25 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.serialization + +import akka.testkit.AkkaSpec +import akka.remote.RemoteProtocol.MessageProtocol +import akka.actor.ProtobufProtocol.MyMessage + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class ProtobufSerializerSpec extends AkkaSpec { + + val ser = SerializationExtension(system) + + "Serialization" must { + + "resolve protobuf serializer" in { + ser.serializerFor(classOf[MessageProtocol]).getClass must be(classOf[ProtobufSerializer]) + ser.serializerFor(classOf[MyMessage]).getClass must be(classOf[ProtobufSerializer]) + } + + } +} + From 239df9d5fbd47ab15b8f0ea48b2e7dad0a0c52c7 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 6 Feb 2012 22:20:38 +0100 Subject: [PATCH 10/34] Improvements after review --- .../scala/akka/serialization/SerializeSpec.scala | 4 ++-- .../scala/akka/serialization/Serialization.scala | 12 ++++++------ 2 files changed, 8 insertions(+), 8 deletions(-) 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 8cf314e0d5..7b91e29bcb 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -80,8 +80,8 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) { "Serialization" must { "have correct bindings" in { - ser.bindings.find(_._1 == addr.getClass).map(_._2.getClass) must be(Some(classOf[JavaSerializer])) - ser.bindings.find(_._1 == classOf[PlainMessage]).map(_._2.getClass) must be(Some(classOf[TestSerializer])) + ser.bindings.collectFirst { case (c, s) if c == addr.getClass ⇒ s.getClass } must be(Some(classOf[JavaSerializer])) + ser.bindings.collectFirst { case (c, s) if c == classOf[PlainMessage] ⇒ s.getClass } must be(Some(classOf[TestSerializer])) } "serialize Address" in { diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 2d313b05a0..1a23833383 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -125,11 +125,11 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { def serializerFor(clazz: Class[_]): Serializer = serializerMap.get(clazz) match { case null ⇒ - val ser = bindings.find { case (c, s) ⇒ c.isAssignableFrom(clazz) } match { - case None ⇒ throw new NotSerializableException( - "No configured serialization-bindings for class [%s]" format clazz.getName) - case Some((c, s)) ⇒ s - } + val ser = bindings.collectFirst { + case (c, s) if c.isAssignableFrom(clazz) ⇒ s + } getOrElse (throw new NotSerializableException( + "No configured serialization-bindings for class [%s]" format clazz.getName)) + // memorize for performance serializerMap.putIfAbsent(clazz, ser) match { case null ⇒ @@ -161,7 +161,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { */ private[akka] val bindings: Seq[ClassSerializer] = { val configuredBindings = for ((k: String, v: String) ← settings.SerializationBindings) yield { - val c = ReflectiveAccess.getClassFor(k, system.internalClassLoader).fold(throw _, (c: Class[_]) ⇒ c) + val c = ReflectiveAccess.getClassFor(k, system.internalClassLoader).fold(throw _, identity[Class[_]]) (c, serializers(v)) } sort(configuredBindings) From 0bd4663c911d7ddf4a5a7d9c1636982e29d88218 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 7 Feb 2012 09:50:03 +0100 Subject: [PATCH 11/34] PinnedDispatcher config and docs for dispatcher executor. * Update PinnedDispatcher config in tests and docs. See #1796 * Update dispatchers doc with info about executor. See #1795 --- .../scala/akka/actor/SupervisorMiscSpec.scala | 1 + .../akka/actor/dispatch/ActorModelSpec.scala | 2 ++ .../akka/actor/dispatch/PinnedActorSpec.scala | 1 + .../CallingThreadDispatcherModelSpec.scala | 1 + akka-actor/src/main/resources/reference.conf | 3 +- .../src/main/scala/akka/AkkaException.scala | 1 + akka-agent/src/main/resources/reference.conf | 2 ++ akka-docs/java/dispatchers.rst | 17 +++++++++-- .../docs/dispatcher/DispatcherDocSpec.scala | 30 ++++++++++++++++++- akka-docs/scala/dispatchers.rst | 17 +++++++++-- akka-remote/src/main/resources/reference.conf | 1 + akka-zeromq/src/main/resources/reference.conf | 1 + 12 files changed, 71 insertions(+), 6 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index fccfc75d98..caeacfb3db 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -14,6 +14,7 @@ import akka.util.duration._ object SupervisorMiscSpec { val config = """ pinned-dispatcher { + executor = thread-pool-executor type = PinnedDispatcher } test-dispatcher { 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 45e1954486..8c949f8776 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 @@ -435,6 +435,7 @@ object DispatcherModelSpec { val config = { """ boss { + executor = thread-pool-executor type = PinnedDispatcher } """ + @@ -506,6 +507,7 @@ object BalancingDispatcherModelSpec { val config = { """ boss { + executor = thread-pool-executor type = PinnedDispatcher } """ + diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala index 6c66784e5d..cf8dd5eab5 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala @@ -12,6 +12,7 @@ import akka.pattern.ask object PinnedActorSpec { val config = """ pinned-dispatcher { + executor = thread-pool-executor type = PinnedDispatcher } """ diff --git a/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala b/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala index ab0f7ec6eb..4693a56536 100644 --- a/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala @@ -16,6 +16,7 @@ object CallingThreadDispatcherModelSpec { val config = { """ boss { + executor = thread-pool-executor type = PinnedDispatcher } """ + diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index cdab8e968e..94e78fbbb7 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -156,7 +156,8 @@ akka { # the same type), PinnedDispatcher, or a FQCN to a class inheriting # MessageDispatcherConfigurator with a constructor with # com.typesafe.config.Config parameter and akka.dispatch.DispatcherPrerequisites - # parameters + # parameters. + # PinnedDispatcher must be used toghether with executor=thread-pool-executor. type = "Dispatcher" # Which kind of ExecutorService to use for this dispatcher diff --git a/akka-actor/src/main/scala/akka/AkkaException.scala b/akka-actor/src/main/scala/akka/AkkaException.scala index 47f7465535..ed079e678b 100644 --- a/akka-actor/src/main/scala/akka/AkkaException.scala +++ b/akka-actor/src/main/scala/akka/AkkaException.scala @@ -23,6 +23,7 @@ object AkkaException { sb.append("\tat %s\n" format trace(i)) sb.toString } + } /** diff --git a/akka-agent/src/main/resources/reference.conf b/akka-agent/src/main/resources/reference.conf index 67da6e3821..7009c0f432 100644 --- a/akka-agent/src/main/resources/reference.conf +++ b/akka-agent/src/main/resources/reference.conf @@ -10,11 +10,13 @@ akka { # The dispatcher used for agent-send-off actor send-off-dispatcher { + executor = thread-pool-executor type = PinnedDispatcher } # The dispatcher used for agent-alter-off actor alter-off-dispatcher { + executor = thread-pool-executor type = PinnedDispatcher } diff --git a/akka-docs/java/dispatchers.rst b/akka-docs/java/dispatchers.rst index fd117f65f9..fceb94abbc 100644 --- a/akka-docs/java/dispatchers.rst +++ b/akka-docs/java/dispatchers.rst @@ -55,6 +55,17 @@ Default values are taken from ``default-dispatcher``, i.e. all options doesn't n :ref:`configuration` for the default values of the ``default-dispatcher``. You can also override the values for the ``default-dispatcher`` in your configuration. +There are two different executor services: + +* executor = "fork-join-executor", ``ExecutorService`` based on ForkJoinPool (jsr166y). This is used by default for + ``default-dispatcher``. +* executor = "thread-pool-executor", ``ExecutorService`` based on ``java.util.concurrent.ThreadPoolExecutor``. + +Note that the pool size is configured differently for the two executor services. The configuration above +is an example for ``fork-join-executor``. Below is an example for ``thread-pool-executor``: + +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config + Let's now walk through the different dispatchers in more detail. Thread-based @@ -67,9 +78,11 @@ has worse performance and scalability than the event-based dispatcher but works a low frequency of messages and are allowed to go off and do their own thing for a longer period of time. Another advantage with this dispatcher is that Actors do not block threads for each other. -The ``PinnedDispatcher`` can't be configured, but is created and associated with an actor like this: +The ``PinnedDispatcher`` is configured like this: -.. includecode:: code/akka/docs/dispatcher/DispatcherDocTestBase.java#defining-pinned-dispatcher +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-pinned-dispatcher-config + +Note that it must be used with ``executor = "thread-pool-executor"``. Event-based ^^^^^^^^^^^ diff --git a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala index 0df4e3ca5b..cd57fbeddc 100644 --- a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala @@ -20,6 +20,27 @@ object DispatcherDocSpec { val config = """ //#my-dispatcher-config my-dispatcher { + # Dispatcher is the name of the event-based dispatcher + type = Dispatcher + # What kind of ExecutionService to use + executor = "fork-join-executor" + # Configuration for the fork join pool + fork-join-executor { + # Min number of threads to cap factor-based parallelism number to + parallelism-min = 2 + # Parallelism (threads) ... ceil(available processors * factor) + parallelism-factor = 2.0 + # Max number of threads to cap factor-based parallelism number to + parallelism-max = 10 + } + # Throughput defines the number of messages that are processed in a batch before the + # thread is returned to the pool. Set to 1 for as fair as possible. + throughput = 100 + } + //#my-dispatcher-config + + //#my-thread-pool-dispatcher-config + my-thread-pool-dispatcher { # Dispatcher is the name of the event-based dispatcher type = Dispatcher # What kind of ExecutionService to use @@ -37,7 +58,14 @@ object DispatcherDocSpec { # thread is returned to the pool. Set to 1 for as fair as possible. throughput = 100 } - //#my-dispatcher-config + //#my-thread-pool-dispatcher-config + + //#my-pinned-dispatcher-config + my-pinned-dispatcher { + executor = "thread-pool-executor" + type = PinnedDispatcher + } + //#my-pinned-dispatcher-config //#my-bounded-config my-dispatcher-bounded-queue { diff --git a/akka-docs/scala/dispatchers.rst b/akka-docs/scala/dispatchers.rst index c9923cf459..c6e6ae23e3 100644 --- a/akka-docs/scala/dispatchers.rst +++ b/akka-docs/scala/dispatchers.rst @@ -54,6 +54,17 @@ Default values are taken from ``default-dispatcher``, i.e. all options doesn't n :ref:`configuration` for the default values of the ``default-dispatcher``. You can also override the values for the ``default-dispatcher`` in your configuration. +There are two different executor services: + +* executor = "fork-join-executor", ``ExecutorService`` based on ForkJoinPool (jsr166y). This is used by default for + ``default-dispatcher``. +* executor = "thread-pool-executor", ``ExecutorService`` based on ``java.util.concurrent.ThreadPoolExecutor``. + +Note that the pool size is configured differently for the two executor services. The configuration above +is an example for ``fork-join-executor``. Below is an example for ``thread-pool-executor``: + +.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config + Let's now walk through the different dispatchers in more detail. Thread-based @@ -66,9 +77,11 @@ has worse performance and scalability than the event-based dispatcher but works a low frequency of messages and are allowed to go off and do their own thing for a longer period of time. Another advantage with this dispatcher is that Actors do not block threads for each other. -The ``PinnedDispatcher`` can't be configured, but is created and associated with an actor like this: +The ``PinnedDispatcher`` is configured like this: -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#defining-pinned-dispatcher +.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-pinned-dispatcher-config + +Note that it must be used with ``executor = "thread-pool-executor"``. Event-based ^^^^^^^^^^^ diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 1158d12295..4c5bca190f 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -133,6 +133,7 @@ akka { # The dispatcher used for the system actor "network-event-sender" network-event-sender-dispatcher { + executor = thread-pool-executor type = PinnedDispatcher } } diff --git a/akka-zeromq/src/main/resources/reference.conf b/akka-zeromq/src/main/resources/reference.conf index 54922b8386..cfb5756156 100644 --- a/akka-zeromq/src/main/resources/reference.conf +++ b/akka-zeromq/src/main/resources/reference.conf @@ -15,6 +15,7 @@ akka { 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 + executor = thread-pool-executor type = "PinnedDispatcher" } } From 9c8c0d42c56c81d54aeef9376d0393645ed36411 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 6 Feb 2012 14:19:59 +0100 Subject: [PATCH 12/34] Add @SerialVersionUID to Serializable classes. See #1786 * @SerialVersionUID(1L) * UntypedActorFactory serialization, and test * Removed Serializable from Serializers --- .../akka/serialization/SerializeSpec.scala | 26 +++++++++++++++---- .../src/main/scala/akka/AkkaException.scala | 1 + .../src/main/scala/akka/actor/ActorPath.scala | 5 +++- .../src/main/scala/akka/actor/ActorRef.scala | 2 ++ .../src/main/scala/akka/actor/Deployer.scala | 3 +++ .../src/main/scala/akka/actor/Props.scala | 1 + .../main/scala/akka/actor/TypedActor.scala | 1 + .../main/scala/akka/actor/UntypedActor.scala | 2 +- .../akka/dispatch/AbstractDispatcher.scala | 2 +- .../src/main/scala/akka/routing/Routing.scala | 7 +++++ .../scala/akka/serialization/Format.scala | 6 ++--- .../scala/akka/serialization/Serializer.scala | 2 +- .../src/main/scala/akka/util/Duration.scala | 4 +++ .../akka/remote/RemoteActorRefProvider.scala | 1 + 14 files changed, 51 insertions(+), 12 deletions(-) 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..6501d8d43d 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -203,6 +203,10 @@ object VerifySerializabilitySpec { } } + class FooUntypedActor extends UntypedActor { + def onReceive(message: Any) {} + } + class NonSerializableActor(system: ActorSystem) extends Actor { def receive = { case s: String ⇒ sender ! s @@ -210,6 +214,7 @@ object VerifySerializabilitySpec { } } +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class VerifySerializabilitySpec extends AkkaSpec(VerifySerializabilitySpec.conf) { import VerifySerializabilitySpec._ implicit val timeout = Timeout(5 seconds) @@ -221,17 +226,28 @@ class VerifySerializabilitySpec extends AkkaSpec(VerifySerializabilitySpec.conf) "verify creators" in { val a = system.actorOf(Props[FooActor]) - intercept[NotSerializableException] { - Await.result(a ? new AnyRef, timeout.duration) - } system stop a + + val b = system.actorOf(Props(new FooActor)) + system stop b + + val c = system.actorOf(Props().withCreator(new UntypedActorFactory { + def create() = new FooUntypedActor + })) + system stop c + + intercept[java.io.NotSerializableException] { + val d = system.actorOf(Props(new NonSerializableActor(system))) + } + } "verify messages" in { val a = system.actorOf(Props[FooActor]) Await.result(a ? "pigdog", timeout.duration) must be("pigdog") - intercept[java.io.NotSerializableException] { - val b = system.actorOf(Props(new NonSerializableActor(system))) + + intercept[NotSerializableException] { + Await.result(a ? new AnyRef, timeout.duration) } system stop a } diff --git a/akka-actor/src/main/scala/akka/AkkaException.scala b/akka-actor/src/main/scala/akka/AkkaException.scala index 47f7465535..ce1c01dcc5 100644 --- a/akka-actor/src/main/scala/akka/AkkaException.scala +++ b/akka-actor/src/main/scala/akka/AkkaException.scala @@ -33,6 +33,7 @@ object AkkaException { *
  • toLongString which also includes the stack trace
  • * */ +@SerialVersionUID(1L) class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable { val uuid = "%s_%s".format(AkkaException.hostname, newUuid) diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index 8997b119e8..751aa05768 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -38,6 +38,7 @@ object ActorPath { * is sorted by path elements FROM RIGHT TO LEFT, where RootActorPath > * ChildActorPath in case the number of elements is different. */ +@SerialVersionUID(1L) sealed trait ActorPath extends Comparable[ActorPath] with Serializable { /** * The Address under which this path can be reached; walks up the tree to @@ -108,6 +109,7 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable { * Root of the hierarchy of ActorPaths. There is exactly root per ActorSystem * and node (for remote-enabled or clustered systems). */ +@SerialVersionUID(1L) final case class RootActorPath(address: Address, name: String = "/") extends ActorPath { def parent: ActorPath = this @@ -130,6 +132,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act } } +@SerialVersionUID(1L) final class ChildActorPath(val parent: ActorPath, val name: String) extends ActorPath { if (name.indexOf('/') != -1) throw new IllegalArgumentException("/ is a path separator and is not legal in ActorPath names: [%s]" format name) @@ -157,7 +160,7 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto // TODO research whether this should be cached somehow (might be fast enough, but creates GC pressure) /* - * idea: add one field which holds the total length (because that is known) + * idea: add one field which holds the total length (because that is known) * so that only one String needs to be allocated before traversal; this is * cheaper than any cache */ diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 38e8ab679f..621e1902b4 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -333,6 +333,7 @@ private[akka] class LocalActorRef private[akka] ( /** * Memento pattern for serializing ActorRefs transparently */ +@SerialVersionUID(1L) case class SerializedActorRef private (path: String) { import akka.serialization.Serialization.currentSystem @@ -397,6 +398,7 @@ private[akka] object MinimalActorRef { case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) private[akka] object DeadLetterActorRef { + @SerialVersionUID(1L) class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance? @throws(classOf[java.io.ObjectStreamException]) private def readResolve(): AnyRef = Serialization.currentSystem.value.deadLetters diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index d561c74413..dbd2f92aa0 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -25,6 +25,7 @@ import akka.util.ReflectiveAccess * context.actorOf(someProps, "someName", Deploy(scope = RemoteScope("someOtherNodeName"))) * }}} */ +@SerialVersionUID(1L) final case class Deploy( path: String = "", config: Config = ConfigFactory.empty, @@ -62,6 +63,7 @@ trait Scope { def withFallback(other: Scope): Scope } +@SerialVersionUID(1L) case object LocalScope extends Scope { /** * Java API @@ -74,6 +76,7 @@ case object LocalScope extends Scope { /** * This is the default value and as such allows overrides. */ +@SerialVersionUID(1L) case object NoScopeGiven extends Scope { def withFallback(other: Scope): Scope = other } diff --git a/akka-actor/src/main/scala/akka/actor/Props.scala b/akka-actor/src/main/scala/akka/actor/Props.scala index 74cf45ebe1..ff25324402 100644 --- a/akka-actor/src/main/scala/akka/actor/Props.scala +++ b/akka-actor/src/main/scala/akka/actor/Props.scala @@ -103,6 +103,7 @@ object Props { * Props props = new Props(MyActor.class).withRouter(new RoundRobinRouter(..)); * }}} */ +@SerialVersionUID(1L) case class Props( creator: () ⇒ Actor = Props.defaultCreator, dispatcher: String = Dispatchers.DefaultDispatcherId, diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 41eba86bbd..c98f6aa55e 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -429,6 +429,7 @@ object TypedProps { * TypedProps is a TypedActor configuration object, that is thread safe and fully sharable. * It's used in TypedActorFactory.typedActorOf to configure a TypedActor instance. */ +@SerialVersionUID(1L) case class TypedProps[T <: AnyRef] protected[TypedProps] ( interfaces: Seq[Class[_]], creator: () ⇒ T, diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index daa7467196..a5ebeb851c 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -158,4 +158,4 @@ abstract class UntypedActor extends Actor { /** * Factory closure for an UntypedActor, to be used with 'Actors.actorOf(factory)'. */ -trait UntypedActorFactory extends Creator[Actor] +trait UntypedActorFactory extends Creator[Actor] with Serializable diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 0cf6c4a77b..e32631108c 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -168,7 +168,7 @@ object MessageDispatcher { implicit def defaultDispatcher(implicit system: ActorSystem): MessageDispatcher = system.dispatcher } -abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) extends AbstractMessageDispatcher with Serializable with Executor with ExecutionContext { +abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) extends AbstractMessageDispatcher with Executor with ExecutionContext { import MessageDispatcher._ import AbstractMessageDispatcher.{ inhabitantsUpdater, shutdownScheduleUpdater } diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index f2e4e9827c..62a00d510e 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -299,6 +299,7 @@ case class Destination(sender: ActorRef, recipient: ActorRef) * from lower-precendence sources. The decision whether or not to create a * router is taken in the LocalActorRefProvider based on Props. */ +@SerialVersionUID(1L) case object NoRouter extends RouterConfig { def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null override def withFallback(other: RouterConfig): RouterConfig = other @@ -315,6 +316,7 @@ case object FromConfig extends RouterConfig { /** * Java API: Router configuration which has no default, i.e. external configuration is required. */ +@SerialVersionUID(1L) case class 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)") @@ -345,6 +347,7 @@ object RoundRobinRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ +@SerialVersionUID(1L) case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None) extends RouterConfig with RoundRobinLike { @@ -424,6 +427,7 @@ object RandomRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ +@SerialVersionUID(1L) case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None) extends RouterConfig with RandomLike { @@ -509,6 +513,7 @@ object SmallestMailboxRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ +@SerialVersionUID(1L) case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None) extends RouterConfig with SmallestMailboxLike { @@ -653,6 +658,7 @@ object BroadcastRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ +@SerialVersionUID(1L) case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None) extends RouterConfig with BroadcastLike { @@ -724,6 +730,7 @@ object ScatterGatherFirstCompletedRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ +@SerialVersionUID(1L) case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: Duration, override val resizer: Option[Resizer] = None) extends RouterConfig with ScatterGatherFirstCompletedLike { diff --git a/akka-actor/src/main/scala/akka/serialization/Format.scala b/akka-actor/src/main/scala/akka/serialization/Format.scala index 41a8eed658..57ae659cec 100644 --- a/akka-actor/src/main/scala/akka/serialization/Format.scala +++ b/akka-actor/src/main/scala/akka/serialization/Format.scala @@ -7,7 +7,7 @@ package akka.serialization import akka.actor.Actor /** - * trait Serializer extends scala.Serializable { + * trait Serializer { * @volatile * var classLoader: Option[ClassLoader] = None * def deepClone(obj: AnyRef): AnyRef = fromBinary(toBinary(obj), Some(obj.getClass)) @@ -74,7 +74,7 @@ trait Format[T <: Actor] extends FromBinary[T] with ToBinary[T] * } * */ -trait StatelessActorFormat[T <: Actor] extends Format[T] with scala.Serializable { +trait StatelessActorFormat[T <: Actor] extends Format[T] { def fromBinary(bytes: Array[Byte], act: T) = act def toBinary(ac: T) = Array.empty[Byte] @@ -94,7 +94,7 @@ trait StatelessActorFormat[T <: Actor] extends Format[T] with scala.Serializable * } * */ -trait SerializerBasedActorFormat[T <: Actor] extends Format[T] with scala.Serializable { +trait SerializerBasedActorFormat[T <: Actor] extends Format[T] { val serializer: Serializer def fromBinary(bytes: Array[Byte], act: T) = serializer.fromBinary(bytes, Some(act.getClass)).asInstanceOf[T] diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index ffe7f50de9..5a9ba97a98 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -10,7 +10,7 @@ import akka.util.ClassLoaderObjectInputStream /** * A Serializer represents a bimap between an object and an array of bytes representing that object */ -trait Serializer extends scala.Serializable { +trait Serializer { /** * 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 diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index 2b6aae1eb3..176960b067 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -8,6 +8,7 @@ import java.util.concurrent.TimeUnit import TimeUnit._ import java.lang.{ Double ⇒ JDouble } +@SerialVersionUID(1L) case class Deadline private (time: Duration) { def +(other: Duration): Deadline = copy(time = time + other) def -(other: Duration): Deadline = copy(time = time - other) @@ -232,6 +233,7 @@ object Duration { * val d3 = d2 + 1.millisecond * */ +@SerialVersionUID(1L) abstract class Duration extends Serializable with Ordered[Duration] { def length: Long def unit: TimeUnit @@ -276,6 +278,7 @@ object FiniteDuration { } } +@SerialVersionUID(1L) class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration { import Duration._ @@ -525,6 +528,7 @@ class DurationDouble(d: Double) { def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS)) } +@SerialVersionUID(1L) case class Timeout(duration: Duration) { def this(timeout: Long) = this(Duration(timeout, TimeUnit.MILLISECONDS)) def this(length: Long, unit: TimeUnit) = this(Duration(length, unit)) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 872be5aa41..36ed73c769 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -222,6 +222,7 @@ trait RemoteRef extends ActorRefScope { * Remote ActorRef that is used when referencing the Actor on a different node than its "home" node. * This reference is network-aware (remembers its origin) and immutable. */ +@SerialVersionUID(1L) private[akka] class RemoteActorRef private[akka] ( val provider: RemoteActorRefProvider, remote: RemoteTransport, From 2a167acd924dc0c744ae659196bd69d8fd53ad3d Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 7 Feb 2012 10:28:42 +0100 Subject: [PATCH 13/34] Some cleanup and additional tests for Serializable. See #1786 * Tried to add @SerialVersionUID(1L) but that removed static forwarders, due to https://issues.scala-lang.org/browse/SI-4804 * Added Serializable to UntypedActorFactory * Removed some unecessary Serializable --- akka-actor/src/main/scala/akka/AkkaException.scala | 2 +- .../src/main/scala/akka/actor/ActorPath.scala | 6 +++--- .../src/main/scala/akka/actor/ActorRef.scala | 4 ++-- .../src/main/scala/akka/actor/Deployer.scala | 6 +++--- .../src/main/scala/akka/actor/FaultHandling.scala | 4 ++-- akka-actor/src/main/scala/akka/actor/Props.scala | 2 +- .../src/main/scala/akka/actor/TypedActor.scala | 2 +- .../src/main/scala/akka/routing/Routing.scala | 14 +++++++------- akka-actor/src/main/scala/akka/util/Duration.scala | 8 ++++---- .../scala/akka/remote/RemoteActorRefProvider.scala | 1 - 10 files changed, 24 insertions(+), 25 deletions(-) diff --git a/akka-actor/src/main/scala/akka/AkkaException.scala b/akka-actor/src/main/scala/akka/AkkaException.scala index ce1c01dcc5..91d88fe641 100644 --- a/akka-actor/src/main/scala/akka/AkkaException.scala +++ b/akka-actor/src/main/scala/akka/AkkaException.scala @@ -33,7 +33,7 @@ object AkkaException { *
  • toLongString which also includes the stack trace
  • * */ -@SerialVersionUID(1L) +//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) diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index 751aa05768..4ebcec0dbb 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -38,7 +38,7 @@ object ActorPath { * is sorted by path elements FROM RIGHT TO LEFT, where RootActorPath > * ChildActorPath in case the number of elements is different. */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed sealed trait ActorPath extends Comparable[ActorPath] with Serializable { /** * The Address under which this path can be reached; walks up the tree to @@ -109,7 +109,7 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable { * Root of the hierarchy of ActorPaths. There is exactly root per ActorSystem * and node (for remote-enabled or clustered systems). */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed final case class RootActorPath(address: Address, name: String = "/") extends ActorPath { def parent: ActorPath = this @@ -132,7 +132,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act } } -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed final class ChildActorPath(val parent: ActorPath, val name: String) extends ActorPath { if (name.indexOf('/') != -1) throw new IllegalArgumentException("/ is a path separator and is not legal in ActorPath names: [%s]" format name) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 621e1902b4..7485d8b11c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -333,7 +333,7 @@ private[akka] class LocalActorRef private[akka] ( /** * Memento pattern for serializing ActorRefs transparently */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class SerializedActorRef private (path: String) { import akka.serialization.Serialization.currentSystem @@ -398,7 +398,7 @@ private[akka] object MinimalActorRef { case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) private[akka] object DeadLetterActorRef { - @SerialVersionUID(1L) + //TODO add @SerialVersionUID(1L) when SI-4804 is fixed class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance? @throws(classOf[java.io.ObjectStreamException]) private def readResolve(): AnyRef = Serialization.currentSystem.value.deadLetters diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index dbd2f92aa0..3da1a7ca35 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -25,7 +25,7 @@ import akka.util.ReflectiveAccess * context.actorOf(someProps, "someName", Deploy(scope = RemoteScope("someOtherNodeName"))) * }}} */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed final case class Deploy( path: String = "", config: Config = ConfigFactory.empty, @@ -63,7 +63,7 @@ trait Scope { def withFallback(other: Scope): Scope } -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed case object LocalScope extends Scope { /** * Java API @@ -76,7 +76,7 @@ case object LocalScope extends Scope { /** * This is the default value and as such allows overrides. */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed case object NoScopeGiven extends Scope { def withFallback(other: Scope): Scope = other } diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 8a21f841bb..8b6abc5f5d 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -233,7 +233,7 @@ abstract class SupervisorStrategy { * * @param maxNrOfRetries the number of times an actor is allowed to be restarted, negative value means no limit * @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window - * @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a + * @param decider mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a * `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates. */ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider) @@ -279,7 +279,7 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration * * @param maxNrOfRetries the number of times an actor is allowed to be restarted, negative value means no limit * @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window - * @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a + * @param decider mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a * `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates. */ case class OneForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider) diff --git a/akka-actor/src/main/scala/akka/actor/Props.scala b/akka-actor/src/main/scala/akka/actor/Props.scala index ff25324402..b78c0a5eb4 100644 --- a/akka-actor/src/main/scala/akka/actor/Props.scala +++ b/akka-actor/src/main/scala/akka/actor/Props.scala @@ -103,7 +103,7 @@ object Props { * Props props = new Props(MyActor.class).withRouter(new RoundRobinRouter(..)); * }}} */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed when SI-4804 is fixed case class Props( creator: () ⇒ Actor = Props.defaultCreator, dispatcher: String = Dispatchers.DefaultDispatcherId, diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index c98f6aa55e..d86dc7dca4 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -429,7 +429,7 @@ object TypedProps { * TypedProps is a TypedActor configuration object, that is thread safe and fully sharable. * It's used in TypedActorFactory.typedActorOf to configure a TypedActor instance. */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class TypedProps[T <: AnyRef] protected[TypedProps] ( interfaces: Seq[Class[_]], creator: () ⇒ T, diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 62a00d510e..c200be0871 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -299,7 +299,7 @@ case class Destination(sender: ActorRef, recipient: ActorRef) * from lower-precendence sources. The decision whether or not to create a * router is taken in the LocalActorRefProvider based on Props. */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed case object NoRouter extends RouterConfig { def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null override def withFallback(other: RouterConfig): RouterConfig = other @@ -316,7 +316,7 @@ case object FromConfig extends RouterConfig { /** * Java API: Router configuration which has no default, i.e. external configuration is required. */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class 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)") @@ -347,7 +347,7 @@ object RoundRobinRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ -@SerialVersionUID(1L) +//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) extends RouterConfig with RoundRobinLike { @@ -427,7 +427,7 @@ object RandomRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ -@SerialVersionUID(1L) +//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) extends RouterConfig with RandomLike { @@ -513,7 +513,7 @@ object SmallestMailboxRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ -@SerialVersionUID(1L) +//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) extends RouterConfig with SmallestMailboxLike { @@ -658,7 +658,7 @@ object BroadcastRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ -@SerialVersionUID(1L) +//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) extends RouterConfig with BroadcastLike { @@ -730,7 +730,7 @@ object ScatterGatherFirstCompletedRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ -@SerialVersionUID(1L) +//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) extends RouterConfig with ScatterGatherFirstCompletedLike { diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index 176960b067..67056d7425 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -8,7 +8,7 @@ import java.util.concurrent.TimeUnit import TimeUnit._ import java.lang.{ Double ⇒ JDouble } -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class Deadline private (time: Duration) { def +(other: Duration): Deadline = copy(time = time + other) def -(other: Duration): Deadline = copy(time = time - other) @@ -233,7 +233,7 @@ object Duration { * val d3 = d2 + 1.millisecond * */ -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed abstract class Duration extends Serializable with Ordered[Duration] { def length: Long def unit: TimeUnit @@ -278,7 +278,7 @@ object FiniteDuration { } } -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration { import Duration._ @@ -528,7 +528,7 @@ class DurationDouble(d: Double) { def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS)) } -@SerialVersionUID(1L) +//TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class Timeout(duration: Duration) { def this(timeout: Long) = this(Duration(timeout, TimeUnit.MILLISECONDS)) def this(length: Long, unit: TimeUnit) = this(Duration(length, unit)) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 36ed73c769..872be5aa41 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -222,7 +222,6 @@ trait RemoteRef extends ActorRefScope { * Remote ActorRef that is used when referencing the Actor on a different node than its "home" node. * This reference is network-aware (remembers its origin) and immutable. */ -@SerialVersionUID(1L) private[akka] class RemoteActorRef private[akka] ( val provider: RemoteActorRefProvider, remote: RemoteTransport, From e12804660fcc481da13adedc2911f168cc26a08c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 7 Feb 2012 11:02:01 +0100 Subject: [PATCH 14/34] Fixing doc error in remoting --- akka-docs/java/remoting.rst | 2 +- akka-docs/scala/remoting.rst | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-docs/java/remoting.rst b/akka-docs/java/remoting.rst index 196d7a40a5..9e0eb4eec9 100644 --- a/akka-docs/java/remoting.rst +++ b/akka-docs/java/remoting.rst @@ -26,7 +26,7 @@ to your ``application.conf`` file:: } remote { transport = "akka.remote.netty.NettyRemoteTransport" - server { + netty { hostname = "127.0.0.1" port = 2552 } diff --git a/akka-docs/scala/remoting.rst b/akka-docs/scala/remoting.rst index ae2aa0f411..f874e15a1b 100644 --- a/akka-docs/scala/remoting.rst +++ b/akka-docs/scala/remoting.rst @@ -27,7 +27,7 @@ to your ``application.conf`` file:: } remote { transport = "akka.remote.netty.NettyRemoteTransport" - server { + netty { hostname = "127.0.0.1" port = 2552 } From 3f067581428daea82491171a5ddd9c8374ceaa23 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 7 Feb 2012 11:14:09 +0100 Subject: [PATCH 15/34] Adding link to config for application.conf --- akka-kernel/src/main/dist/config/application.conf | 1 + 1 file changed, 1 insertion(+) diff --git a/akka-kernel/src/main/dist/config/application.conf b/akka-kernel/src/main/dist/config/application.conf index 2f7ad95abd..682d41e484 100644 --- a/akka-kernel/src/main/dist/config/application.conf +++ b/akka-kernel/src/main/dist/config/application.conf @@ -1,2 +1,3 @@ # In this file you can override any option defined in the 'reference.conf' files. # Copy in all or parts of the 'reference.conf' files and modify as you please. +# For more info about config, please visit the Akka Documentation: http://akka.io/docs/akka/2.0-SNAPSHOT/ From ac3868659305b2e1a891e7ac88ee66e2eb7ee9e8 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 7 Feb 2012 12:00:50 +0100 Subject: [PATCH 16/34] System.exit on fatal error and removed actor null check. See #1799 and #1768 * Config property jvmExitOnFatalError * System.exit in case of fatal error, such as OutOfMemoryError * Adjusted NonFatal extractor, ok with StackOverflowError, not ok with LinkageError * Removed the actor null check in ActorCell --- .../src/test/scala/akka/config/ConfigSpec.scala | 3 +++ .../src/test/scala/akka/util/NonFatalSpec.scala | 12 +++++------- akka-actor/src/main/resources/reference.conf | 5 ++++- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 5 +---- .../src/main/scala/akka/actor/ActorSystem.scala | 2 ++ akka-actor/src/main/scala/akka/util/NonFatal.scala | 10 ++++++---- 6 files changed, 21 insertions(+), 16 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 ad39057d1d..9a0eab8830 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -34,6 +34,9 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) { getMilliseconds("akka.scheduler.tickDuration") must equal(100) settings.SchedulerTickDuration must equal(100 millis) + + settings.Daemonicity must be(false) + settings.JvmExitOnFatalError must be(true) } { diff --git a/akka-actor-tests/src/test/scala/akka/util/NonFatalSpec.scala b/akka-actor-tests/src/test/scala/akka/util/NonFatalSpec.scala index f7d9789ec3..0c4bc295fb 100644 --- a/akka-actor-tests/src/test/scala/akka/util/NonFatalSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/util/NonFatalSpec.scala @@ -19,18 +19,16 @@ class NonFatalSpec extends AkkaSpec with MustMatchers { } } - "not match StackOverflowError" in { + "match StackOverflowError" in { //not @tailrec def blowUp(n: Long): Long = { blowUp(n + 1) + 1 } - intercept[StackOverflowError] { - try { - blowUp(0) - } catch { - case NonFatal(e) ⇒ assert(false) - } + try { + blowUp(0) + } catch { + case NonFatal(e) ⇒ // as expected } } diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index cdab8e968e..db81180fe7 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -36,6 +36,9 @@ akka { # Toggles whether the threads created by this ActorSystem should be daemons or not daemonic = off + # JVM shutdown, System.exit(-1), in case of a fatal error, such as OutOfMemoryError + jvmExitOnFatalError = on + actor { provider = "akka.actor.LocalActorRefProvider" @@ -97,7 +100,7 @@ akka { paths = [] } - # Routers with dynamically resizable number of routees; this feature is enabled + # Routers with dynamically resizable number of routees; this feature is enabled # by including (parts of) this section in the deployment resizer { diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index f86f64fa99..2b56bef5d1 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -476,10 +476,7 @@ private[akka] class ActorCell( cancelReceiveTimeout() // FIXME: leave this here??? messageHandle.message match { case msg: AutoReceivedMessage ⇒ autoReceiveMessage(messageHandle) - // FIXME: actor can be null when creation fails with fatal error, why? - case msg if actor == null ⇒ - system.eventStream.publish(Warning(self.path.toString, this.getClass, "Ignoring message due to null actor [%s]" format msg)) - case msg ⇒ actor(msg) + case msg ⇒ actor(msg) } currentMessage = null // reset current message after successful invocation } catch { diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index b20cdf5f1c..72f2505994 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -92,6 +92,7 @@ object ActorSystem { final val SchedulerTickDuration = Duration(getMilliseconds("akka.scheduler.tickDuration"), MILLISECONDS) final val SchedulerTicksPerWheel = getInt("akka.scheduler.ticksPerWheel") final val Daemonicity = getBoolean("akka.daemonic") + final val JvmExitOnFatalError = getBoolean("akka.jvmExitOnFatalError") if (ConfigVersion != Version) throw new ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]") @@ -348,6 +349,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten log.error(cause, "Uncaught error from thread [{}]", thread.getName) cause match { case NonFatal(_) | _: InterruptedException ⇒ + case _ if settings.JvmExitOnFatalError ⇒ System.exit(-1) case _ ⇒ shutdown() } } diff --git a/akka-actor/src/main/scala/akka/util/NonFatal.scala b/akka-actor/src/main/scala/akka/util/NonFatal.scala index 36bb1960e9..ae7d91c7a3 100644 --- a/akka-actor/src/main/scala/akka/util/NonFatal.scala +++ b/akka-actor/src/main/scala/akka/util/NonFatal.scala @@ -5,8 +5,9 @@ package akka.util /** * Extractor of non-fatal Throwables. Will not match fatal errors - * like VirtualMachineError (OutOfMemoryError, StackOverflowError) - * ThreadDeath, and InterruptedException. + * like VirtualMachineError (OutOfMemoryError) + * ThreadDeath, LinkageError and InterruptedException. + * StackOverflowError is matched, i.e. considered non-fatal. * * Usage to catch all harmless throwables: * {{{ @@ -20,8 +21,9 @@ package akka.util object NonFatal { def unapply(t: Throwable): Option[Throwable] = t match { - // VirtualMachineError includes OutOfMemoryError, StackOverflowError and other fatal errors - case _: VirtualMachineError | _: ThreadDeath | _: InterruptedException ⇒ None + case e: StackOverflowError ⇒ Some(e) // StackOverflowError ok even though it is a VirtualMachineError + // VirtualMachineError includes OutOfMemoryError and other fatal errors + case _: VirtualMachineError | _: ThreadDeath | _: InterruptedException | _: LinkageError ⇒ None case e ⇒ Some(e) } From 8b9f1caf67b0c9bd7c75358b17c3584fb39e9ef5 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 7 Feb 2012 15:11:16 +0100 Subject: [PATCH 17/34] change serialization to strictly rely on subtyping MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - when encountering new message type, check all bindings which map apply - if multiple are found, choose the most specific one if that exists or verify that all mappings yield the same serializer - in case of remaining ambiguity, throw exception - also add special handling for “none” serializer mapping: turn off a default --- .../akka/serialization/SerializeSpec.scala | 12 +++----- akka-actor/src/main/resources/reference.conf | 9 +++--- .../akka/serialization/Serialization.scala | 28 +++++++++++-------- akka-docs/java/serialization.rst | 28 +++++++++++++------ akka-docs/scala/serialization.rst | 28 +++++++++++++------ akka-remote/src/main/resources/reference.conf | 11 ++++---- 6 files changed, 72 insertions(+), 44 deletions(-) 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 7b91e29bcb..9ab93ed974 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -179,14 +179,10 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) { ser.serializerFor(classOf[ExtendedPlainMessage]).getClass must be(classOf[TestSerializer]) } - "resolve serializer for message with several bindings" in { - ser.serializerFor(classOf[Both]).getClass must be(classOf[TestSerializer]) - } - - "resolve serializer in the order of the bindings" in { - ser.serializerFor(classOf[A]).getClass must be(classOf[JavaSerializer]) - ser.serializerFor(classOf[B]).getClass must be(classOf[TestSerializer]) - ser.serializerFor(classOf[C]).getClass must be(classOf[JavaSerializer]) + "throw exception for message with several bindings" in { + intercept[java.io.NotSerializableException] { + ser.serializerFor(classOf[Both]) + } } "resolve serializer in the order of most specific binding first" in { diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index fb0ad24a75..4e6c42c704 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -269,15 +269,16 @@ akka { # Entries for pluggable serializers and their bindings. serializers { java = "akka.serialization.JavaSerializer" - # proto = "akka.serialization.ProtobufSerializer" } # Class to Serializer binding. You only need to specify the name of an interface - # or abstract base class of the messages. In case of ambiguity it is primarily - # using the most specific configured class, and secondly the entry configured first. + # or abstract base class of the messages. In case of ambiguity it is + # using the most specific configured class, throwing an exception otherwise. + # + # To disable one of the default serializers, assign its class to "none", like + # "java.io.Serializable" = none serialization-bindings { "java.io.Serializable" = java - #"com.google.protobuf.Message" = proto } } diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 1a23833383..3e86de6c1c 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -125,17 +125,23 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { def serializerFor(clazz: Class[_]): Serializer = serializerMap.get(clazz) match { case null ⇒ - val ser = bindings.collectFirst { - case (c, s) if c.isAssignableFrom(clazz) ⇒ s - } getOrElse (throw new NotSerializableException( - "No configured serialization-bindings for class [%s]" format clazz.getName)) + // bindings are ordered from most specific to least specific + def unique(cs: Seq[Class[_]], ser: Set[Serializer]): Boolean = (cs forall (_ isAssignableFrom cs(0))) || ser.size == 1 - // memorize for performance - serializerMap.putIfAbsent(clazz, ser) match { - case null ⇒ - log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName) - ser - case some ⇒ some + val possible = bindings filter { _._1 isAssignableFrom clazz } + possible.size match { + case 0 ⇒ + throw new NotSerializableException("No configured serialization-bindings for class [%s]" format clazz.getName) + case x if x == 1 || unique(possible map (_._1), possible.map(_._2)(scala.collection.breakOut)) ⇒ + val ser = possible(0)._2 + serializerMap.putIfAbsent(clazz, ser) match { + case null ⇒ + log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName) + ser + case some ⇒ some + } + case _ ⇒ + throw new NotSerializableException("Multiple serializers found for " + clazz + ": " + possible) } case ser ⇒ ser } @@ -160,7 +166,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * It is primarily ordered by the most specific classes first, and secondly in the configured order. */ private[akka] val bindings: Seq[ClassSerializer] = { - val configuredBindings = for ((k: String, v: String) ← settings.SerializationBindings) yield { + val configuredBindings = for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield { val c = ReflectiveAccess.getClassFor(k, system.internalClassLoader).fold(throw _, identity[Class[_]]) (c, serializers(v)) } diff --git a/akka-docs/java/serialization.rst b/akka-docs/java/serialization.rst index b0721e82cc..dad2174f91 100644 --- a/akka-docs/java/serialization.rst +++ b/akka-docs/java/serialization.rst @@ -32,14 +32,26 @@ should be serialized using which ``Serializer``, this is done in the "akka.actor .. includecode:: ../scala/code/akka/docs/serialization/SerializationDocSpec.scala#serialization-bindings-config -You only need to specify the name of an interface or abstract base class of the messages. In case of ambiguity, -i.e. the message implements several of the configured classes, it is primarily using the most specific -configured class, and secondly the entry configured first. +You only need to specify the name of an interface or abstract base class of the +messages. In case of ambiguity, i.e. the message implements several of the +configured classes, the most specific configured class will be used, i.e. the +one of which all other candidates are superclasses. If this condition cannot be +met, because e.g. ``java.io.Serializable`` and ``MyOwnSerializable`` both apply +and neither is a subtype of the other, an exception will be thrown during +serialization. -Akka provides serializers for ``java.io.Serializable`` and `protobuf `_ -``com.google.protobuf.Message`` by default, so normally you don't need to add configuration for that, but -it can be done to force a specific serializer in case messages implements both ``java.io.Serializable`` -and ``com.google.protobuf.Message``. +Akka provides serializers for :class:`java.io.Serializable` and `protobuf +`_ +:class:`com.google.protobuf.GeneratedMessage` by default (the latter only if +depending on the akka-remote module), so normally you don't need to add +configuration for that; since :class:`com.google.protobuf.GeneratedMessage` +implements :class:`java.io.Serializable`, protobuf messages will always by +serialized using the protobuf protocol unless specifically overridden. In order +to disable a default serializer, map its marker type to “none”:: + + akka.actor.serialization-bindings { + "java.io.Serializable" = none + } Verification ------------ @@ -91,4 +103,4 @@ 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. diff --git a/akka-docs/scala/serialization.rst b/akka-docs/scala/serialization.rst index 9cfaf909b9..ec0f4ca706 100644 --- a/akka-docs/scala/serialization.rst +++ b/akka-docs/scala/serialization.rst @@ -32,14 +32,26 @@ should be serialized using which ``Serializer``, this is done in the "akka.actor .. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala#serialization-bindings-config -You only need to specify the name of an interface or abstract base class of the messages. In case of ambiguity, -i.e. the message implements several of the configured classes, it is primarily using the most specific -configured class, and secondly the entry configured first. +You only need to specify the name of an interface or abstract base class of the +messages. In case of ambiguity, i.e. the message implements several of the +configured classes, the most specific configured class will be used, i.e. the +one of which all other candidates are superclasses. If this condition cannot be +met, because e.g. ``java.io.Serializable`` and ``MyOwnSerializable`` both apply +and neither is a subtype of the other, an exception will be thrown during +serialization. -Akka provides serializers for ``java.io.Serializable`` and `protobuf `_ -``com.google.protobuf.Message`` by default, so normally you don't need to add configuration for that, but -it can be done to force a specific serializer in case messages implements both ``java.io.Serializable`` -and ``com.google.protobuf.Message``. +Akka provides serializers for :class:`java.io.Serializable` and `protobuf +`_ +:class:`com.google.protobuf.GeneratedMessage` by default (the latter only if +depending on the akka-remote module), so normally you don't need to add +configuration for that; since :class:`com.google.protobuf.GeneratedMessage` +implements :class:`java.io.Serializable`, protobuf messages will always by +serialized using the protobuf protocol unless specifically overridden. In order +to disable a default serializer, map its marker type to “none”:: + + akka.actor.serialization-bindings { + "java.io.Serializable" = none + } Verification ------------ @@ -89,4 +101,4 @@ 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. diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 38e0de27cc..a8854fc9df 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -11,17 +11,18 @@ akka { # Entries for pluggable serializers and their bindings. serializers { - java = "akka.serialization.JavaSerializer" proto = "akka.serialization.ProtobufSerializer" } # Class to Serializer binding. You only need to specify the name of an interface - # or abstract base class of the messages. In case of ambiguity it is primarily - # using the most specific configured class, and secondly the entry configured first. + # or abstract base class of the messages. In case of ambiguity it is + # using the most specific configured class, giving an error if two mappings are found + # which cannot be decided by sub-typing relation. serialization-bindings { - "com.google.protobuf.Message" = proto - "java.io.Serializable" = java + # Since com.google.protobuf.Message does not extend Serializable but GeneratedMessage + # does, need to use the more specific one here in order to avoid ambiguity + "com.google.protobuf.GeneratedMessage" = proto } deployment { From 224ce7f773f0ca8aeb2a7391596036e4b33fad7e Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 7 Feb 2012 15:51:41 +0100 Subject: [PATCH 18/34] tone it down: just a Warning in case of ambiguous serializers --- .../akka/serialization/SerializeSpec.scala | 16 ++++++++++++---- akka-actor/src/main/resources/reference.conf | 4 ++-- .../akka/serialization/Serialization.scala | 19 ++++++++++--------- akka-docs/java/serialization.rst | 3 +-- akka-docs/scala/serialization.rst | 3 +-- akka-remote/src/main/resources/reference.conf | 8 +++----- 6 files changed, 29 insertions(+), 24 deletions(-) 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 9ab93ed974..f510f848f8 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -4,7 +4,7 @@ package akka.serialization -import akka.testkit.AkkaSpec +import akka.testkit.{ AkkaSpec, EventFilter } import akka.actor._ import java.io._ import akka.dispatch.Await @@ -179,9 +179,17 @@ class SerializeSpec extends AkkaSpec(SerializeSpec.config) { ser.serializerFor(classOf[ExtendedPlainMessage]).getClass must be(classOf[TestSerializer]) } - "throw exception for message with several bindings" in { - intercept[java.io.NotSerializableException] { - ser.serializerFor(classOf[Both]) + "give warning for message with several bindings" in { + EventFilter.warning(start = "Multiple serializers found", occurrences = 1) intercept { + ser.serializerFor(classOf[Both]).getClass must be(classOf[TestSerializer]) + } + } + + "resolve serializer in the order of the bindings" in { + ser.serializerFor(classOf[A]).getClass must be(classOf[JavaSerializer]) + ser.serializerFor(classOf[B]).getClass must be(classOf[TestSerializer]) + EventFilter.warning(start = "Multiple serializers found", occurrences = 1) intercept { + ser.serializerFor(classOf[C]).getClass must be(classOf[JavaSerializer]) } } diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 4e6c42c704..ce9bf684e8 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -272,8 +272,8 @@ akka { } # Class to Serializer binding. You only need to specify the name of an interface - # or abstract base class of the messages. In case of ambiguity it is - # using the most specific configured class, throwing an exception otherwise. + # or abstract base class of the messages. In case of ambiguity it is using the + # most specific configured class, or giving a warning and choosing the “first” one. # # To disable one of the default serializers, assign its class to "none", like # "java.io.Serializable" = none diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 3e86de6c1c..b26adcfc36 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -129,19 +129,20 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { def unique(cs: Seq[Class[_]], ser: Set[Serializer]): Boolean = (cs forall (_ isAssignableFrom cs(0))) || ser.size == 1 val possible = bindings filter { _._1 isAssignableFrom clazz } - possible.size match { + val ser = possible.size match { case 0 ⇒ throw new NotSerializableException("No configured serialization-bindings for class [%s]" format clazz.getName) case x if x == 1 || unique(possible map (_._1), possible.map(_._2)(scala.collection.breakOut)) ⇒ - val ser = possible(0)._2 - serializerMap.putIfAbsent(clazz, ser) match { - case null ⇒ - log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName) - ser - case some ⇒ some - } + possible(0)._2 case _ ⇒ - throw new NotSerializableException("Multiple serializers found for " + clazz + ": " + possible) + log.warning("Multiple serializers found for " + clazz + ", choosing first: " + possible) + possible(0)._2 + } + serializerMap.putIfAbsent(clazz, ser) match { + case null ⇒ + log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName) + ser + case some ⇒ some } case ser ⇒ ser } diff --git a/akka-docs/java/serialization.rst b/akka-docs/java/serialization.rst index dad2174f91..e44d69f162 100644 --- a/akka-docs/java/serialization.rst +++ b/akka-docs/java/serialization.rst @@ -37,8 +37,7 @@ messages. In case of ambiguity, i.e. the message implements several of the configured classes, the most specific configured class will be used, i.e. the one of which all other candidates are superclasses. If this condition cannot be met, because e.g. ``java.io.Serializable`` and ``MyOwnSerializable`` both apply -and neither is a subtype of the other, an exception will be thrown during -serialization. +and neither is a subtype of the other, a warning will be issued. Akka provides serializers for :class:`java.io.Serializable` and `protobuf `_ diff --git a/akka-docs/scala/serialization.rst b/akka-docs/scala/serialization.rst index ec0f4ca706..4735548140 100644 --- a/akka-docs/scala/serialization.rst +++ b/akka-docs/scala/serialization.rst @@ -37,8 +37,7 @@ messages. In case of ambiguity, i.e. the message implements several of the configured classes, the most specific configured class will be used, i.e. the one of which all other candidates are superclasses. If this condition cannot be met, because e.g. ``java.io.Serializable`` and ``MyOwnSerializable`` both apply -and neither is a subtype of the other, an exception will be thrown during -serialization. +and neither is a subtype of the other, a warning will be issued Akka provides serializers for :class:`java.io.Serializable` and `protobuf `_ diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index a8854fc9df..017b531faa 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -5,20 +5,18 @@ # This the reference config file has all the default settings. # Make your edits/overrides in your application.conf. +# comments above akka.actor settings left out where they are already in akka- +# actor.jar, because otherwise they would be repeated in config rendering. + akka { actor { - # Entries for pluggable serializers and their bindings. serializers { proto = "akka.serialization.ProtobufSerializer" } - # Class to Serializer binding. You only need to specify the name of an interface - # or abstract base class of the messages. In case of ambiguity it is - # using the most specific configured class, giving an error if two mappings are found - # which cannot be decided by sub-typing relation. serialization-bindings { # Since com.google.protobuf.Message does not extend Serializable but GeneratedMessage # does, need to use the more specific one here in order to avoid ambiguity From c40d76c0ab4cc77615c2c1d5fdb6291368008dc2 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 7 Feb 2012 16:21:48 +0100 Subject: [PATCH 19/34] review-based improvements on Serialization --- .../akka/serialization/Serialization.scala | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index b26adcfc36..fbcd5e9f9e 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -126,17 +126,18 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { serializerMap.get(clazz) match { case null ⇒ // bindings are ordered from most specific to least specific - def unique(cs: Seq[Class[_]], ser: Set[Serializer]): Boolean = (cs forall (_ isAssignableFrom cs(0))) || ser.size == 1 + def unique(possibilities: Seq[(Class[_], Serializer)]): Boolean = + possibilities.size == 1 || + (possibilities map (_._1) forall (_ isAssignableFrom possibilities(0)._1)) || + (possibilities map (_._2) forall (_ == possibilities(0)._2)) - val possible = bindings filter { _._1 isAssignableFrom clazz } - val ser = possible.size match { - case 0 ⇒ + val ser = bindings filter { _._1 isAssignableFrom clazz } match { + case Seq() ⇒ throw new NotSerializableException("No configured serialization-bindings for class [%s]" format clazz.getName) - case x if x == 1 || unique(possible map (_._1), possible.map(_._2)(scala.collection.breakOut)) ⇒ - possible(0)._2 - case _ ⇒ - log.warning("Multiple serializers found for " + clazz + ", choosing first: " + possible) - possible(0)._2 + case possibilities ⇒ + if (!unique(possibilities)) + log.warning("Multiple serializers found for " + clazz + ", choosing first: " + possibilities) + possibilities(0)._2 } serializerMap.putIfAbsent(clazz, ser) match { case null ⇒ From 4e00eb889f6229757d13b5e3bd9cbc8160be8c34 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 7 Feb 2012 11:21:49 +0100 Subject: [PATCH 20/34] Test for Slf4jEventHandler. See #1625 * Test of MDC parameters and other output using a custom logback appender * Logback test dependency * Changed to encoding instead of deprecated layout --- akka-docs/java/logging.rst | 8 +- akka-docs/scala/logging.rst | 8 +- .../sample/src/main/config/logback.xml | 4 +- .../{SLF4J.scala => Slf4jEventHandler.scala} | 0 .../src/test/resources/logback-test.xml | 26 ++++ .../event/slf4j/Slf4jEventHandlerSpec.scala | 127 ++++++++++++++++++ project/AkkaBuild.scala | 2 +- 7 files changed, 164 insertions(+), 11 deletions(-) rename akka-slf4j/src/main/scala/akka/event/slf4j/{SLF4J.scala => Slf4jEventHandler.scala} (100%) create mode 100644 akka-slf4j/src/test/resources/logback-test.xml create mode 100644 akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala diff --git a/akka-docs/java/logging.rst b/akka-docs/java/logging.rst index 398a940468..01dedee0ad 100644 --- a/akka-docs/java/logging.rst +++ b/akka-docs/java/logging.rst @@ -218,9 +218,9 @@ Mapped Diagnostic Context (MDC) with attribute name ``sourceThread``. With Logback the thread name is available with ``%X{sourceThread}`` specifier within the pattern layout configuration:: - + %date{ISO8601} %-5level %logger{36} %X{sourceThread} - %msg%n - + .. note:: @@ -235,9 +235,9 @@ is available for associating log messages e.g. with members of a router. This information is available in the MDC with attribute name ``akkaSource``:: - + %date{ISO8601} %-5level %logger{36} %X{akkaSource} - %msg%n - + For more details on what this attribute contains—also for non-actors—please see diff --git a/akka-docs/scala/logging.rst b/akka-docs/scala/logging.rst index dc87126c5b..19630cda18 100644 --- a/akka-docs/scala/logging.rst +++ b/akka-docs/scala/logging.rst @@ -251,9 +251,9 @@ Mapped Diagnostic Context (MDC) with attribute name ``sourceThread``. With Logback the thread name is available with ``%X{sourceThread}`` specifier within the pattern layout configuration:: - + %date{ISO8601} %-5level %logger{36} %X{sourceThread} - %msg%n - + .. note:: @@ -268,9 +268,9 @@ is available for associating log messages e.g. with members of a router. This information is available in the MDC with attribute name ``akkaSource``:: - + %date{ISO8601} %-5level %logger{36} %X{akkaSource} - %msg%n - + For more details on what this attribute contains—also for non-actors—please see diff --git a/akka-sbt-plugin/sample/src/main/config/logback.xml b/akka-sbt-plugin/sample/src/main/config/logback.xml index 019d298192..20d035b1d2 100644 --- a/akka-sbt-plugin/sample/src/main/config/logback.xml +++ b/akka-sbt-plugin/sample/src/main/config/logback.xml @@ -3,9 +3,9 @@ - + %date{ISO8601} %-5level %X{akkaSource} %X{sourceThread} - %msg%n - + diff --git a/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala b/akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jEventHandler.scala similarity index 100% rename from akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala rename to akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jEventHandler.scala diff --git a/akka-slf4j/src/test/resources/logback-test.xml b/akka-slf4j/src/test/resources/logback-test.xml new file mode 100644 index 0000000000..96674387c5 --- /dev/null +++ b/akka-slf4j/src/test/resources/logback-test.xml @@ -0,0 +1,26 @@ + + + + + + + %date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n + + + + + + %date{ISO8601} level=[%level] logger=[%logger] akkaSource=[%X{akkaSource}] sourceThread=[%X{sourceThread}] - msg=[%msg]%n----%n + + + + + + + + + + + + diff --git a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala new file mode 100644 index 0000000000..17af919fcc --- /dev/null +++ b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala @@ -0,0 +1,127 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.event.slf4j + +import akka.testkit.AkkaSpec +import akka.actor.Actor +import akka.actor.ActorLogging +import akka.util.duration._ +import akka.event.Logging +import akka.actor.Props +import ch.qos.logback.core.OutputStreamAppender +import java.io.StringWriter +import java.io.ByteArrayOutputStream +import org.scalatest.BeforeAndAfterEach + +object Slf4jEventHandlerSpec { + + // This test depends on logback configuration in src/test/resources/logback-test.xml + + val config = """ + akka { + loglevel = INFO + event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] + } + """ + + class LogProducer extends Actor with ActorLogging { + def receive = { + case e: Exception ⇒ + log.error(e, e.getMessage) + case (s: String, x: Int, y: Int) ⇒ + log.info(s, x, y) + } + } + + class MyLogSource + + val output = new ByteArrayOutputStream + def outputString: String = output.toString("UTF-8") + + class TestAppender extends OutputStreamAppender { + + override def start(): Unit = { + setOutputStream(output) + super.start() + } + } + +} + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with BeforeAndAfterEach { + import Slf4jEventHandlerSpec._ + + val producer = system.actorOf(Props[LogProducer], name = "logProducer") + + override def beforeEach(): Unit = { + output.reset() + } + + "Slf4jEventHandler" must { + + "log error with stackTrace" in { + producer ! new RuntimeException("Simulated error") + + awaitCond(outputString.contains("----"), 5 seconds) + val s = outputString + s must include("akkaSource=[akka://Slf4jEventHandlerSpec/user/logProducer]") + s must include("level=[ERROR]") + s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$LogProducer]") + s must include regex ("sourceThread=\\[ForkJoinPool-[1-9][0-9]*-worker-[1-9][0-9]*\\]") + s must include("msg=[Simulated error]") + s must include("java.lang.RuntimeException: Simulated error") + s must include("at akka.event.slf4j.Slf4jEventHandlerSpec") + } + + "log info with parameters" in { + producer ! ("test x={} y={}", 3, 17) + + awaitCond(outputString.contains("----"), 5 seconds) + val s = outputString + s must include("akkaSource=[akka://Slf4jEventHandlerSpec/user/logProducer]") + s must include("level=[INFO]") + s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$LogProducer]") + s must include regex ("sourceThread=\\[ForkJoinPool-[1-9][0-9]*-worker-[1-9][0-9]*\\]") + s must include("msg=[test x=3 y=17]") + } + + "include system info in akkaSource when creating Logging with system" in { + val log = Logging(system, "akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource") + log.info("test") + awaitCond(outputString.contains("----"), 5 seconds) + val s = outputString + s must include("akkaSource=[akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource(akka://Slf4jEventHandlerSpec)]") + s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource(akka://Slf4jEventHandlerSpec)]") + } + + "not include system info in akkaSource when creating Logging with system.eventStream" in { + val log = Logging(system.eventStream, "akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource") + log.info("test") + awaitCond(outputString.contains("----"), 5 seconds) + val s = outputString + s must include("akkaSource=[akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource]") + s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource]") + } + + "use short class name and include system info in akkaSource when creating Logging with system and class" in { + val log = Logging(system, classOf[MyLogSource]) + log.info("test") + awaitCond(outputString.contains("----"), 5 seconds) + val s = outputString + s must include("akkaSource=[Slf4jEventHandlerSpec$MyLogSource(akka://Slf4jEventHandlerSpec)]") + s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$MyLogSource]") + } + + "use short class name in akkaSource when creating Logging with system.eventStream and class" in { + val log = Logging(system.eventStream, classOf[MyLogSource]) + log.info("test") + awaitCond(outputString.contains("----"), 5 seconds) + val s = outputString + s must include("akkaSource=[Slf4jEventHandlerSpec$MyLogSource]") + s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$MyLogSource]") + } + } + +} diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index f810da86a6..306aa38e8f 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -431,7 +431,7 @@ object Dependencies { val cluster = Seq(Test.junit, Test.scalatest) - val slf4j = Seq(slf4jApi) + val slf4j = Seq(slf4jApi, Test.logback) val agent = Seq(scalaStm, Test.scalatest, Test.junit) From 761a61eb6bffba61c9dc828e58627a5fabf40871 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 8 Feb 2012 09:36:17 +0100 Subject: [PATCH 21/34] remove sbteclipse plugin from project/plugins.sbt MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit this should be in each user’s .sbt directory, so that everybody is free to use which version they like --- project/plugins.sbt | 2 -- 1 file changed, 2 deletions(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 52704b5935..80ff9db95a 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -5,8 +5,6 @@ addSbtPlugin("com.typesafe.sbtmultijvm" % "sbt-multi-jvm" % "0.1.9") addSbtPlugin("com.typesafe.schoir" % "schoir" % "0.1.2") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse" % "1.5.0") - addSbtPlugin("com.typesafe.sbtscalariform" % "sbtscalariform" % "0.3.1") resolvers ++= Seq( From b25d25e6f730e725f03c534a787f6b5fef997ba1 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 8 Feb 2012 10:04:47 +0100 Subject: [PATCH 22/34] DOC: Error Kernel example in Actor docs doesn't do what is intended. See #1803 --- akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index 8747060554..1c739bc6ac 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -48,9 +48,7 @@ class FirstActor extends Actor { context.stop(self) } def doSomeDangerousWork(msg: ImmutableMessage): String = { "done" } - })) ! m - - case replyMsg: String ⇒ sender ! replyMsg + })) forward m } //#anonymous-actor } From 7c6b9db17ba0d907bfc8a90887fe88fcaaaa9c51 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 8 Feb 2012 10:42:17 +0100 Subject: [PATCH 23/34] mark two doc tests as TimingTests --- akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index 1c739bc6ac..d2fd56dac3 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -233,7 +233,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { system.stop(myActor) } - "using implicit timeout" in { + "using implicit timeout" taggedAs TimingTest in { val myActor = system.actorOf(Props(new FirstActor)) //#using-implicit-timeout import akka.util.duration._ @@ -246,7 +246,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { } - "using explicit timeout" in { + "using explicit timeout" taggedAs TimingTest in { val myActor = system.actorOf(Props(new FirstActor)) //#using-explicit-timeout import akka.util.duration._ From 771da2c7e2868568aab9921bb7036e8828270ec2 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 8 Feb 2012 11:16:07 +0100 Subject: [PATCH 24/34] clarify wording or Pi sample configuration section also move Pi.scala into the correct directory so that it can actually be launched as Scala Application from Eclipse --- akka-docs/intro/getting-started-first-java.rst | 18 +++++++++++++----- .../getting-started-first-scala-eclipse.rst | 18 +++++++++++++----- .../intro/getting-started-first-scala.rst | 18 +++++++++++++----- .../{ => akka/tutorial/first/scala}/Pi.scala | 0 4 files changed, 39 insertions(+), 15 deletions(-) rename akka-tutorials/akka-tutorial-first/src/main/scala/{ => akka/tutorial/first/scala}/Pi.scala (100%) diff --git a/akka-docs/intro/getting-started-first-java.rst b/akka-docs/intro/getting-started-first-java.rst index 5f45a51528..0a048a050f 100644 --- a/akka-docs/intro/getting-started-first-java.rst +++ b/akka-docs/intro/getting-started-first-java.rst @@ -397,17 +397,25 @@ When this in done we can run our application directly inside Maven:: Yippee! It is working. -Overriding Configuration Externally ------------------------------------ +Overriding Configuration Externally (Optional) +---------------------------------------------- The sample project includes an ``application.conf`` file in the resources directory: .. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/resources/application.conf If you uncomment the two lines, you should see a change in performance, -hopefully for the better. It should be noted that overriding only works if a -router type is given, so just uncommenting ``nr-of-instances`` does not work; -see :ref:`routing-java` for more details. +hopefully for the better (you might want to increase the number of messages in +the code to prolong the time the application runs). It should be noted that +overriding only works if a router type is given, so just uncommenting +``nr-of-instances`` does not work; see :ref:`routing-java` for more details. + +.. note:: + + Make sure that your ``application.conf`` is on the class path when you run + the application. If running from inside Maven that should already be the + case, otherwise you need to add the directory containing this file to the + JVM’s ``-classpath`` option. Conclusion ---------- diff --git a/akka-docs/intro/getting-started-first-scala-eclipse.rst b/akka-docs/intro/getting-started-first-scala-eclipse.rst index 9d45270ebd..4e8fdfdae9 100644 --- a/akka-docs/intro/getting-started-first-scala-eclipse.rst +++ b/akka-docs/intro/getting-started-first-scala-eclipse.rst @@ -421,17 +421,25 @@ arguments to the JVM on the ``Arguments`` page, for instance to define where :re Once you finished your run configuration, click ``Run``. You should see the same output in the ``Console`` window. You can use the same configuration for debugging the application, by choosing ``Run/Debug History`` or just ``Debug As``. -Overriding Configuration Externally ------------------------------------ +Overriding Configuration Externally (Optional) +---------------------------------------------- The sample project includes an ``application.conf`` file in the resources directory: .. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/resources/application.conf If you uncomment the two lines, you should see a change in performance, -hopefully for the better. It should be noted that overriding only works if a -router type is given, so just uncommenting ``nr-of-instances`` does not work; -see :ref:`routing-java` for more details. +hopefully for the better (you might want to increase the number of messages in +the code to prolong the time the application runs). It should be noted that +overriding only works if a router type is given, so just uncommenting +``nr-of-instances`` does not work; see :ref:`routing-scala` for more details. + +.. note:: + + Make sure that your ``application.conf`` is on the class path when you run + the application. If running from inside SBT that should already be the case, + otherwise you need to add the directory containing this file to the JVM’s + ``-classpath`` option. Conclusion ---------- diff --git a/akka-docs/intro/getting-started-first-scala.rst b/akka-docs/intro/getting-started-first-scala.rst index 10386ca334..88a2b2c130 100644 --- a/akka-docs/intro/getting-started-first-scala.rst +++ b/akka-docs/intro/getting-started-first-scala.rst @@ -442,17 +442,25 @@ When this in done we can run our application directly inside SBT:: Yippee! It is working. -Overriding Configuration Externally -=================================== +Overriding Configuration Externally (Optional) +============================================== The sample project includes an ``application.conf`` file in the resources directory: .. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/resources/application.conf If you uncomment the two lines, you should see a change in performance, -hopefully for the better. It should be noted that overriding only works if a -router type is given, so just uncommenting ``nr-of-instances`` does not work; -see :ref:`routing-scala` for more details. +hopefully for the better (you might want to increase the number of messages in +the code to prolong the time the application runs). It should be noted that +overriding only works if a router type is given, so just uncommenting +``nr-of-instances`` does not work; see :ref:`routing-scala` for more details. + +.. note:: + + Make sure that your ``application.conf`` is on the class path when you run + the application. If running from inside SBT that should already be the case, + otherwise you need to add the directory containing this file to the JVM’s + ``-classpath`` option. Conclusion ========== diff --git a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala similarity index 100% rename from akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala rename to akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala From 1f988889c21984efad2ad396523f05c2fcc4e924 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 8 Feb 2012 11:53:55 +0100 Subject: [PATCH 25/34] Proper thread names for fork-join-executor. See #1805 --- .../akka/actor/dispatch/DispatchersSpec.scala | 41 ++++++++++++++++++- .../akka/dispatch/AbstractDispatcher.scala | 20 ++++++--- .../main/scala/akka/dispatch/Dispatcher.scala | 7 +--- .../akka/dispatch/ThreadPoolBuilder.scala | 13 ++++-- .../event/slf4j/Slf4jEventHandlerSpec.scala | 6 ++- 5 files changed, 68 insertions(+), 19 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala index 5d6dd65529..0556a1762e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala @@ -7,8 +7,12 @@ import java.util.concurrent.{ CountDownLatch, TimeUnit } import scala.reflect.{ Manifest } import akka.dispatch._ import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender import scala.collection.JavaConverters._ import com.typesafe.config.ConfigFactory +import akka.actor.Actor +import akka.actor.Props +import akka.util.duration._ object DispatchersSpec { val config = """ @@ -16,13 +20,22 @@ object DispatchersSpec { mydispatcher { throughput = 17 } + thread-pool-dispatcher { + executor = thread-pool-executor + } } """ + + class ThreadNameEcho extends Actor { + def receive = { + case _ ⇒ sender ! Thread.currentThread.getName + } + } } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) { - +class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) with ImplicitSender { + import DispatchersSpec._ val df = system.dispatchers import df._ @@ -92,6 +105,30 @@ class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) { d1 must be === d2 } + "include system name and dispatcher id in thread names for fork-join-executor" in { + system.actorOf(Props[ThreadNameEcho].withDispatcher("myapp.mydispatcher")) ! "what's the name?" + val Expected = "(DispatchersSpec-myapp.mydispatcher-[1-9][0-9]*-worker-[1-9][0-9]*)".r + expectMsgPF(5 seconds) { + case Expected(x) ⇒ + } + } + + "include system name and dispatcher id in thread names for thread-pool-executor" in { + system.actorOf(Props[ThreadNameEcho].withDispatcher("myapp.thread-pool-dispatcher")) ! "what's the name?" + val Expected = "(DispatchersSpec-myapp.thread-pool-dispatcher-[1-9][0-9]*)".r + expectMsgPF(5 seconds) { + case Expected(x) ⇒ + } + } + + "include system name and dispatcher id in thread names for default-dispatcher" in { + system.actorOf(Props[ThreadNameEcho]) ! "what's the name?" + val Expected = "(DispatchersSpec-akka.actor.default-dispatcher-[1-9][0-9]*-worker-[1-9][0-9]*)".r + expectMsgPF(5 seconds) { + case Expected(x) ⇒ + } + } + } } diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index e32631108c..cdafb37dc0 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -420,8 +420,13 @@ class ThreadPoolExecutorConfigurator(config: Config, prerequisites: DispatcherPr })(queueFactory ⇒ _.setQueueFactory(queueFactory))) } - def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory = - threadPoolConfig.createExecutorServiceFactory(name, threadFactory) + def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory = { + val tf = threadFactory match { + case m: MonitorableThreadFactory ⇒ m.copy(m.name + "-" + id) + case other ⇒ other + } + threadPoolConfig.createExecutorServiceFactory(id, tf) + } } object ForkJoinExecutorConfigurator { @@ -460,7 +465,7 @@ object ForkJoinExecutorConfigurator { class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceConfigurator(config, prerequisites) { import ForkJoinExecutorConfigurator._ - def validate(t: ThreadFactory): ForkJoinPool.ForkJoinWorkerThreadFactory = prerequisites.threadFactory match { + def validate(t: ThreadFactory): ForkJoinPool.ForkJoinWorkerThreadFactory = t match { case correct: ForkJoinPool.ForkJoinWorkerThreadFactory ⇒ correct case x ⇒ throw new IllegalStateException("The prerequisites for the ForkJoinExecutorConfigurator is a ForkJoinPool.ForkJoinWorkerThreadFactory!") } @@ -469,11 +474,16 @@ class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrer val parallelism: Int) extends ExecutorServiceFactory { def createExecutorService: ExecutorService = new AkkaForkJoinPool(parallelism, threadFactory, MonitorableThreadFactory.doNothing) } - final def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory = + final def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory = { + val tf = threadFactory match { + case m: MonitorableThreadFactory ⇒ m.copy(m.name + "-" + id) + case other ⇒ other + } new ForkJoinExecutorServiceFactory( - validate(threadFactory), + validate(tf), ThreadPoolConfig.scaledPoolSize( config.getInt("parallelism-min"), config.getDouble("parallelism-factor"), config.getInt("parallelism-max"))) + } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 906c160dce..a735ea367e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -33,12 +33,7 @@ class Dispatcher( extends MessageDispatcher(_prerequisites) { protected[akka] val executorServiceFactory: ExecutorServiceFactory = - executorServiceFactoryProvider.createExecutorServiceFactory( - id, - prerequisites.threadFactory match { - case m: MonitorableThreadFactory ⇒ m.copy(m.name + "-" + id) - case other ⇒ other - }) + executorServiceFactoryProvider.createExecutorServiceFactory(id, prerequisites.threadFactory) protected[akka] val executorService = new AtomicReference[ExecutorService](new ExecutorServiceDelegate { lazy val executor = executorServiceFactory.createExecutorService diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index 5be5f1b0e1..4b95ab2931 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -66,7 +66,7 @@ trait ExecutorServiceFactory { * Generic way to specify an ExecutorService to a Dispatcher, create it with the given name if desired */ trait ExecutorServiceFactoryProvider { - def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory + def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory } /** @@ -93,7 +93,7 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def service } } - final def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory = + final def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory = new ThreadPoolExecutorServiceFactory(threadFactory) } @@ -170,9 +170,14 @@ case class MonitorableThreadFactory(name: String, extends ThreadFactory with ForkJoinPool.ForkJoinWorkerThreadFactory { protected val counter = new AtomicLong - def newThread(pool: ForkJoinPool): ForkJoinWorkerThread = wire(ForkJoinPool.defaultForkJoinWorkerThreadFactory.newThread(pool)) + def newThread(pool: ForkJoinPool): ForkJoinWorkerThread = { + val t = wire(ForkJoinPool.defaultForkJoinWorkerThreadFactory.newThread(pool)) + // Name of the threads for the ForkJoinPool are not customizable. Change it here. + if (t.getName.startsWith("ForkJoinPool-")) t.setName(name + "-" + t.getName.substring("ForkJoinPool-".length)) + t + } - def newThread(runnable: Runnable): Thread = wire(new Thread(runnable, name + counter.incrementAndGet())) + def newThread(runnable: Runnable): Thread = wire(new Thread(runnable, name + "-" + counter.incrementAndGet())) protected def wire[T <: Thread](t: T): T = { t.setUncaughtExceptionHandler(exceptionHandler) diff --git a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala index 17af919fcc..f66a06c6ab 100644 --- a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala +++ b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala @@ -59,6 +59,8 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with output.reset() } + val sourceThreadRegex = "sourceThread=\\[Slf4jEventHandlerSpec-akka.actor.default-dispatcher-[1-9][0-9]*-worker-[1-9][0-9]*\\]" + "Slf4jEventHandler" must { "log error with stackTrace" in { @@ -69,7 +71,7 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with s must include("akkaSource=[akka://Slf4jEventHandlerSpec/user/logProducer]") s must include("level=[ERROR]") s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$LogProducer]") - s must include regex ("sourceThread=\\[ForkJoinPool-[1-9][0-9]*-worker-[1-9][0-9]*\\]") + s must include regex (sourceThreadRegex) s must include("msg=[Simulated error]") s must include("java.lang.RuntimeException: Simulated error") s must include("at akka.event.slf4j.Slf4jEventHandlerSpec") @@ -83,7 +85,7 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with s must include("akkaSource=[akka://Slf4jEventHandlerSpec/user/logProducer]") s must include("level=[INFO]") s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$LogProducer]") - s must include regex ("sourceThread=\\[ForkJoinPool-[1-9][0-9]*-worker-[1-9][0-9]*\\]") + s must include regex (sourceThreadRegex) s must include("msg=[test x=3 y=17]") } From 9a7b9d7ab9ff0c1254257614d00169da9ae9f3c0 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 8 Feb 2012 13:16:59 +0100 Subject: [PATCH 26/34] =?UTF-8?q?remove=20unnecessary=20=E2=80=9Creturn?= =?UTF-8?q?=E2=80=9D=20in=20fault=20handling=20and=20make=20code=20look=20?= =?UTF-8?q?nicer?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- akka-actor/src/main/scala/akka/actor/FaultHandling.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 8b6abc5f5d..28694dd3d3 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -159,7 +159,10 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { */ def makeDecider(flat: Iterable[CauseDirective]): Decider = { val directives = sort(flat) - return { case x ⇒ directives find (_._1 isInstance x) map (_._2) getOrElse Escalate } + + { + case x ⇒ directives find (_._1 isInstance x) map (_._2) getOrElse Escalate + } } def makeDecider(func: JDecider): Decider = { From c43b024cacdc6889e338cd907b2536e7f4c2a859 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 8 Feb 2012 13:37:00 +0100 Subject: [PATCH 27/34] Fixed the failing doc test --- .../code/akka/docs/actor/ActorDocSpec.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index d2fd56dac3..aee30e3c56 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -37,6 +37,12 @@ case class Message(s: String) class FirstActor extends Actor { val myActor = context.actorOf(Props[MyActor], name = "myactor") //#context-actorOf + def receive = { + case x ⇒ sender ! x + } +} + +class AnonymousActor extends Actor { //#anonymous-actor def receive = { case m: DoIt ⇒ @@ -233,27 +239,27 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { system.stop(myActor) } - "using implicit timeout" taggedAs TimingTest in { + "using implicit timeout" in { val myActor = system.actorOf(Props(new FirstActor)) //#using-implicit-timeout import akka.util.duration._ import akka.util.Timeout import akka.pattern.ask - implicit val timeout = Timeout(500 millis) + implicit val timeout = Timeout(5 seconds) val future = myActor ? "hello" //#using-implicit-timeout Await.result(future, timeout.duration) must be("hello") } - "using explicit timeout" taggedAs TimingTest in { + "using explicit timeout" in { val myActor = system.actorOf(Props(new FirstActor)) //#using-explicit-timeout import akka.util.duration._ import akka.pattern.ask - val future = myActor.ask("hello")(500 millis) + val future = myActor.ask("hello")(5 seconds) //#using-explicit-timeout - Await.result(future, 500 millis) must be("hello") + Await.result(future, 5 seconds) must be("hello") } "using receiveTimeout" in { From 81451208f115dd696a17a779bf4a62cd20fdf8c9 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 8 Feb 2012 13:37:19 +0100 Subject: [PATCH 28/34] Pi is not a constant, it has changed --- .../getting-started-first-scala-eclipse.rst | 18 +++++++------- .../intro/getting-started-first-scala.rst | 24 +++++++++---------- 2 files changed, 21 insertions(+), 21 deletions(-) diff --git a/akka-docs/intro/getting-started-first-scala-eclipse.rst b/akka-docs/intro/getting-started-first-scala-eclipse.rst index 4e8fdfdae9..9e02f99a71 100644 --- a/akka-docs/intro/getting-started-first-scala-eclipse.rst +++ b/akka-docs/intro/getting-started-first-scala-eclipse.rst @@ -284,7 +284,7 @@ In Scala we have 'case classes' which make excellent messages. So let's start by We also create a common base trait for our messages (that we define as being ``sealed`` in order to prevent creating messages outside our control): -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#messages +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#messages Creating the worker @@ -295,7 +295,7 @@ trait and defining the ``receive`` method. The ``receive`` method defines our message handler. We expect it to be able to handle the ``Work`` message so we need to add a handler for this message: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#worker +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#worker :exclude: calculatePiFor The ``Actor`` trait is defined in ``akka.actor`` and you can either import it explicitly, @@ -318,7 +318,7 @@ The only thing missing in our ``Worker`` actor is the implementation on the algorithm in Scala, in this introductory tutorial we have chosen an imperative style using a for comprehension and an accumulator: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#calculatePiFor +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#calculatePiFor Creating the master @@ -328,11 +328,11 @@ Now create a new class for the master actor. The master actor is a little bit more involved. In its constructor we create a round-robin router to make it easier to spread out the work evenly between the workers. First we need to add some imports: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#imports +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#imports and then we can create the router: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#create-router +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#create-router Now we have a router that is representing all our workers in a single abstraction. So now let's create the master actor. We pass it three integer variables: @@ -343,7 +343,7 @@ abstraction. So now let's create the master actor. We pass it three integer vari Here is the master actor: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#master :exclude: handle-messages A couple of things are worth explaining further. @@ -370,7 +370,7 @@ will propagate down to all its supervised 'children'. Let's capture this in code: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master-receive +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#master-receive Creating the result listener ---------------------------- @@ -378,7 +378,7 @@ Creating the result listener The listener is straightforward. When it receives the ``PiApproximation`` from the ``Master`` it prints the result and shuts down the ``ActorSystem``. -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#result-listener +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#result-listener Bootstrap the calculation ------------------------- @@ -390,7 +390,7 @@ which means that we will be able to run this as an application directly from the The ``Pi`` object is a perfect container module for our actors and messages, so let's put them all there. We also create a method ``calculate`` in which we start up the ``Master`` actor and wait for it to finish: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#app +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#app :exclude: actors-and-messages As you can see the *calculate* method above it creates an ``ActorSystem`` and this is the Akka container which diff --git a/akka-docs/intro/getting-started-first-scala.rst b/akka-docs/intro/getting-started-first-scala.rst index 88a2b2c130..0d66c11eea 100644 --- a/akka-docs/intro/getting-started-first-scala.rst +++ b/akka-docs/intro/getting-started-first-scala.rst @@ -49,7 +49,7 @@ check out the full tutorial from the Akka GitHub repository. It is in the `here`__, with the actual source code `here`__. __ https://github.com/jboner/akka/tree/master/akka-tutorials/akka-tutorial-first -__ https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala +__ https://github.com/jboner/akka/blob/master/akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala To check out the code using Git invoke the following:: @@ -244,7 +244,7 @@ Now it's about time to start hacking. We start by creating a ``Pi.scala`` file and adding these import statements at the top of the file: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#imports +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#imports If you are using SBT in this tutorial then create the file in the ``src/main/scala`` directory. @@ -282,7 +282,7 @@ start by creating three messages as case classes. We also create a common base trait for our messages (that we define as being ``sealed`` in order to prevent creating messages outside our control): -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#messages +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#messages Creating the worker @@ -293,7 +293,7 @@ trait and defining the ``receive`` method. The ``receive`` method defines our message handler. We expect it to be able to handle the ``Work`` message so we need to add a handler for this message: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#worker +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#worker :exclude: calculatePiFor As you can see we have now created an ``Actor`` with a ``receive`` method as a @@ -308,7 +308,7 @@ The only thing missing in our ``Worker`` actor is the implementation on the algorithm in Scala, in this introductory tutorial we have chosen an imperative style using a for comprehension and an accumulator: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#calculatePiFor +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#calculatePiFor Creating the master @@ -317,7 +317,7 @@ Creating the master The master actor is a little bit more involved. In its constructor we create a round-robin router to make it easier to spread out the work evenly between the workers. Let's do that first: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#create-router +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#create-router Now we have a router that is representing all our workers in a single abstraction. So now let's create the master actor. We pass it three integer variables: @@ -328,7 +328,7 @@ abstraction. So now let's create the master actor. We pass it three integer vari Here is the master actor: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#master :exclude: handle-messages A couple of things are worth explaining further. @@ -355,7 +355,7 @@ will propagate down to all its supervised 'children'. Let's capture this in code: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master-receive +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#master-receive Creating the result listener @@ -364,7 +364,7 @@ Creating the result listener The listener is straightforward. When it receives the ``PiApproximation`` from the ``Master`` it prints the result and shuts down the ``ActorSystem``. -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#result-listener +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#result-listener Bootstrap the calculation ========================= @@ -378,7 +378,7 @@ The ``Pi`` object is a perfect container module for our actors and messages, so let's put them all there. We also create a method ``calculate`` in which we start up the ``Master`` actor and wait for it to finish: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#app +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala#app :exclude: actors-and-messages As you can see the *calculate* method above it creates an ``ActorSystem`` and this is the Akka container which @@ -392,14 +392,14 @@ That's it. Now we are done. But before we package it up and run it, let's take a look at the full code now, with package declaration, imports and all: -.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala +.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala Run it as a command line application ==================================== If you have not typed in (or copied) the code for the tutorial as in -``$AKKA_HOME/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala`` then now is the time. +``$AKKA_HOME/akka-tutorials/akka-tutorial-first/src/main/scala/akka/tutorial/first/scala/Pi.scala`` then now is the time. When that's done open up a shell and step in to the Akka distribution (``cd $AKKA_HOME``). First we need to compile the source file. That is done with Scala's compiler From df1606a8eec75af73917072f27d7bf6a49bd4d7c Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 8 Feb 2012 14:03:31 +0100 Subject: [PATCH 29/34] Adjustments based on review. See #1805 --- .../scala/akka/actor/dispatch/DispatchersSpec.scala | 4 ++-- .../scala/akka/dispatch/AbstractDispatcher.scala | 12 ++++++++---- .../main/scala/akka/dispatch/ThreadPoolBuilder.scala | 2 +- .../akka/event/slf4j/Slf4jEventHandlerSpec.scala | 2 +- 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala index 0556a1762e..03c5f21711 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala @@ -107,7 +107,7 @@ class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) with ImplicitSend "include system name and dispatcher id in thread names for fork-join-executor" in { system.actorOf(Props[ThreadNameEcho].withDispatcher("myapp.mydispatcher")) ! "what's the name?" - val Expected = "(DispatchersSpec-myapp.mydispatcher-[1-9][0-9]*-worker-[1-9][0-9]*)".r + val Expected = "(DispatchersSpec-myapp.mydispatcher-[1-9][0-9]*)".r expectMsgPF(5 seconds) { case Expected(x) ⇒ } @@ -123,7 +123,7 @@ class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) with ImplicitSend "include system name and dispatcher id in thread names for default-dispatcher" in { system.actorOf(Props[ThreadNameEcho]) ! "what's the name?" - val Expected = "(DispatchersSpec-akka.actor.default-dispatcher-[1-9][0-9]*-worker-[1-9][0-9]*)".r + val Expected = "(DispatchersSpec-akka.actor.default-dispatcher-[1-9][0-9]*)".r expectMsgPF(5 seconds) { case Expected(x) ⇒ } diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index cdafb37dc0..8ac5aba564 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -422,8 +422,10 @@ class ThreadPoolExecutorConfigurator(config: Config, prerequisites: DispatcherPr def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory = { val tf = threadFactory match { - case m: MonitorableThreadFactory ⇒ m.copy(m.name + "-" + id) - case other ⇒ other + case m: MonitorableThreadFactory ⇒ + // add the dispatcher id to the thread names + m.copy(m.name + "-" + id) + case other ⇒ other } threadPoolConfig.createExecutorServiceFactory(id, tf) } @@ -476,8 +478,10 @@ class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrer } final def createExecutorServiceFactory(id: String, threadFactory: ThreadFactory): ExecutorServiceFactory = { val tf = threadFactory match { - case m: MonitorableThreadFactory ⇒ m.copy(m.name + "-" + id) - case other ⇒ other + case m: MonitorableThreadFactory ⇒ + // add the dispatcher id to the thread names + m.copy(m.name + "-" + id) + case other ⇒ other } new ForkJoinExecutorServiceFactory( validate(tf), diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index 4b95ab2931..1c63831013 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -173,7 +173,7 @@ case class MonitorableThreadFactory(name: String, def newThread(pool: ForkJoinPool): ForkJoinWorkerThread = { val t = wire(ForkJoinPool.defaultForkJoinWorkerThreadFactory.newThread(pool)) // Name of the threads for the ForkJoinPool are not customizable. Change it here. - if (t.getName.startsWith("ForkJoinPool-")) t.setName(name + "-" + t.getName.substring("ForkJoinPool-".length)) + t.setName(name + "-" + counter.incrementAndGet()) t } diff --git a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala index f66a06c6ab..87d4869fef 100644 --- a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala +++ b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jEventHandlerSpec.scala @@ -59,7 +59,7 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with output.reset() } - val sourceThreadRegex = "sourceThread=\\[Slf4jEventHandlerSpec-akka.actor.default-dispatcher-[1-9][0-9]*-worker-[1-9][0-9]*\\]" + val sourceThreadRegex = "sourceThread=\\[Slf4jEventHandlerSpec-akka.actor.default-dispatcher-[1-9][0-9]*\\]" "Slf4jEventHandler" must { From 34d9714e22a355558522d074ac7655f659e84795 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 8 Feb 2012 15:54:35 +0100 Subject: [PATCH 30/34] add clarification to dispatcher-id documentation --- akka-docs/java/dispatchers.rst | 7 +++++++ akka-docs/scala/dispatchers.rst | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/akka-docs/java/dispatchers.rst b/akka-docs/java/dispatchers.rst index fceb94abbc..2dcd5049c5 100644 --- a/akka-docs/java/dispatchers.rst +++ b/akka-docs/java/dispatchers.rst @@ -55,6 +55,13 @@ Default values are taken from ``default-dispatcher``, i.e. all options doesn't n :ref:`configuration` for the default values of the ``default-dispatcher``. You can also override the values for the ``default-dispatcher`` in your configuration. +.. note:: + + It should be noted that the ``dispatcher-id`` used in :class:`Props` is in + fact an absolute path into the configuration object, i.e. you can declare a + dispatcher configuration nested within other configuration objects and refer + to it like so: ``"my.config.object.myAwesomeDispatcher"`` + There are two different executor services: * executor = "fork-join-executor", ``ExecutorService`` based on ForkJoinPool (jsr166y). This is used by default for diff --git a/akka-docs/scala/dispatchers.rst b/akka-docs/scala/dispatchers.rst index c6e6ae23e3..50807fbc55 100644 --- a/akka-docs/scala/dispatchers.rst +++ b/akka-docs/scala/dispatchers.rst @@ -54,6 +54,13 @@ Default values are taken from ``default-dispatcher``, i.e. all options doesn't n :ref:`configuration` for the default values of the ``default-dispatcher``. You can also override the values for the ``default-dispatcher`` in your configuration. +.. note:: + + It should be noted that the ``dispatcher-id`` used in :class:`Props` is in + fact an absolute path into the configuration object, i.e. you can declare a + dispatcher configuration nested within other configuration objects and refer + to it like so: ``"my.config.object.myAwesomeDispatcher"`` + There are two different executor services: * executor = "fork-join-executor", ``ExecutorService`` based on ForkJoinPool (jsr166y). This is used by default for From 51a218b87f9ea319f056c153e3b05a6f968fb821 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bone=CC=81r?= Date: Thu, 9 Feb 2012 20:40:09 +0100 Subject: [PATCH 31/34] Fixes ticket #1809 - Harmonize configuration option names. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Jonas Bonér --- .../test/scala/akka/config/ConfigSpec.scala | 6 +- .../akka/dispatch/MailboxConfigSpec.scala | 2 +- .../dispatch/PriorityDispatcherSpec.scala | 4 +- akka-actor/src/main/resources/reference.conf | 56 +++++++++---------- .../main/scala/akka/actor/ActorSystem.scala | 8 +-- .../akka/dispatch/AbstractDispatcher.scala | 4 +- akka-agent/src/main/resources/reference.conf | 1 - .../src/main/resources/reference.conf | 2 +- .../scala/akka/cluster/ClusterSettings.scala | 2 +- akka-docs/general/configuration.rst | 4 +- akka-docs/java/logging.rst | 4 +- .../actor/mailbox/DurableMailboxDocSpec.scala | 2 +- akka-docs/modules/durable-mailbox.rst | 10 ++-- .../project/migration-guide-1.3.x-2.0.x.rst | 2 +- .../docs/dispatcher/DispatcherDocSpec.scala | 4 +- akka-docs/scala/logging.rst | 4 +- .../mailbox/BeanstalkBasedMailboxSpec.scala | 2 +- .../actor/mailbox/FileBasedMailboxSpec.scala | 2 +- .../actor/mailbox/MongoBasedMailboxSpec.scala | 2 +- .../actor/mailbox/RedisBasedMailboxSpec.scala | 2 +- .../mailbox/ZooKeeperBasedMailboxSpec.scala | 2 +- 21 files changed, 62 insertions(+), 63 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 9a0eab8830..dd5149ad8e 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -29,10 +29,10 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) { getBoolean("akka.actor.serialize-messages") must equal(false) settings.SerializeAllMessages must equal(false) - getInt("akka.scheduler.ticksPerWheel") must equal(512) + getInt("akka.scheduler.ticks-per-wheel") must equal(512) settings.SchedulerTicksPerWheel must equal(512) - getMilliseconds("akka.scheduler.tickDuration") must equal(100) + getMilliseconds("akka.scheduler.tick-duration") must equal(100) settings.SchedulerTickDuration must equal(100 millis) settings.Daemonicity must be(false) @@ -49,7 +49,7 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) { c.getString("executor") must equal("fork-join-executor") c.getInt("mailbox-capacity") must equal(-1) c.getMilliseconds("mailbox-push-timeout-time") must equal(10 * 1000) - c.getString("mailboxType") must be("") + c.getString("mailbox-type") must be("") c.getMilliseconds("shutdown-timeout") must equal(1 * 1000) c.getInt("throughput") must equal(5) c.getMilliseconds("throughput-deadline-time") must equal(0) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala index b797827680..4f787a730f 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala @@ -153,7 +153,7 @@ class PriorityMailboxSpec extends MailboxSpec { object CustomMailboxSpec { val config = """ my-dispatcher { - mailboxType = "akka.dispatch.CustomMailboxSpec$MyMailboxType" + mailbox-type = "akka.dispatch.CustomMailboxSpec$MyMailboxType" } """ diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala index 2bb4ab73e5..855c4f6965 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -10,10 +10,10 @@ import com.typesafe.config.Config object PriorityDispatcherSpec { val config = """ unbounded-prio-dispatcher { - mailboxType = "akka.dispatch.PriorityDispatcherSpec$Unbounded" + mailbox-type = "akka.dispatch.PriorityDispatcherSpec$Unbounded" } bounded-prio-dispatcher { - mailboxType = "akka.dispatch.PriorityDispatcherSpec$Bounded" + mailbox-type = "akka.dispatch.PriorityDispatcherSpec$Bounded" } """ diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index ce9bf684e8..bc52938a7d 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -26,7 +26,7 @@ akka { # Log the complete configuration at INFO level when the actor system is started. # This is useful when you are uncertain of what configuration is used. - logConfigOnStart = off + log-config-on-start = off # List FQCN of extensions which shall be loaded at actor system startup. # Should be on the format: 'extensions = ["foo", "bar"]' etc. @@ -37,7 +37,7 @@ akka { daemonic = off # JVM shutdown, System.exit(-1), in case of a fatal error, such as OutOfMemoryError - jvmExitOnFatalError = on + jvm-exit-on-fatal-error = on actor { @@ -185,37 +185,37 @@ akka { # This will be used if you have set "executor = "thread-pool-executor"" thread-pool-executor { - # Keep alive time for threads - keep-alive-time = 60s + # Keep alive time for threads + keep-alive-time = 60s - # Min number of threads to cap factor-based core number to - core-pool-size-min = 8 + # Min number of threads to cap factor-based core number to + core-pool-size-min = 8 - # No of core threads ... ceil(available processors * factor) - core-pool-size-factor = 3.0 + # No of core threads ... ceil(available processors * factor) + core-pool-size-factor = 3.0 - # Max number of threads to cap factor-based number to - core-pool-size-max = 64 + # Max number of threads to cap factor-based number to + core-pool-size-max = 64 - # Hint: max-pool-size is only used for bounded task queues - # minimum number of threads to cap factor-based max number to - max-pool-size-min = 8 + # Hint: max-pool-size is only used for bounded task queues + # minimum number of threads to cap factor-based max number to + max-pool-size-min = 8 - # Max no of threads ... ceil(available processors * factor) - max-pool-size-factor = 3.0 + # Max no of threads ... ceil(available processors * factor) + max-pool-size-factor = 3.0 - # Max number of threads to cap factor-based max number to - max-pool-size-max = 64 + # Max number of threads to cap factor-based max number to + max-pool-size-max = 64 - # Specifies the bounded capacity of the task queue (< 1 == unbounded) - task-queue-size = -1 + # Specifies the bounded capacity of the task queue (< 1 == unbounded) + task-queue-size = -1 - # Specifies which type of task queue will be used, can be "array" or - # "linked" (default) - task-queue-type = "linked" + # Specifies which type of task queue will be used, can be "array" or + # "linked" (default) + task-queue-type = "linked" - # Allow core threads to time out - allow-core-timeout = on + # Allow core threads to time out + allow-core-timeout = on } # How long time the dispatcher will wait for new actors until it shuts down @@ -245,7 +245,7 @@ akka { # FQCN of the MailboxType, if not specified the default bounded or unbounded # mailbox is used. The Class of the FQCN must have a constructor with a # com.typesafe.config.Config parameter. - mailboxType = "" + mailbox-type = "" } debug { @@ -272,7 +272,7 @@ akka { } # Class to Serializer binding. You only need to specify the name of an interface - # or abstract base class of the messages. In case of ambiguity it is using the + # or abstract base class of the messages. In case of ambiguity it is using the # most specific configured class, or giving a warning and choosing the “first” one. # # To disable one of the default serializers, assign its class to "none", like @@ -294,7 +294,7 @@ akka { # or larger tick duration. # If you are scheduling a lot of tasks you should consider increasing the ticks per wheel. # For more information see: http://www.jboss.org/netty/ - tickDuration = 100ms - ticksPerWheel = 512 + tick-duration = 100ms + ticks-per-wheel = 512 } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 72f2505994..98ac6897e1 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -77,7 +77,7 @@ object ActorSystem { final val LogLevel = getString("akka.loglevel") final val StdoutLogLevel = getString("akka.stdout-loglevel") final val EventHandlers: Seq[String] = getStringList("akka.event-handlers").asScala - final val LogConfigOnStart = config.getBoolean("akka.logConfigOnStart") + final val LogConfigOnStart = config.getBoolean("akka.log-config-on-start") final val AddLoggingReceive = getBoolean("akka.actor.debug.receive") final val DebugAutoReceive = getBoolean("akka.actor.debug.autoreceive") final val DebugLifecycle = getBoolean("akka.actor.debug.lifecycle") @@ -89,10 +89,10 @@ object ActorSystem { case x ⇒ Some(x) } - final val SchedulerTickDuration = Duration(getMilliseconds("akka.scheduler.tickDuration"), MILLISECONDS) - final val SchedulerTicksPerWheel = getInt("akka.scheduler.ticksPerWheel") + final val SchedulerTickDuration = Duration(getMilliseconds("akka.scheduler.tick-duration"), MILLISECONDS) + final val SchedulerTicksPerWheel = getInt("akka.scheduler.ticks-per-wheel") final val Daemonicity = getBoolean("akka.daemonic") - final val JvmExitOnFatalError = getBoolean("akka.jvmExitOnFatalError") + final val JvmExitOnFatalError = getBoolean("akka.jvm-exit-on-fatal-error") if (ConfigVersion != Version) throw new ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]") diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 8ac5aba564..75ffe376ee 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -354,12 +354,12 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit /** * Returns a factory for the [[akka.dispatch.Mailbox]] given the configuration. * Default implementation instantiate the [[akka.dispatch.MailboxType]] specified - * as FQCN in mailboxType config property. If mailboxType is unspecified (empty) + * as FQCN in mailbox-type config property. If mailbox-type is unspecified (empty) * then [[akka.dispatch.UnboundedMailbox]] is used when capacity is < 1, * otherwise [[akka.dispatch.BoundedMailbox]]. */ def mailboxType(): MailboxType = { - config.getString("mailboxType") match { + config.getString("mailbox-type") match { case "" ⇒ val capacity = config.getInt("mailbox-capacity") if (capacity < 1) UnboundedMailbox() diff --git a/akka-agent/src/main/resources/reference.conf b/akka-agent/src/main/resources/reference.conf index 7009c0f432..2c3b121ba7 100644 --- a/akka-agent/src/main/resources/reference.conf +++ b/akka-agent/src/main/resources/reference.conf @@ -19,6 +19,5 @@ akka { executor = thread-pool-executor type = PinnedDispatcher } - } } diff --git a/akka-cluster/src/main/resources/reference.conf b/akka-cluster/src/main/resources/reference.conf index 749c138a26..e097d34f3e 100644 --- a/akka-cluster/src/main/resources/reference.conf +++ b/akka-cluster/src/main/resources/reference.conf @@ -26,7 +26,7 @@ akka { } gossip { - initialDelay = 5s + initial-delay = 5s frequency = 1s } } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala index 820290ea14..e88c3ae72c 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala @@ -18,7 +18,7 @@ class ClusterSettings(val config: Config, val systemName: String) { val FailureDetectorMaxSampleSize = getInt("akka.cluster.failure-detector.max-sample-size") val SeedNodeConnectionTimeout = Duration(config.getMilliseconds("akka.cluster.seed-node-connection-timeout"), MILLISECONDS) val MaxTimeToRetryJoiningCluster = Duration(config.getMilliseconds("akka.cluster.max-time-to-retry-joining-cluster"), MILLISECONDS) - val InitialDelayForGossip = Duration(getMilliseconds("akka.cluster.gossip.initialDelay"), MILLISECONDS) + val InitialDelayForGossip = Duration(getMilliseconds("akka.cluster.gossip.initial-delay"), MILLISECONDS) val GossipFrequency = Duration(getMilliseconds("akka.cluster.gossip.frequency"), MILLISECONDS) val SeedNodes = Set.empty[Address] ++ getStringList("akka.cluster.seed-nodes").asScala.collect { case AddressExtractor(addr) ⇒ addr diff --git a/akka-docs/general/configuration.rst b/akka-docs/general/configuration.rst index 1a01ba24c9..3e2bca240d 100644 --- a/akka-docs/general/configuration.rst +++ b/akka-docs/general/configuration.rst @@ -172,12 +172,12 @@ More advanced include and substitution mechanisms are explained in the `HOCON .. note:: - + It will probably be a good idea to use the ``sourceThread`` MDC value also in non-Akka parts of the application in order to have this property consistently available in the logs. diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala index 827b4200d8..2f67c607ed 100644 --- a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala +++ b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala @@ -23,7 +23,7 @@ object DurableMailboxDocSpec { val config = """ //#dispatcher-config my-dispatcher { - mailboxType = akka.actor.mailbox.FileBasedMailboxType + mailbox-type = akka.actor.mailbox.FileBasedMailboxType } //#dispatcher-config """ diff --git a/akka-docs/modules/durable-mailbox.rst b/akka-docs/modules/durable-mailbox.rst index dc83522705..e8df16c97f 100644 --- a/akka-docs/modules/durable-mailbox.rst +++ b/akka-docs/modules/durable-mailbox.rst @@ -99,7 +99,7 @@ You configure durable mailboxes through the dispatcher, as described in Config:: my-dispatcher { - mailboxType = akka.actor.mailbox.FileBasedMailboxType + mailbox-type = akka.actor.mailbox.FileBasedMailboxType } You can also configure and tune the file-based durable mailbox. This is done in @@ -124,7 +124,7 @@ You configure durable mailboxes through the dispatcher, as described in Config:: my-dispatcher { - mailboxType = akka.actor.mailbox.RedisBasedMailboxType + mailbox-type = akka.actor.mailbox.RedisBasedMailboxType } You also need to configure the IP and port for the Redis server. This is done in @@ -150,7 +150,7 @@ You configure durable mailboxes through the dispatcher, as described in Config:: my-dispatcher { - mailboxType = akka.actor.mailbox.ZooKeeperBasedMailboxType + mailbox-type = akka.actor.mailbox.ZooKeeperBasedMailboxType } You also need to configure ZooKeeper server addresses, timeouts, etc. This is @@ -173,7 +173,7 @@ You configure durable mailboxes through the dispatcher, as described in Config:: my-dispatcher { - mailboxType = akka.actor.mailbox.BeanstalkBasedMailboxType + mailbox-type = akka.actor.mailbox.BeanstalkBasedMailboxType } You also need to configure the IP, and port, and so on, for the Beanstalk @@ -202,7 +202,7 @@ You configure durable mailboxes through the dispatcher, as described in Config:: my-dispatcher { - mailboxType = akka.actor.mailbox.MongoBasedMailboxType + mailbox-type = akka.actor.mailbox.MongoBasedMailboxType } You will need to configure the URI for the MongoDB server, using the URI Format specified in the diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 8a46d5f654..853381ca0d 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -326,7 +326,7 @@ v1.3:: v2.0:: -Dconfig.file= - -Dakka.logConfigOnStart=on + -Dakka.log-config-on-start=on Several configuration properties have been changed, such as: diff --git a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala index cd57fbeddc..4de32c3b57 100644 --- a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala @@ -94,13 +94,13 @@ object DispatcherDocSpec { //#prio-dispatcher-config prio-dispatcher { - mailboxType = "akka.docs.dispatcher.DispatcherDocSpec$PrioMailbox" + mailbox-type = "akka.docs.dispatcher.DispatcherDocSpec$PrioMailbox" } //#prio-dispatcher-config //#prio-dispatcher-config-java prio-dispatcher-java { - mailboxType = "akka.docs.dispatcher.DispatcherDocTestBase$PrioMailbox" + mailbox-type = "akka.docs.dispatcher.DispatcherDocTestBase$PrioMailbox" } //#prio-dispatcher-config-java """ diff --git a/akka-docs/scala/logging.rst b/akka-docs/scala/logging.rst index 19630cda18..d8cbb948c1 100644 --- a/akka-docs/scala/logging.rst +++ b/akka-docs/scala/logging.rst @@ -58,7 +58,7 @@ This config option is very good if you want to know what config settings are loa akka { # Log the complete configuration at INFO level when the actor system is started. # This is useful when you are uncertain of what configuration is used. - logConfigOnStart = on + log-config-on-start = on } If you want very detailed logging of all user-level messages that are processed @@ -257,7 +257,7 @@ With Logback the thread name is available with ``%X{sourceThread}`` specifier wi .. note:: - + It will probably be a good idea to use the ``sourceThread`` MDC value also in non-Akka parts of the application in order to have this property consistently available in the logs. diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/test/scala/akka/actor/mailbox/BeanstalkBasedMailboxSpec.scala b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/test/scala/akka/actor/mailbox/BeanstalkBasedMailboxSpec.scala index f7ed2e71ac..4eb370ab63 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/test/scala/akka/actor/mailbox/BeanstalkBasedMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/test/scala/akka/actor/mailbox/BeanstalkBasedMailboxSpec.scala @@ -3,7 +3,7 @@ package akka.actor.mailbox object BeanstalkBasedMailboxSpec { val config = """ Beanstalkd-dispatcher { - mailboxType = akka.actor.mailbox.BeanstalkBasedMailboxType + mailbox-type = akka.actor.mailbox.BeanstalkBasedMailboxType throughput = 1 } """ diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/test/scala/akka/actor/mailbox/FileBasedMailboxSpec.scala b/akka-durable-mailboxes/akka-file-mailbox/src/test/scala/akka/actor/mailbox/FileBasedMailboxSpec.scala index 3f202ddc5a..274bc36cc1 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/test/scala/akka/actor/mailbox/FileBasedMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/test/scala/akka/actor/mailbox/FileBasedMailboxSpec.scala @@ -5,7 +5,7 @@ import org.apache.commons.io.FileUtils object FileBasedMailboxSpec { val config = """ File-dispatcher { - mailboxType = akka.actor.mailbox.FileBasedMailboxType + mailbox-type = akka.actor.mailbox.FileBasedMailboxType throughput = 1 } """ diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/test/scala/akka/actor/mailbox/MongoBasedMailboxSpec.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/test/scala/akka/actor/mailbox/MongoBasedMailboxSpec.scala index 16fcde321e..7001d8de99 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/test/scala/akka/actor/mailbox/MongoBasedMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/test/scala/akka/actor/mailbox/MongoBasedMailboxSpec.scala @@ -12,7 +12,7 @@ import akka.dispatch.MessageDispatcher object MongoBasedMailboxSpec { val config = """ mongodb-dispatcher { - mailboxType = akka.actor.mailbox.MongoBasedMailboxType + mailbox-type = akka.actor.mailbox.MongoBasedMailboxType throughput = 1 } """ diff --git a/akka-durable-mailboxes/akka-redis-mailbox/src/test/scala/akka/actor/mailbox/RedisBasedMailboxSpec.scala b/akka-durable-mailboxes/akka-redis-mailbox/src/test/scala/akka/actor/mailbox/RedisBasedMailboxSpec.scala index 15bad81d2f..6e78d6b74a 100644 --- a/akka-durable-mailboxes/akka-redis-mailbox/src/test/scala/akka/actor/mailbox/RedisBasedMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-redis-mailbox/src/test/scala/akka/actor/mailbox/RedisBasedMailboxSpec.scala @@ -3,7 +3,7 @@ package akka.actor.mailbox object RedisBasedMailboxSpec { val config = """ Redis-dispatcher { - mailboxType = akka.actor.mailbox.RedisBasedMailboxType + mailbox-type = akka.actor.mailbox.RedisBasedMailboxType throughput = 1 } """ diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala index 4febbafe6f..9264fbccce 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala @@ -9,7 +9,7 @@ import akka.actor.ActorRef object ZooKeeperBasedMailboxSpec { val config = """ ZooKeeper-dispatcher { - mailboxType = akka.actor.mailbox.ZooKeeperBasedMailboxType + mailbox-type = akka.actor.mailbox.ZooKeeperBasedMailboxType throughput = 1 } """ From 4a5f5eef21d9374cb2c635ce54896128c8fd3192 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 9 Feb 2012 21:21:31 +0100 Subject: [PATCH 32/34] 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 d910eeae6945b9af35570ee61e22dd46ab589f6f Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 10 Feb 2012 08:20:36 +0100 Subject: [PATCH 33/34] 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 5b50ca96f2b3e4fc32f077e730e050f99cddfe79 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 10 Feb 2012 10:40:24 +0100 Subject: [PATCH 34/34] 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() }