From c7d58c600b878ab9bd63dfee51d6ee7a0181f683 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 26 Aug 2011 17:25:18 +0200 Subject: [PATCH] Adding initial support for Props --- .../java/akka/dispatch/JavaFutureTests.java | 23 + .../ActorFireForgetRequestReplySpec.scala | 6 +- .../scala/akka/actor/actor/ActorRefSpec.scala | 72 +- .../akka/actor/actor/ActorRestartSpec.scala | 24 +- .../akka/actor/actor/ActorTimeoutSpec.scala | 26 +- .../scala/akka/actor/actor/DeployerSpec.scala | 2 +- .../akka/actor/actor/FSMTimingSpec.scala | 14 +- .../akka/actor/actor/FSMTransitionSpec.scala | 5 +- .../akka/actor/actor/ForwardActorSpec.scala | 73 +- .../test/scala/akka/actor/actor/IOActor.scala | 5 +- .../akka/actor/actor/LoggingReceiveSpec.scala | 9 +- .../supervisor/RestartStrategySpec.scala | 45 +- .../supervisor/SupervisorHierarchySpec.scala | 24 +- .../actor/supervisor/SupervisorMiscSpec.scala | 22 +- .../actor/supervisor/SupervisorSpec.scala | 16 +- .../actor/supervisor/SupervisorTreeSpec.scala | 11 +- .../scala/akka/dispatch/ActorModelSpec.scala | 26 +- .../dispatch/BalancingDispatcherSpec.scala | 34 +- .../akka/dispatch/DispatcherActorSpec.scala | 56 +- .../test/scala/akka/dispatch/FutureSpec.scala | 14 + .../akka/dispatch/MailboxConfigSpec.scala | 4 +- .../scala/akka/dispatch/PinnedActorSpec.scala | 27 +- .../dispatch/PriorityDispatcherSpec.scala | 8 +- .../scala/akka/misc/ActorRegistrySpec.scala | 15 +- .../test/scala/akka/misc/SchedulerSpec.scala | 2 - .../trading/common/AkkaPerformanceTest.scala | 9 +- .../trading/common/MatchingEngine.scala | 6 +- .../trading/common/OrderReceiver.scala | 7 +- .../trading/common/TradingSystem.scala | 14 +- .../trading/oneway/OneWayMatchingEngine.scala | 3 +- .../trading/oneway/OneWayOrderReceiver.scala | 3 +- .../oneway/OneWayPerformanceTest.scala | 8 +- .../trading/oneway/OneWayTradingSystem.scala | 14 +- .../scala/akka/routing/ActorPoolSpec.scala | 63 +- .../test/scala/akka/routing/RoutingSpec.scala | 1 - .../scala/akka/ticket/Ticket703Spec.scala | 5 +- .../src/main/scala/akka/actor/Actor.scala | 53 +- .../src/main/scala/akka/actor/ActorRef.scala | 740 +++++++----------- .../src/main/scala/akka/actor/Channel.scala | 4 +- .../src/main/scala/akka/actor/Deployer.scala | 6 +- .../scala/akka/actor/DeploymentConfig.scala | 19 +- akka-actor/src/main/scala/akka/actor/IO.scala | 2 +- .../main/scala/akka/actor/Supervisor.scala | 63 +- .../main/scala/akka/actor/TypedActor.scala | 7 +- .../main/scala/akka/actor/UntypedActor.scala | 4 +- .../src/main/scala/akka/actor/package.scala | 7 +- .../akka/dispatch/BalancingDispatcher.scala | 15 +- .../main/scala/akka/dispatch/Dispatcher.scala | 19 +- .../src/main/scala/akka/dispatch/Future.scala | 43 +- .../scala/akka/dispatch/MessageHandling.scala | 22 +- .../akka/dispatch/PinnedDispatcher.scala | 6 +- .../main/scala/akka/event/EventHandler.scala | 2 +- .../src/main/scala/akka/japi/JavaAPI.scala | 12 +- .../remoteinterface/RemoteEventHandler.scala | 3 - .../src/main/scala/akka/routing/Pool.scala | 50 +- .../src/main/scala/akka/routing/Routing.scala | 83 +- .../main/scala/akka/camel/TypedConsumer.scala | 11 +- .../src/main/scala/akka/camel/Consumer.scala | 14 +- .../akka/camel/component/ActorComponent.scala | 25 +- .../scala/akka/camel/CamelTestSupport.scala | 1 - .../camel/ConsumerPublishRequestorTest.scala | 8 +- .../akka/camel/ConsumerRegisteredTest.scala | 7 +- .../scala/akka/camel/ConsumerScalaTest.scala | 5 +- .../akka/camel/ProducerFeatureTest.scala | 11 +- .../component/ActorComponentFeatureTest.scala | 4 +- .../camel/component/ActorProducerTest.scala | 3 +- .../src/main/scala/akka/cluster/Cluster.scala | 67 +- .../scala/akka/cluster/ClusterActorRef.scala | 9 +- .../serialization/SerializationProtocol.scala | 149 ++-- .../serialization/ActorSerializeSpec.scala | 23 +- akka-docs/cluster/durable-mailbox.rst | 3 +- akka-docs/general/jmm.rst | 17 +- .../intro/getting-started-first-java.rst | 10 +- .../getting-started-first-scala-eclipse.rst | 47 +- .../intro/getting-started-first-scala.rst | 55 +- akka-docs/intro/getting-started.rst | 50 +- akka-docs/java/fault-tolerance.rst | 5 +- akka-docs/scala/fault-tolerance.rst | 4 +- akka-docs/scala/index.rst | 1 - .../actor/mailbox/BeanstalkBasedMailbox.scala | 4 +- .../actor/mailbox/FiledBasedMailbox.scala | 4 +- .../actor/mailbox/DurableDispatcher.scala | 18 +- .../akka/actor/mailbox/DurableMailbox.scala | 26 +- .../actor/mailbox/DurableMailboxSpec.scala | 12 +- .../actor/mailbox/BSONSerialization.scala | 33 +- .../actor/mailbox/MongoBasedMailbox.scala | 4 +- .../actor/mailbox/MongoDurableMessage.scala | 4 +- .../actor/mailbox/RedisBasedMailbox.scala | 4 +- .../actor/mailbox/ZooKeeperBasedMailbox.scala | 6 +- .../mailbox/ZooKeeperBasedMailboxSpec.scala | 12 +- akka-http/src/main/scala/akka/http/Mist.scala | 13 +- .../src/main/scala/ChatServer.scala | 2 +- .../main/scala/akka/event/slf4j/SLF4J.scala | 2 - .../src/main/scala/akka/agent/Agent.scala | 11 +- .../testkit/CallingThreadDispatcher.scala | 14 +- .../scala/akka/testkit/TestActorRef.scala | 31 +- .../main/scala/akka/testkit/TestFSMRef.scala | 8 +- .../src/main/scala/akka/testkit/TestKit.scala | 4 +- .../scala/akka/testkit/TestActorRefSpec.scala | 35 +- .../java/akka/tutorial/first/java/Pi.java | 2 +- .../java/akka/tutorial/java/second/Pi.java | 2 +- project/AkkaBuild.scala | 29 +- 102 files changed, 1141 insertions(+), 1524 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 5bd58e4108..904b1243e1 100644 --- a/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java +++ b/akka-actor-tests/src/test/java/akka/dispatch/JavaFutureTests.java @@ -1,5 +1,6 @@ package akka.dispatch; +import akka.actor.Timeout; import org.junit.Test; import static org.junit.Assert.*; import java.util.concurrent.Callable; @@ -11,6 +12,7 @@ import java.util.concurrent.TimeUnit; import akka.japi.Function; import akka.japi.Function2; import akka.japi.Procedure; +import akka.japi.Option; import scala.Some; import scala.Right; import static akka.dispatch.Futures.*; @@ -231,4 +233,25 @@ public class JavaFutureTests { assertEquals(result.get(), expectedStrings); } + + @Test public void findForJavaApiMustWork() { + LinkedList> listFutures = new LinkedList>(); + for (int i = 0; i < 10; i++) { + final Integer fi = i; + listFutures.add(future(new Callable() { + public Integer call() { + return fi; + } + })); + } + final Integer expect = 5; + Future> f = Futures.find(listFutures, new Function() { + public Boolean apply(Integer i) { + return i == 5; + } + }, Timeout.getDefault()); + + final Integer got = f.get().get(); + assertEquals(expect, got); + } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/ActorFireForgetRequestReplySpec.scala index 5846fae439..f3703c17c9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/ActorFireForgetRequestReplySpec.scala @@ -27,9 +27,7 @@ object ActorFireForgetRequestReplySpec { } } - class CrashingTemporaryActor extends Actor { - self.lifeCycle = Temporary - + class CrashingActor extends Actor { def receive = { case "Die" ⇒ state.finished.await @@ -86,7 +84,7 @@ class ActorFireForgetRequestReplySpec extends WordSpec with MustMatchers with Be "should shutdown crashed temporary actor" in { filterEvents(EventFilter[Exception]("Expected")) { - val actor = actorOf[CrashingTemporaryActor].start() + val actor = actorOf(Props[CrashingActor].withLifeCycle(Temporary)).start() actor.isRunning must be(true) actor ! "Die" state.finished.await diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/ActorRefSpec.scala index c5f93f831f..debb3e5527 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/ActorRefSpec.scala @@ -15,6 +15,7 @@ import akka.dispatch.Future import java.util.concurrent.{ TimeUnit, CountDownLatch } import java.lang.IllegalStateException import akka.util.ReflectiveAccess +import akka.actor.Actor.actorOf object ActorRefSpec { @@ -26,11 +27,11 @@ object ActorRefSpec { def receive = { case "complexRequest" ⇒ { replyTo = self.channel - val worker = Actor.actorOf[WorkerActor].start() + val worker = actorOf(Props[WorkerActor]) worker ! "work" } case "complexRequest2" ⇒ - val worker = Actor.actorOf[WorkerActor].start() + val worker = actorOf(Props[WorkerActor]) worker ! self.channel case "workDone" ⇒ replyTo ! "complexReply" case "simpleRequest" ⇒ self.reply("simpleReply") @@ -122,7 +123,7 @@ class ActorRefSpec extends WordSpec with MustMatchers { } intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new Actor { + actorOf(new Actor { val nested = new Actor { def receive = { case _ ⇒ } } def receive = { case _ ⇒ } }).start() @@ -133,43 +134,43 @@ class ActorRefSpec extends WordSpec with MustMatchers { refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new FailingOuterActor(Actor.actorOf(new InnerActor).start)).start() + actorOf(new FailingOuterActor(actorOf(new InnerActor).start)).start() } refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new OuterActor(Actor.actorOf(new FailingInnerActor).start)).start() + actorOf(new OuterActor(actorOf(new FailingInnerActor).start)).start() } refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new FailingInheritingOuterActor(Actor.actorOf(new InnerActor).start)).start() + actorOf(new FailingInheritingOuterActor(actorOf(new InnerActor).start)).start() } refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new FailingOuterActor(Actor.actorOf(new FailingInheritingInnerActor).start)).start() + actorOf(new FailingOuterActor(actorOf(new FailingInheritingInnerActor).start)).start() } refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new FailingInheritingOuterActor(Actor.actorOf(new FailingInheritingInnerActor).start)).start() + actorOf(new FailingInheritingOuterActor(actorOf(new FailingInheritingInnerActor).start)).start() } refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new FailingInheritingOuterActor(Actor.actorOf(new FailingInnerActor).start)).start() + actorOf(new FailingInheritingOuterActor(actorOf(new FailingInnerActor).start)).start() } refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new OuterActor(Actor.actorOf(new InnerActor { + actorOf(new OuterActor(actorOf(new InnerActor { val a = new InnerActor }).start)).start() } @@ -177,32 +178,32 @@ class ActorRefSpec extends WordSpec with MustMatchers { refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new FailingOuterActor(Actor.actorOf(new FailingInheritingInnerActor).start)).start() + actorOf(new FailingOuterActor(actorOf(new FailingInheritingInnerActor).start)).start() } refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new OuterActor(Actor.actorOf(new FailingInheritingInnerActor).start)).start() + actorOf(new OuterActor(actorOf(new FailingInheritingInnerActor).start)).start() } refStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { - Actor.actorOf(new OuterActor(Actor.actorOf({ new InnerActor; new InnerActor }).start)).start() + actorOf(new OuterActor(actorOf({ new InnerActor; new InnerActor }).start)).start() } refStackMustBeEmpty (intercept[java.lang.IllegalStateException] { - Actor.actorOf(new OuterActor(Actor.actorOf({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor }).start)).start() + actorOf(new OuterActor(actorOf({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor }).start)).start() }).getMessage must be === "Ur state be b0rked" refStackMustBeEmpty } "be serializable using Java Serialization on local node" in { - val a = Actor.actorOf[InnerActor].start + val a = actorOf[InnerActor].start import java.io._ @@ -225,7 +226,7 @@ class ActorRefSpec extends WordSpec with MustMatchers { "must throw exception on deserialize if not present in local registry and remoting is not enabled" in { ReflectiveAccess.RemoteModule.isEnabled must be === false - val a = Actor.actorOf[InnerActor].start + val a = actorOf[InnerActor].start val inetAddress = ReflectiveAccess.RemoteModule.configDefaultAddress @@ -255,8 +256,8 @@ class ActorRefSpec extends WordSpec with MustMatchers { } "support nested actorOfs" in { - val a = Actor.actorOf(new Actor { - val nested = Actor.actorOf(new Actor { def receive = { case _ ⇒ } }).start() + val a = actorOf(new Actor { + val nested = actorOf(new Actor { def receive = { case _ ⇒ } }).start() def receive = { case _ ⇒ self reply nested } }).start() @@ -267,7 +268,7 @@ class ActorRefSpec extends WordSpec with MustMatchers { } "support advanced nested actorOfs" in { - val a = Actor.actorOf(new OuterActor(Actor.actorOf(new InnerActor).start)).start + val a = actorOf(Props(new OuterActor(actorOf(Props(new InnerActor))))) val inner = (a ? "innerself").as[Any].get (a ? a).as[ActorRef].get must be(a) @@ -278,8 +279,8 @@ class ActorRefSpec extends WordSpec with MustMatchers { } "support reply via channel" in { - val serverRef = Actor.actorOf[ReplyActor].start() - val clientRef = Actor.actorOf(new SenderActor(serverRef)).start() + val serverRef = actorOf(Props[ReplyActor]) + val clientRef = actorOf(Props(new SenderActor(serverRef))) clientRef ! "complex" clientRef ! "simple" @@ -302,16 +303,14 @@ class ActorRefSpec extends WordSpec with MustMatchers { } "stop when sent a poison pill" in { - val ref = Actor.actorOf( - new Actor { - def receive = { - case 5 ⇒ self tryReply "five" - case null ⇒ self tryReply "null" - } - }).start() + val timeout = Timeout(20000) + val ref = actorOf(Props(self ⇒ { + case 5 ⇒ self tryReply "five" + case null ⇒ self tryReply "null" + })) - val ffive = (ref ? 5).mapTo[String] - val fnull = (ref ? null).mapTo[String] + val ffive = (ref ? (5, timeout)).mapTo[String] + val fnull = (ref ? (null, timeout)).mapTo[String] intercept[ActorKilledException] { (ref ? PoisonPill).get @@ -329,20 +328,17 @@ class ActorRefSpec extends WordSpec with MustMatchers { filterException[ActorKilledException] { val latch = new CountDownLatch(2) - val boss = Actor.actorOf(new Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), scala.Some(2), scala.Some(1000)) + val boss = actorOf(Props(new Actor { - val ref = Actor.actorOf( - new Actor { + val ref = actorOf( + Props(new Actor { def receive = { case _ ⇒ } override def preRestart(reason: Throwable, msg: Option[Any]) = latch.countDown() override def postRestart(reason: Throwable) = latch.countDown() - }).start() - - self link ref + }).withSupervisor(self)) protected def receive = { case "sendKill" ⇒ ref ! Kill } - }).start() + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 1000))) boss ! "sendKill" latch.await(5, TimeUnit.SECONDS) must be === true diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/ActorRestartSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/ActorRestartSpec.scala index 94e3b843cc..040e06e62a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/ActorRestartSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/ActorRestartSpec.scala @@ -59,7 +59,6 @@ object ActorRestartSpec { } class Supervisor extends Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 5, 5000) def receive = { case _ ⇒ } @@ -79,21 +78,24 @@ class ActorRestartSpec extends WordSpec with MustMatchers with TestKit with Befo } private var toStop = new java.util.concurrent.ConcurrentSkipListSet[ActorRef] - private def newActor(f: ⇒ Actor): ActorRef = { - val ref = actorOf(f) + private def collect(f: ⇒ ActorRef): ActorRef = { + val ref = f toStop add ref ref.start() } + private def createSupervisor = + actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000))) + val expectedEvents = Seq(EventFilter[ActorKilledException], EventFilter[IllegalActorStateException]("expected")) "An Actor restart" must { "invoke preRestart, preStart, postRestart" in { filterEvents(expectedEvents) { - val actor = newActor(new Restarter(testActor)) + val actor = collect(actorOf(new Restarter(testActor))) expectMsg(1 second, ("preStart", 1)) - val supervisor = newActor(new Supervisor) + val supervisor = collect(createSupervisor) supervisor link actor actor ! Kill within(1 second) { @@ -107,9 +109,9 @@ class ActorRestartSpec extends WordSpec with MustMatchers with TestKit with Befo "support creation of nested actors in freshInstance()" in { filterEvents(expectedEvents) { - val actor = newActor(new Restarter(testActor)) + val actor = collect(actorOf(new Restarter(testActor))) expectMsg(1 second, ("preStart", 1)) - val supervisor = newActor(new Supervisor) + val supervisor = collect(createSupervisor) supervisor link actor actor ! Nested actor ! Kill @@ -128,9 +130,9 @@ class ActorRestartSpec extends WordSpec with MustMatchers with TestKit with Befo "use freshInstance() if available" in { filterEvents(expectedEvents) { - val actor = newActor(new Restarter(testActor)) + val actor = collect(actorOf(new Restarter(testActor))) expectMsg(1 second, ("preStart", 1)) - val supervisor = newActor(new Supervisor) + val supervisor = collect(createSupervisor) supervisor link actor actor ! 42 actor ! Handover @@ -148,9 +150,9 @@ class ActorRestartSpec extends WordSpec with MustMatchers with TestKit with Befo "fall back to default factory if freshInstance() fails" in { filterEvents(expectedEvents) { - val actor = newActor(new Restarter(testActor)) + val actor = collect(actorOf(new Restarter(testActor))) expectMsg(1 second, ("preStart", 1)) - val supervisor = newActor(new Supervisor) + val supervisor = collect(createSupervisor) supervisor link actor actor ! 42 actor ! Fail diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/ActorTimeoutSpec.scala index 2589282d75..d474824d25 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/ActorTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/ActorTimeoutSpec.scala @@ -15,38 +15,42 @@ class ActorTimeoutSpec with MustMatchers with TestKit { - val echo = Actor.actorOf(new Actor { + def actorWithTimeout(t: Timeout): ActorRef = Actor.actorOf(Props(creator = () ⇒ new Actor { def receive = { case x ⇒ } - }).start() + }, timeout = t)).start() val testTimeout = if (Timeout.default.duration < 400.millis) 500 millis else 100 millis - override def afterAll { echo.stop() } - "An Actor-based Future" must { "use the global default timeout if no implicit in scope" in { - echo.timeout = 12 within((Actor.TIMEOUT - 100).millis, (Actor.TIMEOUT + 400).millis) { - val f = echo ? "hallo" - intercept[FutureTimeoutException] { f.await } + val echo = actorWithTimeout(Timeout(12)) + try { + val f = echo ? "hallo" + intercept[FutureTimeoutException] { f.await } + } finally { echo.stop } } } "use implicitly supplied timeout" in { implicit val timeout = Timeout(testTimeout) within(testTimeout - 100.millis, testTimeout + 300.millis) { - val f = (echo ? "hallo").mapTo[String] - intercept[FutureTimeoutException] { f.await } - f.value must be(None) + val echo = actorWithTimeout(Props.defaultTimeout) + try { + val f = (echo ? "hallo").mapTo[String] + intercept[FutureTimeoutException] { f.await } + f.value must be(None) + } finally { echo.stop } } } "use explicitly supplied timeout" in { within(testTimeout - 100.millis, testTimeout + 300.millis) { - (echo.?("hallo", testTimeout)).as[String] must be(None) + val echo = actorWithTimeout(Props.defaultTimeout) + try { (echo.?("hallo", testTimeout)).as[String] must be(None) } finally { echo.stop } } } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala index b2e41a02c7..f79a0fef88 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala @@ -22,7 +22,7 @@ class DeployerSpec extends WordSpec with MustMatchers { LeastCPU, Clustered( Vector(Node("node1")), - ReplicationFactor(3), + new ReplicationFactor(3), Replication( TransactionLog, WriteThrough))))) diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/FSMTimingSpec.scala index dc8f6a570a..1a86cb4b56 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/FSMTimingSpec.scala @@ -70,7 +70,7 @@ class FSMTimingSpec extends WordSpec with MustMatchers with TestKit { fsm ! Tick expectMsg(100 millis, Tick) Thread.sleep(200) - fsm.dispatcher resume fsm + resume(fsm) expectMsg(100 millis, Transition(fsm, TestCancelStateTimerInNamedTimerMessage, TestCancelStateTimerInNamedTimerMessage2)) fsm ! Cancel within(100 millis) { @@ -119,6 +119,16 @@ class FSMTimingSpec extends WordSpec with MustMatchers with TestKit { object FSMTimingSpec { + def suspend(actorRef: ActorRef): Unit = actorRef match { + case l: LocalActorRef ⇒ l.dispatcher.suspend(l) + case _ ⇒ + } + + def resume(actorRef: ActorRef): Unit = actorRef match { + case l: LocalActorRef ⇒ l.dispatcher.resume(l) + case _ ⇒ + } + trait State case object Initial extends State case object TestStateTimeout extends State @@ -189,7 +199,7 @@ object FSMTimingSpec { when(TestCancelStateTimerInNamedTimerMessage) { // FSM is suspended after processing this message and resumed 200ms later case Ev(Tick) ⇒ - self.dispatcher suspend self + suspend(self) setTimer("named", Tock, 10 millis, false) stay forMax (100 millis) replying Tick case Ev(Tock) ⇒ diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/FSMTransitionSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/FSMTransitionSpec.scala index 2c825f9063..ecd2cd8932 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/FSMTransitionSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/FSMTransitionSpec.scala @@ -17,7 +17,6 @@ import FSM._ object FSMTransitionSpec { class Supervisor extends Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, None) def receive = { case _ ⇒ } } @@ -63,7 +62,7 @@ class FSMTransitionSpec extends WordSpec with MustMatchers with TestKit { "not fail when listener goes away" in { val forward = Actor.actorOf(new Forwarder(testActor)).start() val fsm = Actor.actorOf(new MyFSM(testActor)).start() - val sup = Actor.actorOf[Supervisor].start() + val sup = Actor.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) sup link fsm within(300 millis) { fsm ! SubscribeTransitionCallBack(forward) @@ -77,7 +76,7 @@ class FSMTransitionSpec extends WordSpec with MustMatchers with TestKit { "not fail when listener is invalid" in { val forward = Actor.actorOf(new Forwarder(testActor)) val fsm = Actor.actorOf(new MyFSM(testActor)).start() - val sup = Actor.actorOf[Supervisor].start() + val sup = Actor.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) sup link fsm within(300 millis) { filterEvents(EventFilter.custom { diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/ForwardActorSpec.scala index 68d4149e5a..c69b4f089e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/ForwardActorSpec.scala @@ -11,49 +11,22 @@ import akka.testkit._ import akka.util.duration._ import Actor._ +import akka.util.Duration object ForwardActorSpec { - object ForwardState { - var sender: Option[ActorRef] = None - } + val ExpectedMessage = "FOO" - class ReceiverActor extends Actor { - val latch = TestLatch() - def receive = { - case "SendBang" ⇒ { - ForwardState.sender = self.sender - latch.countDown() - } - case "SendBangBang" ⇒ self.reply("SendBangBang") - } - } + def createForwardingChain(): ActorRef = { + val replier = actorOf(new Actor { + def receive = { case x ⇒ self reply x } + }).start() - class ForwardActor extends Actor { - val receiverActor = actorOf[ReceiverActor] - receiverActor.start() - def receive = { - case "SendBang" ⇒ receiverActor.forward("SendBang") - case "SendBangBang" ⇒ receiverActor.forward("SendBangBang") - } - } + def mkforwarder(forwardTo: ActorRef) = actorOf( + new Actor { + def receive = { case x ⇒ forwardTo forward x } + }).start() - class BangSenderActor extends Actor { - val forwardActor = actorOf[ForwardActor] - forwardActor.start() - forwardActor ! "SendBang" - def receive = { - case _ ⇒ {} - } - } - - class BangBangSenderActor extends Actor { - val latch = TestLatch() - val forwardActor = actorOf[ForwardActor] - forwardActor.start() - forwardActor ? "SendBangBang" onComplete { _ ⇒ latch.countDown() } - def receive = { - case _ ⇒ {} - } + mkforwarder(mkforwarder(mkforwarder(replier))) } } @@ -61,23 +34,21 @@ class ForwardActorSpec extends WordSpec with MustMatchers { import ForwardActorSpec._ "A Forward Actor" must { + "forward actor reference when invoking forward on bang" in { - val senderActor = actorOf[BangSenderActor] - val latch = senderActor.actor.asInstanceOf[BangSenderActor] - .forwardActor.actor.asInstanceOf[ForwardActor] - .receiverActor.actor.asInstanceOf[ReceiverActor] - .latch - senderActor.start() - latch.await - ForwardState.sender must not be (null) - senderActor.toString must be(ForwardState.sender.get.toString) + val latch = new TestLatch(1) + + val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage ⇒ latch.countDown() } }).start() + + val chain = createForwardingChain() + + chain.tell(ExpectedMessage, replyTo) + latch.await(Duration(5, "s")) must be === true } "forward actor reference when invoking forward on bang bang" in { - val senderActor = actorOf[BangBangSenderActor] - senderActor.start() - val latch = senderActor.actor.asInstanceOf[BangBangSenderActor].latch - latch.await + val chain = createForwardingChain() + chain.ask(ExpectedMessage, 5000).get must be === ExpectedMessage } } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/IOActor.scala index faf36acf6b..036378f68b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/IOActor.scala @@ -34,7 +34,8 @@ object IOActorSpec { }) def receive = { - case msg: NewClient ⇒ self startLink createWorker forward msg + case msg: NewClient ⇒ + self.link(createWorker).start() forward msg } } @@ -103,7 +104,7 @@ object IOActorSpec { }) def receive = { - case msg: NewClient ⇒ self startLink createWorker forward msg + case msg: NewClient ⇒ self.link(createWorker).start() forward msg case ('set, key: String, value: ByteString) ⇒ kvs += (key -> value) self tryReply (()) diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/LoggingReceiveSpec.scala index b21584dfc0..5c91dd5601 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/LoggingReceiveSpec.scala @@ -131,12 +131,9 @@ class LoggingReceiveSpec "log LifeCycle changes if requested" in { within(2 seconds) { - val supervisor = TestActorRef(new Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 5, 5000) - def receive = { - case _ ⇒ - } - }).start() + val supervisor = TestActorRef(Props(new Actor { + def receive = { case _ ⇒ } + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000))) val f = Actor.getClass.getDeclaredField("debugLifecycle") f.setAccessible(true) f.setBoolean(Actor, true) diff --git a/akka-actor-tests/src/test/scala/akka/actor/supervisor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/supervisor/RestartStrategySpec.scala index 03855c6aad..89830c37fd 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/supervisor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/supervisor/RestartStrategySpec.scala @@ -34,10 +34,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { @Test def slaveShouldStayDeadAfterMaxRestartsWithinTimeRange = { - val boss = actorOf(new Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), Some(1000)) + val boss = actorOf(Props(new Actor { protected def receive = { case _ ⇒ () } - }).start() + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 1000))) val restartLatch = new StandardLatch val secondRestartLatch = new StandardLatch @@ -61,7 +60,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { stopLatch.open } }) - boss.startLink(slave) + boss.link(slave).start() slave ! Ping slave ! Crash @@ -88,10 +87,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { @Test def slaveShouldBeImmortalWithoutMaxRestartsAndTimeRange = { - val boss = actorOf(new Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, None) - protected def receive = { case _ ⇒ () } - }).start() + val boss = actorOf(Props(new Actor { + def receive = { case _ ⇒ () } + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) val countDownLatch = new CountDownLatch(100) @@ -106,7 +104,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { } }) - boss.startLink(slave) + boss.link(slave).start() (1 to 100) foreach { _ ⇒ slave ! Crash } assert(countDownLatch.await(120, TimeUnit.SECONDS)) assert(slave.isRunning) @@ -115,10 +113,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { @Test def slaveShouldRestartAfterNumberOfCrashesNotWithinTimeRange = { - val boss = actorOf(new Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), Some(500)) - protected def receive = { case _ ⇒ () } - }).start() + val boss = actorOf(Props(new Actor { + def receive = { case _ ⇒ () } + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 500))) val restartLatch = new StandardLatch val secondRestartLatch = new StandardLatch @@ -148,7 +145,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { } } }) - boss.startLink(slave) + boss.link(slave).start() slave ! Ping slave ! Crash @@ -176,10 +173,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { @Test def slaveShouldNotRestartAfterMaxRetries = { - val boss = actorOf(new Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), None) - protected def receive = { case _ ⇒ () } - }).start() + val boss = actorOf(Props(new Actor { + def receive = { case _ ⇒ () } + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), Some(2), None))) val restartLatch = new StandardLatch val secondRestartLatch = new StandardLatch @@ -203,7 +199,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { stopLatch.open } }) - boss.startLink(slave) + boss.link(slave).start() slave ! Ping slave ! Crash @@ -235,12 +231,9 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { val restartLatch, stopLatch, maxNoOfRestartsLatch = new StandardLatch val countDownLatch = new CountDownLatch(2) - val boss = actorOf(new Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, Some(1000)) - protected def receive = { - case m: MaximumNumberOfRestartsWithinTimeRangeReached ⇒ maxNoOfRestartsLatch.open - } - }).start() + val boss = actorOf(Props(new Actor { + def receive = { case m: MaximumNumberOfRestartsWithinTimeRangeReached ⇒ maxNoOfRestartsLatch.open } + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, Some(1000)))) val slave = actorOf(new Actor { @@ -257,7 +250,7 @@ class RestartStrategySpec extends JUnitSuite with BeforeAndAfterAll { stopLatch.open } }) - boss.startLink(slave) + boss.link(slave).start() slave ! Ping slave ! Crash diff --git a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorHierarchySpec.scala index 8ed783e544..df0d550add 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorHierarchySpec.scala @@ -36,18 +36,16 @@ class SupervisorHierarchySpec extends JUnitSuite { val workerTwo = actorOf(new CountDownActor(countDown)) val workerThree = actorOf(new CountDownActor(countDown)) - val boss = actorOf(new Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 5, 1000) - + val boss = actorOf(Props(new Actor { protected def receive = { case _ ⇒ () } - }).start() + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 1000))) val manager = actorOf(new CountDownActor(countDown)) - boss.startLink(manager) + boss.link(manager).start() - manager.startLink(workerOne) - manager.startLink(workerTwo) - manager.startLink(workerThree) + manager.link(workerOne).start() + manager.link(workerTwo).start() + manager.link(workerThree).start() workerOne ! Death(workerOne, new FireWorkerException("Fire the worker!")) @@ -61,14 +59,12 @@ class SupervisorHierarchySpec extends JUnitSuite { def supervisorShouldReceiveNotificationMessageWhenMaximumNumberOfRestartsWithinTimeRangeIsReached = { val countDown = new CountDownLatch(2) val crasher = actorOf(new CountDownActor(countDown)) - val boss = actorOf(new Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 1, 5000) + val boss = actorOf(Props(new Actor { protected def receive = { - case MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _) ⇒ - countDown.countDown() + case MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _) ⇒ countDown.countDown() } - }).start() - boss.startLink(crasher) + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 1, 5000))) + boss.link(crasher).start() crasher ! Death(crasher, new FireWorkerException("Fire the worker!")) crasher ! Death(crasher, new FireWorkerException("Fire the worker!")) diff --git a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala index 4d7731d883..0c8e606247 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala @@ -5,10 +5,10 @@ package akka.actor import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers -import akka.dispatch.Dispatchers import akka.config.Supervision.{ SupervisorConfig, OneForOneStrategy, Supervise, Permanent } import java.util.concurrent.CountDownLatch import akka.testkit.{ filterEvents, EventFilter } +import akka.dispatch.{ PinnedDispatcher, Dispatchers } class SupervisorMiscSpec extends WordSpec with MustMatchers { "A Supervisor" should { @@ -17,45 +17,41 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers { filterEvents(EventFilter[Exception]("killed")) { val countDownLatch = new CountDownLatch(4) - val actor1 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newPinnedDispatcher(self) + val actor1 = Actor.actorOf(Props(new Actor { override def postRestart(cause: Throwable) { countDownLatch.countDown() } protected def receive = { case "kill" ⇒ throw new Exception("killed") case _ ⇒ println("received unknown message") } - }).start() + }).withDispatcher(new PinnedDispatcher())) - val actor2 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newPinnedDispatcher(self) + val actor2 = Actor.actorOf(Props(new Actor { override def postRestart(cause: Throwable) { countDownLatch.countDown() } protected def receive = { case "kill" ⇒ throw new Exception("killed") case _ ⇒ println("received unknown message") } - }).start() + }).withDispatcher(new PinnedDispatcher())) - val actor3 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newDispatcher("test").build + val actor3 = Actor.actorOf(Props(new Actor { override def postRestart(cause: Throwable) { countDownLatch.countDown() } protected def receive = { case "kill" ⇒ throw new Exception("killed") case _ ⇒ println("received unknown message") } - }).start() + }).withDispatcher(Dispatchers.newDispatcher("test").build)) - val actor4 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newPinnedDispatcher(self) + val actor4 = Actor.actorOf(Props(new Actor { override def postRestart(cause: Throwable) { countDownLatch.countDown() } protected def receive = { case "kill" ⇒ throw new Exception("killed") case _ ⇒ println("received unknown message") } - }).start() + }).withDispatcher(new PinnedDispatcher())) val sup = Supervisor( SupervisorConfig( diff --git a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala index dc5b20d325..242802589a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala @@ -55,15 +55,10 @@ object SupervisorSpec { } } - class TemporaryActor extends PingPongActor { - self.lifeCycle = Temporary - } - class Master extends Actor { - self.faultHandler = OneForOneStrategy(List(classOf[Exception]), 5, (1 second).dilated.toMillis.toInt) val temp = { - val a = actorOf[TemporaryActor] + val a = actorOf(Props[PingPongActor].withLifeCycle(Temporary)) self link a a.start } @@ -79,7 +74,7 @@ object SupervisorSpec { // ===================================================== def temporaryActorAllForOne = { - val temporaryActor = actorOf[TemporaryActor].start() + val temporaryActor = actorOf(Props[PingPongActor].withLifeCycle(Temporary)) val supervisor = Supervisor( SupervisorConfig( @@ -226,7 +221,7 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach "A supervisor" must { "not restart programmatically linked temporary actor" in { - val master = actorOf[Master].start() + val master = actorOf(Props[Master].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 5, (1 second).dilated.toMillis.toInt))) intercept[RuntimeException] { (master.?(Die, TimeoutMillis)).get @@ -369,8 +364,7 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach "must attempt restart when exception during restart" in { val inits = new AtomicInteger(0) - val dyingActor = actorOf(new Actor { - self.lifeCycle = Permanent + val dyingActor = actorOf(Props(new Actor { inits.incrementAndGet if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!") @@ -379,7 +373,7 @@ class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach case Ping ⇒ self.tryReply(PongMessage) case Die ⇒ throw new Exception("expected") } - }) + })) val supervisor = Supervisor( diff --git a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorTreeSpec.scala index 86591fd2c8..ab90c2b524 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorTreeSpec.scala @@ -17,9 +17,7 @@ class SupervisorTreeSpec extends WordSpec with MustMatchers { var log = "" case object Die - class Chainer(a: Option[ActorRef] = None) extends Actor { - self.lifeCycle = Permanent - self.faultHandler = OneForOneStrategy(List(classOf[Exception]), 3, 1000) + class Chainer(a: Option[ActorRef]) extends Actor { a.foreach(self.link(_)) def receive = { @@ -37,9 +35,10 @@ class SupervisorTreeSpec extends WordSpec with MustMatchers { filterException[Exception] { log = "INIT" - val lastActor = actorOf(new Chainer, "lastActor").start - val middleActor = actorOf(new Chainer(Some(lastActor)), "middleActor").start - val headActor = actorOf(new Chainer(Some(middleActor)), "headActor").start + val p = Props.default.withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 1000)) + val lastActor = actorOf(p.withCreator(new Chainer(None)), "lastActor") + val middleActor = actorOf(p.withCreator(new Chainer(Some(lastActor))), "middleActor") + val headActor = actorOf(p.withCreator(new Chainer(Some(middleActor))), "headActor") middleActor ! Die sleepFor(500 millis) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala index aeb219f1cd..1849f7d143 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala @@ -12,9 +12,9 @@ import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.{ ConcurrentHashMap, CountDownLatch, TimeUnit } import akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor import akka.util.Switch -import akka.actor.{ ActorKilledException, PoisonPill, ActorRef, Actor } import java.rmi.RemoteException import org.junit.{ After, Test } +import akka.actor._ object ActorModelSpec { @@ -49,11 +49,11 @@ object ActorModelSpec { val Ping = "Ping" val Pong = "Pong" - class DispatcherActor(dispatcher: MessageDispatcherInterceptor) extends Actor { - self.dispatcher = dispatcher.asInstanceOf[MessageDispatcher] - + class DispatcherActor extends Actor { private val busy = new Switch(false) + def dispatcher = self.dispatcher.asInstanceOf[MessageDispatcherInterceptor] + def ack { if (!busy.switchOn()) { throw new Exception("isolation violated") @@ -103,22 +103,22 @@ object ActorModelSpec { stats.get(actorRef) } - abstract override def suspend(actorRef: ActorRef) { + abstract override def suspend(actorRef: LocalActorRef) { super.suspend(actorRef) getStats(actorRef).suspensions.incrementAndGet() } - abstract override def resume(actorRef: ActorRef) { + abstract override def resume(actorRef: LocalActorRef) { super.resume(actorRef) getStats(actorRef).resumes.incrementAndGet() } - private[akka] abstract override def register(actorRef: ActorRef) { + private[akka] abstract override def register(actorRef: LocalActorRef) { super.register(actorRef) getStats(actorRef).registers.incrementAndGet() } - private[akka] abstract override def unregister(actorRef: ActorRef) { + private[akka] abstract override def unregister(actorRef: LocalActorRef) { super.unregister(actorRef) getStats(actorRef).unregisters.incrementAndGet() } @@ -183,7 +183,7 @@ object ActorModelSpec { msgsReceived: Long = statsFor(actorRef).msgsReceived.get(), msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(), restarts: Long = statsFor(actorRef).restarts.get()) { - val stats = statsFor(actorRef, if (dispatcher eq null) actorRef.dispatcher else dispatcher) + val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[SelfActorRef].dispatcher)) assert(stats.suspensions.get() === suspensions, "Suspensions") assert(stats.resumes.get() === resumes, "Resumes") assert(stats.registers.get() === registers, "Registers") @@ -207,7 +207,7 @@ object ActorModelSpec { false } - def newTestActor(implicit d: MessageDispatcherInterceptor) = actorOf(new DispatcherActor(d)) + def newTestActor(implicit d: MessageDispatcherInterceptor) = actorOf(Props[DispatcherActor].withDispatcher(d)) } abstract class ActorModelSpec extends JUnitSuite { @@ -219,8 +219,8 @@ abstract class ActorModelSpec extends JUnitSuite { @Test def dispatcherShouldDynamicallyHandleItsOwnLifeCycle { implicit val dispatcher = newInterceptedDispatcher - val a = newTestActor assertDispatcher(dispatcher)(starts = 0, stops = 0) + val a = newTestActor a.start() assertDispatcher(dispatcher)(starts = 1, stops = 0) a.stop() @@ -346,7 +346,7 @@ abstract class ActorModelSpec extends JUnitSuite { @Test def dispatcherShouldNotProcessMessagesForASuspendedActor { implicit val dispatcher = newInterceptedDispatcher - val a = newTestActor.start() + val a = newTestActor.start().asInstanceOf[LocalActorRef] val done = new CountDownLatch(1) dispatcher.suspend(a) a ! CountDown(done) @@ -384,7 +384,7 @@ abstract class ActorModelSpec extends JUnitSuite { @Test def dispatcherShouldCompleteAllUncompletedSenderFuturesOnDeregister { implicit val dispatcher = newInterceptedDispatcher - val a = newTestActor.start() + val a = newTestActor.start().asInstanceOf[LocalActorRef] dispatcher.suspend(a) val f1: Future[String] = a ? Reply("foo") mapTo manifest[String] val stopped = a ? PoisonPill diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/BalancingDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/BalancingDispatcherSpec.scala index 9ce2c9dccd..b587719b32 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/BalancingDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/BalancingDispatcherSpec.scala @@ -6,9 +6,9 @@ import org.scalatest.junit.JUnitSuite import org.junit.Test import java.util.concurrent.{ TimeUnit, CountDownLatch } -import akka.actor.{ IllegalActorStateException, Actor } -import Actor._ +import akka.actor.Actor._ import akka.dispatch.{ MessageQueue, Dispatchers } +import akka.actor.{ LocalActorRef, IllegalActorStateException, Actor, Props } object BalancingDispatcherSpec { @@ -17,7 +17,6 @@ object BalancingDispatcherSpec { val delayableActorDispatcher, sharedActorDispatcher, parentActorDispatcher = newWorkStealer() class DelayableActor(delay: Int, finishedCounter: CountDownLatch) extends Actor { - self.dispatcher = delayableActorDispatcher @volatile var invocationCount = 0 @@ -31,17 +30,14 @@ object BalancingDispatcherSpec { } class FirstActor extends Actor { - self.dispatcher = sharedActorDispatcher def receive = { case _ ⇒ {} } } class SecondActor extends Actor { - self.dispatcher = sharedActorDispatcher def receive = { case _ ⇒ {} } } class ParentActor extends Actor { - self.dispatcher = parentActorDispatcher def receive = { case _ ⇒ {} } } @@ -59,8 +55,8 @@ class BalancingDispatcherSpec extends JUnitSuite with MustMatchers { def fastActorShouldStealWorkFromSlowActor { val finishedCounter = new CountDownLatch(110) - val slow = actorOf(new DelayableActor(50, finishedCounter), "slow").start - val fast = actorOf(new DelayableActor(10, finishedCounter), "fast").start + val slow = actorOf(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher), "slow").asInstanceOf[LocalActorRef] + val fast = actorOf(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher), "fast").asInstanceOf[LocalActorRef] var sentToFast = 0 @@ -86,32 +82,32 @@ class BalancingDispatcherSpec extends JUnitSuite with MustMatchers { finishedCounter.await(5, TimeUnit.SECONDS) fast.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true) slow.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true) - fast.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast - fast.actor.asInstanceOf[DelayableActor].invocationCount must be > - (slow.actor.asInstanceOf[DelayableActor].invocationCount) + fast.actorInstance.get().asInstanceOf[DelayableActor].invocationCount must be > sentToFast + fast.actorInstance.get().asInstanceOf[DelayableActor].invocationCount must be > + (slow.actorInstance.get().asInstanceOf[DelayableActor].invocationCount) slow.stop() fast.stop() } @Test def canNotUseActorsOfDifferentTypesInSameDispatcher(): Unit = { - val first = actorOf[FirstActor] - val second = actorOf[SecondActor] + val first = actorOf(Props[FirstActor].withDispatcher(sharedActorDispatcher)) - first.start() intercept[IllegalActorStateException] { - second.start() + actorOf(Props[SecondActor].withDispatcher(sharedActorDispatcher)) } + first.stop() } @Test def canNotUseActorsOfDifferentSubTypesInSameDispatcher(): Unit = { - val parent = actorOf[ParentActor] - val child = actorOf[ChildActor] + val parent = actorOf(Props[ParentActor].withDispatcher(parentActorDispatcher)) - parent.start() intercept[IllegalActorStateException] { - child.start() + val child = actorOf(Props[ChildActor].withDispatcher(parentActorDispatcher)) + child.stop() } + + parent.stop() } } diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/DispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/DispatcherActorSpec.scala index 2d5b3ecb16..f0b538b022 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/DispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/DispatcherActorSpec.scala @@ -3,20 +3,17 @@ package akka.actor.dispatch import java.util.concurrent.{ CountDownLatch, TimeUnit } import org.scalatest.junit.JUnitSuite import org.junit.Test -import akka.dispatch.{ Dispatchers, Dispatcher } -import akka.actor.Actor -import Actor._ +import akka.actor.Actor._ import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger } import akka.testkit.{ filterEvents, EventFilter } +import akka.dispatch.{ PinnedDispatcher, Dispatchers, Dispatcher } +import akka.actor.{ Props, Actor } object DispatcherActorSpec { class TestActor extends Actor { - self.dispatcher = Dispatchers.newDispatcher(self.uuid.toString).build def receive = { - case "Hello" ⇒ - self.reply("World") - case "Failure" ⇒ - throw new RuntimeException("Expected exception; to test fault-tolerance") + case "Hello" ⇒ self.reply("World") + case "Failure" ⇒ throw new RuntimeException("Expected exception; to test fault-tolerance") } } @@ -24,7 +21,6 @@ object DispatcherActorSpec { val oneWay = new CountDownLatch(1) } class OneWayTestActor extends Actor { - self.dispatcher = Dispatchers.newDispatcher(self.uuid.toString).build def receive = { case "OneWay" ⇒ OneWayTestActor.oneWay.countDown() } @@ -37,7 +33,7 @@ class DispatcherActorSpec extends JUnitSuite { @Test def shouldTell = { - val actor = actorOf[OneWayTestActor].start() + val actor = actorOf(Props[OneWayTestActor].withDispatcher(new PinnedDispatcher())) val result = actor ! "OneWay" assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS)) actor.stop() @@ -45,7 +41,7 @@ class DispatcherActorSpec extends JUnitSuite { @Test def shouldSendReplySync = { - val actor = actorOf[TestActor].start() + val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher())) val result = (actor.?("Hello", 10000)).as[String] assert("World" === result.get) actor.stop() @@ -53,7 +49,7 @@ class DispatcherActorSpec extends JUnitSuite { @Test def shouldSendReplyAsync = { - val actor = actorOf[TestActor].start() + val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher())) val result = (actor ? "Hello").as[String] assert("World" === result.get) actor.stop() @@ -62,7 +58,7 @@ class DispatcherActorSpec extends JUnitSuite { @Test def shouldSendReceiveException = { filterEvents(EventFilter[RuntimeException]("Expected")) { - val actor = actorOf[TestActor].start() + val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher())) try { (actor ? "Failure").get fail("Should have thrown an exception") @@ -85,19 +81,13 @@ class DispatcherActorSpec extends JUnitSuite { val latch = new CountDownLatch(100) val start = new CountDownLatch(1) val fastOne = actorOf( - new Actor { - self.dispatcher = throughputDispatcher - def receive = { case "sabotage" ⇒ works.set(false) } - }).start() + Props(self ⇒ { case "sabotage" ⇒ works.set(false) }).withDispatcher(throughputDispatcher)) val slowOne = actorOf( - new Actor { - self.dispatcher = throughputDispatcher - def receive = { - case "hogexecutor" ⇒ start.await - case "ping" ⇒ if (works.get) latch.countDown() - } - }).start() + Props(self ⇒ { + case "hogexecutor" ⇒ start.await + case "ping" ⇒ if (works.get) latch.countDown() + }).withDispatcher(throughputDispatcher)) slowOne ! "hogexecutor" (1 to 100) foreach { _ ⇒ slowOne ! "ping" } @@ -122,19 +112,15 @@ class DispatcherActorSpec extends JUnitSuite { val ready = new CountDownLatch(1) val fastOne = actorOf( - new Actor { - self.dispatcher = throughputDispatcher - def receive = { case "ping" ⇒ if (works.get) latch.countDown(); self.stop() } - }).start() + Props(self ⇒ { + case "ping" ⇒ if (works.get) latch.countDown(); self.stop() + }).withDispatcher(throughputDispatcher)) val slowOne = actorOf( - new Actor { - self.dispatcher = throughputDispatcher - def receive = { - case "hogexecutor" ⇒ ready.countDown(); start.await - case "ping" ⇒ works.set(false); self.stop() - } - }).start() + Props(self ⇒ { + case "hogexecutor" ⇒ ready.countDown(); start.await + case "ping" ⇒ works.set(false); self.stop() + }).withDispatcher(throughputDispatcher)) slowOne ! "hogexecutor" slowOne ! "ping" diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index 2f2317c5f7..4763ab4f92 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -289,6 +289,20 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd } } + "firstCompletedOf" in { + val futures = Vector.fill[Future[Int]](10)(new DefaultPromise[Int]()) :+ new KeptPromise[Int](Right(5)) + Futures.firstCompletedOf(futures).get must be(5) + } + + "find" in { + val futures = for (i ← 1 to 10) yield Future { i } + val result = Futures.find[Int](_ == 3)(futures) + result.get must be(Some(3)) + + val notFound = Futures.find[Int](_ == 11)(futures) + notFound.get must be(None) + } + "fold" in { val actors = (1 to 10).toList map { _ ⇒ actorOf(new Actor { 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 678cbd2e86..2dc2caa11e 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala @@ -4,12 +4,12 @@ import org.scalatest.matchers.MustMatchers import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import org.scalatest.junit.JUnitRunner import org.junit.runner.RunWith -import akka.actor.{ Actor, ActorRegistry, NullChannel } import akka.actor.Actor.{ actorOf } import java.util.concurrent.{ TimeUnit, CountDownLatch, BlockingQueue } import java.util.{ Queue } import akka.util._ import akka.util.Duration._ +import akka.actor.{ LocalActorRef, Actor, ActorRegistry, NullChannel } @RunWith(classOf[JUnitRunner]) abstract class MailboxSpec extends WordSpec with MustMatchers with BeforeAndAfterAll with BeforeAndAfterEach { @@ -84,7 +84,7 @@ abstract class MailboxSpec extends WordSpec with MustMatchers with BeforeAndAfte new MessageInvocation( actorOf(new Actor { //Dummy actor def receive = { case _ ⇒ } - }), msg, NullChannel) + }).asInstanceOf[LocalActorRef], msg, NullChannel) } def ensureInitialMailboxState(config: MailboxType, q: MessageQueue) { diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PinnedActorSpec.scala index 48918d994d..aa477ce9a4 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PinnedActorSpec.scala @@ -4,23 +4,19 @@ import java.util.concurrent.{ CountDownLatch, TimeUnit } import org.scalatest.junit.JUnitSuite import org.junit.{ Test, Before, After } -import akka.dispatch.Dispatchers -import akka.actor.Actor -import Actor._ +import akka.actor.Actor._ import akka.event.EventHandler import akka.testkit.TestEvent._ import akka.testkit.EventFilter +import akka.dispatch.{ PinnedDispatcher, Dispatchers } +import akka.actor.{ Props, Actor } object PinnedActorSpec { class TestActor extends Actor { - self.dispatcher = Dispatchers.newPinnedDispatcher(self) - def receive = { - case "Hello" ⇒ - self.reply("World") - case "Failure" ⇒ - throw new RuntimeException("Expected exception; to test fault-tolerance") + case "Hello" ⇒ self.reply("World") + case "Failure" ⇒ throw new RuntimeException("Expected exception; to test fault-tolerance") } } } @@ -43,12 +39,7 @@ class PinnedActorSpec extends JUnitSuite { @Test def shouldTell { var oneWay = new CountDownLatch(1) - val actor = actorOf(new Actor { - self.dispatcher = Dispatchers.newPinnedDispatcher(self) - def receive = { - case "OneWay" ⇒ oneWay.countDown() - } - }).start() + val actor = actorOf(Props(self ⇒ { case "OneWay" ⇒ oneWay.countDown() }).withDispatcher(new PinnedDispatcher())) val result = actor ! "OneWay" assert(oneWay.await(1, TimeUnit.SECONDS)) actor.stop() @@ -56,7 +47,7 @@ class PinnedActorSpec extends JUnitSuite { @Test def shouldSendReplySync = { - val actor = actorOf[TestActor].start() + val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher())) val result = (actor.?("Hello", 10000)).as[String] assert("World" === result.get) actor.stop() @@ -64,7 +55,7 @@ class PinnedActorSpec extends JUnitSuite { @Test def shouldSendReplyAsync = { - val actor = actorOf[TestActor].start() + val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher())) val result = (actor ? "Hello").as[String] assert("World" === result.get) actor.stop() @@ -72,7 +63,7 @@ class PinnedActorSpec extends JUnitSuite { @Test def shouldSendReceiveException = { - val actor = actorOf[TestActor].start() + val actor = actorOf(Props[TestActor].withDispatcher(new PinnedDispatcher())) try { (actor ? "Failure").get fail("Should have thrown an exception") diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala index c2fbc2d09c..d6b490d00e 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -1,10 +1,9 @@ package akka.dispatch import akka.actor.Actor._ -import akka.actor.Actor import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers -import java.util.concurrent.CountDownLatch +import akka.actor.{ Props, LocalActorRef, Actor } class PriorityDispatcherSpec extends WordSpec with MustMatchers { @@ -27,15 +26,14 @@ class PriorityDispatcherSpec extends WordSpec with MustMatchers { throughput = 1, mailboxType = mboxType) - val actor = actorOf(new Actor { - self.dispatcher = dispatcher + val actor = actorOf(Props(new Actor { var acc: List[Int] = Nil def receive = { case i: Int ⇒ acc = i :: acc case 'Result ⇒ self tryReply acc } - }).start() + }).withDispatcher(dispatcher)).asInstanceOf[LocalActorRef] dispatcher.suspend(actor) //Make sure the actor isn't treating any messages, let it buffer the incoming messages diff --git a/akka-actor-tests/src/test/scala/akka/misc/ActorRegistrySpec.scala b/akka-actor-tests/src/test/scala/akka/misc/ActorRegistrySpec.scala index 0dfba61920..1939ccba1f 100644 --- a/akka-actor-tests/src/test/scala/akka/misc/ActorRegistrySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/misc/ActorRegistrySpec.scala @@ -63,11 +63,10 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll { @Test def shouldFindThingsFromLocalActorRegistry { Actor.registry.local.shutdownAll - val actor = actorOf[TestActor]("test-actor-1") - actor.start - val found = Actor.registry.local.find({ case a: ActorRef if a.actor.isInstanceOf[TestActor] ⇒ a }) + val actor = actorOf[TestActor]("test-actor-1").start() + val found: Option[LocalActorRef] = Actor.registry.local.find({ case a: LocalActorRef if a.actorInstance.get().isInstanceOf[TestActor] ⇒ a }) assert(found.isDefined) - assert(found.get.actor.isInstanceOf[TestActor]) + assert(found.get.actorInstance.get().isInstanceOf[TestActor]) assert(found.get.address === "test-actor-1") actor.stop } @@ -75,13 +74,13 @@ class ActorRegistrySpec extends JUnitSuite with BeforeAndAfterAll { @Test def shouldGetAllActorsFromLocalActorRegistry { Actor.registry.local.shutdownAll - val actor1 = actorOf[TestActor]("test-actor-1").start - val actor2 = actorOf[TestActor]("test-actor-2").start + val actor1 = actorOf[TestActor]("test-actor-1").start() + val actor2 = actorOf[TestActor]("test-actor-2").start() val actors = Actor.registry.local.actors assert(actors.size === 2) - assert(actors.head.actor.isInstanceOf[TestActor]) + assert(actors.head.asInstanceOf[LocalActorRef].actorInstance.get().isInstanceOf[TestActor]) assert(actors.head.address === "test-actor-2") - assert(actors.last.actor.isInstanceOf[TestActor]) + assert(actors.last.asInstanceOf[LocalActorRef].actorInstance.get().isInstanceOf[TestActor]) assert(actors.last.address === "test-actor-1") actor1.stop actor2.stop diff --git a/akka-actor-tests/src/test/scala/akka/misc/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/misc/SchedulerSpec.scala index 9fa7a8cced..437f1db2e6 100644 --- a/akka-actor-tests/src/test/scala/akka/misc/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/misc/SchedulerSpec.scala @@ -119,8 +119,6 @@ class SchedulerSpec extends JUnitSuite { val pingLatch = new CountDownLatch(6) val actor = actorOf(new Actor { - self.lifeCycle = Permanent - def receive = { case Ping ⇒ pingLatch.countDown() case Crash ⇒ throw new Exception("CRASH") diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala index 5d5bf98943..bf4a08b425 100755 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala @@ -6,12 +6,10 @@ import java.util.concurrent.CountDownLatch import java.util.concurrent.TimeUnit import akka.performance.trading.domain._ import akka.performance.trading.common._ -import akka.actor.ActorRef -import akka.actor.Actor import akka.actor.Actor.actorOf import akka.dispatch.Dispatchers -import akka.actor.PoisonPill import akka.event.EventHandler +import akka.actor.{ Props, ActorRef, Actor, PoisonPill } abstract class AkkaPerformanceTest extends BenchmarkScenarios { @@ -38,7 +36,7 @@ abstract class AkkaPerformanceTest extends BenchmarkScenarios { val receivers = tradingSystem.orderReceivers.toIndexedSeq val clients = (for (i ← 0 until numberOfClients) yield { val receiver = receivers(i % receivers.size) - actorOf(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), delayMs)) + actorOf(Props(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), delayMs)).withDispatcher(clientDispatcher)) }).toList clients.foreach(_.start) @@ -54,9 +52,6 @@ abstract class AkkaPerformanceTest extends BenchmarkScenarios { } class Client(orderReceiver: ActorRef, orders: List[Order], latch: CountDownLatch, repeat: Int, delayMs: Int) extends Actor { - - self.dispatcher = clientDispatcher - def this(orderReceiver: ActorRef, orders: List[Order], latch: CountDownLatch, repeat: Int) { this(orderReceiver, orders, latch, repeat, 0) } diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/MatchingEngine.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/MatchingEngine.scala index 7b531444f0..03c6fa79b1 100755 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/MatchingEngine.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/MatchingEngine.scala @@ -16,13 +16,9 @@ trait MatchingEngine { } -class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook], disp: Option[MessageDispatcher]) +class AkkaMatchingEngine(val meId: String, val orderbooks: List[Orderbook]) extends Actor with MatchingEngine { - for (d ← disp) { - self.dispatcher = d - } - var standby: Option[ActorRef] = None def receive = { diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala index ae8e879421..9a6ba3ed9f 100755 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala @@ -25,14 +25,9 @@ trait OrderReceiver { } -class AkkaOrderReceiver(disp: Option[MessageDispatcher]) - extends Actor with OrderReceiver { +class AkkaOrderReceiver extends Actor with OrderReceiver { type ME = ActorRef - for (d ← disp) { - self.dispatcher = d - } - def receive = { case routing @ MatchingEngineRouting(mapping) ⇒ refreshMatchingEnginePartitions(routing.asInstanceOf[MatchingEngineRouting[ActorRef]]) diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/TradingSystem.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/TradingSystem.scala index ed822cf1be..38ad3f7c3c 100755 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/TradingSystem.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/TradingSystem.scala @@ -3,9 +3,8 @@ package akka.performance.trading.common import akka.performance.trading.domain.Orderbook import akka.performance.trading.domain.OrderbookRepository import akka.actor.Actor._ -import akka.actor.ActorRef -import akka.actor.PoisonPill import akka.dispatch.MessageDispatcher +import akka.actor.{ Props, ActorRef, PoisonPill } trait TradingSystem { type ME @@ -69,7 +68,10 @@ class AkkaTradingSystem extends TradingSystem { } def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = - actorOf(new AkkaMatchingEngine(meId, orderbooks, meDispatcher)) + meDispatcher match { + case Some(d) ⇒ actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d)) + case _ ⇒ actorOf(Props(new AkkaMatchingEngine(meId, orderbooks))) + } override def createOrderReceivers: List[ActorRef] = { (1 to 10).toList map (i ⇒ createOrderReceiver()) @@ -87,8 +89,10 @@ class AkkaTradingSystem extends TradingSystem { MatchingEngineRouting(Map() ++ rules) } - def createOrderReceiver() = - actorOf(new AkkaOrderReceiver(orDispatcher)) + def createOrderReceiver() = orDispatcher match { + case Some(d) ⇒ actorOf(Props(new AkkaOrderReceiver()).withDispatcher(d)) + case _ ⇒ actorOf(Props(new AkkaOrderReceiver())) + } override def start() { for (MatchingEngineInfo(p, s, o) ← matchingEngines) { diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayMatchingEngine.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayMatchingEngine.scala index 2b48107f8d..728c86c67c 100755 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayMatchingEngine.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayMatchingEngine.scala @@ -7,8 +7,7 @@ import akka.performance.trading.domain.Order import akka.performance.trading.domain.Orderbook import akka.performance.trading.common.AkkaMatchingEngine -class OneWayMatchingEngine(meId: String, orderbooks: List[Orderbook], disp: Option[MessageDispatcher]) - extends AkkaMatchingEngine(meId, orderbooks, disp) { +class OneWayMatchingEngine(meId: String, orderbooks: List[Orderbook]) extends AkkaMatchingEngine(meId, orderbooks) { override def handleOrder(order: Order) { orderbooksMap.get(order.orderbookSymbol) match { diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayOrderReceiver.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayOrderReceiver.scala index 16e3a41048..604dcadb5f 100755 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayOrderReceiver.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayOrderReceiver.scala @@ -6,8 +6,7 @@ import akka.event.EventHandler import akka.performance.trading.domain._ import akka.performance.trading.common.AkkaOrderReceiver -class OneWayOrderReceiver(disp: Option[MessageDispatcher]) - extends AkkaOrderReceiver(disp) { +class OneWayOrderReceiver extends AkkaOrderReceiver { override def placeOrder(order: Order) = { val matchingEngine = matchingEngineForOrderbook.get(order.orderbookSymbol) diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayPerformanceTest.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayPerformanceTest.scala index 169dd02ebf..9ad42baa74 100755 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayPerformanceTest.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayPerformanceTest.scala @@ -5,16 +5,18 @@ import java.util.concurrent.TimeUnit import org.junit.Test import akka.actor.Actor.actorOf -import akka.actor.ActorRef import akka.performance.trading.common.AkkaPerformanceTest import akka.performance.trading.common.Rsp import akka.performance.trading.domain._ +import akka.actor.{ Props, ActorRef } class OneWayPerformanceTest extends AkkaPerformanceTest { override def createTradingSystem: TS = new OneWayTradingSystem { - override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = - actorOf(new OneWayMatchingEngine(meId, orderbooks, meDispatcher) with LatchMessageCountDown) + override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match { + case Some(d) ⇒ actorOf(Props(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown).withDispatcher(d)) + case _ ⇒ actorOf(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown) + } } override def placeOrder(orderReceiver: ActorRef, order: Order): Rsp = { diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayTradingSystem.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayTradingSystem.scala index 57737e9c0e..56fcea8ad8 100755 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayTradingSystem.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayTradingSystem.scala @@ -1,16 +1,20 @@ package akka.performance.trading.oneway import akka.actor.Actor.actorOf -import akka.actor.ActorRef import akka.performance.trading.common.AkkaTradingSystem import akka.performance.trading.domain.Orderbook +import akka.actor.{ Props, ActorRef } class OneWayTradingSystem extends AkkaTradingSystem { - override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = - actorOf(new OneWayMatchingEngine(meId, orderbooks, meDispatcher)) + override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match { + case Some(d) ⇒ actorOf(Props(new OneWayMatchingEngine(meId, orderbooks)).withDispatcher(d)) + case _ ⇒ actorOf(Props(new OneWayMatchingEngine(meId, orderbooks))) + } - override def createOrderReceiver() = - actorOf(new OneWayOrderReceiver(orDispatcher)) + override def createOrderReceiver() = orDispatcher match { + case Some(d) ⇒ actorOf(Props[OneWayOrderReceiver].withDispatcher(d)) + case _ ⇒ actorOf(Props[OneWayOrderReceiver]) + } } \ No newline at end of file diff --git a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala index cce7a14d11..8a30118433 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala @@ -6,9 +6,10 @@ import akka.dispatch.{ KeptPromise, Future } import java.util.concurrent.atomic.AtomicInteger import akka.actor.Actor._ import akka.testkit.Testing._ -import akka.actor.{ TypedActor, Actor } +import akka.actor.{ TypedActor, Actor, Props } import akka.testkit.{ TestLatch, filterEvents, EventFilter, filterException } import akka.util.duration._ +import akka.config.Supervision.OneForOneStrategy object ActorPoolSpec { @@ -22,6 +23,8 @@ object ActorPoolSpec { new KeptPromise(Right(x * x)) } } + + val faultHandler = OneForOneStrategy(List(classOf[Exception]), 5, 1000) } class ActorPoolSpec extends WordSpec with MustMatchers { @@ -34,7 +37,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { val count = new AtomicInteger(0) val pool = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with FixedCapacityStrategy with SmallestMailboxSelector { + Props(new Actor with DefaultActorPool with FixedCapacityStrategy with SmallestMailboxSelector { def factory = actorOf(new Actor { def receive = { case _ ⇒ @@ -49,7 +52,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { def partialFill = true def instance = factory def receive = _route - }).start() + }).withFaultHandler(faultHandler)) val successes = TestLatch(2) val successCounter = actorOf(new Actor { @@ -74,7 +77,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { "pass ticket #705" in { val pool = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicFilter { + Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicFilter { def lowerBound = 2 def upperBound = 20 def rampupRate = 0.1 @@ -93,7 +96,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { } } }) - }).start() + }).withFaultHandler(faultHandler)) try { (for (count ← 1 to 500) yield pool.?("Test", 20000)) foreach { @@ -112,7 +115,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { val count = new AtomicInteger(0) val pool = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { + Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { def factory = actorOf(new Actor { def receive = { case n: Int ⇒ @@ -129,7 +132,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { def selectionCount = 1 def instance = factory def receive = _route - }).start() + }).withFaultHandler(faultHandler)) // first message should create the minimum number of delgates @@ -177,7 +180,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { val count = new AtomicInteger(0) val pool = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { + Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { def factory = actorOf(new Actor { def receive = { case n: Int ⇒ @@ -195,7 +198,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { def selectionCount = 1 def instance = factory def receive = _route - }).start() + }).withFaultHandler(faultHandler)) var loops = 0 def loop(t: Int) = { @@ -231,7 +234,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { val delegates = new java.util.concurrent.ConcurrentHashMap[String, String] val pool1 = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter { + Props(new Actor with DefaultActorPool with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter { def factory = actorOf(new Actor { def receive = { case _ ⇒ @@ -246,7 +249,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { def partialFill = true def instance = factory def receive = _route - }).start() + }).withFaultHandler(faultHandler)) pool1 ! "a" pool1 ! "b" @@ -260,7 +263,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { delegates.clear() val pool2 = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter { + Props(new Actor with DefaultActorPool with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter { def factory = actorOf(new Actor { def receive = { case _ ⇒ @@ -275,7 +278,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { def partialFill = false def instance = factory def receive = _route - }).start() + }).withFaultHandler(faultHandler)) pool2 ! "a" pool2 ! "b" @@ -290,7 +293,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { val latch = TestLatch(10) val pool = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup { + Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup { def factory = actorOf(new Actor { def receive = { case n: Int ⇒ @@ -309,7 +312,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { def backoffThreshold = 0.50 def instance = factory def receive = _route - }).start() + }).withFaultHandler(faultHandler)) // put some pressure on the pool @@ -336,7 +339,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { "support typed actors" in { import RoutingSpec._ import TypedActor._ - def createPool = new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup { + def createPool = new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup { def lowerBound = 1 def upperBound = 5 def pressureThreshold = 1 @@ -349,7 +352,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { def receive = _route } - val pool = createProxy[Foo](createPool) + val pool = createProxy[Foo](createPool, Props().withFaultHandler(faultHandler)) val results = for (i ← 1 to 100) yield (i, pool.sq(i, 100)) @@ -364,7 +367,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { var keepDying = false val pool1 = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { + Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { def lowerBound = 2 def upperBound = 5 def rampupRate = 0.1 @@ -385,10 +388,10 @@ class ActorPoolSpec extends WordSpec with MustMatchers { case _ ⇒ pingCount.incrementAndGet } }).start() - }).start() + }).withFaultHandler(faultHandler)) val pool2 = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { + Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { def lowerBound = 2 def upperBound = 5 def rampupRate = 0.1 @@ -400,7 +403,6 @@ class ActorPoolSpec extends WordSpec with MustMatchers { def receive = _route def pressureThreshold = 1 def factory = actorOf(new Actor { - self.lifeCycle = Permanent if (deathCount.get > 5) deathCount.set(0) if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } def receive = { @@ -410,10 +412,10 @@ class ActorPoolSpec extends WordSpec with MustMatchers { case _ ⇒ pingCount.incrementAndGet } }).start() - }).start() + }).withFaultHandler(faultHandler)) val pool3 = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with RoundRobinSelector with BasicFilter { + Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with RoundRobinSelector with BasicFilter { def lowerBound = 2 def upperBound = 5 def rampupRate = 0.1 @@ -424,8 +426,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { def instance = factory def receive = _route def pressureThreshold = 1 - def factory = actorOf(new Actor { - self.lifeCycle = Temporary + def factory = actorOf(Props(new Actor { if (deathCount.get > 5) deathCount.set(0) if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } def receive = { @@ -434,8 +435,8 @@ class ActorPoolSpec extends WordSpec with MustMatchers { throw new RuntimeException case _ ⇒ pingCount.incrementAndGet } - }).start() - }).start() + }).withLifeCycle(Temporary)) + }).withFaultHandler(faultHandler)) // default lifecycle // actor comes back right away @@ -508,14 +509,10 @@ class ActorPoolSpec extends WordSpec with MustMatchers { val deathCount = new AtomicInteger(0) var keepDying = false - trait LimitedTrapSupervisionConfig extends ActorPoolSupervisionConfig { - def poolFaultHandler = OneForOneStrategy(List(classOf[IllegalStateException]), 5, 1000) - } - object BadState val pool1 = actorOf( - new Actor with DefaultActorPool with LimitedTrapSupervisionConfig with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { + Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { def lowerBound = 2 def upperBound = 5 def rampupRate = 0.1 @@ -538,7 +535,7 @@ class ActorPoolSpec extends WordSpec with MustMatchers { case _ ⇒ pingCount.incrementAndGet } }).start() - }).start() + }).withFaultHandler(OneForOneStrategy(List(classOf[IllegalStateException]), 5, 1000))) // actor comes back right away pingCount.set(0) 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 1460c81e03..749e42dbd3 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -3,7 +3,6 @@ package akka.routing import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import akka.routing._ -import akka.routing.Router import java.util.concurrent.atomic.AtomicInteger import akka.actor.Actor._ import akka.actor.{ ActorRef, Actor } diff --git a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala index 849070752b..f024546e15 100644 --- a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala @@ -5,13 +5,14 @@ import akka.actor._ import akka.routing._ import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers +import akka.config.Supervision.OneForOneStrategy class Ticket703Spec extends WordSpec with MustMatchers { "A ? call to an actor pool" should { "reuse the proper timeout" in { val actorPool = actorOf( - new Actor with DefaultActorPool with DefaultActorPoolSupervisionConfig with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { + Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { def lowerBound = 2 def upperBound = 20 def rampupRate = 0.1 @@ -27,7 +28,7 @@ class Ticket703Spec extends WordSpec with MustMatchers { self.tryReply("Response") } }) - }).start() + }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 5, 1000))) (actorPool.?("Ping", 7000)).await.result must be === Some("Response") } } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 6a8408f6d9..50c20a90b7 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -36,7 +36,7 @@ sealed trait LifeCycleMessage extends Serializable */ sealed trait AutoReceivedMessage { self: LifeCycleMessage ⇒ } -case class HotSwap(code: ActorRef ⇒ Actor.Receive, discardOld: Boolean = true) extends AutoReceivedMessage with LifeCycleMessage { +case class HotSwap(code: SelfActorRef ⇒ Actor.Receive, discardOld: Boolean = true) extends AutoReceivedMessage with LifeCycleMessage { /** * Java API @@ -137,6 +137,7 @@ object Timeout { /** * The default timeout, based on the config setting 'akka.actor.timeout' */ + @BeanProperty implicit val default = new Timeout(Actor.TIMEOUT) /** @@ -172,8 +173,8 @@ object Actor { */ type Receive = PartialFunction[Any, Unit] - private[actor] val actorRefInCreation = new ThreadLocal[Stack[ActorRef]] { - override def initialValue = Stack[ActorRef]() + private[actor] val actorRefInCreation = new ThreadLocal[Stack[ScalaActorRef with SelfActorRef]] { + override def initialValue = Stack[ScalaActorRef with SelfActorRef]() } private[akka] val TIMEOUT = Duration(config.getInt("akka.actor.timeout", 5), TIME_UNIT).toMillis @@ -343,7 +344,7 @@ object Actor { * */ def actorOf[T <: Actor](creator: ⇒ T, address: String): ActorRef = { - createActor(address, () ⇒ new LocalActorRef(() ⇒ creator, address)) + createActor(address, () ⇒ new LocalActorRef(Props(creator = () ⇒ creator), address)) } /** @@ -366,28 +367,22 @@ object Actor { * JAVA API */ def actorOf[T <: Actor](creator: Creator[T], address: String): ActorRef = { - createActor(address, () ⇒ new LocalActorRef(() ⇒ creator.create, address)) + createActor(address, () ⇒ new LocalActorRef(Props(creator = () ⇒ creator.create), address)) } + def actorOf(props: Props): ActorRef = actorOf(props, newUuid.toString) //TODO FIXME - def actorOf(props: Props): ActorRef = { + def actorOf(props: Props, address: String): ActorRef = { //TODO Implement support for configuring by deployment ID etc //TODO If localOnly = true, never use the config file deployment and always create a new actor //TODO If deployId matches an already created actor (Ahead-of-time deployed) return that actor //TODO If deployId exists in config, it will override the specified Props (should we attempt to merge?) - val address = props.deployId match { //TODO handle deployId separately from address? + /*val address = props.deployId match { //TODO handle deployId separately from address? case "" | null ⇒ newUuid().toString case other ⇒ other - } - val newActor = new LocalActorRef(props.creator, address) - newActor.dispatcher = props.dispatcher - newActor.faultHandler = props.faultHandler - newActor.lifeCycle = props.lifeCycle - newActor.timeout = props.timeout.duration.toMillis - newActor.receiveTimeout = props.receiveTimeout.map(_.toMillis) - props.supervisor.foreach(newActor.link(_)) - newActor.start + } */ + new LocalActorRef(props, address).start() } def localActorOf[T <: Actor: Manifest]: ActorRef = { @@ -407,11 +402,11 @@ object Actor { } def localActorOf[T <: Actor](factory: ⇒ T): ActorRef = { - new LocalActorRef(() ⇒ factory, newUuid().toString) + new LocalActorRef(Props(creator = () ⇒ factory), newUuid().toString) } def localActorOf[T <: Actor](factory: ⇒ T, address: String): ActorRef = { - new LocalActorRef(() ⇒ factory, address) + new LocalActorRef(Props(creator = () ⇒ factory), address) } /** @@ -432,13 +427,9 @@ object Actor { * */ def spawn(body: ⇒ Unit)(implicit dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher): Unit = { - case object Spawn - actorOf(new Actor() { - self.dispatcher = dispatcher - def receive = { - case Spawn ⇒ try { body } finally { self.stop() } - } - }).start() ! Spawn + actorOf(Props(new Actor() { + def receive = { case "go" ⇒ try { body } finally { self.stop() } } + }).withDispatcher(dispatcher)) ! "go" } /** @@ -464,7 +455,7 @@ object Actor { } private[akka] def newLocalActorRef(clazz: Class[_ <: Actor], address: String): ActorRef = { - new LocalActorRef(() ⇒ { + new LocalActorRef(Props(creator = () ⇒ { import ReflectiveAccess.{ createInstance, noParams, noArgs } createInstance[Actor](clazz.asInstanceOf[Class[_]], noParams, noArgs) match { case Right(actor) ⇒ actor @@ -480,7 +471,7 @@ object Actor { "\nif so put it outside the class/trait, f.e. in a companion object," + "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.", cause) } - }, address) + }), address) } private[akka] def newClusterActorRef(factory: () ⇒ ActorRef, address: String, deploy: Deploy): ActorRef = @@ -548,7 +539,7 @@ object Actor { *

* Here you find functions like: * - !, ? and forward - * - link, unlink, startLink etc + * - link, unlink etc * - start, stop * - etc. * @@ -593,7 +584,7 @@ trait Actor { * the 'forward' function. */ @transient - val someSelf: Some[ActorRef] = { + val someSelf: Some[ScalaActorRef with SelfActorRef] = { val refStack = Actor.actorRefInCreation.get if (refStack.isEmpty) throw new ActorInitializationException( "\n\tYou can not create an instance of an " + getClass.getName + " explicitly using 'new MyActor'." + @@ -619,7 +610,7 @@ trait Actor { * Mainly for internal use, functions as the implicit sender references when invoking * one of the message send functions ('!' and '?'). */ - def optionSelf: Option[ActorRef] = someSelf + def optionSelf: Option[ScalaActorRef with SelfActorRef] = someSelf /** * The 'self' field holds the ActorRef for this actor. @@ -649,7 +640,7 @@ trait Actor { * */ @transient - implicit val self: ScalaActorRef = someSelf.get + implicit val self = someSelf.get /** * User overridable callback/setting. diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 13becad246..57c837e0c7 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -47,55 +47,78 @@ private[akka] object ActorRefInternals { * TODO document me */ object Props { - object Default { - val creator: () ⇒ Actor = () ⇒ throw new UnsupportedOperationException("No actor creator specified!") - val deployId: String = "" - val dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher - val timeout: Timeout = Timeout(Duration(Actor.TIMEOUT, "millis")) - val receiveTimeout: Option[Duration] = None - val lifeCycle: LifeCycle = Permanent - val faultHandler: FaultHandlingStrategy = NoFaultHandlingStrategy - val supervisor: Option[ActorRef] = None - val localOnly: Boolean = false - } + final val defaultCreator: () ⇒ Actor = () ⇒ throw new UnsupportedOperationException("No actor creator specified!") + final val defaultDeployId: String = "" + final val defaultDispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher + final val defaultTimeout: Timeout = Timeout(Duration(Actor.TIMEOUT, "millis")) + final val defaultLifeCycle: LifeCycle = Permanent + final val defaultFaultHandler: FaultHandlingStrategy = NoFaultHandlingStrategy + final val defaultSupervisor: Option[ActorRef] = None + final val defaultLocalOnly: Boolean = false - val default = new Props() + /** + * The default Props instance, uses the settings from the Props object starting with default* + */ + final val default = new Props() - def apply[T <: Actor: Manifest]: Props = - default.withCreator(() ⇒ implicitly[Manifest[T]].erasure.asInstanceOf[Class[_ <: Actor]].newInstance) + /** + * Returns a cached default implementation of Props + */ + def apply(): Props = default + /** + * Returns a Props that has default values except for "creator" which will be a function that creates an instance + * of the supplied type using the default constructor + */ + def apply[T <: Actor: ClassManifest]: Props = + default.withCreator(implicitly[ClassManifest[T]].erasure.asInstanceOf[Class[_ <: Actor]].newInstance) + + /** + * Returns a Props that has default values except for "creator" which will be a function that creates an instance + * of the supplied class using the default constructor + */ def apply(actorClass: Class[_ <: Actor]): Props = - default.withCreator(() ⇒ actorClass.newInstance) + default.withCreator(actorClass.newInstance) + + /** + * Returns a Props that has default values except for "creator" which will be a function that creates an instance + * using the supplied thunk + */ + def apply(creator: ⇒ Actor): Props = default.withCreator(creator) + + def apply(behavior: (ScalaActorRef with SelfActorRef) ⇒ Actor.Receive): Props = + apply(new Actor { def receive = behavior(self) }) } /** * ActorRef configuration object, this is thread safe and fully sharable */ -case class Props(creator: () ⇒ Actor = Props.Default.creator, - deployId: String = Props.Default.deployId, - dispatcher: MessageDispatcher = Props.Default.dispatcher, - timeout: Timeout = Props.Default.timeout, - receiveTimeout: Option[Duration] = Props.Default.receiveTimeout, - lifeCycle: LifeCycle = Props.Default.lifeCycle, - faultHandler: FaultHandlingStrategy = Props.Default.faultHandler, - supervisor: Option[ActorRef] = Props.Default.supervisor, - localOnly: Boolean = Props.Default.localOnly) { - +case class Props(creator: () ⇒ Actor = Props.defaultCreator, + deployId: String = Props.defaultDeployId, + dispatcher: MessageDispatcher = Props.defaultDispatcher, + timeout: Timeout = Props.defaultTimeout, + lifeCycle: LifeCycle = Props.defaultLifeCycle, + faultHandler: FaultHandlingStrategy = Props.defaultFaultHandler, + supervisor: Option[ActorRef] = Props.defaultSupervisor, + localOnly: Boolean = Props.defaultLocalOnly) { + /** + * No-args constructor that sets all the default values + * Java API + */ def this() = this( - creator = Props.Default.creator, - deployId = Props.Default.deployId, - dispatcher = Props.Default.dispatcher, - timeout = Props.Default.timeout, - receiveTimeout = Props.Default.receiveTimeout, - lifeCycle = Props.Default.lifeCycle, - faultHandler = Props.Default.faultHandler, - supervisor = Props.Default.supervisor, - localOnly = Props.Default.localOnly) + creator = Props.defaultCreator, + deployId = Props.defaultDeployId, + dispatcher = Props.defaultDispatcher, + timeout = Props.defaultTimeout, + lifeCycle = Props.defaultLifeCycle, + faultHandler = Props.defaultFaultHandler, + supervisor = Props.defaultSupervisor, + localOnly = Props.defaultLocalOnly) /** * Returns a new Props with the specified creator set * Scala API */ - def withCreator(c: () ⇒ Actor) = copy(creator = c) + def withCreator(c: ⇒ Actor) = copy(creator = () ⇒ c) /** * Returns a new Props with the specified creator set @@ -151,24 +174,6 @@ case class Props(creator: () ⇒ Actor = Props.Default.creator, */ def withSupervisor(s: scala.Option[ActorRef]) = copy(supervisor = s) - /** - * Returns a new Props with the specified receive timeout set, if null, it's equivalent to withReceiveTimeout(Option.none()) - * Java API - */ - def withReceiveTimeout(r: Duration) = copy(receiveTimeout = Option(r)) - - /** - * Returns a new Props with the specified receive timeout set - * Java API - */ - def withReceiveTimeout(r: akka.japi.Option[Duration]) = copy(receiveTimeout = r.asScala) - - /** - * Returns a new Props with the specified receive timeout set - * Scala API - */ - def withReceiveTimeout(r: scala.Option[Duration]) = copy(receiveTimeout = r) - /** * Returns a new Props with the specified localOnly set * Java and Scala API @@ -208,7 +213,7 @@ case class Props(creator: () ⇒ Actor = Props.Default.creator, * * @author Jonas Bonér */ -abstract class ActorRef extends ActorRefShared with ForwardableChannel with ReplyChannel[Any] with java.lang.Comparable[ActorRef] with Serializable { +abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyChannel[Any] with java.lang.Comparable[ActorRef] with Serializable { scalaRef: ScalaActorRef ⇒ // Only mutable for RemoteServer in order to maintain identity across nodes @volatile @@ -218,97 +223,6 @@ abstract class ActorRef extends ActorRefShared with ForwardableChannel with Repl def address: String - /** - * User overridable callback/setting. - *

- * Defines the default timeout for '?'/'ask' invocations, - * e.g. the timeout for the future returned by the call to '?'/'ask'. - */ - @deprecated("Will be replaced by implicit-scoped timeout on all methods that needs it, will default to timeout specified in config", "1.1") - @BeanProperty - @volatile - var timeout: Long = Actor.TIMEOUT - - /** - * User overridable callback/setting. - *

- * Defines the default timeout for an initial receive invocation. - * When specified, the receive function should be able to handle a 'ReceiveTimeout' message. - */ - @volatile - var receiveTimeout: Option[Long] = None - - /** - * Akka Java API.

- * Defines the default timeout for an initial receive invocation. - * When specified, the receive function should be able to handle a 'ReceiveTimeout' message. - */ - def setReceiveTimeout(timeout: Long) { - this.receiveTimeout = Some(timeout) - } - - def getReceiveTimeout: Option[Long] = receiveTimeout - - /** - * Akka Java API.

- * A faultHandler defines what should be done when a linked actor signals an error. - *

- * Can be one of: - *

-   * getContext().setFaultHandler(new AllForOneStrategy(new Class[]{Throwable.class},maxNrOfRetries, withinTimeRange));
-   * 
- * Or: - *
-   * getContext().setFaultHandler(new OneForOneStrategy(new Class[]{Throwable.class},maxNrOfRetries, withinTimeRange));
-   * 
- */ - def setFaultHandler(handler: FaultHandlingStrategy) - - def getFaultHandler: FaultHandlingStrategy - - /** - * Akka Java API.

- * A lifeCycle defines whether the actor will be stopped on error (Temporary) or if it can be restarted (Permanent) - *

- * Can be one of: - * - * import static akka.config.Supervision.*; - *

-   * getContext().setLifeCycle(permanent());
-   * 
- * Or: - *
-   * getContext().setLifeCycle(temporary());
-   * 
- */ - def setLifeCycle(lifeCycle: LifeCycle) - - def getLifeCycle: LifeCycle - - /** - * Akka Java API.

- * The default dispatcher is the Dispatchers.globalDispatcher. - * This means that all actors will share the same event-driven executor based dispatcher. - *

- * You can override it so it fits the specific use-case that the actor is used for. - * See the akka.dispatch.Dispatchers class for the different - * dispatchers available. - *

- * The default is also that all actors that are created and spawned from within this actor - * is sharing the same dispatcher as its creator. - */ - def setDispatcher(dispatcher: MessageDispatcher) { - this.dispatcher = dispatcher - } - - def getDispatcher: MessageDispatcher = dispatcher - - /** - * Holds the hot swapped partial function. - */ - @volatile - protected[akka] var hotswap = Stack[PartialFunction[Any, Unit]]() - /** * This is a reference to the message currently being processed by the actor */ @@ -327,27 +241,6 @@ abstract class ActorRef extends ActorRefShared with ForwardableChannel with Repl def uuid = _uuid - /** - * Akka Java API.

- * The reference sender Actor of the last received message. - * Is defined if the message was sent from another Actor, else None. - */ - @deprecated("will be removed in 2.0, use channel instead", "1.2") - def getSender: Option[ActorRef] = sender - - /** - * Akka Java API.

- * The reference sender future of the last received message. - * Is defined if the message was sent with sent with '?'/'ask', else None. - */ - @deprecated("will be removed in 2.0, use channel instead", "1.2") - def getSenderFuture: Option[Promise[Any]] = senderFuture - - /** - * Is the actor being restarted? - */ - def isBeingRestarted: Boolean = _status == ActorRefInternals.BEING_RESTARTED - /** * Is the actor running? */ @@ -361,11 +254,6 @@ abstract class ActorRef extends ActorRefShared with ForwardableChannel with Repl */ def isShutdown: Boolean = _status == ActorRefInternals.SHUTDOWN - /** - * Is the actor ever started? - */ - def isUnstarted: Boolean = _status == ActorRefInternals.UNSTARTED - /** * Only for internal use. UUID is effectively final. */ @@ -373,6 +261,13 @@ abstract class ActorRef extends ActorRefShared with ForwardableChannel with Repl _uuid = uid } + protected[akka] def timeout: Long = Props.defaultTimeout.duration.toMillis //TODO Remove me if possible + + /** + * Defines the life-cycle for a supervised actor. + */ + protected[akka] def lifeCycle: LifeCycle = UndefinedLifeCycle //TODO Remove me if possible + /** * Akka Java API.

* @see ask(message: AnyRef, sender: ActorRef): Future[_] @@ -412,46 +307,11 @@ abstract class ActorRef extends ActorRefShared with ForwardableChannel with Repl * Akka Java API.

* Forwards the message specified to this actor and preserves the original sender of the message */ - def forward(message: AnyRef, sender: ActorRef) { + def forward(message: AnyRef, sender: SelfActorRef) { if (sender eq null) throw new IllegalArgumentException("The 'sender' argument to 'forward' can't be null") else forward(message)(sender) } - /** - * Akka Scala & Java API - * Use self.reply(..) to reply with a message to the original sender of the message currently - * being processed. This method fails if the original sender of the message could not be determined with an - * IllegalStateException. - * - * If you don't want deal with this IllegalStateException, but just a boolean, just use the tryReply(...) - * version. - * - *

- * Throws an IllegalStateException if unable to determine what to reply to. - */ - def reply(message: Any) = channel.!(message)(this) - - /** - * Akka Scala & Java API - * Use tryReply(..) to try reply with a message to the original sender of the message currently - * being processed. This method - *

- * Returns true if reply was sent, and false if unable to determine what to reply to. - * - * If you would rather have an exception, check the reply(..) version. - */ - def tryReply(message: Any): Boolean = channel.tryTell(message)(this) - - /** - * Sets the dispatcher for this actor. Needs to be invoked before the actor is started. - */ - def dispatcher_=(md: MessageDispatcher) - - /** - * Get the dispatcher for this actor. - */ - def dispatcher: MessageDispatcher - /** * Starts up the actor and its message queue. */ @@ -478,17 +338,12 @@ abstract class ActorRef extends ActorRefShared with ForwardableChannel with Repl * 'trap' these exceptions and automatically restart the linked actors according to the restart strategy * defined by the 'faultHandler'. */ - def link(actorRef: ActorRef) + def link(actorRef: ActorRef): ActorRef /** * Unlink the actor. */ - def unlink(actorRef: ActorRef) - - /** - * Atomically start and link an actor. - */ - def startLink(actorRef: ActorRef): ActorRef + def unlink(actorRef: ActorRef): ActorRef /** * Returns the supervisor, if there is one. @@ -501,6 +356,127 @@ abstract class ActorRef extends ActorRefShared with ForwardableChannel with Repl */ def getSupervisor: ActorRef = supervisor getOrElse null + protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit + + protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( + message: Any, + timeout: Timeout, + channel: UntypedChannel): Future[Any] + + protected[akka] def supervisor_=(sup: Option[ActorRef]) + + protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) + + override def hashCode: Int = HashCode.hash(HashCode.SEED, uuid) + + override def equals(that: Any): Boolean = { + that.isInstanceOf[ActorRef] && + that.asInstanceOf[ActorRef].uuid == uuid + } + + override def toString = "Actor[%s:%s]".format(address, uuid) +} + +abstract class SelfActorRef extends ActorRef with ForwardableChannel { self: LocalActorRef with ScalaActorRef ⇒ + /** + * Holds the hot swapped partial function. + * WARNING: DO NOT USE THIS, IT IS INTERNAL AKKA USE ONLY + */ + @volatile + protected[akka] var hotswap = Stack[PartialFunction[Any, Unit]]() + + /** + * User overridable callback/setting. + *

+ * Defines the default timeout for an initial receive invocation. + * When specified, the receive function should be able to handle a 'ReceiveTimeout' message. + */ + @volatile + var receiveTimeout: Option[Long] = None + + /** + * Akka Java API.

+ * The reference sender Actor of the last received message. + * Is defined if the message was sent from another Actor, else None. + */ + @deprecated("will be removed in 2.0, use channel instead", "1.2") + def getSender: Option[ActorRef] = sender + + /** + * Akka Java API.

+ * The reference sender future of the last received message. + * Is defined if the message was sent with sent with '?'/'ask', else None. + */ + @deprecated("will be removed in 2.0, use channel instead", "1.2") + def getSenderFuture: Option[Promise[Any]] = senderFuture + + /** + * The reference sender Actor of the last received message. + * Is defined if the message was sent from another Actor, else None. + */ + @deprecated("will be removed in 2.0, use channel instead", "1.2") + def sender: Option[ActorRef] + + /** + * The reference sender future of the last received message. + * Is defined if the message was sent with sent with '?'/'ask', else None. + */ + @deprecated("will be removed in 2.0, use channel instead", "1.2") + def senderFuture(): Option[Promise[Any]] + + /** + * Abstraction for unification of sender and senderFuture for later reply + */ + def channel: UntypedChannel = self.currentMessage match { + case null ⇒ NullChannel + case msg ⇒ msg.channel + } + + /** + * Akka Java API.

+ * Defines the default timeout for an initial receive invocation. + * When specified, the receive function should be able to handle a 'ReceiveTimeout' message. + */ + def setReceiveTimeout(timeout: Long): Unit = this.receiveTimeout = Some(timeout) + + /** + * Akka Java API.

+ * Gets the current receive timeout + * When specified, the receive method should be able to handle a 'ReceiveTimeout' message. + */ + def getReceiveTimeout: Option[Long] = receiveTimeout + + /** + * Java API.

+ * Abstraction for unification of sender and senderFuture for later reply + */ + def getChannel: UntypedChannel = channel + + /** + * Akka Scala & Java API + * Use self.reply(..) to reply with a message to the original sender of the message currently + * being processed. This method fails if the original sender of the message could not be determined with an + * IllegalStateException. + * + * If you don't want deal with this IllegalStateException, but just a boolean, just use the tryReply(...) + * version. + * + *

+ * Throws an IllegalStateException if unable to determine what to reply to. + */ + def reply(message: Any) = channel.!(message)(this) + + /** + * Akka Scala & Java API + * Use tryReply(..) to try reply with a message to the original sender of the message currently + * being processed. This method + *

+ * Returns true if reply was sent, and false if unable to determine what to reply to. + * + * If you would rather have an exception, check the reply(..) version. + */ + def tryReply(message: Any): Boolean = channel.tryTell(message)(this) + /** * Returns an unmodifiable Java Map containing the linked actors, * please note that the backing map is thread-safe but not immutable @@ -515,53 +491,19 @@ abstract class ActorRef extends ActorRefShared with ForwardableChannel with Repl def getLinkedActors: JMap[Uuid, ActorRef] = linkedActors /** - * Abstraction for unification of sender and senderFuture for later reply + * Scala API + * Returns the dispatcher (MessageDispatcher) that is used for this Actor */ - def channel: UntypedChannel = { - val msg = currentMessage - if (msg ne null) msg.channel - else NullChannel - } + def dispatcher: MessageDispatcher /** - * Java API.

- * Abstraction for unification of sender and senderFuture for later reply + * Java API + * Returns the dispatcher (MessageDispatcher) that is used for this Actor */ - def getChannel: UntypedChannel = channel - - protected[akka] def invoke(messageHandle: MessageInvocation) - - protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit - - protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( - message: Any, - timeout: Timeout, - channel: UntypedChannel): Future[Any] - - protected[akka] def actorInstance: AtomicReference[Actor] - - protected[akka] def actor: Actor = actorInstance.get - - protected[akka] def supervisor_=(sup: Option[ActorRef]) - - protected[akka] def mailbox: AnyRef - - protected[akka] def mailbox_=(value: AnyRef): AnyRef + final def getDispatcher(): MessageDispatcher = dispatcher + /** INTERNAL API ONLY **/ protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) - - protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) - - protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) - - override def hashCode: Int = HashCode.hash(HashCode.SEED, uuid) - - override def equals(that: Any): Boolean = { - that.isInstanceOf[ActorRef] && - that.asInstanceOf[ActorRef].uuid == uuid - } - - override def toString = "Actor[%s:%s]".format(address, uuid) } /** @@ -569,8 +511,8 @@ abstract class ActorRef extends ActorRefShared with ForwardableChannel with Repl * * @author Jonas Bonér */ -class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, val address: String) - extends ActorRef with ScalaActorRef { +class LocalActorRef private[akka] (private[this] val props: Props, val address: String) + extends SelfActorRef with ScalaActorRef { protected[akka] val guard = new ReentrantGuard @@ -590,14 +532,17 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, private var restartTimeWindowStartNanos: Long = 0L @volatile - private var _mailbox: AnyRef = _ - - @volatile - private[akka] var _dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher + protected[akka] var mailbox: AnyRef = _ protected[akka] val actorInstance = guard.withGuard { - new AtomicReference[Actor](newActor) - } + new AtomicReference[Actor]({ + if (props.supervisor.isDefined) props.supervisor.get.link(this) + newActor + }) + } //TODO Why is the guard needed here? + + protected[akka] override def timeout: Long = props.timeout.duration.toMillis //TODO remove this if possible + protected[akka] override def lifeCycle: LifeCycle = props.lifeCycle //TODO remove this if possible private def serializer: Serializer = //TODO Is this used or needed? try { Serialization.serializerFor(this.getClass) } catch { @@ -633,44 +578,25 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, private[akka] def this( __uuid: Uuid, __address: String, - __timeout: Long, + __props: Props, __receiveTimeout: Option[Long], - __lifeCycle: LifeCycle, - __supervisor: Option[ActorRef], - __hotswap: Stack[PartialFunction[Any, Unit]], - __factory: () ⇒ Actor) = { + __hotswap: Stack[PartialFunction[Any, Unit]]) = { - this(__factory, __address) + this(__props, __address) _uuid = __uuid - timeout = __timeout - receiveTimeout = __receiveTimeout - lifeCycle = __lifeCycle - _supervisor = __supervisor hotswap = __hotswap - setActorSelfFields(actor, this) + receiveTimeout = __receiveTimeout + setActorSelfFields(actorInstance.get(), this) //TODO Why is this needed? start() } // ========= PUBLIC FUNCTIONS ========= /** - * Sets the dispatcher for this actor. Needs to be invoked before the actor is started. + * Returns the dispatcher (MessageDispatcher) that is used for this Actor */ - def dispatcher_=(md: MessageDispatcher) { - guard.withGuard { - if (!isBeingRestarted) { - if (!isRunning) _dispatcher = md - else throw new ActorInitializationException( - "Can not swap dispatcher for " + toString + " after it has been started") - } - } - } - - /** - * Get the dispatcher for this actor. - */ - def dispatcher: MessageDispatcher = _dispatcher + def dispatcher: MessageDispatcher = props.dispatcher /** * Starts up the actor and its message queue. @@ -711,7 +637,7 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, _status = ActorRefInternals.SHUTDOWN dispatcher.detach(this) try { - val a = actor + val a = actorInstance.get() if (Actor.debugLifecycle) EventHandler.debug(a, "stopping") a.postStop() } finally { @@ -719,7 +645,7 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, setActorSelfFields(actorInstance.get, null) } - } //else if (isBeingRestarted) throw new ActorKilledException("Actor [" + toString + "] is being restarted.") + } if (hasReplicationStorage) replicationStorage.get.delete() //TODO shouldn't this be inside the if (isRunning?) } @@ -734,12 +660,13 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, * defined by the 'faultHandler'. *

* To be invoked from within the actor itself. + * Returns the ref that was passed into it */ - def link(actorRef: ActorRef) { + def link(actorRef: ActorRef): ActorRef = { guard.withGuard { val actorRefSupervisor = actorRef.supervisor val hasSupervisorAlready = actorRefSupervisor.isDefined - if (hasSupervisorAlready && actorRefSupervisor.get.uuid == uuid) return // we already supervise this guy + if (hasSupervisorAlready && actorRefSupervisor.get.uuid == uuid) return actorRef // we already supervise this guy else if (hasSupervisorAlready) throw new IllegalActorStateException( "Actor can only have one supervisor [" + actorRef + "], e.g. link(actor) fails") else { @@ -747,42 +674,52 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, actorRef.supervisor = Some(this) } } - if (Actor.debugLifecycle) EventHandler.debug(actor, "now supervising " + actorRef) + if (Actor.debugLifecycle) EventHandler.debug(actorInstance.get(), "now supervising " + actorRef) + actorRef } /** * Unlink the actor. *

* To be invoked from within the actor itself. + * Returns the ref that was passed into it */ - def unlink(actorRef: ActorRef) { + def unlink(actorRef: ActorRef): ActorRef = { guard.withGuard { if (_linkedActors.remove(actorRef.uuid) eq null) throw new IllegalActorStateException("Actor [" + actorRef + "] is not a linked actor, can't unlink") actorRef.supervisor = None - if (Actor.debugLifecycle) EventHandler.debug(actor, "stopped supervising " + actorRef) + if (Actor.debugLifecycle) EventHandler.debug(actorInstance.get(), "stopped supervising " + actorRef) } - } - - /** - * Atomically start and link an actor. - *

- * To be invoked from within the actor itself. - */ - def startLink(actorRef: ActorRef): ActorRef = guard.withGuard { - link(actorRef) - actorRef.start() actorRef } /** - * Returns the mailbox. + * The reference sender Actor of the last received message. + * Is defined if the message was sent from another Actor, else None. */ - def mailbox: AnyRef = _mailbox + @deprecated("will be removed in 2.0, use channel instead", "1.2") + def sender: Option[ActorRef] = { + val msg = currentMessage + if (msg eq null) None + else msg.channel match { + case ref: ActorRef ⇒ Some(ref) + case _ ⇒ None + } + } - protected[akka] def mailbox_=(value: AnyRef): AnyRef = { - _mailbox = value; - value + /** + * The reference sender future of the last received message. + * Is defined if the message was sent with sent with '?'/'ask', else None. + */ + @deprecated("will be removed in 2.0, use channel instead", "1.2") + def senderFuture(): Option[Promise[Any]] = { + val msg = currentMessage + if (msg eq null) None + else msg.channel match { + case f: ActorPromise ⇒ Some(f) + case _ ⇒ None + } } /** @@ -829,7 +766,7 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, try { try { cancelReceiveTimeout() // FIXME: leave this here? - actor(messageHandle.message) + actorInstance.get().apply(messageHandle.message) currentMessage = null // reset current message after successful invocation } catch { case e: InterruptedException ⇒ @@ -842,7 +779,7 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, } } catch { case e ⇒ - EventHandler.error(e, actor, messageHandle.message.toString) + EventHandler.error(e, actorInstance.get(), messageHandle.message.toString) throw e } } else { @@ -855,7 +792,7 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, } protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) { - faultHandler match { + props.faultHandler match { case AllForOneStrategy(trapExit, maxRetries, within) if trapExit.exists(_.isAssignableFrom(reason.getClass)) ⇒ restartLinkedActors(reason, maxRetries, within) @@ -988,7 +925,7 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, refStack.set(stackBefore.push(this)) try { if (_status == ActorRefInternals.BEING_RESTARTED) { - val a = actor + val a = actorInstance.get() val fresh = try a.freshInstance catch { case e ⇒ EventHandler.error(e, a, "freshInstance() failed, falling back to initial actor factory") @@ -996,10 +933,10 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, } fresh match { case Some(ref) ⇒ ref - case None ⇒ actorFactory() + case None ⇒ props.creator() } } else { - actorFactory() + props.creator() } } finally { val stackAfter = refStack.get @@ -1082,11 +1019,11 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, } } - lookupAndSetSelfFields(actor.getClass, actor, value) + lookupAndSetSelfFields(actorInstance.get().getClass, actorInstance.get(), value) } private def initializeActorInstance() { - val a = actor + val a = actorInstance.get() if (Actor.debugLifecycle) EventHandler.debug(a, "started") a.preStart() // run actor preStart Actor.registry.register(this) @@ -1094,9 +1031,10 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, protected[akka] def checkReceiveTimeout() { cancelReceiveTimeout() - if (receiveTimeout.isDefined && dispatcher.mailboxIsEmpty(this)) { + val recvtimeout = receiveTimeout + if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) { //Only reschedule if desired and there are currently no more messages to be processed - _futureTimeout = Some(Scheduler.scheduleOnce(this, ReceiveTimeout, receiveTimeout.get, TimeUnit.MILLISECONDS)) + _futureTimeout = Some(Scheduler.scheduleOnce(this, ReceiveTimeout, recvtimeout.get, TimeUnit.MILLISECONDS)) } } @@ -1132,15 +1070,12 @@ private[akka] case class RemoteActorRef private[akka] ( ClusterModule.ensureEnabled() - timeout = _timeout - start() + protected[akka] override def timeout: Long = _timeout + def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = { - val chSender = channel match { - case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None - } + val chSender = if (channel.isInstanceOf[ActorRef]) Some(channel.asInstanceOf[ActorRef]) else None Actor.remote.send[Any](message, chSender, None, remoteAddress, timeout, true, this, loader) } @@ -1148,14 +1083,8 @@ private[akka] case class RemoteActorRef private[akka] ( message: Any, timeout: Timeout, channel: UntypedChannel): Future[Any] = { - val chSender = channel match { - case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None - } - val chFuture = channel match { - case f: Promise[_] ⇒ Some(f.asInstanceOf[Promise[Any]]) - case _ ⇒ None - } + val chSender = if (channel.isInstanceOf[ActorRef]) Some(channel.asInstanceOf[ActorRef]) else None + val chFuture = if (channel.isInstanceOf[Promise[_]]) Some(channel.asInstanceOf[Promise[Any]]) else None val future = Actor.remote.send[Any](message, chSender, chFuture, remoteAddress, timeout.duration.toMillis, false, this, loader) if (future.isDefined) ActorPromise(future.get) else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString) @@ -1181,53 +1110,20 @@ private[akka] case class RemoteActorRef private[akka] ( SerializedActorRef(uuid, address, remoteAddress.getAddress.getHostAddress, remoteAddress.getPort, timeout) } - // ==== NOT SUPPORTED ==== - def dispatcher_=(md: MessageDispatcher) { - unsupported - } + def link(actorRef: ActorRef): ActorRef = unsupported - def dispatcher: MessageDispatcher = unsupported - - def link(actorRef: ActorRef) { - unsupported - } - - def unlink(actorRef: ActorRef) { - unsupported - } - - def startLink(actorRef: ActorRef): ActorRef = unsupported + def unlink(actorRef: ActorRef): ActorRef = unsupported def supervisor: Option[ActorRef] = unsupported - def linkedActors: JMap[Uuid, ActorRef] = unsupported - - protected[akka] def mailbox: AnyRef = unsupported - - protected[akka] def mailbox_=(value: AnyRef): AnyRef = unsupported - - protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) { - unsupported - } - protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) { unsupported } - protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) { - unsupported - } - - protected[akka] def invoke(messageHandle: MessageInvocation) { - unsupported - } - protected[akka] def supervisor_=(sup: Option[ActorRef]) { unsupported } - protected[akka] def actorInstance: AtomicReference[Actor] = unsupported - private def unsupported = throw new UnsupportedOperationException("Not supported for RemoteActorRef") } @@ -1259,69 +1155,9 @@ trait ActorRefShared { * There are implicit conversions in ../actor/Implicits.scala * from ActorRef -> ScalaActorRef and back */ -trait ScalaActorRef extends ActorRefShared with ForwardableChannel with ReplyChannel[Any] { +trait ScalaActorRef extends ActorRefShared with ReplyChannel[Any] { ref: ActorRef ⇒ - /** - * Address for actor, must be a unique one. - */ - def address: String - - /** - * User overridable callback/setting. - *

- * Defines the life-cycle for a supervised actor. - */ - @volatile - @BeanProperty - var lifeCycle: LifeCycle = UndefinedLifeCycle - - /** - * User overridable callback/setting. - *

- * Don't forget to supply a List of exception types to intercept (trapExit) - *

- * Can be one of: - *

-   *  faultHandler = AllForOneStrategy(trapExit = List(classOf[Exception]), maxNrOfRetries, withinTimeRange)
-   * 
- * Or: - *
-   *  faultHandler = OneForOneStrategy(trapExit = List(classOf[Exception]), maxNrOfRetries, withinTimeRange)
-   * 
- */ - @volatile - @BeanProperty - var faultHandler: FaultHandlingStrategy = NoFaultHandlingStrategy - - /** - * The reference sender Actor of the last received message. - * Is defined if the message was sent from another Actor, else None. - */ - @deprecated("will be removed in 2.0, use channel instead", "1.2") - def sender: Option[ActorRef] = { - val msg = currentMessage - if (msg eq null) None - else msg.channel match { - case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None - } - } - - /** - * The reference sender future of the last received message. - * Is defined if the message was sent with sent with '?'/'ask', else None. - */ - @deprecated("will be removed in 2.0, use channel instead", "1.2") - def senderFuture(): Option[Promise[Any]] = { - val msg = currentMessage - if (msg eq null) None - else msg.channel match { - case f: ActorPromise ⇒ Some(f) - case _ ⇒ None - } - } - /** * Sends a one-way asynchronous message. E.g. fire-and-forget semantics. *

@@ -1373,6 +1209,9 @@ trait ScalaActorRef extends ActorRefShared with ForwardableChannel with ReplyCha } } +/** + * Memento pattern for serializing ActorRefs transparently + */ case class SerializedActorRef(uuid: Uuid, address: String, hostname: String, @@ -1382,6 +1221,7 @@ case class SerializedActorRef(uuid: Uuid, def readResolve(): AnyRef = Actor.registry.local.actorFor(uuid) match { case Some(actor) ⇒ actor case None ⇒ + //TODO FIXME Add case for when hostname+port == remote.address.hostname+port, should return a DeadActorRef or something if (ReflectiveAccess.RemoteModule.isEnabled) RemoteActorRef(new InetSocketAddress(hostname, port), address, timeout, None) else @@ -1396,31 +1236,13 @@ case class SerializedActorRef(uuid: Uuid, */ trait UnsupportedActorRef extends ActorRef with ScalaActorRef { - def dispatcher_=(md: MessageDispatcher) { - unsupported - } + def link(actorRef: ActorRef): ActorRef = unsupported - def dispatcher: MessageDispatcher = unsupported - - def link(actorRef: ActorRef) { - unsupported - } - - def unlink(actorRef: ActorRef) { - unsupported - } - - def startLink(actorRef: ActorRef): ActorRef = unsupported + def unlink(actorRef: ActorRef): ActorRef = unsupported def supervisor: Option[ActorRef] = unsupported - def linkedActors: JMap[Uuid, ActorRef] = unsupported - - protected[akka] def mailbox: AnyRef = unsupported - - protected[akka] def mailbox_=(value: AnyRef): AnyRef = unsupported - - protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) { + protected[akka] def supervisor_=(sup: Option[ActorRef]) { unsupported } @@ -1428,19 +1250,5 @@ trait UnsupportedActorRef extends ActorRef with ScalaActorRef { unsupported } - protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) { - unsupported - } - - protected[akka] def invoke(messageHandle: MessageInvocation) { - unsupported - } - - protected[akka] def supervisor_=(sup: Option[ActorRef]) { - unsupported - } - - protected[akka] def actorInstance: AtomicReference[Actor] = unsupported - private def unsupported = throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName)) } diff --git a/akka-actor/src/main/scala/akka/actor/Channel.scala b/akka-actor/src/main/scala/akka/actor/Channel.scala index d860eafdff..888699cada 100644 --- a/akka-actor/src/main/scala/akka/actor/Channel.scala +++ b/akka-actor/src/main/scala/akka/actor/Channel.scala @@ -155,7 +155,7 @@ trait ForwardableChannel extends UntypedChannel with AvailableChannel[Any] { } object ForwardableChannel { - implicit def someS2FC(sender: Some[ActorRef]): ForwardableChannel = sender.get - implicit def someIS2FC(implicit sender: Some[ActorRef]): ForwardableChannel = sender.get + implicit def someS2FC(sender: Some[SelfActorRef]): ForwardableChannel = sender.get + implicit def someIS2FC(implicit sender: Some[SelfActorRef]): ForwardableChannel = sender.get } diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index dcee047659..6ab2afd972 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -193,16 +193,16 @@ object Deployer extends ActorDeployer { // akka.actor.deployment.

.clustered.replicas // -------------------------------- val replicationFactor = { - if (router == Direct) ReplicationFactor(1) + if (router == Direct) new ReplicationFactor(1) else { clusteredConfig.getAny("replication-factor", "0") match { case "auto" ⇒ AutoReplicationFactor case "0" ⇒ ZeroReplicationFactor case nrOfReplicas: String ⇒ try { - ReplicationFactor(nrOfReplicas.toInt) + new ReplicationFactor(nrOfReplicas.toInt) } catch { - case e: NumberFormatException ⇒ + case e: Exception ⇒ throw new ConfigurationException( "Config option [" + addressPath + ".clustered.replicas] needs to be either [\"auto\"] or [0-N] - was [" + diff --git a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala index ff539f57b8..79cf40f50a 100644 --- a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala +++ b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala @@ -80,17 +80,18 @@ object DeploymentConfig { // -------------------------------- // --- Replicas // -------------------------------- - sealed case class ReplicationFactor(val factor: Int) { + + case class ReplicationFactor(val factor: Int) { if (factor < 0) throw new IllegalArgumentException("replication-factor can not be negative") } // For Java API - case class AutoReplicationFactor() extends ReplicationFactor(-1) - case class ZeroReplicationFactor() extends ReplicationFactor(0) + class AutoReplicationFactor extends ReplicationFactor(-1) + class ZeroReplicationFactor extends ReplicationFactor(0) // For Scala API - case object AutoReplicationFactor extends ReplicationFactor(-1) - case object ZeroReplicationFactor extends ReplicationFactor(0) + case object AutoReplicationFactor extends AutoReplicationFactor + case object ZeroReplicationFactor extends ZeroReplicationFactor // -------------------------------- // --- Replication @@ -125,12 +126,12 @@ object DeploymentConfig { sealed trait ReplicationStrategy // For Java API - case class WriteBehind() extends ReplicationStrategy - case class WriteThrough() extends ReplicationStrategy + sealed class WriteBehind extends ReplicationStrategy + sealed class WriteThrough extends ReplicationStrategy // For Scala API - case object WriteBehind extends ReplicationStrategy - case object WriteThrough extends ReplicationStrategy + case object WriteBehind extends WriteBehind + case object WriteThrough extends WriteThrough // -------------------------------- // --- Helper methods for parsing diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 8e3a79404f..60f8e71a99 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -200,7 +200,7 @@ trait IO { if (!reinvoked && (_next eq Idle) && _messages.nonEmpty) { try { reinvoked = true - while ((_next eq Idle) && _messages.nonEmpty) self invoke _messages.dequeue + while ((_next eq Idle) && _messages.nonEmpty) self.asInstanceOf[LocalActorRef] invoke _messages.dequeue } finally { reinvoked = false } diff --git a/akka-actor/src/main/scala/akka/actor/Supervisor.scala b/akka-actor/src/main/scala/akka/actor/Supervisor.scala index 1463c1b6cb..3a80f66e37 100644 --- a/akka-actor/src/main/scala/akka/actor/Supervisor.scala +++ b/akka-actor/src/main/scala/akka/actor/Supervisor.scala @@ -8,9 +8,9 @@ import akka.AkkaException import akka.util._ import ReflectiveAccess._ import Actor._ - -import java.util.concurrent.{ CopyOnWriteArrayList, ConcurrentHashMap } +import java.util.concurrent.{ CopyOnWriteArrayList } import akka.config.Supervision._ +import collection.mutable.ListBuffer class SupervisorException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) { def this(msg: String) = this(msg, null); @@ -110,10 +110,10 @@ case class SupervisorFactory(val config: SupervisorConfig) { sealed class Supervisor(handler: FaultHandlingStrategy, maxRestartsHandler: (ActorRef, MaximumNumberOfRestartsWithinTimeRangeReached) ⇒ Unit) { import Supervisor._ - private val _childActors = new ConcurrentHashMap[String, List[ActorRef]] + private val _childActors = new CopyOnWriteArrayList[ActorRef] private val _childSupervisors = new CopyOnWriteArrayList[Supervisor] - private[akka] val supervisor = localActorOf(new SupervisorActor(handler, maxRestartsHandler)).start() + private[akka] val supervisor = actorOf(Props(new SupervisorActor(maxRestartsHandler)).withFaultHandler(handler)) def uuid = supervisor.uuid @@ -125,39 +125,39 @@ sealed class Supervisor(handler: FaultHandlingStrategy, maxRestartsHandler: (Act def link(child: ActorRef) = supervisor.link(child) - def startLink(child: ActorRef) = supervisor.startLink(child) - def unlink(child: ActorRef) = supervisor.unlink(child) - def children: List[ActorRef] = - _childActors.values.toArray.toList.asInstanceOf[List[List[ActorRef]]].flatten + def children: List[ActorRef] = { + val buf = new ListBuffer[ActorRef] + val i = _childActors.iterator() + while (i.hasNext) buf += i.next() + buf.toList + } - def childSupervisors: List[Supervisor] = - _childActors.values.toArray.toList.asInstanceOf[List[Supervisor]] + def childSupervisors: List[Supervisor] = { + val buf = new ListBuffer[Supervisor] + val i = _childSupervisors.iterator() + while (i.hasNext) buf += i.next() + buf.toList + } def configure(config: SupervisorConfig): Unit = config match { case SupervisorConfig(_, servers, _) ⇒ + servers foreach { + case Supervise(actorRef, lifeCycle, registerAsRemoteService) ⇒ + // actorRef.lifeCycle = lifeCycle THIS IS NOT COOL, BUT WAITING FOR https://www.assembla.com/spaces/akka/tickets/1124-supervisor-dsl-doesn-t-make-much-sense-after-the-introduction-of-props + supervisor.link(actorRef) + actorRef.start() - servers.map(server ⇒ - server match { - case Supervise(actorRef, lifeCycle, registerAsRemoteService) ⇒ - actorRef.start() - val className = actorRef.actor.getClass.getName - val currentActors = { - val list = _childActors.get(className) - if (list eq null) List[ActorRef]() - else list - } - _childActors.put(className, actorRef :: currentActors) - actorRef.lifeCycle = lifeCycle - supervisor.link(actorRef) - if (ClusterModule.isEnabled && registerAsRemoteService) - Actor.remote.register(actorRef) - case supervisorConfig @ SupervisorConfig(_, _, _) ⇒ // recursive supervisor configuration - val childSupervisor = Supervisor(supervisorConfig) - supervisor.link(childSupervisor.supervisor) - _childSupervisors.add(childSupervisor) - }) + _childActors.add(actorRef) //TODO Why do we keep this here, mem leak? + + if (ClusterModule.isEnabled && registerAsRemoteService) + Actor.remote.register(actorRef) + case supervisorConfig @ SupervisorConfig(_, _, _) ⇒ // recursive supervisor configuration + val childSupervisor = Supervisor(supervisorConfig) + supervisor.link(childSupervisor.supervisor) + _childSupervisors.add(childSupervisor) + } } } @@ -166,8 +166,7 @@ sealed class Supervisor(handler: FaultHandlingStrategy, maxRestartsHandler: (Act * * @author Jonas Bonér */ -final class SupervisorActor private[akka] (handler: FaultHandlingStrategy, maxRestartsHandler: (ActorRef, MaximumNumberOfRestartsWithinTimeRangeReached) ⇒ Unit) extends Actor { - self.faultHandler = handler +final class SupervisorActor private[akka] (maxRestartsHandler: (ActorRef, MaximumNumberOfRestartsWithinTimeRangeReached) ⇒ Unit) extends Actor { override def postStop(): Unit = { val i = self.linkedActors.values.iterator diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 88c377b7a2..1a256291bb 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -212,7 +212,7 @@ object TypedActor { private[akka] def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: (AtomVar[R]) ⇒ Actor, props: Props, loader: ClassLoader): R = { val proxyVar = new AtomVar[R] - configureAndProxyLocalActorRef[R](interfaces, proxyVar, props.withCreator(() ⇒ constructor(proxyVar)), loader) + configureAndProxyLocalActorRef[R](interfaces, proxyVar, props.withCreator(constructor(proxyVar)), loader) } private[akka] def createProxyAndTypedActor[R <: AnyRef, T <: R](interface: Class[_], constructor: ⇒ T, props: Props, loader: ClassLoader): R = @@ -221,7 +221,7 @@ object TypedActor { private[akka] def configureAndProxyLocalActorRef[T <: AnyRef](interfaces: Array[Class[_]], proxyVar: AtomVar[T], props: Props, loader: ClassLoader): T = { //Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling val actorVar = new AtomVar[ActorRef](null) - val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)).asInstanceOf[T] + val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)(props.timeout)).asInstanceOf[T] proxyVar.set(proxy) // Chicken and egg situation we needed to solve, set the proxy so that we can set the self-reference inside each receive val ref = actorOf(props) actorVar.set(ref) //Make sure the InvocationHandler gets ahold of the actor reference, this is not a problem since the proxy hasn't escaped this method yet @@ -252,7 +252,7 @@ object TypedActor { } } - private[akka] class TypedActorInvocationHandler(actorVar: AtomVar[ActorRef]) extends InvocationHandler { + private[akka] class TypedActorInvocationHandler(actorVar: AtomVar[ActorRef])(implicit timeout: Timeout) extends InvocationHandler { def actor = actorVar.get def invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = method.getName match { @@ -260,7 +260,6 @@ object TypedActor { case "equals" ⇒ (args.length == 1 && (proxy eq args(0)) || actor == getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean case "hashCode" ⇒ actor.hashCode.asInstanceOf[AnyRef] case _ ⇒ - implicit val timeout = Timeout(actor.timeout) MethodCall(method, args) match { case m if m.isOneWay ⇒ actor ! m; null //Null return value case m if m.returnsFuture_? ⇒ actor ? m diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index fa93cc0e57..8442df5270 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -65,12 +65,12 @@ abstract class UntypedActor extends Actor { /** * Returns the 'self' reference with the API. */ - def getContext(): ActorRef = self + def getContext(): SelfActorRef = self /** * Returns the 'self' reference with the API. */ - def context(): ActorRef = self + def context(): SelfActorRef = self /** * Java API for become diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala index 98d87106cc..9893c3a431 100644 --- a/akka-actor/src/main/scala/akka/actor/package.scala +++ b/akka-actor/src/main/scala/akka/actor/package.scala @@ -5,11 +5,8 @@ package akka package object actor { - implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = - ref.asInstanceOf[ScalaActorRef] - - implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = - ref.asInstanceOf[ActorRef] + implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef] + implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef] type Uuid = com.eaio.uuid.UUID diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index a5787e6c0d..ffff55422d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -4,9 +4,8 @@ package akka.dispatch -import akka.actor.{ ActorRef, Actor, IllegalActorStateException } - import util.DynamicVariable +import akka.actor.{ LocalActorRef, ActorRef, Actor, IllegalActorStateException } /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -53,15 +52,15 @@ class BalancingDispatcher( @volatile private var actorType: Option[Class[_]] = None @volatile - private var members = Vector[ActorRef]() + private var members = Vector[LocalActorRef]() private val donationInProgress = new DynamicVariable(false) - private[akka] override def register(actorRef: ActorRef) = { + private[akka] override def register(actorRef: LocalActorRef) = { //Verify actor type conformity actorType match { - case None ⇒ actorType = Some(actorRef.actor.getClass) + case None ⇒ actorType = Some(actorRef.actorInstance.get().getClass) case Some(aType) ⇒ - if (aType != actorRef.actor.getClass) + if (aType != actorRef.actorInstance.get().getClass) throw new IllegalActorStateException(String.format( "Can't register actor %s in a work stealing dispatcher which already knows actors of type %s", actorRef, aType)) @@ -71,7 +70,7 @@ class BalancingDispatcher( super.register(actorRef) } - private[akka] override def unregister(actorRef: ActorRef) = { + private[akka] override def unregister(actorRef: LocalActorRef) = { synchronized { members = members.filterNot(actorRef eq) } //Update members super.unregister(actorRef) } @@ -137,7 +136,7 @@ class BalancingDispatcher( /** * Returns an available recipient for the message, if any */ - protected def doFindDonorRecipient(donorMbox: MessageQueue with ExecutableMailbox, potentialRecipients: Vector[ActorRef], startIndex: Int): ActorRef = { + protected def doFindDonorRecipient(donorMbox: MessageQueue with ExecutableMailbox, potentialRecipients: Vector[LocalActorRef], startIndex: Int): ActorRef = { val prSz = potentialRecipients.size var i = 0 var recipient: ActorRef = null diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index d2b880041c..410e3120cc 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -7,7 +7,7 @@ package akka.dispatch import akka.event.EventHandler import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue } -import akka.actor.{ ActorKilledException, ActorRef } +import akka.actor.{ LocalActorRef, ActorKilledException, ActorRef } /** * Default settings are: @@ -108,13 +108,13 @@ class Dispatcher( /** * @return the mailbox associated with the actor */ - protected def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox] + protected def getMailbox(receiver: LocalActorRef) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox] - def mailboxIsEmpty(actorRef: ActorRef): Boolean = getMailbox(actorRef).isEmpty + override def mailboxIsEmpty(actorRef: LocalActorRef): Boolean = getMailbox(actorRef).isEmpty - override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size + override def mailboxSize(actorRef: LocalActorRef): Int = getMailbox(actorRef).size - def createMailbox(actorRef: ActorRef): AnyRef = mailboxType match { + def createMailbox(actorRef: LocalActorRef): AnyRef = mailboxType match { case b: UnboundedMailbox ⇒ new ConcurrentLinkedQueue[MessageInvocation] with MessageQueue with ExecutableMailbox { @inline @@ -160,7 +160,7 @@ class Dispatcher( private[akka] def reRegisterForExecution(mbox: MessageQueue with ExecutableMailbox): Unit = registerForExecution(mbox) - protected override def cleanUpMailboxFor(actorRef: ActorRef) { + protected override def cleanUpMailboxFor(actorRef: LocalActorRef) { val m = getMailbox(actorRef) if (!m.isEmpty) { var invocation = m.dequeue @@ -174,11 +174,10 @@ class Dispatcher( override val toString = getClass.getSimpleName + "[" + name + "]" - def suspend(actorRef: ActorRef) { + def suspend(actorRef: LocalActorRef): Unit = getMailbox(actorRef).suspended.tryLock - } - def resume(actorRef: ActorRef) { + def resume(actorRef: LocalActorRef): Unit = { val mbox = getMailbox(actorRef) mbox.suspended.tryUnlock reRegisterForExecution(mbox) @@ -297,7 +296,7 @@ class PriorityDispatcher( trait PriorityMailbox { self: Dispatcher ⇒ def comparator: java.util.Comparator[MessageInvocation] - override def createMailbox(actorRef: ActorRef): AnyRef = self.mailboxType match { + override def createMailbox(actorRef: LocalActorRef): AnyRef = self.mailboxType match { case b: UnboundedMailbox ⇒ new UnboundedPriorityMessageQueue(comparator) with ExecutableMailbox { @inline diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index da0aeb21de..bd1d9c4fb1 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -7,7 +7,8 @@ package akka.dispatch import akka.AkkaException import akka.event.EventHandler import akka.actor.{ Actor, ForwardableChannel, UntypedChannel, Scheduler, Timeout, ExceptionChannel } -import akka.japi.{ Procedure, Function ⇒ JFunc } +import scala.Option +import akka.japi.{ Procedure, Function ⇒ JFunc, Option ⇒ JOption } import scala.util.continuations._ @@ -19,8 +20,7 @@ import java.util.{ LinkedList ⇒ JLinkedList } import scala.annotation.tailrec import scala.collection.mutable.Stack import akka.util.{ Switch, Duration, BoxedType } - -import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean } +import java.util.concurrent.atomic.{ AtomicInteger, AtomicReference, AtomicBoolean } class FutureTimeoutException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { def this(message: String) = this(message, null) @@ -71,16 +71,45 @@ object Futures { val futureResult = new DefaultPromise[T](timeout) val completeFirst: Future[T] ⇒ Unit = _.value.foreach(futureResult complete _) - for (f ← futures) f onComplete completeFirst + futures.foreach(_ onComplete completeFirst) futureResult } + /** + * Returns a Future that will hold the optional result of the first Future with a result that matches the predicate + */ + def find[T](predicate: T ⇒ Boolean, timeout: Timeout = Timeout.default)(futures: Iterable[Future[T]]): Future[Option[T]] = { + if (futures.isEmpty) new KeptPromise[Option[T]](Right(None)) + else { + val result = new DefaultPromise[Option[T]](timeout) + val ref = new AtomicInteger(futures.size) + val search: Future[T] ⇒ Unit = f ⇒ try { + f.result.filter(predicate).foreach(r ⇒ result completeWithResult Some(r)) + } finally { + if (ref.decrementAndGet == 0) + result completeWithResult None + } + futures.foreach(_ onComplete search) + + result + } + } + + /** + * Java API. + * Returns a Future that will hold the optional result of the first Future with a result that matches the predicate + */ + def find[T <: AnyRef](futures: JIterable[Future[T]], predicate: JFunc[T, java.lang.Boolean], timeout: Timeout): Future[JOption[T]] = { + val pred: T ⇒ Boolean = predicate.apply(_) + find[T](pred, timeout)(scala.collection.JavaConversions.iterableAsScalaIterable(futures)).map(JOption.fromScalaOption(_)) + } + /** * Java API. * Returns a Future to the result of the first future in the list that is completed */ - def firstCompletedOf[T <: AnyRef](futures: java.lang.Iterable[Future[T]], timeout: Timeout): Future[T] = + def firstCompletedOf[T <: AnyRef](futures: JIterable[Future[T]], timeout: Timeout): Future[T] = firstCompletedOf(scala.collection.JavaConversions.iterableAsScalaIterable(futures), timeout) /** @@ -329,7 +358,7 @@ object Future { try { next.apply() } catch { - case e ⇒ // FIXME + case e ⇒ // TODO FIXME: Throwable or Exception, log or do what? } } } finally { _taskStack set None } @@ -645,7 +674,7 @@ package japi { /* Java API */ trait Future[+T] { self: akka.dispatch.Future[T] ⇒ private[japi] final def onTimeout[A >: T](proc: Procedure[akka.dispatch.Future[A]]): this.type = self.onTimeout(proc(_)) - private[japi] final def onResult[A >: T](proc: Procedure[A]): this.type = self.onResult({ case r: A ⇒ proc(r) }: PartialFunction[T, Unit]) + private[japi] final def onResult[A >: T](proc: Procedure[A]): this.type = self.onResult({ case r ⇒ proc(r.asInstanceOf[A]) }: PartialFunction[T, Unit]) private[japi] final def onException(proc: Procedure[Throwable]): this.type = self.onException({ case t: Throwable ⇒ proc(t) }: PartialFunction[Throwable, Unit]) private[japi] final def onComplete[A >: T](proc: Procedure[akka.dispatch.Future[A]]): this.type = self.onComplete(proc(_)) private[japi] final def map[A >: T, B](f: JFunc[A, B]): akka.dispatch.Future[B] = self.map(f(_)) diff --git a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala index af924595b4..9eefc29e53 100644 --- a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala @@ -16,7 +16,7 @@ import akka.actor._ /** * @author Jonas Bonér */ -final case class MessageInvocation(val receiver: ActorRef, +final case class MessageInvocation(val receiver: LocalActorRef, val message: Any, val channel: UntypedChannel) { if (receiver eq null) throw new IllegalArgumentException("Receiver can't be null") @@ -63,7 +63,7 @@ abstract class MessageDispatcher { /** * Creates and returns a mailbox for the given actor. */ - private[akka] def createMailbox(actorRef: ActorRef): AnyRef + private[akka] def createMailbox(actorRef: LocalActorRef): AnyRef /** * Name of this dispatcher. @@ -73,7 +73,7 @@ abstract class MessageDispatcher { /** * Attaches the specified actorRef to this dispatcher */ - final def attach(actorRef: ActorRef) { + final def attach(actorRef: LocalActorRef) { guard withGuard { register(actorRef) } @@ -82,7 +82,7 @@ abstract class MessageDispatcher { /** * Detaches the specified actorRef from this dispatcher */ - final def detach(actorRef: ActorRef) { + final def detach(actorRef: LocalActorRef) { guard withGuard { unregister(actorRef) } @@ -129,7 +129,7 @@ abstract class MessageDispatcher { * Only "private[akka] for the sake of intercepting calls, DO NOT CALL THIS OUTSIDE OF THE DISPATCHER, * and only call it under the dispatcher-guard, see "attach" for the only invocation */ - private[akka] def register(actorRef: ActorRef) { + private[akka] def register(actorRef: LocalActorRef) { if (actorRef.mailbox eq null) actorRef.mailbox = createMailbox(actorRef) @@ -145,7 +145,7 @@ abstract class MessageDispatcher { * Only "private[akka] for the sake of intercepting calls, DO NOT CALL THIS OUTSIDE OF THE DISPATCHER, * and only call it under the dispatcher-guard, see "detach" for the only invocation */ - private[akka] def unregister(actorRef: ActorRef) = { + private[akka] def unregister(actorRef: LocalActorRef) = { if (uuids remove actorRef.uuid) { cleanUpMailboxFor(actorRef) actorRef.mailbox = null @@ -166,7 +166,7 @@ abstract class MessageDispatcher { * Overridable callback to clean up the mailbox for a given actor, * called when an actor is unregistered. */ - protected def cleanUpMailboxFor(actorRef: ActorRef) {} + protected def cleanUpMailboxFor(actorRef: LocalActorRef) {} /** * Traverses the list of actors (uuids) currently being attached to this dispatcher and stops those actors @@ -211,12 +211,12 @@ abstract class MessageDispatcher { /** * After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference */ - def suspend(actorRef: ActorRef) + def suspend(actorRef: LocalActorRef) /* * After the call to this method, the dispatcher must begin any new message processing for the specified reference */ - def resume(actorRef: ActorRef) + def resume(actorRef: LocalActorRef) /** * Will be called when the dispatcher is to queue an invocation for execution @@ -238,12 +238,12 @@ abstract class MessageDispatcher { /** * Returns the size of the mailbox for the specified actor */ - def mailboxSize(actorRef: ActorRef): Int + def mailboxSize(actorRef: LocalActorRef): Int /** * Returns the "current" emptiness status of the mailbox for the specified actor */ - def mailboxIsEmpty(actorRef: ActorRef): Boolean + def mailboxIsEmpty(actorRef: LocalActorRef): Boolean /** * Returns the amount of tasks queued for execution diff --git a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala index fd7c7bb031..7d3e539c1c 100644 --- a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala @@ -5,7 +5,7 @@ package akka.dispatch import java.util.concurrent.atomic.AtomicReference -import akka.actor.ActorRef +import akka.actor.{ LocalActorRef, ActorRef } /** * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. @@ -33,14 +33,14 @@ class PinnedDispatcher(_actor: ActorRef, _name: String, _mailboxType: MailboxTyp private[akka] val owner = new AtomicReference[ActorRef](_actor) //Relies on an external lock provided by MessageDispatcher.attach - private[akka] override def register(actorRef: ActorRef) = { + private[akka] override def register(actorRef: LocalActorRef) = { val actor = owner.get() if ((actor ne null) && actorRef != actor) throw new IllegalArgumentException("Cannot register to anyone but " + actor) owner.compareAndSet(null, actorRef) //Register if unregistered super.register(actorRef) } //Relies on an external lock provided by MessageDispatcher.detach - private[akka] override def unregister(actorRef: ActorRef) = { + private[akka] override def unregister(actorRef: LocalActorRef) = { super.unregister(actorRef) owner.compareAndSet(actorRef, null) //Unregister (prevent memory leak) } diff --git a/akka-actor/src/main/scala/akka/event/EventHandler.scala b/akka-actor/src/main/scala/akka/event/EventHandler.scala index c00da99a77..ec19ea3187 100644 --- a/akka-actor/src/main/scala/akka/event/EventHandler.scala +++ b/akka-actor/src/main/scala/akka/event/EventHandler.scala @@ -119,7 +119,7 @@ object EventHandler extends ListenerManagement { defaultListeners foreach { listenerName ⇒ try { ReflectiveAccess.getClassFor[Actor](listenerName) match { - case Right(actorClass) ⇒ addListener(Actor.localActorOf(actorClass).start()) + case Right(actorClass) ⇒ addListener(Actor.actorOf(Props(actorClass).withDispatcher(EventHandlerDispatcher))) case Left(exception) ⇒ throw exception } } catch { diff --git a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala index f5c4ccdcaa..7a94ef7c50 100644 --- a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala +++ b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala @@ -1,5 +1,7 @@ package akka.japi +import scala.Some + /** * A Function interface. Used to create first-class-functions is Java (sort of). */ @@ -84,6 +86,14 @@ object Option { */ def option[A](v: A): Option[A] = if (v == null) none else some(v) + /** + * Converts a Scala Option to a Java Option + */ + def fromScalaOption[T](scalaOption: scala.Option[T]): Option[T] = scalaOption match { + case scala.Some(r) ⇒ some(r) + case scala.None ⇒ none + } + /** * Class Some[A] represents existing values of type * A. @@ -104,5 +114,5 @@ object Option { } implicit def java2ScalaOption[A](o: Option[A]): scala.Option[A] = o.asScala - implicit def scala2JavaOption[A](o: scala.Option[A]): Option[A] = option(o.get) + implicit def scala2JavaOption[A](o: scala.Option[A]): Option[A] = if (o.isDefined) some(o.get) else none } diff --git a/akka-actor/src/main/scala/akka/remoteinterface/RemoteEventHandler.scala b/akka-actor/src/main/scala/akka/remoteinterface/RemoteEventHandler.scala index ee24efe99a..064463f308 100644 --- a/akka-actor/src/main/scala/akka/remoteinterface/RemoteEventHandler.scala +++ b/akka-actor/src/main/scala/akka/remoteinterface/RemoteEventHandler.scala @@ -13,9 +13,6 @@ import akka.event.EventHandler * @author Jonas Bonér */ class RemoteEventHandler extends Actor { - import EventHandler._ - - self.dispatcher = EventHandlerDispatcher def receive = { diff --git a/akka-actor/src/main/scala/akka/routing/Pool.scala b/akka-actor/src/main/scala/akka/routing/Pool.scala index 2e930e912e..053650005b 100644 --- a/akka-actor/src/main/scala/akka/routing/Pool.scala +++ b/akka-actor/src/main/scala/akka/routing/Pool.scala @@ -4,9 +4,9 @@ package akka.routing -import akka.actor.{ Actor, ActorRef, PoisonPill, Death, MaximumNumberOfRestartsWithinTimeRangeReached } import akka.dispatch.{ Promise } import akka.config.Supervision._ +import akka.actor._ /** * Actor pooling @@ -59,28 +59,6 @@ trait ActorPool { def select(delegates: Seq[ActorRef]): Seq[ActorRef] } -/** - * Defines the configuration options for how the pool supervises the actors. - */ -trait ActorPoolSupervisionConfig { - /** - * Defines the default fault handling strategy to be employed by the pool. - */ - def poolFaultHandler: FaultHandlingStrategy -} - -/** - * Provides a default implementation of the supervision configuration by - * defining a One-for-One fault handling strategy, trapping exceptions, - * limited to 5 retries within 1 second. - * - * This is just a basic strategy and implementors are encouraged to define - * something more appropriate for their needs. - */ -trait DefaultActorPoolSupervisionConfig extends ActorPoolSupervisionConfig { - def poolFaultHandler = OneForOneStrategy(List(classOf[Exception]), 5, 1000) -} - /** * A default implementation of a pool that: * First, invokes the pool's capacitor that tells it, based on the current delegate count @@ -104,14 +82,11 @@ trait DefaultActorPoolSupervisionConfig extends ActorPoolSupervisionConfig { * * Lastly, routes by forwarding, the incoming message to each delegate in the selected set. */ -trait DefaultActorPool extends ActorPool { this: Actor with ActorPoolSupervisionConfig ⇒ +trait DefaultActorPool extends ActorPool { this: Actor ⇒ import ActorPool._ protected[akka] var _delegates = Vector[ActorRef]() - override def preStart() { - self.faultHandler = poolFaultHandler - } override def postStop() { _delegates foreach { delegate ⇒ try { @@ -141,8 +116,8 @@ trait DefaultActorPool extends ActorPool { this: Actor with ActorPoolSupervision _delegates ++ { for (i ← 0 until requestedCapacity) yield { val delegate = instance() - self startLink delegate - delegate + self link delegate + delegate.start() } } case qty if qty < 0 ⇒ @@ -176,8 +151,13 @@ trait SmallestMailboxSelector { var set: Seq[ActorRef] = Nil var take = if (partialFill) math.min(selectionCount, delegates.length) else selectionCount + def mailboxSize(a: ActorRef): Int = a match { + case l: LocalActorRef ⇒ l.dispatcher.mailboxSize(l) + case _ ⇒ Int.MaxValue //Non-local actors mailbox size is unknown, so consider them lowest priority + } + while (take > 0) { - set = delegates.sortWith((a, b) ⇒ a.dispatcher.mailboxSize(a) < b.dispatcher.mailboxSize(b)).take(take) ++ set //Question, doesn't this risk selecting the same actor multiple times? + set = delegates.sortWith((a, b) ⇒ mailboxSize(a) < mailboxSize(b)).take(take) ++ set //Question, doesn't this risk selecting the same actor multiple times? take -= set.size } @@ -257,7 +237,10 @@ trait BoundedCapacitor { trait MailboxPressureCapacitor { def pressureThreshold: Int def pressure(delegates: Seq[ActorRef]): Int = - delegates count { a ⇒ a.dispatcher.mailboxSize(a) > pressureThreshold } + delegates count { + case a: LocalActorRef ⇒ a.dispatcher.mailboxSize(a) > pressureThreshold + case _ ⇒ false + } } /** @@ -265,7 +248,10 @@ trait MailboxPressureCapacitor { */ trait ActiveFuturesPressureCapacitor { def pressure(delegates: Seq[ActorRef]): Int = - delegates count { _.channel.isInstanceOf[Promise[_]] } + delegates count { + case fc: ForwardableChannel ⇒ fc.channel.isInstanceOf[Promise[_]] + case _ ⇒ false + } } /** diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 66febb36b8..7bc99c6d2e 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -158,34 +158,30 @@ object Routing { * how many connections it can handle. */ def actorOf(actorAddress: String, connections: Iterable[ActorRef], routerType: RouterType): ActorRef = { - val ref = routerType match { + val router = routerType match { + case RouterType.Direct if connections.size > 1 ⇒ + throw new IllegalArgumentException("A direct router can't have more than 1 connection") case RouterType.Direct ⇒ - if (connections.size > 1) - throw new IllegalArgumentException("A direct router can't have more than 1 connection") - actorOf(actorAddress, connections, new DirectRouter()) - + new DirectRouter() case RouterType.Random ⇒ - actorOf(actorAddress, connections, new RandomRouter()) - + new RandomRouter() case RouterType.RoundRobin ⇒ - actorOf(actorAddress, connections, new RoundRobinRouter()) - - case _ ⇒ throw new IllegalArgumentException("Unsupported routerType " + routerType) + new RoundRobinRouter() + case r ⇒ + throw new IllegalArgumentException("Unsupported routerType " + r) } - ref.start() + actorOf(actorAddress, connections, router).start() } - def actorOf(actorAddress: String, connections: Iterable[ActorRef], router: Router): ActorRef = { - if (connections.size == 0) + def actorOf(actorAddress: String, connections: Iterable[ActorRef], router: Router): ActorRef = + if (connections.isEmpty) throw new IllegalArgumentException("To create a routed actor ref, at least one connection is required") + else + new RoutedActorRef(actorAddress, router, connections) - new RoutedActorRef(actorAddress, router, connections) - } - - def actorOfWithRoundRobin(actorAddress: String, connections: Iterable[ActorRef]): ActorRef = { + def actorOfWithRoundRobin(actorAddress: String, connections: Iterable[ActorRef]): ActorRef = actorOf(actorAddress, connections, akka.routing.RouterType.RoundRobin) - } } /** @@ -214,7 +210,8 @@ class RoutedActorRef(val address: String, val router: Router, val connectionIter } def start(): this.type = synchronized[this.type] { - _status = ActorRefInternals.RUNNING + if (_status == ActorRefInternals.UNSTARTED) + _status = ActorRefInternals.RUNNING this } @@ -370,13 +367,10 @@ class DirectRouter extends BasicRouter { throw new RoutingException("A DirectRouter can't have more than 1 connected Actor, but found [%s]".format(connectionIterable.size)) val newState = new DirectRouterState(connectionIterable.head, version) - if (state.compareAndSet(currentState, newState)) { - //we are lucky since we just updated the state, so we can send it back as the state to use + if (state.compareAndSet(currentState, newState)) //we are lucky since we just updated the state, so we can send it back as the state to use newState - } else { - //we failed to update the state, lets try again... better luck next time. + else //we failed to update the state, lets try again... better luck next time. getState() - } } } @@ -394,16 +388,11 @@ class RandomRouter extends BasicRouter { private val state = new AtomicReference[RandomRouterState]() //FIXME: threadlocal random? - private val random = new java.util.Random(System.currentTimeMillis) + private val random = new java.util.Random(System.nanoTime()) - def next: Option[ActorRef] = { - val state = getState() - if (state.array.isEmpty) { - None - } else { - val index = random.nextInt(state.array.length) - Some(state.array(index)) - } + def next: Option[ActorRef] = getState().array match { + case a if a.isEmpty ⇒ None + case a ⇒ Some(a(random.nextInt(a.length))) } @tailrec @@ -416,19 +405,16 @@ class RandomRouter extends BasicRouter { } else { //there has been a change in connections, or it was the first try, so we need to update the internal state val (version, connectionIterable) = connections.versionedIterator - val newState = new RandomRouterState(connectionIterable.toArray[ActorRef], version) + val newState = new RandomRouterState(connectionIterable.toIndexedSeq, version) - if (state.compareAndSet(currentState, newState)) { - //we are lucky since we just updated the state, so we can send it back as the state to use + if (state.compareAndSet(currentState, newState)) //we are lucky since we just updated the state, so we can send it back as the state to use newState - } else { - //we failed to update the state, lets try again... better luck next time. + else //we failed to update the state, lets try again... better luck next time. getState() - } } } - private case class RandomRouterState(val array: Array[ActorRef], val version: Long) + private case class RandomRouterState(array: IndexedSeq[ActorRef], version: Long) } /** @@ -452,19 +438,16 @@ class RoundRobinRouter extends BasicRouter { } else { //there has been a change in connections, or it was the first try, so we need to update the internal state val (version, connectionIterable) = connections.versionedIterator - val newState = new RoundRobinState(connectionIterable.toArray[ActorRef], version) + val newState = new RoundRobinState(connectionIterable.toIndexedSeq, version) - if (state.compareAndSet(currentState, newState)) { - //we are lucky since we just updated the state, so we can send it back as the state to use + if (state.compareAndSet(currentState, newState)) //we are lucky since we just updated the state, so we can send it back as the state to use newState - } else { - //we failed to update the state, lets try again... better luck next time. + else //we failed to update the state, lets try again... better luck next time. getState() - } } } - private case class RoundRobinState(val array: Array[ActorRef], val version: Long) { + private case class RoundRobinState(array: IndexedSeq[ActorRef], version: Long) { private val index = new AtomicInteger(0) @@ -510,10 +493,10 @@ trait ScatterGatherRouter extends BasicRouter with Serializable { } } - if (responses.size == 0) + if (responses.isEmpty) throw new RoutingException("No connections can process the message [%s] sent to scatter-gather router" format (message)) - - gather(responses) + else + gather(responses) } override def route[T](message: Any, timeout: Timeout)(implicit sender: Option[ActorRef]): Future[T] = message match { diff --git a/akka-camel-typed/src/main/scala/akka/camel/TypedConsumer.scala b/akka-camel-typed/src/main/scala/akka/camel/TypedConsumer.scala index 3ed0dff0c4..06ba255c4d 100644 --- a/akka-camel-typed/src/main/scala/akka/camel/TypedConsumer.scala +++ b/akka-camel-typed/src/main/scala/akka/camel/TypedConsumer.scala @@ -7,7 +7,7 @@ package akka.camel import java.lang.reflect.Method import java.lang.reflect.Proxy._ -import akka.actor.{ TypedActor, ActorRef } +import akka.actor.{ LocalActorRef, TypedActor, ActorRef } import akka.actor.TypedActor._ /** @@ -42,10 +42,11 @@ private[camel] object TypedConsumer { if (m.isAnnotationPresent(classOf[consume])) } yield f(tc, m) - private def withConsumeAnnotatedMethodsonImplClass[T](tc: AnyRef, actorRef: ActorRef, f: (AnyRef, Method) ⇒ T): List[T] = { - val implClass = actorRef.actor.asInstanceOf[TypedActor.TypedActor[AnyRef, AnyRef]].me.getClass - for (m ← implClass.getDeclaredMethods.toList; if (m.isAnnotationPresent(classOf[consume]))) yield f(tc, m) - + private def withConsumeAnnotatedMethodsonImplClass[T](tc: AnyRef, actorRef: ActorRef, f: (AnyRef, Method) ⇒ T): List[T] = actorRef match { + case l: LocalActorRef ⇒ + val implClass = l.actorInstance.get().asInstanceOf[TypedActor.TypedActor[AnyRef, AnyRef]].me.getClass + for (m ← implClass.getDeclaredMethods.toList; if (m.isAnnotationPresent(classOf[consume]))) yield f(tc, m) + case _ ⇒ Nil } private class ProxyClass(c: Class[_]) { diff --git a/akka-camel/src/main/scala/akka/camel/Consumer.scala b/akka-camel/src/main/scala/akka/camel/Consumer.scala index cfa348e38e..d5843ebe0b 100644 --- a/akka-camel/src/main/scala/akka/camel/Consumer.scala +++ b/akka-camel/src/main/scala/akka/camel/Consumer.scala @@ -139,12 +139,12 @@ private[camel] object Consumer { * to a consumer actor, Some contained the return value of f * otherwise. */ - def withConsumer[T](actorRef: ActorRef)(f: Consumer ⇒ T): Option[T] = { - if (!actorRef.actor.isInstanceOf[Consumer]) None - - // TODO: check if this is needed at all - //else if (actorRef.homeAddress.isDefined) None - - else Some(f(actorRef.actor.asInstanceOf[Consumer])) + def withConsumer[T](actorRef: ActorRef)(f: Consumer ⇒ T): Option[T] = actorRef match { + case l: LocalActorRef ⇒ + l.actorInstance.get() match { + case c: Consumer ⇒ Some(f(c)) + case _ ⇒ None + } + case _ ⇒ None } } diff --git a/akka-camel/src/main/scala/akka/camel/component/ActorComponent.scala b/akka-camel/src/main/scala/akka/camel/component/ActorComponent.scala index c63ca8a289..c6f7d29695 100644 --- a/akka-camel/src/main/scala/akka/camel/component/ActorComponent.scala +++ b/akka-camel/src/main/scala/akka/camel/component/ActorComponent.scala @@ -291,37 +291,16 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall callback.done(false) } - def actorClass: Class[_ <: Actor] = unsupported def dispatcher_=(md: MessageDispatcher): Unit = unsupported def dispatcher: MessageDispatcher = unsupported - def makeRemote(hostname: String, port: Int): Unit = unsupported - def makeRemote(address: InetSocketAddress): Unit = unsupported - def homeAddress_=(address: InetSocketAddress): Unit = unsupported - def remoteAddress: Option[InetSocketAddress] = unsupported - def link(actorRef: ActorRef): Unit = unsupported - def unlink(actorRef: ActorRef): Unit = unsupported - def startLink(actorRef: ActorRef): ActorRef = unsupported - def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = unsupported - def spawn(clazz: Class[_ <: Actor]): ActorRef = unsupported - def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef = unsupported - def spawnLink(clazz: Class[_ <: Actor]): ActorRef = unsupported - def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef = unsupported + def link(actorRef: ActorRef): ActorRef = unsupported + def unlink(actorRef: ActorRef): ActorRef = unsupported def shutdownLinkedActors: Unit = unsupported def supervisor: Option[ActorRef] = unsupported - def homeAddress: Option[InetSocketAddress] = None protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout(message: Any, timeout: Timeout, channel: UntypedChannel) = unsupported - protected[akka] def mailbox: AnyRef = unsupported - protected[akka] def mailbox_=(msg: AnyRef): AnyRef = unsupported protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported - protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported - protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported - def linkedActors: JMap[Uuid, ActorRef] = unsupported - protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported - protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported - protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported protected[akka] def registerSupervisorAsRemoteActor = unsupported protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = unsupported - protected[akka] def actorInstance: AtomicReference[Actor] = unsupported private def unsupported = throw new UnsupportedOperationException("Not supported for %s" format classOf[AsyncCallbackAdapter].getName) } diff --git a/akka-camel/src/test/scala/akka/camel/CamelTestSupport.scala b/akka-camel/src/test/scala/akka/camel/CamelTestSupport.scala index 519fbc07bd..d190347acf 100644 --- a/akka-camel/src/test/scala/akka/camel/CamelTestSupport.scala +++ b/akka-camel/src/test/scala/akka/camel/CamelTestSupport.scala @@ -28,7 +28,6 @@ object CamelTestSupport { } class Tester3 extends TestActor with Noop { - self.timeout = 1 def handler = noop } diff --git a/akka-camel/src/test/scala/akka/camel/ConsumerPublishRequestorTest.scala b/akka-camel/src/test/scala/akka/camel/ConsumerPublishRequestorTest.scala index 655ce0adc2..5e1baaf305 100644 --- a/akka-camel/src/test/scala/akka/camel/ConsumerPublishRequestorTest.scala +++ b/akka-camel/src/test/scala/akka/camel/ConsumerPublishRequestorTest.scala @@ -14,7 +14,7 @@ class ConsumerPublishRequestorTest extends JUnitSuite { var publisher: ActorRef = _ var requestor: ActorRef = _ - var consumer: ActorRef = _ + var consumer: LocalActorRef = _ @Before def setUp: Unit = { @@ -24,7 +24,7 @@ class ConsumerPublishRequestorTest extends JUnitSuite { consumer = actorOf(new Actor with Consumer { def endpointUri = "mock:test" protected def receive = null - }).start + }).start.asInstanceOf[LocalActorRef] } @After @@ -39,7 +39,7 @@ class ConsumerPublishRequestorTest extends JUnitSuite { requestor ! ActorRegistered(consumer.address, consumer, None) assert(latch.await(5000, TimeUnit.MILLISECONDS)) assert((publisher ? GetRetainedMessage).get === - ConsumerActorRegistered(consumer, consumer.actor.asInstanceOf[Consumer])) + ConsumerActorRegistered(consumer, consumer.actorInstance.get.asInstanceOf[Consumer])) } @Test @@ -48,7 +48,7 @@ class ConsumerPublishRequestorTest extends JUnitSuite { requestor ! ActorUnregistered(consumer.address, consumer, None) assert(latch.await(5000, TimeUnit.MILLISECONDS)) assert((publisher ? GetRetainedMessage).get === - ConsumerActorUnregistered(consumer, consumer.actor.asInstanceOf[Consumer])) + ConsumerActorUnregistered(consumer, consumer.actorInstance.get.asInstanceOf[Consumer])) } } diff --git a/akka-camel/src/test/scala/akka/camel/ConsumerRegisteredTest.scala b/akka-camel/src/test/scala/akka/camel/ConsumerRegisteredTest.scala index aaab35a69c..be91f3ac2e 100644 --- a/akka-camel/src/test/scala/akka/camel/ConsumerRegisteredTest.scala +++ b/akka-camel/src/test/scala/akka/camel/ConsumerRegisteredTest.scala @@ -2,7 +2,7 @@ package akka.camel import org.junit.Test import org.scalatest.junit.JUnitSuite -import akka.actor.{ ActorRef, Actor } +import akka.actor.{ ActorRef, Actor, LocalActorRef } class ConsumerRegisteredTest extends JUnitSuite { import ConsumerRegisteredTest._ @@ -48,7 +48,10 @@ class ConsumerRegisteredTest extends JUnitSuite { assert(event === None) } - private def consumerOf(ref: ActorRef) = ref.actor.asInstanceOf[Consumer] + private def consumerOf(ref: ActorRef) = ref match { + case l: LocalActorRef ⇒ l.actorInstance.get.asInstanceOf[Consumer] + case _ ⇒ null: Consumer + } } object ConsumerRegisteredTest { diff --git a/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala b/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala index 52a1e91246..56e2b7e00a 100644 --- a/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala +++ b/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala @@ -103,7 +103,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher "receiving an in-out message exchange" must { "lead to a TimeoutException" in { service.awaitEndpointActivation(1) { - actorOf(new TestBlocker("direct:publish-test-5")).start + actorOf(Props(creator = () ⇒ new TestBlocker("direct:publish-test-5"), timeout = Timeout(1000))).start } must be(true) try { @@ -188,7 +188,7 @@ class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatcher } "be able to reply on failure during postStop" in { - val consumer = Actor.actorOf(new SupervisedConsumer("reply-channel-test-3")) + val consumer = Actor.actorOf(Props(new SupervisedConsumer("reply-channel-test-3")).withLifeCycle(Temporary)) val supervisor = Supervisor( SupervisorConfig( OneForOneStrategy(List(classOf[Exception]), 2, 10000), @@ -216,7 +216,6 @@ object ConsumerScalaTest { } class TestBlocker(uri: String) extends Actor with BlockingConsumer { - self.timeout = 1000 def endpointUri = uri protected def receive = { case msg: Message ⇒ { /* do not reply */ } diff --git a/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala b/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala index a7755f1346..30dd3f43a3 100644 --- a/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala +++ b/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala @@ -95,11 +95,10 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before when("a test message is sent to the producer with ?") val message = Message("test", Map(Message.MessageExchangeId -> "123")) - val result = (producer ? message).get + val result = (producer ? message).as[Message].get then("a normal response should have been returned by the producer") - val expected = Message("received test", Map(Message.MessageExchangeId -> "123")) - assert(result === expected) + assert(result.headers(Message.MessageExchangeId) === "123") } scenario("produce message and receive failure response") { @@ -188,11 +187,11 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before when("a test message is sent to the producer with ?") val message = Message("test", Map(Message.MessageExchangeId -> "123")) - val result = (producer ? message).get + val result = (producer ? message).as[Message].get then("a normal response should have been returned by the forward target") - val expected = Message("received test", Map(Message.MessageExchangeId -> "123", "test" -> "result")) - assert(result === expected) + assert(result.headers(Message.MessageExchangeId) === "123") + assert(result.headers("test") === "result") } scenario("produce message, forward failure response to a replying target actor and receive response") { diff --git a/akka-camel/src/test/scala/akka/camel/component/ActorComponentFeatureTest.scala b/akka-camel/src/test/scala/akka/camel/component/ActorComponentFeatureTest.scala index 3c1ce039b1..c837075368 100644 --- a/akka-camel/src/test/scala/akka/camel/component/ActorComponentFeatureTest.scala +++ b/akka-camel/src/test/scala/akka/camel/component/ActorComponentFeatureTest.scala @@ -7,7 +7,7 @@ import org.apache.camel.builder.RouteBuilder import org.apache.camel.component.mock.MockEndpoint import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec } -import akka.actor.Actor +import akka.actor.{ Actor, Props, Timeout } import akka.actor.Actor._ import akka.camel.{ Failure, Message, CamelContextManager } import akka.camel.CamelTestSupport._ @@ -47,7 +47,7 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with } scenario("two-way communication with timeout") { - val actor = actorOf[Tester3].start + val actor = actorOf(Props[Tester3].withTimeout(Timeout(1))).start intercept[RuntimeCamelException] { mandatoryTemplate.requestBody("actor:uuid:%s?blocking=true" format actor.uuid, "Martin") } diff --git a/akka-camel/src/test/scala/akka/camel/component/ActorProducerTest.scala b/akka-camel/src/test/scala/akka/camel/component/ActorProducerTest.scala index d1ebb5357b..29571a2b87 100644 --- a/akka-camel/src/test/scala/akka/camel/component/ActorProducerTest.scala +++ b/akka-camel/src/test/scala/akka/camel/component/ActorProducerTest.scala @@ -10,6 +10,7 @@ import org.junit.{ After, Test } import org.scalatest.junit.JUnitSuite import org.scalatest.BeforeAndAfterAll +import akka.actor.{ Props, Timeout } import akka.actor.Actor._ import akka.camel.{ Failure, Message } import akka.camel.CamelTestSupport._ @@ -228,7 +229,7 @@ class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll { @Test def shouldSendMessageToActorAndTimeout(): Unit = { - val actor = actorOf[Tester3].start + val actor = actorOf(Props[Tester3].withTimeout(Timeout(1))).start val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid) val exchange = endpoint.createExchange(ExchangePattern.InOut) exchange.getIn.setBody("Martin") diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 45323b880e..b4af807a69 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -15,8 +15,6 @@ import org.I0Itec.zkclient.exception._ import java.util.{ List ⇒ JList } import java.util.concurrent.atomic.{ AtomicBoolean, AtomicReference } import java.net.InetSocketAddress -import javax.management.StandardMBean - import scala.collection.mutable.ConcurrentMap import scala.collection.JavaConversions._ @@ -29,7 +27,7 @@ import Status._ import DeploymentConfig._ import akka.event.EventHandler -import akka.dispatch.{ Dispatchers, Future } +import akka.dispatch.{ Dispatchers, Future, PinnedDispatcher } import akka.remoteinterface._ import akka.routing.RouterType @@ -53,6 +51,7 @@ import com.google.protobuf.ByteString import java.util.concurrent.{ CopyOnWriteArrayList, Callable, ConcurrentHashMap } import annotation.tailrec +import javax.management.{ StandardMBean } // FIXME add watch for each node that when the entry for the node is removed then the node shuts itself down @@ -277,15 +276,15 @@ class DefaultClusterNode private[akka] ( // private val connectToAllNewlyArrivedMembershipNodesInClusterLock = new AtomicBoolean(false) - private[cluster] lazy val remoteClientLifeCycleListener = localActorOf(new Actor { + private[cluster] lazy val remoteClientLifeCycleListener = actorOf(Props(new Actor { def receive = { case RemoteClientError(cause, client, address) ⇒ client.shutdownClientModule() case RemoteClientDisconnected(client, address) ⇒ client.shutdownClientModule() case _ ⇒ //ignore other } - }, "akka.cluster.RemoteClientLifeCycleListener").start() + }), "akka.cluster.RemoteClientLifeCycleListener") - private[cluster] lazy val remoteDaemon = localActorOf(new RemoteClusterDaemon(this), RemoteClusterDaemon.Address).start() + private[cluster] lazy val remoteDaemon = actorOf(Props(new RemoteClusterDaemon(this)).copy(dispatcher = new PinnedDispatcher(), localOnly = true), RemoteClusterDaemon.Address) private[cluster] lazy val remoteDaemonSupervisor = Supervisor( SupervisorConfig( @@ -1714,8 +1713,6 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { import RemoteClusterDaemon._ import Cluster._ - self.dispatcher = Dispatchers.newPinnedDispatcher(self) - override def preRestart(reason: Throwable, msg: Option[Any]) { EventHandler.debug(this, "RemoteClusterDaemon failed due to [%s] restarting...".format(reason)) } @@ -1863,59 +1860,25 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { } def handle_fun0_unit(message: ClusterProtocol.RemoteDaemonMessageProtocol) { - localActorOf(new Actor() { - self.dispatcher = computeGridDispatcher - - def receive = { - case f: Function0[_] ⇒ try { - f() - } finally { - self.stop() - } - } - }).start ! payloadFor(message, classOf[Function0[Unit]]) + actorOf(Props( + self ⇒ { case f: Function0[_] ⇒ try { f() } finally { self.stop() } }).copy(dispatcher = computeGridDispatcher, localOnly = true)) ! payloadFor(message, classOf[Function0[Unit]]) } def handle_fun0_any(message: ClusterProtocol.RemoteDaemonMessageProtocol) { - localActorOf(new Actor() { - self.dispatcher = computeGridDispatcher - - def receive = { - case f: Function0[_] ⇒ try { - self.reply(f()) - } finally { - self.stop() - } - } - }).start forward payloadFor(message, classOf[Function0[Any]]) + actorOf(Props( + self ⇒ { case f: Function0[_] ⇒ try { self.reply(f()) } finally { self.stop() } }).copy(dispatcher = computeGridDispatcher, localOnly = true)) forward payloadFor(message, classOf[Function0[Any]]) } def handle_fun1_arg_unit(message: ClusterProtocol.RemoteDaemonMessageProtocol) { - localActorOf(new Actor() { - self.dispatcher = computeGridDispatcher - - def receive = { - case (fun: Function[_, _], param: Any) ⇒ try { - fun.asInstanceOf[Any ⇒ Unit].apply(param) - } finally { - self.stop() - } - } - }).start ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) + actorOf(Props( + self ⇒ { case (fun: Function[_, _], param: Any) ⇒ try { fun.asInstanceOf[Any ⇒ Unit].apply(param) } finally { self.stop() } }).copy(dispatcher = computeGridDispatcher, localOnly = true)) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) } def handle_fun1_arg_any(message: ClusterProtocol.RemoteDaemonMessageProtocol) { - localActorOf(new Actor() { - self.dispatcher = computeGridDispatcher - - def receive = { - case (fun: Function[_, _], param: Any) ⇒ try { - self.reply(fun.asInstanceOf[Any ⇒ Any](param)) - } finally { - self.stop() - } - } - }).start forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) + actorOf(Props( + self ⇒ { + case (fun: Function[_, _], param: Any) ⇒ try { self.reply(fun.asInstanceOf[Any ⇒ Any](param)) } finally { self.stop() } + }).copy(dispatcher = computeGridDispatcher, localOnly = true)) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) } def handleFailover(message: ClusterProtocol.RemoteDaemonMessageProtocol) { diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala index 28ddd922fc..6061b514db 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala @@ -54,18 +54,13 @@ object ClusterActorRef { * @author Jonas Bonér */ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], - _address: String, - _timeout: Long, + val address: String, + protected[akka] override val timeout: Long, val router: Router) extends UnsupportedActorRef { ClusterModule.ensureEnabled() - // val address = Address.clusterActorRefPrefix + _address - val address = _address - - timeout = _timeout - val connections = new ClusterActorRefConnections((Map[InetSocketAddress, ActorRef]() /: inetSocketAddresses) { case (map, (uuid, inetSocketAddress)) ⇒ map + (inetSocketAddress -> createRemoteActorRef(address, inetSocketAddress)) }) diff --git a/akka-cluster/src/main/scala/akka/serialization/SerializationProtocol.scala b/akka-cluster/src/main/scala/akka/serialization/SerializationProtocol.scala index 810f813efb..1e44a456f9 100644 --- a/akka-cluster/src/main/scala/akka/serialization/SerializationProtocol.scala +++ b/akka-cluster/src/main/scala/akka/serialization/SerializationProtocol.scala @@ -9,7 +9,7 @@ import akka.actor.{ uuidFrom, newUuid } import akka.actor._ import DeploymentConfig._ import akka.dispatch.MessageInvocation -import akka.util.ReflectiveAccess +import akka.util.{ ReflectiveAccess, Duration } import akka.remote.{ RemoteClientSettings, MessageSerializer } import akka.remote.protocol.RemoteProtocol import RemoteProtocol._ @@ -22,6 +22,7 @@ import com.google.protobuf.ByteString import com.eaio.uuid.UUID import akka.event.EventHandler +import java.util.{ LinkedList, Collections } /** * Module for local actor serialization. @@ -60,6 +61,11 @@ object ActorSerialization { serializeMailBox: Boolean, replicationScheme: ReplicationScheme): SerializedActorRefProtocol = { + val localRef: Option[LocalActorRef] = actorRef match { + case l: LocalActorRef ⇒ Some(l) + case _ ⇒ None + } + val lifeCycleProtocol: Option[LifeCycleProtocol] = { actorRef.lifeCycle match { case Permanent ⇒ Some(LifeCycleProtocol.newBuilder.setLifeCycle(LifeCycleType.PERMANENT).build) @@ -71,9 +77,11 @@ object ActorSerialization { val builder = SerializedActorRefProtocol.newBuilder .setUuid(UuidProtocol.newBuilder.setHigh(actorRef.uuid.getTime).setLow(actorRef.uuid.getClockSeqAndNode).build) .setAddress(actorRef.address) - .setActorClassname(actorRef.actorInstance.get.getClass.getName) .setTimeout(actorRef.timeout) + if (localRef.isDefined) + builder.setActorClassname(localRef.get.actorInstance.get.getClass.getName) //TODO FIXME Why is the classname needed anymore? + replicationScheme match { case _: Transient | Transient ⇒ builder.setReplicationStorage(ReplicationStorageType.TRANSIENT) @@ -86,47 +94,53 @@ object ActorSerialization { builder.setReplicationStorage(storageType) val strategyType = strategy match { - case _: WriteBehind | WriteBehind ⇒ ReplicationStrategyType.WRITE_BEHIND - case _: WriteThrough | WriteThrough ⇒ ReplicationStrategyType.WRITE_THROUGH + case _: WriteBehind ⇒ ReplicationStrategyType.WRITE_BEHIND + case _: WriteThrough ⇒ ReplicationStrategyType.WRITE_THROUGH } builder.setReplicationStrategy(strategyType) } - if (serializeMailBox == true) { - if (actorRef.mailbox eq null) throw new IllegalActorStateException("Can't serialize an actor that has not been started.") - val messages = - actorRef.mailbox match { + lifeCycleProtocol.foreach(builder.setLifeCycle(_)) + actorRef.supervisor.foreach(s ⇒ builder.setSupervisor(RemoteActorSerialization.toRemoteActorRefProtocol(s))) + + localRef foreach { l ⇒ + if (serializeMailBox) { + l.mailbox match { + case null ⇒ throw new IllegalActorStateException("Can't serialize an actor that has not been started.") case q: java.util.Queue[_] ⇒ val l = new scala.collection.mutable.ListBuffer[MessageInvocation] val it = q.iterator while (it.hasNext) l += it.next.asInstanceOf[MessageInvocation] - l + + l map { m ⇒ + RemoteActorSerialization.createRemoteMessageProtocolBuilder( + Option(m.receiver), + Left(actorRef.uuid), + actorRef.address, + actorRef.timeout, + Right(m.message), + false, + m.channel match { + case a: ActorRef ⇒ Some(a) + case _ ⇒ None + }) + } foreach { + builder.addMessages(_) + } } + } - val requestProtocols = - messages.map(m ⇒ - RemoteActorSerialization.createRemoteMessageProtocolBuilder( - Some(actorRef), - Left(actorRef.uuid), - actorRef.address, - actorRef.timeout, - Right(m.message), - false, - actorRef.getSender)) - - requestProtocols.foreach(builder.addMessages(_)) + l.receiveTimeout.foreach(builder.setReceiveTimeout(_)) + val actorInstance = l.actorInstance.get + Serialization.serialize(actorInstance.asInstanceOf[T]) match { + case Right(bytes) ⇒ builder.setActorInstance(ByteString.copyFrom(bytes)) + case Left(exception) ⇒ throw new Exception("Error serializing : " + actorInstance.getClass.getName) + } + val stack = l.hotswap + if (!stack.isEmpty) + builder.setHotswapStack(ByteString.copyFrom(akka.serialization.JavaSerializer.toBinary(stack))) } - actorRef.receiveTimeout.foreach(builder.setReceiveTimeout(_)) - Serialization.serialize(actorRef.actor.asInstanceOf[T]) match { - case Right(bytes) ⇒ builder.setActorInstance(ByteString.copyFrom(bytes)) - case Left(exception) ⇒ throw new Exception("Error serializing : " + actorRef.actor.getClass.getName) - } - - lifeCycleProtocol.foreach(builder.setLifeCycle(_)) - actorRef.supervisor.foreach(s ⇒ builder.setSupervisor(RemoteActorSerialization.toRemoteActorRefProtocol(s))) - // if (!actorRef.hotswap.isEmpty) builder.setHotswapStack(ByteString.copyFrom(Serializers.Java.toBinary(actorRef.hotswap))) - if (!actorRef.hotswap.isEmpty) builder.setHotswapStack(ByteString.copyFrom(akka.serialization.JavaSerializer.toBinary(actorRef.hotswap))) builder.build } @@ -145,19 +159,6 @@ object ActorSerialization { EventHandler.debug(this, "Deserializing SerializedActorRefProtocol to LocalActorRef:\n%s".format(protocol)) - val lifeCycle = - if (protocol.hasLifeCycle) { - protocol.getLifeCycle.getLifeCycle match { - case LifeCycleType.PERMANENT ⇒ Permanent - case LifeCycleType.TEMPORARY ⇒ Temporary - case unknown ⇒ throw new IllegalActorStateException("LifeCycle type is not valid [" + unknown + "]") - } - } else UndefinedLifeCycle - - val supervisor = - if (protocol.hasSupervisor) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader)) - else None - // import ReplicationStorageType._ // import ReplicationStrategyType._ // val replicationScheme = @@ -180,7 +181,7 @@ object ActorSerialization { // } // } else Transient - val hotswap = + val storedHotswap = try { Serialization.deserialize( protocol.getHotswapStack.toByteArray, @@ -193,17 +194,26 @@ object ActorSerialization { case e: Exception ⇒ Stack[PartialFunction[Any, Unit]]() } - val classLoader = loader.getOrElse(this.getClass.getClassLoader) - - val factory = () ⇒ { - val actorClass = classLoader.loadClass(protocol.getActorClassname) - try { - Serialization.deserialize(protocol.getActorInstance.toByteArray, actorClass, loader) match { - case Right(r) ⇒ r.asInstanceOf[Actor] - case Left(ex) ⇒ throw new Exception("Cannot de-serialize : " + actorClass) + val storedLifeCycle = + if (protocol.hasLifeCycle) { + protocol.getLifeCycle.getLifeCycle match { + case LifeCycleType.PERMANENT ⇒ Permanent + case LifeCycleType.TEMPORARY ⇒ Temporary + case unknown ⇒ UndefinedLifeCycle } - } catch { - case e: Exception ⇒ actorClass.newInstance.asInstanceOf[Actor] + } else UndefinedLifeCycle + + val storedSupervisor = + if (protocol.hasSupervisor) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader)) + else None + + val classLoader = loader.getOrElse(this.getClass.getClassLoader) + val bytes = protocol.getActorInstance.toByteArray + val actorClass = classLoader.loadClass(protocol.getActorClassname) + val factory = () ⇒ { + Serialization.deserialize(bytes, actorClass, loader) match { + case Right(r) ⇒ r.asInstanceOf[Actor] + case Left(ex) ⇒ throw new Exception("Cannot de-serialize : " + actorClass) } } @@ -212,18 +222,25 @@ object ActorSerialization { case None ⇒ uuidFrom(protocol.getUuid.getHigh, protocol.getUuid.getLow) } - val ar = new LocalActorRef( - actorUuid, - protocol.getAddress, - if (protocol.hasTimeout) protocol.getTimeout else Actor.TIMEOUT, - if (protocol.hasReceiveTimeout) Some(protocol.getReceiveTimeout) else None, - lifeCycle, - supervisor, - hotswap, - factory) + val props = Props(creator = factory, + timeout = if (protocol.hasTimeout) protocol.getTimeout else Timeout.default, + lifeCycle = storedLifeCycle, + supervisor = storedSupervisor //TODO what dispatcher should it use? + //TODO what faultHandler should it use? + // + ) + + val receiveTimeout = if (protocol.hasReceiveTimeout) Some(protocol.getReceiveTimeout) else None //TODO FIXME, I'm expensive and slow + + val ar = new LocalActorRef(actorUuid, protocol.getAddress, props, receiveTimeout, storedHotswap) + + //Deserialize messages + { + val iterator = protocol.getMessagesList.iterator() + while (iterator.hasNext()) + ar ! MessageSerializer.deserialize(iterator.next().getMessage, Some(classLoader)) //TODO This is broken, why aren't we preserving the sender? + } - val messages = protocol.getMessagesList.toArray.toList.asInstanceOf[List[RemoteMessageProtocol]] - messages.foreach(message ⇒ ar ! MessageSerializer.deserialize(message.getMessage, Some(classLoader))) ar } } diff --git a/akka-cluster/src/test/scala/akka/serialization/ActorSerializeSpec.scala b/akka-cluster/src/test/scala/akka/serialization/ActorSerializeSpec.scala index 1430ba4c9e..67a29396f0 100644 --- a/akka-cluster/src/test/scala/akka/serialization/ActorSerializeSpec.scala +++ b/akka-cluster/src/test/scala/akka/serialization/ActorSerializeSpec.scala @@ -21,13 +21,12 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll describe("Serializable actor") { it("should be able to serialize and de-serialize a stateful actor with a given serializer") { - val actor1 = localActorOf[MyJavaSerializableActor].start() + val actor1 = localActorOf[MyJavaSerializableActor].start().asInstanceOf[LocalActorRef] (actor1 ? "hello").get should equal("world 1") (actor1 ? "hello").get should equal("world 2") val bytes = toBinary(actor1) - val actor2 = fromBinary(bytes) - actor2.start() + val actor2 = fromBinary(bytes).start().asInstanceOf[LocalActorRef] (actor2 ? "hello").get should equal("world 3") actor2.receiveTimeout should equal(Some(1000)) @@ -37,7 +36,7 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll it("should be able to serialize and deserialize a MyStatelessActorWithMessagesInMailbox") { - val actor1 = localActorOf[MyStatelessActorWithMessagesInMailbox].start() + val actor1 = localActorOf[MyStatelessActorWithMessagesInMailbox].start().asInstanceOf[LocalActorRef] (actor1 ! "hello") (actor1 ! "hello") (actor1 ! "hello") @@ -49,12 +48,12 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll (actor1 ! "hello") (actor1 ! "hello") actor1.getDispatcher.mailboxSize(actor1) should be > (0) - val actor2 = fromBinary(toBinary(actor1)) + val actor2 = fromBinary(toBinary(actor1)).asInstanceOf[LocalActorRef] Thread.sleep(1000) actor2.getDispatcher.mailboxSize(actor1) should be > (0) (actor2 ? "hello-reply").get should equal("world") - val actor3 = fromBinary(toBinary(actor1, false)) + val actor3 = fromBinary(toBinary(actor1, false)).asInstanceOf[LocalActorRef] Thread.sleep(1000) actor3.getDispatcher.mailboxSize(actor1) should equal(0) (actor3 ? "hello-reply").get should equal("world") @@ -63,7 +62,7 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll it("should be able to serialize and deserialize a PersonActorWithMessagesInMailbox") { val p1 = Person("debasish ghosh", 25, SerializeSpec.Address("120", "Monroe Street", "Santa Clara", "95050")) - val actor1 = localActorOf[PersonActorWithMessagesInMailbox].start() + val actor1 = localActorOf[PersonActorWithMessagesInMailbox].start().asInstanceOf[LocalActorRef] (actor1 ! p1) (actor1 ! p1) (actor1 ! p1) @@ -75,12 +74,12 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll (actor1 ! p1) (actor1 ! p1) actor1.getDispatcher.mailboxSize(actor1) should be > (0) - val actor2 = fromBinary(toBinary(actor1)) + val actor2 = fromBinary(toBinary(actor1)).asInstanceOf[LocalActorRef] Thread.sleep(1000) actor2.getDispatcher.mailboxSize(actor1) should be > (0) (actor2 ? "hello-reply").get should equal("hello") - val actor3 = fromBinary(toBinary(actor1, false)) + val actor3 = fromBinary(toBinary(actor1, false)).asInstanceOf[LocalActorRef] Thread.sleep(1000) actor3.getDispatcher.mailboxSize(actor1) should equal(0) (actor3 ? "hello-reply").get should equal("hello") @@ -107,7 +106,7 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll describe("serialize actor that accepts protobuf message") { it("should serialize") { - val actor1 = localActorOf[MyActorWithProtobufMessagesInMailbox].start() + val actor1 = localActorOf[MyActorWithProtobufMessagesInMailbox].start().asInstanceOf[LocalActorRef] val msg = MyMessage(123, "debasish ghosh", true) val b = ProtobufProtocol.MyMessage.newBuilder.setId(msg.id).setName(msg.name).setStatus(msg.status).build (actor1 ! b) @@ -121,12 +120,12 @@ class ActorSerializeSpec extends Spec with ShouldMatchers with BeforeAndAfterAll (actor1 ! b) (actor1 ! b) actor1.getDispatcher.mailboxSize(actor1) should be > (0) - val actor2 = fromBinary(toBinary(actor1)) + val actor2 = fromBinary(toBinary(actor1)).asInstanceOf[LocalActorRef] Thread.sleep(1000) actor2.getDispatcher.mailboxSize(actor1) should be > (0) (actor2 ? "hello-reply").get should equal("world") - val actor3 = fromBinary(toBinary(actor1, false)) + val actor3 = fromBinary(toBinary(actor1, false)).asInstanceOf[LocalActorRef] Thread.sleep(1000) actor3.getDispatcher.mailboxSize(actor1) should equal(0) (actor3 ? "hello-reply").get should equal("world") diff --git a/akka-docs/cluster/durable-mailbox.rst b/akka-docs/cluster/durable-mailbox.rst index 1c9b577997..4965eeff8a 100644 --- a/akka-docs/cluster/durable-mailbox.rst +++ b/akka-docs/cluster/durable-mailbox.rst @@ -30,6 +30,7 @@ The durable mailboxes currently supported are: - ``ZooKeeperDurableMailboxStorage`` -- backed by ZooKeeper - ``BeanstalkDurableMailboxStorage`` -- backed by Beanstalkd - ``MongoNaiveDurableMailboxStorage`` -- backed by MongoDB + We'll walk through each one of these in detail in the sections below. Soon Akka will also have: @@ -235,7 +236,7 @@ MongoDB is a fast, lightweight and scalable document-oriented database. It cont features cohesive to a fast, reliable & durable queueing mechanism which the Akka Mailbox takes advantage of. -Akka's implementations of MongoDB mailboxes are built on top of the purely asynchronous MongoDB driver (often known as `Hammersmith `_ and ``com.mongodb.async``) and as such are purely callback based with a Netty network layer. This makes them extremely fast & lightweight versus building on other MongoDB implementations such as `mongo-java-driver `_ and `Casbah `_ and ``com.mongodb.async``) and as such are purely callback based with a Netty network layer. This makes them extremely fast & lightweight versus building on other MongoDB implementations such as `mongo-java-driver `_ and `Casbah `_. You will need to configure the URI for the MongoDB server, using the URI Format specified in the `MongoDB Documentation `_. This is done in the ``akka.actor.mailbox.mongodb`` section in the ``akka.conf`` configuration diff --git a/akka-docs/general/jmm.rst b/akka-docs/general/jmm.rst index df25f983c4..4c907be85c 100644 --- a/akka-docs/general/jmm.rst +++ b/akka-docs/general/jmm.rst @@ -9,31 +9,38 @@ The Java Memory Model --------------------- Prior to Java 5, the Java Memory Model (JMM) was ill defined. It was possible to get all kinds of strange results when shared memory was accessed by multiple threads, such as: + * a thread not seeing values written by other threads: a visibility problem * a thread observing 'impossible' behavior of other threads, caused by instructions not being executed in the order - expected: an instruction reordering problem. + +expected: an instruction reordering problem. With the implementation of JSR 133 in Java 5, a lot of these issues have been resolved. The JMM is a set of rules based on the "happens-before" relation, which constrain when one memory access must happen before another, and conversely, when they are allowed to happen out of order. Two examples of these rules are: + * **The monitor lock rule:** a release of a lock happens before every subsequent acquire of the same lock. * **The volatile variable rule:** a write of a volatile variable happens before every subsequent read of the same volatile variable + Although the JMM can seem complicated, the specification tries to find a balance between ease of use and the ability to write performant and scalable concurrent data structures. Actors and the Java Memory Model -------------------------------- With the Actors implementation in Akka, there are two ways multiple threads can execute actions on shared memory: + * if a message is sent to an actor (e.g. by another actor). In most cases messages are immutable, but if that message - is not a properly constructed immutable object, without a "happens before" rule, it would be possible for the receiver - to see partially initialized data structures and possibly even values out of thin air (longs/doubles). +is not a properly constructed immutable object, without a "happens before" rule, it would be possible for the receiver +to see partially initialized data structures and possibly even values out of thin air (longs/doubles). * if an actor makes changes to its internal state while processing a message, and accesses that state while processing - another message moments later. It is important to realize that with the actor model you don't get any guarantee that - the same thread will be executing the same actor for different messages. +another message moments later. It is important to realize that with the actor model you don't get any guarantee that +the same thread will be executing the same actor for different messages. To prevent visibility and reordering problems on actors, Akka guarantees the following two "happens before" rules: + * **The actor send rule:** the send of the message to an actor happens before the receive of that message by the same actor. * **The actor subsequent processing rule:** processing of one message happens before processing of the next message by the same actor. + Both rules only apply for the same actor instance and are not valid if different actors are used. STM and the Java Memory Model diff --git a/akka-docs/intro/getting-started-first-java.rst b/akka-docs/intro/getting-started-first-java.rst index 43bb638bfa..da606c7d84 100644 --- a/akka-docs/intro/getting-started-first-java.rst +++ b/akka-docs/intro/getting-started-first-java.rst @@ -159,7 +159,9 @@ Here is the layout that Maven created:: As you can see we already have a Java source file called ``App.java``, let's now rename it to ``Pi.java``. -We also need to edit the ``pom.xml`` build file. Let's add the dependency we need as well as the Maven repository it should download it from. It should now look something like this: +We also need to edit the ``pom.xml`` build file. Let's add the dependency we need as well as the Maven repository it should download it from. The Akka Maven repository can be found at ``_ +and Typesafe provides ``_ that proxies several other repositories, including akka.io. +It should now look something like this: .. code-block:: xml @@ -186,9 +188,9 @@ We also need to edit the ``pom.xml`` build file. Let's add the dependency we nee - Akka - Akka Maven2 Repository - http://akka.io/repository/ + typesafe + Typesafe Repository + http://repo.typesafe.com/typesafe/releases/ diff --git a/akka-docs/intro/getting-started-first-scala-eclipse.rst b/akka-docs/intro/getting-started-first-scala-eclipse.rst index 39629578e7..ebe3785828 100644 --- a/akka-docs/intro/getting-started-first-scala-eclipse.rst +++ b/akka-docs/intro/getting-started-first-scala-eclipse.rst @@ -158,44 +158,21 @@ If you have not already done so, now is the time to create an Eclipse project fo Using SBT in Eclipse ^^^^^^^^^^^^^^^^^^^^ -If you are an `SBT `_ user, you can follow the :ref:`getting-started-first-scala-download-sbt` instruction and additionally install the ``sbt-eclipse`` plugin. This adds support for generating Eclipse project files from your SBT project. You need to update your SBT plugins definition in ``project/plugins``:: - - import sbt._ - - class TutorialPlugins(info: ProjectInfo) extends PluginDefinition(info) { - // eclipsify plugin - lazy val eclipse = "de.element34" % "sbt-eclipsify" % "0.7.0" - - val akkaRepo = "Akka Repo" at "http://akka.io/repository" - val akkaPlugin = "se.scalablesolutions.akka" % "akka-sbt-plugin" % "2.0-SNAPSHOT" - } - -and then update your SBT project definition by mixing in ``Eclipsify`` in your project definition:: - - import sbt._ - import de.element34.sbteclipsify._ - - class MySbtProject(info: ProjectInfo) extends DefaultProject(info) - with Eclipsify with AkkaProject { - // the project definition here - // akka dependencies - } +If you are an `SBT `_ user, you can follow the :ref:`getting-started-first-scala-download-sbt` instruction and additionally install the ``sbteclipse`` plugin. This adds support for generating Eclipse project files from your SBT project. +You need to install the plugin as described in the `README of sbteclipse `_ Then run the ``eclipse`` target to generate the Eclipse project:: - dragos@dragos-imac pi $ sbt eclipse - [info] Building project AkkaPi 1.0 against Scala 2.9.0 - [info] using MySbtProject with sbt 0.7.4 and Scala 2.7.7 - [info] - [info] == eclipse == - [info] Creating eclipse project... - [info] == eclipse == - [success] Successful. - [info] - [info] Total time: 0 s, completed Apr 20, 2011 2:48:03 PM - [info] - [info] Total session time: 1 s, completed Apr 20, 2011 2:48:03 PM - [success] Build completed successfully. + $ sbt + > eclipse + +The options `create-src` and `with-sources` are useful:: + + $ sbt + > eclipse create-src with-sources + +* create-src to create the common source directories, e.g. src/main/scala, src/main/test +* with-sources to create source attachments for the library dependencies Next you need to import this project in Eclipse, by choosing ``Eclipse/Import.. Existing Projects into Workspace``. Navigate to the directory where you defined your SBT project and choose import: diff --git a/akka-docs/intro/getting-started-first-scala.rst b/akka-docs/intro/getting-started-first-scala.rst index c86e4e51f6..71591a3134 100644 --- a/akka-docs/intro/getting-started-first-scala.rst +++ b/akka-docs/intro/getting-started-first-scala.rst @@ -146,59 +146,38 @@ Downloading and installing SBT SBT, short for 'Simple Build Tool' is an excellent build system written in Scala. It uses Scala to write the build scripts which gives you a lot of power. It has a plugin architecture with many plugins available, something that we will take advantage of soon. SBT is the preferred way of building software in Scala and is probably the easiest way of getting through this tutorial. If you want to use SBT for this tutorial then follow the following instructions, if not you can skip this section and the next. -First browse to `http://code.google.com/p/simple-build-tool/downloads/list `_ and download the ``0.7.6.RC0`` distribution. - -To install SBT and create a project for this tutorial it is easiest to follow the instructions on `http://code.google.com/p/simple-build-tool/wiki/Setup `_. +To install SBT and create a project for this tutorial it is easiest to follow the instructions on `https://github.com/harrah/xsbt/wiki/Setup `_. Now we need to create our first Akka project. You could add the dependencies manually to the build script, but the easiest way is to use Akka's SBT Plugin, covered in the next section. Creating an Akka SBT project ---------------------------- -If you have not already done so, now is the time to create an SBT project for our tutorial. You do that by stepping into the directory you want to create your project in and invoking the ``sbt`` command answering the questions for setting up your project (just pressing ENTER will choose the default in square brackets):: +If you have not already done so, now is the time to create an SBT project for our tutorial. You do that by adding the following content to ``build.sbt`` file in the directory you want to create your project in:: - $ sbt - Project does not exist, create new project? (y/N/s) y - Name: Tutorial 1 - Organization: Hakkers Inc - Version [1.0]: - Scala version [2.9.0]: - sbt version [0.7.6.RC0]: + name := "My Project" -Now we have the basis for an SBT project. Akka has an SBT Plugin making it very easy to use Akka is an SBT-based project so let's use that. + version := "1.0" -To use the plugin, first add a plugin definition to your SBT project by creating a ``Plugins.scala`` file in the ``project/plugins`` directory containing:: + scalaVersion := "2.9.0-1" - import sbt._ + resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" - class Plugins(info: ProjectInfo) extends PluginDefinition(info) { - val akkaRepo = "Akka Repo" at "http://akka.io/repository" - val akkaPlugin = "se.scalablesolutions.akka" % "akka-sbt-plugin" % "2.0-SNAPSHOT" - } + libraryDependencies += "se.scalablesolutions.akka" % "akka-actor" % "1.2-SNAPSHOT" -Now we need to create a project definition using our Akka SBT plugin. We do that by creating a ``project/build/Project.scala`` file containing:: +Create a directory ``src/main/scala`` in which you will store the Scala source files. - import sbt._ +Not needed in this tutorial, but if you would like to use additional Akka modules beyond ``akka-actor``, you can add these as ``libraryDependencies`` in ``build.sbt``. Note that there must be a blank line between each. Here is an example adding ``akka-remote`` and ``akka-stm``:: - class TutorialOneProject(info: ProjectInfo) extends DefaultProject(info) with AkkaProject + libraryDependencies += "se.scalablesolutions.akka" % "akka-actor" % "1.2-SNAPSHOT" + + libraryDependencies += "se.scalablesolutions.akka" % "akka-remote" % "1.2-SNAPSHOT" + + libraryDependencies += "se.scalablesolutions.akka" % "akka-stm" % "1.2-SNAPSHOT" -The magic is in mixing in the ``AkkaProject`` trait. +So, now we are all set. -Not needed in this tutorial, but if you would like to use additional Akka modules beyond ``akka-actor``, you can add these as "module configurations" in the project file. Here is an example adding ``akka-remote`` and ``akka-stm``:: - - class AkkaSampleProject(info: ProjectInfo) extends DefaultProject(info) with AkkaProject { - val akkaSTM = akkaModule("stm") - val akkaRemote = akkaModule("remote") - } - -So, now we are all set. Just one final thing to do; make SBT download the dependencies it needs. That is done by invoking:: - - > reload - > update - -The first reload command is needed because we have changed the project definition since the sbt session started. - -SBT itself needs a whole bunch of dependencies but our project will only need one; ``akka-actor-2.0-SNAPSHOT.jar``. SBT downloads that as well. +SBT itself needs a whole bunch of dependencies but our project will only need one; ``akka-actor-2.0-SNAPSHOT.jar``. SBT will download that as well. Start writing the code ---------------------- @@ -537,8 +516,6 @@ Run it inside SBT If you used SBT, then you can run the application directly inside SBT. First you need to compile the project:: $ sbt - > update - ... > compile ... diff --git a/akka-docs/intro/getting-started.rst b/akka-docs/intro/getting-started.rst index 377f2c8de8..5edf2a627c 100644 --- a/akka-docs/intro/getting-started.rst +++ b/akka-docs/intro/getting-started.rst @@ -75,7 +75,8 @@ More information is available in the documentation of the :ref:`microkernel`. Using a build tool ------------------ -Akka can be used with build tools that support Maven repositories. The Akka Maven repository can be found at ``_. +Akka can be used with build tools that support Maven repositories. The Akka Maven repository can be found at ``_ +and Typesafe provides ``_ that proxies several other repositories, including akka.io. Using Akka with Maven --------------------- @@ -90,9 +91,9 @@ Summary of the essential parts for using Akka with Maven: .. code-block:: xml - Akka - Akka Maven2 Repository - http://akka.io/repository/ + typesafe + Typesafe Repository + http://repo.typesafe.com/typesafe/releases/ 2) Add the Akka dependencies. For example, here is the dependency for Akka Actor 2.0-SNAPSHOT: @@ -115,44 +116,19 @@ can be found in the :ref:`getting-started-first-scala`. Summary of the essential parts for using Akka with SBT: -1) Akka has an SBT plugin which makes it very easy to get started with Akka and SBT. +SBT installation instructions on `https://github.com/harrah/xsbt/wiki/Setup `_ -The Scala version in your SBT project needs to match the version that Akka is built against. For Akka 2.0-SNAPSHOT this is -Scala version 2.9.0. +``build.sbt`` file:: -To use the plugin, first add a plugin definition to your SBT project by creating project/plugins/Plugins.scala with: + name := "My Project" -.. code-block:: scala + version := "1.0" - import sbt._ + scalaVersion := "2.9.0-1" - class Plugins(info: ProjectInfo) extends PluginDefinition(info) { - val akkaRepo = "Akka Repo" at "http://akka.io/repository" - val akkaPlugin = "se.scalablesolutions.akka" % "akka-sbt-plugin" % "2.0-SNAPSHOT" - } + resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" -*Note: the plugin version matches the Akka version provided. The current release is 2.0-SNAPSHOT.* - -2) Then mix the AkkaProject trait into your project definition. For example: - -.. code-block:: scala - - class MyProject(info: ProjectInfo) extends DefaultProject(info) with AkkaProject - -*Note: This adds akka-actor as a dependency by default.* - -If you also want to include other Akka modules there is a convenience method: ``akkaModule``. For example, you can add extra Akka modules by adding any of the following lines to your project class: - -.. code-block:: scala - - val akkaStm = akkaModule("stm") - val akkaTypedActor = akkaModule("typed-actor") - val akkaRemote = akkaModule("remote") - val akkaHttp = akkaModule("http") - val akkaAmqp = akkaModule("amqp") - val akkaCamel = akkaModule("camel") - val akkaCamelTyped = akkaModule("camel-typed") - val akkaSpring = akkaModule("spring") + libraryDependencies += "se.scalablesolutions.akka" % "akka-actor" % "2.0-SNAPSHOT" Using Akka with Eclipse @@ -161,6 +137,8 @@ Using Akka with Eclipse Information about how to use Akka with Eclipse, including how to create an Akka Eclipse project from scratch, can be found in the :ref:`getting-started-first-scala-eclipse`. +Setup SBT project and then use `sbteclipse `_ to generate Eclipse project. + Using Akka with IntelliJ IDEA ----------------------------- diff --git a/akka-docs/java/fault-tolerance.rst b/akka-docs/java/fault-tolerance.rst index 1b1a06245c..7ad7553031 100644 --- a/akka-docs/java/fault-tolerance.rst +++ b/akka-docs/java/fault-tolerance.rst @@ -233,16 +233,13 @@ Here is the API and how to use it from within an 'Actor': getContext().unlink(actorRef); // starts and links Actors atomically - getContext().startLink(actorRef); - getContext().startLinkRemote(actorRef); + getContext().link(actorRef).start(); // spawns (creates and starts) actors getContext().spawn(MyActor.class); - getContext().spawnRemote(MyActor.class); // spawns and links Actors atomically getContext().spawnLink(MyActor.class); - getContext().spawnLinkRemote(MyActor.class); A child actor can tell the supervising actor to unlink him by sending him the 'Unlink(this)' message. When the supervisor receives the message he will unlink and shut down the child. The supervisor for an actor is available in the 'supervisor: Option[Actor]' method in the 'ActorRef' class. Here is how it can be used. diff --git a/akka-docs/scala/fault-tolerance.rst b/akka-docs/scala/fault-tolerance.rst index 507c3a3b88..b6a6d1d1ae 100644 --- a/akka-docs/scala/fault-tolerance.rst +++ b/akka-docs/scala/fault-tolerance.rst @@ -213,8 +213,8 @@ Here is the API and how to use it from within an 'Actor': self.link(actorRef) self.unlink(actorRef) - // starts and links Actors atomically - self.startLink(actorRef) + // link first, then start actor + self.link(actorRef).start() // spawns (creates and starts) actors self.spawn[MyActor] diff --git a/akka-docs/scala/index.rst b/akka-docs/scala/index.rst index 7998ae2765..0c8111cab0 100644 --- a/akka-docs/scala/index.rst +++ b/akka-docs/scala/index.rst @@ -21,6 +21,5 @@ Scala API routing fsm http - security testing tutorial-chat-server 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 c873117d41..3b9f6c7357 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 @@ -5,7 +5,7 @@ package akka.actor.mailbox import MailboxProtocol._ -import akka.actor.ActorRef +import akka.actor.LocalActorRef import akka.dispatch._ import akka.config.Config._ import akka.util.Duration @@ -20,7 +20,7 @@ class BeanstalkBasedMailboxException(message: String) extends AkkaException(mess /** * @author Jonas Bonér */ -class BeanstalkBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(owner) { +class BeanstalkBasedMailbox(val owner: LocalActorRef) extends DurableExecutableMailbox(owner) { val hostname = config.getString("akka.actor.mailbox.beanstalk.hostname", "0.0.0.0") val port = config.getInt("akka.actor.mailbox.beanstalk.port", 11300) 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 4795953af3..36bf0c7c9e 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 @@ -4,7 +4,7 @@ package akka.actor.mailbox -import akka.actor.ActorRef +import akka.actor.LocalActorRef import akka.dispatch._ import akka.config.Config._ import akka.event.EventHandler @@ -18,7 +18,7 @@ private[akka] object FileBasedMailboxUtil { val queuePath = config.getString("akka.actor.mailbox.file-based.directory-path", "./_mb") // /var/spool/akka } -class FileBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(owner) { +class FileBasedMailbox(val owner: LocalActorRef) extends DurableExecutableMailbox(owner) { import FileBasedMailboxUtil._ private val queue = try { diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala index 9adb6312b7..7262992759 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala @@ -3,21 +3,21 @@ */ package akka.actor.mailbox -import akka.actor.{ newUuid, ActorRef } import akka.util.ReflectiveAccess import akka.dispatch._ import akka.config._ import akka.event.EventHandler import java.lang.reflect.InvocationTargetException +import akka.actor.{ LocalActorRef, newUuid, ActorRef } /** * @author Jonas Bonér */ sealed abstract class DurableMailboxStorage(mailboxFQN: String) { - val constructorSignature = Array[Class[_]](classOf[ActorRef]) + val constructorSignature = Array[Class[_]](classOf[LocalActorRef]) - val mailboxClass: Class[_] = ReflectiveAccess.getClassFor(mailboxFQN, classOf[ActorRef].getClassLoader) match { + val mailboxClass: Class[_] = ReflectiveAccess.getClassFor(mailboxFQN, classOf[LocalActorRef].getClassLoader) match { case Right(clazz) ⇒ clazz case Left(exception) ⇒ val cause = exception match { @@ -81,14 +81,14 @@ case class DurableDispatcher( def this(_name: String, _storage: DurableMailboxStorage) = this(_name, _storage, Dispatchers.THROUGHPUT, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, Dispatchers.MAILBOX_TYPE) // Needed for Java API usage - override def register(actorRef: ActorRef) { + override def register(actorRef: LocalActorRef) { super.register(actorRef) - val mbox = actorRef.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox] + val mbox = getMailbox(actorRef) if (mbox ne null) //Schedule the ActorRef for initial execution, because we might be resuming operations after a failure super.registerForExecution(mbox) } - override def createMailbox(actorRef: ActorRef): AnyRef = _storage.createFor(actorRef) + override def createMailbox(actorRef: LocalActorRef): AnyRef = _storage.createFor(actorRef) private[akka] override def dispatch(invocation: MessageInvocation): Unit = { if (invocation.channel.isInstanceOf[ActorPromise]) @@ -96,7 +96,7 @@ case class DurableDispatcher( super.dispatch(invocation) } - protected override def cleanUpMailboxFor(actorRef: ActorRef) {} //No need to clean up Futures since we don't support them + protected override def cleanUpMailboxFor(actorRef: LocalActorRef) {} //No need to clean up Futures since we don't support them } /** @@ -118,14 +118,14 @@ case class DurablePinnedDispatcher( def this(actor: ActorRef, _storage: DurableMailboxStorage, capacity: Int, pushTimeOut: akka.util.Duration) = //For Java API this(actor, _storage, BoundedMailbox(capacity, pushTimeOut)) - override def register(actorRef: ActorRef) { + override def register(actorRef: LocalActorRef) { super.register(actorRef) val mbox = actorRef.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox] if (mbox ne null) //Schedule the ActorRef for initial execution, because we might be resuming operations after a failure super.registerForExecution(mbox) } - override def createMailbox(actorRef: ActorRef): AnyRef = _storage.createFor(actorRef) + override def createMailbox(actorRef: LocalActorRef): AnyRef = _storage.createFor(actorRef) private[akka] override def dispatch(invocation: MessageInvocation): Unit = { if (invocation.channel.isInstanceOf[ActorPromise]) 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 381987e9f3..74fdf0d181 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 @@ -5,12 +5,12 @@ package akka.actor.mailbox import MailboxProtocol._ -import akka.actor.{ Actor, ActorRef, NullChannel } import akka.dispatch._ import akka.event.EventHandler import akka.remote.MessageSerializer import akka.remote.protocol.RemoteProtocol.MessageProtocol import akka.AkkaException +import akka.actor._ /** * @author Jonas Bonér @@ -24,12 +24,14 @@ private[akka] object DurableExecutableMailboxConfig { val Name = "[\\.\\/\\$\\s]".r } -class DurableMailboxException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) +class DurableMailboxException private[akka] (message: String, cause: Throwable) extends AkkaException(message, cause) { + def this(message: String) = this(message, null) +} /** * @author Jonas Bonér */ -abstract class DurableExecutableMailbox(owner: ActorRef) extends MessageQueue with ExecutableMailbox with DurableMailboxBase { +abstract class DurableExecutableMailbox(owner: LocalActorRef) extends MessageQueue with ExecutableMailbox with DurableMailboxBase { import DurableExecutableMailboxConfig._ val ownerAddress = owner.address @@ -58,19 +60,17 @@ abstract class DurableExecutableMailbox(owner: ActorRef) extends MessageQueue wi //TODO: switch to RemoteProtocol def deserialize(bytes: Array[Byte]) = { val durableMessage = DurableMailboxMessageProtocol.parseFrom(bytes) + + val owner = Actor.registry.actorFor(durableMessage.getOwnerAddress) match { + case Some(l: LocalActorRef) ⇒ l + case Some(a) ⇒ throw new DurableMailboxException("Recipient of message is not a LocalActorRef: " + a) + case None ⇒ throw new DurableMailboxException("No actor could be found for address [" + ownerAddress + "], could not deserialize message.") + } + val messageProtocol = MessageProtocol.parseFrom(durableMessage.getMessage) val message = MessageSerializer.deserialize(messageProtocol) - val ownerAddress = durableMessage.getOwnerAddress - val owner = Actor.registry.actorFor(ownerAddress).getOrElse( - throw new DurableMailboxException("No actor could be found for address [" + ownerAddress + "], could not deserialize message.")) - val senderOption = if (durableMessage.hasSenderAddress) { - Actor.registry.actorFor(durableMessage.getSenderAddress) - } else None - val sender = senderOption match { - case Some(ref) ⇒ ref - case None ⇒ NullChannel - } + val sender = if (durableMessage.hasSenderAddress) Actor.registry.actorFor(durableMessage.getSenderAddress).getOrElse(NullChannel) else NullChannel new MessageInvocation(owner, message, sender) } diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala index 6bece5925b..c79246966c 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala @@ -15,17 +15,11 @@ import akka.dispatch.MessageDispatcher object DurableMailboxSpecActorFactory { class MailboxTestActor extends Actor { - self.lifeCycle = Temporary - def receive = { - case "sum" ⇒ self.reply("sum") - } + def receive = { case "sum" ⇒ self.reply("sum") } } - def createMailboxTestActor(id: String)(implicit dispatcher: MessageDispatcher): ActorRef = { - val queueActor = localActorOf[MailboxTestActor] - queueActor.dispatcher = dispatcher - queueActor.start - } + def createMailboxTestActor(id: String)(implicit dispatcher: MessageDispatcher): ActorRef = + actorOf(Props[MailboxTestActor].withDispatcher(dispatcher).withLifeCycle(Temporary)) } abstract class DurableMailboxSpec(val backendName: String, val storage: DurableMailboxStorage) extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll { 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 63808addd5..a5fed22719 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 @@ -3,7 +3,7 @@ */ package akka.actor.mailbox -import akka.actor.{ Actor, ActorRef, NullChannel } +import akka.actor.{ Actor, ActorRef, LocalActorRef, NullChannel, Channel } import akka.config.Config.config import akka.dispatch._ import akka.event.EventHandler @@ -37,7 +37,7 @@ object BSONSerializableMailbox extends SerializableBSONObject[MongoDurableMessag msg.channel match { case a: ActorRef ⇒ { b += "senderAddress" -> a.address } - case _ ⇒ + case _ ⇒ () } /** * TODO - Figure out a way for custom serialization of the message instance @@ -72,24 +72,25 @@ object BSONSerializableMailbox extends SerializableBSONObject[MongoDurableMessag def decode(in: InputStream): MongoDurableMessage = { val deserializer = new DefaultBSONDeserializer // TODO - Skip the whole doc step for performance, fun, and profit! (Needs Salat / custom Deser) - val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument] + val doc: BSONDocument = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument] EventHandler.debug(this, "Deserializing a durable message from MongoDB: %s".format(doc)) - val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData) - val msg = MessageSerializer.deserialize(msgData) + val ownerAddress = doc.as[String]("ownerAddress") - val owner = Actor.registry.actorFor(ownerAddress).getOrElse( - throw new DurableMailboxException("No actor could be found for address [" + ownerAddress + "], could not deserialize message.")) - - val senderOption = if (doc.contains("senderAddress")) { - Actor.registry.actorFor(doc.as[String]("senderAddress")) - } else None - - val sender = senderOption match { - case Some(ref) ⇒ ref - case None ⇒ NullChannel + val owner = Actor.registry.actorFor(ownerAddress) match { + case Some(l: LocalActorRef) ⇒ l + case Some(a) ⇒ throw new DurableMailboxException("Recipient of message is not a LocalActorRef: " + a) + case None ⇒ throw new DurableMailboxException("No actor could be found for address [" + ownerAddress + "], could not deserialize message.") } - MongoDurableMessage(ownerAddress, owner, msg, sender) + val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData) + val msg = MessageSerializer.deserialize(msgData) + + val sender = if (doc.contains("senderAddress")) + Actor.registry.actorFor(doc.as[String]("senderAddress")) + else + None + + MongoDurableMessage(ownerAddress, owner, msg, sender.getOrElse(NullChannel)) } def checkObject(msg: MongoDurableMessage, isQuery: Boolean = false) = {} // object expected to be OK with this message type. 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 d5c31261ec..e8ce6957b3 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 @@ -3,7 +3,7 @@ */ package akka.actor.mailbox -import akka.actor.ActorRef +import akka.actor.{ ActorRef, LocalActorRef } import akka.config.Config.config import akka.dispatch._ import akka.event.EventHandler @@ -28,7 +28,7 @@ class MongoBasedMailboxException(message: String) extends AkkaException(message) * * @author Brendan W. McAdams */ -class MongoBasedNaiveMailbox(val owner: ActorRef) extends DurableExecutableMailbox(owner) { +class MongoBasedNaiveMailbox(val owner: LocalActorRef) extends DurableExecutableMailbox(owner) { // this implicit object provides the context for reading/writing things as MongoDurableMessage implicit val mailboxBSONSer = BSONSerializableMailbox implicit val safeWrite = WriteConcern.Safe // TODO - Replica Safe when appropriate! diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala index 7cdca7f8b0..93d4951fe7 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala @@ -3,7 +3,7 @@ */ package akka.actor.mailbox -import akka.actor.{ ActorRef, UntypedChannel, NullChannel } +import akka.actor.{ LocalActorRef, UntypedChannel, NullChannel } import akka.config.Config.config import akka.dispatch._ import akka.event.EventHandler @@ -33,7 +33,7 @@ import org.bson.collection._ * @author Brendan W. McAdams */ case class MongoDurableMessage(val ownerAddress: String, - val receiver: ActorRef, + val receiver: LocalActorRef, val message: Any, val channel: UntypedChannel, val _id: ObjectId = new ObjectId) { 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 3b4efac5e9..2753c4672b 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 @@ -3,7 +3,7 @@ */ package akka.actor.mailbox -import akka.actor.ActorRef +import akka.actor.LocalActorRef import akka.config.Config.config import akka.dispatch._ import akka.event.EventHandler @@ -18,7 +18,7 @@ class RedisBasedMailboxException(message: String) extends AkkaException(message) /** * @author Jonas Bonér */ -class RedisBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(owner) { +class RedisBasedMailbox(val owner: LocalActorRef) extends DurableExecutableMailbox(owner) { @volatile private var clients = connect() // returns a RedisClientPool for multiple asynchronous message handling 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 f924c21935..7936aedb88 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 @@ -5,7 +5,7 @@ package akka.actor.mailbox import MailboxProtocol._ -import akka.actor.ActorRef +import akka.actor.LocalActorRef import akka.dispatch._ import akka.config.Config._ import akka.event.EventHandler @@ -43,7 +43,7 @@ private[akka] object ZooKeeperMailboxConfig { /** * @author Jonas Bonér */ -class ZooKeeperBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(owner) { +class ZooKeeperBasedMailbox(val owner: LocalActorRef) extends DurableExecutableMailbox(owner) { import ZooKeeperMailboxConfig._ private val zkClient = new AkkaZkClient(zkServerAddresses, sessionTimeout, connectionTimeout) @@ -79,5 +79,5 @@ class ZooKeeperBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbo case e ⇒ false } - def close = zkClient.close + def close() = zkClient.close } diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala index 62ab97f750..9b970b46e7 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala @@ -1,6 +1,6 @@ package akka.actor.mailbox -import akka.actor.Actor +import akka.actor.{ Actor, LocalActorRef } import akka.cluster.zookeeper._ import org.I0Itec.zkclient._ @@ -17,10 +17,12 @@ class ZooKeeperBasedMailboxSpec extends DurableMailboxSpec("ZooKeeper", ZooKeepe } override def afterEach() { - Actor.registry.local.actors.foreach(_.mailbox match { - case zkm: ZooKeeperBasedMailbox ⇒ zkm.close - case _ ⇒ () - }) + Actor.registry.local.actors foreach { + case l: LocalActorRef ⇒ l.mailbox match { + case zk: ZooKeeperBasedMailbox ⇒ zk.close() + case _ ⇒ + } + } super.afterEach } diff --git a/akka-http/src/main/scala/akka/http/Mist.scala b/akka-http/src/main/scala/akka/http/Mist.scala index 532475b995..ef0ad699e4 100644 --- a/akka-http/src/main/scala/akka/http/Mist.scala +++ b/akka-http/src/main/scala/akka/http/Mist.scala @@ -4,7 +4,6 @@ package akka.http -import akka.actor.{ ActorRef, Actor } import akka.event.EventHandler import akka.config.ConfigurationException @@ -12,6 +11,7 @@ import javax.servlet.http.{ HttpServletResponse, HttpServletRequest } import javax.servlet.http.HttpServlet import javax.servlet.Filter import java.lang.UnsupportedOperationException +import akka.actor.{ NullChannel, ActorRef, Actor } /** * @author Garrick Evans @@ -205,7 +205,7 @@ object Endpoint { /** * leverage the akka config to tweak the dispatcher for our endpoints */ - val Dispatcher = Dispatchers.fromConfig("akka.http.mist-dispatcher") + lazy val Dispatcher = Dispatchers.fromConfig("akka.http.mist-dispatcher") type Hook = PartialFunction[String, ActorRef] @@ -258,9 +258,9 @@ trait Endpoint { this: Actor ⇒ if (!endpoints.isEmpty) endpoints.foreach { _.apply(uri) ! req } else { - self.sender match { - case Some(s) ⇒ s reply NoneAvailable(uri, req) - case None ⇒ _na(uri, req) + self.channel match { + case null | NullChannel ⇒ _na(uri, req) + case channel ⇒ channel ! NoneAvailable(uri, req) } } } @@ -280,9 +280,6 @@ class RootEndpoint extends Actor with Endpoint { final val Root = "/" - // use the configurable dispatcher - self.dispatcher = Endpoint.Dispatcher - override def preStart() = _attachments ::= { case `Root` ⇒ this.actor } diff --git a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala index c278efecc5..630b45e5cf 100644 --- a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala +++ b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala @@ -169,7 +169,7 @@ * Creates and links a MemoryChatStorage. */ trait MemoryChatStorageFactory { this: Actor => - val storage = this.self.startLink(actorOf[MemoryChatStorage]) // starts and links ChatStorage + val storage = this.self.link(actorOf[MemoryChatStorage]).start() // starts and links ChatStorage } /** diff --git a/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala b/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala index 289d4aa5df..40898b221d 100644 --- a/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala +++ b/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala @@ -34,8 +34,6 @@ object Logger { class Slf4jEventHandler extends Actor with Logging { import EventHandler._ - self.dispatcher = EventHandlerDispatcher - def receive = { case Error(cause, instance, message) ⇒ log.error("\n\t[{}]\n\t[{}]\n\t[{}]", diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala index 3c32bfa911..a07fd1cde3 100644 --- a/akka-stm/src/main/scala/akka/agent/Agent.scala +++ b/akka-stm/src/main/scala/akka/agent/Agent.scala @@ -5,9 +5,10 @@ package akka.agent import akka.stm._ -import akka.actor.Actor import akka.japi.{ Function ⇒ JFunc, Procedure ⇒ JProc } -import akka.dispatch.{ DefaultPromise, Dispatchers, Future } +import akka.dispatch.{ PinnedDispatcher, DefaultPromise, Dispatchers, Future } +import akka.actor.{ Props, LocalActorRef, Actor } +import akka.actor.Actor._ /** * Used internally to send functions. @@ -94,7 +95,7 @@ object Agent { */ class Agent[T](initialValue: T) { private[akka] val ref = Ref(initialValue) - private[akka] val updater = Actor.actorOf(new AgentUpdater(this)).start() + private[akka] val updater = actorOf(Props(new AgentUpdater(this))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow? /** * Read the internal state of the agent. @@ -151,7 +152,7 @@ class Agent[T](initialValue: T) { */ def sendOff(f: T ⇒ T): Unit = send((value: T) ⇒ { suspend - val threadBased = Actor.actorOf(new ThreadBasedAgentUpdater(this)).start() + val threadBased = actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(new PinnedDispatcher())) threadBased ! Update(f) value }) @@ -293,8 +294,6 @@ class AgentUpdater[T](agent: Agent[T]) extends Actor { * Thread-based agent updater actor. Used internally for `sendOff` actions. */ class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor { - self.dispatcher = Dispatchers.newPinnedDispatcher(self) - val txFactory = TransactionFactory(familyName = "ThreadBasedAgentUpdater", readonly = false) def receive = { diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index fc89966964..51fba43376 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -4,7 +4,6 @@ package akka.testkit import akka.event.EventHandler -import akka.actor.ActorRef import akka.dispatch.{ MessageDispatcher, MessageInvocation, TaskInvocation, Promise, ActorPromise } import java.util.concurrent.locks.ReentrantLock import java.util.LinkedList @@ -12,6 +11,7 @@ import java.util.concurrent.RejectedExecutionException import akka.util.Switch import java.lang.ref.WeakReference import scala.annotation.tailrec +import akka.actor.{ LocalActorRef, ActorRef } /* * Locking rules: @@ -107,9 +107,9 @@ object CallingThreadDispatcher { class CallingThreadDispatcher(val name: String = "calling-thread", val warnings: Boolean = true) extends MessageDispatcher { import CallingThreadDispatcher._ - private[akka] override def createMailbox(actor: ActorRef) = new CallingThreadMailbox + private[akka] override def createMailbox(actor: LocalActorRef) = new CallingThreadMailbox - private def getMailbox(actor: ActorRef) = actor.mailbox.asInstanceOf[CallingThreadMailbox] + private def getMailbox(actor: LocalActorRef) = actor.mailbox.asInstanceOf[CallingThreadMailbox] private[akka] override def start() {} @@ -117,11 +117,11 @@ class CallingThreadDispatcher(val name: String = "calling-thread", val warnings: private[akka] override def timeoutMs = 100L - override def suspend(actor: ActorRef) { + override def suspend(actor: LocalActorRef) { getMailbox(actor).suspended.switchOn } - override def resume(actor: ActorRef) { + override def resume(actor: LocalActorRef) { val mbox = getMailbox(actor) val queue = mbox.queue val wasActive = queue.isActive @@ -133,9 +133,9 @@ class CallingThreadDispatcher(val name: String = "calling-thread", val warnings: } } - override def mailboxSize(actor: ActorRef) = getMailbox(actor).queue.size + override def mailboxSize(actor: LocalActorRef) = getMailbox(actor).queue.size - def mailboxIsEmpty(actorRef: ActorRef): Boolean = getMailbox(actorRef).queue.isEmpty + override def mailboxIsEmpty(actorRef: LocalActorRef): Boolean = getMailbox(actorRef).queue.isEmpty private[akka] override def dispatch(handle: MessageInvocation) { val mbox = getMailbox(handle.receiver) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 116450665a..194af5f613 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -9,6 +9,7 @@ import akka.util.ReflectiveAccess import akka.event.EventHandler import com.eaio.uuid.UUID +import akka.actor.Props._ /** * This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it @@ -18,24 +19,20 @@ import com.eaio.uuid.UUID * @author Roland Kuhn * @since 1.1 */ -class TestActorRef[T <: Actor](factory: () ⇒ T, address: String) extends LocalActorRef(factory, address) { - - dispatcher = CallingThreadDispatcher.global - receiveTimeout = None - +class TestActorRef[T <: Actor](props: Props, address: String) extends LocalActorRef(props.withDispatcher(CallingThreadDispatcher.global), address) { /** * Directly inject messages into actor receive behavior. Any exceptions * thrown will be available to you, while still being able to use * become/unbecome and their message counterparts. */ - def apply(o: Any) { actor(o) } + def apply(o: Any) { actorInstance.get().apply(o) } /** * Retrieve reference to the underlying actor, where the static type matches the factory used inside the * constructor. Beware that this reference is discarded by the ActorRef upon restarting the actor (should this * reference be linked to a supervisor). The old Actor may of course still be used in post-mortem assertions. */ - def underlyingActor: T = actor.asInstanceOf[T] + def underlyingActor: T = actorInstance.get().asInstanceOf[T] override def toString = "TestActor[" + address + ":" + uuid + "]" @@ -49,9 +46,10 @@ class TestActorRef[T <: Actor](factory: () ⇒ T, address: String) extends Local * supervisor, but then you just asked for trouble. */ override def supervisor_=(a: Option[ActorRef]) { - for (ref ← a) { - if (!ref.dispatcher.isInstanceOf[CallingThreadDispatcher]) - EventHandler.warning(this, "supervisor " + ref + " does not use CallingThreadDispatcher") + a match { //TODO This should probably be removed since the Supervisor could be a remote actor for all we know + case Some(l: SelfActorRef) if !l.dispatcher.isInstanceOf[CallingThreadDispatcher] ⇒ + EventHandler.warning(this, "supervisor " + l + " does not use CallingThreadDispatcher") + case _ ⇒ } super.supervisor_=(a) } @@ -60,14 +58,17 @@ class TestActorRef[T <: Actor](factory: () ⇒ T, address: String) extends Local object TestActorRef { - def apply[T <: Actor](factory: ⇒ T): TestActorRef[T] = apply[T](factory, new UUID().toString) + def apply[T <: Actor](factory: ⇒ T): TestActorRef[T] = apply[T](Props(factory), new UUID().toString) - def apply[T <: Actor](factory: ⇒ T, address: String): TestActorRef[T] = new TestActorRef(() ⇒ factory, address) + def apply[T <: Actor](factory: ⇒ T, address: String): TestActorRef[T] = apply[T](Props(factory), address) + + def apply[T <: Actor](props: Props): TestActorRef[T] = apply[T](props, new UUID().toString).start() + + def apply[T <: Actor](props: Props, address: String): TestActorRef[T] = new TestActorRef(props, address).start() def apply[T <: Actor: Manifest]: TestActorRef[T] = apply[T](new UUID().toString) - def apply[T <: Actor: Manifest](address: String): TestActorRef[T] = new TestActorRef[T]({ () ⇒ - + def apply[T <: Actor: Manifest](address: String): TestActorRef[T] = apply[T](Props({ import ReflectiveAccess.{ createInstance, noParams, noArgs } createInstance[T](manifest[T].erasure, noParams, noArgs) match { case Right(value) ⇒ value @@ -77,5 +78,5 @@ object TestActorRef { "\nif so put it outside the class/trait, f.e. in a companion object," + "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.", exception) } - }, address) + }), address) } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index 86e77896c3..783456e223 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -34,9 +34,9 @@ import com.eaio.uuid.UUID * @author Roland Kuhn * @since 1.2 */ -class TestFSMRef[S, D, T <: Actor](factory: () ⇒ T, address: String)(implicit ev: T <:< FSM[S, D]) extends TestActorRef(factory, address) { +class TestFSMRef[S, D, T <: Actor](props: Props, address: String)(implicit ev: T <:< FSM[S, D]) extends TestActorRef(props, address) { - private def fsm = underlyingActor + private def fsm: T = underlyingActor /** * Get current state name of this FSM. @@ -79,8 +79,8 @@ class TestFSMRef[S, D, T <: Actor](factory: () ⇒ T, address: String)(implicit object TestFSMRef { - def apply[S, D, T <: Actor](factory: ⇒ T)(implicit ev: T <:< FSM[S, D]): TestFSMRef[S, D, T] = new TestFSMRef(() ⇒ factory, new UUID().toString) + def apply[S, D, T <: Actor](factory: ⇒ T)(implicit ev: T <:< FSM[S, D]): TestFSMRef[S, D, T] = new TestFSMRef(Props(creator = () ⇒ factory), new UUID().toString) - def apply[S, D, T <: Actor](factory: ⇒ T, address: String)(implicit ev: T <:< FSM[S, D]): TestFSMRef[S, D, T] = new TestFSMRef(() ⇒ factory, address) + def apply[S, D, T <: Actor](factory: ⇒ T, address: String)(implicit ev: T <:< FSM[S, D]): TestFSMRef[S, D, T] = new TestFSMRef(Props(creator = () ⇒ factory), address) } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index 1dd1498ad2..bfa8ce6850 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -34,8 +34,6 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor with FSM[ import FSM._ import TestActor._ - self.dispatcher = CallingThreadDispatcher.global - startWith(0, None) when(0, stateTimeout = 5 seconds) { case Ev(SetTimeout(d)) ⇒ @@ -100,7 +98,7 @@ trait TestKitLight { * ActorRef of the test actor. Access is provided to enable e.g. * registration as message target. */ - val testActor = localActorOf(new TestActor(queue), "testActor" + TestKit.testActorId.incrementAndGet()).start() + val testActor = actorOf(Props(new TestActor(queue)).copy(dispatcher = CallingThreadDispatcher.global, localOnly = true), "testActor" + TestKit.testActorId.incrementAndGet()) /** * Implicit sender reference so that replies are possible for messages sent diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 0fab0a54b6..28d8c5e0cd 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -41,11 +41,11 @@ object TestActorRefSpec { def receiveT = { case "complexRequest" ⇒ { replyTo = self.channel - val worker = TestActorRef[WorkerActor].start() + val worker = TestActorRef(Props[WorkerActor]) worker ! "work" } case "complexRequest2" ⇒ - val worker = TestActorRef[WorkerActor].start() + val worker = TestActorRef(Props[WorkerActor]) worker ! self.channel case "workDone" ⇒ replyTo ! "complexReply" case "simpleRequest" ⇒ self.reply("simpleReply") @@ -109,10 +109,10 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac "support nested Actor creation" when { "used with TestActorRef" in { - val a = TestActorRef(new Actor { - val nested = TestActorRef(new Actor { def receive = { case _ ⇒ } }).start() + val a = TestActorRef(Props(new Actor { + val nested = TestActorRef(Props(self ⇒ { case _ ⇒ })) def receive = { case _ ⇒ self reply nested } - }).start() + })) a must not be (null) val nested = (a ? "any").as[ActorRef].get nested must not be (null) @@ -120,10 +120,10 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac } "used with ActorRef" in { - val a = TestActorRef(new Actor { - val nested = Actor.actorOf(new Actor { def receive = { case _ ⇒ } }).start() + val a = TestActorRef(Props(new Actor { + val nested = Actor.actorOf(Props(self ⇒ { case _ ⇒ })) def receive = { case _ ⇒ self reply nested } - }).start() + })) a must not be (null) val nested = (a ? "any").as[ActorRef].get nested must not be (null) @@ -133,8 +133,8 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac } "support reply via channel" in { - val serverRef = TestActorRef[ReplyActor].start() - val clientRef = TestActorRef(new SenderActor(serverRef)).start() + val serverRef = TestActorRef(Props[ReplyActor]) + val clientRef = TestActorRef(Props(new SenderActor(serverRef))) counter = 4 @@ -159,7 +159,7 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac "stop when sent a poison pill" in { filterEvents(EventFilter[ActorKilledException]) { - val a = TestActorRef[WorkerActor].start() + val a = TestActorRef(Props[WorkerActor]) intercept[ActorKilledException] { (a ? PoisonPill).get } @@ -173,17 +173,16 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac filterEvents(EventFilter[ActorKilledException]) { counter = 2 - val boss = TestActorRef(new TActor { - self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), Some(1000)) - val ref = TestActorRef(new TActor { + val boss = TestActorRef(Props(new TActor { + + val ref = TestActorRef(Props(new TActor { def receiveT = { case _ ⇒ } override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 } override def postRestart(reason: Throwable) { counter -= 1 } - }).start() - self.dispatcher = CallingThreadDispatcher.global - self link ref + }).withSupervisor(self)) + def receiveT = { case "sendKill" ⇒ ref ! Kill } - }).start() + }).withFaultHandler(OneForOneStrategy(List(classOf[ActorKilledException]), 5, 1000))) boss ! "sendKill" diff --git a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java index 0442fabc45..b0f973d985 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java +++ b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java @@ -108,7 +108,7 @@ public class Pi { workers.add(worker); } - router = Routing.actorOfWithRoundRobin("pi", JavaConversions.asIterable(workers)); + router = Routing.actorOfWithRoundRobin("pi", JavaConversions.collectionAsScalaIterable(workers)); } // message handler diff --git a/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java b/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java index 2dc7da4399..98cd30353d 100644 --- a/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java +++ b/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java @@ -102,7 +102,7 @@ public class Pi { workers.add(worker); } - router = Routing.actorOfWithRoundRobin("pi", JavaConversions.asIterable(workers)); + router = Routing.actorOfWithRoundRobin("pi", JavaConversions.collectionAsScalaIterable(workers)); } @Override diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 12cc305c80..3ecd291749 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -237,7 +237,6 @@ object AkkaBuild extends Build { base = file("akka-samples/akka-sample-camel"), dependencies = Seq(actor, camelTyped, testkit % "test"), settings = defaultSettings ++ Seq( - ivyXML := Dependencies.sampleCamelXML, libraryDependencies ++= Dependencies.sampleCamel ) ) @@ -387,29 +386,16 @@ object Dependencies { val camel = Seq(camelCore, Test.junit, Test.scalatest, Test.logback) - val spring = Seq(springBeans, springContext, Test.camelSpring, Test.junit, Test.scalatest) + val spring = Seq(springBeans, springContext, camelSpring, Test.junit, Test.scalatest) val kernel = Seq( jettyUtil, jettyXml, jettyServlet, jerseyCore, jerseyJson, jerseyScala, jacksonCore, staxApi, Provided.jerseyServer ) - val sampleCamel = Seq(camelCore, commonsCodec, Runtime.activemq, Runtime.springJms, + // TODO: resolve Jetty version conflict + val sampleCamel = Seq(camelCore, camelSpring, commonsCodec, Runtime.camelJms, Runtime.activemq, Runtime.springJms, Test.junit, Test.scalatest, Test.logback) - - val sampleCamelXML = - - - - - - - - - - } object Dependency { @@ -417,8 +403,7 @@ object Dependency { // Versions object V { - val Camel = "2.7.1" - val CamelPatch = "2.7.1.1" + val Camel = "2.8.0" val Jackson = "1.8.0" val JavaxServlet = "3.0" val Jersey = "1.3" @@ -437,7 +422,8 @@ object Dependency { val beanstalk = "beanstalk" % "beanstalk_client" % "1.4.5" // New BSD val bookkeeper = "org.apache.hadoop.zookeeper" % "bookkeeper" % V.Zookeeper // ApacheV2 - val camelCore = "org.apache.camel" % "camel-core" % V.CamelPatch // ApacheV2 + val camelCore = "org.apache.camel" % "camel-core" % V.Camel // ApacheV2 + val camelSpring = "org.apache.camel" % "camel-spring" % V.Camel // ApacheV2 val commonsCodec = "commons-codec" % "commons-codec" % "1.4" // ApacheV2 val commonsIo = "commons-io" % "commons-io" % "2.0.1" // ApacheV2 val guice = "org.guiceyfruit" % "guice-all" % "2.0" // ApacheV2 @@ -481,7 +467,7 @@ object Dependency { object Runtime { val activemq = "org.apache.activemq" % "activemq-core" % "5.4.2" % "runtime" // ApacheV2 - val camelJetty = "org.apache.camel" % "camel-jetty" % V.CamelPatch % "runtime" // ApacheV2 + val camelJetty = "org.apache.camel" % "camel-jetty" % V.Camel % "runtime" // ApacheV2 val camelJms = "org.apache.camel" % "camel-jms" % V.Camel % "runtime" // ApacheV2 val logback = "ch.qos.logback" % "logback-classic" % V.Logback % "runtime" // MIT val springJms = "org.springframework" % "spring-jms" % V.Spring % "runtime" // ApacheV2 @@ -490,7 +476,6 @@ object Dependency { // Test object Test { - val camelSpring = "org.apache.camel" % "camel-spring" % V.Camel % "test" // ApacheV2 val commonsColl = "commons-collections" % "commons-collections" % "3.2.1" % "test" // ApacheV2 val commonsMath = "org.apache.commons" % "commons-math" % "2.1" % "test" // ApacheV2 val jetty = "org.eclipse.jetty" % "jetty-server" % V.Jetty % "test" // Eclipse license