From e88f2bd936496787ee4515d3c3b1ae105e0327c7 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 16 Feb 2012 12:31:49 +0100 Subject: [PATCH 01/46] First stab, do we want to go for throws Throwable or even just switch to RuntimeExceptions? --- .../java/akka/dispatch/JavaFutureTests.java | 20 ++++++------- .../src/main/scala/akka/dispatch/Future.scala | 6 ++++ .../docs/actor/FaultHandlingTestBase.java | 2 +- .../docs/actor/UntypedActorDocTestBase.java | 6 ++-- .../akka/docs/future/FutureDocTestBase.java | 28 +++++++++---------- .../jrouting/CustomRouterDocTestBase.java | 2 +- .../code/akka/docs/jrouting/ParentActor.java | 2 +- .../docs/transactor/TransactorDocTest.java | 6 ++-- .../main/scala/akka/testkit/TestLatch.scala | 3 +- .../UntypedCoordinatedIncrementTest.java | 4 +-- .../transactor/UntypedTransactorTest.java | 4 +-- 11 files changed, 45 insertions(+), 38 deletions(-) diff --git a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java index b03fe3b5fc..bb6b3b79a6 100644 --- a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java +++ b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java @@ -39,7 +39,7 @@ public class JavaFutureTests { } @Test - public void mustBeAbleToMapAFuture() { + public void mustBeAbleToMapAFuture() throws Exception { Future f1 = Futures.future(new Callable() { public String call() { @@ -163,7 +163,7 @@ public class JavaFutureTests { // TODO: Improve this test, perhaps with an Actor @Test - public void mustSequenceAFutureList() { + public void mustSequenceAFutureList() throws Exception{ LinkedList> listFutures = new LinkedList>(); LinkedList listExpected = new LinkedList(); @@ -183,7 +183,7 @@ public class JavaFutureTests { // TODO: Improve this test, perhaps with an Actor @Test - public void foldForJavaApiMustWork() { + public void foldForJavaApiMustWork() throws Exception{ LinkedList> listFutures = new LinkedList>(); StringBuilder expected = new StringBuilder(); @@ -206,7 +206,7 @@ public class JavaFutureTests { } @Test - public void reduceForJavaApiMustWork() { + public void reduceForJavaApiMustWork() throws Exception{ LinkedList> listFutures = new LinkedList>(); StringBuilder expected = new StringBuilder(); @@ -229,7 +229,7 @@ public class JavaFutureTests { } @Test - public void traverseForJavaApiMustWork() { + public void traverseForJavaApiMustWork() throws Exception{ LinkedList listStrings = new LinkedList(); LinkedList expectedStrings = new LinkedList(); @@ -252,7 +252,7 @@ public class JavaFutureTests { } @Test - public void findForJavaApiMustWork() { + public void findForJavaApiMustWork() throws Exception{ LinkedList> listFutures = new LinkedList>(); for (int i = 0; i < 10; i++) { final Integer fi = i; @@ -273,7 +273,7 @@ public class JavaFutureTests { } @Test - public void blockMustBeCallable() { + public void blockMustBeCallable() throws Exception { Promise p = Futures.promise(system.dispatcher()); Duration d = Duration.create(1, TimeUnit.SECONDS); p.success("foo"); @@ -282,7 +282,7 @@ public class JavaFutureTests { } @Test - public void mapToMustBeCallable() { + public void mapToMustBeCallable() throws Exception { Promise p = Futures.promise(system.dispatcher()); Future f = p.future().mapTo(manifest(String.class)); Duration d = Duration.create(1, TimeUnit.SECONDS); @@ -292,7 +292,7 @@ public class JavaFutureTests { } @Test - public void recoverToMustBeCallable() { + public void recoverToMustBeCallable() throws Exception { final IllegalStateException fail = new IllegalStateException("OHNOES"); Promise p = Futures.promise(system.dispatcher()); Future f = p.future().recover(new Recover() { @@ -309,7 +309,7 @@ public class JavaFutureTests { } @Test - public void recoverWithToMustBeCallable() { + public void recoverWithToMustBeCallable() throws Exception{ final IllegalStateException fail = new IllegalStateException("OHNOES"); Promise p = Futures.promise(system.dispatcher()); Future f = p.future().recoverWith(new Recover>() { diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index d6f4751d6e..959bd53503 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -38,12 +38,14 @@ object Await { * Should throw [[java.util.concurrent.TimeoutException]] if times out * This method should not be called directly. */ + @throws(classOf[TimeoutException]) def ready(atMost: Duration)(implicit permit: CanAwait): this.type /** * Throws exceptions if cannot produce a T within the specified time * This method should not be called directly. */ + @throws(classOf[Exception]) def result(atMost: Duration)(implicit permit: CanAwait): T } @@ -56,6 +58,7 @@ object Await { * @throws [[java.util.concurrent.TimeoutException]] if times out * @return The returned value as returned by Awaitable.ready */ + @throws(classOf[TimeoutException]) def ready[T <: Awaitable[_]](awaitable: T, atMost: Duration): T = awaitable.ready(atMost) /** @@ -65,6 +68,7 @@ object Await { * @throws [[java.util.concurrent.TimeoutException]] if times out * @return The returned value as returned by Awaitable.result */ + @throws(classOf[Exception]) def result[T](awaitable: Awaitable[T], atMost: Duration): T = awaitable.result(atMost) } @@ -818,10 +822,12 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac awaitUnsafe(if (atMost.isFinite) atMost.toNanos else Long.MaxValue) } + @throws(classOf[TimeoutException]) def ready(atMost: Duration)(implicit permit: CanAwait): this.type = if (isCompleted || tryAwait(atMost)) this else throw new TimeoutException("Futures timed out after [" + atMost.toMillis + "] milliseconds") + @throws(classOf[Exception]) def result(atMost: Duration)(implicit permit: CanAwait): T = ready(atMost).value.get match { case Left(e: AskTimeoutException) ⇒ throw new AskTimeoutException(e.getMessage, e) // to get meaningful stack trace diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java index bb8f11467c..dc2ce9bae7 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java @@ -150,7 +150,7 @@ public class FaultHandlingTestBase { } @Test - public void mustEmploySupervisorStrategy() { + public void mustEmploySupervisorStrategy() throws Exception { // code here //#testkit EventFilter ex1 = (EventFilter) new ErrorFilter(ArithmeticException.class); diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java index 749dd1e1d9..1237797b62 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java @@ -130,7 +130,7 @@ public class UntypedActorDocTestBase { } @Test - public void usingAsk() { + public void usingAsk() throws Exception { ActorSystem system = ActorSystem.create("MySystem"); ActorRef myActor = system.actorOf(new Props(new UntypedActorFactory() { public UntypedActor create() { @@ -188,7 +188,7 @@ public class UntypedActorDocTestBase { } @Test - public void useWatch() { + public void useWatch() throws Exception { ActorSystem system = ActorSystem.create("MySystem"); ActorRef myActor = system.actorOf(new Props(WatchActor.class)); Future future = Patterns.ask(myActor, "kill", 1000); @@ -197,7 +197,7 @@ public class UntypedActorDocTestBase { } @Test - public void usePatternsGracefulStop() { + public void usePatternsGracefulStop() throws Exception { ActorSystem system = ActorSystem.create("MySystem"); ActorRef actorRef = system.actorOf(new Props(MyUntypedActor.class)); //#gracefulStop diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index e541c925c1..5a6d2e5acd 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -74,7 +74,7 @@ public class FutureDocTestBase { } @Test - public void useBlockingFromActor() { + public void useBlockingFromActor() throws Exception { ActorRef actor = system.actorOf(new Props(MyActor.class)); String msg = "hello"; //#ask-blocking @@ -86,7 +86,7 @@ public class FutureDocTestBase { } @Test - public void useFutureEval() { + public void useFutureEval() throws Exception { //#future-eval Future f = future(new Callable() { public String call() { @@ -99,7 +99,7 @@ public class FutureDocTestBase { } @Test - public void useMap() { + public void useMap() throws Exception { //#map Future f1 = future(new Callable() { public String call() { @@ -162,7 +162,7 @@ public class FutureDocTestBase { } @Test - public void useFlatMap() { + public void useFlatMap() throws Exception { //#flat-map Future f1 = future(new Callable() { public String call() { @@ -186,7 +186,7 @@ public class FutureDocTestBase { } @Test - public void useSequence() { + public void useSequence() throws Exception { List> source = new ArrayList>(); source.add(Futures.successful(1, system.dispatcher())); source.add(Futures.successful(2, system.dispatcher())); @@ -214,7 +214,7 @@ public class FutureDocTestBase { } @Test - public void useTraverse() { + public void useTraverse() throws Exception { //#traverse //Just a sequence of Strings Iterable listStrings = Arrays.asList("a", "b", "c"); @@ -236,7 +236,7 @@ public class FutureDocTestBase { } @Test - public void useFold() { + public void useFold() throws Exception { List> source = new ArrayList>(); source.add(Futures.successful("a", system.dispatcher())); source.add(Futures.successful("b", system.dispatcher())); @@ -258,7 +258,7 @@ public class FutureDocTestBase { } @Test - public void useReduce() { + public void useReduce() throws Exception { List> source = new ArrayList>(); source.add(Futures.successful("a", system.dispatcher())); source.add(Futures.successful("b", system.dispatcher())); @@ -280,7 +280,7 @@ public class FutureDocTestBase { } @Test - public void useSuccessfulAndFailed() { + public void useSuccessfulAndFailed() throws Exception { //#successful Future future = Futures.successful("Yay!", system.dispatcher()); //#successful @@ -294,7 +294,7 @@ public class FutureDocTestBase { } @Test - public void useFilter() { + public void useFilter() throws Exception { //#filter Future future1 = Futures.successful(4, system.dispatcher()); Future successfulFilter = future1.filter(new Filter() { @@ -338,7 +338,7 @@ public class FutureDocTestBase { } @Test - public void useRecover() { + public void useRecover() throws Exception { //#recover Future future = future(new Callable() { public Integer call() { @@ -358,7 +358,7 @@ public class FutureDocTestBase { } @Test - public void useTryRecover() { + public void useTryRecover() throws Exception { //#try-recover Future future = future(new Callable() { public Integer call() { @@ -382,7 +382,7 @@ public class FutureDocTestBase { } @Test - public void useOnSuccessOnFailureAndOnComplete() { + public void useOnSuccessOnFailureAndOnComplete() throws Exception { { Future future = Futures.successful("foo", system.dispatcher()); @@ -429,7 +429,7 @@ public class FutureDocTestBase { } @Test - public void useOrAndZip() { + public void useOrAndZip() throws Exception { { //#zip Future future1 = Futures.successful("foo", system.dispatcher()); diff --git a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java b/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java index a20a351f06..6620c9e130 100644 --- a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java +++ b/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java @@ -55,7 +55,7 @@ public class CustomRouterDocTestBase { //#crTest @Test - public void countVotesAsIntendedNotAsInFlorida() { + public void countVotesAsIntendedNotAsInFlorida() throws Exception { ActorRef routedActor = system.actorOf(new Props().withRouter(new VoteCountRouter())); routedActor.tell(DemocratVote); routedActor.tell(DemocratVote); diff --git a/akka-docs/java/code/akka/docs/jrouting/ParentActor.java b/akka-docs/java/code/akka/docs/jrouting/ParentActor.java index cf1e2b9cee..32a33b3a1b 100644 --- a/akka-docs/java/code/akka/docs/jrouting/ParentActor.java +++ b/akka-docs/java/code/akka/docs/jrouting/ParentActor.java @@ -18,7 +18,7 @@ import akka.dispatch.Await; //#parentActor public class ParentActor extends UntypedActor { - public void onReceive(Object msg) { + public void onReceive(Object msg) throws Exception { if (msg.equals("rrr")) { //#roundRobinRouter ActorRef roundRobinRouter = getContext().actorOf( diff --git a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java index f00713a007..bb1d38651b 100644 --- a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java +++ b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java @@ -20,7 +20,7 @@ import static java.util.concurrent.TimeUnit.SECONDS; public class TransactorDocTest { @Test - public void coordinatedExample() { + public void coordinatedExample() throws Exception { //#coordinated-example ActorSystem system = ActorSystem.create("CoordinatedExample"); @@ -63,7 +63,7 @@ public class TransactorDocTest { } @Test - public void counterTransactor() { + public void counterTransactor() throws Exception { ActorSystem system = ActorSystem.create("CounterTransactor"); ActorRef counter = system.actorOf(new Props(Counter.class)); @@ -79,7 +79,7 @@ public class TransactorDocTest { } @Test - public void friendlyCounterTransactor() { + public void friendlyCounterTransactor() throws Exception { ActorSystem system = ActorSystem.create("FriendlyCounterTransactor"); ActorRef friend = system.actorOf(new Props(Counter.class)); ActorRef friendlyCounter = system.actorOf(new Props(FriendlyCounter.class)); diff --git a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala index d57ed76877..f9e426e20e 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala @@ -30,13 +30,14 @@ class TestLatch(count: Int = 1)(implicit system: ActorSystem) extends Awaitable[ def open() = while (!isOpen) countDown() def reset() = latch = new CountDownLatch(count) + @throws(classOf[TimeoutException]) def ready(atMost: Duration)(implicit permit: CanAwait) = { val opened = latch.await(atMost.dilated.toNanos, TimeUnit.NANOSECONDS) if (!opened) throw new TimeoutException( "Timeout of %s with time factor of %s" format (atMost.toString, TestKitExtension(system).TestTimeFactor)) this } - + @throws(classOf[Exception]) def result(atMost: Duration)(implicit permit: CanAwait): Unit = { ready(atMost) } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java index 66d666dfd3..60a887f554 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java @@ -72,7 +72,7 @@ public class UntypedCoordinatedIncrementTest { } @Test - public void incrementAllCountersWithSuccessfulTransaction() { + public void incrementAllCountersWithSuccessfulTransaction() throws Exception { CountDownLatch incrementLatch = new CountDownLatch(numCounters); Increment message = new Increment(counters.subList(1, counters.size()), incrementLatch); counters.get(0).tell(new Coordinated(message, timeout)); @@ -88,7 +88,7 @@ public class UntypedCoordinatedIncrementTest { } @Test - public void incrementNoCountersWithFailingTransaction() { + public void incrementNoCountersWithFailingTransaction() throws Exception { EventFilter expectedFailureFilter = (EventFilter) new ErrorFilter(ExpectedFailureException.class); EventFilter coordinatedFilter = (EventFilter) new ErrorFilter(CoordinatedTransactionException.class); Seq ignoreExceptions = seq(expectedFailureFilter, coordinatedFilter); diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java index 5820c82573..cadc4828b1 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java @@ -73,7 +73,7 @@ public class UntypedTransactorTest { } @Test - public void incrementAllCountersWithSuccessfulTransaction() { + public void incrementAllCountersWithSuccessfulTransaction() throws Exception { CountDownLatch incrementLatch = new CountDownLatch(numCounters); Increment message = new Increment(counters.subList(1, counters.size()), incrementLatch); counters.get(0).tell(message); @@ -89,7 +89,7 @@ public class UntypedTransactorTest { } @Test - public void incrementNoCountersWithFailingTransaction() { + public void incrementNoCountersWithFailingTransaction() throws Exception { EventFilter expectedFailureFilter = (EventFilter) new ErrorFilter(ExpectedFailureException.class); EventFilter coordinatedFilter = (EventFilter) new ErrorFilter(CoordinatedTransactionException.class); Seq ignoreExceptions = seq(expectedFailureFilter, coordinatedFilter); From c4705542dd12481bca236621167e0afcca958314 Mon Sep 17 00:00:00 2001 From: Roland Date: Sat, 18 Feb 2012 12:33:32 +0100 Subject: [PATCH 02/46] improve Scaladoc of ActorContext.children --- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index a5331e9f49..0dc2d01846 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -93,7 +93,15 @@ trait ActorContext extends ActorRefFactory { def sender: ActorRef /** - * Returns all supervised children. + * Returns all supervised children; this method returns a view onto the + * internal collection of children. Targeted lookups should be using + * `actorFor` instead for performance reasons: + * + * {{{ + * val badLookup = context.children find (_.path.name == "kid") + * // should better be expressed as: + * val goodLookup = context.actorFor("kid") + * }}} */ def children: Iterable[ActorRef] From 1589e1de9f4e0c82a59ee0395802073b1c0be855 Mon Sep 17 00:00:00 2001 From: Roland Date: Sat, 18 Feb 2012 12:39:38 +0100 Subject: [PATCH 03/46] make ActorSystem.shutdown() asynchronous, see #1840 - it was asynchronous by accident before, but should be explicitly so --- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 2 +- akka-actor/src/main/scala/akka/actor/ActorSystem.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 0dc2d01846..39bd52aa87 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -96,7 +96,7 @@ trait ActorContext extends ActorRefFactory { * Returns all supervised children; this method returns a view onto the * internal collection of children. Targeted lookups should be using * `actorFor` instead for performance reasons: - * + * * {{{ * val badLookup = context.children find (_.path.name == "kid") * // should better be expressed as: diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index bb0510752e..6cdcf8817d 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -500,7 +500,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf def awaitTermination(timeout: Duration) { Await.ready(terminationCallbacks, timeout) } def awaitTermination() = awaitTermination(Duration.Inf) - def shutdown(): Unit = stop(guardian) + def shutdown(): Unit = guardian.stop() /** * Create the scheduler service. This one needs one special behavior: if From c2376d01c0e3f86c4d9e76a5a070e1ee6856713e Mon Sep 17 00:00:00 2001 From: Roland Date: Sat, 18 Feb 2012 12:46:42 +0100 Subject: [PATCH 04/46] explicitly set supervisorStrategy for both guardians, see #1839 also obtain them from a protected def to make them overridable in subclasses --- .../scala/akka/actor/ActorRefProvider.scala | 36 ++++++++++++++----- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index a176c6271d..255a42d87c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -380,6 +380,18 @@ class LocalActorRefProvider( } } + /** + * Overridable supervision strategy to be used by the “/user” guardian. + */ + protected def guardianSupervisionStrategy = { + import akka.actor.SupervisorStrategy._ + OneForOneStrategy() { + case _: ActorKilledException ⇒ Stop + case _: ActorInitializationException ⇒ Stop + case _: Exception ⇒ Restart + } + } + /* * Guardians can be asked by ActorSystem to create children, i.e. top-level * actors. Therefore these need to answer to these requests, forwarding any @@ -387,14 +399,7 @@ class LocalActorRefProvider( */ private class Guardian extends Actor { - override val supervisorStrategy = { - import akka.actor.SupervisorStrategy._ - OneForOneStrategy() { - case _: ActorKilledException ⇒ Stop - case _: ActorInitializationException ⇒ Stop - case _: Exception ⇒ Restart - } - } + override val supervisorStrategy = guardianSupervisionStrategy def receive = { case Terminated(_) ⇒ context.stop(self) @@ -408,12 +413,27 @@ class LocalActorRefProvider( override def preRestart(cause: Throwable, msg: Option[Any]) {} } + /** + * Overridable supervision strategy to be used by the “/system” guardian. + */ + protected def systemGuardianSupervisionStrategy = { + import akka.actor.SupervisorStrategy._ + OneForOneStrategy() { + case _: ActorKilledException ⇒ Stop + case _: ActorInitializationException ⇒ Stop + case _: Exception ⇒ Restart + } + } + /* * Guardians can be asked by ActorSystem to create children, i.e. top-level * actors. Therefore these need to answer to these requests, forwarding any * exceptions which might have occurred. */ private class SystemGuardian extends Actor { + + override val supervisorStrategy = systemGuardianSupervisionStrategy + def receive = { case Terminated(_) ⇒ eventStream.stopDefaultLoggers() From 0f48b9f3eb148dbdcd3a99e755ca5ee2f1c3fb29 Mon Sep 17 00:00:00 2001 From: Roland Date: Sat, 18 Feb 2012 22:15:39 +0100 Subject: [PATCH 05/46] make supervisorStrategy of Router configurable, see #1835 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - also do not scrap router’s children upon restart - and add docs and tests --- .../test/scala/akka/routing/RoutingSpec.scala | 40 ++++ .../src/main/scala/akka/routing/Routing.scala | 202 +++++++++++++++++- .../jrouting/CustomRouterDocTestBase.java | 14 ++ akka-docs/java/routing.rst | 23 ++ akka-docs/scala/routing.rst | 24 +++ .../akka/routing/RemoteRouterConfig.scala | 3 + 6 files changed, 300 insertions(+), 6 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index ad3702d556..645d68a47b 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -53,6 +53,7 @@ object RoutingSpec { } } def routerDispatcher: String = Dispatchers.DefaultDispatcherId + def supervisorStrategy: SupervisorStrategy = SupervisorStrategy.defaultStrategy } } @@ -126,6 +127,44 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with system.stop(router) } + "set supplied supervisorStrategy" in { + //#supervision + val escalator = OneForOneStrategy() { + //#custom-strategy + case e ⇒ testActor ! e; SupervisorStrategy.Escalate + //#custom-strategy + } + val router = system.actorOf(Props.empty.withRouter( + RoundRobinRouter(1, supervisorStrategy = escalator))) + //#supervision + router ! CurrentRoutees + EventFilter[ActorKilledException](occurrences = 2) intercept { + expectMsgType[RouterRoutees].routees.head ! Kill + } + expectMsgType[ActorKilledException] + } + + "default to all-for-one-always-escalate strategy" in { + val restarter = OneForOneStrategy() { + case e ⇒ testActor ! e; SupervisorStrategy.Restart + } + val supervisor = system.actorOf(Props(new Supervisor(restarter))) + supervisor ! Props(new Actor { + def receive = { + case x: String ⇒ throw new Exception(x) + } + override def postRestart(reason: Throwable): Unit = testActor ! "restarted" + }).withRouter(RoundRobinRouter(3)) + val router = expectMsgType[ActorRef] + EventFilter[Exception]("die", occurrences = 2) intercept { + router ! "die" + } + expectMsgType[Exception].getMessage must be("die") + expectMsg("restarted") + expectMsg("restarted") + expectMsg("restarted") + } + } "no router" must { @@ -542,6 +581,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with case class VoteCountRouter() extends RouterConfig { def routerDispatcher: String = Dispatchers.DefaultDispatcherId + def supervisorStrategy: SupervisorStrategy = SupervisorStrategy.defaultStrategy //#crRoute def createRoute(routeeProps: Props, routeeProvider: RouteeProvider): Route = { diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 44faa67d47..e988206e9f 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -171,7 +171,14 @@ trait RouterConfig { def createRouteeProvider(context: ActorContext) = new RouteeProvider(context, resizer) - def createActor(): Router = new Router {} + def createActor(): Router = new Router { + override def supervisorStrategy: SupervisorStrategy = RouterConfig.this.supervisorStrategy + } + + /** + * SupervisorStrategy for the created Router actor. + */ + def supervisorStrategy: SupervisorStrategy /** * Dispatcher ID to use for running the “head” actor, i.e. the [[akka.routing.Router]]. @@ -308,10 +315,19 @@ trait Router extends Actor { def routerReceive: Receive = { case _ ⇒ } + + override def preRestart(cause: Throwable, msg: Option[Any]): Unit = { + // do not scrap children + } } private object Router { + case object Resize + + val defaultSupervisorStrategy: SupervisorStrategy = OneForOneStrategy() { + case _ ⇒ SupervisorStrategy.Escalate + } } /** @@ -353,6 +369,7 @@ case class Destination(sender: ActorRef, recipient: ActorRef) case object NoRouter extends RouterConfig { def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null def routerDispatcher: String = "" + def supervisorStrategy = null override def withFallback(other: RouterConfig): RouterConfig = other } @@ -363,6 +380,7 @@ case object FromConfig extends RouterConfig { def createRoute(props: Props, routeeProvider: RouteeProvider): Route = throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)") def routerDispatcher: String = Dispatchers.DefaultDispatcherId + def supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy } /** @@ -378,6 +396,8 @@ case class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatch def createRoute(props: Props, routeeProvider: RouteeProvider): Route = throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)") + + def supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy } object RoundRobinRouter { @@ -402,12 +422,40 @@ object RoundRobinRouter { * if you provide either 'nrOfInstances' or 'routees' during instantiation they will * be ignored if the router is defined in the configuration file for the actor being used. * + *

Supervision Setup

+ * + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to + * different groups of children. + * + * {{{ + * class MyActor extends Actor { + * override val supervisorStrategy = ... + * + * val poolAsAWhole = context.actorOf(Props[SomeActor].withRouter(RoundRobinRouter(5))) + * + * val poolIndividuals = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = this.supervisorStrategy))) + * + * val specialChild = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = OneForOneStrategy() { + * ... + * }))) + * } + * }}} + * * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, - val routerDispatcher: String = Dispatchers.DefaultDispatcherId) + val routerDispatcher: String = Dispatchers.DefaultDispatcherId, + val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) extends RouterConfig with RoundRobinLike { /** @@ -438,6 +486,12 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = * Java API for setting routerDispatcher */ def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + + /** + * Java API for setting the supervisor strategy to be used for the “head” + * Router actor. + */ + def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) } trait RoundRobinLike { this: RouterConfig ⇒ @@ -488,12 +542,40 @@ object RandomRouter { * if you provide either 'nrOfInstances' or 'routees' during instantiation they will * be ignored if the router is defined in the configuration file for the actor being used. * + *

Supervision Setup

+ * + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to + * different groups of children. + * + * {{{ + * class MyActor extends Actor { + * override val supervisorStrategy = ... + * + * val poolAsAWhole = context.actorOf(Props[SomeActor].withRouter(RoundRobinRouter(5))) + * + * val poolIndividuals = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = this.supervisorStrategy))) + * + * val specialChild = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = OneForOneStrategy() { + * ... + * }))) + * } + * }}} + * * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, - val routerDispatcher: String = Dispatchers.DefaultDispatcherId) + val routerDispatcher: String = Dispatchers.DefaultDispatcherId, + val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) extends RouterConfig with RandomLike { /** @@ -524,6 +606,12 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, * Java API for setting routerDispatcher */ def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + + /** + * Java API for setting the supervisor strategy to be used for the “head” + * Router actor. + */ + def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) } trait RandomLike { this: RouterConfig ⇒ @@ -580,12 +668,40 @@ object SmallestMailboxRouter { * if you provide either 'nrOfInstances' or 'routees' during instantiation they will * be ignored if the router is defined in the configuration file for the actor being used. * + *

Supervision Setup

+ * + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to + * different groups of children. + * + * {{{ + * class MyActor extends Actor { + * override val supervisorStrategy = ... + * + * val poolAsAWhole = context.actorOf(Props[SomeActor].withRouter(RoundRobinRouter(5))) + * + * val poolIndividuals = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = this.supervisorStrategy))) + * + * val specialChild = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = OneForOneStrategy() { + * ... + * }))) + * } + * }}} + * * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, - val routerDispatcher: String = Dispatchers.DefaultDispatcherId) + val routerDispatcher: String = Dispatchers.DefaultDispatcherId, + val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) extends RouterConfig with SmallestMailboxLike { /** @@ -616,6 +732,12 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin * Java API for setting routerDispatcher */ def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + + /** + * Java API for setting the supervisor strategy to be used for the “head” + * Router actor. + */ + def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) } trait SmallestMailboxLike { this: RouterConfig ⇒ @@ -731,12 +853,40 @@ object BroadcastRouter { * if you provide either 'nrOfInstances' or 'routees' during instantiation they will * be ignored if the router is defined in the configuration file for the actor being used. * + *

Supervision Setup

+ * + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to + * different groups of children. + * + * {{{ + * class MyActor extends Actor { + * override val supervisorStrategy = ... + * + * val poolAsAWhole = context.actorOf(Props[SomeActor].withRouter(RoundRobinRouter(5))) + * + * val poolIndividuals = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = this.supervisorStrategy))) + * + * val specialChild = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = OneForOneStrategy() { + * ... + * }))) + * } + * }}} + * * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, - val routerDispatcher: String = Dispatchers.DefaultDispatcherId) + val routerDispatcher: String = Dispatchers.DefaultDispatcherId, + val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) extends RouterConfig with BroadcastLike { /** @@ -767,6 +917,12 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N * Java API for setting routerDispatcher */ def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + + /** + * Java API for setting the supervisor strategy to be used for the “head” + * Router actor. + */ + def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) } trait BroadcastLike { this: RouterConfig ⇒ @@ -808,13 +964,41 @@ object ScatterGatherFirstCompletedRouter { * if you provide either 'nrOfInstances' or 'routees' during instantiation they will * be ignored if the router is defined in the configuration file for the actor being used. * + *

Supervision Setup

+ * + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to + * different groups of children. + * + * {{{ + * class MyActor extends Actor { + * override val supervisorStrategy = ... + * + * val poolAsAWhole = context.actorOf(Props[SomeActor].withRouter(RoundRobinRouter(5))) + * + * val poolIndividuals = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = this.supervisorStrategy))) + * + * val specialChild = context.actorOf(Props[SomeActor].withRouter( + * RoundRobinRouter(5, supervisorStrategy = OneForOneStrategy() { + * ... + * }))) + * } + * }}} + * * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ //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, - val routerDispatcher: String = Dispatchers.DefaultDispatcherId) + val routerDispatcher: String = Dispatchers.DefaultDispatcherId, + val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) extends RouterConfig with ScatterGatherFirstCompletedLike { if (within <= Duration.Zero) throw new IllegalArgumentException( @@ -848,6 +1032,12 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It * Java API for setting routerDispatcher */ def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + + /** + * Java API for setting the supervisor strategy to be used for the “head” + * Router actor. + */ + def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) } trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒ diff --git a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java b/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java index a20a351f06..922195abd3 100644 --- a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java +++ b/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java @@ -52,6 +52,16 @@ public class CustomRouterDocTestBase { .withDispatcher("workers")); // MyActor “workers” run on "workers" dispatcher //#dispatchers } + + @Test + public void demonstrateSupervisor() { + //#supervision + final SupervisorStrategy strategy = new OneForOneStrategy(5, Duration.parse("1 minute"), + new Class[] { Exception }); + final ActorRef router = system.actorOf(new Props(MyActor.class) + .withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy))); + //#supervision + } //#crTest @Test @@ -123,6 +133,10 @@ public class CustomRouterDocTestBase { @Override public String routerDispatcher() { return Dispatchers.DefaultDispatcherId(); } + + @Override public SupervisorStrategy supervisorStrategy() { + return SupervisorStrategy.defaultStrategy(); + } //#crRoute @Override diff --git a/akka-docs/java/routing.rst b/akka-docs/java/routing.rst index ebb219ae5d..aa1daa19db 100644 --- a/akka-docs/java/routing.rst +++ b/akka-docs/java/routing.rst @@ -92,6 +92,29 @@ to the actor hierarchy, changing the actor paths of all children of the router. The routees especially do need to know that they are routed to in order to choose the sender reference for any messages they dispatch as shown above. +Routers vs. Supervision +^^^^^^^^^^^^^^^^^^^^^^^ + +As explained in the previous section, routers create new actor instances as +children of the “head” router, who therefor also is their supervisor. The +supervisor strategy of this actor can be configured by means of the +:meth:`RouterConfig.supervisorStrategy` property, which is supported for all +built-in router types. It defaults to “always escalate”, which leads to the +application of the router’s parent’s supervision directive to all children of +the router uniformly (i.e. not only the one which failed). It should be +mentioned that the router overrides the default behavior of terminating all +children upon restart, which means that a restart—while re-creating them—does +not have an effect on the number of actors in the pool. + +Setting the strategy is easily done: + +.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java + :include: supervision + +Another potentially useful approach is to give the router the same strategy as +its parent, which effectively treats all actors in the pool as if they were +direct children of their grand-parent instead. + Router usage ^^^^^^^^^^^^ diff --git a/akka-docs/scala/routing.rst b/akka-docs/scala/routing.rst index dfae20fd6b..7071747374 100644 --- a/akka-docs/scala/routing.rst +++ b/akka-docs/scala/routing.rst @@ -92,6 +92,30 @@ to the actor hierarchy, changing the actor paths of all children of the router. The routees especially do need to know that they are routed to in order to choose the sender reference for any messages they dispatch as shown above. +Routers vs. Supervision +^^^^^^^^^^^^^^^^^^^^^^^ + +As explained in the previous section, routers create new actor instances as +children of the “head” router, who therefor also is their supervisor. The +supervisor strategy of this actor can be configured by means of the +:meth:`RouterConfig.supervisorStrategy` property, which is supported for all +built-in router types. It defaults to “always escalate”, which leads to the +application of the router’s parent’s supervision directive to all children of +the router uniformly (i.e. not only the one which failed). It should be +mentioned that the router overrides the default behavior of terminating all +children upon restart, which means that a restart—while re-creating them—does +not have an effect on the number of actors in the pool. + +Setting the strategy is easily done: + +.. includecode:: ../../akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala + :include: supervision + :exclude: custom-strategy + +Another potentially useful approach is to give the router the same strategy as +its parent, which effectively treats all actors in the pool as if they were +direct children of their grand-parent instead. + Router usage ^^^^^^^^^^^^ diff --git a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala index 3b1791db8e..afa94433aa 100644 --- a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala +++ b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala @@ -13,6 +13,7 @@ import akka.actor.Props import akka.config.ConfigurationException import akka.remote.RemoteScope import akka.actor.AddressExtractor +import akka.actor.SupervisorStrategy /** * [[akka.routing.RouterConfig]] implementation for remote deployment on defined @@ -30,6 +31,8 @@ case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[String]) exte override def createActor(): Router = local.createActor() + override def supervisorStrategy: SupervisorStrategy = local.supervisorStrategy + override def routerDispatcher: String = local.routerDispatcher override def resizer: Option[Resizer] = local.resizer From b5826f9bd916ad53231742d1642f671215f5d4a7 Mon Sep 17 00:00:00 2001 From: Roland Date: Sat, 18 Feb 2012 22:32:41 +0100 Subject: [PATCH 06/46] make LoggingReceiveSpec more robust --- .../scala/akka/event/LoggingReceiveSpec.scala | 2 + .../test/scala/akka/routing/RoutingSpec.scala | 2 +- .../src/main/scala/akka/routing/Routing.scala | 70 +++++++++---------- 3 files changed, 38 insertions(+), 36 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala index 1ac765e400..a7218230e4 100644 --- a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala @@ -56,6 +56,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd "decorate a Receive" in { new TestKit(appLogging) { system.eventStream.subscribe(testActor, classOf[Logging.Debug]) + system.eventStream.subscribe(testActor, classOf[UnhandledMessage]) val a = system.actorOf(Props(new Actor { def receive = new LoggingReceive(Some("funky"), { case null ⇒ @@ -63,6 +64,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd })) a ! "hallo" expectMsg(1 second, Logging.Debug("funky", classOf[DummyClassForStringSources], "received unhandled message hallo")) + expectMsgType[UnhandledMessage](1 second) } } diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index 645d68a47b..89b6f5274a 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -135,7 +135,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with //#custom-strategy } val router = system.actorOf(Props.empty.withRouter( - RoundRobinRouter(1, supervisorStrategy = escalator))) + RoundRobinRouter(1, supervisorStrategy = escalator))) //#supervision router ! CurrentRoutees EventFilter[ActorKilledException](occurrences = 2) intercept { diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index e988206e9f..57847f3553 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -424,13 +424,13 @@ object RoundRobinRouter { * *

Supervision Setup

* - * The router creates a “head” actor which supervises and/or monitors the - * routees. Instances are created as children of this actor, hence the - * children are not supervised by the parent of the router. Common choices are - * to always escalate (meaning that fault handling is always applied to all - * children simultaneously; this is the default) or use the parent’s strategy, - * which will result in routed children being treated individually, but it is - * possible as well to use Routers to give different supervisor strategies to + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to * different groups of children. * * {{{ @@ -544,13 +544,13 @@ object RandomRouter { * *

Supervision Setup

* - * The router creates a “head” actor which supervises and/or monitors the - * routees. Instances are created as children of this actor, hence the - * children are not supervised by the parent of the router. Common choices are - * to always escalate (meaning that fault handling is always applied to all - * children simultaneously; this is the default) or use the parent’s strategy, - * which will result in routed children being treated individually, but it is - * possible as well to use Routers to give different supervisor strategies to + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to * different groups of children. * * {{{ @@ -670,13 +670,13 @@ object SmallestMailboxRouter { * *

Supervision Setup

* - * The router creates a “head” actor which supervises and/or monitors the - * routees. Instances are created as children of this actor, hence the - * children are not supervised by the parent of the router. Common choices are - * to always escalate (meaning that fault handling is always applied to all - * children simultaneously; this is the default) or use the parent’s strategy, - * which will result in routed children being treated individually, but it is - * possible as well to use Routers to give different supervisor strategies to + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to * different groups of children. * * {{{ @@ -855,13 +855,13 @@ object BroadcastRouter { * *

Supervision Setup

* - * The router creates a “head” actor which supervises and/or monitors the - * routees. Instances are created as children of this actor, hence the - * children are not supervised by the parent of the router. Common choices are - * to always escalate (meaning that fault handling is always applied to all - * children simultaneously; this is the default) or use the parent’s strategy, - * which will result in routed children being treated individually, but it is - * possible as well to use Routers to give different supervisor strategies to + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to * different groups of children. * * {{{ @@ -966,13 +966,13 @@ object ScatterGatherFirstCompletedRouter { * *

Supervision Setup

* - * The router creates a “head” actor which supervises and/or monitors the - * routees. Instances are created as children of this actor, hence the - * children are not supervised by the parent of the router. Common choices are - * to always escalate (meaning that fault handling is always applied to all - * children simultaneously; this is the default) or use the parent’s strategy, - * which will result in routed children being treated individually, but it is - * possible as well to use Routers to give different supervisor strategies to + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to * different groups of children. * * {{{ From ec15fd6cfdf4dda4158410da32e1db3f0302799a Mon Sep 17 00:00:00 2001 From: Roland Date: Sun, 19 Feb 2012 00:09:04 +0100 Subject: [PATCH 07/46] add TestKit.setAutoPilot, see #1807 --- akka-docs/scala/testing.rst | 23 +++++++++++++++++ .../src/main/scala/akka/testkit/TestKit.scala | 22 +++++++++++++--- .../scala/akka/testkit/TestProbeSpec.scala | 25 +++++++++++++++++++ 3 files changed, 67 insertions(+), 3 deletions(-) diff --git a/akka-docs/scala/testing.rst b/akka-docs/scala/testing.rst index db0f670c93..e99f6b6838 100644 --- a/akka-docs/scala/testing.rst +++ b/akka-docs/scala/testing.rst @@ -421,6 +421,14 @@ using a small example: .. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala :include: imports-test-probe,my-double-echo,test-probe +Here a the system under test is simulated by :class:`MyDoubleEcho`, which is +supposed to mirror its input to two outputs. Attaching two test probes enables +verification of the (simplistic) behavior. Another example would be two actors +A and B which collaborate by A sending messages to B. In order to verify this +message flow, a :class:`TestProbe` could be inserted as target of A, using the +forwarding capabilities or auto-pilot described below to include a real B in +the test setup. + Probes may also be equipped with custom assertions to make your test code even more concise and clear: @@ -455,6 +463,21 @@ network functioning: The ``dest`` actor will receive the same message invocation as if no test probe had intervened. +Auto-Pilot +^^^^^^^^^^ + +Receiving messages in a queue for later inspection is nice, but in order to +keep a test running and verify traces later you can also install an +:class:`AutoPilot` in the participating test probes (actually in any +:class:`TestKit`) which is invoked before enqueueing to the inspection queue. +This code can be used to forward messages, e.g. in a chain ``A --> Probe --> +B``, as long as a certain protocol is obeyed. + +.. includecode:: ../../akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala#autopilot + +The :meth:`run` method must return the auto-pilot for the next message, wrapped +in an :class:`Option`; setting it to :obj:`None` terminates the auto-pilot. + Caution about Timing Assertions ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index bdfab36ede..965fc555e1 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -16,9 +16,14 @@ import akka.util.Timeout object TestActor { type Ignore = Option[PartialFunction[AnyRef, Boolean]] + trait AutoPilot { + def run(sender: ActorRef, msg: Any): Option[AutoPilot] + } + case class SetIgnore(i: Ignore) case class Watch(ref: ActorRef) case class UnWatch(ref: ActorRef) + case class SetAutoPilot(ap: AutoPilot) trait Message { def msg: AnyRef @@ -36,11 +41,15 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor { var ignore: Ignore = None + var autopilot: Option[AutoPilot] = None + def receive = { - case SetIgnore(ign) ⇒ ignore = ign - case x @ Watch(ref) ⇒ context.watch(ref); queue.offerLast(RealMessage(x, self)) - case x @ UnWatch(ref) ⇒ context.unwatch(ref); queue.offerLast(RealMessage(x, self)) + case SetIgnore(ign) ⇒ ignore = ign + case x @ Watch(ref) ⇒ context.watch(ref); queue.offerLast(RealMessage(x, self)) + case x @ UnWatch(ref) ⇒ context.unwatch(ref); queue.offerLast(RealMessage(x, self)) + case SetAutoPilot(pilot) ⇒ autopilot = Some(pilot) case x: AnyRef ⇒ + autopilot = autopilot.flatMap(_.run(sender, x)) val observe = ignore map (ignoreFunc ⇒ if (ignoreFunc isDefinedAt x) !ignoreFunc(x) else true) getOrElse true if (observe) queue.offerLast(RealMessage(x, sender)) } @@ -148,6 +157,13 @@ class TestKit(_system: ActorSystem) { expectMsg(msg) } + /** + * Install an AutoPilot to drive the testActor: the AutoPilot will be run + * for each received message and can be used to send or forward messages, + * etc. Each invocation must return the AutoPilot for the next round. + */ + def setAutoPilot(pilot: TestActor.AutoPilot): Unit = testActor ! TestActor.SetAutoPilot(pilot) + /** * Obtain current time (`System.nanoTime`) as Duration. */ diff --git a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala index 516bba7322..b272544141 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala @@ -40,6 +40,31 @@ class TestProbeSpec extends AkkaSpec with DefaultTimeout { probe1.expectMsg(0 millis, "world") } + "have an AutoPilot" in { + //#autopilot + val probe = TestProbe() + probe.setAutoPilot(new TestActor.AutoPilot { + def run(sender: ActorRef, msg: Any): Option[TestActor.AutoPilot] = + msg match { + case "stop" ⇒ None + case x ⇒ testActor.tell(x, sender); Some(this) + } + }) + //#autopilot + probe.ref ! "hallo" + probe.ref ! "welt" + probe.ref ! "stop" + expectMsg("hallo") + expectMsg("welt") + probe.expectMsg("hallo") + probe.expectMsg("welt") + probe.expectMsg("stop") + probe.ref ! "hallo" + probe.expectMsg("hallo") + testActor ! "end" + expectMsg("end") // verify that "hallo" did not get through + } + } } From 62be4dafeea369e15ba466b409a0d8d07db8677b Mon Sep 17 00:00:00 2001 From: Roland Date: Sun, 19 Feb 2012 10:12:20 +0100 Subject: [PATCH 08/46] fix Java API for creating SupervisorStrategy from array --- akka-actor/src/main/scala/akka/actor/FaultHandling.scala | 6 +++--- .../code/akka/docs/jrouting/CustomRouterDocTestBase.java | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 28694dd3d3..6074780801 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -135,7 +135,7 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { * Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ - def makeDecider(trapExit: Array[Class[_ <: Throwable]]): Decider = + def makeDecider(trapExit: Array[Class[_]]): Decider = { case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate } /** @@ -248,7 +248,7 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) = this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) - def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_ <: Throwable]]) = + def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_]]) = this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) /* @@ -294,7 +294,7 @@ case class OneForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) = this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) - def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_ <: Throwable]]) = + def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_]]) = this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) /* diff --git a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java b/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java index 922195abd3..2d19ea3956 100644 --- a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java +++ b/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java @@ -57,7 +57,7 @@ public class CustomRouterDocTestBase { public void demonstrateSupervisor() { //#supervision final SupervisorStrategy strategy = new OneForOneStrategy(5, Duration.parse("1 minute"), - new Class[] { Exception }); + new Class[] { Exception.class }); final ActorRef router = system.actorOf(new Props(MyActor.class) .withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy))); //#supervision From 2f3737195b848d0de05479d2fa9b1507f9adb729 Mon Sep 17 00:00:00 2001 From: Roland Date: Sun, 19 Feb 2012 10:28:56 +0100 Subject: [PATCH 09/46] split Mailbox and MessageQueue, see #1844 - this enables using any MessageQueue in BalancingDispatcher, CallingThreadDispatcher and in general leads to less conflation of concepts - add MessageQueue.cleanUp(owner, deadLetterQueue) for the benefit of durable mailboxes - change MailboxType.create to take an optional owner and generate only a MessageQueue, not a Mailbox --- .../test/scala/akka/actor/FSMTimingSpec.scala | 4 +- .../akka/actor/dispatch/ActorModelSpec.scala | 2 +- .../dispatch/BalancingDispatcherSpec.scala | 4 +- .../akka/dispatch/MailboxConfigSpec.scala | 28 +++---- .../CallingThreadDispatcherModelSpec.scala | 3 +- .../src/main/scala/akka/actor/ActorCell.scala | 2 +- .../main/scala/akka/actor/ActorSystem.scala | 20 +++-- .../akka/dispatch/BalancingDispatcher.scala | 63 ++++++--------- .../main/scala/akka/dispatch/Dispatcher.scala | 4 +- .../main/scala/akka/dispatch/Mailbox.scala | 78 ++++++++++--------- .../src/main/scala/akka/routing/Routing.scala | 8 +- .../actor/mailbox/BeanstalkBasedMailbox.scala | 9 ++- .../actor/mailbox/FiledBasedMailbox.scala | 10 ++- .../akka/actor/mailbox/DurableMailbox.scala | 4 +- .../actor/mailbox/MongoBasedMailbox.scala | 11 ++- .../actor/mailbox/RedisBasedMailbox.scala | 9 ++- .../actor/mailbox/ZooKeeperBasedMailbox.scala | 9 ++- .../testkit/CallingThreadDispatcher.scala | 47 ++++++----- 18 files changed, 171 insertions(+), 144 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index 59468125eb..1f708983bd 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -182,7 +182,7 @@ object FSMTimingSpec { when(TestCancelTimer) { case Event(Tick, _) ⇒ setTimer("hallo", Tock, 1 milli, false) - TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.hasMessages, 1 second) + TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.messageQueue.hasMessages, 1 second) cancelTimer("hallo") sender ! Tick setTimer("hallo", Tock, 500 millis, false) @@ -209,7 +209,7 @@ object FSMTimingSpec { case Event(Tick, _) ⇒ suspend(self) setTimer("named", Tock, 1 millis, false) - TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.hasMessages, 1 second) + TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.messageQueue.hasMessages, 1 second) stay forMax (1 millis) replying Tick case Event(Tock, _) ⇒ goto(TestCancelStateTimerInNamedTimerMessage2) 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 88358e9f16..87a9cf9734 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 @@ -374,7 +374,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa def compare(l: AnyRef, r: AnyRef) = (l, r) match { case (ll: ActorCell, rr: ActorCell) ⇒ ll.self.path compareTo rr.self.path } } foreach { case cell: ActorCell ⇒ - System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain())) + System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.messageQueue.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain())) } System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala index 4060587b73..ec0982982c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala @@ -76,8 +76,8 @@ class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) { } finishedCounter.await(5, TimeUnit.SECONDS) - fast.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false) - slow.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false) + fast.underlying.mailbox.asInstanceOf[Mailbox].messageQueue.hasMessages must be(false) + slow.underlying.mailbox.asInstanceOf[Mailbox].messageQueue.hasMessages must be(false) fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be > (slow.underlying.actor.asInstanceOf[DelayableActor].invocationCount) 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 4f787a730f..21c6c75cb2 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala @@ -6,15 +6,14 @@ import java.util.concurrent.ConcurrentLinkedQueue import akka.util._ import akka.util.duration._ import akka.testkit.AkkaSpec -import akka.actor.ActorRef -import akka.actor.ActorContext +import akka.actor.{ ActorRef, ActorContext, Props, LocalActorRef } import com.typesafe.config.Config @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach { def name: String - def factory: MailboxType ⇒ Mailbox + def factory: MailboxType ⇒ MessageQueue name should { "create an unbounded mailbox" in { @@ -77,7 +76,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters)(system) - def ensureInitialMailboxState(config: MailboxType, q: Mailbox) { + def ensureInitialMailboxState(config: MailboxType, q: MessageQueue) { q must not be null q match { case aQueue: BlockingQueue[_] ⇒ @@ -136,8 +135,8 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn class DefaultMailboxSpec extends MailboxSpec { lazy val name = "The default mailbox implementation" def factory = { - case u: UnboundedMailbox ⇒ u.create(null) - case b: BoundedMailbox ⇒ b.create(null) + case u: UnboundedMailbox ⇒ u.create(None) + case b: BoundedMailbox ⇒ b.create(None) } } @@ -145,8 +144,8 @@ class PriorityMailboxSpec extends MailboxSpec { val comparator = PriorityGenerator(_.##) lazy val name = "The priority mailbox implementation" def factory = { - case UnboundedMailbox() ⇒ UnboundedPriorityMailbox(comparator).create(null) - case BoundedMailbox(capacity, pushTimeOut) ⇒ BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(null) + case UnboundedMailbox() ⇒ UnboundedPriorityMailbox(comparator).create(None) + case BoundedMailbox(capacity, pushTimeOut) ⇒ BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(None) } } @@ -158,11 +157,13 @@ object CustomMailboxSpec { """ class MyMailboxType(config: Config) extends MailboxType { - override def create(owner: ActorContext) = new MyMailbox(owner) + override def create(owner: Option[ActorContext]) = owner match { + case Some(o) ⇒ new MyMailbox(o) + case None ⇒ throw new Exception("no mailbox owner given") + } } - class MyMailbox(owner: ActorContext) extends CustomMailbox(owner) - with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { + class MyMailbox(owner: ActorContext) extends QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final val queue = new ConcurrentLinkedQueue[Envelope]() } } @@ -171,8 +172,9 @@ object CustomMailboxSpec { class CustomMailboxSpec extends AkkaSpec(CustomMailboxSpec.config) { "Dispatcher configuration" must { "support custom mailboxType" in { - val dispatcher = system.dispatchers.lookup("my-dispatcher") - dispatcher.createMailbox(null).getClass must be(classOf[CustomMailboxSpec.MyMailbox]) + val actor = system.actorOf(Props.empty.withDispatcher("my-dispatcher")) + val queue = actor.asInstanceOf[LocalActorRef].underlying.mailbox.messageQueue + queue.getClass must be(classOf[CustomMailboxSpec.MyMailbox]) } } } 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 4693a56536..5b023054d4 100644 --- a/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala @@ -9,6 +9,7 @@ import com.typesafe.config.Config import akka.dispatch.DispatcherPrerequisites import akka.dispatch.MessageDispatcher import akka.dispatch.MessageDispatcherConfigurator +import akka.dispatch.UnboundedMailbox object CallingThreadDispatcherModelSpec { import ActorModelSpec._ @@ -31,7 +32,7 @@ object CallingThreadDispatcherModelSpec { extends MessageDispatcherConfigurator(config, prerequisites) { private val instance: MessageDispatcher = - new CallingThreadDispatcher(prerequisites) with MessageDispatcherInterceptor { + new CallingThreadDispatcher(prerequisites, UnboundedMailbox()) with MessageDispatcherInterceptor { override def id: String = config.getString("id") } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 39bd52aa87..0a77457cf9 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -590,7 +590,7 @@ private[akka] class ActorCell( final def checkReceiveTimeout() { val recvtimeout = receiveTimeoutData - if (recvtimeout._1 > 0 && !mailbox.hasMessages) { + if (recvtimeout._1 > 0 && !mailbox.messageQueue.hasMessages) { recvtimeout._2.cancel() //Cancel any ongoing future //Only reschedule if desired and there are currently no more messages to be processed receiveTimeoutData = (recvtimeout._1, system.scheduler.scheduleOnce(Duration(recvtimeout._1, TimeUnit.MILLISECONDS), self, ReceiveTimeout)) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 6cdcf8817d..f31e0d830f 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -452,15 +452,19 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf def deadLetters: ActorRef = provider.deadLetters - val deadLetterMailbox: Mailbox = new Mailbox(null) { + val deadLetterQueue: MessageQueue = new MessageQueue { + def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } + def dequeue() = null + def hasMessages = false + def numberOfMessages = 0 + def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = () + } + + val deadLetterMailbox: Mailbox = new Mailbox(null, deadLetterQueue) { becomeClosed() - override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } - override def dequeue() = null - override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) } - override def systemDrain(): SystemMessage = null - override def hasMessages = false - override def hasSystemMessages = false - override def numberOfMessages = 0 + def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit = deadLetters ! DeadLetter(handle, receiver, receiver) + def systemDrain(): SystemMessage = null + def hasSystemMessages = false } def locker: Locker = provider.locker diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index a7f03db5a0..e95f54b88b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -40,47 +40,9 @@ class BalancingDispatcher( def compare(l: ActorCell, r: ActorCell) = l.self.path compareTo r.self.path })) - val messageQueue: MessageQueue = mailboxType match { - case UnboundedMailbox() ⇒ - new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { - final val queue = new ConcurrentLinkedQueue[Envelope] - } + val messageQueue: MessageQueue = mailboxType.create(None) - case BoundedMailbox(cap, timeout) ⇒ - new QueueBasedMessageQueue with BoundedMessageQueueSemantics { - final val queue = new LinkedBlockingQueue[Envelope](cap) - final val pushTimeOut = timeout - } - - case other ⇒ throw new IllegalArgumentException("Only handles BoundedMailbox and UnboundedMailbox, but you specified [" + other + "]") - } - - protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor) - - class SharingMailbox(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue { - final def enqueue(receiver: ActorRef, handle: Envelope) = messageQueue.enqueue(receiver, handle) - - final def dequeue(): Envelope = messageQueue.dequeue() - - final def numberOfMessages: Int = messageQueue.numberOfMessages - - final def hasMessages: Boolean = messageQueue.hasMessages - - override def cleanUp(): Unit = { - //Don't call the original implementation of this since it scraps all messages, and we don't want to do that - if (hasSystemMessages) { - val dlq = actor.systemImpl.deadLetterMailbox - var message = systemDrain() - while (message ne null) { - // message must be “virgin” before being able to systemEnqueue again - val next = message.next - message.next = null - dlq.systemEnqueue(actor.self, message) - message = next - } - } - } - } + protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor, messageQueue) protected[akka] override def register(actor: ActorCell): Unit = { super.register(actor) @@ -111,4 +73,23 @@ class BalancingDispatcher( scheduleOne() } -} \ No newline at end of file +} + +class SharingMailbox(_actor: ActorCell, _messageQueue: MessageQueue) + extends Mailbox(_actor, _messageQueue) with DefaultSystemMessageQueue { + + override def cleanUp(): Unit = { + //Don't call the original implementation of this since it scraps all messages, and we don't want to do that + if (hasSystemMessages) { + val dlq = actor.systemImpl.deadLetterMailbox + var message = systemDrain() + while (message ne null) { + // message must be “virgin” before being able to systemEnqueue again + val next = message.next + message.next = null + dlq.systemEnqueue(actor.self, message) + message = next + } + } + } +} diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 2046f02286..46d7b249df 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -40,7 +40,7 @@ class Dispatcher( protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { val mbox = receiver.mailbox - mbox.enqueue(receiver.self, invocation) + mbox.messageQueue.enqueue(receiver.self, invocation) registerForExecution(mbox, true, false) } @@ -65,7 +65,7 @@ class Dispatcher( } } - protected[akka] def createMailbox(actor: ActorCell): Mailbox = mailboxType.create(actor) + protected[akka] def createMailbox(actor: ActorCell): Mailbox = new Mailbox(actor, mailboxType.create(Some(actor))) with DefaultSystemMessageQueue protected[akka] def shutdown: Unit = Option(executorService.getAndSet(new ExecutorServiceDelegate { diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 4c50cb5c8d..1a6515ea2d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -34,24 +34,14 @@ object Mailbox { final val debug = false } -/** - * Custom mailbox implementations are implemented by extending this class. - * E.g. - * - * class MyMailbox(owner: ActorContext) extends CustomMailbox(owner) - * with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { - * val queue = new ConcurrentLinkedQueue[Envelope]() - * } - * - */ -abstract class CustomMailbox(val actorContext: ActorContext) extends Mailbox(actorContext.asInstanceOf[ActorCell]) - /** * Mailbox and InternalMailbox is separated in two classes because ActorCell is needed for implementation, * but can't be exposed to user defined mailbox subclasses. * */ -private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue with SystemMessageQueue with Runnable { +private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: MessageQueue) + extends SystemMessageQueue with Runnable { + import Mailbox._ @volatile @@ -152,7 +142,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue Unsafe.instance.compareAndSwapObject(this, AbstractMailbox.systemMessageOffset, _old, _new) final def canBeScheduledForExecution(hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = status match { - case Open | Scheduled ⇒ hasMessageHint || hasSystemMessageHint || hasSystemMessages || hasMessages + case Open | Scheduled ⇒ hasMessageHint || hasSystemMessageHint || hasSystemMessages || messageQueue.hasMessages case Closed ⇒ false case _ ⇒ hasSystemMessageHint || hasSystemMessages } @@ -176,7 +166,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue left: Int = java.lang.Math.max(dispatcher.throughput, 1), deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined == true) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0L): Unit = if (shouldProcessMessage) { - val next = dequeue() + val next = messageQueue.dequeue() if (next ne null) { if (Mailbox.debug) println(actor.self + " processing message " + next) actor invoke next @@ -216,25 +206,20 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue */ protected[dispatch] def cleanUp(): Unit = if (actor ne null) { // actor is null for the deadLetterMailbox - val dlq = actor.systemImpl.deadLetterMailbox + val dlm = actor.systemImpl.deadLetterMailbox if (hasSystemMessages) { var message = systemDrain() while (message ne null) { // message must be “virgin” before being able to systemEnqueue again val next = message.next message.next = null - dlq.systemEnqueue(actor.self, message) + dlm.systemEnqueue(actor.self, message) message = next } } - if (hasMessages) { - var envelope = dequeue - while (envelope ne null) { - dlq.enqueue(actor.self, envelope) - envelope = dequeue - } - } + if (messageQueue ne null) // needed for CallingThreadDispatcher, which never calls Mailbox.run() + messageQueue.cleanUp(actor, actor.systemImpl.deadLetterQueue) } } @@ -260,9 +245,20 @@ trait MessageQueue { * Indicates whether this queue is non-empty. */ def hasMessages: Boolean + + /** + * Called when the mailbox this queue belongs to is disposed of. Normally it + * is expected to transfer all remaining messages into the dead letter queue + * which is passed in. The owner of this MessageQueue is passed in if + * available (e.g. for creating DeadLetters()), “/deadletters” otherwise. + */ + def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit } -trait SystemMessageQueue { +/** + * Internal mailbox implementation detail. + */ +private[akka] trait SystemMessageQueue { /** * Enqueue a new system message, e.g. by prepending atomically as new head of a single-linked list. */ @@ -276,7 +272,10 @@ trait SystemMessageQueue { def hasSystemMessages: Boolean } -trait DefaultSystemMessageQueue { self: Mailbox ⇒ +/** + * Internal mailbox implementation detail. + */ +private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒ @tailrec final def systemEnqueue(receiver: ActorRef, message: SystemMessage): Unit = { @@ -329,21 +328,30 @@ trait QueueBasedMessageQueue extends MessageQueue { def queue: Queue[Envelope] def numberOfMessages = queue.size def hasMessages = !queue.isEmpty + def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = { + if (hasMessages) { + var envelope = dequeue + while (envelope ne null) { + deadLetters.enqueue(owner.self, envelope) + envelope = dequeue + } + } + } } /** * Mailbox configuration. */ trait MailboxType { - def create(receiver: ActorContext): Mailbox + def create(owner: Option[ActorContext]): MessageQueue } /** * It's a case class for Java (new UnboundedMailbox) */ case class UnboundedMailbox() extends MailboxType { - final override def create(receiver: ActorContext): Mailbox = - new Mailbox(receiver.asInstanceOf[ActorCell]) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { + final override def create(owner: Option[ActorContext]): MessageQueue = + new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final val queue = new ConcurrentLinkedQueue[Envelope]() } } @@ -353,16 +361,16 @@ 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") - final override def create(receiver: ActorContext): Mailbox = - new Mailbox(receiver.asInstanceOf[ActorCell]) with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue { + final override def create(owner: Option[ActorContext]): MessageQueue = + new QueueBasedMessageQueue with BoundedMessageQueueSemantics { final val queue = new LinkedBlockingQueue[Envelope](capacity) final val pushTimeOut = BoundedMailbox.this.pushTimeOut } } case class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType { - final override def create(receiver: ActorContext): Mailbox = - new Mailbox(receiver.asInstanceOf[ActorCell]) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { + final override def create(owner: Option[ActorContext]): MessageQueue = + new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final val queue = new PriorityBlockingQueue[Envelope](11, cmp) } } @@ -372,8 +380,8 @@ 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") - final override def create(receiver: ActorContext): Mailbox = - new Mailbox(receiver.asInstanceOf[ActorCell]) with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue { + final override def create(owner: Option[ActorContext]): MessageQueue = + new QueueBasedMessageQueue with BoundedMessageQueueSemantics { final val queue = new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut } diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 57847f3553..e77d6ac469 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -773,7 +773,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ * routers based on mailbox and actor internal state. */ protected def hasMessages(a: ActorRef): Boolean = a match { - case x: LocalActorRef ⇒ x.underlying.mailbox.hasMessages + case x: LocalActorRef ⇒ x.underlying.mailbox.messageQueue.hasMessages case _ ⇒ false } @@ -797,7 +797,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ * routers based on mailbox and actor internal state. */ protected def numberOfMessages(a: ActorRef): Int = a match { - case x: LocalActorRef ⇒ x.underlying.mailbox.numberOfMessages + case x: LocalActorRef ⇒ x.underlying.mailbox.messageQueue.numberOfMessages case _ ⇒ 0 } @@ -1249,9 +1249,9 @@ case class DefaultResizer( case a: LocalActorRef ⇒ val cell = a.underlying pressureThreshold match { - case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.hasMessages + case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.messageQueue.hasMessages case i if i < 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null - case threshold ⇒ cell.mailbox.numberOfMessages >= threshold + case threshold ⇒ cell.mailbox.messageQueue.numberOfMessages >= threshold } case x ⇒ false diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala index 489d97d176..68aaa6067e 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala @@ -12,11 +12,16 @@ import akka.event.Logging import akka.actor.ActorRef import akka.dispatch.MailboxType import com.typesafe.config.Config +import akka.config.ConfigurationException +import akka.dispatch.MessageQueue class BeanstalkBasedMailboxException(message: String) extends AkkaException(message) {} class BeanstalkBasedMailboxType(config: Config) extends MailboxType { - override def create(owner: ActorContext) = new BeanstalkBasedMailbox(owner) + override def create(owner: Option[ActorContext]): MessageQueue = owner match { + case Some(o) ⇒ new BeanstalkBasedMailbox(o) + case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") + } } /** @@ -110,4 +115,6 @@ class BeanstalkBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) private def reconnect(name: String): ThreadLocal[Client] = { new ThreadLocal[Client] { override def initialValue: Client = connect(name) } } + + def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = () } diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala index ccdbdc4145..70368e2f40 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala @@ -6,15 +6,19 @@ package akka.actor.mailbox import org.apache.commons.io.FileUtils import akka.actor.ActorContext -import akka.dispatch.Envelope +import akka.dispatch.{ Envelope, MessageQueue } import akka.event.Logging import akka.actor.ActorRef import akka.dispatch.MailboxType import com.typesafe.config.Config import akka.util.NonFatal +import akka.config.ConfigurationException class FileBasedMailboxType(config: Config) extends MailboxType { - override def create(owner: ActorContext) = new FileBasedMailbox(owner) + override def create(owner: Option[ActorContext]): MessageQueue = owner match { + case Some(o) ⇒ new FileBasedMailbox(o) + case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") + } } class FileBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { @@ -72,4 +76,6 @@ class FileBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with case NonFatal(_) ⇒ false } + def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = () + } diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala index 69f7fb50c1..92072c8cf3 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -4,7 +4,7 @@ package akka.actor.mailbox import akka.actor.{ ActorContext, ActorRef, ExtendedActorSystem } -import akka.dispatch.{ Envelope, DefaultSystemMessageQueue, CustomMailbox } +import akka.dispatch.{ Envelope, MessageQueue } import akka.remote.MessageSerializer import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol } @@ -12,7 +12,7 @@ private[akka] object DurableExecutableMailboxConfig { val Name = "[\\.\\/\\$\\s]".r } -abstract class DurableMailbox(val owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue { +abstract class DurableMailbox(val owner: ActorContext) extends MessageQueue { import DurableExecutableMailboxConfig._ def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem] diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala index d17a1221a8..ac969695bb 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala @@ -14,11 +14,16 @@ import akka.dispatch.{ Await, Promise, Envelope } import java.util.concurrent.TimeoutException import akka.dispatch.MailboxType import com.typesafe.config.Config +import akka.config.ConfigurationException +import akka.dispatch.MessageQueue class MongoBasedMailboxException(message: String) extends AkkaException(message) class MongoBasedMailboxType(config: Config) extends MailboxType { - override def create(owner: ActorContext) = new MongoBasedMailbox(owner) + override def create(owner: Option[ActorContext]): MessageQueue = owner match { + case Some(o) ⇒ new MongoBasedMailbox(o) + case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") + } } /** @@ -37,6 +42,8 @@ class MongoBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) { implicit val mailboxBSONSer = new BSONSerializableMailbox(system) implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate! + private val dispatcher = owner.dispatcher + private val settings = MongoBasedMailboxExtension(owner.system) val log = Logging(system, "MongoBasedMailbox") @@ -132,4 +139,6 @@ class MongoBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) { } } } + + def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = () } diff --git a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala index b6cf3febc6..9a01d462f8 100644 --- a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala @@ -12,11 +12,16 @@ import akka.actor.ActorRef import akka.dispatch.MailboxType import com.typesafe.config.Config import akka.util.NonFatal +import akka.config.ConfigurationException +import akka.dispatch.MessageQueue class RedisBasedMailboxException(message: String) extends AkkaException(message) class RedisBasedMailboxType(config: Config) extends MailboxType { - override def create(owner: ActorContext) = new RedisBasedMailbox(owner) + override def create(owner: Option[ActorContext]): MessageQueue = owner match { + case Some(o) ⇒ new RedisBasedMailbox(o) + case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") + } } class RedisBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { @@ -80,5 +85,7 @@ class RedisBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) wit throw error } } + + def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = () } diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala index 90fd381af1..1434f6c1d9 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala @@ -13,11 +13,16 @@ import akka.actor.ActorRef import akka.dispatch.MailboxType import com.typesafe.config.Config import akka.util.NonFatal +import akka.config.ConfigurationException +import akka.dispatch.MessageQueue class ZooKeeperBasedMailboxException(message: String) extends AkkaException(message) class ZooKeeperBasedMailboxType(config: Config) extends MailboxType { - override def create(owner: ActorContext) = new ZooKeeperBasedMailbox(owner) + override def create(owner: Option[ActorContext]): MessageQueue = owner match { + case Some(o) ⇒ new ZooKeeperBasedMailbox(o) + case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") + } } class ZooKeeperBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { @@ -62,7 +67,7 @@ class ZooKeeperBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) case e: Exception ⇒ false } - override def cleanUp() { + def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = { try { zkClient.close() } catch { diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 8b2d15a079..8cbe954aac 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -9,10 +9,12 @@ import java.util.LinkedList import scala.annotation.tailrec import com.typesafe.config.Config import akka.actor.{ ActorInitializationException, ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorRef, ActorCell } -import akka.dispatch.{ TaskInvocation, SystemMessage, Suspend, Resume, MessageDispatcherConfigurator, MessageDispatcher, Mailbox, Envelope, DispatcherPrerequisites, DefaultSystemMessageQueue } +import akka.dispatch.{ MailboxType, TaskInvocation, SystemMessage, Suspend, Resume, MessageDispatcherConfigurator, MessageDispatcher, Mailbox, Envelope, DispatcherPrerequisites, DefaultSystemMessageQueue } import akka.util.duration.intToDurationInt import akka.util.{ Switch, Duration } import akka.util.NonFatal +import akka.actor.ActorContext +import akka.dispatch.MessageQueue /* * Locking rules: @@ -75,9 +77,12 @@ private[testkit] class CallingThreadDispatcherQueues extends Extension { val q = ref.get if (q ne null) && (q ne own) } { - while (q.peek ne null) { + val owner = mbox.actor.self + var msg = q.q.dequeue() + while (msg ne null) { // this is safe because this method is only ever called while holding the suspendSwitch monitor - own.push(q.pop) + own.q.enqueue(owner, msg) + msg = q.q.dequeue() } } } @@ -115,6 +120,7 @@ object CallingThreadDispatcher { */ class CallingThreadDispatcher( _prerequisites: DispatcherPrerequisites, + val mailboxType: MailboxType, val name: String = "calling-thread") extends MessageDispatcher(_prerequisites) { import CallingThreadDispatcher._ @@ -122,7 +128,7 @@ class CallingThreadDispatcher( override def id: String = Id - protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(actor) + protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(actor, mailboxType) protected[akka] override def shutdown() {} @@ -183,17 +189,17 @@ class CallingThreadDispatcher( case mbox: CallingThreadMailbox ⇒ val queue = mbox.queue val execute = mbox.suspendSwitch.fold { - queue.push(handle) + queue.q.enqueue(receiver.self, handle) false } { - queue.push(handle) + queue.q.enqueue(receiver.self, handle) if (!queue.isActive) { queue.enter true } else false } if (execute) runQueue(mbox, queue) - case m ⇒ m.enqueue(receiver.self, handle) + case m ⇒ m.messageQueue.enqueue(receiver.self, handle) } } @@ -219,7 +225,7 @@ class CallingThreadDispatcher( queue.leave null } { - val ret = if (mbox.isClosed) null else queue.pop + val ret = if (mbox.isClosed) null else queue.q.dequeue() if (ret eq null) queue.leave ret } @@ -261,19 +267,13 @@ class CallingThreadDispatcher( class CallingThreadDispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends MessageDispatcherConfigurator(config, prerequisites) { - private val instance = new CallingThreadDispatcher(prerequisites) + + private val instance = new CallingThreadDispatcher(prerequisites, mailboxType()) override def dispatcher(): MessageDispatcher = instance } -class NestingQueue { - private var q = new LinkedList[Envelope]() - def size = q.size - def isEmpty = q.isEmpty - def push(handle: Envelope) { q.offer(handle) } - def peek = q.peek - def pop = q.poll - +class NestingQueue(val q: MessageQueue) { @volatile private var active = false def enter { if (active) sys.error("already active") else active = true } @@ -281,11 +281,11 @@ class NestingQueue { def isActive = active } -class CallingThreadMailbox(_receiver: ActorCell) extends Mailbox(_receiver) with DefaultSystemMessageQueue { +class CallingThreadMailbox(_receiver: ActorCell, val mailboxType: MailboxType) extends Mailbox(_receiver, null) with DefaultSystemMessageQueue { private val q = new ThreadLocal[NestingQueue]() { override def initialValue = { - val queue = new NestingQueue + val queue = new NestingQueue(mailboxType.create(Some(actor))) CallingThreadDispatcherQueues(actor.system).registerQueue(CallingThreadMailbox.this, queue) queue } @@ -296,11 +296,6 @@ class CallingThreadMailbox(_receiver: ActorCell) extends Mailbox(_receiver) with val ctdLock = new ReentrantLock val suspendSwitch = new Switch - override def enqueue(receiver: ActorRef, msg: Envelope) {} - override def dequeue() = null - override def hasMessages = queue.isEmpty - override def numberOfMessages = queue.size - override def cleanUp(): Unit = { /* * This is called from dispatcher.unregister, i.e. under this.lock. If @@ -308,8 +303,10 @@ class CallingThreadMailbox(_receiver: ActorCell) extends Mailbox(_receiver) with * the gather operation, tough luck: no guaranteed delivery to deadLetters. */ suspendSwitch.locked { - CallingThreadDispatcherQueues(actor.system).gatherFromAllOtherQueues(this, queue) + val q = queue + CallingThreadDispatcherQueues(actor.system).gatherFromAllOtherQueues(this, q) super.cleanUp() + q.q.cleanUp(actor, actor.systemImpl.deadLetterQueue) } } } From 95b1590cd9aef0d4ca3513c9879ac37ded376130 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 19 Feb 2012 18:25:22 +0100 Subject: [PATCH 10/46] Adding support for DeathWatcher for TypedActors --- .../src/main/scala/akka/actor/TypedActor.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index da375a5f21..07c494fec9 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -283,6 +283,9 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi TypedActor.selfReference set null TypedActor.currentContext set null } + + case t: Terminated if me.isInstanceOf[DeathWatcher] => + me.asInstanceOf[DeathWatcher].onTermination(t.actor) } } @@ -297,6 +300,18 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi def supervisorStrategy(): SupervisorStrategy } + /** + * Mix this into your TypedActor to be able to intercept Terminated messages + */ + trait DeathWatcher { + + /** + * User overridable callback to intercept Terminated messages. + * @param actor + */ + def onTermination(actor: ActorRef): Unit + } + /** * Mix this into your TypedActor to be able to hook into its lifecycle */ From ff294986691dfe69835b560d07b091797eb936a9 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 19 Feb 2012 22:28:54 +0100 Subject: [PATCH 11/46] Adding support for Receiver and yanking out DeathWatcher --- .../main/scala/akka/actor/TypedActor.scala | 71 +++++++++---------- 1 file changed, 35 insertions(+), 36 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 07c494fec9..3d1fd7bed3 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -256,36 +256,40 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi case _ ⇒ super.postRestart(reason) } - def receive = { - case m: MethodCall ⇒ - TypedActor.selfReference set proxyVar.get - TypedActor.currentContext set context - try { - if (m.isOneWay) m(me) - else { - try { - if (m.returnsFuture_?) { - val s = sender - m(me).asInstanceOf[Future[Any]] onComplete { - case Left(f) ⇒ s ! Status.Failure(f) - case Right(r) ⇒ s ! r - } - } else { - sender ! m(me) - } - } catch { - case NonFatal(e) ⇒ - sender ! Status.Failure(e) - throw e - } - } - } finally { - TypedActor.selfReference set null - TypedActor.currentContext set null - } + protected def withContext[T](unitOfWork: ⇒ T): T = { + TypedActor.selfReference set proxyVar.get + TypedActor.currentContext set context + try unitOfWork finally { + TypedActor.selfReference set null + TypedActor.currentContext set null + } + } - case t: Terminated if me.isInstanceOf[DeathWatcher] => - me.asInstanceOf[DeathWatcher].onTermination(t.actor) + def receive = { + case m: MethodCall ⇒ withContext { + if (m.isOneWay) m(me) + else { + try { + if (m.returnsFuture_?) { + val s = sender + m(me).asInstanceOf[Future[Any]] onComplete { + case Left(f) ⇒ s ! Status.Failure(f) + case Right(r) ⇒ s ! r + } + } else { + sender ! m(me) + } + } catch { + case NonFatal(e) ⇒ + sender ! Status.Failure(e) + throw e + } + } + } + + case msg if me.isInstanceOf[Receiver] ⇒ withContext { + me.asInstanceOf[Receiver].onReceive(msg, sender) + } } } @@ -303,13 +307,8 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi /** * Mix this into your TypedActor to be able to intercept Terminated messages */ - trait DeathWatcher { - - /** - * User overridable callback to intercept Terminated messages. - * @param actor - */ - def onTermination(actor: ActorRef): Unit + trait Receiver { + def onReceive(message: Any, sender: ActorRef): Unit } /** From ccae535c798e64aa34ddbaee15aad98edea39fa4 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 20 Feb 2012 00:44:56 +0100 Subject: [PATCH 12/46] Adding testcase for Receiver --- .../scala/akka/actor/TypedActorSpec.scala | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index 5a9fab6c63..26f510d08a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -12,13 +12,13 @@ import java.util.concurrent.atomic.AtomicReference import annotation.tailrec import akka.testkit.{ EventFilter, filterEvents, AkkaSpec } import akka.serialization.SerializationExtension -import akka.actor.TypedActor.{ PostRestart, PreRestart, PostStop, PreStart } import java.util.concurrent.{ TimeUnit, CountDownLatch } import akka.japi.{ Creator, Option ⇒ JOption } import akka.testkit.DefaultTimeout import akka.dispatch.{ Await, Dispatchers, Future, Promise } import akka.pattern.ask import akka.serialization.JavaSerializer +import akka.actor.TypedActor._ object TypedActorSpec { @@ -160,7 +160,7 @@ object TypedActorSpec { def crash(): Unit } - class LifeCyclesImpl(val latch: CountDownLatch) extends PreStart with PostStop with PreRestart with PostRestart with LifeCycles { + class LifeCyclesImpl(val latch: CountDownLatch) extends PreStart with PostStop with PreRestart with PostRestart with LifeCycles with Receiver { override def crash(): Unit = throw new IllegalStateException("Crash!") @@ -171,6 +171,12 @@ object TypedActorSpec { override def preRestart(reason: Throwable, message: Option[Any]): Unit = for (i ← 1 to 5) latch.countDown() override def postRestart(reason: Throwable): Unit = for (i ← 1 to 7) latch.countDown() + + override def onReceive(msg: Any, sender: ActorRef): Unit = { + msg match { + case "pigdog" ⇒ sender ! "dogpig" + } + } } } @@ -415,6 +421,16 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) EventFilter[IllegalStateException]("Crash!", occurrences = 1) intercept { t.crash() } + + //Sneak in a check for the Receiver override + val ref = ta getActorRefFor t + + ref.tell("pigdog", testActor) + + expectMsg(timeout.duration, "dogpig") + + //Done with that now + ta.poisonPill(t) latch.await(10, TimeUnit.SECONDS) must be === true } From 69195dae1f5a747d85f5d8ac52ed09058088d303 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 20 Feb 2012 11:06:12 +0100 Subject: [PATCH 13/46] Stab at rectifying the TypedActor Serialization issue --- akka-actor/src/main/scala/akka/actor/TypedActor.scala | 4 ++-- .../src/main/scala/akka/serialization/Serializer.scala | 6 ++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index da375a5f21..85306c7203 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -128,11 +128,11 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi case null ⇒ SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, null) case ps if ps.length == 0 ⇒ SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, Array()) case ps ⇒ + val serialization = SerializationExtension(akka.serialization.JavaSerializer.currentSystem.value) val serializedParameters = Array.ofDim[(Int, Class[_], Array[Byte])](ps.length) for (i ← 0 until ps.length) { val p = ps(i) - val system = akka.serialization.JavaSerializer.currentSystem.value - val s = SerializationExtension(system).findSerializerFor(p) + val s = serialization.findSerializerFor(p) val m = if (s.includeManifest) p.getClass else null serializedParameters(i) = (s.identifier, m, s toBinary parameters(i)) //Mutable for the sake of sanity } diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index 2721432460..5696201f62 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -111,16 +111,14 @@ class JavaSerializer(val system: ExtendedActorSystem) extends Serializer { def toBinary(o: AnyRef): Array[Byte] = { val bos = new ByteArrayOutputStream val out = new ObjectOutputStream(bos) - out.writeObject(o) + JavaSerializer.currentSystem.withValue(system) { out.writeObject(o) } out.close() bos.toByteArray } def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = { val in = new ClassLoaderObjectInputStream(system.dynamicAccess.classLoader, new ByteArrayInputStream(bytes)) - val obj = JavaSerializer.currentSystem.withValue(system) { - in.readObject - } + val obj = JavaSerializer.currentSystem.withValue(system) { in.readObject } in.close() obj } From 46fb8c72cd8f15cd15cb9e8e0e59cc356f95e753 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 20 Feb 2012 14:20:13 +0100 Subject: [PATCH 14/46] Adding more detailed logs to remote messages --- akka-remote/src/main/scala/akka/remote/RemoteTransport.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index 256451bc0a..e3c09b5f5d 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -300,9 +300,9 @@ trait RemoteMarshallingOps { case AddressExtractor(address) if address == provider.transport.address ⇒ // if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed) r.!(remoteMessage.payload)(remoteMessage.sender) - case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r) + case r ⇒ log.error("dropping message {} for non-local recipient {} at {} local is {}", remoteMessage.payload, r, address, provider.transport.address) } - case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r) + case r ⇒ log.error("dropping message {} for non-local recipient {} of type {}", remoteMessage.payload, r, if (r ne null) r.getClass else "null") } } } From da00a37f64a9db61573671d01e5c1a1a39f4999e Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 20 Feb 2012 15:14:23 +0100 Subject: [PATCH 15/46] Removing unused RemoteServerWriteFailed event, and added docs describing the possibility to listen to remote events --- akka-docs/java/remoting.rst | 43 +++++++++++++++++++ akka-docs/scala/remoting.rst | 42 ++++++++++++++++++ .../scala/akka/remote/RemoteTransport.scala | 13 ------ 3 files changed, 85 insertions(+), 13 deletions(-) diff --git a/akka-docs/java/remoting.rst b/akka-docs/java/remoting.rst index 9e0eb4eec9..941dc4eb93 100644 --- a/akka-docs/java/remoting.rst +++ b/akka-docs/java/remoting.rst @@ -249,5 +249,48 @@ Observe how the name of the server actor matches the deployment given in the configuration file, which will transparently delegate the actor creation to the remote node. +Remote Events +------------- +It is possible to listen to events that occur in Akka Remote, and to subscribe/unsubscribe to there events, +you simply register as listener to the below described types in on the ``ActorSystem.eventStream``. + +.. note:: + To subscribe to any outbound-related events, subscribe to ``RemoteClientLifeCycleEvent`` + To subscribe to any inbound-related events, subscribe to ``RemoteServerLifeCycleEvent`` + To subscribe to any remote events, subscribe to ``RemoteLifeCycleEvent`` + +To intercept when an outbound connection is disconnected, you listen to ``RemoteClientDisconnected`` which +holds the transport used (RemoteTransport) and the outbound address that was disconnected (Address). + +To intercept when an outbound connection is connected, you listen to ``RemoteClientConnected`` which +holds the transport used (RemoteTransport) and the outbound address that was connected to (Address). + +To intercept when an outbound client is started you listen to ``RemoteClientStarted`` +which holds the transport used (RemoteTransport) and the outbound address that it is connected to (Address). + +To intercept when an outbound client is shut down you listen to ``RemoteClientShutdown`` +which holds the transport used (RemoteTransport) and the outbound address that it was connected to (Address). + +To intercept when an outbound message cannot be sent, you listen to ``RemoteClientWriteFailed`` which holds +the payload that was not written (AnyRef), the cause of the failed send (Throwable), +the transport used (RemoteTransport) and the outbound address that was the destination (Address). + +For general outbound-related errors, that do not classify as any of the others, you can listen to ``RemoteClientError``, +which holds the cause (Throwable), the transport used (RemoteTransport) and the outbound address (Address). + +To intercept when an inbound server is started (typically only once) you listen to ``RemoteServerStarted`` +which holds the transport that it will use (RemoteTransport). + +To intercept when an inbound server is shut down (typically only once) you listen to ``RemoteServerShutdown`` +which holds the transport that it used (RemoteTransport). + +To intercept when an inbound connection has been established you listen to ``RemoteServerClientConnected`` +which holds the transport used (RemoteTransport) and optionally the address that connected (Option
). + +To intercept when an inbound connection has been disconnected you listen to ``RemoteServerClientDisconnected`` +which holds the transport used (RemoteTransport) and optionally the address that disconnected (Option
). + +To intercept when an inbound remote client has been closed you listen to ``RemoteServerClientClosed`` +which holds the transport used (RemoteTransport) and optionally the address of the remote client that was closed (Option
). diff --git a/akka-docs/scala/remoting.rst b/akka-docs/scala/remoting.rst index f874e15a1b..5e613dd7e4 100644 --- a/akka-docs/scala/remoting.rst +++ b/akka-docs/scala/remoting.rst @@ -261,5 +261,47 @@ Observe how the name of the server actor matches the deployment given in the configuration file, which will transparently delegate the actor creation to the remote node. +Remote Events +------------- +It is possible to listen to events that occur in Akka Remote, and to subscribe/unsubscribe to there events, +you simply register as listener to the below described types in on the ``ActorSystem.eventStream``. +.. note:: + To subscribe to any outbound-related events, subscribe to ``RemoteClientLifeCycleEvent`` + To subscribe to any inbound-related events, subscribe to ``RemoteServerLifeCycleEvent`` + To subscribe to any remote events, subscribe to ``RemoteLifeCycleEvent`` + +To intercept when an outbound connection is disconnected, you listen to ``RemoteClientDisconnected`` which +holds the transport used (RemoteTransport) and the outbound address that was disconnected (Address). + +To intercept when an outbound connection is connected, you listen to ``RemoteClientConnected`` which +holds the transport used (RemoteTransport) and the outbound address that was connected to (Address). + +To intercept when an outbound client is started you listen to ``RemoteClientStarted`` +which holds the transport used (RemoteTransport) and the outbound address that it is connected to (Address). + +To intercept when an outbound client is shut down you listen to ``RemoteClientShutdown`` +which holds the transport used (RemoteTransport) and the outbound address that it was connected to (Address). + +To intercept when an outbound message cannot be sent, you listen to ``RemoteClientWriteFailed`` which holds +the payload that was not written (AnyRef), the cause of the failed send (Throwable), +the transport used (RemoteTransport) and the outbound address that was the destination (Address). + +For general outbound-related errors, that do not classify as any of the others, you can listen to ``RemoteClientError``, +which holds the cause (Throwable), the transport used (RemoteTransport) and the outbound address (Address). + +To intercept when an inbound server is started (typically only once) you listen to ``RemoteServerStarted`` +which holds the transport that it will use (RemoteTransport). + +To intercept when an inbound server is shut down (typically only once) you listen to ``RemoteServerShutdown`` +which holds the transport that it used (RemoteTransport). + +To intercept when an inbound connection has been established you listen to ``RemoteServerClientConnected`` +which holds the transport used (RemoteTransport) and optionally the address that connected (Option[Address]). + +To intercept when an inbound connection has been disconnected you listen to ``RemoteServerClientDisconnected`` +which holds the transport used (RemoteTransport) and optionally the address that disconnected (Option[Address]). + +To intercept when an inbound remote client has been closed you listen to ``RemoteServerClientClosed`` +which holds the transport used (RemoteTransport) and optionally the address of the remote client that was closed (Option[Address]). diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index e3c09b5f5d..2937ac233b 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -133,19 +133,6 @@ case class RemoteServerClientClosed( ": Client[" + clientAddress.getOrElse("no address") + "]" } -case class RemoteServerWriteFailed( - @BeanProperty request: AnyRef, - @BeanProperty cause: Throwable, - @BeanProperty remote: RemoteTransport, - @BeanProperty remoteAddress: Option[Address]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.WarningLevel - override def toString = - "RemoteServerWriteFailed@" + remote + - ": ClientAddress[" + remoteAddress + - "] MessageClass[" + (if (request ne null) request.getClass.getName else "no message") + - "] Error[" + AkkaException.toStringWithStackTrace(cause) + "]" -} - /** * Thrown for example when trying to send a message using a RemoteClient that is either not started or shut down. */ From 587950b863d23367fbe623bd0d12469e53046adf Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 20 Feb 2012 15:15:20 +0100 Subject: [PATCH 16/46] =?UTF-8?q?fix=20BSONSerializer=E2=80=99s=20use=20of?= =?UTF-8?q?=20paths?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/scala/akka/actor/mailbox/BSONSerialization.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala index 5aa314eb55..8da4db9376 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala @@ -26,7 +26,7 @@ class BSONSerializableMailbox(system: ExtendedActorSystem) extends SerializableB val b = Map.newBuilder[String, Any] b += "_id" -> msg._id b += "ownerPath" -> msg.ownerPath - b += "senderPath" -> msg.sender.path + b += "senderPath" -> msg.sender.path.toString /** * TODO - Figure out a way for custom serialization of the message instance From 0f685bdaae5effda07358fa8bc0579072baa3527 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 20 Feb 2012 15:43:17 +0100 Subject: [PATCH 17/46] Adding example on how to roll your own ExecutionContext --- .../akka/docs/future/FutureDocTestBase.java | 44 ++++++++++++++----- akka-docs/java/futures.rst | 3 ++ .../code/akka/docs/future/FutureDocSpec.scala | 20 +++++++-- akka-docs/scala/futures.rst | 3 ++ 4 files changed, 55 insertions(+), 15 deletions(-) diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index 5a6d2e5acd..42dc0e2a12 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -5,8 +5,6 @@ package akka.docs.future; //#imports1 import akka.dispatch.*; -import akka.japi.Procedure; -import akka.japi.Procedure2; import akka.util.Timeout; //#imports1 @@ -41,9 +39,17 @@ import static akka.dispatch.Futures.reduce; //#imports6 +//#imports7 +import akka.dispatch.ExecutionContexts; +import akka.dispatch.ExecutionContextExecutorService; + +//#imports7 + import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import org.junit.After; import org.junit.Before; @@ -73,6 +79,20 @@ public class FutureDocTestBase { system.shutdown(); } + @Test public void useCustomExecutionContext() throws Exception { + ExecutorService yourExecutorServiceGoesHere = Executors.newSingleThreadExecutor(); + //#diy-execution-context + ExecutionContextExecutorService ec = + ExecutionContexts.fromExecutorService(yourExecutorServiceGoesHere); + + //Use ec with your Futures + Future f1 = Futures.successful("foo", ec); + + // Then you shut the ec down somewhere at the end of your program/application. + ec.shutdown(); + //#diy-execution-context + } + @Test public void useBlockingFromActor() throws Exception { ActorRef actor = system.actorOf(new Props(MyActor.class)); @@ -324,10 +344,10 @@ public class FutureDocTestBase { public void useAndThen() { //#and-then Future future1 = Futures.successful("value", system.dispatcher()).andThen(new OnComplete() { - public void onComplete(Throwable failure, String result) { - if (failure != null) - sendToIssueTracker(failure); - } + public void onComplete(Throwable failure, String result) { + if (failure != null) + sendToIssueTracker(failure); + } }).andThen(new OnComplete() { public void onComplete(Throwable failure, String result) { if (result != null) @@ -416,13 +436,13 @@ public class FutureDocTestBase { Future future = Futures.successful("foo", system.dispatcher()); //#onComplete future.onComplete(new OnComplete() { - public void onComplete(Throwable failure, String result) { - if (failure != null) { - //We got a failure, handle it here - } else { - // We got a result, do something with it + public void onComplete(Throwable failure, String result) { + if (failure != null) { + //We got a failure, handle it here + } else { + // We got a result, do something with it + } } - } }); //#onComplete } diff --git a/akka-docs/java/futures.rst b/akka-docs/java/futures.rst index c738167a50..0f66f7fcfa 100644 --- a/akka-docs/java/futures.rst +++ b/akka-docs/java/futures.rst @@ -22,6 +22,9 @@ which is very similar to a ``java.util.concurrent.Executor``. if you have an ``A it will use its default dispatcher as the ``ExecutionContext``, or you can use the factory methods provided by the ``ExecutionContexts`` class to wrap ``Executors`` and ``ExecutorServices``, or even create your own. +.. includecode:: code/akka/docs/future/FutureDocTestBase.java + :include: imports1,imports7,diy-execution-context + Use with Actors --------------- diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala index 3d1ca946a7..cee2eaeef8 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala @@ -9,12 +9,10 @@ import akka.testkit._ import akka.actor.Actor import akka.actor.Props import akka.actor.Status.Failure -import akka.dispatch.Future -import akka.dispatch.Await import akka.util.Timeout import akka.util.duration._ -import akka.dispatch.Promise import java.lang.IllegalStateException +import akka.dispatch.{ ExecutionContext, Future, Await, Promise } object FutureDocSpec { @@ -41,6 +39,22 @@ object FutureDocSpec { class FutureDocSpec extends AkkaSpec { import FutureDocSpec._ + "demonstrate usage custom ExecutionContext" in { + val yourExecutorServiceGoesHere = java.util.concurrent.Executors.newSingleThreadExecutor() + //#diy-execution-context + import akka.dispatch.{ ExecutionContext, Promise } + + implicit val ec = ExecutionContext.fromExecutorService(yourExecutorServiceGoesHere) + + // Do stuff with your brand new shiny ExecutionContext + val f = Promise.successful("foo") + + // Then shut your ExecutionContext down at some + // appropriate place in your program/application + ec.shutdown() + //#diy-execution-context + } + "demonstrate usage of blocking from actor" in { val actor = system.actorOf(Props[MyActor]) val msg = "hello" diff --git a/akka-docs/scala/futures.rst b/akka-docs/scala/futures.rst index 03d9004512..11e514f981 100644 --- a/akka-docs/scala/futures.rst +++ b/akka-docs/scala/futures.rst @@ -22,6 +22,9 @@ which is very similar to a ``java.util.concurrent.Executor``. if you have an ``A it will use its default dispatcher as the ``ExecutionContext``, or you can use the factory methods provided by the ``ExecutionContext`` companion object to wrap ``Executors`` and ``ExecutorServices``, or even create your own. +.. includecode:: code/akka/docs/future/FutureDocSpec.scala + :include: diy-execution-context + Use With Actors --------------- From 7c43d049a9a6c749a47757374037439a1340fdbf Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 20 Feb 2012 16:25:01 +0100 Subject: [PATCH 18/46] Adding Receiver-docs --- akka-docs/java/typed-actors.rst | 8 ++++++++ akka-docs/scala/typed-actors.rst | 8 ++++++++ 2 files changed, 16 insertions(+) diff --git a/akka-docs/java/typed-actors.rst b/akka-docs/java/typed-actors.rst index d6b016e3b5..a54e5767d9 100644 --- a/akka-docs/java/typed-actors.rst +++ b/akka-docs/java/typed-actors.rst @@ -160,6 +160,14 @@ By having your Typed Actor implementation class implement ``TypedActor.Superviso you can define the strategy to use for supervising child actors, as described in :ref:`supervision` and :ref:`fault-tolerance-java`. +Receive arbitrary messages +-------------------------- + +If your implementation class of your TypedActor extends ``akka.actor.TypedActor.Receiver``, +all messages that are not ``MethodCall``s will be passed into the ``onReceive``-method. + +This allows you to react to DeathWatch ``Terminated``-messages and other typed of messages. + Lifecycle callbacks ------------------- diff --git a/akka-docs/scala/typed-actors.rst b/akka-docs/scala/typed-actors.rst index ba326b8a27..4dee51a920 100644 --- a/akka-docs/scala/typed-actors.rst +++ b/akka-docs/scala/typed-actors.rst @@ -172,6 +172,14 @@ By having your Typed Actor implementation class implement any and all of the fol You can hook into the lifecycle of your Typed Actor. +Receive arbitrary messages +-------------------------- + +If your implementation class of your TypedActor extends ``akka.actor.TypedActor.Receiver``, +all messages that are not ``MethodCall``s will be passed into the ``onReceive``-method. + +This allows you to react to DeathWatch ``Terminated``-messages and other typed of messages. + Supercharging ------------- From d6ca3c9b5f0bf5c911b2fc71765a2e12402e4a4f Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 20 Feb 2012 16:37:14 +0100 Subject: [PATCH 19/46] Clarifying the need for Future.blocking --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 80d40a5cc7..d7b4f17922 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -160,9 +160,9 @@ object Futures { /** * 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. + * an action that will take a non-trivial amount of time, perhaps by using blocking.IO or using a lot of CPU time, + * 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 @@ -316,9 +316,9 @@ object Future { /** * 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. + * an action that will take a non-trivial amount of time, perhaps by using blocking.IO or using a lot of CPU time, + * 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 From 127b2a305bf4489045f8c3fa8c95c7bdb6d1f836 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 20 Feb 2012 19:33:52 +0100 Subject: [PATCH 20/46] Switching to using a ChannelLocal to keep tabs on the inbound clients address --- .../main/scala/akka/remote/netty/Server.scala | 48 +++++++++---------- 1 file changed, 23 insertions(+), 25 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/netty/Server.scala b/akka-remote/src/main/scala/akka/remote/netty/Server.scala index 0bafb1c712..1f18b27c8c 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -10,7 +10,6 @@ import org.jboss.netty.bootstrap.ServerBootstrap import org.jboss.netty.channel.ChannelHandler.Sharable import org.jboss.netty.channel.group.ChannelGroup import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory -import org.jboss.netty.channel.{ StaticChannelPipeline, SimpleChannelUpstreamHandler, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelHandler, Channel } import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder } import org.jboss.netty.handler.execution.ExecutionHandler import akka.event.Logging @@ -19,8 +18,7 @@ import akka.remote.{ RemoteServerShutdown, RemoteServerError, RemoteServerClient import akka.actor.Address import java.net.InetAddress import akka.actor.ActorSystemImpl -import org.jboss.netty.channel.ChannelLocal -import org.jboss.netty.channel.ChannelEvent +import org.jboss.netty.channel._ class NettyRemoteServer(val netty: NettyRemoteTransport) { @@ -135,6 +133,10 @@ class RemoteServerHandler( val openChannels: ChannelGroup, val netty: NettyRemoteTransport) extends SimpleChannelUpstreamHandler { + val channelAddress = new ChannelLocal[Option[Address]](false) { + override def initialValue(channel: Channel) = None + } + import netty.settings private var addressToSet = true @@ -154,23 +156,20 @@ class RemoteServerHandler( */ override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel) - override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { - val clientAddress = getClientAddress(ctx.getChannel) - netty.notifyListeners(RemoteServerClientConnected(netty, clientAddress)) - } + // TODO might want to log or otherwise signal that a TCP connection has been established here. + override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = () override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { - val clientAddress = getClientAddress(ctx.getChannel) - netty.notifyListeners(RemoteServerClientDisconnected(netty, clientAddress)) + netty.notifyListeners(RemoteServerClientDisconnected(netty, channelAddress.get(ctx.getChannel))) } - override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = getClientAddress(ctx.getChannel) match { - case s @ Some(address) ⇒ - if (settings.UsePassiveConnections) - netty.unbindClient(address) - netty.notifyListeners(RemoteServerClientClosed(netty, s)) - case None ⇒ - netty.notifyListeners(RemoteServerClientClosed(netty, None)) + override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { + val address = channelAddress.get(ctx.getChannel) + if (address.isDefined && settings.UsePassiveConnections) + netty.unbindClient(address.get) + + netty.notifyListeners(RemoteServerClientClosed(netty, address)) + channelAddress.remove(ctx.getChannel) } override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try { @@ -181,11 +180,16 @@ class RemoteServerHandler( case remote: AkkaRemoteProtocol if remote.hasInstruction ⇒ val instruction = remote.getInstruction instruction.getCommandType match { - case CommandType.CONNECT if settings.UsePassiveConnections ⇒ + case CommandType.CONNECT ⇒ val origin = instruction.getOrigin val inbound = Address("akka", origin.getSystem, origin.getHostname, origin.getPort) - val client = new PassiveRemoteClient(event.getChannel, netty, inbound) - netty.bindClient(inbound, client) + channelAddress.set(event.getChannel, Option(inbound)) + + //If we want to reuse the inbound connections as outbound we need to get busy + if (settings.UsePassiveConnections) + netty.bindClient(inbound, new PassiveRemoteClient(event.getChannel, netty, inbound)) + + netty.notifyListeners(RemoteServerClientConnected(netty, Option(inbound))) case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed case CommandType.HEARTBEAT ⇒ //Other guy is still alive case _ ⇒ //Unknown command @@ -200,11 +204,5 @@ class RemoteServerHandler( netty.notifyListeners(RemoteServerError(event.getCause, netty)) event.getChannel.close() } - - private def getClientAddress(c: Channel): Option[Address] = - c.getRemoteAddress match { - case inet: InetSocketAddress ⇒ Some(Address("akka", "unknown(yet)", inet.getAddress.toString, inet.getPort)) - case _ ⇒ None - } } From a5664a48e43e38150cba1b0b37563a36a8cf56e9 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 20 Feb 2012 19:35:15 +0100 Subject: [PATCH 21/46] add ActorSystem.isTerminated, see #1849 --- .../scala/akka/actor/ActorSystemSpec.scala | 8 ++++++++ .../main/scala/akka/actor/ActorSystem.scala | 18 ++++++++++++++++-- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index ea8ce4c7d0..5a5a377efc 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -74,6 +74,14 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt callbackWasRun must be(true) } + "return isTerminated status correctly" in { + val system = ActorSystem() + system.isTerminated must be(false) + system.shutdown() + system.awaitTermination() + system.isTerminated must be(true) + } + "throw RejectedExecutionException when shutdown" in { val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf) system2.shutdown() diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 6cdcf8817d..63bb2caa54 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -182,12 +182,12 @@ abstract class ActorSystem extends ActorRefFactory { /** * Start-up time in milliseconds since the epoch. */ - val startTime = System.currentTimeMillis + val startTime: Long = System.currentTimeMillis /** * Up-time of this actor system in seconds. */ - def uptime = (System.currentTimeMillis - startTime) / 1000 + def uptime: Long = (System.currentTimeMillis - startTime) / 1000 /** * Main event bus of this actor system, used for example for logging. @@ -253,6 +253,8 @@ abstract class ActorSystem extends ActorRefFactory { * Block current thread until the system has been shutdown, or the specified * timeout has elapsed. This will block until after all on termination * callbacks have been run. + * + * @throws TimeoutException in case of timeout */ def awaitTermination(timeout: Duration): Unit @@ -270,6 +272,15 @@ abstract class ActorSystem extends ActorRefFactory { */ def shutdown(): Unit + /** + * Query the termination status: if it returns true, all callbacks have run + * and the ActorSystem has been fully stopped, i.e. + * `awaitTermination(0 seconds)` would return normally. If this method + * returns `false`, the status is actually unknown, since it might have + * changed since you queried it. + */ + def isTerminated: Boolean + /** * Registers the provided extension and creates its payload, if this extension isn't already registered * This method has putIfAbsent-semantics, this method can potentially block, waiting for the initialization @@ -499,6 +510,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf def registerOnTermination(code: Runnable) { terminationCallbacks.add(code) } def awaitTermination(timeout: Duration) { Await.ready(terminationCallbacks, timeout) } def awaitTermination() = awaitTermination(Duration.Inf) + def isTerminated = terminationCallbacks.isTerminated def shutdown(): Unit = guardian.stop() @@ -634,5 +646,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf } final def result(atMost: Duration)(implicit permit: CanAwait): Unit = ready(atMost) + + final def isTerminated: Boolean = latch.getCount == 0 } } From 9debd392b12912f90a5bdda450225eff0bbf39f0 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 20 Feb 2012 21:01:48 +0100 Subject: [PATCH 22/46] remove termination-triggered slaughter, see #1851 --- akka-actor/src/main/scala/akka/actor/FaultHandling.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 6074780801..70246bab30 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -260,10 +260,7 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration SupervisorStrategy.maxNrOfRetriesOption(maxNrOfRetries), SupervisorStrategy.withinTimeRangeOption(withinTimeRange).map(_.toMillis.toInt)) - def handleChildTerminated(context: ActorContext, child: ActorRef, children: Iterable[ActorRef]): Unit = { - children foreach (context.stop(_)) - //TODO optimization to drop all children here already? - } + def handleChildTerminated(context: ActorContext, child: ActorRef, children: Iterable[ActorRef]): Unit = {} def processFailure(context: ActorContext, restart: Boolean, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Unit = { if (children.nonEmpty) { From a2a9996ee6db5bc7f9f2819d60f746c2edea37c7 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 20 Feb 2012 22:02:15 +0100 Subject: [PATCH 23/46] Fixing typo --- akka-docs/java/typed-actors.rst | 3 ++- akka-docs/scala/typed-actors.rst | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/akka-docs/java/typed-actors.rst b/akka-docs/java/typed-actors.rst index a54e5767d9..8d5a13b8b2 100644 --- a/akka-docs/java/typed-actors.rst +++ b/akka-docs/java/typed-actors.rst @@ -166,7 +166,8 @@ Receive arbitrary messages If your implementation class of your TypedActor extends ``akka.actor.TypedActor.Receiver``, all messages that are not ``MethodCall``s will be passed into the ``onReceive``-method. -This allows you to react to DeathWatch ``Terminated``-messages and other typed of messages. +This allows you to react to DeathWatch ``Terminated``-messages and other types of messages, +e.g. when interfacing with untyped actors. Lifecycle callbacks ------------------- diff --git a/akka-docs/scala/typed-actors.rst b/akka-docs/scala/typed-actors.rst index 4dee51a920..2d47063664 100644 --- a/akka-docs/scala/typed-actors.rst +++ b/akka-docs/scala/typed-actors.rst @@ -178,7 +178,8 @@ Receive arbitrary messages If your implementation class of your TypedActor extends ``akka.actor.TypedActor.Receiver``, all messages that are not ``MethodCall``s will be passed into the ``onReceive``-method. -This allows you to react to DeathWatch ``Terminated``-messages and other typed of messages. +This allows you to react to DeathWatch ``Terminated``-messages and other types of messages, +e.g. when interfacing with untyped actors. Supercharging ------------- From 72ab6429c21e4db949d8623251b412aed5e487c7 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 21 Feb 2012 08:51:51 +0100 Subject: [PATCH 24/46] document AllForOneStrategy better, see #1851 --- akka-docs/general/supervision.rst | 38 +++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/akka-docs/general/supervision.rst b/akka-docs/general/supervision.rst index 2aa9ceba0a..99efb0378d 100644 --- a/akka-docs/general/supervision.rst +++ b/akka-docs/general/supervision.rst @@ -7,6 +7,8 @@ This chapter outlines the concept behind supervision, the primitives offered and their semantics. For details on how that translates into real code, please refer to the corresponding chapters for Scala and Java APIs. +.. _supervision-directives: + What Supervision Means ---------------------- @@ -110,3 +112,39 @@ external resource, which may also be one of its own children. If a third party terminates a child by way of the ``system.stop(child)`` method or sending a :class:`PoisonPill`, the supervisor might well be affected. +One-For-One Strategy vs. All-For-One Strategy +--------------------------------------------- + +There are two classes of supervision strategies which come with Akka: +:class:`OneForOneStrategy` and :class:`AllForOneStrategy`. Both are configured +with a mapping from exception type to supervision directive (see +:ref:`above `) and limits on how often a child is allowed to fail +before terminating it. The difference between them is that the former applies +the obtained directive only to the failed child, whereas the latter applies it +to all siblings as well. Normally, you should use the +:class:`OneForOneStrategy`, which also is the default if none is specified +explicitly. + +The :class:`AllForOneStrategy` is applicable in cases where the ensemble of +children has so tight dependencies among them, that a failure of one child +affects the function of the others, i.e. they are intricably linked. Since a +restart does not clear out the mailbox, it often is best to stop the children +upon failure and re-create them explicitly from the supervisor (by watching the +children’s lifecycle); otherwise you have to make sure that it is no problem +for any of the actors to receive a message which was queued before the restart +but processed afterwards. + +Normally stopping a child (i.e. not in response to a failure) will not +automatically terminate the other children in an all-for-one strategy, that can +easily be done by watching their lifecycle: if the :class:`Terminated` message +is not handled by the supervisor, it will throw a :class:`DeathPathException` +which (depending on its supervisor) will restart it, and the default +:meth:`preRestart` action will terminate all children. Of course this can be +handled explicitly as well. + +Please note that creating one-off actors from an all-for-one supervisor entails +that failures escalated by the temporary actor will affect all the permanent +ones. If this is not desired, install an intermediate supervisor; this can very +easily be done by declaring a router of size 1 for the worker, see +:ref:`routing-scala` or :ref:`routing-java`. + From 2bf95e15a840c393fe13b282087be9d2876122c3 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 21 Feb 2012 09:51:09 +0100 Subject: [PATCH 25/46] Update to version as of 2012-02-21 --- .../main/java/akka/jsr166y/ForkJoinPool.java | 998 +++++++++++------- .../main/java/akka/jsr166y/ForkJoinTask.java | 332 +++--- .../akka/jsr166y/ForkJoinWorkerThread.java | 6 +- 3 files changed, 755 insertions(+), 581 deletions(-) diff --git a/akka-actor/src/main/java/akka/jsr166y/ForkJoinPool.java b/akka-actor/src/main/java/akka/jsr166y/ForkJoinPool.java index f92e5541f4..3cda4d8af1 100644 --- a/akka-actor/src/main/java/akka/jsr166y/ForkJoinPool.java +++ b/akka-actor/src/main/java/akka/jsr166y/ForkJoinPool.java @@ -5,7 +5,6 @@ */ package akka.jsr166y; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -178,7 +177,10 @@ public class ForkJoinPool extends AbstractExecutorService { * If an attempted steal fails, a thief always chooses a different * random victim target to try next. So, in order for one thief to * progress, it suffices for any in-progress poll or new push on - * any empty queue to complete. + * any empty queue to complete. (This is why we normally use + * method pollAt and its variants that try once at the apparent + * base index, else consider alternative actions, rather than + * method poll.) * * This approach also enables support of a user mode in which local * task processing is in FIFO, not LIFO order, simply by using @@ -208,7 +210,8 @@ public class ForkJoinPool extends AbstractExecutorService { * lock (mainly to protect in the case of resizing) but we use * only a simple spinlock (using bits in field runState), because * submitters encountering a busy queue move on to try or create - * other queues, so never block. + * other queues -- they block only when creating and registering + * new queues. * * Management * ========== @@ -234,10 +237,7 @@ public class ForkJoinPool extends AbstractExecutorService { * deregister WorkQueues, as well as to enable shutdown. It is * only modified under a lock (normally briefly held, but * occasionally protecting allocations and resizings) but even - * when locked remains available to check consistency. An - * auxiliary field "growHints", also only modified under lock, - * contains a candidate index for the next WorkQueue and - * a mask for submission queue indices. + * when locked remains available to check consistency. * * Recording WorkQueues. WorkQueues are recorded in the * "workQueues" array that is created upon pool construction and @@ -249,12 +249,7 @@ public class ForkJoinPool extends AbstractExecutorService { * readers must tolerate null slots. Shared (submission) queues * are at even indices, worker queues at odd indices. Grouping * them together in this way simplifies and speeds up task - * scanning. To avoid flailing during start-up, the array is - * presized to hold twice #parallelism workers (which is unlikely - * to need further resizing during execution). But to avoid - * dealing with so many null slots, variable runState includes a - * mask for the nearest power of two that contains all currently - * used indices. + * scanning. * * All worker thread creation is on-demand, triggered by task * submissions, replacement of terminated workers, and/or @@ -386,8 +381,8 @@ public class ForkJoinPool extends AbstractExecutorService { * (http://portal.acm.org/citation.cfm?id=155354). It differs in * that: (1) We only maintain dependency links across workers upon * steals, rather than use per-task bookkeeping. This sometimes - * requires a linear scan of workQueues array to locate stealers, but - * often doesn't because stealers leave hints (that may become + * requires a linear scan of workQueues array to locate stealers, + * but often doesn't because stealers leave hints (that may become * stale/wrong) of where to locate them. A stealHint is only a * hint because a worker might have had multiple steals and the * hint records only one of them (usually the most current). @@ -398,22 +393,43 @@ public class ForkJoinPool extends AbstractExecutorService { * which means that we miss links in the chain during long-lived * tasks, GC stalls etc (which is OK since blocking in such cases * is usually a good idea). (4) We bound the number of attempts - * to find work (see MAX_HELP_DEPTH) and fall back to suspending - * the worker and if necessary replacing it with another. + * to find work (see MAX_HELP) and fall back to suspending the + * worker and if necessary replacing it with another. * * It is impossible to keep exactly the target parallelism number * of threads running at any given time. Determining the * existence of conservatively safe helping targets, the * availability of already-created spares, and the apparent need * to create new spares are all racy, so we rely on multiple - * retries of each. Currently, in keeping with on-demand - * signalling policy, we compensate only if blocking would leave - * less than one active (non-waiting, non-blocked) worker. - * Additionally, to avoid some false alarms due to GC, lagging - * counters, system activity, etc, compensated blocking for joins - * is only attempted after rechecks stabilize in - * ForkJoinTask.awaitJoin. (Retries are interspersed with - * Thread.yield, for good citizenship.) + * retries of each. Compensation in the apparent absence of + * helping opportunities is challenging to control on JVMs, where + * GC and other activities can stall progress of tasks that in + * turn stall out many other dependent tasks, without us being + * able to determine whether they will ever require compensation. + * Even though work-stealing otherwise encounters little + * degradation in the presence of more threads than cores, + * aggressively adding new threads in such cases entails risk of + * unwanted positive feedback control loops in which more threads + * cause more dependent stalls (as well as delayed progress of + * unblocked threads to the point that we know they are available) + * leading to more situations requiring more threads, and so + * on. This aspect of control can be seen as an (analytically + * intractable) game with an opponent that may choose the worst + * (for us) active thread to stall at any time. We take several + * precautions to bound losses (and thus bound gains), mainly in + * methods tryCompensate and awaitJoin: (1) We only try + * compensation after attempting enough helping steps (measured + * via counting and timing) that we have already consumed the + * estimated cost of creating and activating a new thread. (2) We + * allow up to 50% of threads to be blocked before initially + * adding any others, and unless completely saturated, check that + * some work is available for a new worker before adding. Also, we + * create up to only 50% more threads until entering a mode that + * only adds a thread if all others are possibly blocked. All + * together, this means that we might be half as fast to react, + * and create half as many threads as possible in the ideal case, + * but present vastly fewer anomalies in all other cases compared + * to both more aggressive and more conservative alternatives. * * Style notes: There is a lot of representation-level coupling * among classes ForkJoinPool, ForkJoinWorkerThread, and @@ -449,19 +465,6 @@ public class ForkJoinPool extends AbstractExecutorService { // Static utilities - /** - * Computes an initial hash code (also serving as a non-zero - * random seed) for a thread id. This method is expected to - * provide higher-quality hash codes than using method hashCode(). - */ - static final int hashId(long id) { - int h = (int)id ^ (int)(id >>> 32); // Use MurmurHash of thread id - h ^= h >>> 16; h *= 0x85ebca6b; - h ^= h >>> 13; h *= 0xc2b2ae35; - h ^= h >>> 16; - return (h == 0) ? 1 : h; // ensure nonzero - } - /** * If there is a security manager, makes sure caller has * permission to modify threads. @@ -593,10 +596,14 @@ public class ForkJoinPool extends AbstractExecutorService { static final class WorkQueue { /** * Capacity of work-stealing queue array upon initialization. - * Must be a power of two; at least 4, but set larger to - * reduce cacheline sharing among queues. + * Must be a power of two; at least 4, but should be larger to + * reduce or eliminate cacheline sharing among queues. + * Currently, it is much larger, as a partial workaround for + * the fact that JVMs often place arrays in locations that + * share GC bookkeeping (especially cardmarks) such that + * per-write accesses encounter serious memory contention. */ - static final int INITIAL_QUEUE_CAPACITY = 1 << 8; + static final int INITIAL_QUEUE_CAPACITY = 1 << 13; /** * Maximum size for queue arrays. Must be a power of two less @@ -620,43 +627,61 @@ public class ForkJoinPool extends AbstractExecutorService { volatile int base; // index of next slot for poll int top; // index of next slot for push ForkJoinTask[] array; // the elements (initially unallocated) + final ForkJoinPool pool; // the containing pool (may be null) final ForkJoinWorkerThread owner; // owning thread or null if shared volatile Thread parker; // == owner during call to park; else null ForkJoinTask currentJoin; // task being joined in awaitJoin ForkJoinTask currentSteal; // current non-local task being executed // Heuristic padding to ameliorate unfortunate memory placements - Object p00, p01, p02, p03, p04, p05, p06, p07, p08, p09, p0a; + Object p00, p01, p02, p03, p04, p05, p06, p07; + Object p08, p09, p0a, p0b, p0c, p0d, p0e; - WorkQueue(ForkJoinWorkerThread owner, int mode) { - this.owner = owner; + WorkQueue(ForkJoinPool pool, ForkJoinWorkerThread owner, int mode) { this.mode = mode; + this.pool = pool; + this.owner = owner; // Place indices in the center of array (that is not yet allocated) base = top = INITIAL_QUEUE_CAPACITY >>> 1; } /** - * Returns number of tasks in the queue. + * Returns the approximate number of tasks in the queue. */ final int queueSize() { - int n = base - top; // non-owner callers must read base first - return (n >= 0) ? 0 : -n; + int n = base - top; // non-owner callers must read base first + return (n >= 0) ? 0 : -n; // ignore transient negative + } + + /** + * Provides a more accurate estimate of whether this queue has + * any tasks than does queueSize, by checking whether a + * near-empty queue has at least one unclaimed task. + */ + final boolean isEmpty() { + ForkJoinTask[] a; int m, s; + int n = base - (s = top); + return (n >= 0 || + (n == -1 && + ((a = array) == null || + (m = a.length - 1) < 0 || + U.getObjectVolatile + (a, ((m & (s - 1)) << ASHIFT) + ABASE) == null))); } /** * Pushes a task. Call only by owner in unshared queues. * * @param task the task. Caller must ensure non-null. - * @param p if non-null, pool to signal if necessary * @throw RejectedExecutionException if array cannot be resized */ - final void push(ForkJoinTask task, ForkJoinPool p) { - ForkJoinTask[] a; + final void push(ForkJoinTask task) { + ForkJoinTask[] a; ForkJoinPool p; int s = top, m, n; if ((a = array) != null) { // ignore if queue removed U.putOrderedObject (a, (((m = a.length - 1) & s) << ASHIFT) + ABASE, task); if ((n = (top = s + 1) - base) <= 2) { - if (p != null) + if ((p = pool) != null) p.signalWork(); } else if (n >= m) @@ -691,23 +716,6 @@ public class ForkJoinPool extends AbstractExecutorService { return submitted; } - /** - * Takes next task, if one exists, in FIFO order. - */ - final ForkJoinTask poll() { - ForkJoinTask[] a; int b; ForkJoinTask t; - while ((b = base) - top < 0 && (a = array) != null) { - int j = (((a.length - 1) & b) << ASHIFT) + ABASE; - if ((t = (ForkJoinTask)U.getObjectVolatile(a, j)) != null && - base == b && - U.compareAndSwapObject(a, j, t, null)) { - base = b + 1; - return t; - } - } - return null; - } - /** * Takes next task, if one exists, in LIFO order. Call only * by owner in unshared queues. (We do not have a shared @@ -730,6 +738,49 @@ public class ForkJoinPool extends AbstractExecutorService { return null; } + /** + * Takes a task in FIFO order if b is base of queue and a task + * can be claimed without contention. Specialized versions + * appear in ForkJoinPool methods scan and tryHelpStealer. + */ + final ForkJoinTask pollAt(int b) { + ForkJoinTask t; ForkJoinTask[] a; + if ((a = array) != null) { + int j = (((a.length - 1) & b) << ASHIFT) + ABASE; + if ((t = (ForkJoinTask)U.getObjectVolatile(a, j)) != null && + base == b && + U.compareAndSwapObject(a, j, t, null)) { + base = b + 1; + return t; + } + } + return null; + } + + /** + * Takes next task, if one exists, in FIFO order. + */ + final ForkJoinTask poll() { + ForkJoinTask[] a; int b; ForkJoinTask t; + while ((b = base) - top < 0 && (a = array) != null) { + int j = (((a.length - 1) & b) << ASHIFT) + ABASE; + t = (ForkJoinTask)U.getObjectVolatile(a, j); + if (t != null) { + if (base == b && + U.compareAndSwapObject(a, j, t, null)) { + base = b + 1; + return t; + } + } + else if (base == b) { + if (b + 1 == top) + break; + Thread.yield(); // wait for lagging update + } + } + return null; + } + /** * Takes next task, if one exists, in order specified by mode. */ @@ -749,23 +800,6 @@ public class ForkJoinPool extends AbstractExecutorService { return (ForkJoinTask)U.getObjectVolatile(a, j); } - /** - * Returns task at index b if b is current base of queue. - */ - final ForkJoinTask pollAt(int b) { - ForkJoinTask t; ForkJoinTask[] a; - if ((a = array) != null) { - int j = (((a.length - 1) & b) << ASHIFT) + ABASE; - if ((t = (ForkJoinTask)U.getObjectVolatile(a, j)) != null && - base == b && - U.compareAndSwapObject(a, j, t, null)) { - base = b + 1; - return t; - } - } - return null; - } - /** * Pops the given task only if it is at the current top. */ @@ -893,8 +927,8 @@ public class ForkJoinPool extends AbstractExecutorService { * Computes next value for random probes. Scans don't require * a very high quality generator, but also not a crummy one. * Marsaglia xor-shift is cheap and works well enough. Note: - * This is manually inlined in several usages in ForkJoinPool - * to avoid writes inside busy scan loops. + * This is manually inlined in its usages in ForkJoinPool to + * avoid writes inside busy scan loops. */ final int nextSeed() { int r = seed; @@ -907,12 +941,46 @@ public class ForkJoinPool extends AbstractExecutorService { /** * Removes and runs tasks until empty, using local mode - * ordering. + * ordering. Normally called only after checking for apparent + * non-emptiness. */ final void runLocalTasks() { - if (base - top < 0) { - for (ForkJoinTask t; (t = nextLocalTask()) != null; ) - t.doExec(); + // hoist checks from repeated pop/poll + ForkJoinTask[] a; int m; + if ((a = array) != null && (m = a.length - 1) >= 0) { + if (mode == 0) { + for (int s; (s = top - 1) - base >= 0;) { + int j = ((m & s) << ASHIFT) + ABASE; + ForkJoinTask t = + (ForkJoinTask)U.getObjectVolatile(a, j); + if (t != null) { + if (U.compareAndSwapObject(a, j, t, null)) { + top = s; + t.doExec(); + } + } + else + break; + } + } + else { + for (int b; (b = base) - top < 0;) { + int j = ((m & b) << ASHIFT) + ABASE; + ForkJoinTask t = + (ForkJoinTask)U.getObjectVolatile(a, j); + if (t != null) { + if (base == b && + U.compareAndSwapObject(a, j, t, null)) { + base = b + 1; + t.doExec(); + } + } else if (base == b) { + if (b + 1 == top) + break; + Thread.yield(); // wait for lagging update + } + } + } } } @@ -927,11 +995,12 @@ public class ForkJoinPool extends AbstractExecutorService { if (t != null) { currentSteal = t; t.doExec(); - runLocalTasks(); + if (top != base) // conservative guard + runLocalTasks(); ++nsteals; currentSteal = null; } - else if (runState < 0) // terminating + else if (runState < 0) // terminating alive = false; return alive; } @@ -1006,10 +1075,22 @@ public class ForkJoinPool extends AbstractExecutorService { * submission queues in method doSubmit. In the future, this may * also incorporate a means to implement different task rejection * and resubmission policies. + * + * Seeds for submitters and workers/workQueues work in basically + * the same way but are initialized and updated using slightly + * different mechanics. Both are initialized using the same + * approach as in class ThreadLocal, where successive values are + * unlikely to collide with previous values. This is done during + * registration for workers, but requires a separate AtomicInteger + * for submitters. Seeds are then randomly modified upon + * collisions using xorshifts, which requires a non-zero seed. */ static final class Submitter { int seed; - Submitter() { seed = hashId(Thread.currentThread().getId()); } + Submitter() { + int s = nextSubmitterSeed.getAndAdd(SEED_INCREMENT); + seed = (s == 0) ? 1 : s; // ensure non-zero + } } /** ThreadLocal class for Submitters */ @@ -1031,6 +1112,12 @@ public class ForkJoinPool extends AbstractExecutorService { */ private static final AtomicInteger poolNumberGenerator; + /** + * Generator for initial hashes/seeds for submitters. Accessed by + * Submitter class constructor. + */ + static final AtomicInteger nextSubmitterSeed; + /** * Permission required for callers of methods that may start or * kill threads. @@ -1064,13 +1151,33 @@ public class ForkJoinPool extends AbstractExecutorService { private static final long SHRINK_TIMEOUT = SHRINK_RATE - (SHRINK_RATE / 10); /** - * The maximum stolen->joining link depth allowed in tryHelpStealer. - * Depths for legitimate chains are unbounded, but we use a fixed - * constant to avoid (otherwise unchecked) cycles and to bound - * staleness of traversal parameters at the expense of sometimes - * blocking when we could be helping. + * The maximum stolen->joining link depth allowed in method + * tryHelpStealer. Must be a power of two. This value also + * controls the maximum number of times to try to help join a task + * without any apparent progress or change in pool state before + * giving up and blocking (see awaitJoin). Depths for legitimate + * chains are unbounded, but we use a fixed constant to avoid + * (otherwise unchecked) cycles and to bound staleness of + * traversal parameters at the expense of sometimes blocking when + * we could be helping. */ - private static final int MAX_HELP_DEPTH = 16; + private static final int MAX_HELP = 32; + + /** + * Secondary time-based bound (in nanosecs) for helping attempts + * before trying compensated blocking in awaitJoin. Used in + * conjunction with MAX_HELP to reduce variance due to different + * polling rates associated with different helping options. The + * value should roughly approximate the time required to create + * and/or activate a worker thread. + */ + private static final long COMPENSATION_DELAY = 100L * 1000L; // 0.1 millisec + + /** + * Increment for seed generators. See class ThreadLocal for + * explanation. + */ + private static final int SEED_INCREMENT = 0x61c88647; /** * Bits and masks for control variables @@ -1102,15 +1209,12 @@ public class ForkJoinPool extends AbstractExecutorService { * * Field runState is an int packed with: * SHUTDOWN: true if shutdown is enabled (1 bit) - * SEQ: a sequence number updated upon (de)registering workers (15 bits) - * MASK: mask (power of 2 - 1) covering all registered poolIndexes (16 bits) + * SEQ: a sequence number updated upon (de)registering workers (30 bits) + * INIT: set true after workQueues array construction (1 bit) * - * The combination of mask and sequence number enables simple - * consistency checks: Staleness of read-only operations on the - * workQueues array can be checked by comparing runState before vs - * after the reads. The low 16 bits (i.e, anding with SMASK) hold - * the smallest power of two covering all indices, minus - * one. + * The sequence number enables simple consistency checks: + * Staleness of read-only operations on the workQueues array can + * be checked by comparing runState before vs after the reads. */ // bit positions/shifts for fields @@ -1120,8 +1224,8 @@ public class ForkJoinPool extends AbstractExecutorService { private static final int EC_SHIFT = 16; // bounds - private static final int POOL_MAX = 0x7fff; // max #workers - 1 private static final int SMASK = 0xffff; // short bits + private static final int MAX_CAP = 0x7fff; // max #workers - 1 private static final int SQMASK = 0xfffe; // even short bits private static final int SHORT_SIGN = 1 << 15; private static final int INT_SIGN = 1 << 31; @@ -1149,8 +1253,6 @@ public class ForkJoinPool extends AbstractExecutorService { // runState bits private static final int SHUTDOWN = 1 << 31; - private static final int RS_SEQ = 1 << 16; - private static final int RS_SEQ_MASK = 0x7fff0000; // access mode for WorkQueue static final int LIFO_QUEUE = 0; @@ -1169,8 +1271,9 @@ public class ForkJoinPool extends AbstractExecutorService { volatile long ctl; // main pool control final int parallelism; // parallelism level final int localMode; // per-worker scheduling mode - int growHints; // for expanding indices/ranges - volatile int runState; // shutdown status, seq, and mask + final int submitMask; // submit queue index bound + int nextSeed; // for initializing worker seeds + volatile int runState; // shutdown status and seq WorkQueue[] workQueues; // main registry final Mutex lock; // for registration final Condition termination; // for awaitTermination @@ -1180,7 +1283,7 @@ public class ForkJoinPool extends AbstractExecutorService { final AtomicInteger nextWorkerNumber; // to create worker name string final String workerNamePrefix; // to create worker name string - // Creating, registering, deregistering and running workers + // Creating, registering, and deregistering workers /** * Tries to create and start a worker @@ -1211,34 +1314,33 @@ public class ForkJoinPool extends AbstractExecutorService { } /** - * Callback from ForkJoinWorkerThread constructor to establish and - * record its WorkQueue. + * Callback from ForkJoinWorkerThread constructor to establish its + * poolIndex and record its WorkQueue. To avoid scanning bias due + * to packing entries in front of the workQueues array, we treat + * the array as a simple power-of-two hash table using per-thread + * seed as hash, expanding as needed. * - * @param wt the worker thread + * @param w the worker's queue */ - final void registerWorker(ForkJoinWorkerThread wt) { - WorkQueue w = wt.workQueue; + final void registerWorker(WorkQueue w) { Mutex lock = this.lock; lock.lock(); try { - int g = growHints, k = g & SMASK; WorkQueue[] ws = workQueues; - if (ws != null) { // ignore on shutdown - int n = ws.length; - if ((k & 1) == 0 || k >= n || ws[k] != null) { - for (k = 1; k < n && ws[k] != null; k += 2) - ; // workers are at odd indices - if (k >= n) // resize - workQueues = ws = Arrays.copyOf(ws, n << 1); - } - w.eventCount = w.poolIndex = k; // establish before recording - ws[k] = w; - growHints = (g & ~SMASK) | ((k + 2) & SMASK); - int rs = runState; - int m = rs & SMASK; // recalculate runState mask - if (k > m) - m = (m << 1) + 1; - runState = (rs & SHUTDOWN) | ((rs + RS_SEQ) & RS_SEQ_MASK) | m; + if (w != null && ws != null) { // skip on shutdown/failure + int rs, n; + while ((n = ws.length) < // ensure can hold total + (parallelism + (short)(ctl >>> TC_SHIFT) << 1)) + workQueues = ws = Arrays.copyOf(ws, n << 1); + int m = n - 1; + int s = nextSeed += SEED_INCREMENT; // rarely-colliding sequence + w.seed = (s == 0) ? 1 : s; // ensure non-zero seed + int r = (s << 1) | 1; // use odd-numbered indices + while (ws[r &= m] != null) // step by approx half size + r += ((n >>> 1) & SQMASK) + 2; + w.eventCount = w.poolIndex = r; // establish before recording + ws[r] = w; // also update seq + runState = ((rs = runState) & SHUTDOWN) | ((rs + 2) & ~SHUTDOWN); } } finally { lock.unlock(); @@ -1255,19 +1357,17 @@ public class ForkJoinPool extends AbstractExecutorService { * @param ex the exception causing failure, or null if none */ final void deregisterWorker(ForkJoinWorkerThread wt, Throwable ex) { + Mutex lock = this.lock; WorkQueue w = null; if (wt != null && (w = wt.workQueue) != null) { w.runState = -1; // ensure runState is set stealCount.getAndAdd(w.totalSteals + w.nsteals); int idx = w.poolIndex; - Mutex lock = this.lock; lock.lock(); try { // remove record from array WorkQueue[] ws = workQueues; - if (ws != null && idx >= 0 && idx < ws.length && ws[idx] == w) { + if (ws != null && idx >= 0 && idx < ws.length && ws[idx] == w) ws[idx] = null; - growHints = (growHints & ~SMASK) | idx; - } } finally { lock.unlock(); } @@ -1291,73 +1391,51 @@ public class ForkJoinPool extends AbstractExecutorService { U.throwException(ex); } - /** - * Top-level runloop for workers, called by ForkJoinWorkerThread.run. - */ - final void runWorker(ForkJoinWorkerThread wt) { - // Initialize queue array and seed in this thread - WorkQueue w = wt.workQueue; - w.growArray(false); - w.seed = hashId(Thread.currentThread().getId()); - - do {} while (w.runTask(scan(w))); - } // Submissions /** * Unless shutting down, adds the given task to a submission queue * at submitter's current queue index (modulo submission - * range). If no queue exists at the index, one is created unless - * pool lock is busy. If the queue and/or lock are busy, another - * index is randomly chosen. The mask in growHints controls the - * effective index range of queues considered. The mask is - * expanded, up to the current workerQueue mask, upon any detected - * contention but otherwise remains small to avoid needlessly - * creating queues when there is no contention. + * range). If no queue exists at the index, one is created. If + * the queue is busy, another index is randomly chosen. The + * submitMask bounds the effective number of queues to the + * (nearest power of two for) parallelism level. + * + * @param task the task. Caller must ensure non-null. */ private void doSubmit(ForkJoinTask task) { - if (task == null) - throw new NullPointerException(); Submitter s = submitters.get(); - for (int r = s.seed, m = growHints >>> 16;;) { - WorkQueue[] ws; WorkQueue q; Mutex lk; + for (int r = s.seed, m = submitMask;;) { + WorkQueue[] ws; WorkQueue q; int k = r & m & SQMASK; // use only even indices if (runState < 0 || (ws = workQueues) == null || ws.length <= k) throw new RejectedExecutionException(); // shutting down - if ((q = ws[k]) == null && (lk = lock).tryAcquire(0)) { - try { // try to create new queue - if (ws == workQueues && (q = ws[k]) == null) { - int rs; // update runState seq - ws[k] = q = new WorkQueue(null, SHARED_QUEUE); - runState = (((rs = runState) & SHUTDOWN) | - ((rs + RS_SEQ) & ~SHUTDOWN)); + else if ((q = ws[k]) == null) { // create new queue + WorkQueue nq = new WorkQueue(this, null, SHARED_QUEUE); + Mutex lock = this.lock; // construct outside lock + lock.lock(); + try { // recheck under lock + int rs = runState; // to update seq + if (ws == workQueues && ws[k] == null) { + ws[k] = nq; + runState = ((rs & SHUTDOWN) | ((rs + 2) & ~SHUTDOWN)); } } finally { - lk.unlock(); + lock.unlock(); } } - if (q != null) { - if (q.trySharedPush(task)) { - signalWork(); - return; - } - else if (m < parallelism - 1 && m < (runState & SMASK)) { - Mutex lock = this.lock; - lock.lock(); // block until lock free - int g = growHints; - if (g >>> 16 == m) // expand range - growHints = (((m << 1) + 1) << 16) | (g & SMASK); - lock.unlock(); // no need for try/finally - } - else if ((r & m) == 0) - Thread.yield(); // occasionally yield if busy + else if (q.trySharedPush(task)) { + signalWork(); + return; } - if (m == (m = growHints >>> 16)) { - r ^= r << 13; // update seed unless new range - r ^= r >>> 17; // same xorshift as WorkQueues + else if (m > 1) { // move to a different index + r ^= r << 13; // same xorshift as WorkQueues + r ^= r >>> 17; s.seed = r ^= r << 5; } + else + Thread.yield(); // yield if no alternatives } } @@ -1406,66 +1484,33 @@ public class ForkJoinPool extends AbstractExecutorService { } } - /** - * Tries to decrement active count (sometimes implicitly) and - * possibly release or create a compensating worker in preparation - * for blocking. Fails on contention or termination. - * - * @return true if the caller can block, else should recheck and retry - */ - final boolean tryCompensate() { - WorkQueue w; Thread p; - int pc = parallelism, e, u, ac, tc, i; - long c = ctl; - WorkQueue[] ws = workQueues; - if ((e = (int)c) >= 0) { - if ((ac = ((u = (int)(c >>> 32)) >> UAC_SHIFT)) <= 0 && - e != 0 && ws != null && (i = e & SMASK) < ws.length && - (w = ws[i]) != null) { - long nc = (long)(w.nextWait & E_MASK) | (c & (AC_MASK|TC_MASK)); - if (w.eventCount == (e | INT_SIGN) && - U.compareAndSwapLong(this, CTL, c, nc)) { - w.eventCount = (e + E_SEQ) & E_MASK; - if ((p = w.parker) != null) - U.unpark(p); - return true; // release an idle worker - } - } - else if ((tc = (short)(u >>> UTC_SHIFT)) >= 0 && ac + pc > 1) { - long nc = ((c - AC_UNIT) & AC_MASK) | (c & ~AC_MASK); - if (U.compareAndSwapLong(this, CTL, c, nc)) - return true; // no compensation needed - } - else if (tc + pc < POOL_MAX) { - long nc = ((c + TC_UNIT) & TC_MASK) | (c & ~TC_MASK); - if (U.compareAndSwapLong(this, CTL, c, nc)) { - addWorker(); - return true; // create replacement - } - } - } - return false; - } // Scanning for tasks + /** + * Top-level runloop for workers, called by ForkJoinWorkerThread.run. + */ + final void runWorker(WorkQueue w) { + w.growArray(false); // initialize queue array in this thread + do {} while (w.runTask(scan(w))); + } + /** * Scans for and, if found, returns one task, else possibly * inactivates the worker. This method operates on single reads of - * volatile state and is designed to be re-invoked continuously in - * part because it returns upon detecting inconsistencies, + * volatile state and is designed to be re-invoked continuously, + * in part because it returns upon detecting inconsistencies, * contention, or state changes that indicate possible success on * re-invocation. * - * The scan searches for tasks across queues, randomly selecting - * the first #queues probes, favoring steals over submissions - * (by exploiting even/odd indexing), and then performing a - * circular sweep of all queues. The scan terminates upon either - * finding a non-empty queue, or completing a full sweep. If the - * worker is not inactivated, it takes and returns a task from - * this queue. On failure to find a task, we take one of the - * following actions, after which the caller will retry calling - * this method unless terminated. + * The scan searches for tasks across a random permutation of + * queues (starting at a random index and stepping by a random + * relative prime, checking each at least once). The scan + * terminates upon either finding a non-empty queue, or completing + * the sweep. If the worker is not inactivated, it takes and + * returns a task from this queue. On failure to find a task, we + * take one of the following actions, after which the caller will + * retry calling this method unless terminated. * * * If pool is terminating, terminate the worker. * @@ -1476,94 +1521,94 @@ public class ForkJoinPool extends AbstractExecutorService { * another worker, but with same net effect. Releasing in other * cases as well ensures that we have enough workers running. * - * * If the caller has run a task since the last empty scan, - * return (to allow rescan) if other workers are not also yet - * enqueued. Field WorkQueue.rescans counts down on each scan to - * ensure eventual inactivation and blocking. - * * * If not already enqueued, try to inactivate and enqueue the - * worker on wait queue. + * worker on wait queue. Or, if inactivating has caused the pool + * to be quiescent, relay to idleAwaitWork to check for + * termination and possibly shrink pool. * - * * If already enqueued and none of the above apply, either park - * awaiting signal, or if this is the most recent waiter and pool - * is quiescent, relay to idleAwaitWork to check for termination - * and possibly shrink pool. + * * If already inactive, and the caller has run a task since the + * last empty scan, return (to allow rescan) unless others are + * also inactivated. Field WorkQueue.rescans counts down on each + * scan to ensure eventual inactivation and blocking. + * + * * If already enqueued and none of the above apply, park + * awaiting signal, * * @param w the worker (via its WorkQueue) * @return a task or null of none found */ private final ForkJoinTask scan(WorkQueue w) { - boolean swept = false; // true after full empty scan - WorkQueue[] ws; // volatile read order matters - int r = w.seed, ec = w.eventCount; // ec is negative if inactive - int rs = runState, m = rs & SMASK; - if ((ws = workQueues) != null && ws.length > m) { // consistency check - for (int k = 0, j = -1 - m; ; ++j) { - WorkQueue q; int b; - if (j < 0) { // random probes while j negative - r ^= r << 13; r ^= r >>> 17; k = (r ^= r << 5) | (j & 1); - } // worker (not submit) for odd j - else // cyclic scan when j >= 0 - k += 7; // step 7 reduces array packing bias - if ((q = ws[k & m]) != null && (b = q.base) - q.top < 0) { - ForkJoinTask t = (ec >= 0) ? q.pollAt(b) : null; - w.seed = r; // save seed for next scan - if (t != null) + WorkQueue[] ws; // first update random seed + int r = w.seed; r ^= r << 13; r ^= r >>> 17; w.seed = r ^= r << 5; + int rs = runState, m; // volatile read order matters + if ((ws = workQueues) != null && (m = ws.length - 1) > 0) { + int ec = w.eventCount; // ec is negative if inactive + int step = (r >>> 16) | 1; // relative prime + for (int j = (m + 1) << 2; ; r += step) { + WorkQueue q; ForkJoinTask t; ForkJoinTask[] a; int b; + if ((q = ws[r & m]) != null && (b = q.base) - q.top < 0 && + (a = q.array) != null) { // probably nonempty + int i = (((a.length - 1) & b) << ASHIFT) + ABASE; + t = (ForkJoinTask)U.getObjectVolatile(a, i); + if (q.base == b && ec >= 0 && t != null && + U.compareAndSwapObject(a, i, t, null)) { + q.base = b + 1; // specialization of pollAt return t; - break; + } + else if ((t != null || b + 1 != q.top) && + (ec < 0 || j <= m)) { + rs = 0; // mark scan as imcomplete + break; // caller can retry after release + } } - else if (j - m > m) { - if (rs == runState) // staleness check - swept = true; + if (--j < 0) break; - } } - - // Decode ctl on empty scan long c = ctl; int e = (int)c, a = (int)(c >> AC_SHIFT), nr, ns; - if (e < 0) // pool is terminating - w.runState = -1; - else if (!swept) { // try to release a waiter - WorkQueue v; Thread p; - if (e > 0 && a < 0 && (v = ws[e & m]) != null && - v.eventCount == (e | INT_SIGN)) { + if (e < 0) // decode ctl on empty scan + w.runState = -1; // pool is terminating + else if (rs == 0 || rs != runState) { // incomplete scan + WorkQueue v; Thread p; // try to release a waiter + if (e > 0 && a < 0 && w.eventCount == ec && + (v = ws[e & m]) != null && v.eventCount == (e | INT_SIGN)) { long nc = ((long)(v.nextWait & E_MASK) | ((c + AC_UNIT) & (AC_MASK|TC_MASK))); - if (U.compareAndSwapLong(this, CTL, c, nc)) { + if (ctl == c && U.compareAndSwapLong(this, CTL, c, nc)) { v.eventCount = (e + E_SEQ) & E_MASK; if ((p = v.parker) != null) U.unpark(p); } } } - else if ((nr = w.rescans) > 0) { // continue rescanning - int ac = a + parallelism; - if (((w.rescans = (ac < nr) ? ac : nr - 1) & 3) == 0 && - w.eventCount == ec) - Thread.yield(); // occasionally yield - } - else if (ec >= 0) { // try to enqueue + else if (ec >= 0) { // try to enqueue/inactivate long nc = (long)ec | ((c - AC_UNIT) & (AC_MASK|TC_MASK)); w.nextWait = e; - w.eventCount = ec | INT_SIGN;// mark as inactive - if (!U.compareAndSwapLong(this, CTL, c, nc)) - w.eventCount = ec; // unmark on CAS failure - else if ((ns = w.nsteals) != 0) { - w.nsteals = 0; // set rescans if ran task - w.rescans = a + parallelism; - w.totalSteals += ns; + w.eventCount = ec | INT_SIGN; // mark as inactive + if (ctl != c || !U.compareAndSwapLong(this, CTL, c, nc)) + w.eventCount = ec; // unmark on CAS failure + else { + if ((ns = w.nsteals) != 0) { + w.nsteals = 0; // set rescans if ran task + w.rescans = (a > 0) ? 0 : a + parallelism; + w.totalSteals += ns; + } + if (a == 1 - parallelism) // quiescent + idleAwaitWork(w, nc, c); } } - else { // already queued - if (parallelism == -a) - idleAwaitWork(w); // quiescent - if (w.eventCount == ec) { - Thread.interrupted(); // clear status - ForkJoinWorkerThread wt = w.owner; + else if (w.eventCount < 0) { // already queued + if ((nr = w.rescans) > 0) { // continue rescanning + int ac = a + parallelism; + if (((w.rescans = (ac < nr) ? ac : nr - 1) & 3) == 0) + Thread.yield(); // yield before block + } + else { + Thread.interrupted(); // clear status + Thread wt = Thread.currentThread(); U.putObject(wt, PARKBLOCKER, this); - w.parker = wt; // emulate LockSupport.park - if (w.eventCount == ec) // recheck - U.park(false, 0L); // block + w.parker = wt; // emulate LockSupport.park + if (w.eventCount < 0) // recheck + U.park(false, 0L); w.parker = null; U.putObject(wt, PARKBLOCKER, null); } @@ -1573,39 +1618,37 @@ public class ForkJoinPool extends AbstractExecutorService { } /** - * If inactivating worker w has caused pool to become quiescent, - * checks for pool termination, and, so long as this is not the - * only worker, waits for event for up to SHRINK_RATE nanosecs. - * On timeout, if ctl has not changed, terminates the worker, - * which will in turn wake up another worker to possibly repeat - * this process. + * If inactivating worker w has caused the pool to become + * quiescent, checks for pool termination, and, so long as this is + * not the only worker, waits for event for up to SHRINK_RATE + * nanosecs. On timeout, if ctl has not changed, terminates the + * worker, which will in turn wake up another worker to possibly + * repeat this process. * * @param w the calling worker + * @param currentCtl the ctl value triggering possible quiescence + * @param prevCtl the ctl value to restore if thread is terminated */ - private void idleAwaitWork(WorkQueue w) { - long c; int nw, ec; - if (!tryTerminate(false, false) && - (int)((c = ctl) >> AC_SHIFT) + parallelism == 0 && - (ec = w.eventCount) == ((int)c | INT_SIGN) && - (nw = w.nextWait) != 0) { - long nc = ((long)(nw & E_MASK) | // ctl to restore on timeout - ((c + AC_UNIT) & AC_MASK) | (c & TC_MASK)); - ForkJoinWorkerThread wt = w.owner; - while (ctl == c) { + private void idleAwaitWork(WorkQueue w, long currentCtl, long prevCtl) { + if (w.eventCount < 0 && !tryTerminate(false, false) && + (int)prevCtl != 0 && ctl == currentCtl) { + Thread wt = Thread.currentThread(); + Thread.yield(); // yield before block + while (ctl == currentCtl) { long startTime = System.nanoTime(); Thread.interrupted(); // timed variant of version in scan() U.putObject(wt, PARKBLOCKER, this); w.parker = wt; - if (ctl == c) + if (ctl == currentCtl) U.park(false, SHRINK_RATE); w.parker = null; U.putObject(wt, PARKBLOCKER, null); - if (ctl != c) + if (ctl != currentCtl) break; if (System.nanoTime() - startTime >= SHRINK_TIMEOUT && - U.compareAndSwapLong(this, CTL, c, nc)) { - w.eventCount = (ec + E_SEQ) | E_MASK; - w.runState = -1; // shrink + U.compareAndSwapLong(this, CTL, currentCtl, prevCtl)) { + w.eventCount = (w.eventCount + E_SEQ) | E_MASK; + w.runState = -1; // shrink break; } } @@ -1623,32 +1666,31 @@ public class ForkJoinPool extends AbstractExecutorService { * leaves hints in workers to speed up subsequent calls. The * implementation is very branchy to cope with potential * inconsistencies or loops encountering chains that are stale, - * unknown, or of length greater than MAX_HELP_DEPTH links. All - * of these cases are dealt with by just retrying by caller. + * unknown, or so long that they are likely cyclic. All of these + * cases are dealt with by just retrying by caller. * * @param joiner the joining worker * @param task the task to join * @return true if found or ran a task (and so is immediately retryable) */ - final boolean tryHelpStealer(WorkQueue joiner, ForkJoinTask task) { - ForkJoinTask subtask; // current target + private boolean tryHelpStealer(WorkQueue joiner, ForkJoinTask task) { + WorkQueue[] ws; + int m, depth = MAX_HELP; // remaining chain depth boolean progress = false; - int depth = 0; // current chain depth - int m = runState & SMASK; - WorkQueue[] ws = workQueues; - - if (ws != null && ws.length > m && (subtask = task).status >= 0) { - outer:for (WorkQueue j = joiner;;) { - // Try to find the stealer of subtask, by first using hint - WorkQueue stealer = null; - WorkQueue v = ws[j.stealHint & m]; + if ((ws = workQueues) != null && (m = ws.length - 1) > 0 && + task.status >= 0) { + ForkJoinTask subtask = task; // current target + outer: for (WorkQueue j = joiner;;) { + WorkQueue stealer = null; // find stealer of subtask + WorkQueue v = ws[j.stealHint & m]; // try hint if (v != null && v.currentSteal == subtask) stealer = v; - else { + else { // scan for (int i = 1; i <= m; i += 2) { - if ((v = ws[i]) != null && v.currentSteal == subtask) { + if ((v = ws[i]) != null && v.currentSteal == subtask && + v != joiner) { stealer = v; - j.stealHint = i; // save hint + j.stealHint = i; // save hint break; } } @@ -1656,24 +1698,30 @@ public class ForkJoinPool extends AbstractExecutorService { break; } - for (WorkQueue q = stealer;;) { // Try to help stealer - ForkJoinTask t; int b; + for (WorkQueue q = stealer;;) { // try to help stealer + ForkJoinTask[] a; ForkJoinTask t; int b; if (task.status < 0) break outer; - if ((b = q.base) - q.top < 0) { + if ((b = q.base) - q.top < 0 && (a = q.array) != null) { progress = true; - if (subtask.status < 0) - break outer; // stale - if ((t = q.pollAt(b)) != null) { - stealer.stealHint = joiner.poolIndex; + int i = (((a.length - 1) & b) << ASHIFT) + ABASE; + t = (ForkJoinTask)U.getObjectVolatile(a, i); + if (subtask.status < 0) // must recheck before taking + break outer; + if (t != null && + q.base == b && + U.compareAndSwapObject(a, i, t, null)) { + q.base = b + 1; joiner.runSubtask(t); } + else if (q.base == b) + break outer; // possibly stalled } - else { // empty - try to descend to find stealer's stealer + else { // descend ForkJoinTask next = stealer.currentJoin; - if (++depth == MAX_HELP_DEPTH || subtask.status < 0 || + if (--depth <= 0 || subtask.status < 0 || next == null || next == subtask) - break outer; // max depth, stale, dead-end, cyclic + break outer; // stale, dead-end, or cyclic subtask = next; j = stealer; break; @@ -1690,11 +1738,10 @@ public class ForkJoinPool extends AbstractExecutorService { * @param joiner the joining worker * @param task the task */ - final void tryPollForAndExec(WorkQueue joiner, ForkJoinTask task) { + private void tryPollForAndExec(WorkQueue joiner, ForkJoinTask task) { WorkQueue[] ws; - int m = runState & SMASK; - if ((ws = workQueues) != null && ws.length > m) { - for (int j = 1; j <= m && task.status >= 0; j += 2) { + if ((ws = workQueues) != null) { + for (int j = 1; j < ws.length && task.status >= 0; j += 2) { WorkQueue q = ws[j]; if (q != null && q.pollFor(task)) { joiner.runSubtask(task); @@ -1705,30 +1752,164 @@ public class ForkJoinPool extends AbstractExecutorService { } /** - * Returns a non-empty steal queue, if one is found during a random, - * then cyclic scan, else null. This method must be retried by - * caller if, by the time it tries to use the queue, it is empty. + * Tries to decrement active count (sometimes implicitly) and + * possibly release or create a compensating worker in preparation + * for blocking. Fails on contention or termination. Otherwise, + * adds a new thread if no idle workers are available and either + * pool would become completely starved or: (at least half + * starved, and fewer than 50% spares exist, and there is at least + * one task apparently available). Even though the availability + * check requires a full scan, it is worthwhile in reducing false + * alarms. + * + * @param task if non-null, a task being waited for + * @param blocker if non-null, a blocker being waited for + * @return true if the caller can block, else should recheck and retry + */ + final boolean tryCompensate(ForkJoinTask task, ManagedBlocker blocker) { + int pc = parallelism, e; + long c = ctl; + WorkQueue[] ws = workQueues; + if ((e = (int)c) >= 0 && ws != null) { + int u, a, ac, hc; + int tc = (short)((u = (int)(c >>> 32)) >>> UTC_SHIFT) + pc; + boolean replace = false; + if ((a = u >> UAC_SHIFT) <= 0) { + if ((ac = a + pc) <= 1) + replace = true; + else if ((e > 0 || (task != null && + ac <= (hc = pc >>> 1) && tc < pc + hc))) { + WorkQueue w; + for (int j = 0; j < ws.length; ++j) { + if ((w = ws[j]) != null && !w.isEmpty()) { + replace = true; + break; // in compensation range and tasks available + } + } + } + } + if ((task == null || task.status >= 0) && // recheck need to block + (blocker == null || !blocker.isReleasable()) && ctl == c) { + if (!replace) { // no compensation + long nc = ((c - AC_UNIT) & AC_MASK) | (c & ~AC_MASK); + if (U.compareAndSwapLong(this, CTL, c, nc)) + return true; + } + else if (e != 0) { // release an idle worker + WorkQueue w; Thread p; int i; + if ((i = e & SMASK) < ws.length && (w = ws[i]) != null) { + long nc = ((long)(w.nextWait & E_MASK) | + (c & (AC_MASK|TC_MASK))); + if (w.eventCount == (e | INT_SIGN) && + U.compareAndSwapLong(this, CTL, c, nc)) { + w.eventCount = (e + E_SEQ) & E_MASK; + if ((p = w.parker) != null) + U.unpark(p); + return true; + } + } + } + else if (tc < MAX_CAP) { // create replacement + long nc = ((c + TC_UNIT) & TC_MASK) | (c & ~TC_MASK); + if (U.compareAndSwapLong(this, CTL, c, nc)) { + addWorker(); + return true; + } + } + } + } + return false; + } + + /** + * Helps and/or blocks until the given task is done. + * + * @param joiner the joining worker + * @param task the task + * @return task status on exit + */ + final int awaitJoin(WorkQueue joiner, ForkJoinTask task) { + ForkJoinTask prevJoin = joiner.currentJoin; + joiner.currentJoin = task; + long startTime = 0L; + for (int k = 0, s; ; ++k) { + if ((joiner.isEmpty() ? // try to help + !tryHelpStealer(joiner, task) : + !joiner.tryRemoveAndExec(task))) { + if (k == 0) { + startTime = System.nanoTime(); + tryPollForAndExec(joiner, task); // check uncommon case + } + else if ((k & (MAX_HELP - 1)) == 0 && + System.nanoTime() - startTime >= COMPENSATION_DELAY && + tryCompensate(task, null)) { + if (task.trySetSignal() && task.status >= 0) { + synchronized (task) { + if (task.status >= 0) { + try { // see ForkJoinTask + task.wait(); // for explanation + } catch (InterruptedException ie) { + } + } + else + task.notifyAll(); + } + } + long c; // re-activate + do {} while (!U.compareAndSwapLong + (this, CTL, c = ctl, c + AC_UNIT)); + } + } + if ((s = task.status) < 0) { + joiner.currentJoin = prevJoin; + return s; + } + else if ((k & (MAX_HELP - 1)) == MAX_HELP >>> 1) + Thread.yield(); // for politeness + } + } + + /** + * Stripped-down variant of awaitJoin used by timed joins. Tries + * to help join only while there is continuous progress. (Caller + * will then enter a timed wait.) + * + * @param joiner the joining worker + * @param task the task + * @return task status on exit + */ + final int helpJoinOnce(WorkQueue joiner, ForkJoinTask task) { + int s; + while ((s = task.status) >= 0 && + (joiner.isEmpty() ? + tryHelpStealer(joiner, task) : + joiner.tryRemoveAndExec(task))) + ; + return s; + } + + /** + * Returns a (probably) non-empty steal queue, if one is found + * during a random, then cyclic scan, else null. This method must + * be retried by caller if, by the time it tries to use the queue, + * it is empty. */ private WorkQueue findNonEmptyStealQueue(WorkQueue w) { - int r = w.seed; // Same idea as scan(), but ignoring submissions + // Similar to loop in scan(), but ignoring submissions + int r = w.seed; r ^= r << 13; r ^= r >>> 17; w.seed = r ^= r << 5; + int step = (r >>> 16) | 1; for (WorkQueue[] ws;;) { - int m = runState & SMASK; - if ((ws = workQueues) == null) + int rs = runState, m; + if ((ws = workQueues) == null || (m = ws.length - 1) < 1) return null; - if (ws.length > m) { - WorkQueue q; - for (int k = 0, j = -1 - m;; ++j) { - if (j < 0) { - r ^= r << 13; r ^= r >>> 17; k = r ^= r << 5; - } - else - k += 7; - if ((q = ws[(k | 1) & m]) != null && q.base - q.top < 0) { - w.seed = r; - return q; - } - else if (j - m > m) + for (int j = (m + 1) << 2; ; r += step) { + WorkQueue q = ws[((r << 1) | 1) & m]; + if (q != null && !q.isEmpty()) + return q; + else if (--j < 0) { + if (runState == rs) return null; + break; } } } @@ -1742,17 +1923,18 @@ public class ForkJoinPool extends AbstractExecutorService { */ final void helpQuiescePool(WorkQueue w) { for (boolean active = true;;) { - w.runLocalTasks(); // exhaust local queue + if (w.base - w.top < 0) + w.runLocalTasks(); // exhaust local queue WorkQueue q = findNonEmptyStealQueue(w); if (q != null) { - ForkJoinTask t; + ForkJoinTask t; int b; if (!active) { // re-establish active count long c; active = true; do {} while (!U.compareAndSwapLong (this, CTL, c = ctl, c + AC_UNIT)); } - if ((t = q.poll()) != null) + if ((b = q.base) - q.top < 0 && (t = q.pollAt(b)) != null) w.runSubtask(t); } else { @@ -1780,12 +1962,12 @@ public class ForkJoinPool extends AbstractExecutorService { */ final ForkJoinTask nextTaskFor(WorkQueue w) { for (ForkJoinTask t;;) { - WorkQueue q; + WorkQueue q; int b; if ((t = w.nextLocalTask()) != null) return t; if ((q = findNonEmptyStealQueue(w)) == null) return null; - if ((t = q.poll()) != null) + if ((b = q.base) - q.top < 0 && (t = q.pollAt(b)) != null) return t; } } @@ -1960,30 +2142,31 @@ public class ForkJoinPool extends AbstractExecutorService { checkPermission(); if (factory == null) throw new NullPointerException(); - if (parallelism <= 0 || parallelism > POOL_MAX) + if (parallelism <= 0 || parallelism > MAX_CAP) throw new IllegalArgumentException(); this.parallelism = parallelism; this.factory = factory; this.ueh = handler; this.localMode = asyncMode ? FIFO_QUEUE : LIFO_QUEUE; - this.growHints = 1; long np = (long)(-parallelism); // offset ctl counts this.ctl = ((np << AC_SHIFT) & AC_MASK) | ((np << TC_SHIFT) & TC_MASK); - // initialize workQueues array with room for 2*parallelism if possible - int n = parallelism << 1; - if (n >= POOL_MAX) - n = POOL_MAX; - else { // See Hackers Delight, sec 3.2, where n < (1 << 16) - n |= n >>> 1; n |= n >>> 2; n |= n >>> 4; n |= n >>> 8; - } - this.workQueues = new WorkQueue[(n + 1) << 1]; // #slots = 2 * #workers + // Use nearest power 2 for workQueues size. See Hackers Delight sec 3.2. + int n = parallelism - 1; + n |= n >>> 1; n |= n >>> 2; n |= n >>> 4; n |= n >>> 8; n |= n >>> 16; + int size = (n + 1) << 1; // #slots = 2*#workers + this.submitMask = size - 1; // room for max # of submit queues + this.workQueues = new WorkQueue[size]; this.termination = (this.lock = new Mutex()).newCondition(); this.stealCount = new AtomicLong(); this.nextWorkerNumber = new AtomicInteger(); + int pn = poolNumberGenerator.incrementAndGet(); StringBuilder sb = new StringBuilder("ForkJoinPool-"); - sb.append(poolNumberGenerator.incrementAndGet()); + sb.append(Integer.toString(pn)); sb.append("-worker-"); this.workerNamePrefix = sb.toString(); + lock.lock(); + this.runState = 1; // set init flag + lock.unlock(); } // Execution methods @@ -2005,6 +2188,8 @@ public class ForkJoinPool extends AbstractExecutorService { * scheduled for execution */ public T invoke(ForkJoinTask task) { + if (task == null) + throw new NullPointerException(); doSubmit(task); return task.join(); } @@ -2018,6 +2203,8 @@ public class ForkJoinPool extends AbstractExecutorService { * scheduled for execution */ public void execute(ForkJoinTask task) { + if (task == null) + throw new NullPointerException(); doSubmit(task); } @@ -2035,7 +2222,7 @@ public class ForkJoinPool extends AbstractExecutorService { if (task instanceof ForkJoinTask) // avoid re-wrap job = (ForkJoinTask) task; else - job = ForkJoinTask.adapt(task, null); + job = new ForkJoinTask.AdaptedRunnableAction(task); doSubmit(job); } @@ -2049,6 +2236,8 @@ public class ForkJoinPool extends AbstractExecutorService { * scheduled for execution */ public ForkJoinTask submit(ForkJoinTask task) { + if (task == null) + throw new NullPointerException(); doSubmit(task); return task; } @@ -2059,9 +2248,7 @@ public class ForkJoinPool extends AbstractExecutorService { * scheduled for execution */ public ForkJoinTask submit(Callable task) { - if (task == null) - throw new NullPointerException(); - ForkJoinTask job = ForkJoinTask.adapt(task); + ForkJoinTask job = new ForkJoinTask.AdaptedCallable(task); doSubmit(job); return job; } @@ -2072,9 +2259,7 @@ public class ForkJoinPool extends AbstractExecutorService { * scheduled for execution */ public ForkJoinTask submit(Runnable task, T result) { - if (task == null) - throw new NullPointerException(); - ForkJoinTask job = ForkJoinTask.adapt(task, result); + ForkJoinTask job = new ForkJoinTask.AdaptedRunnable(task, result); doSubmit(job); return job; } @@ -2091,7 +2276,7 @@ public class ForkJoinPool extends AbstractExecutorService { if (task instanceof ForkJoinTask) // avoid re-wrap job = (ForkJoinTask) task; else - job = ForkJoinTask.adapt(task, null); + job = new ForkJoinTask.AdaptedRunnableAction(task); doSubmit(job); return job; } @@ -2113,7 +2298,7 @@ public class ForkJoinPool extends AbstractExecutorService { boolean done = false; try { for (Callable t : tasks) { - ForkJoinTask f = ForkJoinTask.adapt(t); + ForkJoinTask f = new ForkJoinTask.AdaptedCallable(t); doSubmit(f); fs.add(f); } @@ -2299,7 +2484,7 @@ public class ForkJoinPool extends AbstractExecutorService { WorkQueue[] ws; WorkQueue w; if ((ws = workQueues) != null) { for (int i = 0; i < ws.length; i += 2) { - if ((w = ws[i]) != null && w.queueSize() != 0) + if ((w = ws[i]) != null && !w.isEmpty()) return true; } } @@ -2613,7 +2798,7 @@ public class ForkJoinPool extends AbstractExecutorService { ForkJoinPool p = ((t instanceof ForkJoinWorkerThread) ? ((ForkJoinWorkerThread)t).pool : null); while (!blocker.isReleasable()) { - if (p == null || p.tryCompensate()) { + if (p == null || p.tryCompensate(null, blocker)) { try { do {} while (!blocker.isReleasable() && !blocker.block()); } finally { @@ -2630,35 +2815,45 @@ public class ForkJoinPool extends AbstractExecutorService { // implement RunnableFuture. protected RunnableFuture newTaskFor(Runnable runnable, T value) { - return (RunnableFuture) ForkJoinTask.adapt(runnable, value); + return new ForkJoinTask.AdaptedRunnable(runnable, value); } protected RunnableFuture newTaskFor(Callable callable) { - return (RunnableFuture) ForkJoinTask.adapt(callable); + return new ForkJoinTask.AdaptedCallable(callable); } // Unsafe mechanics private static final sun.misc.Unsafe U; private static final long CTL; private static final long PARKBLOCKER; + private static final int ABASE; + private static final int ASHIFT; static { poolNumberGenerator = new AtomicInteger(); + nextSubmitterSeed = new AtomicInteger(0x55555555); modifyThreadPermission = new RuntimePermission("modifyThread"); defaultForkJoinWorkerThreadFactory = new DefaultForkJoinWorkerThreadFactory(); submitters = new ThreadSubmitter(); + int s; try { U = getUnsafe(); Class k = ForkJoinPool.class; + Class ak = ForkJoinTask[].class; CTL = U.objectFieldOffset (k.getDeclaredField("ctl")); Class tk = Thread.class; PARKBLOCKER = U.objectFieldOffset (tk.getDeclaredField("parkBlocker")); + ABASE = U.arrayBaseOffset(ak); + s = U.arrayIndexScale(ak); } catch (Exception e) { throw new Error(e); } + if ((s & (s-1)) != 0) + throw new Error("data type scale not a power of two"); + ASHIFT = 31 - Integer.numberOfLeadingZeros(s); } /** @@ -2671,4 +2866,5 @@ public class ForkJoinPool extends AbstractExecutorService { private static sun.misc.Unsafe getUnsafe() { return Unsafe.instance; } + } diff --git a/akka-actor/src/main/java/akka/jsr166y/ForkJoinTask.java b/akka-actor/src/main/java/akka/jsr166y/ForkJoinTask.java index 9fe2005698..fe12152c3a 100644 --- a/akka-actor/src/main/java/akka/jsr166y/ForkJoinTask.java +++ b/akka-actor/src/main/java/akka/jsr166y/ForkJoinTask.java @@ -5,8 +5,6 @@ */ package akka.jsr166y; -import akka.util.Unsafe; - import java.io.Serializable; import java.util.Collection; import java.util.List; @@ -23,6 +21,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.locks.ReentrantLock; import java.lang.reflect.Constructor; +import akka.util.Unsafe; /** * Abstract base class for tasks that run within a {@link ForkJoinPool}. @@ -199,45 +198,37 @@ public abstract class ForkJoinTask implements Future, Serializable { * methods in a way that flows well in javadocs. */ - /** - * The number of times to try to help join a task without any - * apparent progress before giving up and blocking. The value is - * arbitrary but should be large enough to cope with transient - * stalls (due to GC etc) that can cause helping methods not to be - * able to proceed because other workers have not progressed to - * the point where subtasks can be found or taken. - */ - private static final int HELP_RETRIES = 32; - /* * The status field holds run control status bits packed into a * single int to minimize footprint and to ensure atomicity (via * CAS). Status is initially zero, and takes on nonnegative - * values until completed, upon which status holds value - * NORMAL, CANCELLED, or EXCEPTIONAL. Tasks undergoing blocking - * waits by other threads have the SIGNAL bit set. Completion of - * a stolen task with SIGNAL set awakens any waiters via - * notifyAll. Even though suboptimal for some purposes, we use - * basic builtin wait/notify to take advantage of "monitor - * inflation" in JVMs that we would otherwise need to emulate to - * avoid adding further per-task bookkeeping overhead. We want - * these monitors to be "fat", i.e., not use biasing or thin-lock - * techniques, so use some odd coding idioms that tend to avoid - * them. + * values until completed, upon which status (anded with + * DONE_MASK) holds value NORMAL, CANCELLED, or EXCEPTIONAL. Tasks + * undergoing blocking waits by other threads have the SIGNAL bit + * set. Completion of a stolen task with SIGNAL set awakens any + * waiters via notifyAll. Even though suboptimal for some + * purposes, we use basic builtin wait/notify to take advantage of + * "monitor inflation" in JVMs that we would otherwise need to + * emulate to avoid adding further per-task bookkeeping overhead. + * We want these monitors to be "fat", i.e., not use biasing or + * thin-lock techniques, so use some odd coding idioms that tend + * to avoid them, mainly by arranging that every synchronized + * block performs a wait, notifyAll or both. */ /** The run status of this task */ volatile int status; // accessed directly by pool and workers - static final int NORMAL = 0xfffffffc; // negative with low 2 bits 0 - static final int CANCELLED = 0xfffffff8; // must be < NORMAL - static final int EXCEPTIONAL = 0xfffffff4; // must be < CANCELLED + static final int DONE_MASK = 0xf0000000; // mask out non-completion bits + static final int NORMAL = 0xf0000000; // must be negative + static final int CANCELLED = 0xc0000000; // must be < NORMAL + static final int EXCEPTIONAL = 0x80000000; // must be < CANCELLED static final int SIGNAL = 0x00000001; static final int MARKED = 0x00000002; /** * Marks completion and wakes up threads waiting to join this - * task, also clearing signal request bits. A specialization for - * NORMAL completion is in method doExec. + * task. A specialization for NORMAL completion is in method + * doExec. * * @param completion one of NORMAL, CANCELLED, EXCEPTIONAL * @return completion status on exit @@ -246,7 +237,7 @@ public abstract class ForkJoinTask implements Future, Serializable { for (int s;;) { if ((s = status) < 0) return s; - if (U.compareAndSwapInt(this, STATUS, s, (s & ~SIGNAL)|completion)) { + if (U.compareAndSwapInt(this, STATUS, s, s | completion)) { if ((s & SIGNAL) != 0) synchronized (this) { notifyAll(); } return completion; @@ -270,7 +261,7 @@ public abstract class ForkJoinTask implements Future, Serializable { return setExceptionalCompletion(rex); } while ((s = status) >= 0 && completed) { - if (U.compareAndSwapInt(this, STATUS, s, (s & ~SIGNAL)|NORMAL)) { + if (U.compareAndSwapInt(this, STATUS, s, s | NORMAL)) { if ((s & SIGNAL) != 0) synchronized (this) { notifyAll(); } return NORMAL; @@ -280,47 +271,58 @@ public abstract class ForkJoinTask implements Future, Serializable { return s; } + /** + * Tries to set SIGNAL status. Used by ForkJoinPool. Other + * variants are directly incorporated into externalAwaitDone etc. + * + * @return true if successful + */ + final boolean trySetSignal() { + int s; + return U.compareAndSwapInt(this, STATUS, s = status, s | SIGNAL); + } + /** * Blocks a non-worker-thread until completion. * @return status upon completion */ private int externalAwaitDone() { + boolean interrupted = false; int s; - if ((s = status) >= 0) { - boolean interrupted = false; - synchronized (this) { - while ((s = status) >= 0) { - if (U.compareAndSwapInt(this, STATUS, s, s | SIGNAL)) { + while ((s = status) >= 0) { + if (U.compareAndSwapInt(this, STATUS, s, s | SIGNAL)) { + synchronized (this) { + if (status >= 0) { try { wait(); } catch (InterruptedException ie) { interrupted = true; } } + else + notifyAll(); } } - if (interrupted) - Thread.currentThread().interrupt(); } + if (interrupted) + Thread.currentThread().interrupt(); return s; } /** - * Blocks a non-worker-thread until completion or interruption or timeout. + * Blocks a non-worker-thread until completion or interruption. */ - private int externalInterruptibleAwaitDone(long millis) - throws InterruptedException { + private int externalInterruptibleAwaitDone() throws InterruptedException { int s; if (Thread.interrupted()) throw new InterruptedException(); - if ((s = status) >= 0) { - synchronized (this) { - while ((s = status) >= 0) { - if (U.compareAndSwapInt(this, STATUS, s, s | SIGNAL)) { - wait(millis); - if (millis > 0L) - break; - } + while ((s = status) >= 0) { + if (U.compareAndSwapInt(this, STATUS, s, s | SIGNAL)) { + synchronized (this) { + if (status >= 0) + wait(); + else + notifyAll(); } } } @@ -331,80 +333,37 @@ public abstract class ForkJoinTask implements Future, Serializable { /** * Implementation for join, get, quietlyJoin. Directly handles * only cases of already-completed, external wait, and - * unfork+exec. Others are relayed to awaitJoin. + * unfork+exec. Others are relayed to ForkJoinPool.awaitJoin. * * @return status upon completion */ private int doJoin() { int s; Thread t; ForkJoinWorkerThread wt; ForkJoinPool.WorkQueue w; if ((s = status) >= 0) { - if (!((t = Thread.currentThread()) instanceof ForkJoinWorkerThread)) + if (((t = Thread.currentThread()) instanceof ForkJoinWorkerThread)) { + if (!(w = (wt = (ForkJoinWorkerThread)t).workQueue). + tryUnpush(this) || (s = doExec()) >= 0) + s = wt.pool.awaitJoin(w, this); + } + else s = externalAwaitDone(); - else if (!(w = (wt = (ForkJoinWorkerThread)t).workQueue). - tryUnpush(this) || (s = doExec()) >= 0) - s = awaitJoin(w, wt.pool); } return s; } - /** - * Helps and/or blocks until joined. - * - * @param w the joiner - * @param p the pool - * @return status upon completion - */ - private int awaitJoin(ForkJoinPool.WorkQueue w, ForkJoinPool p) { - int s; - ForkJoinTask prevJoin = w.currentJoin; - w.currentJoin = this; - for (int k = HELP_RETRIES; (s = status) >= 0;) { - if ((w.queueSize() > 0) ? - w.tryRemoveAndExec(this) : // self-help - p.tryHelpStealer(w, this)) // help process tasks - k = HELP_RETRIES; // reset if made progress - else if ((s = status) < 0) // recheck - break; - else if (--k > 0) { - if ((k & 3) == 1) - Thread.yield(); // occasionally yield - } - else if (k == 0) - p.tryPollForAndExec(w, this); // uncommon self-help case - else if (p.tryCompensate()) { // true if can block - try { - int ss = status; - if (ss >= 0 && // assert need signal - U.compareAndSwapInt(this, STATUS, ss, ss | SIGNAL)) { - synchronized (this) { - if (status >= 0) // block - wait(); - } - } - } catch (InterruptedException ignore) { - } finally { - p.incrementActiveCount(); // re-activate - } - } - } - w.currentJoin = prevJoin; - return s; - } - /** * Implementation for invoke, quietlyInvoke. * * @return status upon completion */ private int doInvoke() { - int s; Thread t; + int s; Thread t; ForkJoinWorkerThread wt; if ((s = doExec()) >= 0) { - if (!((t = Thread.currentThread()) instanceof ForkJoinWorkerThread)) + if ((t = Thread.currentThread()) instanceof ForkJoinWorkerThread) + s = (wt = (ForkJoinWorkerThread)t).pool.awaitJoin(wt.workQueue, + this); + else s = externalAwaitDone(); - else { - ForkJoinWorkerThread wt = (ForkJoinWorkerThread)t; - s = awaitJoin(wt.workQueue, wt.pool); - } } return s; } @@ -541,7 +500,7 @@ public abstract class ForkJoinTask implements Future, Serializable { * @return the exception, or null if none */ private Throwable getThrowableException() { - if (status != EXCEPTIONAL) + if ((status & DONE_MASK) != EXCEPTIONAL) return null; int h = System.identityHashCode(this); ExceptionNode e; @@ -626,16 +585,14 @@ public abstract class ForkJoinTask implements Future, Serializable { } /** - * Report the result of invoke or join; called only upon - * non-normal return of internal versions. + * Throws exception, if any, associated with the given status. */ - private V reportResult() { - int s; Throwable ex; - if ((s = status) == CANCELLED) - throw new CancellationException(); - if (s == EXCEPTIONAL && (ex = getThrowableException()) != null) + private void reportException(int s) { + Throwable ex = ((s == CANCELLED) ? new CancellationException() : + (s == EXCEPTIONAL) ? getThrowableException() : + null); + if (ex != null) U.throwException(ex); - return getRawResult(); } // public methods @@ -659,9 +616,7 @@ public abstract class ForkJoinTask implements Future, Serializable { * @return {@code this}, to simplify usage */ public final ForkJoinTask fork() { - ForkJoinWorkerThread wt; - (wt = (ForkJoinWorkerThread)Thread.currentThread()). - workQueue.push(this, wt.pool); + ((ForkJoinWorkerThread)Thread.currentThread()).workQueue.push(this); return this; } @@ -677,10 +632,10 @@ public abstract class ForkJoinTask implements Future, Serializable { * @return the computed result */ public final V join() { - if (doJoin() != NORMAL) - return reportResult(); - else - return getRawResult(); + int s; + if ((s = doJoin() & DONE_MASK) != NORMAL) + reportException(s); + return getRawResult(); } /** @@ -692,10 +647,10 @@ public abstract class ForkJoinTask implements Future, Serializable { * @return the computed result */ public final V invoke() { - if (doInvoke() != NORMAL) - return reportResult(); - else - return getRawResult(); + int s; + if ((s = doInvoke() & DONE_MASK) != NORMAL) + reportException(s); + return getRawResult(); } /** @@ -722,9 +677,12 @@ public abstract class ForkJoinTask implements Future, Serializable { * @throws NullPointerException if any task is null */ public static void invokeAll(ForkJoinTask t1, ForkJoinTask t2) { + int s1, s2; t2.fork(); - t1.invoke(); - t2.join(); + if ((s1 = t1.doInvoke() & DONE_MASK) != NORMAL) + t1.reportException(s1); + if ((s2 = t2.doJoin() & DONE_MASK) != NORMAL) + t2.reportException(s2); } /** @@ -861,7 +819,7 @@ public abstract class ForkJoinTask implements Future, Serializable { * @return {@code true} if this task is now cancelled */ public boolean cancel(boolean mayInterruptIfRunning) { - return setCompletion(CANCELLED) == CANCELLED; + return (setCompletion(CANCELLED) & DONE_MASK) == CANCELLED; } public final boolean isDone() { @@ -869,7 +827,7 @@ public abstract class ForkJoinTask implements Future, Serializable { } public final boolean isCancelled() { - return status == CANCELLED; + return (status & DONE_MASK) == CANCELLED; } /** @@ -889,7 +847,7 @@ public abstract class ForkJoinTask implements Future, Serializable { * exception and was not cancelled */ public final boolean isCompletedNormally() { - return status == NORMAL; + return (status & DONE_MASK) == NORMAL; } /** @@ -900,7 +858,7 @@ public abstract class ForkJoinTask implements Future, Serializable { * @return the exception, or {@code null} if none */ public final Throwable getException() { - int s = status; + int s = status & DONE_MASK; return ((s >= NORMAL) ? null : (s == CANCELLED) ? new CancellationException() : getThrowableException()); @@ -962,9 +920,9 @@ public abstract class ForkJoinTask implements Future, Serializable { */ public final V get() throws InterruptedException, ExecutionException { int s = (Thread.currentThread() instanceof ForkJoinWorkerThread) ? - doJoin() : externalInterruptibleAwaitDone(0L); + doJoin() : externalInterruptibleAwaitDone(); Throwable ex; - if (s == CANCELLED) + if ((s &= DONE_MASK) == CANCELLED) throw new CancellationException(); if (s == EXCEPTIONAL && (ex = getThrowableException()) != null) throw new ExecutionException(ex); @@ -987,52 +945,60 @@ public abstract class ForkJoinTask implements Future, Serializable { */ public final V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - // Messy in part because we measure in nanos, but wait in millis - int s; long millis, nanos; - Thread t = Thread.currentThread(); - if (!(t instanceof ForkJoinWorkerThread)) { - if ((millis = unit.toMillis(timeout)) > 0L) - s = externalInterruptibleAwaitDone(millis); - else - s = status; - } - else if ((s = status) >= 0 && (nanos = unit.toNanos(timeout)) > 0L) { - long deadline = System.nanoTime() + nanos; - ForkJoinWorkerThread wt = (ForkJoinWorkerThread)t; - ForkJoinPool.WorkQueue w = wt.workQueue; - ForkJoinPool p = wt.pool; - if (w.tryUnpush(this)) - doExec(); - boolean blocking = false; + if (Thread.interrupted()) + throw new InterruptedException(); + // Messy in part because we measure in nanosecs, but wait in millisecs + int s; long ns, ms; + if ((s = status) >= 0 && (ns = unit.toNanos(timeout)) > 0L) { + long deadline = System.nanoTime() + ns; + ForkJoinPool p = null; + ForkJoinPool.WorkQueue w = null; + Thread t = Thread.currentThread(); + if (t instanceof ForkJoinWorkerThread) { + ForkJoinWorkerThread wt = (ForkJoinWorkerThread)t; + p = wt.pool; + w = wt.workQueue; + s = p.helpJoinOnce(w, this); // no retries on failure + } + boolean canBlock = false; + boolean interrupted = false; try { while ((s = status) >= 0) { - if (w.runState < 0) + if (w != null && w.runState < 0) cancelIgnoringExceptions(this); - else if (!blocking) - blocking = p.tryCompensate(); + else if (!canBlock) { + if (p == null || p.tryCompensate(this, null)) + canBlock = true; + } else { - millis = TimeUnit.NANOSECONDS.toMillis(nanos); - if (millis > 0L && + if ((ms = TimeUnit.NANOSECONDS.toMillis(ns)) > 0L && U.compareAndSwapInt(this, STATUS, s, s | SIGNAL)) { - try { - synchronized (this) { - if (status >= 0) - wait(millis); + synchronized (this) { + if (status >= 0) { + try { + wait(ms); + } catch (InterruptedException ie) { + if (p == null) + interrupted = true; + } } - } catch (InterruptedException ie) { + else + notifyAll(); } } - if ((s = status) < 0 || - (nanos = deadline - System.nanoTime()) <= 0L) + if ((s = status) < 0 || interrupted || + (ns = deadline - System.nanoTime()) <= 0L) break; } } } finally { - if (blocking) + if (p != null && canBlock) p.incrementActiveCount(); } + if (interrupted) + throw new InterruptedException(); } - if (s != NORMAL) { + if ((s &= DONE_MASK) != NORMAL) { Throwable ex; if (s == CANCELLED) throw new CancellationException(); @@ -1099,7 +1065,7 @@ public abstract class ForkJoinTask implements Future, Serializable { * setRawResult(null)}. */ public void reinitialize() { - if (status == EXCEPTIONAL) + if ((status & DONE_MASK) == EXCEPTIONAL) clearExceptionalCompletion(); else status = 0; @@ -1387,21 +1353,33 @@ public abstract class ForkJoinTask implements Future, Serializable { static final class AdaptedRunnable extends ForkJoinTask implements RunnableFuture { final Runnable runnable; - final T resultOnCompletion; T result; AdaptedRunnable(Runnable runnable, T result) { if (runnable == null) throw new NullPointerException(); this.runnable = runnable; - this.resultOnCompletion = result; + this.result = result; // OK to set this even before completion } - public T getRawResult() { return result; } - public void setRawResult(T v) { result = v; } - public boolean exec() { - runnable.run(); - result = resultOnCompletion; - return true; + public final T getRawResult() { return result; } + public final void setRawResult(T v) { result = v; } + public final boolean exec() { runnable.run(); return true; } + public final void run() { invoke(); } + private static final long serialVersionUID = 5232453952276885070L; + } + + /** + * Adaptor for Runnables without results + */ + static final class AdaptedRunnableAction extends ForkJoinTask + implements RunnableFuture { + final Runnable runnable; + AdaptedRunnableAction(Runnable runnable) { + if (runnable == null) throw new NullPointerException(); + this.runnable = runnable; } - public void run() { invoke(); } + public final Void getRawResult() { return null; } + public final void setRawResult(Void v) { } + public final boolean exec() { runnable.run(); return true; } + public final void run() { invoke(); } private static final long serialVersionUID = 5232453952276885070L; } @@ -1416,9 +1394,9 @@ public abstract class ForkJoinTask implements Future, Serializable { if (callable == null) throw new NullPointerException(); this.callable = callable; } - public T getRawResult() { return result; } - public void setRawResult(T v) { result = v; } - public boolean exec() { + public final T getRawResult() { return result; } + public final void setRawResult(T v) { result = v; } + public final boolean exec() { try { result = callable.call(); return true; @@ -1430,7 +1408,7 @@ public abstract class ForkJoinTask implements Future, Serializable { throw new RuntimeException(ex); } } - public void run() { invoke(); } + public final void run() { invoke(); } private static final long serialVersionUID = 2838392045355241008L; } @@ -1443,7 +1421,7 @@ public abstract class ForkJoinTask implements Future, Serializable { * @return the task */ public static ForkJoinTask adapt(Runnable runnable) { - return new AdaptedRunnable(runnable, null); + return new AdaptedRunnableAction(runnable); } /** diff --git a/akka-actor/src/main/java/akka/jsr166y/ForkJoinWorkerThread.java b/akka-actor/src/main/java/akka/jsr166y/ForkJoinWorkerThread.java index 61b0cce979..4ff31f742d 100644 --- a/akka-actor/src/main/java/akka/jsr166y/ForkJoinWorkerThread.java +++ b/akka-actor/src/main/java/akka/jsr166y/ForkJoinWorkerThread.java @@ -43,8 +43,8 @@ public class ForkJoinWorkerThread extends Thread { if (ueh != null) setUncaughtExceptionHandler(ueh); this.pool = pool; - this.workQueue = new ForkJoinPool.WorkQueue(this, pool.localMode); - pool.registerWorker(this); + pool.registerWorker(this.workQueue = new ForkJoinPool.WorkQueue + (pool, this, pool.localMode)); } /** @@ -101,7 +101,7 @@ public class ForkJoinWorkerThread extends Thread { Throwable exception = null; try { onStart(); - pool.runWorker(this); + pool.runWorker(workQueue); } catch (Throwable ex) { exception = ex; } finally { From 8b10cd97d589d04e600f99d536b251c93f6512fd Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 21 Feb 2012 09:55:49 +0100 Subject: [PATCH 26/46] Updating to HOCON as of 2012-02-21 --- .../main/java/com/typesafe/config/Config.java | 21 ++++- .../com/typesafe/config/ConfigException.java | 0 .../com/typesafe/config/ConfigFactory.java | 0 .../typesafe/config/ConfigIncludeContext.java | 0 .../com/typesafe/config/ConfigIncluder.java | 0 .../java/com/typesafe/config/ConfigList.java | 0 .../com/typesafe/config/ConfigMergeable.java | 0 .../com/typesafe/config/ConfigObject.java | 19 +++++ .../com/typesafe/config/ConfigOrigin.java | 0 .../typesafe/config/ConfigParseOptions.java | 0 .../com/typesafe/config/ConfigParseable.java | 0 .../typesafe/config/ConfigResolveOptions.java | 0 .../com/typesafe/config/ConfigSyntax.java | 0 .../java/com/typesafe/config/ConfigUtil.java | 0 .../java/com/typesafe/config/ConfigValue.java | 0 .../typesafe/config/ConfigValueFactory.java | 0 .../com/typesafe/config/ConfigValueType.java | 0 .../config/impl/AbstractConfigObject.java | 12 +++ .../config/impl/AbstractConfigValue.java | 0 .../typesafe/config/impl/ConfigBoolean.java | 0 .../config/impl/ConfigDelayedMerge.java | 2 + .../config/impl/ConfigDelayedMergeObject.java | 29 ++++++- .../typesafe/config/impl/ConfigDouble.java | 0 .../com/typesafe/config/impl/ConfigImpl.java | 0 .../typesafe/config/impl/ConfigImplUtil.java | 0 .../com/typesafe/config/impl/ConfigInt.java | 0 .../com/typesafe/config/impl/ConfigLong.java | 0 .../com/typesafe/config/impl/ConfigNull.java | 0 .../typesafe/config/impl/ConfigNumber.java | 0 .../typesafe/config/impl/ConfigString.java | 0 .../config/impl/ConfigSubstitution.java | 0 .../config/impl/DefaultTransformer.java | 0 .../com/typesafe/config/impl/FromMapMode.java | 0 .../typesafe/config/impl/MergeableValue.java | 0 .../com/typesafe/config/impl/OriginType.java | 0 .../com/typesafe/config/impl/Parseable.java | 0 .../java/com/typesafe/config/impl/Parser.java | 0 .../java/com/typesafe/config/impl/Path.java | 0 .../com/typesafe/config/impl/PathBuilder.java | 0 .../config/impl/PropertiesParser.java | 0 .../typesafe/config/impl/ResolveStatus.java | 0 .../typesafe/config/impl/SimpleConfig.java | 12 +++ .../config/impl/SimpleConfigList.java | 2 + .../config/impl/SimpleConfigObject.java | 77 +++++++++++++++++++ .../config/impl/SimpleConfigOrigin.java | 0 .../config/impl/SubstitutionExpression.java | 0 .../config/impl/SubstitutionResolver.java | 0 .../java/com/typesafe/config/impl/Token.java | 0 .../com/typesafe/config/impl/TokenType.java | 0 .../com/typesafe/config/impl/Tokenizer.java | 0 .../java/com/typesafe/config/impl/Tokens.java | 0 .../com/typesafe/config/impl/Unmergeable.java | 0 .../java/com/typesafe/config/package.html | 0 53 files changed, 172 insertions(+), 2 deletions(-) mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/Config.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigException.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigList.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigObject.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigUtil.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigValue.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/MergeableValue.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/OriginType.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/Parser.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/Path.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/Token.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java mode change 100644 => 100755 akka-actor/src/main/java/com/typesafe/config/package.html diff --git a/akka-actor/src/main/java/com/typesafe/config/Config.java b/akka-actor/src/main/java/com/typesafe/config/Config.java old mode 100644 new mode 100755 index febeece3e2..629b107be9 --- a/akka-actor/src/main/java/com/typesafe/config/Config.java +++ b/akka-actor/src/main/java/com/typesafe/config/Config.java @@ -418,7 +418,7 @@ public interface Config extends ConfigMergeable { * units suffixes like "10m" or "5ns" as documented in the the * spec. - * + * * @param path * path expression * @return the duration value at the requested path, in milliseconds @@ -487,4 +487,23 @@ public interface Config extends ConfigMergeable { List getMillisecondsList(String path); List getNanosecondsList(String path); + + /** + * Clone the config with only the given path (and its children) retained; + * all sibling paths are removed. + * + * @param path + * path to keep + * @return a copy of the config minus all paths except the one specified + */ + Config withOnlyPath(String path); + + /** + * Clone the config with the given path removed. + * + * @param path + * path to remove + * @return a copy of the config minus the specified path + */ + Config withoutPath(String path); } diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigException.java b/akka-actor/src/main/java/com/typesafe/config/ConfigException.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java b/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java b/akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java b/akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigList.java b/akka-actor/src/main/java/com/typesafe/config/ConfigList.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java b/akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java old mode 100644 new mode 100755 index 16b9837f6d..285bf04e7f --- a/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java @@ -91,4 +91,23 @@ public interface ConfigObject extends ConfigValue, Map { */ @Override ConfigValue get(Object key); + + /** + * Clone the object with only the given key (and its children) retained; all + * sibling keys are removed. + * + * @param key + * key to keep + * @return a copy of the object minus all keys except the one specified + */ + ConfigObject withOnlyKey(String key); + + /** + * Clone the object with the given key removed. + * + * @param key + * key to remove + * @return a copy of the object minus the specified key + */ + ConfigObject withoutKey(String key); } diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java b/akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java b/akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java b/akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java b/akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java b/akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigUtil.java b/akka-actor/src/main/java/com/typesafe/config/ConfigUtil.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigValue.java b/akka-actor/src/main/java/com/typesafe/config/ConfigValue.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java b/akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java b/akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java old mode 100644 new mode 100755 index 6f9fd542ff..c5031fe568 --- a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java @@ -43,6 +43,18 @@ abstract class AbstractConfigObject extends AbstractConfigValue implements return this; } + @Override + abstract public AbstractConfigObject withOnlyKey(String key); + + @Override + abstract public AbstractConfigObject withoutKey(String key); + + abstract protected AbstractConfigObject withOnlyPathOrNull(Path path); + + abstract AbstractConfigObject withOnlyPath(Path path); + + abstract AbstractConfigObject withoutPath(Path path); + /** * This looks up the key with no transformation or type conversion of any * kind, and returns null if the key is not present. diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java old mode 100644 new mode 100755 index b64712ebd5..49eae37c97 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java @@ -25,6 +25,8 @@ import com.typesafe.config.ConfigValueType; final class ConfigDelayedMerge extends AbstractConfigValue implements Unmergeable { + private static final long serialVersionUID = 1L; + // earlier items in the stack win final private List stack; final private boolean ignoresFallbacks; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java old mode 100644 new mode 100755 index c290c47fa5..fa873d9df9 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java @@ -18,9 +18,11 @@ import com.typesafe.config.ConfigValue; // This is just like ConfigDelayedMerge except we know statically // that it will turn out to be an object. -class ConfigDelayedMergeObject extends AbstractConfigObject implements +final class ConfigDelayedMergeObject extends AbstractConfigObject implements Unmergeable { + private static final long serialVersionUID = 1L; + final private List stack; final private boolean ignoresFallbacks; @@ -111,6 +113,31 @@ class ConfigDelayedMergeObject extends AbstractConfigObject implements return (ConfigDelayedMergeObject) super.withFallback(mergeable); } + @Override + public ConfigDelayedMergeObject withOnlyKey(String key) { + throw notResolved(); + } + + @Override + public ConfigDelayedMergeObject withoutKey(String key) { + throw notResolved(); + } + + @Override + protected AbstractConfigObject withOnlyPathOrNull(Path path) { + throw notResolved(); + } + + @Override + AbstractConfigObject withOnlyPath(Path path) { + throw notResolved(); + } + + @Override + AbstractConfigObject withoutPath(Path path) { + throw notResolved(); + } + @Override public Collection unmergedValues() { return stack; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java b/akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java b/akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/MergeableValue.java b/akka-actor/src/main/java/com/typesafe/config/impl/MergeableValue.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/OriginType.java b/akka-actor/src/main/java/com/typesafe/config/impl/OriginType.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java b/akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Parser.java b/akka-actor/src/main/java/com/typesafe/config/impl/Parser.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Path.java b/akka-actor/src/main/java/com/typesafe/config/impl/Path.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java b/akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java b/akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java b/akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java old mode 100644 new mode 100755 index daa9f4c368..0ab776b9af --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java @@ -826,4 +826,16 @@ final class SimpleConfig implements Config, MergeableValue, Serializable { throw new ConfigException.ValidationFailed(problems); } } + + @Override + public SimpleConfig withOnlyPath(String pathExpression) { + Path path = Path.newPath(pathExpression); + return new SimpleConfig(root().withOnlyPath(path)); + } + + @Override + public SimpleConfig withoutPath(String pathExpression) { + Path path = Path.newPath(pathExpression); + return new SimpleConfig(root().withoutPath(path)); + } } diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java old mode 100644 new mode 100755 index 9c31dbf0c1..50d361f35e --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java @@ -18,6 +18,8 @@ import com.typesafe.config.ConfigValueType; final class SimpleConfigList extends AbstractConfigValue implements ConfigList { + private static final long serialVersionUID = 1L; + final private List value; final private boolean resolved; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java old mode 100644 new mode 100755 index c4b8cf461a..a138ec611e --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java @@ -41,6 +41,83 @@ final class SimpleConfigObject extends AbstractConfigObject { this(origin, value, ResolveStatus.fromValues(value.values()), false /* ignoresFallbacks */); } + @Override + public SimpleConfigObject withOnlyKey(String key) { + return withOnlyPath(Path.newKey(key)); + } + + @Override + public SimpleConfigObject withoutKey(String key) { + return withoutPath(Path.newKey(key)); + } + + // gets the object with only the path if the path + // exists, otherwise null if it doesn't. this ensures + // that if we have { a : { b : 42 } } and do + // withOnlyPath("a.b.c") that we don't keep an empty + // "a" object. + @Override + protected SimpleConfigObject withOnlyPathOrNull(Path path) { + String key = path.first(); + Path next = path.remainder(); + AbstractConfigValue v = value.get(key); + + if (next != null) { + if (v != null && (v instanceof AbstractConfigObject)) { + v = ((AbstractConfigObject) v).withOnlyPathOrNull(next); + } else { + // if the path has more elements but we don't have an object, + // then the rest of the path does not exist. + v = null; + } + } + + if (v == null) { + return null; + } else { + return new SimpleConfigObject(origin(), Collections.singletonMap(key, v), + resolveStatus(), ignoresFallbacks); + } + } + + @Override + SimpleConfigObject withOnlyPath(Path path) { + SimpleConfigObject o = withOnlyPathOrNull(path); + if (o == null) { + return new SimpleConfigObject(origin(), + Collections. emptyMap(), resolveStatus(), + ignoresFallbacks); + } else { + return o; + } + } + + @Override + SimpleConfigObject withoutPath(Path path) { + String key = path.first(); + Path next = path.remainder(); + AbstractConfigValue v = value.get(key); + + if (v != null && next != null && v instanceof AbstractConfigObject) { + v = ((AbstractConfigObject) v).withoutPath(next); + Map updated = new HashMap( + value); + updated.put(key, v); + return new SimpleConfigObject(origin(), updated, resolveStatus(), ignoresFallbacks); + } else if (next != null || v == null) { + // can't descend, nothing to remove + return this; + } else { + Map smaller = new HashMap( + value.size() - 1); + for (Map.Entry old : value.entrySet()) { + if (!old.getKey().equals(key)) + smaller.put(old.getKey(), old.getValue()); + } + return new SimpleConfigObject(origin(), smaller, resolveStatus(), ignoresFallbacks); + } + } + @Override protected AbstractConfigValue peek(String key) { return value.get(key); diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java b/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java b/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Token.java b/akka-actor/src/main/java/com/typesafe/config/impl/Token.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java b/akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java b/akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java b/akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java b/akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java old mode 100644 new mode 100755 diff --git a/akka-actor/src/main/java/com/typesafe/config/package.html b/akka-actor/src/main/java/com/typesafe/config/package.html old mode 100644 new mode 100755 From 445ffa3a7ddfe99ff1a0c78edc0a8f516cd7938c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 21 Feb 2012 12:03:14 +0100 Subject: [PATCH 27/46] #1842 - Documenting initial behavior --- akka-docs/general/actors.rst | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/akka-docs/general/actors.rst b/akka-docs/general/actors.rst index 4d8f2c8096..2d9ff67488 100644 --- a/akka-docs/general/actors.rst +++ b/akka-docs/general/actors.rst @@ -66,6 +66,11 @@ behavior logic, or the function itself may be swapped out at runtime, see the during construction of the actor object is special in the sense that a restart of the actor will reset its behavior to this initial one. +.. note:: + The initial behavior of an Actor is extracted prior to constructor is run, + so if you want to base your initial behavior on member state, you should + use ``become`` in the constructor. + Mailbox ------- From 8248379f929ba4cd416d9139f13bb6911e4eaca0 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 21 Feb 2012 12:14:13 +0100 Subject: [PATCH 28/46] Fixing pointless error on ask timeout completion --- akka-actor/src/main/scala/akka/pattern/AskSupport.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala index f0e5939f96..3f11da0543 100644 --- a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala @@ -191,7 +191,7 @@ trait AskSupport { val result = Promise[Any]()(provider.dispatcher) val a = new PromiseActorRef(provider, path, provider.tempContainer, result, provider.deathWatch) provider.registerTempActor(a, path) - val f = provider.scheduler.scheduleOnce(timeout.duration) { result.failure(new AskTimeoutException("Timed out")) } + val f = provider.scheduler.scheduleOnce(timeout.duration) { result.tryComplete(Left(new AskTimeoutException("Timed out"))) } result onComplete { _ ⇒ try { a.stop(); f.cancel() } finally { provider.unregisterTempActor(path) } From 211d5ddab8d6b6bd09d4c52bcec7d391efc63b78 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 21 Feb 2012 12:17:48 +0100 Subject: [PATCH 29/46] Tightening some screws --- akka-actor/src/main/scala/akka/pattern/AskSupport.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala index 3f11da0543..368dd9110a 100644 --- a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala @@ -193,7 +193,7 @@ trait AskSupport { provider.registerTempActor(a, path) val f = provider.scheduler.scheduleOnce(timeout.duration) { result.tryComplete(Left(new AskTimeoutException("Timed out"))) } result onComplete { _ ⇒ - try { a.stop(); f.cancel() } + try { try a.stop() finally f.cancel() } finally { provider.unregisterTempActor(path) } } a From 3eeaadd804431312bd705439dfebae07d4d418ea Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 21 Feb 2012 13:22:25 +0100 Subject: [PATCH 30/46] move back to explicit enqueue/dequeue delegation from Mailbox to MessageQueue, see #1844 --- .../src/test/scala/akka/actor/FSMTimingSpec.scala | 4 ++-- .../test/scala/akka/actor/dispatch/ActorModelSpec.scala | 2 +- .../akka/actor/dispatch/BalancingDispatcherSpec.scala | 4 ++-- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 9 +++++++-- akka-actor/src/main/scala/akka/routing/Routing.scala | 8 ++++---- .../scala/akka/testkit/CallingThreadDispatcher.scala | 2 +- 8 files changed, 19 insertions(+), 14 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index 1f708983bd..59468125eb 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -182,7 +182,7 @@ object FSMTimingSpec { when(TestCancelTimer) { case Event(Tick, _) ⇒ setTimer("hallo", Tock, 1 milli, false) - TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.messageQueue.hasMessages, 1 second) + TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.hasMessages, 1 second) cancelTimer("hallo") sender ! Tick setTimer("hallo", Tock, 500 millis, false) @@ -209,7 +209,7 @@ object FSMTimingSpec { case Event(Tick, _) ⇒ suspend(self) setTimer("named", Tock, 1 millis, false) - TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.messageQueue.hasMessages, 1 second) + TestKit.awaitCond(context.asInstanceOf[ActorCell].mailbox.hasMessages, 1 second) stay forMax (1 millis) replying Tick case Event(Tock, _) ⇒ goto(TestCancelStateTimerInNamedTimerMessage2) 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 87a9cf9734..88358e9f16 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -374,7 +374,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa def compare(l: AnyRef, r: AnyRef) = (l, r) match { case (ll: ActorCell, rr: ActorCell) ⇒ ll.self.path compareTo rr.self.path } } foreach { case cell: ActorCell ⇒ - System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.messageQueue.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain())) + System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain())) } System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala index ec0982982c..4060587b73 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala @@ -76,8 +76,8 @@ class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) { } finishedCounter.await(5, TimeUnit.SECONDS) - fast.underlying.mailbox.asInstanceOf[Mailbox].messageQueue.hasMessages must be(false) - slow.underlying.mailbox.asInstanceOf[Mailbox].messageQueue.hasMessages must be(false) + fast.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false) + slow.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false) fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be > (slow.underlying.actor.asInstanceOf[DelayableActor].invocationCount) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 0a77457cf9..39bd52aa87 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -590,7 +590,7 @@ private[akka] class ActorCell( final def checkReceiveTimeout() { val recvtimeout = receiveTimeoutData - if (recvtimeout._1 > 0 && !mailbox.messageQueue.hasMessages) { + if (recvtimeout._1 > 0 && !mailbox.hasMessages) { recvtimeout._2.cancel() //Cancel any ongoing future //Only reschedule if desired and there are currently no more messages to be processed receiveTimeoutData = (recvtimeout._1, system.scheduler.scheduleOnce(Duration(recvtimeout._1, TimeUnit.MILLISECONDS), self, ReceiveTimeout)) diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 46d7b249df..5537b01244 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -40,7 +40,7 @@ class Dispatcher( protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { val mbox = receiver.mailbox - mbox.messageQueue.enqueue(receiver.self, invocation) + mbox.enqueue(receiver.self, invocation) registerForExecution(mbox, true, false) } diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 1a6515ea2d..f25c6571e8 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -44,6 +44,11 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes import Mailbox._ + def enqueue(receiver: ActorRef, msg: Envelope): Unit = messageQueue.enqueue(receiver, msg) + def dequeue(): Envelope = messageQueue.dequeue() + def hasMessages: Boolean = messageQueue.hasMessages + def numberOfMessages: Int = messageQueue.numberOfMessages + @volatile protected var _statusDoNotCallMeDirectly: Status = _ //0 by default @@ -142,7 +147,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes Unsafe.instance.compareAndSwapObject(this, AbstractMailbox.systemMessageOffset, _old, _new) final def canBeScheduledForExecution(hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = status match { - case Open | Scheduled ⇒ hasMessageHint || hasSystemMessageHint || hasSystemMessages || messageQueue.hasMessages + case Open | Scheduled ⇒ hasMessageHint || hasSystemMessageHint || hasSystemMessages || hasMessages case Closed ⇒ false case _ ⇒ hasSystemMessageHint || hasSystemMessages } @@ -166,7 +171,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes left: Int = java.lang.Math.max(dispatcher.throughput, 1), deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined == true) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0L): Unit = if (shouldProcessMessage) { - val next = messageQueue.dequeue() + val next = dequeue() if (next ne null) { if (Mailbox.debug) println(actor.self + " processing message " + next) actor invoke next diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index e77d6ac469..57847f3553 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -773,7 +773,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ * routers based on mailbox and actor internal state. */ protected def hasMessages(a: ActorRef): Boolean = a match { - case x: LocalActorRef ⇒ x.underlying.mailbox.messageQueue.hasMessages + case x: LocalActorRef ⇒ x.underlying.mailbox.hasMessages case _ ⇒ false } @@ -797,7 +797,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ * routers based on mailbox and actor internal state. */ protected def numberOfMessages(a: ActorRef): Int = a match { - case x: LocalActorRef ⇒ x.underlying.mailbox.messageQueue.numberOfMessages + case x: LocalActorRef ⇒ x.underlying.mailbox.numberOfMessages case _ ⇒ 0 } @@ -1249,9 +1249,9 @@ case class DefaultResizer( case a: LocalActorRef ⇒ val cell = a.underlying pressureThreshold match { - case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.messageQueue.hasMessages + case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.hasMessages case i if i < 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null - case threshold ⇒ cell.mailbox.messageQueue.numberOfMessages >= threshold + case threshold ⇒ cell.mailbox.numberOfMessages >= threshold } case x ⇒ false diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 8cbe954aac..aba582ae68 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -199,7 +199,7 @@ class CallingThreadDispatcher( } else false } if (execute) runQueue(mbox, queue) - case m ⇒ m.messageQueue.enqueue(receiver.self, handle) + case m ⇒ m.enqueue(receiver.self, handle) } } From 882249c5cfef3879a7a1fb61d992f4ac49321d0b Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 21 Feb 2012 13:40:05 +0100 Subject: [PATCH 31/46] rename durable mailboxes to message queues, see #1844 keep *MailboxType in order to not confuse users. --- .../scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala | 6 +++--- .../main/scala/akka/actor/mailbox/FiledBasedMailbox.scala | 6 +++--- .../main/scala/akka/actor/mailbox/DurableMailbox.scala | 4 ++-- .../main/scala/akka/actor/mailbox/BSONSerialization.scala | 2 +- .../main/scala/akka/actor/mailbox/MongoBasedMailbox.scala | 8 ++++---- .../main/scala/akka/actor/mailbox/RedisBasedMailbox.scala | 6 +++--- .../scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala | 6 +++--- 7 files changed, 19 insertions(+), 19 deletions(-) diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala index 68aaa6067e..2e3232b22a 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala @@ -19,7 +19,7 @@ class BeanstalkBasedMailboxException(message: String) extends AkkaException(mess class BeanstalkBasedMailboxType(config: Config) extends MailboxType { override def create(owner: Option[ActorContext]): MessageQueue = owner match { - case Some(o) ⇒ new BeanstalkBasedMailbox(o) + case Some(o) ⇒ new BeanstalkBasedMessageQueue(o) case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") } } @@ -27,13 +27,13 @@ class BeanstalkBasedMailboxType(config: Config) extends MailboxType { /** * @author Jonas Bonér */ -class BeanstalkBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { +class BeanstalkBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) with DurableMessageSerialization { private val settings = BeanstalkBasedMailboxExtension(owner.system) private val messageSubmitDelaySeconds = settings.MessageSubmitDelay.toSeconds.toInt private val messageTimeToLiveSeconds = settings.MessageTimeToLive.toSeconds.toInt - val log = Logging(system, "BeanstalkBasedMailbox") + val log = Logging(system, "BeanstalkBasedMessageQueue") private val queue = new ThreadLocal[Client] { override def initialValue = connect(name) } diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala index 70368e2f40..8f4754ebbf 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala @@ -16,14 +16,14 @@ import akka.config.ConfigurationException class FileBasedMailboxType(config: Config) extends MailboxType { override def create(owner: Option[ActorContext]): MessageQueue = owner match { - case Some(o) ⇒ new FileBasedMailbox(o) + case Some(o) ⇒ new FileBasedMessageQueue(o) case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") } } -class FileBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { +class FileBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) with DurableMessageSerialization { - val log = Logging(system, "FileBasedMailbox") + val log = Logging(system, "FileBasedMessageQueue") private val settings = FileBasedMailboxExtension(owner.system) val queuePath = settings.QueuePath diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala index 92072c8cf3..77b932911d 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -12,7 +12,7 @@ private[akka] object DurableExecutableMailboxConfig { val Name = "[\\.\\/\\$\\s]".r } -abstract class DurableMailbox(val owner: ActorContext) extends MessageQueue { +abstract class DurableMessageQueue(val owner: ActorContext) extends MessageQueue { import DurableExecutableMailboxConfig._ def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem] @@ -22,7 +22,7 @@ abstract class DurableMailbox(val owner: ActorContext) extends MessageQueue { } -trait DurableMessageSerialization { this: DurableMailbox ⇒ +trait DurableMessageSerialization { this: DurableMessageQueue ⇒ def serialize(durableMessage: Envelope): Array[Byte] = { diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala index 8da4db9376..aad04630c8 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala @@ -18,7 +18,7 @@ import akka.remote.RemoteProtocol.MessageProtocol import akka.remote.MessageSerializer import akka.actor.ExtendedActorSystem -class BSONSerializableMailbox(system: ExtendedActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging { +class BSONSerializableMessageQueue(system: ExtendedActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging { protected[akka] def serializeDurableMsg(msg: MongoDurableMessage)(implicit serializer: BSONSerializer) = { diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala index ac969695bb..647411b747 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala @@ -21,7 +21,7 @@ class MongoBasedMailboxException(message: String) extends AkkaException(message) class MongoBasedMailboxType(config: Config) extends MailboxType { override def create(owner: Option[ActorContext]): MessageQueue = owner match { - case Some(o) ⇒ new MongoBasedMailbox(o) + case Some(o) ⇒ new MongoBasedMessageQueue(o) case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") } } @@ -37,16 +37,16 @@ class MongoBasedMailboxType(config: Config) extends MailboxType { * * @author Brendan W. McAdams */ -class MongoBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) { +class MongoBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) { // this implicit object provides the context for reading/writing things as MongoDurableMessage - implicit val mailboxBSONSer = new BSONSerializableMailbox(system) + implicit val mailboxBSONSer = new BSONSerializableMessageQueue(system) implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate! private val dispatcher = owner.dispatcher private val settings = MongoBasedMailboxExtension(owner.system) - val log = Logging(system, "MongoBasedMailbox") + val log = Logging(system, "MongoBasedMessageQueue") @volatile private var mongo = connect() diff --git a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala index 9a01d462f8..78a8bc3b37 100644 --- a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala @@ -19,19 +19,19 @@ class RedisBasedMailboxException(message: String) extends AkkaException(message) class RedisBasedMailboxType(config: Config) extends MailboxType { override def create(owner: Option[ActorContext]): MessageQueue = owner match { - case Some(o) ⇒ new RedisBasedMailbox(o) + case Some(o) ⇒ new RedisBasedMessageQueue(o) case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") } } -class RedisBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { +class RedisBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) with DurableMessageSerialization { private val settings = RedisBasedMailboxExtension(owner.system) @volatile private var clients = connect() // returns a RedisClientPool for multiple asynchronous message handling - val log = Logging(system, "RedisBasedMailbox") + val log = Logging(system, "RedisBasedMessageQueue") def enqueue(receiver: ActorRef, envelope: Envelope) { log.debug("ENQUEUING message in redis-based mailbox [%s]".format(envelope)) diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala index 1434f6c1d9..abeb430c64 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala @@ -20,18 +20,18 @@ class ZooKeeperBasedMailboxException(message: String) extends AkkaException(mess class ZooKeeperBasedMailboxType(config: Config) extends MailboxType { override def create(owner: Option[ActorContext]): MessageQueue = owner match { - case Some(o) ⇒ new ZooKeeperBasedMailbox(o) + case Some(o) ⇒ new ZooKeeperBasedMessageQueue(o) case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") } } -class ZooKeeperBasedMailbox(_owner: ActorContext) extends DurableMailbox(_owner) with DurableMessageSerialization { +class ZooKeeperBasedMessageQueue(_owner: ActorContext) extends DurableMessageQueue(_owner) with DurableMessageSerialization { private val settings = ZooKeeperBasedMailboxExtension(owner.system) val queueNode = "/queues" val queuePathTemplate = queueNode + "/%s" - val log = Logging(system, "ZooKeeperBasedMailbox") + val log = Logging(system, "ZooKeeperBasedMessageQueue") private val zkClient = new AkkaZkClient( settings.ZkServerAddresses, From ef0af0b47f9bcaa380a075be7d2f4b807a67eb18 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 21 Feb 2012 15:37:51 +0100 Subject: [PATCH 32/46] document RemoteRouterConfig, see #1846 - fix Java API - change to pass in Addresses so that validation happens earlier --- .../docs/jrouting/RouterViaProgramExample.java | 11 +++++++++++ akka-docs/java/routing.rst | 12 ++++++++++++ .../docs/routing/RouterViaProgramExample.scala | 10 ++++++++++ akka-docs/scala/routing.rst | 12 ++++++++++++ .../scala/akka/remote/RemoteDeployer.scala | 2 +- .../akka/routing/RemoteRouterConfig.scala | 18 +++++++++--------- .../scala/akka/remote/RemoteRouterSpec.scala | 2 +- 7 files changed, 56 insertions(+), 11 deletions(-) diff --git a/akka-docs/java/code/akka/docs/jrouting/RouterViaProgramExample.java b/akka-docs/java/code/akka/docs/jrouting/RouterViaProgramExample.java index 27e90b63fa..52a721ed0d 100644 --- a/akka-docs/java/code/akka/docs/jrouting/RouterViaProgramExample.java +++ b/akka-docs/java/code/akka/docs/jrouting/RouterViaProgramExample.java @@ -5,10 +5,13 @@ package akka.docs.jrouting; import akka.routing.RoundRobinRouter; import akka.routing.DefaultResizer; +import akka.routing.RemoteRouterConfig; import akka.actor.ActorRef; import akka.actor.Props; import akka.actor.UntypedActor; import akka.actor.ActorSystem; +import akka.actor.Address; +import akka.actor.AddressExtractor; import java.util.Arrays; public class RouterViaProgramExample { @@ -67,6 +70,14 @@ public class RouterViaProgramExample { for (int i = 1; i <= 6; i++) { router3.tell(new ExampleActor.Message(i)); } + + //#remoteRoutees + Address addr1 = new Address("akka", "remotesys", "otherhost", 1234); + Address addr2 = AddressExtractor.parse("akka://othersys@anotherhost:1234"); + Address[] addresses = new Address[] { addr1, addr2 }; + ActorRef routerRemote = system.actorOf(new Props(ExampleActor.class) + .withRouter(new RemoteRouterConfig(new RoundRobinRouter(5), addresses))); + //#remoteRoutees } private class CompileCheckJavaDocsForRouting extends UntypedActor { diff --git a/akka-docs/java/routing.rst b/akka-docs/java/routing.rst index aa1daa19db..aeb89c2506 100644 --- a/akka-docs/java/routing.rst +++ b/akka-docs/java/routing.rst @@ -54,6 +54,18 @@ Once you have the router actor it is just to send messages to it as you would to The router will apply its behavior to the message it receives and forward it to the routees. +Remotely Deploying Routees +************************** + +In addition to being able to supply looked-up remote actors as routees, you can +make the router deploy its created children on a set of remote hosts; this will +be done in round-robin fashion. In order to do that, wrap the router +configuration in a :class:`RemoteRouterConfig`, attaching the remote addresses of +the nodes to deploy to. Naturally, this requires your to include the +``akka-remote`` module on your classpath: + +.. includecode:: code/akka/docs/jrouting/RouterViaProgramExample.java#remoteRoutees + How Routing is Designed within Akka ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala b/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala index 246ddbf5a2..3750e42635 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala +++ b/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala @@ -6,6 +6,7 @@ package akka.docs.routing import akka.routing.RoundRobinRouter import akka.actor.{ ActorRef, Props, Actor, ActorSystem } import akka.routing.DefaultResizer +import akka.routing.RemoteRouterConfig case class Message1(nbr: Int) @@ -40,4 +41,13 @@ object RoutingProgrammaticallyExample extends App { //#programmaticRoutingWithResizer 1 to 6 foreach { i ⇒ router3 ! Message1(i) } + //#remoteRoutees + import akka.actor.{ Address, AddressExtractor } + val addresses = Seq( + Address("akka", "remotesys", "otherhost", 1234), + AddressExtractor("akka://othersys@anotherhost:1234")) + val routerRemote = system.actorOf(Props[ExampleActor1].withRouter( + RemoteRouterConfig(RoundRobinRouter(5), addresses))) + //#remoteRoutees + } \ No newline at end of file diff --git a/akka-docs/scala/routing.rst b/akka-docs/scala/routing.rst index 7071747374..92776fa73f 100644 --- a/akka-docs/scala/routing.rst +++ b/akka-docs/scala/routing.rst @@ -54,6 +54,18 @@ Once you have the router actor it is just to send messages to it as you would to The router will apply its behavior to the message it receives and forward it to the routees. +Remotely Deploying Routees +************************** + +In addition to being able to supply looked-up remote actors as routees, you can +make the router deploy its created children on a set of remote hosts; this will +be done in round-robin fashion. In order to do that, wrap the router +configuration in a :class:`RemoteRouterConfig`, attaching the remote addresses of +the nodes to deploy to. Naturally, this requires your to include the +``akka-remote`` module on your classpath: + +.. includecode:: code/akka/docs/routing/RouterViaProgramExample.scala#remoteRoutees + How Routing is Designed within Akka ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala index 1e1068cd5f..61a3459e9d 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala @@ -23,7 +23,7 @@ class RemoteDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extend case AddressExtractor(r) ⇒ Some(deploy.copy(scope = RemoteScope(r))) case str ⇒ if (!str.isEmpty) throw new ConfigurationException("unparseable remote node name " + str) - val nodes = deploy.config.getStringList("target.nodes").asScala + val nodes = deploy.config.getStringList("target.nodes").asScala map (AddressExtractor(_)) if (nodes.isEmpty || deploy.routerConfig == NoRouter) d else Some(deploy.copy(routerConfig = RemoteRouterConfig(deploy.routerConfig, nodes))) } diff --git a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala index afa94433aa..45e059dab6 100644 --- a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala +++ b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala @@ -14,6 +14,9 @@ import akka.config.ConfigurationException import akka.remote.RemoteScope import akka.actor.AddressExtractor import akka.actor.SupervisorStrategy +import akka.actor.Address + +import scala.collection.JavaConverters._ /** * [[akka.routing.RouterConfig]] implementation for remote deployment on defined @@ -21,7 +24,10 @@ import akka.actor.SupervisorStrategy * which makes it possible to mix this with the built-in routers such as * [[akka.routing.RoundRobinRouter]] or custom routers. */ -case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[String]) extends RouterConfig { +case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[Address]) extends RouterConfig { + + def this(local: RouterConfig, nodes: java.lang.Iterable[Address]) = this(local, nodes.asScala) + def this(local: RouterConfig, nodes: Array[Address]) = this(local, nodes: Iterable[Address]) override def createRouteeProvider(context: ActorContext) = new RemoteRouteeProvider(nodes, context, resizer) @@ -49,17 +55,11 @@ case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[String]) exte * * Routee paths may not be combined with remote target nodes. */ -class RemoteRouteeProvider(nodes: Iterable[String], _context: ActorContext, _resizer: Option[Resizer]) +class RemoteRouteeProvider(nodes: Iterable[Address], _context: ActorContext, _resizer: Option[Resizer]) extends RouteeProvider(_context, _resizer) { // need this iterator as instance variable since Resizer may call createRoutees several times - private val nodeAddressIter = { - val nodeAddresses = nodes map { - case AddressExtractor(a) ⇒ a - case x ⇒ throw new ConfigurationException("unparseable remote node " + x) - } - Stream.continually(nodeAddresses).flatten.iterator - } + private val nodeAddressIter = Stream.continually(nodes).flatten.iterator override def createRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): IndexedSeq[ActorRef] = (nrOfInstances, routees, nodes) match { diff --git a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala index bf79caf847..acf7ba4f40 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala @@ -90,7 +90,7 @@ akka.actor.deployment { "deploy its children on remote host driven by programatic definition" in { val router = system.actorOf(Props[Echo].withRouter(new RemoteRouterConfig(RoundRobinRouter(2), - Seq("akka://remote_sys@localhost:12347"))), "blub2") + Seq(Address("akka", "remote_sys", "localhost", 12347)))), "blub2") val replies = for (i ← 1 to 5) yield { router ! "" expectMsgType[ActorRef].path From c84daf83e801c25888906830243614625f989cfb Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 21 Feb 2012 16:40:34 +0100 Subject: [PATCH 33/46] clean up mailbox types configurability, see #1843 - add (config) constructors to std mailbox types - update docs for prio mailbox to directly implement MailboxType --- .../scala/akka/dispatch/AbstractDispatcher.scala | 10 ++++------ .../src/main/scala/akka/dispatch/Mailbox.scala | 7 +++++++ .../docs/dispatcher/DispatcherDocTestBase.java | 15 ++++++++++++--- .../akka/docs/dispatcher/DispatcherDocSpec.scala | 7 ++++++- 4 files changed, 29 insertions(+), 10 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index f6e3bf922c..631e84ece7 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -383,12 +383,10 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit def mailboxType(): MailboxType = { config.getString("mailbox-type") match { case "" ⇒ - val capacity = config.getInt("mailbox-capacity") - if (capacity < 1) UnboundedMailbox() - else { - val duration = Duration(config.getNanoseconds("mailbox-push-timeout-time"), TimeUnit.NANOSECONDS) - BoundedMailbox(capacity, duration) - } + if (config.getInt("mailbox-capacity") < 1) UnboundedMailbox() + else new BoundedMailbox(config) + case "unbounded" ⇒ UnboundedMailbox() + case "bounded" ⇒ new BoundedMailbox(config) case fqcn ⇒ val args = Seq(classOf[Config] -> config) prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args) match { diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index f25c6571e8..3099fe8f01 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -11,6 +11,7 @@ import java.util.concurrent._ import annotation.tailrec import akka.event.Logging.Error import akka.actor.ActorContext +import com.typesafe.config.Config class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause) @@ -355,6 +356,9 @@ trait MailboxType { * It's a case class for Java (new UnboundedMailbox) */ case class UnboundedMailbox() extends MailboxType { + + def this(config: Config) = this() + final override def create(owner: Option[ActorContext]): MessageQueue = new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final val queue = new ConcurrentLinkedQueue[Envelope]() @@ -363,6 +367,9 @@ case class UnboundedMailbox() extends MailboxType { case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType { + def this(config: Config) = this(config.getInt("mailbox-capacity"), + Duration(config.getNanoseconds("mailbox-push-timeout-time"), TimeUnit.NANOSECONDS)) + 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") diff --git a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java index b1334bb9e5..d005ffcd6b 100644 --- a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java +++ b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java @@ -20,8 +20,11 @@ import akka.event.LoggingAdapter; //#imports-prio //#imports-prio-mailbox +import akka.actor.ActorContext; import akka.dispatch.PriorityGenerator; import akka.dispatch.UnboundedPriorityMailbox; +import akka.dispatch.MailboxType; +import akka.dispatch.MessageQueue; import com.typesafe.config.Config; //#imports-prio-mailbox @@ -120,7 +123,7 @@ public class DispatcherDocTestBase { } //#prio-mailbox - public static class PrioMailbox extends UnboundedPriorityMailbox { + public static class PrioMailbox implements MailboxType { static final PriorityGenerator generator = new PriorityGenerator() { // Create a new PriorityGenerator, lower prio means more important @Override @@ -135,9 +138,15 @@ public class DispatcherDocTestBase { return 50; // We default to 50 } }; + + private UnboundedPriorityMailbox priorityMailbox; - public PrioMailbox(Config config) { - super(generator); + public PrioMailbox(Config config) { // needed for reflective instantiation + priorityMailbox = new UnboundedPriorityMailbox(generator); + } + + public MessageQueue create(Option owner) { + return priorityMailbox.create(owner); } } //#prio-mailbox diff --git a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala index 4de32c3b57..fe8eb8a270 100644 --- a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala @@ -108,6 +108,8 @@ object DispatcherDocSpec { //#prio-mailbox import akka.dispatch.PriorityGenerator import akka.dispatch.UnboundedPriorityMailbox + import akka.dispatch.MailboxType + import akka.actor.ActorContext import com.typesafe.config.Config val generator = PriorityGenerator { // Create a new PriorityGenerator, lower prio means more important @@ -118,7 +120,10 @@ object DispatcherDocSpec { } // We create a new Priority dispatcher and seed it with the priority generator - class PrioMailbox(config: Config) extends UnboundedPriorityMailbox(generator) + class PrioMailbox(config: Config) extends MailboxType { + val priorityMailbox = UnboundedPriorityMailbox(generator) + def create(owner: Option[ActorContext]) = priorityMailbox.create(owner) + } //#prio-mailbox class MyActor extends Actor { From 2b4c8b2508f6a85e03fc106da1f76c5a02ed36d8 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 21 Feb 2012 17:23:54 +0100 Subject: [PATCH 34/46] Changing so that PriorityMailboxes can be used --- .../test/scala/akka/dispatch/MailboxConfigSpec.scala | 4 ++-- akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 10 ++++++++-- .../code/akka/docs/dispatcher/DispatcherDocSpec.scala | 5 +---- 3 files changed, 11 insertions(+), 8 deletions(-) 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 21c6c75cb2..f4d355333a 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala @@ -144,8 +144,8 @@ class PriorityMailboxSpec extends MailboxSpec { val comparator = PriorityGenerator(_.##) lazy val name = "The priority mailbox implementation" def factory = { - case UnboundedMailbox() ⇒ UnboundedPriorityMailbox(comparator).create(None) - case BoundedMailbox(capacity, pushTimeOut) ⇒ BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(None) + case UnboundedMailbox() ⇒ new UnboundedPriorityMailbox(comparator).create(None) + case BoundedMailbox(capacity, pushTimeOut) ⇒ new BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(None) } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 3099fe8f01..6ae7f3ddc7 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -380,14 +380,20 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat } } -case class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType { +/** + * Extend me to provide the comparator + */ +class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType { final override def create(owner: Option[ActorContext]): MessageQueue = new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final val queue = new PriorityBlockingQueue[Envelope](11, cmp) } } -case class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType { +/** + * Extend me to provide the comparator + */ +class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType { 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") diff --git a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala index fe8eb8a270..f26ac791d1 100644 --- a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala @@ -120,10 +120,7 @@ object DispatcherDocSpec { } // We create a new Priority dispatcher and seed it with the priority generator - class PrioMailbox(config: Config) extends MailboxType { - val priorityMailbox = UnboundedPriorityMailbox(generator) - def create(owner: Option[ActorContext]) = priorityMailbox.create(owner) - } + class PrioMailbox(config: Config) extends UnboundedPriorityMailbox(generator) //#prio-mailbox class MyActor extends Actor { From 98b74a974cb8bed79673a4cce50435122c5ab478 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 21 Feb 2012 18:24:38 +0100 Subject: [PATCH 35/46] Last minute awesomization of the priority based mailbox API docs --- .../dispatcher/DispatcherDocTestBase.java | 37 +++++++------------ .../docs/dispatcher/DispatcherDocSpec.scala | 15 ++++---- 2 files changed, 21 insertions(+), 31 deletions(-) diff --git a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java index d005ffcd6b..1aaa76ee11 100644 --- a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java +++ b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java @@ -123,30 +123,21 @@ public class DispatcherDocTestBase { } //#prio-mailbox - public static class PrioMailbox implements MailboxType { - - static final PriorityGenerator generator = new PriorityGenerator() { // Create a new PriorityGenerator, lower prio means more important - @Override - public int gen(Object message) { - if (message.equals("highpriority")) - return 0; // 'highpriority messages should be treated first if possible - else if (message.equals("lowpriority")) - return 100; // 'lowpriority messages should be treated last if possible - else if (message.equals(Actors.poisonPill())) - return 1000; // PoisonPill when no other left - else - return 50; // We default to 50 - } - }; - - private UnboundedPriorityMailbox priorityMailbox; - + public static class PrioMailbox extends UnboundedPriorityMailbox { public PrioMailbox(Config config) { // needed for reflective instantiation - priorityMailbox = new UnboundedPriorityMailbox(generator); - } - - public MessageQueue create(Option owner) { - return priorityMailbox.create(owner); + super(new PriorityGenerator() { // Create a new PriorityGenerator, lower prio means more important + @Override + public int gen(Object message) { + if (message.equals("highpriority")) + return 0; // 'highpriority messages should be treated first if possible + else if (message.equals("lowpriority")) + return 100; // 'lowpriority messages should be treated last if possible + else if (message.equals(Actors.poisonPill())) + return 1000; // PoisonPill when no other left + else + return 50; // We default to 50 + } + }); } } //#prio-mailbox diff --git a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala index f26ac791d1..6717ad96cf 100644 --- a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala @@ -112,15 +112,14 @@ object DispatcherDocSpec { import akka.actor.ActorContext import com.typesafe.config.Config - val generator = PriorityGenerator { // Create a new PriorityGenerator, lower prio means more important - case 'highpriority ⇒ 0 // 'highpriority messages should be treated first if possible - case 'lowpriority ⇒ 100 // 'lowpriority messages should be treated last if possible - case PoisonPill ⇒ 1000 // PoisonPill when no other left - case otherwise ⇒ 50 // We default to 50 - } - // We create a new Priority dispatcher and seed it with the priority generator - class PrioMailbox(config: Config) extends UnboundedPriorityMailbox(generator) + class PrioMailbox(config: Config) extends UnboundedPriorityMailbox( + PriorityGenerator { // Create a new PriorityGenerator, lower prio means more important + case 'highpriority ⇒ 0 // 'highpriority messages should be treated first if possible + case 'lowpriority ⇒ 100 // 'lowpriority messages should be treated last if possible + case PoisonPill ⇒ 1000 // PoisonPill when no other left + case otherwise ⇒ 50 // We default to 50 + }) //#prio-mailbox class MyActor extends Actor { From 4a47ce8ba34a5285f156e145295b465c6db80abd Mon Sep 17 00:00:00 2001 From: Adam Lesperance Date: Tue, 21 Feb 2012 13:35:07 -0600 Subject: [PATCH 36/46] Remove unneeded word "that". --- akka-docs/general/actors.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-docs/general/actors.rst b/akka-docs/general/actors.rst index 2d9ff67488..0b6e5e3f77 100644 --- a/akka-docs/general/actors.rst +++ b/akka-docs/general/actors.rst @@ -132,7 +132,7 @@ Once an actor terminates, i.e. fails in a way which is not handled by a restart, stops itself or is stopped by its supervisor, it will free up its resources, draining all remaining messages from its mailbox into the system’s “dead letter mailbox”. The mailbox is then replaced within the actor reference -with a that system mailbox, redirecting all new messages “into the drain”. This +with a system mailbox, redirecting all new messages “into the drain”. This is done on a best effort basis, though, so do not rely on it in order to construct “guaranteed delivery”. From 1cb657e02e0dc59b153c898aa9b33edcac843cee Mon Sep 17 00:00:00 2001 From: Adam Lesperance Date: Tue, 21 Feb 2012 14:39:08 -0600 Subject: [PATCH 37/46] Reword default config headers --- akka-actor/src/main/resources/reference.conf | 2 +- akka-agent/src/main/resources/reference.conf | 2 +- akka-cluster/src/main/resources/reference.conf | 2 +- .../akka-beanstalk-mailbox/src/main/resources/reference.conf | 2 +- .../akka-file-mailbox/src/main/resources/reference.conf | 2 +- .../akka-mongo-mailbox/src/main/resources/reference.conf | 2 +- .../akka-redis-mailbox/src/main/resources/reference.conf | 2 +- .../akka-zookeeper-mailbox/src/main/resources/reference.conf | 2 +- akka-remote/src/main/resources/reference.conf | 2 +- akka-testkit/src/main/resources/reference.conf | 2 +- akka-transactor/src/main/resources/reference.conf | 2 +- akka-zeromq/src/main/resources/reference.conf | 2 +- 12 files changed, 12 insertions(+), 12 deletions(-) diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 8240aee4d7..e1a603c971 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka Actor Reference Config File # #################################### -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-agent/src/main/resources/reference.conf b/akka-agent/src/main/resources/reference.conf index 2c3b121ba7..cf57b869de 100644 --- a/akka-agent/src/main/resources/reference.conf +++ b/akka-agent/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka Agent Reference Config File # #################################### -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-cluster/src/main/resources/reference.conf b/akka-cluster/src/main/resources/reference.conf index e097d34f3e..3142d548b5 100644 --- a/akka-cluster/src/main/resources/reference.conf +++ b/akka-cluster/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka Cluster Reference Config File # ###################################### -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/resources/reference.conf b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/resources/reference.conf index 82beeeddd8..6ec98a2114 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/resources/reference.conf +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka Beanstalk Mailboxes Reference Config File # ################################################## -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf b/akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf index 93ee52fcc7..9e7069f296 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka File Mailboxes Reference Config File # ############################################# -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/resources/reference.conf b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/resources/reference.conf index ad73037960..e4eb631727 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/resources/reference.conf +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka MongoDB Mailboxes Reference Config File # ################################################ -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-durable-mailboxes/akka-redis-mailbox/src/main/resources/reference.conf b/akka-durable-mailboxes/akka-redis-mailbox/src/main/resources/reference.conf index 7b12dc24b2..93c11a2975 100644 --- a/akka-durable-mailboxes/akka-redis-mailbox/src/main/resources/reference.conf +++ b/akka-durable-mailboxes/akka-redis-mailbox/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka Redis Mailboxes Reference Config File # ############################################## -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/resources/reference.conf b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/resources/reference.conf index 3dfea7a944..4f9a47b084 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/resources/reference.conf +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka ZooKepper Mailboxes Reference Config File # ################################################## -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 8ec036db0e..3d7a6050d0 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka Remote Reference Config File # ##################################### -# This the reference config file has all the default settings. +# This is the reference config file that contains 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- diff --git a/akka-testkit/src/main/resources/reference.conf b/akka-testkit/src/main/resources/reference.conf index 5e70df9403..17da88c22e 100644 --- a/akka-testkit/src/main/resources/reference.conf +++ b/akka-testkit/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka Testkit Reference Config File # ###################################### -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-transactor/src/main/resources/reference.conf b/akka-transactor/src/main/resources/reference.conf index d91a50db33..d1216bd73e 100644 --- a/akka-transactor/src/main/resources/reference.conf +++ b/akka-transactor/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka Transactor Reference Config File # ######################################### -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { diff --git a/akka-zeromq/src/main/resources/reference.conf b/akka-zeromq/src/main/resources/reference.conf index b94a442c78..ed52dc84e1 100644 --- a/akka-zeromq/src/main/resources/reference.conf +++ b/akka-zeromq/src/main/resources/reference.conf @@ -2,7 +2,7 @@ # Akka ZeroMQ Reference Config File # ##################################### -# This the reference config file has all the default settings. +# This is the reference config file that contains all the default settings. # Make your edits/overrides in your application.conf. akka { From 38d4d96ca35435a3701dfa50d8babfd3f31fe29e Mon Sep 17 00:00:00 2001 From: Adam Lesperance Date: Tue, 21 Feb 2012 14:51:43 -0600 Subject: [PATCH 38/46] Fix akka-remote default config's spelling/grammer/formatting --- akka-remote/src/main/resources/reference.conf | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 3d7a6050d0..1438904fe2 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -5,7 +5,7 @@ # This is the reference config file that contains 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- +# comments about akka.actor settings left out where they are already in akka- # actor.jar, because otherwise they would be repeated in config rendering. akka { @@ -60,7 +60,7 @@ akka { # untrusted clients to connect. untrusted-mode = off - # Timeout for ACK of cluster operations, lik checking actor out etc. + # Timeout for ACK of cluster operations, like checking actor out etc. remote-daemon-ack-timeout = 30s # If this is "on", Akka will log all inbound messages at DEBUG level, if off then they are not logged @@ -76,16 +76,16 @@ akka { # to client). netty { - # (O) In case of increased latency / overflow how long - # should we wait (blocking the sender) until we deem the send to be cancelled? + # (O) In case of increased latency / overflow how long should we wait (blocking the sender) + # until we deem the send to be cancelled? # 0 means "never backoff", any positive number will indicate time to block at most. backoff-timeout = 0ms # (I&O) Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh' - # or using 'akka.util.Crypt.generateSecureCookie' + # or using 'akka.util.Crypt.generateSecureCookie' secure-cookie = "" - # (I) Should the remote server require that it peers share the same secure-cookie + # (I) Should the remote server require that its peers share the same secure-cookie # (defined in the 'remote' section)? require-cookie = off From 5a344ac67d77314c4e7359202c5a3951ef43c271 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 22 Feb 2012 10:19:00 +0100 Subject: [PATCH 39/46] fix LoggingAdapter multi-arg handling, see #1856 --- .../src/main/scala/akka/event/Logging.scala | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index a58ea3fdb9..93019318dd 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -734,37 +734,37 @@ trait LoggingAdapter { */ def error(cause: Throwable, message: String) { if (isErrorEnabled) notifyError(cause, message) } - def error(cause: Throwable, template: String, arg1: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1)) } + def error(cause: Throwable, template: String, arg1: Any) { if (isErrorEnabled) notifyError(cause, format1(template, arg1)) } def error(cause: Throwable, template: String, arg1: Any, arg2: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2)) } def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3)) } def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3, arg4)) } def error(message: String) { if (isErrorEnabled) notifyError(message) } - def error(template: String, arg1: Any) { if (isErrorEnabled) notifyError(format(template, arg1)) } + def error(template: String, arg1: Any) { if (isErrorEnabled) notifyError(format1(template, arg1)) } def error(template: String, arg1: Any, arg2: Any) { if (isErrorEnabled) notifyError(format(template, arg1, arg2)) } def error(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3)) } def error(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3, arg4)) } def warning(message: String) { if (isWarningEnabled) notifyWarning(message) } - def warning(template: String, arg1: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1)) } + def warning(template: String, arg1: Any) { if (isWarningEnabled) notifyWarning(format1(template, arg1)) } def warning(template: String, arg1: Any, arg2: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2)) } def warning(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3)) } def warning(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3, arg4)) } def info(message: String) { if (isInfoEnabled) notifyInfo(message) } - def info(template: String, arg1: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1)) } + def info(template: String, arg1: Any) { if (isInfoEnabled) notifyInfo(format1(template, arg1)) } def info(template: String, arg1: Any, arg2: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2)) } def info(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3)) } def info(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3, arg4)) } def debug(message: String) { if (isDebugEnabled) notifyDebug(message) } - def debug(template: String, arg1: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1)) } + def debug(template: String, arg1: Any) { if (isDebugEnabled) notifyDebug(format1(template, arg1)) } def debug(template: String, arg1: Any, arg2: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2)) } def debug(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3)) } def debug(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3, arg4)) } def log(level: Logging.LogLevel, message: String) { if (isEnabled(level)) notifyLog(level, message) } - def log(level: Logging.LogLevel, template: String, arg1: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1)) } + def log(level: Logging.LogLevel, template: String, arg1: Any) { if (isEnabled(level)) notifyLog(level, format1(template, arg1)) } def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2)) } def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3)) } def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3, arg4)) } @@ -783,16 +783,28 @@ trait LoggingAdapter { case Logging.DebugLevel ⇒ if (isDebugEnabled) notifyDebug(message) } + private def format1(t: String, arg: Any) = arg match { + case a: Array[_] if !a.getClass.getComponentType.isPrimitive ⇒ format(t, a: _*) + case a: Array[_] ⇒ format(t, (a map (_.asInstanceOf[AnyRef]): _*)) + case x ⇒ format(t, x) + } + def format(t: String, arg: Any*) = { val sb = new StringBuilder var p = 0 var rest = t while (p < arg.length) { val index = rest.indexOf("{}") - sb.append(rest.substring(0, index)) - sb.append(arg(p)) - rest = rest.substring(index + 2) - p += 1 + if (index == -1) { + sb.append(rest).append(" WARNING arguments left: ").append(arg.length - p) + rest = "" + p = arg.length + } else { + sb.append(rest.substring(0, index)) + sb.append(arg(p)) + rest = rest.substring(index + 2) + p += 1 + } } sb.append(rest) sb.toString From ae4a1960ad461a8bd398850316132f3001ac4885 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 22 Feb 2012 11:00:00 +0100 Subject: [PATCH 40/46] document multi-arg logging, see #1856 --- .../code/akka/docs/event/LoggingDocTestBase.java | 12 +++++++++++- akka-docs/java/logging.rst | 9 +++++++-- .../code/akka/docs/event/LoggingDocSpec.scala | 7 +++++++ akka-docs/scala/logging.rst | 16 ++++++++++++++-- 4 files changed, 39 insertions(+), 5 deletions(-) diff --git a/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java b/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java index 1a1f7867c0..8f7b63d8a8 100644 --- a/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java +++ b/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java @@ -33,7 +33,7 @@ import akka.actor.DeadLetter; //#imports-deadletter public class LoggingDocTestBase { - + @Test public void useLoggingActor() { ActorSystem system = ActorSystem.create("MySystem"); @@ -55,6 +55,16 @@ public class LoggingDocTestBase { //#deadletters system.shutdown(); } + + @Test + public void demonstrateMultipleArgs() { + final ActorSystem system = ActorSystem.create("multiArg"); + //#array + final Object[] args = new Object[] { "The", "brown", "fox", "jumps", 42 }; + system.log().debug("five parameters: {}, {}, {}, {}, {}", args); + //#array + system.shutdown(); + } //#my-actor class MyActor extends UntypedActor { diff --git a/akka-docs/java/logging.rst b/akka-docs/java/logging.rst index fabd9f6e92..333640a41f 100644 --- a/akka-docs/java/logging.rst +++ b/akka-docs/java/logging.rst @@ -30,8 +30,13 @@ object is translated to a String according to the following rules: * in case of a class an approximation of its simpleName * and in all other cases the simpleName of its class -The log message may contain argument placeholders ``{}``, which will be substituted if the log level -is enabled. +The log message may contain argument placeholders ``{}``, which will be +substituted if the log level is enabled. Giving more arguments as there are +placeholders results in a warning being appended to the log statement (i.e. on +the same line with the same severity). You may pass a Java array as the only +substitution argument to have its elements be treated individually: + +.. includecode:: code/akka/docs/event/LoggingDocTestBase.java#array The Java :class:`Class` of the log source is also included in the generated :class:`LogEvent`. In case of a simple string this is replaced with a “marker” diff --git a/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala b/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala index cd6ecdb162..0aa29549c9 100644 --- a/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala @@ -89,4 +89,11 @@ class LoggingDocSpec extends AkkaSpec { //#deadletters } + "demonstrate logging more arguments" in { + //#array + val args = Array("The", "brown", "fox", "jumps", 42) + system.log.debug("five parameters: {}, {}, {}, {}, {}", args) + //#array + } + } diff --git a/akka-docs/scala/logging.rst b/akka-docs/scala/logging.rst index dd58b24fb4..1ccf9583dd 100644 --- a/akka-docs/scala/logging.rst +++ b/akka-docs/scala/logging.rst @@ -34,8 +34,20 @@ The source object is translated to a String according to the following rules: * and in all other cases a compile error occurs unless and implicit :class:`LogSource[T]` is in scope for the type in question. -The log message may contain argument placeholders ``{}``, which will be substituted if the log level -is enabled. +The log message may contain argument placeholders ``{}``, which will be +substituted if the log level is enabled. Giving more arguments as there are +placeholders results in a warning being appended to the log statement (i.e. on +the same line with the same severity). You may pass a Java array as the only +substitution argument to have its elements be treated individually: + +.. includecode:: code/akka/docs/event/LoggingDocSpec.scala#array + +The Java :class:`Class` of the log source is also included in the generated +:class:`LogEvent`. In case of a simple string this is replaced with a “marker” +class :class:`akka.event.DummyClassForStringSources` in order to allow special +treatment of this case, e.g. in the SLF4J event listener which will then use +the string instead of the class’ name for looking up the logger instance to +use. Auxiliary logging options ------------------------- From 0435b75d5c0c195732d789bb5234d4d5559333eb Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 22 Feb 2012 13:29:14 +0100 Subject: [PATCH 41/46] #1855 - JITing 0mq module --- akka-zeromq/src/main/scala/akka/zeromq/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/package.scala b/akka-zeromq/src/main/scala/akka/zeromq/package.scala index 812c3eef50..6eeba5b92a 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/package.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/package.scala @@ -15,7 +15,7 @@ package object zeromq { * @param system * @return An augmented [[akka.actor.ActorSystem]] */ - implicit def zeromqSystem(system: ActorSystem) = system.extension(ZeroMQExtension) + implicit def zeromqSystem(system: ActorSystem): ZeroMQExtension = ZeroMQExtension(system) /** * Convenience accessor to subscribe to all events From 7ce4765fec01119cf9a0f153280e6fb82513d2ee Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 22 Feb 2012 15:34:43 +0100 Subject: [PATCH 42/46] do newActor() and preRestart() in the right order, see #1854 --- .../scala/akka/actor/SupervisorMiscSpec.scala | 17 +++++++++++++++++ .../src/main/scala/akka/actor/ActorCell.scala | 2 +- 2 files changed, 18 insertions(+), 1 deletion(-) 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 caeacfb3db..f644842591 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -60,5 +60,22 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul assert(Await.result(actor4 ? "status", timeout.duration) == "OK", "actor4 is shutdown") } } + + "be able to create named children in its constructor" in { + val a = system.actorOf(Props(new Actor { + context.actorOf(Props.empty, "bob") + def receive = { + case x: Exception ⇒ throw x + } + override def preStart(): Unit = testActor ! "preStart" + })) + val m = "weird message" + EventFilter[Exception](m, occurrences = 1) intercept { + a ! new Exception(m) + } + expectMsg("preStart") + expectMsg("preStart") + a.isTerminated must be(false) + } } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 39bd52aa87..b46d95b669 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -389,7 +389,6 @@ private[akka] class ActorCell( def recreate(cause: Throwable): Unit = try { val failedActor = actor if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(failedActor), "restarting")) - val freshActor = newActor() if (failedActor ne null) { val c = currentMessage //One read only plz try { @@ -398,6 +397,7 @@ private[akka] class ActorCell( clearActorFields() } } + val freshActor = newActor() actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call freshActor.postRestart(cause) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(freshActor), "restarted")) From 848702aecbdf7a27c0b79fcdeac0a235de031613 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 22 Feb 2012 15:50:33 +0100 Subject: [PATCH 43/46] improve comments, see #1854 --- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index b46d95b669..e659928743 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -397,8 +397,8 @@ private[akka] class ActorCell( clearActorFields() } } - val freshActor = newActor() - actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call + val freshActor = newActor() // this must happen after failedActor.preRestart (to scrap those children) + actor = freshActor // this must happen before postRestart has a chance to fail freshActor.postRestart(cause) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(freshActor), "restarted")) From 2c5f65b0b2736a6c54b49aa8fd19c7b488870f26 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 24 Feb 2012 13:13:55 +0100 Subject: [PATCH 44/46] #1860 - Making all default messagequeues be the implementation, and therefor avoiding one level of indirection. --- .../src/main/scala/akka/dispatch/Mailbox.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 6ae7f3ddc7..29260a9f8d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -360,8 +360,8 @@ case class UnboundedMailbox() extends MailboxType { def this(config: Config) = this() final override def create(owner: Option[ActorContext]): MessageQueue = - new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { - final val queue = new ConcurrentLinkedQueue[Envelope]() + new ConcurrentLinkedQueue[Envelope]() with QueueBasedMessageQueue with UnboundedMessageQueueSemantics { + final def queue: Queue[Envelope] = this } } @@ -374,8 +374,8 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null") final override def create(owner: Option[ActorContext]): MessageQueue = - new QueueBasedMessageQueue with BoundedMessageQueueSemantics { - final val queue = new LinkedBlockingQueue[Envelope](capacity) + new LinkedBlockingQueue[Envelope](capacity) with QueueBasedMessageQueue with BoundedMessageQueueSemantics { + final def queue: BlockingQueue[Envelope] = this final val pushTimeOut = BoundedMailbox.this.pushTimeOut } } @@ -385,8 +385,8 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat */ class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType { final override def create(owner: Option[ActorContext]): MessageQueue = - new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { - final val queue = new PriorityBlockingQueue[Envelope](11, cmp) + new PriorityBlockingQueue[Envelope](11, cmp) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics { + final def queue: Queue[Envelope] = this } } @@ -399,8 +399,8 @@ class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val cap if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null") final override def create(owner: Option[ActorContext]): MessageQueue = - new QueueBasedMessageQueue with BoundedMessageQueueSemantics { - final val queue = new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) + new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) with QueueBasedMessageQueue with BoundedMessageQueueSemantics { + final def queue: BlockingQueue[Envelope] = this final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut } } From 1e7ce2bfc736cfa66c9701628d17ec212686e923 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 24 Feb 2012 14:28:17 +0100 Subject: [PATCH 45/46] #1859 - rewriting dispatcher docs --- .../dispatcher/DispatcherDocTestBase.java | 31 +-- akka-docs/java/dispatchers.rst | 233 +++++++----------- .../docs/dispatcher/DispatcherDocSpec.scala | 82 ++++-- akka-docs/scala/dispatchers.rst | 232 ++++++++--------- 4 files changed, 264 insertions(+), 314 deletions(-) diff --git a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java index 1aaa76ee11..14291fc27e 100644 --- a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java +++ b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java @@ -60,19 +60,17 @@ public class DispatcherDocTestBase { @Test public void defineDispatcher() { //#defining-dispatcher - ActorRef myActor1 = system.actorOf(new Props(MyUntypedActor.class).withDispatcher("my-dispatcher"), - "myactor1"); - ActorRef myActor2 = system.actorOf(new Props(MyUntypedActor.class).withDispatcher("my-dispatcher"), - "myactor2"); + ActorRef myActor = + system.actorOf(new Props(MyUntypedActor.class).withDispatcher("my-dispatcher"), + "myactor3"); //#defining-dispatcher } @Test public void definePinnedDispatcher() { //#defining-pinned-dispatcher - String name = "myactor"; ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class) - .withDispatcher("myactor-dispatcher"), name); + .withDispatcher("my-pinned-dispatcher")); //#defining-pinned-dispatcher } @@ -80,11 +78,13 @@ public class DispatcherDocTestBase { public void priorityDispatcher() throws Exception { //#prio-dispatcher - ActorRef myActor = system.actorOf( // We create a new Actor that just prints out what it processes + // We create a new Actor that just prints out what it processes + ActorRef myActor = system.actorOf( new Props().withCreator(new UntypedActorFactory() { public UntypedActor create() { return new UntypedActor() { - LoggingAdapter log = Logging.getLogger(getContext().system(), this); + LoggingAdapter log = + Logging.getLogger(getContext().system(), this); { getSelf().tell("lowpriority"); getSelf().tell("lowpriority"); @@ -101,7 +101,7 @@ public class DispatcherDocTestBase { } }; } - }).withDispatcher("prio-dispatcher-java")); + }).withDispatcher("prio-dispatcher")); /* Logs: @@ -123,19 +123,20 @@ public class DispatcherDocTestBase { } //#prio-mailbox - public static class PrioMailbox extends UnboundedPriorityMailbox { - public PrioMailbox(Config config) { // needed for reflective instantiation - super(new PriorityGenerator() { // Create a new PriorityGenerator, lower prio means more important + public static class MyPrioMailbox extends UnboundedPriorityMailbox { + public MyPrioMailbox(Config config) { // needed for reflective instantiation + // Create a new PriorityGenerator, lower prio means more important + super(new PriorityGenerator() { @Override public int gen(Object message) { if (message.equals("highpriority")) return 0; // 'highpriority messages should be treated first if possible else if (message.equals("lowpriority")) - return 100; // 'lowpriority messages should be treated last if possible + return 2; // 'lowpriority messages should be treated last if possible else if (message.equals(Actors.poisonPill())) - return 1000; // PoisonPill when no other left + return 3; // PoisonPill when no other left else - return 50; // We default to 50 + return 1; // By default they go between high and low prio } }); } diff --git a/akka-docs/java/dispatchers.rst b/akka-docs/java/dispatchers.rst index b7a16d9283..e602c2c77c 100644 --- a/akka-docs/java/dispatchers.rst +++ b/akka-docs/java/dispatchers.rst @@ -7,204 +7,157 @@ Dispatchers (Java) .. contents:: :local: -The Dispatcher is an important piece that allows you to configure the right semantics and parameters for optimal performance, throughput and scalability. Different Actors have different needs. - -Akka supports dispatchers for both event-driven lightweight threads, allowing creation of millions of threads on a single workstation, and thread-based Actors, where each dispatcher is bound to a dedicated OS thread. - -The event-based Actors currently consume ~600 bytes per Actor which means that you can create more than 6.5 million Actors on 4 GB RAM. +An Akka ``MessageDispatcher`` is what makes Akka Actors "tick", it is the engine of the machine so to speak. +All ``MessageDispatcher`` implementations are also an ``ExecutionContext``, which means that they can be used +to execute arbitrary code, for instance :ref:`futures-java`. Default dispatcher ------------------ -For most scenarios the default settings are the best. Here we have one single event-based dispatcher for all Actors created. -The default dispatcher is available from the ``ActorSystem.dispatcher`` and can be configured in the ``akka.actor.default-dispatcher`` -section of the :ref:`configuration`. +Every ``ActorSystem`` will have a default dispatcher that will be used in case nothing else is configured for an ``Actor``. +The default dispatcher can be configured, and is by default a ``Dispatcher`` with a "fork-join-executor", which gives excellent performance in most cases. -If you are starting to get contention on the single dispatcher (the ``Executor`` and its queue) or want to group a specific set of Actors -for a dedicated dispatcher for better flexibility and configurability then you can override the defaults and define your own dispatcher. -See below for details on which ones are available and how they can be configured. +Setting the dispatcher for an Actor +----------------------------------- -.. warning:: - Try to stick to a sensible default dispatcher, that means avoid using CallingThreadDispatcher, BalancingDispatcher or PinnedDispatcher - as the default-dispatcher. This is because they have very specific requirements from the environment in which they are used. +So in case you want to give your ``Actor`` a different dispatcher than the default, you need to do two things, of which the first is: -Setting the dispatcher ----------------------- +.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#defining-dispatcher -You specify the id of the dispatcher to use when creating an actor. The id corresponds to the :ref:`configuration` key -of the dispatcher settings. +.. note:: + The "dispatcherId" you specify in withDispatcher is in fact a path into your configuration. + So in this example it's a top-level section, but you could for instance put it as a sub-section, + where you'd use periods to denote sub-sections, like this: ``"foo.bar.my-dispatcher"`` -.. includecode:: code/akka/docs/dispatcher/DispatcherDocTestBase.java - :include: imports,defining-dispatcher +And then you just need to configure that dispatcher in your configuration: + +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config + +And here's another example that uses the "thread-pool-executor": + +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config + +For more options, see the default-dispatcher section of the :ref:`configuration`. Types of dispatchers -------------------- There are 4 different types of message dispatchers: -* Thread-based (Pinned) -* Event-based -* Priority event-based -* Work-sharing (Balancing) +* Dispatcher -It is recommended to define the dispatcher in :ref:`configuration` to allow for tuning for different environments. + - Sharability: Unlimited -Example of a custom event-based dispatcher, which can be used with -``new Props().withCreator(MyUntypedActor.class).withDispatcher("my-dispatcher")`` -as in the example above: + - Mailboxes: Any, creates one per Actor -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config + - Use cases: Default dispatcher, Bulkheading -Default values are taken from ``default-dispatcher``, i.e. all options doesn't need to be defined. See -:ref:`configuration` for the default values of the ``default-dispatcher``. You can also override -the values for the ``default-dispatcher`` in your configuration. + - Driven by: ``java.util.concurrent.ExecutorService`` + specify using "executor" using "fork-join-executor", + "thread-pool-executor" or the FQCN of + an ``akka.dispatcher.ExecutorServiceConfigurator`` -.. note:: +* PinnedDispatcher - 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"`` + - Sharability: None -There are two different executor services: + - Mailboxes: Any, creates one per Actor -* 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``. + - Use cases: Bulkheading -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``: + - Driven by: Any ``akka.dispatch.ThreadPoolExecutorConfigurator`` + by default a "thread-pool-executor" -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config +* BalancingDispatcher -Let's now walk through the different dispatchers in more detail. + - Sharability: Actors of the same type only -Thread-based -^^^^^^^^^^^^ + - Mailboxes: Any, creates one for all Actors -The ``PinnedDispatcher`` binds a dedicated OS thread to each specific Actor. The messages are posted to a -`LinkedBlockingQueue `_ -which feeds the messages to the dispatcher one by one. A ``PinnedDispatcher`` cannot be shared between actors. This dispatcher -has worse performance and scalability than the event-based dispatcher but works great for creating "daemon" Actors that consumes -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. + - Use cases: Work-sharing -The ``PinnedDispatcher`` is configured like this: + - Driven by: ``java.util.concurrent.ExecutorService`` + specify using "executor" using "fork-join-executor", + "thread-pool-executor" or the FQCN of + an ``akka.dispatcher.ExecutorServiceConfigurator`` + +* CallingThreadDispatcher + + - Sharability: Unlimited + + - Mailboxes: Any, creates one per Actor per Thread (on demand) + + - Use cases: Testing + + - Driven by: The calling thread (duh) + +More dispatcher configuration examples +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Configuring a ``PinnedDispatcher``: .. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-pinned-dispatcher-config -Note that it must be used with ``executor = "thread-pool-executor"``. +And then using it: -Event-based -^^^^^^^^^^^ +.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#defining-pinned-dispatcher -The event-based ``Dispatcher`` binds a set of Actors to a thread pool backed up by a -`BlockingQueue `_. This dispatcher is highly configurable -and supports a fluent configuration API to configure the ``BlockingQueue`` (type of queue, max items etc.) as well as the thread pool. +Mailboxes +--------- -The event-driven dispatchers **must be shared** between multiple Actors. One best practice is to let each top-level Actor, e.g. -the Actors you create from ``system.actorOf`` to get their own dispatcher but reuse the dispatcher for each new Actor -that the top-level Actor creates. But you can also share dispatcher between multiple top-level Actors. This is very use-case specific -and needs to be tried out on a case by case basis. The important thing is that Akka tries to provide you with the freedom you need to -design and implement your system in the most efficient way in regards to performance, throughput and latency. +An Akka ``Mailbox`` holds the messages that are destined for an ``Actor``. +Normally each ``Actor`` has its own mailbox, but with example a ``BalancingDispatcher`` all actors with the same ``BalancingDispatcher`` will share a single instance. -It comes with many different predefined BlockingQueue configurations: +Builtin implementations +^^^^^^^^^^^^^^^^^^^^^^^ -* Bounded `LinkedBlockingQueue `_ -* Unbounded `LinkedBlockingQueue `_ -* Bounded `ArrayBlockingQueue `_ -* Unbounded `ArrayBlockingQueue `_ -* `SynchronousQueue `_ +Akka comes shipped with a number of default mailbox implementations: -When using a bounded queue and it has grown up to limit defined the message processing will run in the caller's -thread as a way to slow him down and balance producer/consumer. +* UnboundedMailbox -Here is an example of a bounded mailbox: + - Backed by a ``java.util.concurrent.ConcurrentLinkedQueue`` -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-bounded-config + - Blocking: No -The standard :class:`Dispatcher` allows you to define the ``throughput`` it -should have, as shown above. This defines the number of messages for a specific -Actor the dispatcher should process in one single sweep; in other words, the -dispatcher will batch process up to ``throughput`` messages together when -having elected an actor to run. Setting this to a higher number will increase -throughput but lower fairness, and vice versa. If you don't specify it explicitly -then it uses the value (5) defined for ``default-dispatcher`` in the :ref:`configuration`. + - Bounded: No -Browse the `ScalaDoc `_ or look at the code for all the options available. +* BoundedMailbox -Priority event-based -^^^^^^^^^^^^^^^^^^^^ + - Backed by a ``java.util.concurrent.LinkedBlockingQueue`` -Sometimes it's useful to be able to specify priority order of messages, that is done by using Dispatcher and supply -an UnboundedPriorityMailbox or BoundedPriorityMailbox with a ``java.util.Comparator[Envelope]`` or use a -``akka.dispatch.PriorityGenerator`` (recommended). + - Blocking: Yes -Creating a Dispatcher with a mailbox using PriorityGenerator: + - Bounded: Yes -Config: +* UnboundedPriorityMailbox -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala - :include: prio-dispatcher-config-java + - Backed by a ``java.util.concurrent.PriorityBlockingQueue`` -Priority mailbox: + - Blocking: Yes -.. includecode:: code/akka/docs/dispatcher/DispatcherDocTestBase.java - :include: imports-prio-mailbox,prio-mailbox + - Bounded: No -Usage: +* BoundedPriorityMailbox -.. includecode:: code/akka/docs/dispatcher/DispatcherDocTestBase.java - :include: imports-prio,prio-dispatcher + - Backed by a ``java.util.PriorityBlockingQueue`` wrapped in an ``akka.util.BoundedBlockingQueue`` + - Blocking: Yes -Work-sharing event-based -^^^^^^^^^^^^^^^^^^^^^^^^^ + - Bounded: Yes -The ``BalancingDispatcher`` is a variation of the ``Dispatcher`` in which Actors of the same type can be set up to -share this dispatcher and during execution time the different actors will steal messages from other actors if they -have less messages to process. -Although the technique used in this implementation is commonly known as "work stealing", the actual implementation is probably -best described as "work donating" because the actor of which work is being stolen takes the initiative. -This can be a great way to improve throughput at the cost of a little higher latency. +* Durable mailboxes, see :ref:`durable-mailboxes`. -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-balancing-config +Mailbox configuration examples +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -Here is an article with some more information: `Load Balancing Actors with Work Stealing Techniques `_ -Here is another article discussing this particular dispatcher: `Flexible load balancing with Akka in Scala `_ +How to create a PriorityMailbox: -Making the Actor mailbox bounded --------------------------------- +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherTestBase.java#prio-mailbox -Global configuration -^^^^^^^^^^^^^^^^^^^^ +And then add it to the configuration: -You can make the Actor mailbox bounded by a capacity in two ways. Either you define it in the :ref:`configuration` file under -``default-dispatcher``. This will set it globally as default for the DefaultDispatcher and for other configured dispatchers, -if not specified otherwise. +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher-config -.. code-block:: ruby - - akka { - actor { - default-dispatcher { - # If negative (or zero) then an unbounded mailbox is used (default) - # If positive then a bounded mailbox is used and the capacity is set to the number specified - mailbox-capacity = 1000 - } - } - } - -Per-instance based configuration -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -You can also do it on a specific dispatcher instance. - -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-bounded-config - - -For the ``PinnedDispatcher``, it is non-shareable between actors, and associates a dedicated Thread with the actor. -Making it bounded (by specifying a capacity) is optional, but if you do, you need to provide a pushTimeout (default is 10 seconds). -When trying to send a message to the Actor it will throw a MessageQueueAppendFailedException("BlockingMessageTransferQueue transfer timed out") -if the message cannot be added to the mailbox within the time specified by the pushTimeout. +And then an example on how you would use it: +.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#prio-dispatcher \ No newline at end of file diff --git a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala index 6717ad96cf..32ecfa1705 100644 --- a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala @@ -6,15 +6,10 @@ package akka.docs.dispatcher import org.scalatest.{ BeforeAndAfterAll, WordSpec } import org.scalatest.matchers.MustMatchers import akka.testkit.AkkaSpec -import akka.actor.Props -import akka.actor.Actor import akka.event.Logging import akka.event.LoggingAdapter import akka.util.duration._ -import akka.actor.PoisonPill -import akka.dispatch.MessageDispatcherConfigurator -import akka.dispatch.MessageDispatcher -import akka.dispatch.DispatcherPrerequisites +import akka.actor.{ Props, Actor, PoisonPill } object DispatcherDocSpec { val config = """ @@ -33,8 +28,9 @@ object DispatcherDocSpec { # 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 defines the maximum number of messages to be + # processed per actor before the thread jumps to the next actor. + # Set to 1 for as fair as possible. throughput = 100 } //#my-dispatcher-config @@ -54,8 +50,9 @@ object DispatcherDocSpec { # maximum number of threads to cap factor-based number to core-pool-size-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 defines the maximum number of messages to be + # processed per actor before the thread jumps to the next actor. + # Set to 1 for as fair as possible. throughput = 100 } //#my-thread-pool-dispatcher-config @@ -94,13 +91,14 @@ object DispatcherDocSpec { //#prio-dispatcher-config prio-dispatcher { - mailbox-type = "akka.docs.dispatcher.DispatcherDocSpec$PrioMailbox" + mailbox-type = "akka.docs.dispatcher.DispatcherDocSpec$MyPrioMailbox" } //#prio-dispatcher-config //#prio-dispatcher-config-java prio-dispatcher-java { - mailbox-type = "akka.docs.dispatcher.DispatcherDocTestBase$PrioMailbox" + mailbox-type = "akka.docs.dispatcher.DispatcherDocTestBase$MyPrioMailbox" + //Other dispatcher configuration goes here } //#prio-dispatcher-config-java """ @@ -108,17 +106,24 @@ object DispatcherDocSpec { //#prio-mailbox import akka.dispatch.PriorityGenerator import akka.dispatch.UnboundedPriorityMailbox - import akka.dispatch.MailboxType - import akka.actor.ActorContext import com.typesafe.config.Config - // We create a new Priority dispatcher and seed it with the priority generator - class PrioMailbox(config: Config) extends UnboundedPriorityMailbox( - PriorityGenerator { // Create a new PriorityGenerator, lower prio means more important - case 'highpriority ⇒ 0 // 'highpriority messages should be treated first if possible - case 'lowpriority ⇒ 100 // 'lowpriority messages should be treated last if possible - case PoisonPill ⇒ 1000 // PoisonPill when no other left - case otherwise ⇒ 50 // We default to 50 + // We inherit, in this case, from UnboundedPriorityMailbox + // and seed it with the priority generator + class MyPrioMailbox(config: Config) extends UnboundedPriorityMailbox( + // Create a new PriorityGenerator, lower prio means more important + PriorityGenerator { + // 'highpriority messages should be treated first if possible + case 'highpriority ⇒ 0 + + // 'lowpriority messages should be treated last if possible + case 'lowpriority ⇒ 2 + + // PoisonPill when no other left + case PoisonPill ⇒ 3 + + // We default to 1, which is in between high and low + case otherwise ⇒ 1 }) //#prio-mailbox @@ -127,6 +132,29 @@ object DispatcherDocSpec { case x ⇒ } } + + //#mailbox-implementation-example + case class MyUnboundedMailbox() extends akka.dispatch.MailboxType { + import akka.actor.ActorContext + import com.typesafe.config.Config + import java.util.concurrent.ConcurrentLinkedQueue + import akka.dispatch.{ + Envelope, + MessageQueue, + QueueBasedMessageQueue, + UnboundedMessageQueueSemantics + } + + // This constructor signature must exist, it will be called by Akka + def this(config: Config) = this() + + // The create method is called to create the MessageQueue + final override def create(owner: Option[ActorContext]): MessageQueue = + new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { + final val queue = new ConcurrentLinkedQueue[Envelope]() + } + //#mailbox-implementation-example + } } class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) { @@ -134,10 +162,11 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) { import DispatcherDocSpec.MyActor "defining dispatcher" in { + val context = system //#defining-dispatcher import akka.actor.Props - val myActor1 = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), name = "myactor1") - val myActor2 = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), name = "myactor2") + val myActor = + context.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), "myactor1") //#defining-dispatcher } @@ -146,15 +175,18 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) { } "defining pinned dispatcher" in { + val context = system //#defining-pinned-dispatcher - val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), name = "myactor") + val myActor = + context.actorOf(Props[MyActor].withDispatcher("my-pinned-dispatcher"), "myactor2") //#defining-pinned-dispatcher } "defining priority dispatcher" in { //#prio-dispatcher - val a = system.actorOf( // We create a new Actor that just prints out what it processes + // We create a new Actor that just prints out what it processes + val a = system.actorOf( Props(new Actor { val log: LoggingAdapter = Logging(context.system, this) diff --git a/akka-docs/scala/dispatchers.rst b/akka-docs/scala/dispatchers.rst index 94ae563e64..18169fb2ef 100644 --- a/akka-docs/scala/dispatchers.rst +++ b/akka-docs/scala/dispatchers.rst @@ -7,202 +7,166 @@ Dispatchers (Scala) .. contents:: :local: -The Dispatcher is an important piece that allows you to configure the right semantics and parameters for optimal performance, throughput and scalability. Different Actors have different needs. - -Akka supports dispatchers for both event-driven lightweight threads, allowing creation of millions of threads on a single workstation, and thread-based Actors, where each dispatcher is bound to a dedicated OS thread. - -The event-based Actors currently consume ~600 bytes per Actor which means that you can create more than 6.5 million Actors on 4 GB RAM. +An Akka ``MessageDispatcher`` is what makes Akka Actors "tick", it is the engine of the machine so to speak. +All ``MessageDispatcher`` implementations are also an ``ExecutionContext``, which means that they can be used +to execute arbitrary code, for instance :ref:`futures-scala`. Default dispatcher ------------------ -For most scenarios the default settings are the best. Here we have one single event-based dispatcher for all Actors created. -The default dispatcher is available from the ``ActorSystem.dispatcher`` and can be configured in the ``akka.actor.default-dispatcher`` -section of the :ref:`configuration`. +Every ``ActorSystem`` will have a default dispatcher that will be used in case nothing else is configured for an ``Actor``. +The default dispatcher can be configured, and is by default a ``Dispatcher`` with a "fork-join-executor", which gives excellent performance in most cases. -If you are starting to get contention on the single dispatcher (the ``Executor`` and its queue) or want to group a specific set of Actors -for a dedicated dispatcher for better flexibility and configurability then you can override the defaults and define your own dispatcher. -See below for details on which ones are available and how they can be configured. +Setting the dispatcher for an Actor +----------------------------------- -.. warning:: - Try to stick to a sensible default dispatcher, that means avoid using CallingThreadDispatcher, BalancingDispatcher or PinnedDispatcher - as the default-dispatcher. This is because they have very specific requirements from the environment in which they are used. +So in case you want to give your ``Actor`` a different dispatcher than the default, you need to do two things, of which the first is: -Setting the dispatcher ----------------------- +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#defining-dispatcher -You specify the id of the dispatcher to use when creating an actor. The id corresponds to the :ref:`configuration` key -of the dispatcher settings. +.. note:: + The "dispatcherId" you specify in withDispatcher is in fact a path into your configuration. + So in this example it's a top-level section, but you could for instance put it as a sub-section, + where you'd use periods to denote sub-sections, like this: ``"foo.bar.my-dispatcher"`` -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala - :include: imports,defining-dispatcher +And then you just need to configure that dispatcher in your configuration: + +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config + +And here's another example that uses the "thread-pool-executor": + +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config + +For more options, see the default-dispatcher section of the :ref:`configuration`. Types of dispatchers -------------------- There are 4 different types of message dispatchers: -* Thread-based (Pinned) -* Event-based -* Priority event-based -* Work-sharing (Balancing) +* Dispatcher -It is recommended to define the dispatcher in :ref:`configuration` to allow for tuning for different environments. + - Sharability: Unlimited -Example of a custom event-based dispatcher, which can be used with ``Props[MyActor].withDispatcher("my-dispatcher")`` -as in the example above: + - Mailboxes: Any, creates one per Actor -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config + - Use cases: Default dispatcher, Bulkheading -Default values are taken from ``default-dispatcher``, i.e. all options doesn't need to be defined. See -:ref:`configuration` for the default values of the ``default-dispatcher``. You can also override -the values for the ``default-dispatcher`` in your configuration. + - Driven by: ``java.util.concurrent.ExecutorService`` + specify using "executor" using "fork-join-executor", + "thread-pool-executor" or the FQCN of + an ``akka.dispatcher.ExecutorServiceConfigurator`` -.. note:: +* PinnedDispatcher - 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"`` + - Sharability: None -There are two different executor services: + - Mailboxes: Any, creates one per Actor -* 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``. + - Use cases: Bulkheading -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``: + - Driven by: Any ``akka.dispatch.ThreadPoolExecutorConfigurator`` + by default a "thread-pool-executor" -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config +* BalancingDispatcher -Let's now walk through the different dispatchers in more detail. + - Sharability: Actors of the same type only -Thread-based -^^^^^^^^^^^^ + - Mailboxes: Any, creates one for all Actors -The ``PinnedDispatcher`` binds a dedicated OS thread to each specific Actor. The messages are posted to a -`LinkedBlockingQueue `_ -which feeds the messages to the dispatcher one by one. A ``PinnedDispatcher`` cannot be shared between actors. This dispatcher -has worse performance and scalability than the event-based dispatcher but works great for creating "daemon" Actors that consumes -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. + - Use cases: Work-sharing -The ``PinnedDispatcher`` is configured like this: + - Driven by: ``java.util.concurrent.ExecutorService`` + specify using "executor" using "fork-join-executor", + "thread-pool-executor" or the FQCN of + an ``akka.dispatcher.ExecutorServiceConfigurator`` -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-pinned-dispatcher-config +* CallingThreadDispatcher -Note that it must be used with ``executor = "thread-pool-executor"``. + - Sharability: Unlimited -Event-based -^^^^^^^^^^^ + - Mailboxes: Any, creates one per Actor per Thread (on demand) -The event-based ``Dispatcher`` binds a set of Actors to a thread pool backed up by a -`BlockingQueue `_. This dispatcher is highly configurable -and supports a fluent configuration API to configure the ``BlockingQueue`` (type of queue, max items etc.) as well as the thread pool. + - Use cases: Testing -The event-driven dispatchers **must be shared** between multiple Actors. One best practice is to let each top-level Actor, e.g. -the Actors you create from ``system.actorOf`` to get their own dispatcher but reuse the dispatcher for each new Actor -that the top-level Actor creates. But you can also share dispatcher between multiple top-level Actors. This is very use-case specific -and needs to be tried out on a case by case basis. The important thing is that Akka tries to provide you with the freedom you need to -design and implement your system in the most efficient way in regards to performance, throughput and latency. + - Driven by: The calling thread (duh) -It comes with many different predefined BlockingQueue configurations: +More dispatcher configuration examples +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -* Bounded `LinkedBlockingQueue `_ -* Unbounded `LinkedBlockingQueue `_ -* Bounded `ArrayBlockingQueue `_ -* Unbounded `ArrayBlockingQueue `_ -* `SynchronousQueue `_ +Configuring a ``PinnedDispatcher``: -When using a bounded queue and it has grown up to limit defined the message processing will run in the caller's -thread as a way to slow him down and balance producer/consumer. +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-pinned-dispatcher-config -Here is an example of a bounded mailbox: +And then using it: -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-bounded-config +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#defining-pinned-dispatcher -The standard :class:`Dispatcher` allows you to define the ``throughput`` it -should have, as shown above. This defines the number of messages for a specific -Actor the dispatcher should process in one single sweep; in other words, the -dispatcher will batch process up to ``throughput`` messages together when -having elected an actor to run. Setting this to a higher number will increase -throughput but lower fairness, and vice versa. If you don't specify it explicitly -then it uses the value (5) defined for ``default-dispatcher`` in the :ref:`configuration`. +Mailboxes +--------- -Browse the `ScalaDoc `_ or look at the code for all the options available. +An Akka ``Mailbox`` holds the messages that are destined for an ``Actor``. +Normally each ``Actor`` has its own mailbox, but with example a ``BalancingDispatcher`` all actors with the same ``BalancingDispatcher`` will share a single instance. -Priority event-based -^^^^^^^^^^^^^^^^^^^^ +Builtin implementations +^^^^^^^^^^^^^^^^^^^^^^^ -Sometimes it's useful to be able to specify priority order of messages, that is done by using Dispatcher and supply -an UnboundedPriorityMailbox or BoundedPriorityMailbox with a ``java.util.Comparator[Envelope]`` or use a -``akka.dispatch.PriorityGenerator`` (recommended). +Akka comes shipped with a number of default mailbox implementations: -Creating a Dispatcher with a mailbox using PriorityGenerator: +* UnboundedMailbox -Config: + - Backed by a ``java.util.concurrent.ConcurrentLinkedQueue`` -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala - :include: prio-dispatcher-config + - Blocking: No -Priority mailbox: + - Bounded: No -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala - :include: prio-mailbox +* BoundedMailbox -Usage: + - Backed by a ``java.util.concurrent.LinkedBlockingQueue`` -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala - :include: prio-dispatcher + - Blocking: Yes -Work-sharing event-based -^^^^^^^^^^^^^^^^^^^^^^^^^ + - Bounded: Yes -The ``BalancingDispatcher`` is a variation of the ``Dispatcher`` in which Actors of the same type can be set up to -share this dispatcher and during execution time the different actors will steal messages from other actors if they -have less messages to process. -Although the technique used in this implementation is commonly known as "work stealing", the actual implementation is probably -best described as "work donating" because the actor of which work is being stolen takes the initiative. -This can be a great way to improve throughput at the cost of a little higher latency. +* UnboundedPriorityMailbox -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-balancing-config + - Backed by a ``java.util.concurrent.PriorityBlockingQueue`` -Here is an article with some more information: `Load Balancing Actors with Work Stealing Techniques `_ -Here is another article discussing this particular dispatcher: `Flexible load balancing with Akka in Scala `_ + - Blocking: Yes -Making the Actor mailbox bounded --------------------------------- + - Bounded: No -Global configuration -^^^^^^^^^^^^^^^^^^^^ +* BoundedPriorityMailbox -You can make the Actor mailbox bounded by a capacity in two ways. Either you define it in the :ref:`configuration` file under -``default-dispatcher``. This will set it globally as default for the DefaultDispatcher and for other configured dispatchers, -if not specified otherwise. + - Backed by a ``java.util.PriorityBlockingQueue`` wrapped in an ``akka.util.BoundedBlockingQueue`` -.. code-block:: ruby + - Blocking: Yes - akka { - actor { - default-dispatcher { - # If negative (or zero) then an unbounded mailbox is used (default) - # If positive then a bounded mailbox is used and the capacity is set to the number specified - mailbox-capacity = 1000 - } - } - } + - Bounded: Yes -Per-instance based configuration -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +* Durable mailboxes, see :ref:`durable-mailboxes`. -You can also do it on a specific dispatcher instance. +Mailbox configuration examples +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-bounded-config +How to create a PriorityMailbox: +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#prio-mailbox -For the ``PinnedDispatcher``, it is non-shareable between actors, and associates a dedicated Thread with the actor. -Making it bounded (by specifying a capacity) is optional, but if you do, you need to provide a pushTimeout (default is 10 seconds). -When trying to send a message to the Actor it will throw a MessageQueueAppendFailedException("BlockingMessageTransferQueue transfer timed out") -if the message cannot be added to the mailbox within the time specified by the pushTimeout. +And then add it to the configuration: +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher-config + +And then an example on how you would use it: + +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher + +Creating your own Mailbox type +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +An example is worth a thousand quacks: + +.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#mailbox-implementation-example + +And then you just specify the FQCN of your MailboxType as the value of the "mailbox-type" in the dispatcher configuration. \ No newline at end of file From 29303f28a4453ec457efa0cc6cd54b788feb685a Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 24 Feb 2012 14:36:39 +0100 Subject: [PATCH 46/46] Adding Gatling to the third-party integrations --- akka-docs/additional/third-party-integrations.rst | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/akka-docs/additional/third-party-integrations.rst b/akka-docs/additional/third-party-integrations.rst index 93b3b36d2e..fea3b0ec13 100644 --- a/akka-docs/additional/third-party-integrations.rst +++ b/akka-docs/additional/third-party-integrations.rst @@ -15,3 +15,9 @@ Scalatra has Akka integration. Read more here: ``_ +Gatling +------- + +Gatling is an Open Source Stress Tool. + +Read more here: ``_ \ No newline at end of file