diff --git a/akka-actor-tests/src/test/java/akka/actor/NonPublicClass.java b/akka-actor-tests/src/test/java/akka/actor/NonPublicClass.java new file mode 100644 index 0000000000..850d82cd62 --- /dev/null +++ b/akka-actor-tests/src/test/java/akka/actor/NonPublicClass.java @@ -0,0 +1,17 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.actor; + +public class NonPublicClass { + public static Props createProps() { + return new Props(MyNonPublicActorClass.class); + } +} + +class MyNonPublicActorClass extends UntypedActor { + @Override public void onReceive(Object msg) { + getSender().tell(msg); + } +} \ No newline at end of file diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorConfigurationVerificationSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorConfigurationVerificationSpec.scala new file mode 100644 index 0000000000..5752bd7806 --- /dev/null +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorConfigurationVerificationSpec.scala @@ -0,0 +1,77 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.actor + +import akka.testkit._ +import akka.testkit.DefaultTimeout +import akka.testkit.TestEvent._ +import akka.util.duration._ +import akka.routing._ +import org.scalatest.BeforeAndAfterEach +import akka.ConfigurationException + +object ActorConfigurationVerificationSpec { + + class TestActor extends Actor { + def receive: Receive = { + case _ ⇒ + } + } + + val config = """ + balancing-dispatcher { + type = BalancingDispatcher + throughput = 1 + } + pinned-dispatcher { + executor = "thread-pool-executor" + type = PinnedDispatcher + } + """ +} + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class ActorConfigurationVerificationSpec extends AkkaSpec(ActorConfigurationVerificationSpec.config) with DefaultTimeout with BeforeAndAfterEach { + import ActorConfigurationVerificationSpec._ + + override def atStartup { + system.eventStream.publish(Mute(EventFilter[ConfigurationException](""))) + } + + "An Actor configured with a BalancingDispatcher" must { + "fail verification with a ConfigurationException if also configured with a RoundRobinRouter" in { + intercept[ConfigurationException] { + system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(2).withDispatcher("balancing-dispatcher"))) + } + } + "fail verification with a ConfigurationException if also configured with a BroadcastRouter" in { + intercept[ConfigurationException] { + system.actorOf(Props[TestActor].withRouter(BroadcastRouter(2).withDispatcher("balancing-dispatcher"))) + } + } + "fail verification with a ConfigurationException if also configured with a RandomRouter" in { + intercept[ConfigurationException] { + system.actorOf(Props[TestActor].withRouter(RandomRouter(2).withDispatcher("balancing-dispatcher"))) + } + } + "fail verification with a ConfigurationException if also configured with a SmallestMailboxRouter" in { + intercept[ConfigurationException] { + system.actorOf(Props[TestActor].withRouter(SmallestMailboxRouter(2).withDispatcher("balancing-dispatcher"))) + } + } + "fail verification with a ConfigurationException if also configured with a ScatterGatherFirstCompletedRouter" in { + intercept[ConfigurationException] { + system.actorOf(Props[TestActor].withRouter(ScatterGatherFirstCompletedRouter(nrOfInstances = 2, within = 2 seconds).withDispatcher("balancing-dispatcher"))) + } + } + "not fail verification with a ConfigurationException also not configured with a Router" in { + system.actorOf(Props[TestActor].withDispatcher("balancing-dispatcher")) + } + } + "An Actor configured with a non-balancing dispatcher" must { + "not fail verification with a ConfigurationException if also configured with a Router" in { + system.actorOf(Props[TestActor].withDispatcher("pinned-dispatcher").withRouter(RoundRobinRouter(2))) + } + } +} diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala index 16b4055d0e..d87aaaaee6 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala @@ -13,6 +13,7 @@ import akka.util.duration._ import java.util.concurrent.atomic._ import akka.dispatch.Await import akka.pattern.ask +import java.util.UUID.{ randomUUID ⇒ newUuid } object ActorLifeCycleSpec { @@ -35,7 +36,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "invoke preRestart, preStart, postRestart when using OneForOneStrategy" in { filterException[ActorKilledException] { - val id = newUuid().toString + val id = newUuid.toString val supervisor = system.actorOf(Props(new Supervisor( OneForOneStrategy(maxNrOfRetries = 3)(List(classOf[Exception]))))) val gen = new AtomicInteger(0) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index e8c667bc7e..a003d25757 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -227,7 +227,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { contextStackMustBeEmpty } - filterException[java.lang.IllegalStateException] { + EventFilter[ActorInitializationException](occurrences = 1) intercept { (intercept[java.lang.IllegalStateException] { wrap(result ⇒ actorOf(Props(new OuterActor(actorOf(Props(promiseIntercept({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor })(result))))))) @@ -257,14 +257,14 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { val in = new ObjectInputStream(new ByteArrayInputStream(bytes)) val readA = in.readObject - a.isInstanceOf[LocalActorRef] must be === true - readA.isInstanceOf[LocalActorRef] must be === true + a.isInstanceOf[ActorRefWithCell] must be === true + readA.isInstanceOf[ActorRefWithCell] must be === true (readA eq a) must be === true } val ser = new JavaSerializer(esys) val readA = ser.fromBinary(bytes, None) - readA.isInstanceOf[LocalActorRef] must be === true + readA.isInstanceOf[ActorRefWithCell] must be === true (readA eq a) must be === true } @@ -358,17 +358,24 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { system.stop(serverRef) } + "support actorOfs where the class of the actor isn't public" in { + val a = system.actorOf(NonPublicClass.createProps()) + a.tell("pigdog", testActor) + expectMsg("pigdog") + system stop a + } + "stop when sent a poison pill" in { val timeout = Timeout(20000) val ref = system.actorOf(Props(new Actor { def receive = { - case 5 ⇒ sender.tell("five") - case null ⇒ sender.tell("null") + case 5 ⇒ sender.tell("five") + case 0 ⇒ sender.tell("null") } })) val ffive = (ref.ask(5)(timeout)).mapTo[String] - val fnull = (ref.ask(null)(timeout)).mapTo[String] + val fnull = (ref.ask(0)(timeout)).mapTo[String] ref ! PoisonPill Await.result(ffive, timeout.duration) must be("five") @@ -393,7 +400,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { override def postRestart(reason: Throwable) = latch.countDown() })) - protected def receive = { case "sendKill" ⇒ ref ! Kill } + def receive = { case "sendKill" ⇒ ref ! Kill } })) boss ! "sendKill" diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index 7ae79fea34..1a2d64bb41 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -10,6 +10,9 @@ import akka.dispatch.Await import akka.util.duration._ import scala.collection.JavaConverters import java.util.concurrent.{ TimeUnit, RejectedExecutionException, CountDownLatch, ConcurrentLinkedQueue } +import akka.pattern.ask +import akka.util.Timeout +import akka.dispatch.Future class JavaExtensionSpec extends JavaExtension with JUnitSuite @@ -21,8 +24,46 @@ object TestExtension extends ExtensionId[TestExtension] with ExtensionIdProvider // Dont't place inside ActorSystemSpec object, since it will not be garbage collected and reference to system remains class TestExtension(val system: ExtendedActorSystem) extends Extension +object ActorSystemSpec { + + class Waves extends Actor { + var master: ActorRef = _ + var terminaters = Set[ActorRef]() + + def receive = { + case n: Int ⇒ + master = sender + terminaters = Set() ++ (for (i ← 1 to n) yield { + val man = context.watch(context.system.actorOf(Props[Terminater])) + man ! "run" + man + }) + case Terminated(child) if terminaters contains child ⇒ + terminaters -= child + if (terminaters.isEmpty) { + master ! "done" + context stop self + } + } + + override def preRestart(cause: Throwable, msg: Option[Any]) { + if (master ne null) { + master ! "failed with " + cause + " while processing " + msg + } + context stop self + } + } + + class Terminater extends Actor { + def receive = { + case "run" ⇒ context.stop(self) + } + } + +} + @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension$"]""") { +class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension$"]""") with ImplicitSender { "An ActorSystem" must { @@ -112,6 +153,35 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt }.getMessage must be("Must be called prior to system shutdown.") } + "reliably create waves of actors" in { + import system.dispatcher + implicit val timeout = Timeout(30 seconds) + val waves = for (i ← 1 to 3) yield system.actorOf(Props[ActorSystemSpec.Waves]) ? 50000 + Await.result(Future.sequence(waves), timeout.duration + 5.seconds) must be === Seq("done", "done", "done") + } + + "reliable deny creation of actors while shutting down" in { + val system = ActorSystem() + system.scheduler.scheduleOnce(200 millis) { system.shutdown() } + var failing = false + var created = Vector.empty[ActorRef] + while (!system.isTerminated && system.uptime < 5) { + try { + val t = system.actorOf(Props[ActorSystemSpec.Terminater]) + failing must not be true // because once failing => always failing (it’s due to shutdown) + created :+= t + } catch { + case _: IllegalStateException ⇒ failing = true + } + } + if (system.uptime >= 5) { + println(created.last) + println(system.asInstanceOf[ExtendedActorSystem].printTree) + system.uptime must be < 5L + } + created filter (ref ⇒ !ref.isTerminated && !ref.asInstanceOf[ActorRefWithCell].underlying.isInstanceOf[UnstartedCell]) must be(Seq()) + } + } -} \ No newline at end of file +} diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorWithBoundedStashSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorWithBoundedStashSpec.scala index 9d411268aa..33283b18cf 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorWithBoundedStashSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorWithBoundedStashSpec.scala @@ -6,7 +6,7 @@ package akka.actor import akka.testkit._ import akka.testkit.DefaultTimeout import akka.testkit.TestEvent._ -import akka.dispatch.{ Await, MessageQueueAppendFailedException, BoundedDequeBasedMailbox } +import akka.dispatch.{ Await, BoundedDequeBasedMailbox } import akka.pattern.ask import akka.util.duration._ import akka.actor.ActorSystem.Settings @@ -17,16 +17,8 @@ object ActorWithBoundedStashSpec { class StashingActor(implicit sys: ActorSystem) extends Actor with Stash { def receive = { - case "hello" ⇒ - stash() - sender ! "OK" - case "world" ⇒ - try { - unstashAll() - } catch { - case e: MessageQueueAppendFailedException ⇒ - expectedException.open() - } + case "hello" ⇒ stash() + case "world" ⇒ unstashAll() } } @@ -36,18 +28,10 @@ object ActorWithBoundedStashSpec { def receive = { case "hello" ⇒ numStashed += 1 - try { - stash() - } catch { - case e: StashOverflowException ⇒ - if (numStashed == 21) stashOverflow.open() - } + try stash() catch { case e: StashOverflowException ⇒ if (numStashed == 21) sender ! "STASHOVERFLOW" } } } - @volatile var expectedException: TestLatch = null - @volatile var stashOverflow: TestLatch = null - val testConf: Config = ConfigFactory.parseString(""" my-dispatcher { mailbox-type = "akka.actor.ActorWithBoundedStashSpec$Bounded" @@ -56,47 +40,42 @@ object ActorWithBoundedStashSpec { """) // bounded deque-based mailbox with capacity 10 - class Bounded(settings: Settings, config: Config) extends BoundedDequeBasedMailbox(10, 5 seconds) - + class Bounded(settings: Settings, config: Config) extends BoundedDequeBasedMailbox(10, 1 seconds) } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class ActorWithBoundedStashSpec extends AkkaSpec(ActorWithBoundedStashSpec.testConf) with DefaultTimeout with BeforeAndAfterEach { +class ActorWithBoundedStashSpec extends AkkaSpec(ActorWithBoundedStashSpec.testConf) with DefaultTimeout with BeforeAndAfterEach with ImplicitSender { import ActorWithBoundedStashSpec._ implicit val sys = system - override def atStartup { - system.eventStream.publish(Mute(EventFilter[Exception]("Crashing..."))) - } + override def atStartup { system.eventStream.publish(Mute(EventFilter[Exception]("Crashing..."))) } def myProps(creator: ⇒ Actor): Props = Props(creator).withDispatcher("my-dispatcher") "An Actor with Stash and BoundedDequeBasedMailbox" must { - "throw a MessageQueueAppendFailedException in case of a capacity violation" in { - ActorWithBoundedStashSpec.expectedException = new TestLatch + "end up in DeadLetters in case of a capacity violation" in { + system.eventStream.subscribe(testActor, classOf[DeadLetter]) + val stasher = system.actorOf(myProps(new StashingActor)) // fill up stash - val futures = for (_ ← 1 to 11) yield { stasher ? "hello" } - futures foreach { Await.ready(_, 10 seconds) } + (1 to 11) foreach { _ ⇒ stasher ! "hello" } // cause unstashAll with capacity violation stasher ! "world" - Await.ready(ActorWithBoundedStashSpec.expectedException, 10 seconds) + expectMsg(DeadLetter("hello", testActor, stasher)) + system.eventStream.unsubscribe(testActor, classOf[DeadLetter]) } - } "An Actor with bounded Stash" must { "throw a StashOverflowException in case of a stash capacity violation" in { - ActorWithBoundedStashSpec.stashOverflow = new TestLatch val stasher = system.actorOf(myProps(new StashingActorWithOverflow)) // fill up stash - for (_ ← 1 to 21) { stasher ! "hello" } - Await.ready(ActorWithBoundedStashSpec.stashOverflow, 10 seconds) + (1 to 21) foreach { _ ⇒ stasher ! "hello" } + expectMsg("STASHOVERFLOW") } - } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorWithStashSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorWithStashSpec.scala index c516a17a42..524913b01d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorWithStashSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorWithStashSpec.scala @@ -131,7 +131,7 @@ class ActorWithStashSpec extends AkkaSpec(ActorWithStashSpec.testConf) with Defa val hasMsgLatch = new TestLatch val slaveProps = myProps(new Actor with Stash { - protected def receive = { + def receive = { case "crash" ⇒ throw new Exception("Crashing...") diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 7a1aa35485..8a21f5f070 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -36,7 +36,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout "notify with one Terminated message when an Actor is stopped" in { val terminal = system.actorOf(Props.empty) startWatching(terminal) ! "hallo" - expectMsg("hallo") // this ensures that the DaemonMsgWatch has been received before we send the PoisonPill + expectMsg("hallo") terminal ! PoisonPill @@ -143,6 +143,26 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout result must be(Seq(1, 2, 3)) } } + + "be able to watch a child with the same name after the old died" in { + val parent = system.actorOf(Props(new Actor { + def receive = { + case "NKOTB" ⇒ + val currentKid = context.watch(context.actorOf(Props(ctx ⇒ { case "NKOTB" ⇒ ctx stop ctx.self }), "kid")) + currentKid forward "NKOTB" + context become { + case Terminated(`currentKid`) ⇒ + testActor ! "GREEN" + context unbecome + } + } + })) + + parent ! "NKOTB" + expectMsg("GREEN") + parent ! "NKOTB" + expectMsg("GREEN") + } } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index ef49cbc18d..cc98a23f1f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -147,7 +147,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im object Hello object Bye val tester = system.actorOf(Props(new Actor { - protected def receive = { + def receive = { case Hello ⇒ lock ! "hello" case "world" ⇒ answerLatch.open case Bye ⇒ lock ! "bye" diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index df47c801bb..76d8df1e92 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -140,13 +140,13 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { object FSMTimingSpec { def suspend(actorRef: ActorRef): Unit = actorRef match { - case l: LocalActorRef ⇒ l.suspend() - case _ ⇒ + case l: ActorRefWithCell ⇒ l.suspend() + case _ ⇒ } def resume(actorRef: ActorRef): Unit = actorRef match { - case l: LocalActorRef ⇒ l.resume() - case _ ⇒ + case l: ActorRefWithCell ⇒ l.resume() + case _ ⇒ } trait State diff --git a/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala index 236d3bd014..120caa3e93 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala @@ -6,10 +6,60 @@ package akka.actor import akka.testkit._ +object HotSwapSpec { + abstract class Becomer extends Actor { + + } +} + @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class HotSwapSpec extends AkkaSpec with ImplicitSender { + import HotSwapSpec.Becomer "An Actor" must { + "be able to become in its constructor" in { + val a = system.actorOf(Props(new Becomer { + context.become { case always ⇒ sender ! always } + def receive = { case always ⇒ sender ! "FAILURE" } + })) + a ! "pigdog" + expectMsg("pigdog") + } + + "be able to become multiple times in its constructor" in { + val a = system.actorOf(Props(new Becomer { + for (i ← 1 to 4) context.become({ case always ⇒ sender ! i + ":" + always }) + def receive = { case always ⇒ sender ! "FAILURE" } + })) + a ! "pigdog" + expectMsg("4:pigdog") + } + + "be able to become with stacking in its constructor" in { + val a = system.actorOf(Props(new Becomer { + context.become({ case always ⇒ sender ! "pigdog:" + always; context.unbecome() }, false) + def receive = { case always ⇒ sender ! "badass:" + always } + })) + a ! "pigdog" + expectMsg("pigdog:pigdog") + a ! "badass" + expectMsg("badass:badass") + } + + "be able to become, with stacking, multiple times in its constructor" in { + val a = system.actorOf(Props(new Becomer { + for (i ← 1 to 4) context.become({ case always ⇒ sender ! i + ":" + always; context.unbecome() }, false) + def receive = { case always ⇒ sender ! "FAILURE" } + })) + a ! "pigdog" + a ! "pigdog" + a ! "pigdog" + a ! "pigdog" + expectMsg("4:pigdog") + expectMsg("3:pigdog") + expectMsg("2:pigdog") + expectMsg("1:pigdog") + } "be able to hotswap its behavior with become(..)" in { val a = system.actorOf(Props(new Actor { @@ -30,13 +80,10 @@ class HotSwapSpec extends AkkaSpec with ImplicitSender { val a = system.actorOf(Props(new Actor { def receive = { case "init" ⇒ sender ! "init" - case "swap" ⇒ - context.become({ - case "swapped" ⇒ - sender ! "swapped" - case "revert" ⇒ - context.unbecome() - }) + case "swap" ⇒ context.become({ + case "swapped" ⇒ sender ! "swapped" + case "revert" ⇒ context.unbecome() + }) } })) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala index dc08df1c98..09fe9c103f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala @@ -22,7 +22,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { val timeoutActor = system.actorOf(Props(new Actor { context.setReceiveTimeout(500 milliseconds) - protected def receive = { + def receive = { case ReceiveTimeout ⇒ timeoutLatch.open } })) @@ -38,7 +38,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { val timeoutActor = system.actorOf(Props(new Actor { context.setReceiveTimeout(500 milliseconds) - protected def receive = { + def receive = { case Tick ⇒ () case ReceiveTimeout ⇒ timeoutLatch.open } @@ -58,7 +58,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { val timeoutActor = system.actorOf(Props(new Actor { context.setReceiveTimeout(500 milliseconds) - protected def receive = { + def receive = { case Tick ⇒ () case ReceiveTimeout ⇒ count.incrementAndGet @@ -78,7 +78,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { val timeoutLatch = TestLatch() val timeoutActor = system.actorOf(Props(new Actor { - protected def receive = { + def receive = { case ReceiveTimeout ⇒ timeoutLatch.open } })) diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index 829ab081e0..8d114bc396 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -40,7 +40,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { val slaveProps = Props(new Actor { - protected def receive = { + def receive = { case Ping ⇒ countDownLatch.countDown() case Crash ⇒ throw new Exception("Crashing...") } @@ -83,7 +83,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { val slaveProps = Props(new Actor { - protected def receive = { + def receive = { case Crash ⇒ throw new Exception("Crashing...") } @@ -110,7 +110,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { val slaveProps = Props(new Actor { - protected def receive = { + def receive = { case Ping ⇒ if (!pingLatch.isOpen) pingLatch.open else secondPingLatch.open case Crash ⇒ throw new Exception("Crashing...") @@ -166,7 +166,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { val slaveProps = Props(new Actor { - protected def receive = { + def receive = { case Ping ⇒ countDownLatch.countDown() case Crash ⇒ throw new Exception("Crashing...") } @@ -221,7 +221,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { val slaveProps = Props(new Actor { - protected def receive = { + def receive = { case Ping ⇒ countDownLatch.countDown() case Crash ⇒ throw new Exception("Crashing...") } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index 3b87af2aad..c67bcb44af 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -18,7 +18,12 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout } override def afterEach { - while (cancellables.peek() ne null) { Option(cancellables.poll()).foreach(_.cancel()) } + while (cancellables.peek() ne null) { + for (c ← Option(cancellables.poll())) { + c.cancel() + c.isCancelled must be === true + } + } } "A Scheduler" must { @@ -43,7 +48,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout assert(countDownLatch2.await(2, TimeUnit.SECONDS)) } - "should stop continuous scheduling if the receiving actor has been terminated" taggedAs TimingTest in { + "stop continuous scheduling if the receiving actor has been terminated" taggedAs TimingTest in { val actor = system.actorOf(Props(new Actor { def receive = { case x ⇒ testActor ! x diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index a04e83f39b..62752d8052 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -20,7 +20,7 @@ object SupervisorHierarchySpec { */ class CountDownActor(countDown: CountDownLatch, override val supervisorStrategy: SupervisorStrategy) extends Actor { - protected def receive = { + def receive = { case p: Props ⇒ sender ! context.actorOf(p) } // test relies on keeping children around during restart @@ -67,7 +67,7 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout { val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, SupervisorStrategy.defaultStrategy)))) - protected def receive = { + def receive = { case "killCrasher" ⇒ crasher ! Kill case Terminated(_) ⇒ countDownMax.countDown() } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index 92af540a9a..197e749d2e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -37,7 +37,7 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul val workerProps = Props(new Actor { override def postRestart(cause: Throwable) { countDownLatch.countDown() } - protected def receive = { + def receive = { case "status" ⇒ this.sender ! "OK" case _ ⇒ this.context.stop(self) } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index d295e6db4f..3db5b5b5dc 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -339,9 +339,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende OneForOneStrategy(maxNrOfRetries = 3, withinTimeRange = 10 seconds)(classOf[Exception] :: Nil)))) val dyingProps = Props(new Actor { - inits.incrementAndGet - - if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!") + if (inits.incrementAndGet % 2 == 0) throw new IllegalStateException("Don't wanna!") def receive = { case Ping ⇒ sender ! PongMessage @@ -366,5 +364,39 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende system.stop(supervisor) } + + "must not lose system messages when a NonFatal exception occurs when processing a system message" in { + val parent = system.actorOf(Props(new Actor { + override val supervisorStrategy = OneForOneStrategy()({ + case e: IllegalStateException if e.getMessage == "OHNOES" ⇒ throw e + case _ ⇒ SupervisorStrategy.Restart + }) + val child = context.watch(context.actorOf(Props(new Actor { + override def postRestart(reason: Throwable): Unit = testActor ! "child restarted" + def receive = { + case "die" ⇒ throw new IllegalStateException("OHNOES") + case "test" ⇒ sender ! "child green" + } + }), "child")) + + override def postRestart(reason: Throwable): Unit = testActor ! "parent restarted" + + def receive = { + case t @ Terminated(`child`) ⇒ testActor ! "child terminated" + case "die" ⇒ child ! "die" + case "test" ⇒ sender ! "green" + case "testchild" ⇒ child forward "test" + } + })) + + parent ! "die" + parent ! "testchild" + expectMsg("parent restarted") + expectMsg("child terminated") + parent ! "test" + expectMsg("green") + parent ! "testchild" + expectMsg("child green") + } } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index 502712872a..b7a5a8f64b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -307,7 +307,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) "be able to call methods returning Scala Options" in { val t = newFooBar(Duration(500, "ms")) t.optionPigdog(200).get must be("Pigdog") - t.optionPigdog(700) must be(None) + t.optionPigdog(1000) must be(None) mustStop(t) } diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index 88358e9f16..da789d9dce 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -3,24 +3,23 @@ */ package akka.actor.dispatch -import org.scalatest.Assertions._ -import akka.testkit._ -import akka.dispatch._ -import akka.util.Timeout -import java.util.concurrent.atomic.AtomicLong -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.{ ConcurrentHashMap, CountDownLatch, TimeUnit } -import akka.util.Switch import java.rmi.RemoteException -import org.junit.{ After, Test } -import akka.actor._ -import util.control.NoStackTrace -import akka.actor.ActorSystem -import akka.util.duration._ -import akka.event.Logging.Error +import java.util.concurrent.{ TimeUnit, CountDownLatch, ConcurrentHashMap } +import java.util.concurrent.atomic.{ AtomicLong, AtomicInteger } + +import org.junit.runner.RunWith +import org.scalatest.Assertions.{ fail, assert } +import org.scalatest.junit.JUnitRunner + import com.typesafe.config.Config -import akka.util.Duration + +import akka.actor._ +import akka.dispatch._ +import akka.event.Logging.Error import akka.pattern.ask +import akka.testkit._ +import akka.util.{ Timeout, Switch, Duration } +import akka.util.duration._ object ActorModelSpec { @@ -201,7 +200,7 @@ object ActorModelSpec { msgsReceived: Long = statsFor(actorRef, dispatcher).msgsReceived.get(), msgsProcessed: Long = statsFor(actorRef, dispatcher).msgsProcessed.get(), restarts: Long = statsFor(actorRef, dispatcher).restarts.get())(implicit system: ActorSystem) { - val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher)) + val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].dispatcher)) val deadline = System.currentTimeMillis + 1000 try { await(deadline)(stats.suspensions.get() == suspensions) @@ -241,6 +240,13 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa def newTestActor(dispatcher: String) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher)) + def awaitStarted(ref: ActorRef): Unit = { + awaitCond(ref match { + case r: RepointableRef ⇒ r.isStarted + case _ ⇒ true + }, 1 second, 10 millis) + } + protected def interceptedDispatcher(): MessageDispatcherInterceptor protected def dispatcherType: String @@ -280,6 +286,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa implicit val dispatcher = interceptedDispatcher() val start, oneAtATime = new CountDownLatch(1) val a = newTestActor(dispatcher.id) + awaitStarted(a) a ! CountDown(start) assertCountDown(start, 3.seconds.dilated.toMillis, "Should process first message within 3 seconds") @@ -328,7 +335,8 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa "not process messages for a suspended actor" in { implicit val dispatcher = interceptedDispatcher() - val a = newTestActor(dispatcher.id).asInstanceOf[LocalActorRef] + val a = newTestActor(dispatcher.id).asInstanceOf[InternalActorRef] + awaitStarted(a) val done = new CountDownLatch(1) a.suspend a ! CountDown(done) @@ -374,7 +382,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa def compare(l: AnyRef, r: AnyRef) = (l, r) match { case (ll: ActorCell, rr: ActorCell) ⇒ ll.self.path compareTo rr.self.path } } foreach { case cell: ActorCell ⇒ - System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain())) + System.err.println(" - " + cell.self.path + " " + cell.isTerminated + " " + cell.mailbox.status + " " + cell.mailbox.numberOfMessages + " " + SystemMessage.size(cell.mailbox.systemDrain(null))) } System.err.println("Mailbox: " + mq.numberOfMessages + " " + mq.hasMessages) @@ -400,17 +408,17 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa val a = newTestActor(dispatcher.id) val f1 = a ? Reply("foo") val f2 = a ? Reply("bar") - val f3 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(ActorInterruptedException(ie)) } + val f3 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(new ActorInterruptedException(ie)) } val f4 = a ? Reply("foo2") - val f5 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(ActorInterruptedException(ie)) } + val f5 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(new ActorInterruptedException(ie)) } val f6 = a ? Reply("bar2") assert(Await.result(f1, timeout.duration) === "foo") assert(Await.result(f2, timeout.duration) === "bar") assert(Await.result(f4, timeout.duration) === "foo2") - assert(intercept[ActorInterruptedException](Await.result(f3, timeout.duration)).getMessage === "Ping!") + assert(intercept[ActorInterruptedException](Await.result(f3, timeout.duration)).getCause.getMessage === "Ping!") assert(Await.result(f6, timeout.duration) === "bar2") - assert(intercept[ActorInterruptedException](Await.result(f5, timeout.duration)).getMessage === "Ping!") + assert(intercept[ActorInterruptedException](Await.result(f5, timeout.duration)).getCause.getMessage === "Ping!") } } @@ -436,6 +444,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa "not double-deregister" in { implicit val dispatcher = interceptedDispatcher() + for (i ← 1 to 1000) system.actorOf(Props.empty) val a = newTestActor(dispatcher.id) a ! DoubleStop awaitCond(statsFor(a, dispatcher).registers.get == 1) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala index 4060587b73..1a5c7e8661 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala @@ -1,8 +1,12 @@ package akka.actor.dispatch import java.util.concurrent.{ TimeUnit, CountDownLatch } -import akka.dispatch.{ Mailbox, Dispatchers } -import akka.actor.{ LocalActorRef, IllegalActorStateException, Actor, Props } + +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner + +import akka.actor.{ Props, ActorRefWithCell, ActorCell, Actor } +import akka.dispatch.Mailbox import akka.testkit.AkkaSpec object BalancingDispatcherSpec { @@ -51,8 +55,8 @@ class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) { "have fast actor stealing work from slow actor" in { val finishedCounter = new CountDownLatch(110) - val slow = system.actorOf(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef] - val fast = system.actorOf(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef] + val slow = system.actorOf(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[ActorRefWithCell] + val fast = system.actorOf(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[ActorRefWithCell] var sentToFast = 0 @@ -76,11 +80,11 @@ class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) { } finishedCounter.await(5, TimeUnit.SECONDS) - fast.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false) - slow.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false) - fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast - fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be > - (slow.underlying.actor.asInstanceOf[DelayableActor].invocationCount) + fast.underlying.asInstanceOf[ActorCell].mailbox.asInstanceOf[Mailbox].hasMessages must be(false) + slow.underlying.asInstanceOf[ActorCell].mailbox.asInstanceOf[Mailbox].hasMessages must be(false) + fast.underlying.asInstanceOf[ActorCell].actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast + fast.underlying.asInstanceOf[ActorCell].actor.asInstanceOf[DelayableActor].invocationCount must be > + (slow.underlying.asInstanceOf[ActorCell].actor.asInstanceOf[DelayableActor].invocationCount) system.stop(slow) system.stop(fast) } 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 8759f1aad9..ba025ffe3c 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala @@ -1,14 +1,17 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ package akka.dispatch -import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } -import java.util.concurrent.{ TimeUnit, BlockingQueue } -import java.util.concurrent.ConcurrentLinkedQueue -import akka.util._ -import akka.util.duration._ -import akka.testkit.AkkaSpec -import akka.actor.{ ActorRef, ActorContext, Props, LocalActorRef } +import java.util.concurrent.{ ConcurrentLinkedQueue, BlockingQueue } + +import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll } + import com.typesafe.config.Config -import akka.actor.ActorSystem + +import akka.actor.{ RepointableRef, Props, DeadLetter, ActorSystem, ActorRefWithCell, ActorRef, ActorCell } +import akka.testkit.AkkaSpec +import akka.util.duration.intToDurationInt @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach { @@ -39,9 +42,10 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn q.numberOfMessages must be === config.capacity q.hasMessages must be === true - intercept[MessageQueueAppendFailedException] { - q.enqueue(null, exampleMessage) - } + system.eventStream.subscribe(testActor, classOf[DeadLetter]) + q.enqueue(testActor, exampleMessage) + expectMsg(DeadLetter(exampleMessage.message, system.deadLetters, testActor)) + system.eventStream.unsubscribe(testActor, classOf[DeadLetter]) q.dequeue must be === exampleMessage q.numberOfMessages must be(config.capacity - 1) @@ -75,7 +79,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn result } - def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters)(system) + def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters, system) def ensureInitialMailboxState(config: MailboxType, q: MessageQueue) { q must not be null @@ -136,8 +140,8 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn class DefaultMailboxSpec extends MailboxSpec { lazy val name = "The default mailbox implementation" def factory = { - case u: UnboundedMailbox ⇒ u.create(None) - case b: BoundedMailbox ⇒ b.create(None) + case u: UnboundedMailbox ⇒ u.create(None, None) + case b: BoundedMailbox ⇒ b.create(None, None) } } @@ -145,8 +149,8 @@ class PriorityMailboxSpec extends MailboxSpec { val comparator = PriorityGenerator(_.##) lazy val name = "The priority mailbox implementation" def factory = { - case UnboundedMailbox() ⇒ new UnboundedPriorityMailbox(comparator).create(None) - case BoundedMailbox(capacity, pushTimeOut) ⇒ new BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(None) + case UnboundedMailbox() ⇒ new UnboundedPriorityMailbox(comparator).create(None, None) + case BoundedMailbox(capacity, pushTimeOut) ⇒ new BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(None, None) } } @@ -158,13 +162,13 @@ object CustomMailboxSpec { """ class MyMailboxType(settings: ActorSystem.Settings, config: Config) extends MailboxType { - override def create(owner: Option[ActorContext]) = owner match { + override def create(owner: Option[ActorRef], system: Option[ActorSystem]) = owner match { case Some(o) ⇒ new MyMailbox(o) case None ⇒ throw new Exception("no mailbox owner given") } } - class MyMailbox(owner: ActorContext) extends QueueBasedMessageQueue with UnboundedMessageQueueSemantics { + class MyMailbox(owner: ActorRef) extends QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final val queue = new ConcurrentLinkedQueue[Envelope]() } } @@ -174,7 +178,11 @@ class CustomMailboxSpec extends AkkaSpec(CustomMailboxSpec.config) { "Dispatcher configuration" must { "support custom mailboxType" in { val actor = system.actorOf(Props.empty.withDispatcher("my-dispatcher")) - val queue = actor.asInstanceOf[LocalActorRef].underlying.mailbox.messageQueue + awaitCond(actor match { + case r: RepointableRef ⇒ r.isStarted + case _ ⇒ true + }, 1 second, 10 millis) + val queue = actor.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].mailbox.messageQueue queue.getClass must be(classOf[CustomMailboxSpec.MyMailbox]) } } 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 a9855fef7d..11f8760320 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -1,12 +1,14 @@ package akka.dispatch -import akka.actor.{ Props, LocalActorRef, Actor } -import akka.testkit.AkkaSpec -import akka.pattern.ask -import akka.util.duration._ -import akka.testkit.DefaultTimeout +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner + import com.typesafe.config.Config -import akka.actor.ActorSystem + +import akka.actor.{ Props, InternalActorRef, ActorSystem, Actor } +import akka.pattern.ask +import akka.testkit.{ DefaultTimeout, AkkaSpec } +import akka.util.duration.intToDurationInt object PriorityDispatcherSpec { val config = """ @@ -54,7 +56,7 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit case i: Int ⇒ acc = i :: acc case 'Result ⇒ sender.tell(acc) } - }).withDispatcher(dispatcherKey)).asInstanceOf[LocalActorRef] + }).withDispatcher(dispatcherKey)).asInstanceOf[InternalActorRef] actor.suspend //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/event/EventStreamSpec.scala b/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala index d2497c4a69..d1846860f3 100644 --- a/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala @@ -19,7 +19,7 @@ object EventStreamSpec { loglevel = INFO event-handlers = ["akka.event.EventStreamSpec$MyLog", "%s"] } - """.format(Logging.StandardOutLoggerName)) + """.format(Logging.StandardOutLogger.getClass.getName)) val configUnhandled = ConfigFactory.parseString(""" akka { @@ -74,6 +74,17 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) { } } + "not allow null as subscriber" in { + val bus = new EventStream(true) + intercept[IllegalArgumentException] { bus.subscribe(null, classOf[M]) }.getMessage must be("subscriber is null") + } + + "not allow null as unsubscriber" in { + val bus = new EventStream(true) + intercept[IllegalArgumentException] { bus.unsubscribe(null, classOf[M]) }.getMessage must be("subscriber is null") + intercept[IllegalArgumentException] { bus.unsubscribe(null) }.getMessage must be("subscriber is null") + } + "be able to log unhandled messages" in { val sys = ActorSystem("EventStreamSpecUnhandled", configUnhandled) try { diff --git a/akka-actor-tests/src/test/scala/akka/pattern/CircuitBreakerMTSpec.scala b/akka-actor-tests/src/test/scala/akka/pattern/CircuitBreakerMTSpec.scala new file mode 100644 index 0000000000..35f55d703d --- /dev/null +++ b/akka-actor-tests/src/test/scala/akka/pattern/CircuitBreakerMTSpec.scala @@ -0,0 +1,121 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.pattern + +import akka.testkit._ +import akka.util.duration._ +import org.scalatest.BeforeAndAfter +import akka.dispatch.{ Promise, Await, Future } + +class CircuitBreakerMTSpec extends AkkaSpec with BeforeAndAfter { + + @volatile + var breakers: BreakerState = null + + class BreakerState { + + val halfOpenLatch = new TestLatch(1) + + val breaker = new CircuitBreaker(system.scheduler, 5, 100.millis.dilated, 500.millis.dilated) + .onHalfOpen(halfOpenLatch.countDown()) + + } + + before { + breakers = new BreakerState() + } + + def unreliableCall(param: String) = { + param match { + case "fail" ⇒ throw new RuntimeException("FAIL") + case _ ⇒ param + } + } + + def openBreaker: Unit = { + for (i ← 1 to 5) + Await.result(breakers.breaker.withCircuitBreaker(Future(unreliableCall("fail"))) recoverWith { + case _ ⇒ Promise.successful("OK") + }, 1.second.dilated) + } + + "A circuit breaker being called by many threads" must { + "allow many calls while in closed state with no errors" in { + + val futures = for (i ← 1 to 100) yield breakers.breaker.withCircuitBreaker(Future { Thread.sleep(10); unreliableCall("succeed") }) + + val futureList = Future.sequence(futures) + + val result = Await.result(futureList, 1.second.dilated) + + result.size must be(100) + result.distinct.size must be(1) + result.distinct must contain("succeed") + + } + + "transition to open state upon reaching failure limit and fail-fast" in { + + openBreaker + + val futures = for (i ← 1 to 100) yield breakers.breaker.withCircuitBreaker(Future { + Thread.sleep(10); unreliableCall("success") + }) recoverWith { + case _: CircuitBreakerOpenException ⇒ Promise.successful("CBO") + } + + val futureList = Future.sequence(futures) + + val result = Await.result(futureList, 1.second.dilated) + + result.size must be(100) + result.distinct.size must be(1) + result.distinct must contain("CBO") + } + + "allow a single call through in half-open state" in { + openBreaker + + Await.ready(breakers.halfOpenLatch, 2.seconds.dilated) + + val futures = for (i ← 1 to 100) yield breakers.breaker.withCircuitBreaker(Future { + Thread.sleep(10); unreliableCall("succeed") + }) recoverWith { + case _: CircuitBreakerOpenException ⇒ Promise.successful("CBO") + } + + val futureList = Future.sequence(futures) + + val result = Await.result(futureList, 1.second.dilated) + + result.size must be(100) + result.distinct.size must be(2) + result.distinct must contain("succeed") + result.distinct must contain("CBO") + } + + "recover and reset the breaker after the reset timeout" in { + openBreaker + + Await.ready(breakers.halfOpenLatch, 2.seconds.dilated) + + Await.ready(breakers.breaker.withCircuitBreaker(Future(unreliableCall("succeed"))), 1.second.dilated) + + val futures = for (i ← 1 to 100) yield breakers.breaker.withCircuitBreaker(Future { + Thread.sleep(10); unreliableCall("succeed") + }) recoverWith { + case _: CircuitBreakerOpenException ⇒ Promise.successful("CBO") + } + + val futureList = Future.sequence(futures) + + val result = Await.result(futureList, 1.second.dilated) + + result.size must be(100) + result.distinct.size must be(1) + result.distinct must contain("succeed") + } + } + +} \ No newline at end of file diff --git a/akka-actor-tests/src/test/scala/akka/pattern/CircuitBreakerSpec.scala b/akka-actor-tests/src/test/scala/akka/pattern/CircuitBreakerSpec.scala new file mode 100644 index 0000000000..2c2a07ee3f --- /dev/null +++ b/akka-actor-tests/src/test/scala/akka/pattern/CircuitBreakerSpec.scala @@ -0,0 +1,243 @@ + +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.pattern + +import akka.util.duration._ +import akka.testkit._ +import org.scalatest.BeforeAndAfter +import akka.dispatch.Future +import akka.dispatch.Await + +object CircuitBreakerSpec { + + class TestException extends RuntimeException + +} + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class CircuitBreakerSpec extends AkkaSpec with BeforeAndAfter { + + import CircuitBreakerSpec.TestException + + val awaitTimeout = 2.seconds.dilated + + @volatile + var breakers: TestCircuitBreakers = null + + class TestCircuitBreakers { + val halfOpenLatch = new TestLatch(1) + val openLatch = new TestLatch(1) + val closedLatch = new TestLatch(1) + + val shortCallTimeoutCb = new CircuitBreaker(system.scheduler, 1, 50.millis.dilated, 500.millis.dilated) + .onClose(closedLatch.countDown()) + .onHalfOpen(halfOpenLatch.countDown()) + .onOpen(openLatch.countDown()) + + val shortResetTimeoutCb = new CircuitBreaker(system.scheduler, 1, 1000.millis.dilated, 50.millis.dilated) + .onClose(closedLatch.countDown()) + .onHalfOpen(halfOpenLatch.countDown()) + .onOpen(openLatch.countDown()) + + val longCallTimeoutCb = new CircuitBreaker(system.scheduler, 1, 5 seconds, 500.millis.dilated) + .onClose(closedLatch.countDown()) + .onHalfOpen(halfOpenLatch.countDown()) + .onOpen(openLatch.countDown()) + + val longResetTimeoutCb = new CircuitBreaker(system.scheduler, 1, 100.millis.dilated, 5 seconds) + .onClose(closedLatch.countDown()) + .onHalfOpen(halfOpenLatch.countDown()) + .onOpen(openLatch.countDown()) + + val multiFailureCb = new CircuitBreaker(system.scheduler, 5, 200.millis.dilated, 500.millis.dilated) + .onClose(closedLatch.countDown()) + .onHalfOpen(halfOpenLatch.countDown()) + .onOpen(openLatch.countDown()) + } + + before { + breakers = new TestCircuitBreakers + } + + def checkLatch(latch: TestLatch) { + Await.ready(latch, awaitTimeout) + } + + def throwException = throw new TestException + + def sayHi = "hi" + + "A synchronous circuit breaker that is open" must { + "throw exceptions when called before reset timeout" in { + + intercept[TestException] { + breakers.longResetTimeoutCb.withSyncCircuitBreaker(throwException) + } + checkLatch(breakers.openLatch) + + intercept[CircuitBreakerOpenException] { + breakers.longResetTimeoutCb.withSyncCircuitBreaker(sayHi) + } + } + + "transition to half-open on reset timeout" in { + intercept[TestException] { + breakers.shortResetTimeoutCb.withSyncCircuitBreaker(throwException) + } + checkLatch(breakers.halfOpenLatch) + } + } + + "A synchronous circuit breaker that is half-open" must { + "pass through next call and close on success" in { + intercept[TestException] { + breakers.shortResetTimeoutCb.withSyncCircuitBreaker(throwException) + } + checkLatch(breakers.halfOpenLatch) + assert("hi" == breakers.shortResetTimeoutCb.withSyncCircuitBreaker(sayHi)) + checkLatch(breakers.closedLatch) + } + + "open on exception in call" in { + intercept[TestException] { + breakers.shortResetTimeoutCb.withSyncCircuitBreaker(throwException) + } + checkLatch(breakers.halfOpenLatch) + intercept[TestException] { + breakers.shortResetTimeoutCb.withSyncCircuitBreaker(throwException) + } + checkLatch(breakers.openLatch) + } + } + + "A synchronous circuit breaker that is closed" must { + "allow calls through" in { + breakers.longCallTimeoutCb.withSyncCircuitBreaker(sayHi) must be("hi") + } + + "increment failure count on failure" in { + intercept[TestException] { + breakers.longCallTimeoutCb.withSyncCircuitBreaker(throwException) + } + checkLatch(breakers.openLatch) + breakers.longCallTimeoutCb.currentFailureCount must be(1) + } + + "reset failure count after success" in { + intercept[TestException] { + breakers.multiFailureCb.withSyncCircuitBreaker(throwException) + } + + breakers.multiFailureCb.currentFailureCount must be(1) + breakers.multiFailureCb.withSyncCircuitBreaker(sayHi) + breakers.multiFailureCb.currentFailureCount must be(0) + } + + "increment failure count on callTimeout" in { + breakers.shortCallTimeoutCb.withSyncCircuitBreaker({ + 100.millis.dilated.sleep() + }) + breakers.shortCallTimeoutCb.currentFailureCount must be(1) + } + } + + "An asynchronous circuit breaker that is open" must { + "throw exceptions when called before reset timeout" in { + breakers.longResetTimeoutCb.withCircuitBreaker(Future(throwException)) + + checkLatch(breakers.openLatch) + + intercept[CircuitBreakerOpenException] { + Await.result( + breakers.longResetTimeoutCb.withCircuitBreaker(Future(sayHi)), + awaitTimeout) + } + } + + "transition to half-open on reset timeout" in { + breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException)) + checkLatch(breakers.halfOpenLatch) + } + } + + "An asynchronous circuit breaker that is half-open" must { + "pass through next call and close on success" in { + breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException)) + checkLatch(breakers.halfOpenLatch) + + Await.result( + breakers.shortResetTimeoutCb.withCircuitBreaker(Future(sayHi)), + awaitTimeout) must be("hi") + checkLatch(breakers.closedLatch) + } + + "re-open on exception in call" in { + breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException)) + checkLatch(breakers.halfOpenLatch) + + intercept[TestException] { + Await.result( + breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException)), + awaitTimeout) + } + checkLatch(breakers.openLatch) + } + + "re-open on async failure" in { + breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException)) + checkLatch(breakers.halfOpenLatch) + + breakers.shortResetTimeoutCb.withCircuitBreaker(Future(throwException)) + checkLatch(breakers.openLatch) + } + } + + "An asynchronous circuit breaker that is closed" must { + "allow calls through" in { + Await.result( + breakers.longCallTimeoutCb.withCircuitBreaker(Future(sayHi)), + awaitTimeout) must be("hi") + } + + "increment failure count on exception" in { + intercept[TestException] { + Await.result( + breakers.longCallTimeoutCb.withCircuitBreaker(Future(throwException)), + awaitTimeout) + } + checkLatch(breakers.openLatch) + breakers.longCallTimeoutCb.currentFailureCount must be(1) + } + + "increment failure count on async failure" in { + breakers.longCallTimeoutCb.withCircuitBreaker(Future(throwException)) + checkLatch(breakers.openLatch) + breakers.longCallTimeoutCb.currentFailureCount must be(1) + } + + "reset failure count after success" in { + breakers.multiFailureCb.withCircuitBreaker(Future(sayHi)) + val latch = TestLatch(4) + for (n ← 1 to 4) breakers.multiFailureCb.withCircuitBreaker(Future(throwException)) + awaitCond(breakers.multiFailureCb.currentFailureCount == 4, awaitTimeout) + breakers.multiFailureCb.withCircuitBreaker(Future(sayHi)) + awaitCond(breakers.multiFailureCb.currentFailureCount == 0, awaitTimeout) + } + + "increment failure count on callTimeout" in { + breakers.shortCallTimeoutCb.withCircuitBreaker { + Future { + 100.millis.dilated.sleep() + sayHi + } + } + + checkLatch(breakers.openLatch) + breakers.shortCallTimeoutCb.currentFailureCount must be(1) + } + } + +} diff --git a/akka-actor-tests/src/test/scala/akka/pattern/PatternSpec.scala b/akka-actor-tests/src/test/scala/akka/pattern/PatternSpec.scala index 2776beabce..68e6d40824 100644 --- a/akka-actor-tests/src/test/scala/akka/pattern/PatternSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/pattern/PatternSpec.scala @@ -7,11 +7,9 @@ package akka.pattern import akka.testkit.AkkaSpec import akka.actor.Props import akka.actor.Actor -import akka.actor.ActorTimeoutException import akka.util.Duration import akka.util.duration._ import akka.dispatch.{ Future, Promise, Await } -import java.lang.IllegalStateException object PatternSpec { case class Work(duration: Duration) @@ -41,13 +39,10 @@ class PatternSpec extends AkkaSpec { Await.ready(gracefulStop(target, 1 millis), 1 second) } - "complete Future with ActorTimeoutException when actor not terminated within timeout" in { + "complete Future with AskTimeoutException when actor not terminated within timeout" in { val target = system.actorOf(Props[TargetActor]) target ! Work(250 millis) - val result = gracefulStop(target, 10 millis) - intercept[ActorTimeoutException] { - Await.result(result, 200 millis) - } + intercept[AskTimeoutException] { Await.result(gracefulStop(target, 10 millis), 200 millis) } } } diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/PerformanceSpec.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/PerformanceSpec.scala index ca6e42d67f..ca23dd5a33 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/workbench/PerformanceSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/PerformanceSpec.scala @@ -3,12 +3,11 @@ package akka.performance.workbench import scala.collection.immutable.TreeMap import org.apache.commons.math.stat.descriptive.DescriptiveStatistics import org.scalatest.BeforeAndAfterEach -import akka.actor.simpleName import akka.testkit.AkkaSpec -import akka.actor.ActorSystem import akka.util.Duration import com.typesafe.config.Config import java.util.concurrent.TimeUnit +import akka.event.Logging abstract class PerformanceSpec(cfg: Config = BenchmarkConfig.config) extends AkkaSpec(cfg) with BeforeAndAfterEach { @@ -36,7 +35,7 @@ abstract class PerformanceSpec(cfg: Config = BenchmarkConfig.config) extends Akk } def logMeasurement(numberOfClients: Int, durationNs: Long, n: Long) { - val name = simpleName(this) + val name = Logging.simpleName(this) val durationS = durationNs.toDouble / 1000000000.0 val stats = Stats( @@ -51,7 +50,7 @@ abstract class PerformanceSpec(cfg: Config = BenchmarkConfig.config) extends Akk } def logMeasurement(numberOfClients: Int, durationNs: Long, stat: DescriptiveStatistics) { - val name = simpleName(this) + val name = Logging.simpleName(this) val durationS = durationNs.toDouble / 1000000000.0 val percentiles = TreeMap[Int, Long]( diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala index d01f1cda04..77ac5daf49 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala @@ -4,15 +4,14 @@ package akka.routing import java.util.concurrent.atomic.AtomicInteger - import org.junit.runner.RunWith - -import akka.actor.{ Props, LocalActorRef, Deploy, Actor, ActorRef } -import akka.config.ConfigurationException +import akka.actor.{ Props, Deploy, Actor, ActorRef } +import akka.ConfigurationException import akka.dispatch.Await import akka.pattern.{ ask, gracefulStop } import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec } import akka.util.duration.intToDurationInt +import akka.actor.UnstartedCell object ConfiguredLocalRoutingSpec { val config = """ @@ -47,6 +46,14 @@ object ConfiguredLocalRoutingSpec { @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.config) with DefaultTimeout with ImplicitSender { + def routerConfig(ref: ActorRef): RouterConfig = ref match { + case r: RoutedActorRef ⇒ + r.underlying match { + case c: RoutedActorCell ⇒ c.routerConfig + case _: UnstartedCell ⇒ awaitCond(r.isStarted, 1 second, 10 millis); routerConfig(ref) + } + } + "RouterConfig" must { "be picked up from Props" in { @@ -55,7 +62,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con case "get" ⇒ sender ! context.props } }).withRouter(RoundRobinRouter(12)), "someOther") - actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RoundRobinRouter(12) + routerConfig(actor) must be === RoundRobinRouter(12) Await.result(gracefulStop(actor, 3 seconds), 3 seconds) } @@ -65,7 +72,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con case "get" ⇒ sender ! context.props } }).withRouter(RoundRobinRouter(12)), "config") - actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RandomRouter(4) + routerConfig(actor) must be === RandomRouter(4) Await.result(gracefulStop(actor, 3 seconds), 3 seconds) } @@ -75,7 +82,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con case "get" ⇒ sender ! context.props } }).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "someOther") - actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RoundRobinRouter(12) + routerConfig(actor) must be === RoundRobinRouter(12) Await.result(gracefulStop(actor, 3 seconds), 3 seconds) } @@ -85,7 +92,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con case "get" ⇒ sender ! context.props } }).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "config") - actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RandomRouter(4) + routerConfig(actor) must be === RandomRouter(4) Await.result(gracefulStop(actor, 3 seconds), 3 seconds) } diff --git a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala index 457c4ab411..0a87273d61 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala @@ -128,7 +128,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with current.routees.size must be(2) } - "resize when busy" in { + "resize when busy" ignore { val busy = new TestLatch(1) 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 2ae32cfcf5..2b946ec1da 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -10,12 +10,13 @@ import akka.testkit._ import akka.util.duration._ import akka.dispatch.Await import akka.util.Duration -import akka.config.ConfigurationException +import akka.ConfigurationException import com.typesafe.config.ConfigFactory -import akka.pattern.ask +import akka.pattern.{ ask, pipe } import java.util.concurrent.ConcurrentHashMap import com.typesafe.config.Config import akka.dispatch.Dispatchers +import akka.util.Timeout object RoutingSpec { @@ -25,6 +26,10 @@ object RoutingSpec { router = round-robin nr-of-instances = 3 } + /router2 { + router = round-robin + nr-of-instances = 3 + } /myrouter { router = "akka.routing.RoutingSpec$MyRouter" foo = bar @@ -73,7 +78,9 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with watch(router) watch(c2) system.stop(c2) - expectMsg(Terminated(c2)) + expectMsgPF() { + case t @ Terminated(`c2`) if t.existenceConfirmed == true ⇒ t + } // it might take a while until the Router has actually processed the Terminated message awaitCond { router ! "" @@ -84,7 +91,9 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with res == Seq(c1, c1) } system.stop(c1) - expectMsg(Terminated(router)) + expectMsgPF() { + case t @ Terminated(`router`) if t.existenceConfirmed == true ⇒ t + } } "be able to send their routees" in { @@ -124,7 +133,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with } "use configured nr-of-instances when router is specified" in { - val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router1") + val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router2") Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(3) system.stop(router) } @@ -167,6 +176,18 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with expectMsg("restarted") } + "must start in-line for context.actorOf()" in { + system.actorOf(Props(new Actor { + def receive = { + case "start" ⇒ + context.actorOf(Props(new Actor { + def receive = { case x ⇒ sender ! x } + }).withRouter(RoundRobinRouter(2))) ? "hello" pipeTo sender + } + })) ! "start" + expectMsg("hello") + } + } "no router" must { @@ -524,7 +545,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with } } "support custom router" in { - val myrouter = system.actorOf(Props().withRouter(FromConfig), "myrouter") + val myrouter = system.actorOf(Props.empty.withRouter(FromConfig), "myrouter") myrouter.isTerminated must be(false) } } @@ -536,7 +557,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with } "count votes as intended - not as in Florida" in { - val routedActor = system.actorOf(Props().withRouter(VoteCountRouter())) + val routedActor = system.actorOf(Props.empty.withRouter(VoteCountRouter())) routedActor ! DemocratVote routedActor ! DemocratVote routedActor ! RepublicanVote diff --git a/akka-actor/src/main/java/akka/actor/AbstractActorCell.java b/akka-actor/src/main/java/akka/actor/AbstractActorCell.java new file mode 100644 index 0000000000..95fb7368bc --- /dev/null +++ b/akka-actor/src/main/java/akka/actor/AbstractActorCell.java @@ -0,0 +1,23 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.actor; + +import akka.util.Unsafe; + +final class AbstractActorCell { + final static long mailboxOffset; + final static long childrenOffset; + final static long nextNameOffset; + + static { + try { + mailboxOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_mailboxDoNotCallMeDirectly")); + childrenOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_childrenRefsDoNotCallMeDirectly")); + nextNameOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_nextNameDoNotCallMeDirectly")); + } catch(Throwable t){ + throw new ExceptionInInitializerError(t); + } + } +} diff --git a/akka-actor/src/main/java/akka/actor/AbstractActorRef.java b/akka-actor/src/main/java/akka/actor/AbstractActorRef.java new file mode 100644 index 0000000000..97ef09c501 --- /dev/null +++ b/akka-actor/src/main/java/akka/actor/AbstractActorRef.java @@ -0,0 +1,19 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.actor; + +import akka.util.Unsafe; + +final class AbstractActorRef { + final static long cellOffset; + + static { + try { + cellOffset = Unsafe.instance.objectFieldOffset(RepointableActorRef.class.getDeclaredField("_cellDoNotCallMeDirectly")); + } catch(Throwable t){ + throw new ExceptionInInitializerError(t); + } + } +} diff --git a/akka-actor/src/main/java/akka/pattern/AbstractCircuitBreaker.java b/akka-actor/src/main/java/akka/pattern/AbstractCircuitBreaker.java new file mode 100644 index 0000000000..44482bb357 --- /dev/null +++ b/akka-actor/src/main/java/akka/pattern/AbstractCircuitBreaker.java @@ -0,0 +1,18 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.pattern; + +import akka.util.Unsafe; + +class AbstractCircuitBreaker { + protected final static long stateOffset; + + static { + try { + stateOffset = Unsafe.instance.objectFieldOffset(CircuitBreaker.class.getDeclaredField("_currentStateDoNotCallMeDirectly")); + } catch(Throwable t){ + throw new ExceptionInInitializerError(t); + } + } +} diff --git a/akka-actor/src/main/java/akka/pattern/AbstractPromiseActorRef.java b/akka-actor/src/main/java/akka/pattern/AbstractPromiseActorRef.java index e21d58204e..bb0f03861c 100644 --- a/akka-actor/src/main/java/akka/pattern/AbstractPromiseActorRef.java +++ b/akka-actor/src/main/java/akka/pattern/AbstractPromiseActorRef.java @@ -8,10 +8,12 @@ import akka.util.Unsafe; final class AbstractPromiseActorRef { final static long stateOffset; + final static long watchedByOffset; static { try { stateOffset = Unsafe.instance.objectFieldOffset(PromiseActorRef.class.getDeclaredField("_stateDoNotCallMeDirectly")); + watchedByOffset = Unsafe.instance.objectFieldOffset(PromiseActorRef.class.getDeclaredField("_watchedByDoNotCallMeDirectly")); } catch(Throwable t){ throw new ExceptionInInitializerError(t); } diff --git a/akka-actor/src/main/java/org/jboss/netty/akka/util/internal/ConcurrentIdentityHashMap.java b/akka-actor/src/main/java/akka/util/internal/ConcurrentIdentityHashMap.java similarity index 99% rename from akka-actor/src/main/java/org/jboss/netty/akka/util/internal/ConcurrentIdentityHashMap.java rename to akka-actor/src/main/java/akka/util/internal/ConcurrentIdentityHashMap.java index ff8a568d02..eb83c98f35 100644 --- a/akka-actor/src/main/java/org/jboss/netty/akka/util/internal/ConcurrentIdentityHashMap.java +++ b/akka-actor/src/main/java/akka/util/internal/ConcurrentIdentityHashMap.java @@ -18,7 +18,7 @@ * Expert Group and released to the public domain, as explained at * http://creativecommons.org/licenses/publicdomain */ -package org.jboss.netty.akka.util.internal; +package akka.util.internal; import java.util.AbstractCollection; import java.util.AbstractMap; diff --git a/akka-actor/src/main/java/org/jboss/netty/akka/util/HashedWheelTimer.java b/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java similarity index 94% rename from akka-actor/src/main/java/org/jboss/netty/akka/util/HashedWheelTimer.java rename to akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java index 6e54fa2233..7a497b8442 100644 --- a/akka-actor/src/main/java/org/jboss/netty/akka/util/HashedWheelTimer.java +++ b/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java @@ -13,12 +13,10 @@ * License for the specific language governing permissions and limitations * under the License. */ -package org.jboss.netty.akka.util; +package akka.util.internal; import akka.event.LoggingAdapter; import akka.util.Duration; -import org.jboss.netty.akka.util.internal.ConcurrentIdentityHashMap; -import org.jboss.netty.akka.util.internal.ReusableIterator; import java.util.*; import java.util.concurrent.ThreadFactory; @@ -34,7 +32,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; *

Tick Duration

* * As described with 'approximated', this timer does not execute the scheduled - * {@link TimerTask} on time. {@link org.jboss.netty.akka.util.HashedWheelTimer}, on every tick, will + * {@link TimerTask} on time. {@link HashedWheelTimer}, on every tick, will * check if there are any {@link TimerTask}s behind the schedule and execute * them. *

@@ -46,7 +44,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; * *

Ticks per Wheel (Wheel Size)

* - * {@link org.jboss.netty.akka.util.HashedWheelTimer} maintains a data structure called 'wheel'. + * {@link HashedWheelTimer} maintains a data structure called 'wheel'. * To put simply, a wheel is a hash table of {@link TimerTask}s whose hash * function is 'dead line of the task'. The default number of ticks per wheel * (i.e. the size of the wheel) is 512. You could specify a larger value @@ -54,7 +52,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; * *

Do not create many instances.

* - * {@link org.jboss.netty.akka.util.HashedWheelTimer} creates a new thread whenever it is instantiated and + * {@link HashedWheelTimer} creates a new thread whenever it is instantiated and * started. Therefore, you should make sure to create only one instance and * share it across your application. One of the common mistakes, that makes * your application unresponsive, is to create a new instance in @@ -63,7 +61,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; * *

Implementation Details

* - * {@link org.jboss.netty.akka.util.HashedWheelTimer} is based on + * {@link HashedWheelTimer} is based on * George Varghese and * Tony Lauck's paper, * 'Hashed @@ -91,6 +89,7 @@ public class HashedWheelTimer implements Timer { final ReusableIterator[] iterators; final int mask; final ReadWriteLock lock = new ReentrantReadWriteLock(); + final boolean isWindows = System.getProperty("os.name", "").toLowerCase().indexOf("win") >= 0; volatile int wheelCursor; private LoggingAdapter logger; @@ -155,7 +154,7 @@ public class HashedWheelTimer implements Timer { ticksPerWheel = normalizeTicksPerWheel(ticksPerWheel); Set[] wheel = new Set[ticksPerWheel]; for (int i = 0; i < wheel.length; i ++) { - wheel[i] = new MapBackedSet(new ConcurrentIdentityHashMap(16, 0.95f, 4)); + wheel[i] = Collections.newSetFromMap(new ConcurrentIdentityHashMap(16, 0.95f, 4)); } return wheel; } @@ -391,7 +390,17 @@ public class HashedWheelTimer implements Timer { for (;;) { final long currentTime = System.nanoTime(); - final long sleepTime = (tickDuration * tick - (currentTime - startTime)); + + long sleepTime = tickDuration * tick - (currentTime - startTime); + + // Check if we run on windows, as if thats the case we will need + // to round the sleepTime as workaround for a bug that only affect + // the JVM if it runs on windows. + // + // See https://github.com/netty/netty/issues/356 + if (isWindows) { + sleepTime = (sleepTime / 10) * 10; + } if (sleepTime <= 0) { break; diff --git a/akka-actor/src/main/java/org/jboss/netty/akka/util/internal/ReusableIterator.java b/akka-actor/src/main/java/akka/util/internal/ReusableIterator.java similarity index 95% rename from akka-actor/src/main/java/org/jboss/netty/akka/util/internal/ReusableIterator.java rename to akka-actor/src/main/java/akka/util/internal/ReusableIterator.java index 210edbe65d..8c8e5e50e5 100644 --- a/akka-actor/src/main/java/org/jboss/netty/akka/util/internal/ReusableIterator.java +++ b/akka-actor/src/main/java/akka/util/internal/ReusableIterator.java @@ -13,7 +13,7 @@ * License for the specific language governing permissions and limitations * under the License. */ -package org.jboss.netty.akka.util.internal; +package akka.util.internal; import java.util.Iterator; diff --git a/akka-actor/src/main/java/org/jboss/netty/akka/util/internal/SystemPropertyUtil.java b/akka-actor/src/main/java/akka/util/internal/SystemPropertyUtil.java similarity index 98% rename from akka-actor/src/main/java/org/jboss/netty/akka/util/internal/SystemPropertyUtil.java rename to akka-actor/src/main/java/akka/util/internal/SystemPropertyUtil.java index bf3e2ac571..affef54bfc 100644 --- a/akka-actor/src/main/java/org/jboss/netty/akka/util/internal/SystemPropertyUtil.java +++ b/akka-actor/src/main/java/akka/util/internal/SystemPropertyUtil.java @@ -13,7 +13,7 @@ * License for the specific language governing permissions and limitations * under the License. */ -package org.jboss.netty.akka.util.internal; +package akka.util.internal; import java.util.regex.Pattern; diff --git a/akka-actor/src/main/java/org/jboss/netty/akka/util/Timeout.java b/akka-actor/src/main/java/akka/util/internal/Timeout.java similarity index 97% rename from akka-actor/src/main/java/org/jboss/netty/akka/util/Timeout.java rename to akka-actor/src/main/java/akka/util/internal/Timeout.java index dbda2110d3..a03534bb8d 100644 --- a/akka-actor/src/main/java/org/jboss/netty/akka/util/Timeout.java +++ b/akka-actor/src/main/java/akka/util/internal/Timeout.java @@ -13,7 +13,7 @@ * License for the specific language governing permissions and limitations * under the License. */ -package org.jboss.netty.akka.util; +package akka.util.internal; /** * A handle associated with a {@link TimerTask} that is returned by a diff --git a/akka-actor/src/main/java/org/jboss/netty/akka/util/Timer.java b/akka-actor/src/main/java/akka/util/internal/Timer.java similarity index 92% rename from akka-actor/src/main/java/org/jboss/netty/akka/util/Timer.java rename to akka-actor/src/main/java/akka/util/internal/Timer.java index b5bd8c6a7c..9cb02794de 100644 --- a/akka-actor/src/main/java/org/jboss/netty/akka/util/Timer.java +++ b/akka-actor/src/main/java/akka/util/internal/Timer.java @@ -13,7 +13,7 @@ * License for the specific language governing permissions and limitations * under the License. */ -package org.jboss.netty.akka.util; +package akka.util.internal; import akka.util.Duration; import java.util.Set; @@ -45,7 +45,7 @@ public interface Timer { Timeout newTimeout(TimerTask task, Duration delay); /** - * Releases all resources acquired by this {@link org.jboss.netty.akka.util.Timer} and cancels all + * Releases all resources acquired by this {@link Timer} and cancels all * tasks which were scheduled but not executed yet. * * @return the handles associated with the tasks which were canceled by diff --git a/akka-actor/src/main/java/org/jboss/netty/akka/util/TimerTask.java b/akka-actor/src/main/java/akka/util/internal/TimerTask.java similarity index 82% rename from akka-actor/src/main/java/org/jboss/netty/akka/util/TimerTask.java rename to akka-actor/src/main/java/akka/util/internal/TimerTask.java index 3d0190d8f5..673dde67c7 100644 --- a/akka-actor/src/main/java/org/jboss/netty/akka/util/TimerTask.java +++ b/akka-actor/src/main/java/akka/util/internal/TimerTask.java @@ -13,11 +13,11 @@ * License for the specific language governing permissions and limitations * under the License. */ -package org.jboss.netty.akka.util; +package akka.util.internal; /** * A task which is executed after the delay specified with - * {@link Timer#newTimeout(org.jboss.netty.akka.util.TimerTask, long, java.util.concurrent.TimeUnit)} + * {@link Timer#newTimeout(TimerTask, long, java.util.concurrent.TimeUnit)} * . * * @author The Netty Project @@ -28,7 +28,7 @@ public interface TimerTask { /** * Executed after the delay specified with - * {@link Timer#newTimeout(org.jboss.netty.akka.util.TimerTask, long, java.util.concurrent.TimeUnit)} + * {@link Timer#newTimeout(TimerTask, long, java.util.concurrent.TimeUnit)} * . * * @param timeout diff --git a/akka-actor/src/main/java/com/eaio/util/lang/Hex.java b/akka-actor/src/main/java/com/eaio/util/lang/Hex.java deleted file mode 100644 index 7794059517..0000000000 --- a/akka-actor/src/main/java/com/eaio/util/lang/Hex.java +++ /dev/null @@ -1,215 +0,0 @@ -/* - * Hex.java - * - * Created 04.07.2003. - * - * eaio: UUID - an implementation of the UUID specification Copyright (c) 2003-2009 Johann Burkard (jb@eaio.com) - * http://eaio.com. - * - * Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated - * documentation files (the "Software"), to deal in the Software without restriction, including without limitation the - * rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to - * permit persons to whom the Software is furnished to do so, subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included in all copies or substantial portions of the - * Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE - * WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR - * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR - * OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - * - */ -package com.eaio.util.lang; - -import java.io.IOException; - -/** - * Number-to-hexadecimal and hexadecimal-to-number conversions. - * - * @see UUID - * @author Johann Burkard - * @version $Id: Hex.java 1888 2009-03-15 12:43:24Z johann $ - */ -public final class Hex { - - /** - * No instances needed. - */ - private Hex() { - super(); - } - - private static final char[] DIGITS = { '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', - 'f' }; - - /** - * Turns a short into hex octets. - * - * @param a the {@link Appendable}, may not be null - * @param in the integer - * @return {@link Appendable} - */ - public static Appendable append(Appendable a, short in) { - return append(a, (long) in, 4); - } - - /** - * Turns a short into hex octets. - * - * @param a the {@link Appendable}, may not be null - * @param in the integer - * @param length the number of octets to produce - * @return {@link Appendable} - */ - public static Appendable append(Appendable a, short in, int length) { - return append(a, (long) in, length); - } - - /** - * Turns an int into hex octets. - * - * @param a the {@link Appendable}, may not be null - * @param in the integer - * @return {@link Appendable} - */ - public static Appendable append(Appendable a, int in) { - return append(a, (long) in, 8); - } - - /** - * Turns an int into hex octets. - * - * @param a the {@link Appendable}, may not be null - * @param in the integer - * @param length the number of octets to produce - * @return {@link Appendable} - */ - public static Appendable append(Appendable a, int in, int length) { - return append(a, (long) in, length); - } - - /** - * Turns a long into hex octets. - * - * @param a the {@link Appendable}, may not be null - * @param in the long - * @return {@link Appendable} - */ - public static Appendable append(Appendable a, long in) { - return append(a, in, 16); - } - - /** - * Turns a long into hex octets. - * - * @param a the {@link Appendable}, may not be null - * @param in the long - * @param length the number of octets to produce - * @return {@link Appendable} - */ - public static Appendable append(Appendable a, long in, int length) { - try { - int lim = (length << 2) - 4; - while (lim >= 0) { - a.append(DIGITS[(byte) (in >> lim) & 0x0f]); - lim -= 4; - } - } - catch (IOException ex) { - // Bla - } - return a; - } - - /** - * Turns a byte array into hex octets. - * - * @param a the {@link Appendable}, may not be null - * @param bytes the byte array - * @return {@link Appendable} - */ - public static Appendable append(Appendable a, byte[] bytes) { - try { - for (byte b : bytes) { - a.append(DIGITS[(byte) ((b & 0xF0) >> 4)]); - a.append(DIGITS[(byte) (b & 0x0F)]); - } - } - catch (IOException ex) { - // Bla - } - return a; - } - - /** - * Parses a long from a hex encoded number. This method will skip all characters that are not 0-9, - * A-F and a-f. - *

- * Returns 0 if the {@link CharSequence} does not contain any interesting characters. - * - * @param s the {@link CharSequence} to extract a long from, may not be null - * @return a long - * @throws NullPointerException if the {@link CharSequence} is null - */ - public static long parseLong(CharSequence s) { - long out = 0; - byte shifts = 0; - char c; - for (int i = 0; i < s.length() && shifts < 16; i++) { - c = s.charAt(i); - if ((c > 47) && (c < 58)) { - ++shifts; - out <<= 4; - out |= c - 48; - } - else if ((c > 64) && (c < 71)) { - ++shifts; - out <<= 4; - out |= c - 55; - } - else if ((c > 96) && (c < 103)) { - ++shifts; - out <<= 4; - out |= c - 87; - } - } - return out; - } - - /** - * Parses a short from a hex encoded number. This method will skip all characters that are not 0-9, - * A-F and a-f. - *

- * Returns 0 if the {@link CharSequence} does not contain any interesting characters. - * - * @param s the {@link CharSequence} to extract a short from, may not be null - * @return a short - * @throws NullPointerException if the {@link CharSequence} is null - */ - public static short parseShort(String s) { - short out = 0; - byte shifts = 0; - char c; - for (int i = 0; i < s.length() && shifts < 4; i++) { - c = s.charAt(i); - if ((c > 47) && (c < 58)) { - ++shifts; - out <<= 4; - out |= c - 48; - } - else if ((c > 64) && (c < 71)) { - ++shifts; - out <<= 4; - out |= c - 55; - } - else if ((c > 96) && (c < 103)) { - ++shifts; - out <<= 4; - out |= c - 87; - } - } - return out; - } - -} diff --git a/akka-actor/src/main/java/com/eaio/uuid/MACAddressParser.java b/akka-actor/src/main/java/com/eaio/uuid/MACAddressParser.java deleted file mode 100644 index c077147470..0000000000 --- a/akka-actor/src/main/java/com/eaio/uuid/MACAddressParser.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * MACAddressParserTest.java - * - * Created 30.01.2006. - * - * eaio: UUID - an implementation of the UUID specification - * Copyright (c) 2003-2009 Johann Burkard (jb@eaio.com) http://eaio.com. - * - * Permission is hereby granted, free of charge, to any person obtaining a - * copy of this software and associated documentation files (the "Software"), - * to deal in the Software without restriction, including without limitation - * the rights to use, copy, modify, merge, publish, distribute, sublicense, - * and/or sell copies of the Software, and to permit persons to whom the - * Software is furnished to do so, subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included - * in all copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS - * OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF - * MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - * NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - * DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR - * OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE - * USE OR OTHER DEALINGS IN THE SOFTWARE. - * - */ -package com.eaio.uuid; - -/** - * The MAC address parser attempts to find the following patterns: - *

    - *
  • .{1,2}:.{1,2}:.{1,2}:.{1,2}:.{1,2}:.{1,2}
  • - *
  • .{1,2}-.{1,2}-.{1,2}-.{1,2}-.{1,2}-.{1,2}
  • - *
- * - * @see UUID - * @author Johann Burkard - * @version $Id: MACAddressParser.java 1888 2009-03-15 12:43:24Z johann $ - */ -class MACAddressParser { - - /** - * No instances needed. - */ - private MACAddressParser() { - super(); - } - - /** - * Attempts to find a pattern in the given String. - * - * @param in the String, may not be null - * @return the substring that matches this pattern or null - */ - static String parse(String in) { - - String out = in; - - // lanscan - - int hexStart = out.indexOf("0x"); - if (hexStart != -1 && out.indexOf("ETHER") != -1) { - int hexEnd = out.indexOf(' ', hexStart); - if (hexEnd > hexStart + 2) { - out = out.substring(hexStart, hexEnd); - } - } - - else { - - int octets = 0; - int lastIndex, old, end; - - if (out.indexOf('-') > -1) { - out = out.replace('-', ':'); - } - - lastIndex = out.lastIndexOf(':'); - - if (lastIndex > out.length() - 2) { - out = null; - } - else { - - end = Math.min(out.length(), lastIndex + 3); - - ++octets; - old = lastIndex; - while (octets != 5 && lastIndex != -1 && lastIndex > 1) { - lastIndex = out.lastIndexOf(':', --lastIndex); - if (old - lastIndex == 3 || old - lastIndex == 2) { - ++octets; - old = lastIndex; - } - } - - if (octets == 5 && lastIndex > 1) { - out = out.substring(lastIndex - 2, end).trim(); - } - else { - out = null; - } - - } - - } - - if (out != null && out.startsWith("0x")) { - out = out.substring(2); - } - - return out; - } - -} diff --git a/akka-actor/src/main/java/com/eaio/uuid/UUID.java b/akka-actor/src/main/java/com/eaio/uuid/UUID.java deleted file mode 100644 index a578a68c6d..0000000000 --- a/akka-actor/src/main/java/com/eaio/uuid/UUID.java +++ /dev/null @@ -1,308 +0,0 @@ -/* - * UUID.java - * - * Created 07.02.2003 - * - * eaio: UUID - an implementation of the UUID specification - * Copyright (c) 2003-2009 Johann Burkard (jb@eaio.com) http://eaio.com. - * - * Permission is hereby granted, free of charge, to any person obtaining a - * copy of this software and associated documentation files (the "Software"), - * to deal in the Software without restriction, including without limitation - * the rights to use, copy, modify, merge, publish, distribute, sublicense, - * and/or sell copies of the Software, and to permit persons to whom the - * Software is furnished to do so, subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included - * in all copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS - * OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF - * MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - * NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - * DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR - * OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE - * USE OR OTHER DEALINGS IN THE SOFTWARE. - * - */ -package com.eaio.uuid; - -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.Serializable; - -import com.eaio.util.lang.Hex; - -/** - * Creates UUIDs according to the DCE Universal Token Identifier specification. - *

- * All you need to know: - *

- * UUID u = new UUID();
- * 
- * - * @see - * http://www.opengroup.org/onlinepubs/9629399/apdxa.htm - * - * @see - * http://www.uddi.org/pubs/draft-leach-uuids-guids-01.txt - * - * @see UUID - * @author Johann Burkard - * @version $Id: UUID.java 1888 2009-03-15 12:43:24Z johann $ - */ -public class UUID implements Comparable, Serializable, Cloneable { - - /** - * Hasn't ever changed between versions. - */ - static final long serialVersionUID = 7435962790062944603L; - - /** - * The time field of the UUID. - * - * @serial - */ - public long time; - - /** - * The clock sequence and node field of the UUID. - * - * @serial - */ - public long clockSeqAndNode; - - /** - * Constructor for UUID. Constructs a new, unique UUID. - * - * @see UUIDGen#newTime() - * @see UUIDGen#getClockSeqAndNode() - */ - public UUID() { - this(UUIDGen.newTime(), UUIDGen.getClockSeqAndNode()); - } - - /** - * Constructor for UUID. Constructs a UUID from two long values. - * - * @param time the upper 64 bits - * @param clockSeqAndNode the lower 64 bits - */ - public UUID(long time, long clockSeqAndNode) { - this.time = time; - this.clockSeqAndNode = clockSeqAndNode; - } - - /** - * Copy constructor for UUID. Values of the given UUID are copied. - * - * @param u the UUID, may not be null - */ - public UUID(UUID u) { - this(u.time, u.clockSeqAndNode); - } - - /** - * Parses a textual representation of a UUID. - *

- * No validation is performed. If the {@link CharSequence} is shorter than 36 characters, - * {@link ArrayIndexOutOfBoundsException}s will be thrown. - * - * @param s the {@link CharSequence}, may not be null - */ - public UUID(CharSequence s) { - this(Hex.parseLong(s.subSequence(0, 18)), Hex.parseLong(s.subSequence( - 19, 36))); - } - - /** - * Compares this UUID to another Object. Throws a {@link ClassCastException} if - * the other Object is not an instance of the UUID class. Returns a value - * smaller than zero if the other UUID is "larger" than this UUID and a value - * larger than zero if the other UUID is "smaller" than this UUID. - * - * @param t the other UUID, may not be null - * @return a value < 0, 0 or a value > 0 - * @see java.lang.Comparable#compareTo(java.lang.Object) - * @throws ClassCastException - */ - public int compareTo(UUID t) { - if (this == t) { - return 0; - } - if (time > t.time) { - return 1; - } - if (time < t.time) { - return -1; - } - if (clockSeqAndNode > t.clockSeqAndNode) { - return 1; - } - if (clockSeqAndNode < t.clockSeqAndNode) { - return -1; - } - return 0; - } - - /** - * Tweaked Serialization routine. - * - * @param out the ObjectOutputStream - * @throws IOException - */ - private void writeObject(ObjectOutputStream out) throws IOException { - out.writeLong(time); - out.writeLong(clockSeqAndNode); - } - - /** - * Tweaked Serialization routine. - * - * @param in the ObjectInputStream - * @throws IOException - */ - private void readObject(ObjectInputStream in) throws IOException { - time = in.readLong(); - clockSeqAndNode = in.readLong(); - } - - /** - * Returns this UUID as a String. - * - * @return a String, never null - * @see java.lang.Object#toString() - * @see #toAppendable(Appendable) - */ - @Override - public final String toString() { - return toAppendable(null).toString(); - } - - /** - * Appends a String representation of this to the given {@link StringBuffer} or - * creates a new one if none is given. - * - * @param in the StringBuffer to append to, may be null - * @return a StringBuffer, never null - * @see #toAppendable(Appendable) - */ - public StringBuffer toStringBuffer(StringBuffer in) { - StringBuffer out = in; - if (out == null) { - out = new StringBuffer(36); - } - else { - out.ensureCapacity(out.length() + 36); - } - return (StringBuffer) toAppendable(out); - } - - /** - * Appends a String representation of this object to the given {@link Appendable} object. - *

- * For reasons I'll probably never understand, Sun has decided to have a number of I/O classes implement - * Appendable which forced them to destroy an otherwise nice and simple interface with {@link IOException}s. - *

- * I decided to ignore any possible IOExceptions in this method. - * - * @param a the Appendable object, may be null - * @return an Appendable object, defaults to a {@link StringBuilder} if a is null - */ - public Appendable toAppendable(Appendable a) { - Appendable out = a; - if (out == null) { - out = new StringBuilder(36); - } - try { - Hex.append(out, (int) (time >> 32)).append('-'); - Hex.append(out, (short) (time >> 16)).append('-'); - Hex.append(out, (short) time).append('-'); - Hex.append(out, (short) (clockSeqAndNode >> 48)).append('-'); - Hex.append(out, clockSeqAndNode, 12); - } - catch (IOException ex) { - // What were they thinking? - } - return out; - } - - /** - * Returns a hash code of this UUID. The hash code is calculated by XOR'ing the - * upper 32 bits of the time and clockSeqAndNode fields and the lower 32 bits of - * the time and clockSeqAndNode fields. - * - * @return an int representing the hash code - * @see java.lang.Object#hashCode() - */ - @Override - public int hashCode() { - return (int) ((time >> 32) ^ time ^ (clockSeqAndNode >> 32) ^ clockSeqAndNode); - } - - /** - * Clones this UUID. - * - * @return a new UUID with identical values, never null - */ - @Override - public Object clone() { - try { - return super.clone(); - } - catch (CloneNotSupportedException ex) { - // One of Sun's most epic fails. - return null; - } - } - - /** - * Returns the time field of the UUID (upper 64 bits). - * - * @return the time field - */ - public final long getTime() { - return time; - } - - /** - * Returns the clock and node field of the UUID (lower 64 bits). - * - * @return the clockSeqAndNode field - */ - public final long getClockSeqAndNode() { - return clockSeqAndNode; - } - - /** - * Compares two Objects for equality. - * - * @see java.lang.Object#equals(Object) - * @param obj the Object to compare this UUID with, may be null - * @return true if the other Object is equal to this UUID, - * false if not - */ - @Override - public boolean equals(Object obj) { - if (!(obj instanceof UUID)) { - return false; - } - return compareTo((UUID) obj) == 0; - } - - /** - * Returns the nil UUID (a UUID whose values are both set to zero). - *

- * Starting with version 2.0, this method does return a new UUID instance every - * time it is called. Earlier versions returned one instance. This has now been - * changed because this UUID has public, non-final instance fields. Returning a - * new instance is therefore more safe. - * - * @return a nil UUID, never null - */ - public static UUID nilUUID() { - return new UUID(0, 0); - } - -} diff --git a/akka-actor/src/main/java/com/eaio/uuid/UUIDGen.java b/akka-actor/src/main/java/com/eaio/uuid/UUIDGen.java deleted file mode 100644 index fb60e1727a..0000000000 --- a/akka-actor/src/main/java/com/eaio/uuid/UUIDGen.java +++ /dev/null @@ -1,368 +0,0 @@ -/* - * UUIDGen.java - * - * Created on 09.08.2003. - * - * eaio: UUID - an implementation of the UUID specification - * Copyright (c) 2003-2009 Johann Burkard (jb@eaio.com) http://eaio.com. - * - * Permission is hereby granted, free of charge, to any person obtaining a - * copy of this software and associated documentation files (the "Software"), - * to deal in the Software without restriction, including without limitation - * the rights to use, copy, modify, merge, publish, distribute, sublicense, - * and/or sell copies of the Software, and to permit persons to whom the - * Software is furnished to do so, subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included - * in all copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS - * OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF - * MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN - * NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, - * DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR - * OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE - * USE OR OTHER DEALINGS IN THE SOFTWARE. - * - */ -package com.eaio.uuid; - -import java.io.BufferedReader; -import java.io.File; -import java.io.IOException; -import java.io.InputStreamReader; -import java.net.InetAddress; -import java.net.InterfaceAddress; -import java.net.NetworkInterface; -import java.net.SocketException; -import java.net.UnknownHostException; -import java.util.Enumeration; -import java.util.concurrent.atomic.AtomicLong; - -import com.eaio.util.lang.Hex; - -/** - * This class contains methods to generate UUID fields. These methods have been - * refactored out of {@link com.eaio.uuid.UUID}. - *

- * Starting with version 2, this implementation tries to obtain the MAC address - * of the network card. Under Microsoft Windows, the ifconfig - * command is used which may pop up a command window in Java Virtual Machines - * prior to 1.4 once this class is initialized. The command window is closed - * automatically. - *

- * The MAC address code has been tested extensively in Microsoft Windows, - * Linux, Solaris 8, HP-UX 11, but should work in MacOS X and BSDs, too. - *

- * If you use JDK 6 or later, the code in {@link InterfaceAddress} will be used. - * - * @see UUID - * @author Johann Burkard - * @version $Id: UUIDGen.java 2914 2010-04-23 11:35:00Z johann $ - * @see com.eaio.uuid.UUID - */ -public final class UUIDGen { - - /** - * No instances needed. - */ - private UUIDGen() { - super(); - } - - /** - * The last time value. Used to remove duplicate UUIDs. - */ - private final static AtomicLong lastTime = new AtomicLong(Long.MIN_VALUE); - - /** - * The cached MAC address. - */ - private static String macAddress = null; - - /** - * The current clock and node value. - */ - private static long clockSeqAndNode = 0x8000000000000000L; - - static { - - try { - Class.forName("java.net.InterfaceAddress"); - macAddress = Class.forName( - "com.eaio.uuid.UUIDGen$HardwareAddressLookup").newInstance().toString(); - } - catch (ExceptionInInitializerError err) { - // Ignored. - } - catch (ClassNotFoundException ex) { - // Ignored. - } - catch (LinkageError err) { - // Ignored. - } - catch (IllegalAccessException ex) { - // Ignored. - } - catch (InstantiationException ex) { - // Ignored. - } - catch (SecurityException ex) { - // Ignored. - } - - if (macAddress == null) { - - Process p = null; - BufferedReader in = null; - - try { - String osname = System.getProperty("os.name", ""); - - if (osname.startsWith("Windows")) { - p = Runtime.getRuntime().exec( - new String[] { "ipconfig", "/all" }, null); - } - // Solaris code must appear before the generic code - else if (osname.startsWith("Solaris") - || osname.startsWith("SunOS")) { - String hostName = getFirstLineOfCommand( - "uname", "-n" ); - if (hostName != null) { - p = Runtime.getRuntime().exec( - new String[] { "/usr/sbin/arp", hostName }, - null); - } - } - else if (new File("/usr/sbin/lanscan").exists()) { - p = Runtime.getRuntime().exec( - new String[] { "/usr/sbin/lanscan" }, null); - } - else if (new File("/sbin/ifconfig").exists()) { - p = Runtime.getRuntime().exec( - new String[] { "/sbin/ifconfig", "-a" }, null); - } - - if (p != null) { - in = new BufferedReader(new InputStreamReader( - p.getInputStream()), 128); - String l = null; - while ((l = in.readLine()) != null) { - macAddress = MACAddressParser.parse(l); - if (macAddress != null - && Hex.parseShort(macAddress) != 0xff) { - break; - } - } - } - - } - catch (SecurityException ex) { - // Ignore it. - } - catch (IOException ex) { - // Ignore it. - } - finally { - if (p != null) { - if (in != null) { - try { - in.close(); - } - catch (IOException ex) { - // Ignore it. - } - } - try { - p.getErrorStream().close(); - } - catch (IOException ex) { - // Ignore it. - } - try { - p.getOutputStream().close(); - } - catch (IOException ex) { - // Ignore it. - } - p.destroy(); - } - } - - } - - if (macAddress != null) { - clockSeqAndNode |= Hex.parseLong(macAddress); - } - else { - try { - byte[] local = InetAddress.getLocalHost().getAddress(); - clockSeqAndNode |= (local[0] << 24) & 0xFF000000L; - clockSeqAndNode |= (local[1] << 16) & 0xFF0000; - clockSeqAndNode |= (local[2] << 8) & 0xFF00; - clockSeqAndNode |= local[3] & 0xFF; - } - catch (UnknownHostException ex) { - clockSeqAndNode |= (long) (Math.random() * 0x7FFFFFFF); - } - } - - // Skip the clock sequence generation process and use random instead. - - clockSeqAndNode |= (long) (Math.random() * 0x3FFF) << 48; - - } - - /** - * Returns the current clockSeqAndNode value. - * - * @return the clockSeqAndNode value - * @see UUID#getClockSeqAndNode() - */ - public static long getClockSeqAndNode() { - return clockSeqAndNode; - } - - /** - * Generates a new time field. Each time field is unique and larger than the - * previously generated time field. - * - * @return a new time value - * @see UUID#getTime() - */ - public static long newTime() { - return createTime(System.currentTimeMillis()); - } - - /** - * Creates a new time field from the given timestamp. Note that even identical - * values of currentTimeMillis will produce different time fields. - * - * @param currentTimeMillis the timestamp - * @return a new time value - * @see UUID#getTime() - */ - public static long createTime(long currentTimeMillis) { - - long time; - - // UTC time - - long timeMillis = (currentTimeMillis * 10000) + 0x01B21DD213814000L; - - // Make sure our time is unique - - for(;;) { - final long c = lastTime.get(); - if (timeMillis <= c) { - timeMillis = lastTime.incrementAndGet(); - break; - } else if(lastTime.compareAndSet(c, timeMillis)) break; - } - - // time low - - time = timeMillis << 32; - - // time mid - - time |= (timeMillis & 0xFFFF00000000L) >> 16; - - // time hi and version - - time |= 0x1000 | ((timeMillis >> 48) & 0x0FFF); // version 1 - - return time; - - } - - /** - * Returns the MAC address. Not guaranteed to return anything. - * - * @return the MAC address, may be null - */ - public static String getMACAddress() { - return macAddress; - } - - /** - * Returns the first line of the shell command. - * - * @param commands the commands to run - * @return the first line of the command - * @throws IOException - */ - static String getFirstLineOfCommand(String... commands) throws IOException { - - Process p = null; - BufferedReader reader = null; - - try { - p = Runtime.getRuntime().exec(commands); - reader = new BufferedReader(new InputStreamReader( - p.getInputStream()), 128); - - return reader.readLine(); - } - finally { - if (p != null) { - if (reader != null) { - try { - reader.close(); - } - catch (IOException ex) { - // Ignore it. - } - } - try { - p.getErrorStream().close(); - } - catch (IOException ex) { - // Ignore it. - } - try { - p.getOutputStream().close(); - } - catch (IOException ex) { - // Ignore it. - } - p.destroy(); - } - } - - } - - /** - * Scans MAC addresses for good ones. - */ - static class HardwareAddressLookup { - - /** - * @see java.lang.Object#toString() - */ - @Override - public String toString() { - String out = null; - try { - Enumeration ifs = NetworkInterface.getNetworkInterfaces(); - if (ifs != null) { - while (ifs.hasMoreElements()) { - NetworkInterface iface = ifs.nextElement(); - byte[] hardware = iface.getHardwareAddress(); - if (hardware != null && hardware.length == 6 - && hardware[1] != (byte) 0xff) { - out = Hex.append(new StringBuilder(36), hardware).toString(); - break; - } - } - } - } - catch (SocketException ex) { - // Ignore it. - } - return out; - } - - } - -} diff --git a/akka-actor/src/main/java/com/typesafe/config/Config.java b/akka-actor/src/main/java/com/typesafe/config/Config.java deleted file mode 100755 index 629b107be9..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/Config.java +++ /dev/null @@ -1,509 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * An immutable map from config paths to config values. - * - *

- * Contrast with {@link ConfigObject} which is a map from config keys, - * rather than paths, to config values. A {@code Config} contains a tree of - * {@code ConfigObject}, and {@link Config#root()} returns the tree's root - * object. - * - *

- * Throughout the API, there is a distinction between "keys" and "paths". A key - * is a key in a JSON object; it's just a string that's the key in a map. A - * "path" is a parseable expression with a syntax and it refers to a series of - * keys. Path expressions are described in the spec for - * Human-Optimized Config Object Notation. In brief, a path is - * period-separated so "a.b.c" looks for key c in object b in object a in the - * root object. Sometimes double quotes are needed around special characters in - * path expressions. - * - *

- * The API for a {@code Config} is in terms of path expressions, while the API - * for a {@code ConfigObject} is in terms of keys. Conceptually, {@code Config} - * is a one-level map from paths to values, while a - * {@code ConfigObject} is a tree of nested maps from keys to values. - * - *

- * Use {@link ConfigUtil#joinPath} and {@link ConfigUtil#splitPath} to convert - * between path expressions and individual path elements (keys). - * - *

- * Another difference between {@code Config} and {@code ConfigObject} is that - * conceptually, {@code ConfigValue}s with a {@link ConfigValue#valueType() - * valueType()} of {@link ConfigValueType#NULL NULL} exist in a - * {@code ConfigObject}, while a {@code Config} treats null values as if they - * were missing. - * - *

- * {@code Config} is an immutable object and thus safe to use from multiple - * threads. There's never a need for "defensive copies." - * - *

- * The "getters" on a {@code Config} all work in the same way. They never return - * null, nor do they return a {@code ConfigValue} with - * {@link ConfigValue#valueType() valueType()} of {@link ConfigValueType#NULL - * NULL}. Instead, they throw {@link ConfigException.Missing} if the value is - * completely absent or set to null. If the value is set to null, a subtype of - * {@code ConfigException.Missing} called {@link ConfigException.Null} will be - * thrown. {@link ConfigException.WrongType} will be thrown anytime you ask for - * a type and the value has an incompatible type. Reasonable type conversions - * are performed for you though. - * - *

- * If you want to iterate over the contents of a {@code Config}, you can get its - * {@code ConfigObject} with {@link #root()}, and then iterate over the - * {@code ConfigObject} (which implements java.util.Map). Or, you - * can use {@link #entrySet()} which recurses the object tree for you and builds - * up a Set of all path-value pairs where the value is not null. - * - *

- * Do not implement {@code Config}; it should only be implemented by - * the config library. Arbitrary implementations will not work because the - * library internals assume a specific concrete implementation. Also, this - * interface is likely to grow new methods over time, so third-party - * implementations will break. - */ -public interface Config extends ConfigMergeable { - /** - * Gets the {@code Config} as a tree of {@link ConfigObject}. This is a - * constant-time operation (it is not proportional to the number of values - * in the {@code Config}). - * - * @return the root object in the configuration - */ - ConfigObject root(); - - /** - * Gets the origin of the {@code Config}, which may be a file, or a file - * with a line number, or just a descriptive phrase. - * - * @return the origin of the {@code Config} for use in error messages - */ - ConfigOrigin origin(); - - @Override - Config withFallback(ConfigMergeable other); - - /** - * Returns a replacement config with all substitutions (the - * ${foo.bar} syntax, see the - * spec) resolved. Substitutions are looked up using this - * Config as the root object, that is, a substitution - * ${foo.bar} will be replaced with the result of - * getValue("foo.bar"). - * - *

- * This method uses {@link ConfigResolveOptions#defaults()}, there is - * another variant {@link Config#resolve(ConfigResolveOptions)} which lets - * you specify non-default options. - * - *

- * A given {@link Config} must be resolved before using it to retrieve - * config values, but ideally should be resolved one time for your entire - * stack of fallbacks (see {@link Config#withFallback}). Otherwise, some - * substitutions that could have resolved with all fallbacks available may - * not resolve, which will be a user-visible oddity. - * - *

- * resolve() should be invoked on root config objects, rather - * than on a subtree (a subtree is the result of something like - * config.getConfig("foo")). The problem with - * resolve() on a subtree is that substitutions are relative to - * the root of the config and the subtree will have no way to get values - * from the root. For example, if you did - * config.getConfig("foo").resolve() on the below config file, - * it would not work: - * - *

-     *   common-value = 10
-     *   foo {
-     *      whatever = ${common-value}
-     *   }
-     * 
- * - * @return an immutable object with substitutions resolved - * @throws ConfigException.UnresolvedSubstitution - * if any substitutions refer to nonexistent paths - * @throws ConfigException - * some other config exception if there are other problems - */ - Config resolve(); - - /** - * Like {@link Config#resolve()} but allows you to specify non-default - * options. - * - * @param options - * resolve options - * @return the resolved Config - */ - Config resolve(ConfigResolveOptions options); - - /** - * Validates this config against a reference config, throwing an exception - * if it is invalid. The purpose of this method is to "fail early" with a - * comprehensive list of problems; in general, anything this method can find - * would be detected later when trying to use the config, but it's often - * more user-friendly to fail right away when loading the config. - * - *

- * Using this method is always optional, since you can "fail late" instead. - * - *

- * You must restrict validation to paths you "own" (those whose meaning are - * defined by your code module). If you validate globally, you may trigger - * errors about paths that happen to be in the config but have nothing to do - * with your module. It's best to allow the modules owning those paths to - * validate them. Also, if every module validates only its own stuff, there - * isn't as much redundant work being done. - * - *

- * If no paths are specified in checkValid()'s parameter list, - * validation is for the entire config. - * - *

- * If you specify paths that are not in the reference config, those paths - * are ignored. (There's nothing to validate.) - * - *

- * Here's what validation involves: - * - *

    - *
  • All paths found in the reference config must be present in this - * config or an exception will be thrown. - *
  • - * Some changes in type from the reference config to this config will cause - * an exception to be thrown. Not all potential type problems are detected, - * in particular it's assumed that strings are compatible with everything - * except objects and lists. This is because string types are often "really" - * some other type (system properties always start out as strings, or a - * string like "5ms" could be used with {@link #getMilliseconds}). Also, - * it's allowed to set any type to null or override null with any type. - *
  • - * Any unresolved substitutions in this config will cause a validation - * failure; both the reference config and this config should be resolved - * before validation. If the reference config is unresolved, it's a bug in - * the caller of this method. - *
- * - *

- * If you want to allow a certain setting to have a flexible type (or - * otherwise want validation to be looser for some settings), you could - * either remove the problematic setting from the reference config provided - * to this method, or you could intercept the validation exception and - * screen out certain problems. Of course, this will only work if all other - * callers of this method are careful to restrict validation to their own - * paths, as they should be. - * - *

- * If validation fails, the thrown exception contains a list of all problems - * found. See {@link ConfigException.ValidationFailed#problems}. The - * exception's getMessage() will have all the problems - * concatenated into one huge string, as well. - * - *

- * Again, checkValid() can't guess every domain-specific way a - * setting can be invalid, so some problems may arise later when attempting - * to use the config. checkValid() is limited to reporting - * generic, but common, problems such as missing settings and blatant type - * incompatibilities. - * - * @param reference - * a reference configuration - * @param restrictToPaths - * only validate values underneath these paths that your code - * module owns and understands - * @throws ConfigException.ValidationFailed - * if there are any validation issues - * @throws ConfigException.NotResolved - * if this config is not resolved - * @throws ConfigException.BugOrBroken - * if the reference config is unresolved or caller otherwise - * misuses the API - */ - void checkValid(Config reference, String... restrictToPaths); - - /** - * Checks whether a value is present and non-null at the given path. This - * differs in two ways from {@code Map.containsKey()} as implemented by - * {@link ConfigObject}: it looks for a path expression, not a key; and it - * returns false for null values, while {@code containsKey()} returns true - * indicating that the object contains a null value for the key. - * - *

- * If a path exists according to {@link #hasPath(String)}, then - * {@link #getValue(String)} will never throw an exception. However, the - * typed getters, such as {@link #getInt(String)}, will still throw if the - * value is not convertible to the requested type. - * - * @param path - * the path expression - * @return true if a non-null value is present at the path - * @throws ConfigException.BadPath - * if the path expression is invalid - */ - boolean hasPath(String path); - - /** - * Returns true if the {@code Config}'s root object contains no key-value - * pairs. - * - * @return true if the configuration is empty - */ - boolean isEmpty(); - - /** - * Returns the set of path-value pairs, excluding any null values, found by - * recursing {@link #root() the root object}. Note that this is very - * different from root().entrySet() which returns the set of - * immediate-child keys in the root object and includes null values. - * - * @return set of paths with non-null values, built up by recursing the - * entire tree of {@link ConfigObject} - */ - Set> entrySet(); - - /** - * - * @param path - * path expression - * @return the boolean value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to boolean - */ - boolean getBoolean(String path); - - /** - * @param path - * path expression - * @return the numeric value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to a number - */ - Number getNumber(String path); - - /** - * @param path - * path expression - * @return the 32-bit integer value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to an int (for example it is out - * of range, or it's a boolean value) - */ - int getInt(String path); - - /** - * @param path - * path expression - * @return the 64-bit long value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to a long - */ - long getLong(String path); - - /** - * @param path - * path expression - * @return the floating-point value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to a double - */ - double getDouble(String path); - - /** - * @param path - * path expression - * @return the string value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to a string - */ - String getString(String path); - - /** - * @param path - * path expression - * @return the {@link ConfigObject} value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to an object - */ - ConfigObject getObject(String path); - - /** - * @param path - * path expression - * @return the nested {@code Config} value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to a Config - */ - Config getConfig(String path); - - /** - * Gets the value at the path as an unwrapped Java boxed value ( - * {@link java.lang.Boolean Boolean}, {@link java.lang.Integer Integer}, and - * so on - see {@link ConfigValue#unwrapped()}). - * - * @param path - * path expression - * @return the unwrapped value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - */ - Object getAnyRef(String path); - - /** - * Gets the value at the given path, unless the value is a - * null value or missing, in which case it throws just like - * the other getters. Use {@code get()} on the {@link - * Config#root()} object (or other object in the tree) if you - * want an unprocessed value. - * - * @param path - * path expression - * @return the value at the requested path - * @throws ConfigException.Missing - * if value is absent or null - */ - ConfigValue getValue(String path); - - /** - * Gets a value as a size in bytes (parses special strings like "128M"). If - * the value is already a number, then it's left alone; if it's a string, - * it's parsed understanding unit suffixes such as "128K", as documented in - * the the - * spec. - * - * @param path - * path expression - * @return the value at the requested path, in bytes - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to Long or String - * @throws ConfigException.BadValue - * if value cannot be parsed as a size in bytes - */ - Long getBytes(String path); - - /** - * Get value as a duration in milliseconds. If the value is already a - * number, then it's left alone; if it's a string, it's parsed understanding - * units suffixes like "10m" or "5ns" as documented in the the - * spec. - * - * @param path - * path expression - * @return the duration value at the requested path, in milliseconds - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to Long or String - * @throws ConfigException.BadValue - * if value cannot be parsed as a number of milliseconds - */ - Long getMilliseconds(String path); - - /** - * Get value as a duration in nanoseconds. If the value is already a number - * it's taken as milliseconds and converted to nanoseconds. If it's a - * string, it's parsed understanding unit suffixes, as for - * {@link #getMilliseconds(String)}. - * - * @param path - * path expression - * @return the duration value at the requested path, in nanoseconds - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to Long or String - * @throws ConfigException.BadValue - * if value cannot be parsed as a number of nanoseconds - */ - Long getNanoseconds(String path); - - /** - * Gets a list value (with any element type) as a {@link ConfigList}, which - * implements {@code java.util.List}. Throws if the path is - * unset or null. - * - * @param path - * the path to the list value. - * @return the {@link ConfigList} at the path - * @throws ConfigException.Missing - * if value is absent or null - * @throws ConfigException.WrongType - * if value is not convertible to a ConfigList - */ - ConfigList getList(String path); - - List getBooleanList(String path); - - List getNumberList(String path); - - List getIntList(String path); - - List getLongList(String path); - - List getDoubleList(String path); - - List getStringList(String path); - - List getObjectList(String path); - - List getConfigList(String path); - - List getAnyRefList(String path); - - List getBytesList(String path); - - List getMillisecondsList(String path); - - List getNanosecondsList(String path); - - /** - * Clone the config with only the given path (and its children) retained; - * all sibling paths are removed. - * - * @param path - * path to keep - * @return a copy of the config minus all paths except the one specified - */ - Config withOnlyPath(String path); - - /** - * Clone the config with the given path removed. - * - * @param path - * path to remove - * @return a copy of the config minus the specified path - */ - Config withoutPath(String path); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigException.java b/akka-actor/src/main/java/com/typesafe/config/ConfigException.java deleted file mode 100755 index 3c31d811c3..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigException.java +++ /dev/null @@ -1,366 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - - -/** - * All exceptions thrown by the library are subclasses of - * ConfigException. - */ -public abstract class ConfigException extends RuntimeException { - private static final long serialVersionUID = 1L; - - final private ConfigOrigin origin; - - protected ConfigException(ConfigOrigin origin, String message, - Throwable cause) { - super(origin.description() + ": " + message, cause); - this.origin = origin; - } - - protected ConfigException(ConfigOrigin origin, String message) { - this(origin.description() + ": " + message, null); - } - - protected ConfigException(String message, Throwable cause) { - super(message, cause); - this.origin = null; - } - - protected ConfigException(String message) { - this(message, null); - } - - /** - * Returns an "origin" (such as a filename and line number) for the - * exception, or null if none is available. If there's no sensible origin - * for a given exception, or the kind of exception doesn't meaningfully - * relate to a particular origin file, this returns null. Never assume this - * will return non-null, it can always return null. - * - * @return origin of the problem, or null if unknown/inapplicable - */ - public ConfigOrigin origin() { - return origin; - } - - /** - * Exception indicating that the type of a value does not match the type you - * requested. - * - */ - public static class WrongType extends ConfigException { - private static final long serialVersionUID = 1L; - - public WrongType(ConfigOrigin origin, String path, String expected, - String actual, - Throwable cause) { - super(origin, path + " has type " + actual + " rather than " - + expected, - cause); - } - - public WrongType(ConfigOrigin origin, String path, String expected, - String actual) { - this(origin, path, expected, actual, null); - } - - WrongType(ConfigOrigin origin, String message, Throwable cause) { - super(origin, message, cause); - } - - WrongType(ConfigOrigin origin, String message) { - this(origin, message, null); - } - } - - /** - * Exception indicates that the setting was never set to anything, not even - * null. - */ - public static class Missing extends ConfigException { - private static final long serialVersionUID = 1L; - - public Missing(String path, Throwable cause) { - super("No configuration setting found for key '" + path + "'", - cause); - } - - public Missing(String path) { - this(path, null); - } - - protected Missing(ConfigOrigin origin, String message, Throwable cause) { - super(origin, message, cause); - } - - protected Missing(ConfigOrigin origin, String message) { - this(origin, message, null); - } - } - - /** - * Exception indicates that the setting was treated as missing because it - * was set to null. - */ - public static class Null extends Missing { - private static final long serialVersionUID = 1L; - - private static String makeMessage(String path, String expected) { - if (expected != null) { - return "Configuration key '" + path - + "' is set to null but expected " + expected; - } else { - return "Configuration key '" + path + "' is null"; - } - } - - public Null(ConfigOrigin origin, String path, String expected, - Throwable cause) { - super(origin, makeMessage(path, expected), cause); - } - - public Null(ConfigOrigin origin, String path, String expected) { - this(origin, path, expected, null); - } - } - - /** - * Exception indicating that a value was messed up, for example you may have - * asked for a duration and the value can't be sensibly parsed as a - * duration. - * - */ - public static class BadValue extends ConfigException { - private static final long serialVersionUID = 1L; - - public BadValue(ConfigOrigin origin, String path, String message, - Throwable cause) { - super(origin, "Invalid value at '" + path + "': " + message, cause); - } - - public BadValue(ConfigOrigin origin, String path, String message) { - this(origin, path, message, null); - } - - public BadValue(String path, String message, Throwable cause) { - super("Invalid value at '" + path + "': " + message, cause); - } - - public BadValue(String path, String message) { - this(path, message, null); - } - } - - /** - * Exception indicating that a path expression was invalid. Try putting - * double quotes around path elements that contain "special" characters. - * - */ - public static class BadPath extends ConfigException { - private static final long serialVersionUID = 1L; - - public BadPath(ConfigOrigin origin, String path, String message, - Throwable cause) { - super(origin, - path != null ? ("Invalid path '" + path + "': " + message) - : message, cause); - } - - public BadPath(ConfigOrigin origin, String path, String message) { - this(origin, path, message, null); - } - - public BadPath(String path, String message, Throwable cause) { - super(path != null ? ("Invalid path '" + path + "': " + message) - : message, cause); - } - - public BadPath(String path, String message) { - this(path, message, null); - } - - public BadPath(ConfigOrigin origin, String message) { - this(origin, null, message); - } - } - - /** - * Exception indicating that there's a bug in something (possibly the - * library itself) or the runtime environment is broken. This exception - * should never be handled; instead, something should be fixed to keep the - * exception from occurring. This exception can be thrown by any method in - * the library. - */ - public static class BugOrBroken extends ConfigException { - private static final long serialVersionUID = 1L; - - public BugOrBroken(String message, Throwable cause) { - super(message, cause); - } - - public BugOrBroken(String message) { - this(message, null); - } - } - - /** - * Exception indicating that there was an IO error. - * - */ - public static class IO extends ConfigException { - private static final long serialVersionUID = 1L; - - public IO(ConfigOrigin origin, String message, Throwable cause) { - super(origin, message, cause); - } - - public IO(ConfigOrigin origin, String message) { - this(origin, message, null); - } - } - - /** - * Exception indicating that there was a parse error. - * - */ - public static class Parse extends ConfigException { - private static final long serialVersionUID = 1L; - - public Parse(ConfigOrigin origin, String message, Throwable cause) { - super(origin, message, cause); - } - - public Parse(ConfigOrigin origin, String message) { - this(origin, message, null); - } - } - - /** - * Exception indicating that a substitution did not resolve to anything. - * Thrown by {@link Config#resolve}. - */ - public static class UnresolvedSubstitution extends Parse { - private static final long serialVersionUID = 1L; - - public UnresolvedSubstitution(ConfigOrigin origin, String expression, Throwable cause) { - super(origin, "Could not resolve substitution to a value: " + expression, cause); - } - - public UnresolvedSubstitution(ConfigOrigin origin, String expression) { - this(origin, expression, null); - } - } - - /** - * Exception indicating that you tried to use a function that requires - * substitutions to be resolved, but substitutions have not been resolved - * (that is, {@link Config#resolve} was not called). This is always a bug in - * either application code or the library; it's wrong to write a handler for - * this exception because you should be able to fix the code to avoid it by - * adding calls to {@link Config#resolve}. - */ - public static class NotResolved extends BugOrBroken { - private static final long serialVersionUID = 1L; - - public NotResolved(String message, Throwable cause) { - super(message, cause); - } - - public NotResolved(String message) { - this(message, null); - } - } - - /** - * Information about a problem that occurred in {@link Config#checkValid}. A - * {@link ConfigException.ValidationFailed} exception thrown from - * checkValid() includes a list of problems encountered. - */ - public static class ValidationProblem { - - final private String path; - final private ConfigOrigin origin; - final private String problem; - - public ValidationProblem(String path, ConfigOrigin origin, String problem) { - this.path = path; - this.origin = origin; - this.problem = problem; - } - - /** Returns the config setting causing the problem. */ - public String path() { - return path; - } - - /** - * Returns where the problem occurred (origin may include info on the - * file, line number, etc.). - */ - public ConfigOrigin origin() { - return origin; - } - - /** Returns a description of the problem. */ - public String problem() { - return problem; - } - } - - /** - * Exception indicating that {@link Config#checkValid} found validity - * problems. The problems are available via the {@link #problems()} method. - * The getMessage() of this exception is a potentially very - * long string listing all the problems found. - */ - public static class ValidationFailed extends ConfigException { - private static final long serialVersionUID = 1L; - - final private Iterable problems; - - public ValidationFailed(Iterable problems) { - super(makeMessage(problems), null); - this.problems = problems; - } - - public Iterable problems() { - return problems; - } - - private static String makeMessage(Iterable problems) { - StringBuilder sb = new StringBuilder(); - for (ValidationProblem p : problems) { - sb.append(p.origin().description()); - sb.append(": "); - sb.append(p.path()); - sb.append(": "); - sb.append(p.problem()); - sb.append(", "); - } - if (sb.length() == 0) - throw new ConfigException.BugOrBroken( - "ValidationFailed must have a non-empty list of problems"); - sb.setLength(sb.length() - 2); // chop comma and space - - return sb.toString(); - } - } - - /** - * Exception that doesn't fall into any other category. - */ - public static class Generic extends ConfigException { - private static final long serialVersionUID = 1L; - - public Generic(String message, Throwable cause) { - super(message, cause); - } - - public Generic(String message) { - this(message, null); - } - } - -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java b/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java deleted file mode 100755 index 981708ae55..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java +++ /dev/null @@ -1,706 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -import java.io.File; -import java.io.Reader; -import java.net.MalformedURLException; -import java.net.URL; -import java.util.Map; -import java.util.Properties; - -import com.typesafe.config.impl.ConfigImpl; -import com.typesafe.config.impl.Parseable; - -/** - * Contains static methods for creating {@link Config} instances. - * - *

- * See also {@link ConfigValueFactory} which contains static methods for - * converting Java values into a {@link ConfigObject}. You can then convert a - * {@code ConfigObject} into a {@code Config} with {@link ConfigObject#toConfig}. - * - *

- * The static methods with "load" in the name do some sort of higher-level - * operation potentially parsing multiple resources and resolving substitutions, - * while the ones with "parse" in the name just create a {@link ConfigValue} - * from a resource and nothing else. - */ -public final class ConfigFactory { - private ConfigFactory() { - } - - /** - * Loads an application's configuration from the given classpath resource or - * classpath resource basename, sandwiches it between default reference - * config and default overrides, and then resolves it. The classpath - * resource is "raw" (it should have no "/" prefix, and is not made relative - * to any package, so it's like {@link ClassLoader#getResource} not - * {@link Class#getResource}). - * - *

- * Resources are loaded from the current thread's - * {@link Thread#getContextClassLoader()}. In general, a library needs its - * configuration to come from the class loader used to load that library, so - * the proper "reference.conf" are present. - * - *

- * The loaded object will already be resolved (substitutions have already - * been processed). As a result, if you add more fallbacks then they won't - * be seen by substitutions. Substitutions are the "${foo.bar}" syntax. If - * you want to parse additional files or something then you need to use - * {@link #load(Config)}. - * - * @param resourceBasename - * name (optionally without extension) of a resource on classpath - * @return configuration for an application relative to context class loader - */ - public static Config load(String resourceBasename) { - return load(Thread.currentThread().getContextClassLoader(), resourceBasename); - } - - /** - * Like {@link #load(String)} but uses the supplied class loader instead of - * the current thread's context class loader. - * - * @param loader - * @param resourceBasename - * @return configuration for an application relative to given class loader - */ - public static Config load(ClassLoader loader, String resourceBasename) { - return load(loader, resourceBasename, ConfigParseOptions.defaults(), - ConfigResolveOptions.defaults()); - } - - /** - * Like {@link #load(String)} but allows you to specify parse and resolve - * options. - * - * @param resourceBasename - * the classpath resource name with optional extension - * @param parseOptions - * options to use when parsing the resource - * @param resolveOptions - * options to use when resolving the stack - * @return configuration for an application - */ - public static Config load(String resourceBasename, ConfigParseOptions parseOptions, - ConfigResolveOptions resolveOptions) { - return load(Thread.currentThread().getContextClassLoader(), resourceBasename, parseOptions, - resolveOptions); - } - - /** - * Like {@link #load(String,ConfigParseOptions,ConfigResolveOptions)} but - * allows you to specify a class loader - * - * @param loader - * class loader in which to find resources - * @param resourceBasename - * the classpath resource name with optional extension - * @param parseOptions - * options to use when parsing the resource - * @param resolveOptions - * options to use when resolving the stack - * @return configuration for an application - */ - public static Config load(ClassLoader loader, String resourceBasename, - ConfigParseOptions parseOptions, ConfigResolveOptions resolveOptions) { - Config appConfig = ConfigFactory.parseResourcesAnySyntax(loader, resourceBasename, - parseOptions); - return load(loader, appConfig, resolveOptions); - } - - /** - * Assembles a standard configuration using a custom Config - * object rather than loading "application.conf". The Config - * object will be sandwiched between the default reference config and - * default overrides and then resolved. - * - * @param config - * the application's portion of the configuration - * @return resolved configuration with overrides and fallbacks added - */ - public static Config load(Config config) { - return load(Thread.currentThread().getContextClassLoader(), config); - } - - public static Config load(ClassLoader loader, Config config) { - return load(loader, config, ConfigResolveOptions.defaults()); - } - - /** - * Like {@link #load(Config)} but allows you to specify - * {@link ConfigResolveOptions}. - * - * @param config - * the application's portion of the configuration - * @param resolveOptions - * options for resolving the assembled config stack - * @return resolved configuration with overrides and fallbacks added - */ - public static Config load(Config config, ConfigResolveOptions resolveOptions) { - return load(Thread.currentThread().getContextClassLoader(), config, resolveOptions); - } - - /** - * Like {@link #load(Config,ConfigResolveOptions)} but allows you to specify - * a class loader other than the context class loader. - * - * @param loader - * class loader to use when looking up override and reference - * configs - * @param config - * the application's portion of the configuration - * @param resolveOptions - * options for resolving the assembled config stack - * @return resolved configuration with overrides and fallbacks added - */ - public static Config load(ClassLoader loader, Config config, ConfigResolveOptions resolveOptions) { - return defaultOverrides(loader).withFallback(config).withFallback(defaultReference(loader)) - .resolve(resolveOptions); - } - - private static Config loadDefaultConfig(ClassLoader loader) { - int specified = 0; - - // override application.conf with config.file, config.resource, - // config.url if requested. - String resource = System.getProperty("config.resource"); - if (resource != null) - specified += 1; - String file = System.getProperty("config.file"); - if (file != null) - specified += 1; - String url = System.getProperty("config.url"); - if (url != null) - specified += 1; - - if (specified == 0) { - return load(loader, "application"); - } else if (specified > 1) { - throw new ConfigException.Generic("You set more than one of config.file='" + file - + "', config.url='" + url + "', config.resource='" + resource - + "'; don't know which one to use!"); - } else { - if (resource != null) { - if (resource.startsWith("/")) - resource = resource.substring(1); - // this deliberately does not parseResourcesAnySyntax; if - // people want that they can use an include statement. - return load(loader, parseResources(loader, resource)); - } else if (file != null) { - return load(loader, parseFile(new File(file))); - } else { - try { - return load(loader, parseURL(new URL(url))); - } catch (MalformedURLException e) { - throw new ConfigException.Generic("Bad URL in config.url system property: '" - + url + "': " + e.getMessage(), e); - } - } - } - } - - /** - * Loads a default configuration, equivalent to {@link #load(String) - * load("application")} in most cases. This configuration should be used by - * libraries and frameworks unless an application provides a different one. - *

- * This method may return a cached singleton. - *

- * If the system properties config.resource, - * config.file, or config.url are set, then the - * classpath resource, file, or URL specified in those properties will be - * used rather than the default - * application.{conf,json,properties} classpath resources. - * These system properties should not be set in code (after all, you can - * just parse whatever you want manually and then use {@link #load(Config)} - * if you don't want to use application.conf). The properties - * are intended for use by the person or script launching the application. - * For example someone might have a production.conf that - * include application.conf but then change a couple of values. - * When launching the app they could specify - * -Dconfig.resource=production.conf to get production mode. - *

- * If no system properties are set to change the location of the default - * configuration, ConfigFactory.load() is equivalent to - * ConfigFactory.load("application"). - * - * @return configuration for an application - */ - public static Config load() { - return load(Thread.currentThread().getContextClassLoader()); - } - - /** - * Like {@link #load()} but allows specifying a class loader other than the - * thread's current context class loader. - * - * @param loader - * class loader for finding resources - * @return configuration for an application - */ - public static Config load(ClassLoader loader) { - return loadDefaultConfig(loader); - } - - /** - * Obtains the default reference configuration, which is currently created - * by merging all resources "reference.conf" found on the classpath and - * overriding the result with system properties. The returned reference - * configuration will already have substitutions resolved. - * - *

- * Libraries and frameworks should ship with a "reference.conf" in their - * jar. - * - *

- * The reference config must be looked up in the class loader that contains - * the libraries that you want to use with this config, so the - * "reference.conf" for each library can be found. Use - * {@link #defaultReference(ClassLoader)} if the context class loader is not - * suitable. - * - *

- * The {@link #load()} methods merge this configuration for you - * automatically. - * - *

- * Future versions may look for reference configuration in more places. It - * is not guaranteed that this method only looks at - * "reference.conf". - * - * @return the default reference config for context class loader - */ - public static Config defaultReference() { - return defaultReference(Thread.currentThread().getContextClassLoader()); - } - - /** - * Like {@link #defaultReference()} but allows you to specify a class loader - * to use rather than the current context class loader. - * - * @param loader - * @return the default reference config for this class loader - */ - public static Config defaultReference(ClassLoader loader) { - return ConfigImpl.defaultReference(loader); - } - - /** - * Obtains the default override configuration, which currently consists of - * system properties. The returned override configuration will already have - * substitutions resolved. - * - *

- * The {@link #load()} methods merge this configuration for you - * automatically. - * - *

- * Future versions may get overrides in more places. It is not guaranteed - * that this method only uses system properties. - * - * @return the default override configuration - */ - public static Config defaultOverrides() { - return systemProperties(); - } - - /** - * Like {@link #defaultOverrides()} but allows you to specify a class loader - * to use rather than the current context class loader. - * - * @param loader - * @return the default override configuration - */ - public static Config defaultOverrides(ClassLoader loader) { - return systemProperties(); - } - - /** - * Gets an empty configuration. See also {@link #empty(String)} to create an - * empty configuration with a description, which may improve user-visible - * error messages. - * - * @return an empty configuration - */ - public static Config empty() { - return empty(null); - } - - /** - * Gets an empty configuration with a description to be used to create a - * {@link ConfigOrigin} for this Config. The description should - * be very short and say what the configuration is, like "default settings" - * or "foo settings" or something. (Presumably you will merge some actual - * settings into this empty config using {@link Config#withFallback}, making - * the description more useful.) - * - * @param originDescription - * description of the config - * @return an empty configuration - */ - public static Config empty(String originDescription) { - return ConfigImpl.emptyConfig(originDescription); - } - - /** - * Gets a Config containing the system properties from - * {@link java.lang.System#getProperties()}, parsed and converted as with - * {@link #parseProperties}. This method can return a global immutable - * singleton, so it's preferred over parsing system properties yourself. - * - *

- * {@link #load} will include the system properties as overrides already, as - * will {@link #defaultReference} and {@link #defaultOverrides}. - * - *

- * Because this returns a singleton, it will not notice changes to system - * properties made after the first time this method is called. - * - * @return system properties parsed into a Config - */ - public static Config systemProperties() { - return ConfigImpl.systemPropertiesAsConfig(); - } - - /** - * Gets a Config containing the system's environment variables. - * This method can return a global immutable singleton. - * - *

- * Environment variables are used as fallbacks when resolving substitutions - * whether or not this object is included in the config being resolved, so - * you probably don't need to use this method for most purposes. It can be a - * nicer API for accessing environment variables than raw - * {@link java.lang.System#getenv(String)} though, since you can use methods - * such as {@link Config#getInt}. - * - * @return system environment variables parsed into a Config - */ - public static Config systemEnvironment() { - return ConfigImpl.envVariablesAsConfig(); - } - - /** - * Converts a Java {@link java.util.Properties} object to a - * {@link ConfigObject} using the rules documented in the HOCON - * spec. The keys in the Properties object are split on the - * period character '.' and treated as paths. The values will all end up as - * string values. If you have both "a=foo" and "a.b=bar" in your properties - * file, so "a" is both the object containing "b" and the string "foo", then - * the string value is dropped. - * - *

- * If you want to have System.getProperties() as a - * ConfigObject, it's better to use the {@link #systemProperties()} method - * which returns a cached global singleton. - * - * @param properties - * a Java Properties object - * @param options - * @return the parsed configuration - */ - public static Config parseProperties(Properties properties, - ConfigParseOptions options) { - return Parseable.newProperties(properties, options).parse().toConfig(); - } - - public static Config parseProperties(Properties properties) { - return parseProperties(properties, ConfigParseOptions.defaults()); - } - - public static Config parseReader(Reader reader, ConfigParseOptions options) { - return Parseable.newReader(reader, options).parse().toConfig(); - } - - public static Config parseReader(Reader reader) { - return parseReader(reader, ConfigParseOptions.defaults()); - } - - public static Config parseURL(URL url, ConfigParseOptions options) { - return Parseable.newURL(url, options).parse().toConfig(); - } - - public static Config parseURL(URL url) { - return parseURL(url, ConfigParseOptions.defaults()); - } - - public static Config parseFile(File file, ConfigParseOptions options) { - return Parseable.newFile(file, options).parse().toConfig(); - } - - public static Config parseFile(File file) { - return parseFile(file, ConfigParseOptions.defaults()); - } - - /** - * Parses a file with a flexible extension. If the fileBasename - * already ends in a known extension, this method parses it according to - * that extension (the file's syntax must match its extension). If the - * fileBasename does not end in an extension, it parses files - * with all known extensions and merges whatever is found. - * - *

- * In the current implementation, the extension ".conf" forces - * {@link ConfigSyntax#CONF}, ".json" forces {@link ConfigSyntax#JSON}, and - * ".properties" forces {@link ConfigSyntax#PROPERTIES}. When merging files, - * ".conf" falls back to ".json" falls back to ".properties". - * - *

- * Future versions of the implementation may add additional syntaxes or - * additional extensions. However, the ordering (fallback priority) of the - * three current extensions will remain the same. - * - *

- * If options forces a specific syntax, this method only parses - * files with an extension matching that syntax. - * - *

- * If {@link ConfigParseOptions#getAllowMissing options.getAllowMissing()} - * is true, then no files have to exist; if false, then at least one file - * has to exist. - * - * @param fileBasename - * a filename with or without extension - * @param options - * parse options - * @return the parsed configuration - */ - public static Config parseFileAnySyntax(File fileBasename, - ConfigParseOptions options) { - return ConfigImpl.parseFileAnySyntax(fileBasename, options).toConfig(); - } - - public static Config parseFileAnySyntax(File fileBasename) { - return parseFileAnySyntax(fileBasename, ConfigParseOptions.defaults()); - } - - /** - * Parses all resources on the classpath with the given name and merges them - * into a single Config. - * - *

- * If the resource name does not begin with a "/", it will have the supplied - * class's package added to it, in the same way as - * {@link java.lang.Class#getResource}. - * - *

- * Duplicate resources with the same name are merged such that ones returned - * earlier from {@link ClassLoader#getResources} fall back to (have higher - * priority than) the ones returned later. This implies that resources - * earlier in the classpath override those later in the classpath when they - * configure the same setting. However, in practice real applications may - * not be consistent about classpath ordering, so be careful. It may be best - * to avoid assuming too much. - * - * @param klass - * klass.getClassLoader() will be used to load - * resources, and non-absolute resource names will have this - * class's package added - * @param resource - * resource to look up, relative to klass's package - * or absolute starting with a "/" - * @param options - * parse options - * @return the parsed configuration - */ - public static Config parseResources(Class klass, String resource, - ConfigParseOptions options) { - return Parseable.newResources(klass, resource, options).parse() - .toConfig(); - } - - public static Config parseResources(Class klass, String resource) { - return parseResources(klass, resource, ConfigParseOptions.defaults()); - } - - /** - * Parses classpath resources with a flexible extension. In general, this - * method has the same behavior as - * {@link #parseFileAnySyntax(File,ConfigParseOptions)} but for classpath - * resources instead, as in {@link #parseResources}. - * - *

- * There is a thorny problem with this method, which is that - * {@link java.lang.ClassLoader#getResources} must be called separately for - * each possible extension. The implementation ends up with separate lists - * of resources called "basename.conf" and "basename.json" for example. As a - * result, the ideal ordering between two files with different extensions is - * unknown; there is no way to figure out how to merge the two lists in - * classpath order. To keep it simple, the lists are simply concatenated, - * with the same syntax priorities as - * {@link #parseFileAnySyntax(File,ConfigParseOptions) parseFileAnySyntax()} - * - all ".conf" resources are ahead of all ".json" resources which are - * ahead of all ".properties" resources. - * - * @param klass - * class which determines the ClassLoader and the - * package for relative resource names - * @param resourceBasename - * a resource name as in {@link java.lang.Class#getResource}, - * with or without extension - * @param options - * parse options - * @return the parsed configuration - */ - public static Config parseResourcesAnySyntax(Class klass, String resourceBasename, - ConfigParseOptions options) { - return ConfigImpl.parseResourcesAnySyntax(klass, resourceBasename, - options).toConfig(); - } - - public static Config parseResourcesAnySyntax(Class klass, String resourceBasename) { - return parseResourcesAnySyntax(klass, resourceBasename, ConfigParseOptions.defaults()); - } - - /** - * Parses all resources on the classpath with the given name and merges them - * into a single Config. - * - *

- * This works like {@link java.lang.ClassLoader#getResource}, not like - * {@link java.lang.Class#getResource}, so the name never begins with a - * slash. - * - *

- * See {@link #parseResources(Class,String,ConfigParseOptions)} for full - * details. - * - * @param loader - * will be used to load resources - * @param resource - * resource to look up - * @param options - * parse options - * @return the parsed configuration - */ - public static Config parseResources(ClassLoader loader, String resource, - ConfigParseOptions options) { - return Parseable.newResources(loader, resource, options).parse().toConfig(); - } - - public static Config parseResources(ClassLoader loader, String resource) { - return parseResources(loader, resource, ConfigParseOptions.defaults()); - } - - /** - * Parses classpath resources with a flexible extension. In general, this - * method has the same behavior as - * {@link #parseFileAnySyntax(File,ConfigParseOptions)} but for classpath - * resources instead, as in - * {@link #parseResources(ClassLoader,String,ConfigParseOptions)}. - * - *

- * {@link #parseResourcesAnySyntax(Class,String,ConfigParseOptions)} differs - * in the syntax for the resource name, but otherwise see - * {@link #parseResourcesAnySyntax(Class,String,ConfigParseOptions)} for - * some details and caveats on this method. - * - * @param loader - * class loader to look up resources in - * @param resourceBasename - * a resource name as in - * {@link java.lang.ClassLoader#getResource}, with or without - * extension - * @param options - * parse options - * @return the parsed configuration - */ - public static Config parseResourcesAnySyntax(ClassLoader loader, String resourceBasename, - ConfigParseOptions options) { - return ConfigImpl.parseResourcesAnySyntax(loader, resourceBasename, options).toConfig(); - } - - public static Config parseResourcesAnySyntax(ClassLoader loader, String resourceBasename) { - return parseResourcesAnySyntax(loader, resourceBasename, ConfigParseOptions.defaults()); - } - - /** - * Like {@link #parseResources(ClassLoader,String,ConfigParseOptions)} but - * uses thread's current context class loader. - */ - public static Config parseResources(String resource, ConfigParseOptions options) { - return Parseable - .newResources(Thread.currentThread().getContextClassLoader(), resource, options) - .parse().toConfig(); - } - - /** - * Like {@link #parseResources(ClassLoader,String)} but uses thread's - * current context class loader. - */ - public static Config parseResources(String resource) { - return parseResources(Thread.currentThread().getContextClassLoader(), resource, - ConfigParseOptions.defaults()); - } - - /** - * Like - * {@link #parseResourcesAnySyntax(ClassLoader,String,ConfigParseOptions)} - * but uses thread's current context class loader. - */ - public static Config parseResourcesAnySyntax(String resourceBasename, ConfigParseOptions options) { - return ConfigImpl.parseResourcesAnySyntax(Thread.currentThread().getContextClassLoader(), - resourceBasename, options).toConfig(); - } - - /** - * Like {@link #parseResourcesAnySyntax(ClassLoader,String)} but uses - * thread's current context class loader. - */ - public static Config parseResourcesAnySyntax(String resourceBasename) { - return parseResourcesAnySyntax(Thread.currentThread().getContextClassLoader(), - resourceBasename, ConfigParseOptions.defaults()); - } - - public static Config parseString(String s, ConfigParseOptions options) { - return Parseable.newString(s, options).parse().toConfig(); - } - - public static Config parseString(String s) { - return parseString(s, ConfigParseOptions.defaults()); - } - - /** - * Creates a {@code Config} based on a {@link java.util.Map} from paths to - * plain Java values. Similar to - * {@link ConfigValueFactory#fromMap(Map,String)}, except the keys in the - * map are path expressions, rather than keys; and correspondingly it - * returns a {@code Config} instead of a {@code ConfigObject}. This is more - * convenient if you are writing literal maps in code, and less convenient - * if you are getting your maps from some data source such as a parser. - * - *

- * An exception will be thrown (and it is a bug in the caller of the method) - * if a path is both an object and a value, for example if you had both - * "a=foo" and "a.b=bar", then "a" is both the string "foo" and the parent - * object of "b". The caller of this method should ensure that doesn't - * happen. - * - * @param values - * @param originDescription - * description of what this map represents, like a filename, or - * "default settings" (origin description is used in error - * messages) - * @return the map converted to a {@code Config} - */ - public static Config parseMap(Map values, - String originDescription) { - return ConfigImpl.fromPathMap(values, originDescription).toConfig(); - } - - /** - * See the other overload of {@link #parseMap(Map, String)} for details, - * this one just uses a default origin description. - * - * @param values - * @return the map converted to a {@code Config} - */ - public static Config parseMap(Map values) { - return parseMap(values, null); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java b/akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java deleted file mode 100755 index ac3644a5af..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java +++ /dev/null @@ -1,30 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - - -/** - * Context provided to a {@link ConfigIncluder}; this interface is only useful - * inside a {@code ConfigIncluder} implementation, and is not intended for apps - * to implement. - */ -public interface ConfigIncludeContext { - /** - * Tries to find a name relative to whatever is doing the including, for - * example in the same directory as the file doing the including. Returns - * null if it can't meaningfully create a relative name. The returned - * parseable may not exist; this function is not required to do any IO, just - * compute what the name would be. - * - * The passed-in filename has to be a complete name (with extension), not - * just a basename. (Include statements in config files are allowed to give - * just a basename.) - * - * @param filename - * the name to make relative to the resource doing the including - * @return parseable item relative to the resource doing the including, or - * null - */ - ConfigParseable relativeTo(String filename); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java b/akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java deleted file mode 100755 index 38e8e35a91..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -/** - * Implement this interface and provide an instance to - * {@link ConfigParseOptions#setIncluder ConfigParseOptions.setIncluder()} to - * customize handling of {@code include} statements in config files. - */ -public interface ConfigIncluder { - /** - * Returns a new includer that falls back to the given includer. This is how - * you can obtain the default includer; it will be provided as a fallback. - * It's up to your includer to chain to it if you want to. You might want to - * merge any files found by the fallback includer with any objects you load - * yourself. - * - * It's important to handle the case where you already have the fallback - * with a "return this", i.e. this method should not create a new object if - * the fallback is the same one you already have. The same fallback may be - * added repeatedly. - * - * @param fallback - * @return a new includer - */ - ConfigIncluder withFallback(ConfigIncluder fallback); - - /** - * Parses another item to be included. The returned object typically would - * not have substitutions resolved. You can throw a ConfigException here to - * abort parsing, or return an empty object, but may not return null. - * - * @param context - * some info about the include context - * @param what - * the include statement's argument - * @return a non-null ConfigObject - */ - ConfigObject include(ConfigIncludeContext context, String what); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigList.java b/akka-actor/src/main/java/com/typesafe/config/ConfigList.java deleted file mode 100755 index 5c694a508e..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigList.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -import java.util.List; - -/** - * Subtype of {@link ConfigValue} representing a list value, as in JSON's - * {@code [1,2,3]} syntax. - * - *

- * {@code ConfigList} implements {@code java.util.List} so you can - * use it like a regular Java list. Or call {@link #unwrapped()} to unwrap the - * list elements into plain Java values. - * - *

- * Like all {@link ConfigValue} subtypes, {@code ConfigList} is immutable. This - * makes it threadsafe and you never have to create "defensive copies." The - * mutator methods from {@link java.util.List} all throw - * {@link java.lang.UnsupportedOperationException}. - * - *

- * The {@link ConfigValue#valueType} method on a list returns - * {@link ConfigValueType#LIST}. - * - *

- * Do not implement {@code ConfigList}; it should only be implemented - * by the config library. Arbitrary implementations will not work because the - * library internals assume a specific concrete implementation. Also, this - * interface is likely to grow new methods over time, so third-party - * implementations will break. - * - */ -public interface ConfigList extends List, ConfigValue { - - /** - * Recursively unwraps the list, returning a list of plain Java values such - * as Integer or String or whatever is in the list. - */ - @Override - List unwrapped(); - -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java b/akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java deleted file mode 100755 index c760bf9d15..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -/** - * Marker for types whose instances can be merged, that is {@link Config} and - * {@link ConfigValue}. Instances of {@code Config} and {@code ConfigValue} can - * be combined into a single new instance using the - * {@link ConfigMergeable#withFallback withFallback()} method. - * - *

- * Do not implement this interface; it should only be implemented by - * the config library. Arbitrary implementations will not work because the - * library internals assume a specific concrete implementation. Also, this - * interface is likely to grow new methods over time, so third-party - * implementations will break. - */ -public interface ConfigMergeable { - /** - * Returns a new value computed by merging this value with another, with - * keys in this value "winning" over the other one. Only - * {@link ConfigObject} and {@link Config} instances do anything in this - * method (they need to merge the fallback keys into themselves). All other - * values just return the original value, since they automatically override - * any fallback. - * - *

- * The semantics of merging are described in the spec - * for HOCON. - * - *

- * Note that objects do not merge "across" non-objects; if you write - * object.withFallback(nonObject).withFallback(otherObject), - * then otherObject will simply be ignored. This is an - * intentional part of how merging works. Both non-objects, and any object - * which has fallen back to a non-object, block subsequent fallbacks. - * - * @param other - * an object whose keys should be used if the keys are not - * present in this one - * @return a new object (or the original one, if the fallback doesn't get - * used) - */ - ConfigMergeable withFallback(ConfigMergeable other); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java deleted file mode 100755 index 285bf04e7f..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java +++ /dev/null @@ -1,113 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -import java.util.Map; - -/** - * Subtype of {@link ConfigValue} representing an object (dictionary, map) - * value, as in JSON's { "a" : 42 } syntax. - * - *

- * {@code ConfigObject} implements {@code java.util.Map} so - * you can use it like a regular Java map. Or call {@link #unwrapped()} to - * unwrap the map to a map with plain Java values rather than - * {@code ConfigValue}. - * - *

- * Like all {@link ConfigValue} subtypes, {@code ConfigObject} is immutable. - * This makes it threadsafe and you never have to create "defensive copies." The - * mutator methods from {@link java.util.Map} all throw - * {@link java.lang.UnsupportedOperationException}. - * - *

- * The {@link ConfigValue#valueType} method on an object returns - * {@link ConfigValueType#OBJECT}. - * - *

- * In most cases you want to use the {@link Config} interface rather than this - * one. Call {@link #toConfig()} to convert a {@code ConfigObject} to a - * {@code Config}. - * - *

- * The API for a {@code ConfigObject} is in terms of keys, while the API for a - * {@link Config} is in terms of path expressions. Conceptually, - * {@code ConfigObject} is a tree of maps from keys to values, while a - * {@code Config} is a one-level map from paths to values. - * - *

- * Use {@link ConfigUtil#joinPath} and {@link ConfigUtil#splitPath} to convert - * between path expressions and individual path elements (keys). - * - *

- * A {@code ConfigObject} may contain null values, which will have - * {@link ConfigValue#valueType()} equal to {@link ConfigValueType#NULL}. If - * {@code get()} returns Java's null then the key was not present in the parsed - * file (or wherever this value tree came from). If {@code get()} returns a - * {@link ConfigValue} with type {@code ConfigValueType#NULL} then the key was - * set to null explicitly in the config file. - * - *

- * Do not implement {@code ConfigObject}; it should only be implemented - * by the config library. Arbitrary implementations will not work because the - * library internals assume a specific concrete implementation. Also, this - * interface is likely to grow new methods over time, so third-party - * implementations will break. - */ -public interface ConfigObject extends ConfigValue, Map { - - /** - * Converts this object to a {@link Config} instance, enabling you to use - * path expressions to find values in the object. This is a constant-time - * operation (it is not proportional to the size of the object). - * - * @return a {@link Config} with this object as its root - */ - Config toConfig(); - - /** - * Recursively unwraps the object, returning a map from String to whatever - * plain Java values are unwrapped from the object's values. - * - * @return a {@link java.util.Map} containing plain Java objects - */ - @Override - Map unwrapped(); - - @Override - ConfigObject withFallback(ConfigMergeable other); - - /** - * Gets a {@link ConfigValue} at the given key, or returns null if there is - * no value. The returned {@link ConfigValue} may have - * {@link ConfigValueType#NULL} or any other type, and the passed-in key - * must be a key in this object, rather than a path expression. - * - * @param key - * key to look up - * - * @return the value at the key or null if none - */ - @Override - ConfigValue get(Object key); - - /** - * Clone the object with only the given key (and its children) retained; all - * sibling keys are removed. - * - * @param key - * key to keep - * @return a copy of the object minus all keys except the one specified - */ - ConfigObject withOnlyKey(String key); - - /** - * Clone the object with the given key removed. - * - * @param key - * key to remove - * @return a copy of the object minus the specified key - */ - ConfigObject withoutKey(String key); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java b/akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java deleted file mode 100755 index c34767fb2e..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java +++ /dev/null @@ -1,82 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -import java.net.URL; -import java.util.List; - - -/** - * Represents the origin (such as filename and line number) of a - * {@link ConfigValue} for use in error messages. Obtain the origin of a value - * with {@link ConfigValue#origin}. Exceptions may have an origin, see - * {@link ConfigException#origin}, but be careful because - * ConfigException.origin() may return null. - * - *

- * It's best to use this interface only for debugging; its accuracy is - * "best effort" rather than guaranteed, and a potentially-noticeable amount of - * memory could probably be saved if origins were not kept around, so in the - * future there might be some option to discard origins. - * - *

- * Do not implement this interface; it should only be implemented by - * the config library. Arbitrary implementations will not work because the - * library internals assume a specific concrete implementation. Also, this - * interface is likely to grow new methods over time, so third-party - * implementations will break. - */ -public interface ConfigOrigin { - /** - * Returns a string describing the origin of a value or exception. This will - * never return null. - * - * @return string describing the origin - */ - public String description(); - - /** - * Returns a filename describing the origin. This will return null if the - * origin was not a file. - * - * @return filename of the origin or null - */ - public String filename(); - - /** - * Returns a URL describing the origin. This will return null if the origin - * has no meaningful URL. - * - * @return url of the origin or null - */ - public URL url(); - - /** - * Returns a classpath resource name describing the origin. This will return - * null if the origin was not a classpath resource. - * - * @return resource name of the origin or null - */ - public String resource(); - - /** - * Returns a line number where the value or exception originated. This will - * return -1 if there's no meaningful line number. - * - * @return line number or -1 if none is available - */ - public int lineNumber(); - - /** - * Returns any comments that appeared to "go with" this place in the file. - * Often an empty list, but never null. The details of this are subject to - * change, but at the moment comments that are immediately before an array - * element or object field, with no blank line after the comment, "go with" - * that element or field. - * - * @return any comments that seemed to "go with" this origin, empty list if - * none - */ - public List comments(); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java b/akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java deleted file mode 100755 index 2d057e812f..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java +++ /dev/null @@ -1,150 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - - -/** - * A set of options related to parsing. - * - *

- * This object is immutable, so the "setters" return a new object. - * - *

- * Here is an example of creating a custom {@code ConfigParseOptions}: - * - *

- *     ConfigParseOptions options = ConfigParseOptions.defaults()
- *         .setSyntax(ConfigSyntax.JSON)
- *         .setAllowMissing(false)
- * 
- * - */ -public final class ConfigParseOptions { - final ConfigSyntax syntax; - final String originDescription; - final boolean allowMissing; - final ConfigIncluder includer; - - protected ConfigParseOptions(ConfigSyntax syntax, String originDescription, - boolean allowMissing, ConfigIncluder includer) { - this.syntax = syntax; - this.originDescription = originDescription; - this.allowMissing = allowMissing; - this.includer = includer; - } - - public static ConfigParseOptions defaults() { - return new ConfigParseOptions(null, null, true, null); - } - - /** - * Set the file format. If set to null, try to guess from any available - * filename extension; if guessing fails, assume {@link ConfigSyntax#CONF}. - * - * @param syntax - * a syntax or {@code null} for best guess - * @return options with the syntax set - */ - public ConfigParseOptions setSyntax(ConfigSyntax syntax) { - if (this.syntax == syntax) - return this; - else - return new ConfigParseOptions(syntax, this.originDescription, - this.allowMissing, this.includer); - } - - public ConfigSyntax getSyntax() { - return syntax; - } - - /** - * Set a description for the thing being parsed. In most cases this will be - * set up for you to something like the filename, but if you provide just an - * input stream you might want to improve on it. Set to null to allow the - * library to come up with something automatically. This description is the - * basis for the {@link ConfigOrigin} of the parsed values. - * - * @param originDescription - * @return options with the origin description set - */ - public ConfigParseOptions setOriginDescription(String originDescription) { - if (this.originDescription == originDescription) - return this; - else if (this.originDescription != null && originDescription != null - && this.originDescription.equals(originDescription)) - return this; - else - return new ConfigParseOptions(this.syntax, originDescription, - this.allowMissing, this.includer); - } - - public String getOriginDescription() { - return originDescription; - } - - /** this is package-private, not public API */ - ConfigParseOptions withFallbackOriginDescription(String originDescription) { - if (this.originDescription == null) - return setOriginDescription(originDescription); - else - return this; - } - - /** - * Set to false to throw an exception if the item being parsed (for example - * a file) is missing. Set to true to just return an empty document in that - * case. - * - * @param allowMissing - * @return options with the "allow missing" flag set - */ - public ConfigParseOptions setAllowMissing(boolean allowMissing) { - if (this.allowMissing == allowMissing) - return this; - else - return new ConfigParseOptions(this.syntax, this.originDescription, - allowMissing, this.includer); - } - - public boolean getAllowMissing() { - return allowMissing; - } - - /** - * Set a ConfigIncluder which customizes how includes are handled. - * - * @param includer - * @return new version of the parse options with different includer - */ - public ConfigParseOptions setIncluder(ConfigIncluder includer) { - if (this.includer == includer) - return this; - else - return new ConfigParseOptions(this.syntax, this.originDescription, - this.allowMissing, includer); - } - - public ConfigParseOptions prependIncluder(ConfigIncluder includer) { - if (this.includer == includer) - return this; - else if (this.includer != null) - return setIncluder(includer.withFallback(this.includer)); - else - return setIncluder(includer); - } - - public ConfigParseOptions appendIncluder(ConfigIncluder includer) { - if (this.includer == includer) - return this; - else if (this.includer != null) - return setIncluder(this.includer.withFallback(includer)); - else - return setIncluder(includer); - } - - public ConfigIncluder getIncluder() { - return includer; - } - -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java b/akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java deleted file mode 100755 index 8c19085513..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java +++ /dev/null @@ -1,42 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - - -/** - * An opaque handle to something that can be parsed, obtained from - * {@link ConfigIncludeContext}. - * - *

- * Do not implement this interface; it should only be implemented by - * the config library. Arbitrary implementations will not work because the - * library internals assume a specific concrete implementation. Also, this - * interface is likely to grow new methods over time, so third-party - * implementations will break. - */ -public interface ConfigParseable { - /** - * Parse whatever it is. The options should come from - * {@link ConfigParseable#options options()} but you could tweak them if you - * like. - * - * @param options - * parse options, should be based on the ones from - * {@link ConfigParseable#options options()} - */ - ConfigObject parse(ConfigParseOptions options); - - /** - * Returns a {@link ConfigOrigin} describing the origin of the parseable - * item. - */ - ConfigOrigin origin(); - - /** - * Get the initial options, which can be modified then passed to parse(). - * These options will have the right description, includer, and other - * parameters already set up. - */ - ConfigParseOptions options(); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java b/akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java deleted file mode 100755 index d82a6be71a..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java +++ /dev/null @@ -1,75 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -/** - * A set of options related to resolving substitutions. Substitutions use the - * ${foo.bar} syntax and are documented in the HOCON - * spec. - *

- * This object is immutable, so the "setters" return a new object. - *

- * Here is an example of creating a custom {@code ConfigResolveOptions}: - * - *

- *     ConfigResolveOptions options = ConfigResolveOptions.defaults()
- *         .setUseSystemEnvironment(false)
- * 
- *

- * In addition to {@link ConfigResolveOptions#defaults}, there's a prebuilt - * {@link ConfigResolveOptions#noSystem} which avoids looking at any system - * environment variables or other external system information. (Right now, - * environment variables are the only example.) - */ -public final class ConfigResolveOptions { - private final boolean useSystemEnvironment; - - private ConfigResolveOptions(boolean useSystemEnvironment) { - this.useSystemEnvironment = useSystemEnvironment; - } - - /** - * Returns the default resolve options. - * - * @return the default resolve options - */ - public static ConfigResolveOptions defaults() { - return new ConfigResolveOptions(true); - } - - /** - * Returns resolve options that disable any reference to "system" data - * (currently, this means environment variables). - * - * @return the resolve options with env variables disabled - */ - public static ConfigResolveOptions noSystem() { - return defaults().setUseSystemEnvironment(false); - } - - /** - * Returns options with use of environment variables set to the given value. - * - * @param value - * true to resolve substitutions falling back to environment - * variables. - * @return options with requested setting for use of environment variables - */ - @SuppressWarnings("static-method") - public ConfigResolveOptions setUseSystemEnvironment(boolean value) { - return new ConfigResolveOptions(value); - } - - /** - * Returns whether the options enable use of system environment variables. - * This method is mostly used by the config lib internally, not by - * applications. - * - * @return true if environment variables should be used - */ - public boolean getUseSystemEnvironment() { - return useSystemEnvironment; - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java b/akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java deleted file mode 100755 index 54529fad0b..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java +++ /dev/null @@ -1,32 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -/** - * The syntax of a character stream, JSON, HOCON - * aka ".conf", or Java properties. - * - */ -public enum ConfigSyntax { - /** - * Pedantically strict JSON format; no - * comments, no unexpected commas, no duplicate keys in the same object. - */ - JSON, - /** - * The JSON-superset HOCON format. - */ - CONF, - /** - * Standard Java properties format. - */ - PROPERTIES; -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigUtil.java b/akka-actor/src/main/java/com/typesafe/config/ConfigUtil.java deleted file mode 100755 index cc936923fe..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigUtil.java +++ /dev/null @@ -1,74 +0,0 @@ -package com.typesafe.config; - -import java.util.List; - -import com.typesafe.config.impl.ConfigImplUtil; - -/** - * Contains static utility methods. - * - */ -public final class ConfigUtil { - private ConfigUtil() { - - } - - /** - * Quotes and escapes a string, as in the JSON specification. - * - * @param s - * a string - * @return the string quoted and escaped - */ - public static String quoteString(String s) { - return ConfigImplUtil.renderJsonString(s); - } - - /** - * Converts a list of keys to a path expression, by quoting the path - * elements as needed and then joining them separated by a period. A path - * expression is usable with a {@link Config}, while individual path - * elements are usable with a {@link ConfigObject}. - * - * @param elements - * the keys in the path - * @return a path expression - * @throws ConfigException - * if there are no elements - */ - public static String joinPath(String... elements) { - return ConfigImplUtil.joinPath(elements); - } - - /** - * Converts a list of strings to a path expression, by quoting the path - * elements as needed and then joining them separated by a period. A path - * expression is usable with a {@link Config}, while individual path - * elements are usable with a {@link ConfigObject}. - * - * @param elements - * the keys in the path - * @return a path expression - * @throws ConfigException - * if the list is empty - */ - public static String joinPath(List elements) { - return ConfigImplUtil.joinPath(elements); - } - - /** - * Converts a path expression into a list of keys, by splitting on period - * and unquoting the individual path elements. A path expression is usable - * with a {@link Config}, while individual path elements are usable with a - * {@link ConfigObject}. - * - * @param path - * a path expression - * @return the individual keys in the path - * @throws ConfigException - * if the path expression is invalid - */ - public static List splitPath(String path) { - return ConfigImplUtil.splitPath(path); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigValue.java b/akka-actor/src/main/java/com/typesafe/config/ConfigValue.java deleted file mode 100755 index 1f389be08f..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigValue.java +++ /dev/null @@ -1,60 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -/** - * An immutable value, following the JSON type - * schema. - * - *

- * Because this object is immutable, it is safe to use from multiple threads and - * there's no need for "defensive copies." - * - *

- * Do not implement {@code ConfigValue}; it should only be implemented - * by the config library. Arbitrary implementations will not work because the - * library internals assume a specific concrete implementation. Also, this - * interface is likely to grow new methods over time, so third-party - * implementations will break. - */ -public interface ConfigValue extends ConfigMergeable { - /** - * The origin of the value (file, line number, etc.), for debugging and - * error messages. - * - * @return where the value came from - */ - ConfigOrigin origin(); - - /** - * The {@link ConfigValueType} of the value; matches the JSON type schema. - * - * @return value's type - */ - ConfigValueType valueType(); - - /** - * Returns the value as a plain Java boxed value, that is, a {@code String}, - * {@code Number}, {@code Boolean}, {@code Map}, - * {@code List}, or {@code null}, matching the {@link #valueType()} - * of this {@code ConfigValue}. If the value is a {@link ConfigObject} or - * {@link ConfigList}, it is recursively unwrapped. - */ - Object unwrapped(); - - /** - * Renders the config value as a HOCON string. This method is primarily - * intended for debugging, so it tries to add helpful comments and - * whitespace. If the config value has not been resolved (see - * {@link Config#resolve}), it's possible that it can't be rendered as valid - * HOCON. In that case the rendering should still be useful for debugging - * but you might not be able to parse it. - * - * @return the rendered value - */ - String render(); - - @Override - ConfigValue withFallback(ConfigMergeable other); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java b/akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java deleted file mode 100755 index babace186e..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java +++ /dev/null @@ -1,141 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -import java.util.Map; - -import com.typesafe.config.impl.ConfigImpl; - -/** - * This class holds some static factory methods for building {@link ConfigValue} - * instances. See also {@link ConfigFactory} which has methods for parsing files - * and certain in-memory data structures. - */ -public final class ConfigValueFactory { - private ConfigValueFactory() { - } - - /** - * Creates a ConfigValue from a plain Java boxed value, which may be a - * Boolean, Number, String, Map, Iterable, or null. A Map must be a Map from - * String to more values that can be supplied to fromAnyRef(). An Iterable - * must iterate over more values that can be supplied to fromAnyRef(). A Map - * will become a ConfigObject and an Iterable will become a ConfigList. If - * the Iterable is not an ordered collection, results could be strange, - * since ConfigList is ordered. - * - *

- * In a Map passed to fromAnyRef(), the map's keys are plain keys, not path - * expressions. So if your Map has a key "foo.bar" then you will get one - * object with a key called "foo.bar", rather than an object with a key - * "foo" containing another object with a key "bar". - * - *

- * The originDescription will be used to set the origin() field on the - * ConfigValue. It should normally be the name of the file the values came - * from, or something short describing the value such as "default settings". - * The originDescription is prefixed to error messages so users can tell - * where problematic values are coming from. - * - *

- * Supplying the result of ConfigValue.unwrapped() to this function is - * guaranteed to work and should give you back a ConfigValue that matches - * the one you unwrapped. The re-wrapped ConfigValue will lose some - * information that was present in the original such as its origin, but it - * will have matching values. - * - *

- * This function throws if you supply a value that cannot be converted to a - * ConfigValue, but supplying such a value is a bug in your program, so you - * should never handle the exception. Just fix your program (or report a bug - * against this library). - * - * @param object - * object to convert to ConfigValue - * @param originDescription - * name of origin file or brief description of what the value is - * @return a new value - */ - public static ConfigValue fromAnyRef(Object object, String originDescription) { - return ConfigImpl.fromAnyRef(object, originDescription); - } - - /** - * See the fromAnyRef() documentation for details. This is a typesafe - * wrapper that only works on {@link java.util.Map} and returns - * {@link ConfigObject} rather than {@link ConfigValue}. - * - *

- * If your Map has a key "foo.bar" then you will get one object with a key - * called "foo.bar", rather than an object with a key "foo" containing - * another object with a key "bar". The keys in the map are keys; not path - * expressions. That is, the Map corresponds exactly to a single - * {@code ConfigObject}. The keys will not be parsed or modified, and the - * values are wrapped in ConfigValue. To get nested {@code ConfigObject}, - * some of the values in the map would have to be more maps. - * - *

- * See also {@link ConfigFactory#parseMap(Map,String)} which interprets the - * keys in the map as path expressions. - * - * @param values - * @param originDescription - * @return a new {@link ConfigObject} value - */ - public static ConfigObject fromMap(Map values, - String originDescription) { - return (ConfigObject) fromAnyRef(values, originDescription); - } - - /** - * See the fromAnyRef() documentation for details. This is a typesafe - * wrapper that only works on {@link java.util.Iterable} and returns - * {@link ConfigList} rather than {@link ConfigValue}. - * - * @param values - * @param originDescription - * @return a new {@link ConfigList} value - */ - public static ConfigList fromIterable(Iterable values, - String originDescription) { - return (ConfigList) fromAnyRef(values, originDescription); - } - - /** - * See the other overload {@link #fromAnyRef(Object,String)} for details, - * this one just uses a default origin description. - * - * @param object - * @return a new {@link ConfigValue} - */ - public static ConfigValue fromAnyRef(Object object) { - return fromAnyRef(object, null); - } - - /** - * See the other overload {@link #fromMap(Map,String)} for details, this one - * just uses a default origin description. - * - *

- * See also {@link ConfigFactory#parseMap(Map)} which interprets the keys in - * the map as path expressions. - * - * @param values - * @return a new {@link ConfigObject} - */ - public static ConfigObject fromMap(Map values) { - return fromMap(values, null); - } - - /** - * See the other overload of {@link #fromIterable(Iterable, String)} for - * details, this one just uses a default origin description. - * - * @param values - * @return a new {@link ConfigList} - */ - public static ConfigList fromIterable(Iterable values) { - return fromIterable(values, null); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java b/akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java deleted file mode 100755 index a15774d3ce..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java +++ /dev/null @@ -1,12 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config; - -/** - * The type of a configuration value (following the JSON type schema). - */ -public enum ConfigValueType { - OBJECT, LIST, NUMBER, BOOLEAN, NULL, STRING -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java deleted file mode 100755 index c5031fe568..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java +++ /dev/null @@ -1,427 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigMergeable; -import com.typesafe.config.ConfigObject; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigResolveOptions; -import com.typesafe.config.ConfigValue; -import com.typesafe.config.ConfigValueType; - -abstract class AbstractConfigObject extends AbstractConfigValue implements - ConfigObject { - - private static final long serialVersionUID = 1L; - - final private SimpleConfig config; - - protected AbstractConfigObject(ConfigOrigin origin) { - super(origin); - this.config = new SimpleConfig(this); - } - - @Override - public SimpleConfig toConfig() { - return config; - } - - @Override - public AbstractConfigObject toFallbackValue() { - return this; - } - - @Override - abstract public AbstractConfigObject withOnlyKey(String key); - - @Override - abstract public AbstractConfigObject withoutKey(String key); - - abstract protected AbstractConfigObject withOnlyPathOrNull(Path path); - - abstract AbstractConfigObject withOnlyPath(Path path); - - abstract AbstractConfigObject withoutPath(Path path); - - /** - * This looks up the key with no transformation or type conversion of any - * kind, and returns null if the key is not present. - * - * @param key - * @return the unmodified raw value or null - */ - protected abstract AbstractConfigValue peek(String key); - - protected AbstractConfigValue peek(String key, - SubstitutionResolver resolver, int depth, - ConfigResolveOptions options) { - AbstractConfigValue v = peek(key); - - if (v != null && resolver != null) { - v = resolver.resolve(v, depth, options); - } - - return v; - } - - /** - * Looks up the path with no transformation, type conversion, or exceptions - * (just returns null if path not found). Does however resolve the path, if - * resolver != null. - */ - protected AbstractConfigValue peekPath(Path path, SubstitutionResolver resolver, - int depth, ConfigResolveOptions options) { - return peekPath(this, path, resolver, depth, options); - } - - AbstractConfigValue peekPath(Path path) { - return peekPath(this, path, null, 0, null); - } - - private static AbstractConfigValue peekPath(AbstractConfigObject self, Path path, - SubstitutionResolver resolver, int depth, - ConfigResolveOptions options) { - String key = path.first(); - Path next = path.remainder(); - - if (next == null) { - AbstractConfigValue v = self.peek(key, resolver, depth, options); - return v; - } else { - // it's important to ONLY resolve substitutions here, not - // all values, because if you resolve arrays or objects - // it creates unnecessary cycles as a side effect (any sibling - // of the object we want to follow could cause a cycle, not just - // the object we want to follow). - - ConfigValue v = self.peek(key); - - if (v instanceof ConfigSubstitution && resolver != null) { - v = resolver.resolve((AbstractConfigValue) v, depth, options); - } - - if (v instanceof AbstractConfigObject) { - return peekPath((AbstractConfigObject) v, next, resolver, - depth, options); - } else { - return null; - } - } - } - - @Override - public ConfigValueType valueType() { - return ConfigValueType.OBJECT; - } - - protected abstract AbstractConfigObject newCopy(ResolveStatus status, boolean ignoresFallbacks, - ConfigOrigin origin); - - @Override - protected AbstractConfigObject newCopy(boolean ignoresFallbacks, ConfigOrigin origin) { - return newCopy(resolveStatus(), ignoresFallbacks, origin); - } - - @Override - protected final AbstractConfigObject mergedWithTheUnmergeable(Unmergeable fallback) { - if (ignoresFallbacks()) - throw new ConfigException.BugOrBroken("should not be reached"); - - List stack = new ArrayList(); - if (this instanceof Unmergeable) { - stack.addAll(((Unmergeable) this).unmergedValues()); - } else { - stack.add(this); - } - stack.addAll(fallback.unmergedValues()); - return new ConfigDelayedMergeObject(mergeOrigins(stack), stack, - ((AbstractConfigValue) fallback).ignoresFallbacks()); - } - - @Override - protected AbstractConfigObject mergedWithObject(AbstractConfigObject fallback) { - if (ignoresFallbacks()) - throw new ConfigException.BugOrBroken("should not be reached"); - - boolean changed = false; - boolean allResolved = true; - Map merged = new HashMap(); - Set allKeys = new HashSet(); - allKeys.addAll(this.keySet()); - allKeys.addAll(fallback.keySet()); - for (String key : allKeys) { - AbstractConfigValue first = this.peek(key); - AbstractConfigValue second = fallback.peek(key); - AbstractConfigValue kept; - if (first == null) - kept = second; - else if (second == null) - kept = first; - else - kept = first.withFallback(second); - - merged.put(key, kept); - - if (first != kept) - changed = true; - - if (kept.resolveStatus() == ResolveStatus.UNRESOLVED) - allResolved = false; - } - - ResolveStatus newResolveStatus = ResolveStatus.fromBoolean(allResolved); - boolean newIgnoresFallbacks = fallback.ignoresFallbacks(); - - if (changed) - return new SimpleConfigObject(mergeOrigins(this, fallback), merged, newResolveStatus, - newIgnoresFallbacks); - else if (newResolveStatus != resolveStatus() || newIgnoresFallbacks != ignoresFallbacks()) - return newCopy(newResolveStatus, newIgnoresFallbacks, origin()); - else - return this; - } - - @Override - public AbstractConfigObject withFallback(ConfigMergeable mergeable) { - return (AbstractConfigObject) super.withFallback(mergeable); - } - - static ConfigOrigin mergeOrigins( - Collection stack) { - if (stack.isEmpty()) - throw new ConfigException.BugOrBroken( - "can't merge origins on empty list"); - List origins = new ArrayList(); - ConfigOrigin firstOrigin = null; - int numMerged = 0; - for (AbstractConfigValue v : stack) { - if (firstOrigin == null) - firstOrigin = v.origin(); - - if (v instanceof AbstractConfigObject - && ((AbstractConfigObject) v).resolveStatus() == ResolveStatus.RESOLVED - && ((ConfigObject) v).isEmpty()) { - // don't include empty files or the .empty() - // config in the description, since they are - // likely to be "implementation details" - } else { - origins.add(v.origin()); - numMerged += 1; - } - } - - if (numMerged == 0) { - // the configs were all empty, so just use the first one - origins.add(firstOrigin); - } - - return SimpleConfigOrigin.mergeOrigins(origins); - } - - static ConfigOrigin mergeOrigins(AbstractConfigObject... stack) { - return mergeOrigins(Arrays.asList(stack)); - } - - private AbstractConfigObject modify(Modifier modifier, - ResolveStatus newResolveStatus) { - Map changes = null; - for (String k : keySet()) { - AbstractConfigValue v = peek(k); - // "modified" may be null, which means remove the child; - // to do that we put null in the "changes" map. - AbstractConfigValue modified = modifier.modifyChild(v); - if (modified != v) { - if (changes == null) - changes = new HashMap(); - changes.put(k, modified); - } - } - if (changes == null) { - return newCopy(newResolveStatus, ignoresFallbacks(), origin()); - } else { - Map modified = new HashMap(); - for (String k : keySet()) { - if (changes.containsKey(k)) { - AbstractConfigValue newValue = changes.get(k); - if (newValue != null) { - modified.put(k, newValue); - } else { - // remove this child; don't put it in the new map. - } - } else { - modified.put(k, peek(k)); - } - } - return new SimpleConfigObject(origin(), modified, newResolveStatus, - ignoresFallbacks()); - } - } - - @Override - AbstractConfigObject resolveSubstitutions(final SubstitutionResolver resolver, - final int depth, - final ConfigResolveOptions options) { - if (resolveStatus() == ResolveStatus.RESOLVED) - return this; - - return modify(new Modifier() { - - @Override - public AbstractConfigValue modifyChild(AbstractConfigValue v) { - return resolver.resolve(v, depth, options); - } - - }, ResolveStatus.RESOLVED); - } - - @Override - AbstractConfigObject relativized(final Path prefix) { - return modify(new Modifier() { - - @Override - public AbstractConfigValue modifyChild(AbstractConfigValue v) { - return v.relativized(prefix); - } - - }, resolveStatus()); - } - - @Override - public AbstractConfigValue get(Object key) { - if (key instanceof String) - return peek((String) key); - else - return null; - } - - @Override - protected void render(StringBuilder sb, int indent, boolean formatted) { - if (isEmpty()) { - sb.append("{}"); - } else { - sb.append("{"); - if (formatted) - sb.append('\n'); - for (String k : keySet()) { - AbstractConfigValue v = peek(k); - if (formatted) { - indent(sb, indent + 1); - sb.append("# "); - sb.append(v.origin().description()); - sb.append("\n"); - for (String comment : v.origin().comments()) { - indent(sb, indent + 1); - sb.append("# "); - sb.append(comment); - sb.append("\n"); - } - indent(sb, indent + 1); - } - v.render(sb, indent + 1, k, formatted); - sb.append(","); - if (formatted) - sb.append('\n'); - } - // chop comma or newline - sb.setLength(sb.length() - 1); - if (formatted) { - sb.setLength(sb.length() - 1); // also chop comma - sb.append("\n"); // put a newline back - indent(sb, indent); - } - sb.append("}"); - } - } - - private static boolean mapEquals(Map a, - Map b) { - Set aKeys = a.keySet(); - Set bKeys = b.keySet(); - - if (!aKeys.equals(bKeys)) - return false; - - for (String key : aKeys) { - if (!a.get(key).equals(b.get(key))) - return false; - } - return true; - } - - private static int mapHash(Map m) { - // the keys have to be sorted, otherwise we could be equal - // to another map but have a different hashcode. - List keys = new ArrayList(); - keys.addAll(m.keySet()); - Collections.sort(keys); - - int valuesHash = 0; - for (String k : keys) { - valuesHash += m.get(k).hashCode(); - } - return 41 * (41 + keys.hashCode()) + valuesHash; - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof ConfigObject; - } - - @Override - public boolean equals(Object other) { - // note that "origin" is deliberately NOT part of equality. - // neither are other "extras" like ignoresFallbacks or resolve status. - if (other instanceof ConfigObject) { - // optimization to avoid unwrapped() for two ConfigObject, - // which is what AbstractConfigValue does. - return canEqual(other) && mapEquals(this, ((ConfigObject) other)); - } else { - return false; - } - } - - @Override - public int hashCode() { - // note that "origin" is deliberately NOT part of equality - // neither are other "extras" like ignoresFallbacks or resolve status. - return mapHash(this); - } - - private static UnsupportedOperationException weAreImmutable(String method) { - return new UnsupportedOperationException( - "ConfigObject is immutable, you can't call Map.'" + method - + "'"); - } - - @Override - public void clear() { - throw weAreImmutable("clear"); - } - - @Override - public ConfigValue put(String arg0, ConfigValue arg1) { - throw weAreImmutable("put"); - } - - @Override - public void putAll(Map arg0) { - throw weAreImmutable("putAll"); - } - - @Override - public ConfigValue remove(Object arg0) { - throw weAreImmutable("remove"); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java deleted file mode 100755 index 7eea5f0a8a..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java +++ /dev/null @@ -1,221 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.Serializable; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigMergeable; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigResolveOptions; -import com.typesafe.config.ConfigValue; - -/** - * - * Trying very hard to avoid a parent reference in config values; when you have - * a tree like this, the availability of parent() tends to result in a lot of - * improperly-factored and non-modular code. Please don't add parent(). - * - */ -abstract class AbstractConfigValue implements ConfigValue, MergeableValue, Serializable { - - private static final long serialVersionUID = 1L; - - final private SimpleConfigOrigin origin; - - AbstractConfigValue(ConfigOrigin origin) { - this.origin = (SimpleConfigOrigin) origin; - } - - @Override - public SimpleConfigOrigin origin() { - return this.origin; - } - - /** - * Called only by SubstitutionResolver object. - * - * @param resolver - * the resolver doing the resolving - * @param depth - * the number of substitutions followed in resolving the current - * one - * @param options - * whether to look at system props and env vars - * @return a new value if there were changes, or this if no changes - */ - AbstractConfigValue resolveSubstitutions(SubstitutionResolver resolver, - int depth, - ConfigResolveOptions options) { - return this; - } - - ResolveStatus resolveStatus() { - return ResolveStatus.RESOLVED; - } - - /** - * This is used when including one file in another; the included file is - * relativized to the path it's included into in the parent file. The point - * is that if you include a file at foo.bar in the parent, and the included - * file as a substitution ${a.b.c}, the included substitution now needs to - * be ${foo.bar.a.b.c} because we resolve substitutions globally only after - * parsing everything. - * - * @param prefix - * @return value relativized to the given path or the same value if nothing - * to do - */ - AbstractConfigValue relativized(Path prefix) { - return this; - } - - protected interface Modifier { - AbstractConfigValue modifyChild(AbstractConfigValue v); - } - - @Override - public AbstractConfigValue toFallbackValue() { - return this; - } - - protected abstract AbstractConfigValue newCopy(boolean ignoresFallbacks, ConfigOrigin origin); - - // this is virtualized rather than a field because only some subclasses - // really need to store the boolean, and they may be able to pack it - // with another boolean to save space. - protected boolean ignoresFallbacks() { - return true; - } - - private ConfigException badMergeException() { - if (ignoresFallbacks()) - throw new ConfigException.BugOrBroken( - "method should not have been called with ignoresFallbacks=true" - + getClass().getSimpleName()); - else - throw new ConfigException.BugOrBroken("should override this in " - + getClass().getSimpleName()); - } - - protected AbstractConfigValue mergedWithTheUnmergeable(Unmergeable fallback) { - throw badMergeException(); - } - - protected AbstractConfigValue mergedWithObject(AbstractConfigObject fallback) { - throw badMergeException(); - } - - protected AbstractConfigValue mergedWithNonObject(AbstractConfigValue fallback) { - // falling back to a non-object doesn't merge anything, and also - // prohibits merging any objects that we fall back to later. - // so we have to switch to ignoresFallbacks mode. - return newCopy(true /* ignoresFallbacks */, origin); - } - - public AbstractConfigValue withOrigin(ConfigOrigin origin) { - if (this.origin == origin) - return this; - else - return newCopy(ignoresFallbacks(), origin); - } - - @Override - public AbstractConfigValue withFallback(ConfigMergeable mergeable) { - if (ignoresFallbacks()) { - return this; - } else { - ConfigValue other = ((MergeableValue) mergeable).toFallbackValue(); - - if (other instanceof Unmergeable) { - return mergedWithTheUnmergeable((Unmergeable) other); - } else if (other instanceof AbstractConfigObject) { - AbstractConfigObject fallback = (AbstractConfigObject) other; - if (fallback.resolveStatus() == ResolveStatus.RESOLVED && fallback.isEmpty()) { - if (fallback.ignoresFallbacks()) - return newCopy(true /* ignoresFallbacks */, origin); - else - return this; - } else { - return mergedWithObject((AbstractConfigObject) other); - } - } else { - return mergedWithNonObject((AbstractConfigValue) other); - } - } - } - - protected boolean canEqual(Object other) { - return other instanceof ConfigValue; - } - - @Override - public boolean equals(Object other) { - // note that "origin" is deliberately NOT part of equality - if (other instanceof ConfigValue) { - return canEqual(other) - && (this.valueType() == - ((ConfigValue) other).valueType()) - && ConfigImplUtil.equalsHandlingNull(this.unwrapped(), - ((ConfigValue) other).unwrapped()); - } else { - return false; - } - } - - @Override - public int hashCode() { - // note that "origin" is deliberately NOT part of equality - Object o = this.unwrapped(); - if (o == null) - return 0; - else - return o.hashCode(); - } - - @Override - public final String toString() { - StringBuilder sb = new StringBuilder(); - render(sb, 0, null /* atKey */, false /* formatted */); - return getClass().getSimpleName() + "(" + sb.toString() + ")"; - } - - protected static void indent(StringBuilder sb, int indent) { - int remaining = indent; - while (remaining > 0) { - sb.append(" "); - --remaining; - } - } - - protected void render(StringBuilder sb, int indent, String atKey, boolean formatted) { - if (atKey != null) { - sb.append(ConfigImplUtil.renderJsonString(atKey)); - sb.append(" : "); - } - render(sb, indent, formatted); - } - - protected void render(StringBuilder sb, int indent, boolean formatted) { - Object u = unwrapped(); - sb.append(u.toString()); - } - - - @Override - public final String render() { - StringBuilder sb = new StringBuilder(); - render(sb, 0, null, true /* formatted */); - return sb.toString(); - } - - // toString() is a debugging-oriented string but this is defined - // to create a string that would parse back to the value in JSON. - // It only works for primitive values (that would be a single token) - // which are auto-converted to strings when concatenating with - // other strings or by the DefaultTransformer. - String transformToString() { - return null; - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java deleted file mode 100755 index 5a5c0f8ffd..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java +++ /dev/null @@ -1,39 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigValueType; - -final class ConfigBoolean extends AbstractConfigValue { - - private static final long serialVersionUID = 1L; - - final private boolean value; - - ConfigBoolean(ConfigOrigin origin, boolean value) { - super(origin); - this.value = value; - } - - @Override - public ConfigValueType valueType() { - return ConfigValueType.BOOLEAN; - } - - @Override - public Boolean unwrapped() { - return value; - } - - @Override - String transformToString() { - return value ? "true" : "false"; - } - - @Override - protected ConfigBoolean newCopy(boolean ignoresFallbacks, ConfigOrigin origin) { - return new ConfigBoolean(origin, value); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java deleted file mode 100755 index 49eae37c97..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java +++ /dev/null @@ -1,244 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.ObjectStreamException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigResolveOptions; -import com.typesafe.config.ConfigValueType; - -/** - * The issue here is that we want to first merge our stack of config files, and - * then we want to evaluate substitutions. But if two substitutions both expand - * to an object, we might need to merge those two objects. Thus, we can't ever - * "override" a substitution when we do a merge; instead we have to save the - * stack of values that should be merged, and resolve the merge when we evaluate - * substitutions. - */ -final class ConfigDelayedMerge extends AbstractConfigValue implements - Unmergeable { - - private static final long serialVersionUID = 1L; - - // earlier items in the stack win - final private List stack; - final private boolean ignoresFallbacks; - - ConfigDelayedMerge(ConfigOrigin origin, List stack, - boolean ignoresFallbacks) { - super(origin); - this.stack = stack; - this.ignoresFallbacks = ignoresFallbacks; - if (stack.isEmpty()) - throw new ConfigException.BugOrBroken( - "creating empty delayed merge value"); - - for (AbstractConfigValue v : stack) { - if (v instanceof ConfigDelayedMerge || v instanceof ConfigDelayedMergeObject) - throw new ConfigException.BugOrBroken( - "placed nested DelayedMerge in a ConfigDelayedMerge, should have consolidated stack"); - } - } - - ConfigDelayedMerge(ConfigOrigin origin, List stack) { - this(origin, stack, false /* ignoresFallbacks */); - } - - @Override - public ConfigValueType valueType() { - throw new ConfigException.NotResolved( - "called valueType() on value with unresolved substitutions, need to resolve first"); - } - - @Override - public Object unwrapped() { - throw new ConfigException.NotResolved( - "called unwrapped() on value with unresolved substitutions, need to resolve first"); - } - - @Override - AbstractConfigValue resolveSubstitutions(SubstitutionResolver resolver, - int depth, ConfigResolveOptions options) { - return resolveSubstitutions(stack, resolver, depth, options); - } - - // static method also used by ConfigDelayedMergeObject - static AbstractConfigValue resolveSubstitutions( - List stack, SubstitutionResolver resolver, - int depth, ConfigResolveOptions options) { - // to resolve substitutions, we need to recursively resolve - // the stack of stuff to merge, and merge the stack so - // we won't be a delayed merge anymore. - - AbstractConfigValue merged = null; - for (AbstractConfigValue v : stack) { - AbstractConfigValue resolved = resolver.resolve(v, depth, options); - if (resolved != null) { - if (merged == null) - merged = resolved; - else - merged = merged.withFallback(resolved); - } - } - - return merged; - } - - @Override - ResolveStatus resolveStatus() { - return ResolveStatus.UNRESOLVED; - } - - @Override - ConfigDelayedMerge relativized(Path prefix) { - List newStack = new ArrayList(); - for (AbstractConfigValue o : stack) { - newStack.add(o.relativized(prefix)); - } - return new ConfigDelayedMerge(origin(), newStack, ignoresFallbacks); - } - - @Override - protected boolean ignoresFallbacks() { - return ignoresFallbacks; - } - - @Override - protected AbstractConfigValue newCopy(boolean newIgnoresFallbacks, ConfigOrigin newOrigin) { - return new ConfigDelayedMerge(newOrigin, stack, newIgnoresFallbacks); - } - - @Override - protected final ConfigDelayedMerge mergedWithTheUnmergeable(Unmergeable fallback) { - if (ignoresFallbacks) - throw new ConfigException.BugOrBroken("should not be reached"); - - // if we turn out to be an object, and the fallback also does, - // then a merge may be required; delay until we resolve. - List newStack = new ArrayList(); - newStack.addAll(stack); - newStack.addAll(fallback.unmergedValues()); - return new ConfigDelayedMerge(AbstractConfigObject.mergeOrigins(newStack), newStack, - ((AbstractConfigValue) fallback).ignoresFallbacks()); - } - - @Override - protected final ConfigDelayedMerge mergedWithObject(AbstractConfigObject fallback) { - if (ignoresFallbacks) - throw new ConfigException.BugOrBroken("should not be reached"); - - // if we turn out to be an object, and the fallback also does, - // then a merge may be required; delay until we resolve. - List newStack = new ArrayList(); - newStack.addAll(stack); - newStack.add(fallback); - return new ConfigDelayedMerge(AbstractConfigObject.mergeOrigins(newStack), newStack, - fallback.ignoresFallbacks()); - } - - @Override - public Collection unmergedValues() { - return stack; - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof ConfigDelayedMerge; - } - - @Override - public boolean equals(Object other) { - // note that "origin" is deliberately NOT part of equality - if (other instanceof ConfigDelayedMerge) { - return canEqual(other) - && this.stack.equals(((ConfigDelayedMerge) other).stack); - } else { - return false; - } - } - - @Override - public int hashCode() { - // note that "origin" is deliberately NOT part of equality - return stack.hashCode(); - } - - @Override - protected void render(StringBuilder sb, int indent, String atKey, boolean formatted) { - render(stack, sb, indent, atKey, formatted); - } - - // static method also used by ConfigDelayedMergeObject. - static void render(List stack, StringBuilder sb, int indent, String atKey, - boolean formatted) { - if (formatted) { - sb.append("# unresolved merge of " + stack.size() + " values follows (\n"); - if (atKey == null) { - indent(sb, indent); - sb.append("# this unresolved merge will not be parseable because it's at the root of the object\n"); - sb.append("# the HOCON format has no way to list multiple root objects in a single file\n"); - } - } - - List reversed = new ArrayList(); - reversed.addAll(stack); - Collections.reverse(reversed); - - int i = 0; - for (AbstractConfigValue v : reversed) { - if (formatted) { - indent(sb, indent); - if (atKey != null) { - sb.append("# unmerged value " + i + " for key " - + ConfigImplUtil.renderJsonString(atKey) + " from "); - } else { - sb.append("# unmerged value " + i + " from "); - } - i += 1; - sb.append(v.origin().description()); - sb.append("\n"); - for (String comment : v.origin().comments()) { - indent(sb, indent); - sb.append("# "); - sb.append(comment); - sb.append("\n"); - } - indent(sb, indent); - } - - if (atKey != null) { - sb.append(ConfigImplUtil.renderJsonString(atKey)); - sb.append(" : "); - } - v.render(sb, indent, formatted); - sb.append(","); - if (formatted) - sb.append('\n'); - } - // chop comma or newline - sb.setLength(sb.length() - 1); - if (formatted) { - sb.setLength(sb.length() - 1); // also chop comma - sb.append("\n"); // put a newline back - indent(sb, indent); - sb.append("# ) end of unresolved merge\n"); - } - } - - // This ridiculous hack is because some JDK versions apparently can't - // serialize an array, which is used to implement ArrayList and EmptyList. - // maybe - // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6446627 - private Object writeReplace() throws ObjectStreamException { - // switch to LinkedList - return new ConfigDelayedMerge(origin(), - new java.util.LinkedList(stack), ignoresFallbacks); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java deleted file mode 100755 index fa873d9df9..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java +++ /dev/null @@ -1,233 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.ObjectStreamException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigMergeable; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigResolveOptions; -import com.typesafe.config.ConfigValue; - -// This is just like ConfigDelayedMerge except we know statically -// that it will turn out to be an object. -final class ConfigDelayedMergeObject extends AbstractConfigObject implements - Unmergeable { - - private static final long serialVersionUID = 1L; - - final private List stack; - final private boolean ignoresFallbacks; - - ConfigDelayedMergeObject(ConfigOrigin origin, - List stack) { - this(origin, stack, false /* ignoresFallbacks */); - } - - ConfigDelayedMergeObject(ConfigOrigin origin, List stack, - boolean ignoresFallbacks) { - super(origin); - this.stack = stack; - this.ignoresFallbacks = ignoresFallbacks; - - if (stack.isEmpty()) - throw new ConfigException.BugOrBroken( - "creating empty delayed merge object"); - if (!(stack.get(0) instanceof AbstractConfigObject)) - throw new ConfigException.BugOrBroken( - "created a delayed merge object not guaranteed to be an object"); - - for (AbstractConfigValue v : stack) { - if (v instanceof ConfigDelayedMerge || v instanceof ConfigDelayedMergeObject) - throw new ConfigException.BugOrBroken( - "placed nested DelayedMerge in a ConfigDelayedMergeObject, should have consolidated stack"); - } - } - - @Override - protected ConfigDelayedMergeObject newCopy(ResolveStatus status, boolean ignoresFallbacks, - ConfigOrigin origin) { - if (status != resolveStatus()) - throw new ConfigException.BugOrBroken( - "attempt to create resolved ConfigDelayedMergeObject"); - return new ConfigDelayedMergeObject(origin, stack, ignoresFallbacks); - } - - @Override - AbstractConfigObject resolveSubstitutions(SubstitutionResolver resolver, - int depth, ConfigResolveOptions options) { - AbstractConfigValue merged = ConfigDelayedMerge.resolveSubstitutions( - stack, resolver, depth, - options); - if (merged instanceof AbstractConfigObject) { - return (AbstractConfigObject) merged; - } else { - throw new ConfigException.BugOrBroken( - "somehow brokenly merged an object and didn't get an object"); - } - } - - @Override - ResolveStatus resolveStatus() { - return ResolveStatus.UNRESOLVED; - } - - @Override - ConfigDelayedMergeObject relativized(Path prefix) { - List newStack = new ArrayList(); - for (AbstractConfigValue o : stack) { - newStack.add(o.relativized(prefix)); - } - return new ConfigDelayedMergeObject(origin(), newStack, - ignoresFallbacks); - } - - @Override - protected boolean ignoresFallbacks() { - return ignoresFallbacks; - } - - @Override - protected ConfigDelayedMergeObject mergedWithObject(AbstractConfigObject fallback) { - if (ignoresFallbacks) - throw new ConfigException.BugOrBroken("should not be reached"); - - // since we are an object, and the fallback is, we'll need to - // merge the fallback once we resolve. - List newStack = new ArrayList(); - newStack.addAll(stack); - newStack.add(fallback); - return new ConfigDelayedMergeObject(AbstractConfigObject.mergeOrigins(newStack), newStack, - fallback.ignoresFallbacks()); - } - - @Override - public ConfigDelayedMergeObject withFallback(ConfigMergeable mergeable) { - return (ConfigDelayedMergeObject) super.withFallback(mergeable); - } - - @Override - public ConfigDelayedMergeObject withOnlyKey(String key) { - throw notResolved(); - } - - @Override - public ConfigDelayedMergeObject withoutKey(String key) { - throw notResolved(); - } - - @Override - protected AbstractConfigObject withOnlyPathOrNull(Path path) { - throw notResolved(); - } - - @Override - AbstractConfigObject withOnlyPath(Path path) { - throw notResolved(); - } - - @Override - AbstractConfigObject withoutPath(Path path) { - throw notResolved(); - } - - @Override - public Collection unmergedValues() { - return stack; - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof ConfigDelayedMergeObject; - } - - @Override - public boolean equals(Object other) { - // note that "origin" is deliberately NOT part of equality - if (other instanceof ConfigDelayedMergeObject) { - return canEqual(other) - && this.stack - .equals(((ConfigDelayedMergeObject) other).stack); - } else { - return false; - } - } - - @Override - public int hashCode() { - // note that "origin" is deliberately NOT part of equality - return stack.hashCode(); - } - - @Override - protected void render(StringBuilder sb, int indent, String atKey, boolean formatted) { - ConfigDelayedMerge.render(stack, sb, indent, atKey, formatted); - } - - private static ConfigException notResolved() { - return new ConfigException.NotResolved( - "bug: this object has not had substitutions resolved, so can't be used"); - } - - @Override - public Map unwrapped() { - throw notResolved(); - } - - @Override - public boolean containsKey(Object key) { - throw notResolved(); - } - - @Override - public boolean containsValue(Object value) { - throw notResolved(); - } - - @Override - public Set> entrySet() { - throw notResolved(); - } - - @Override - public boolean isEmpty() { - throw notResolved(); - } - - @Override - public Set keySet() { - throw notResolved(); - } - - @Override - public int size() { - throw notResolved(); - } - - @Override - public Collection values() { - throw notResolved(); - } - - @Override - protected AbstractConfigValue peek(String key) { - throw notResolved(); - } - - // This ridiculous hack is because some JDK versions apparently can't - // serialize an array, which is used to implement ArrayList and EmptyList. - // maybe - // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6446627 - private Object writeReplace() throws ObjectStreamException { - // switch to LinkedList - return new ConfigDelayedMergeObject(origin(), - new java.util.LinkedList(stack), ignoresFallbacks); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java deleted file mode 100755 index 8b99db9640..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java +++ /dev/null @@ -1,53 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigValueType; - -final class ConfigDouble extends ConfigNumber { - - private static final long serialVersionUID = 1L; - - final private double value; - - ConfigDouble(ConfigOrigin origin, double value, String originalText) { - super(origin, originalText); - this.value = value; - } - - @Override - public ConfigValueType valueType() { - return ConfigValueType.NUMBER; - } - - @Override - public Double unwrapped() { - return value; - } - - @Override - String transformToString() { - String s = super.transformToString(); - if (s == null) - return Double.toString(value); - else - return s; - } - - @Override - protected long longValue() { - return (long) value; - } - - @Override - protected double doubleValue() { - return value; - } - - @Override - protected ConfigDouble newCopy(boolean ignoresFallbacks, ConfigOrigin origin) { - return new ConfigDouble(origin, value, originalText); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java deleted file mode 100755 index fd41dda6a4..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java +++ /dev/null @@ -1,419 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.File; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -import com.typesafe.config.Config; -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigIncludeContext; -import com.typesafe.config.ConfigIncluder; -import com.typesafe.config.ConfigObject; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigParseOptions; -import com.typesafe.config.ConfigParseable; -import com.typesafe.config.ConfigSyntax; -import com.typesafe.config.ConfigValue; - -/** This is public but is only supposed to be used by the "config" package */ -public class ConfigImpl { - - private interface NameSource { - ConfigParseable nameToParseable(String name); - } - - // this function is a little tricky because there are three places we're - // trying to use it; for 'include "basename"' in a .conf file, for - // loading app.{conf,json,properties} from classpath, and for - // loading app.{conf,json,properties} from the filesystem. - private static ConfigObject fromBasename(NameSource source, String name, - ConfigParseOptions options) { - ConfigObject obj; - if (name.endsWith(".conf") || name.endsWith(".json") - || name.endsWith(".properties")) { - ConfigParseable p = source.nameToParseable(name); - - obj = p.parse(p.options().setAllowMissing(options.getAllowMissing())); - } else { - ConfigParseable confHandle = source.nameToParseable(name + ".conf"); - ConfigParseable jsonHandle = source.nameToParseable(name + ".json"); - ConfigParseable propsHandle = source.nameToParseable(name - + ".properties"); - boolean gotSomething = false; - List failMessages = new ArrayList(); - - ConfigSyntax syntax = options.getSyntax(); - - obj = SimpleConfigObject.empty(SimpleConfigOrigin.newSimple(name)); - if (syntax == null || syntax == ConfigSyntax.CONF) { - try { - obj = confHandle.parse(confHandle.options().setAllowMissing(false) - .setSyntax(ConfigSyntax.CONF)); - gotSomething = true; - } catch (ConfigException.IO e) { - failMessages.add(e.getMessage()); - } - } - - if (syntax == null || syntax == ConfigSyntax.JSON) { - try { - ConfigObject parsed = jsonHandle.parse(jsonHandle.options() - .setAllowMissing(false).setSyntax(ConfigSyntax.JSON)); - obj = obj.withFallback(parsed); - gotSomething = true; - } catch (ConfigException.IO e) { - failMessages.add(e.getMessage()); - } - } - - if (syntax == null || syntax == ConfigSyntax.PROPERTIES) { - try { - ConfigObject parsed = propsHandle.parse(propsHandle.options() - .setAllowMissing(false).setSyntax(ConfigSyntax.PROPERTIES)); - obj = obj.withFallback(parsed); - gotSomething = true; - } catch (ConfigException.IO e) { - failMessages.add(e.getMessage()); - } - } - - if (!options.getAllowMissing() && !gotSomething) { - String failMessage; - if (failMessages.isEmpty()) { - // this should not happen - throw new ConfigException.BugOrBroken( - "should not be reached: nothing found but no exceptions thrown"); - } else { - StringBuilder sb = new StringBuilder(); - for (String msg : failMessages) { - sb.append(msg); - sb.append(", "); - } - sb.setLength(sb.length() - 2); - failMessage = sb.toString(); - } - throw new ConfigException.IO(SimpleConfigOrigin.newSimple(name), failMessage); - } - } - - return obj; - } - - /** For use ONLY by library internals, DO NOT TOUCH not guaranteed ABI */ - public static ConfigObject parseResourcesAnySyntax(final Class klass, - String resourceBasename, final ConfigParseOptions baseOptions) { - NameSource source = new NameSource() { - @Override - public ConfigParseable nameToParseable(String name) { - return Parseable.newResources(klass, name, baseOptions); - } - }; - return fromBasename(source, resourceBasename, baseOptions); - } - - /** For use ONLY by library internals, DO NOT TOUCH not guaranteed ABI */ - public static ConfigObject parseResourcesAnySyntax(final ClassLoader loader, - String resourceBasename, final ConfigParseOptions baseOptions) { - NameSource source = new NameSource() { - @Override - public ConfigParseable nameToParseable(String name) { - return Parseable.newResources(loader, name, baseOptions); - } - }; - return fromBasename(source, resourceBasename, baseOptions); - } - - /** For use ONLY by library internals, DO NOT TOUCH not guaranteed ABI */ - public static ConfigObject parseFileAnySyntax(final File basename, - final ConfigParseOptions baseOptions) { - NameSource source = new NameSource() { - @Override - public ConfigParseable nameToParseable(String name) { - return Parseable.newFile(new File(name), baseOptions); - } - }; - return fromBasename(source, basename.getPath(), baseOptions); - } - - static AbstractConfigObject emptyObject(String originDescription) { - ConfigOrigin origin = originDescription != null ? SimpleConfigOrigin - .newSimple(originDescription) : null; - return emptyObject(origin); - } - - /** For use ONLY by library internals, DO NOT TOUCH not guaranteed ABI */ - public static Config emptyConfig(String originDescription) { - return emptyObject(originDescription).toConfig(); - } - - static AbstractConfigObject empty(ConfigOrigin origin) { - return emptyObject(origin); - } - - // default origin for values created with fromAnyRef and no origin specified - final private static ConfigOrigin defaultValueOrigin = SimpleConfigOrigin - .newSimple("hardcoded value"); - final private static ConfigBoolean defaultTrueValue = new ConfigBoolean( - defaultValueOrigin, true); - final private static ConfigBoolean defaultFalseValue = new ConfigBoolean( - defaultValueOrigin, false); - final private static ConfigNull defaultNullValue = new ConfigNull( - defaultValueOrigin); - final private static SimpleConfigList defaultEmptyList = new SimpleConfigList( - defaultValueOrigin, Collections. emptyList()); - final private static SimpleConfigObject defaultEmptyObject = SimpleConfigObject - .empty(defaultValueOrigin); - - private static SimpleConfigList emptyList(ConfigOrigin origin) { - if (origin == null || origin == defaultValueOrigin) - return defaultEmptyList; - else - return new SimpleConfigList(origin, - Collections. emptyList()); - } - - private static AbstractConfigObject emptyObject(ConfigOrigin origin) { - // we want null origin to go to SimpleConfigObject.empty() to get the - // origin "empty config" rather than "hardcoded value" - if (origin == defaultValueOrigin) - return defaultEmptyObject; - else - return SimpleConfigObject.empty(origin); - } - - private static ConfigOrigin valueOrigin(String originDescription) { - if (originDescription == null) - return defaultValueOrigin; - else - return SimpleConfigOrigin.newSimple(originDescription); - } - - /** For use ONLY by library internals, DO NOT TOUCH not guaranteed ABI */ - public static ConfigValue fromAnyRef(Object object, String originDescription) { - ConfigOrigin origin = valueOrigin(originDescription); - return fromAnyRef(object, origin, FromMapMode.KEYS_ARE_KEYS); - } - - /** For use ONLY by library internals, DO NOT TOUCH not guaranteed ABI */ - public static ConfigObject fromPathMap( - Map pathMap, String originDescription) { - ConfigOrigin origin = valueOrigin(originDescription); - return (ConfigObject) fromAnyRef(pathMap, origin, - FromMapMode.KEYS_ARE_PATHS); - } - - static AbstractConfigValue fromAnyRef(Object object, ConfigOrigin origin, - FromMapMode mapMode) { - if (origin == null) - throw new ConfigException.BugOrBroken( - "origin not supposed to be null"); - - if (object == null) { - if (origin != defaultValueOrigin) - return new ConfigNull(origin); - else - return defaultNullValue; - } else if (object instanceof Boolean) { - if (origin != defaultValueOrigin) { - return new ConfigBoolean(origin, (Boolean) object); - } else if ((Boolean) object) { - return defaultTrueValue; - } else { - return defaultFalseValue; - } - } else if (object instanceof String) { - return new ConfigString(origin, (String) object); - } else if (object instanceof Number) { - // here we always keep the same type that was passed to us, - // rather than figuring out if a Long would fit in an Int - // or a Double has no fractional part. i.e. deliberately - // not using ConfigNumber.newNumber() when we have a - // Double, Integer, or Long. - if (object instanceof Double) { - return new ConfigDouble(origin, (Double) object, null); - } else if (object instanceof Integer) { - return new ConfigInt(origin, (Integer) object, null); - } else if (object instanceof Long) { - return new ConfigLong(origin, (Long) object, null); - } else { - return ConfigNumber.newNumber(origin, - ((Number) object).doubleValue(), null); - } - } else if (object instanceof Map) { - if (((Map) object).isEmpty()) - return emptyObject(origin); - - if (mapMode == FromMapMode.KEYS_ARE_KEYS) { - Map values = new HashMap(); - for (Map.Entry entry : ((Map) object).entrySet()) { - Object key = entry.getKey(); - if (!(key instanceof String)) - throw new ConfigException.BugOrBroken( - "bug in method caller: not valid to create ConfigObject from map with non-String key: " - + key); - AbstractConfigValue value = fromAnyRef(entry.getValue(), - origin, mapMode); - values.put((String) key, value); - } - - return new SimpleConfigObject(origin, values); - } else { - return PropertiesParser.fromPathMap(origin, (Map) object); - } - } else if (object instanceof Iterable) { - Iterator i = ((Iterable) object).iterator(); - if (!i.hasNext()) - return emptyList(origin); - - List values = new ArrayList(); - while (i.hasNext()) { - AbstractConfigValue v = fromAnyRef(i.next(), origin, mapMode); - values.add(v); - } - - return new SimpleConfigList(origin, values); - } else { - throw new ConfigException.BugOrBroken( - "bug in method caller: not valid to create ConfigValue from: " - + object); - } - } - - private static class SimpleIncluder implements ConfigIncluder { - - private ConfigIncluder fallback; - - SimpleIncluder(ConfigIncluder fallback) { - this.fallback = fallback; - } - - @Override - public ConfigObject include(final ConfigIncludeContext context, - String name) { - NameSource source = new NameSource() { - @Override - public ConfigParseable nameToParseable(String name) { - ConfigParseable p = context.relativeTo(name); - if (p == null) { - // avoid returning null - return Parseable.newNotFound(name, "include was not found: '" + name + "'", - ConfigParseOptions.defaults()); - } else { - return p; - } - } - }; - - ConfigObject obj = fromBasename(source, name, ConfigParseOptions - .defaults().setAllowMissing(true)); - - // now use the fallback includer if any and merge - // its result. - if (fallback != null) { - return obj.withFallback(fallback.include(context, name)); - } else { - return obj; - } - } - - @Override - public ConfigIncluder withFallback(ConfigIncluder fallback) { - if (this == fallback) { - throw new ConfigException.BugOrBroken( - "trying to create includer cycle"); - } else if (this.fallback == fallback) { - return this; - } else if (this.fallback != null) { - return new SimpleIncluder(this.fallback.withFallback(fallback)); - } else { - return new SimpleIncluder(fallback); - } - } - } - - private static class DefaultIncluderHolder { - static final ConfigIncluder defaultIncluder = new SimpleIncluder(null); - } - - static ConfigIncluder defaultIncluder() { - try { - return DefaultIncluderHolder.defaultIncluder; - } catch (ExceptionInInitializerError e) { - throw ConfigImplUtil.extractInitializerError(e); - } - } - - private static AbstractConfigObject loadSystemProperties() { - return (AbstractConfigObject) Parseable.newProperties(System.getProperties(), - ConfigParseOptions.defaults().setOriginDescription("system properties")).parse(); - } - - private static class SystemPropertiesHolder { - // this isn't final due to the reloadSystemPropertiesConfig() hack below - static AbstractConfigObject systemProperties = loadSystemProperties(); - } - - static AbstractConfigObject systemPropertiesAsConfigObject() { - try { - return SystemPropertiesHolder.systemProperties; - } catch (ExceptionInInitializerError e) { - throw ConfigImplUtil.extractInitializerError(e); - } - } - - /** For use ONLY by library internals, DO NOT TOUCH not guaranteed ABI */ - public static Config systemPropertiesAsConfig() { - return systemPropertiesAsConfigObject().toConfig(); - } - - // this is a hack to let us set system props in the test suite. - // obviously not thread-safe. - static void reloadSystemPropertiesConfig() { - SystemPropertiesHolder.systemProperties = loadSystemProperties(); - } - - private static AbstractConfigObject loadEnvVariables() { - Map env = System.getenv(); - Map m = new HashMap(); - for (Map.Entry entry : env.entrySet()) { - String key = entry.getKey(); - m.put(key, - new ConfigString(SimpleConfigOrigin.newSimple("env var " + key), entry - .getValue())); - } - return new SimpleConfigObject(SimpleConfigOrigin.newSimple("env variables"), - m, ResolveStatus.RESOLVED, false /* ignoresFallbacks */); - } - - private static class EnvVariablesHolder { - static final AbstractConfigObject envVariables = loadEnvVariables(); - } - - static AbstractConfigObject envVariablesAsConfigObject() { - try { - return EnvVariablesHolder.envVariables; - } catch (ExceptionInInitializerError e) { - throw ConfigImplUtil.extractInitializerError(e); - } - } - - /** For use ONLY by library internals, DO NOT TOUCH not guaranteed ABI */ - public static Config envVariablesAsConfig() { - return envVariablesAsConfigObject().toConfig(); - } - - /** For use ONLY by library internals, DO NOT TOUCH not guaranteed ABI */ - public static Config defaultReference(ClassLoader loader) { - Config unresolvedResources = Parseable - .newResources(loader, "reference.conf", ConfigParseOptions.defaults()).parse() - .toConfig(); - return systemPropertiesAsConfig().withFallback(unresolvedResources).resolve(); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java deleted file mode 100755 index 4cbcff7aa3..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java +++ /dev/null @@ -1,185 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.File; -import java.net.URISyntaxException; -import java.net.URL; -import java.util.ArrayList; -import java.util.List; - -import com.typesafe.config.ConfigException; - - -/** This is public just for the "config" package to use, don't touch it */ -final public class ConfigImplUtil { - static boolean equalsHandlingNull(Object a, Object b) { - if (a == null && b != null) - return false; - else if (a != null && b == null) - return false; - else if (a == b) // catches null == null plus optimizes identity case - return true; - else - return a.equals(b); - } - - /** - * This is public ONLY for use by the "config" package, DO NOT USE this ABI - * may change. - */ - public static String renderJsonString(String s) { - StringBuilder sb = new StringBuilder(); - sb.append('"'); - for (int i = 0; i < s.length(); ++i) { - char c = s.charAt(i); - switch (c) { - case '"': - sb.append("\\\""); - break; - case '\\': - sb.append("\\\\"); - break; - case '\n': - sb.append("\\n"); - break; - case '\b': - sb.append("\\b"); - break; - case '\f': - sb.append("\\f"); - break; - case '\r': - sb.append("\\r"); - break; - case '\t': - sb.append("\\t"); - break; - default: - if (Character.isISOControl(c)) - sb.append(String.format("\\u%04x", (int) c)); - else - sb.append(c); - } - } - sb.append('"'); - return sb.toString(); - } - - static boolean isWhitespace(int codepoint) { - switch (codepoint) { - // try to hit the most common ASCII ones first, then the nonbreaking - // spaces that Java brokenly leaves out of isWhitespace. - case ' ': - case '\n': - case '\u00A0': - case '\u2007': - case '\u202F': - return true; - default: - return Character.isWhitespace(codepoint); - } - } - - /** This is public just for the "config" package to use, don't touch it! */ - public static String unicodeTrim(String s) { - // this is dumb because it looks like there aren't any whitespace - // characters that need surrogate encoding. But, points for - // pedantic correctness! It's future-proof or something. - // String.trim() actually is broken, since there are plenty of - // non-ASCII whitespace characters. - final int length = s.length(); - if (length == 0) - return s; - - int start = 0; - while (start < length) { - char c = s.charAt(start); - if (c == ' ' || c == '\n') { - start += 1; - } else { - int cp = s.codePointAt(start); - if (isWhitespace(cp)) - start += Character.charCount(cp); - else - break; - } - } - - int end = length; - while (end > start) { - char c = s.charAt(end - 1); - if (c == ' ' || c == '\n') { - --end; - } else { - int cp; - int delta; - if (Character.isLowSurrogate(c)) { - cp = s.codePointAt(end - 2); - delta = 2; - } else { - cp = s.codePointAt(end - 1); - delta = 1; - } - if (isWhitespace(cp)) - end -= delta; - else - break; - } - } - return s.substring(start, end); - } - - /** This is public just for the "config" package to use, don't touch it! */ - public static ConfigException extractInitializerError(ExceptionInInitializerError e) { - Throwable cause = e.getCause(); - if (cause != null && cause instanceof ConfigException) { - return (ConfigException) cause; - } else { - throw e; - } - } - - static File urlToFile(URL url) { - // this isn't really right, clearly, but not sure what to do. - try { - // this will properly handle hex escapes, etc. - return new File(url.toURI()); - } catch (URISyntaxException e) { - // this handles some stuff like file:///c:/Whatever/ - // apparently but mangles handling of hex escapes - return new File(url.getPath()); - } - } - - /** - * This is public ONLY for use by the "config" package, DO NOT USE this ABI - * may change. You can use the version in ConfigUtil instead. - */ - public static String joinPath(String... elements) { - return (new Path(elements)).render(); - } - - /** - * This is public ONLY for use by the "config" package, DO NOT USE this ABI - * may change. You can use the version in ConfigUtil instead. - */ - public static String joinPath(List elements) { - return joinPath(elements.toArray(new String[0])); - } - - /** - * This is public ONLY for use by the "config" package, DO NOT USE this ABI - * may change. You can use the version in ConfigUtil instead. - */ - public static List splitPath(String path) { - Path p = Path.newPath(path); - List elements = new ArrayList(); - while (p != null) { - elements.add(p.first()); - p = p.remainder(); - } - return elements; - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java deleted file mode 100755 index 8ccb3f872a..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java +++ /dev/null @@ -1,53 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigValueType; - -final class ConfigInt extends ConfigNumber { - - private static final long serialVersionUID = 1L; - - final private int value; - - ConfigInt(ConfigOrigin origin, int value, String originalText) { - super(origin, originalText); - this.value = value; - } - - @Override - public ConfigValueType valueType() { - return ConfigValueType.NUMBER; - } - - @Override - public Integer unwrapped() { - return value; - } - - @Override - String transformToString() { - String s = super.transformToString(); - if (s == null) - return Integer.toString(value); - else - return s; - } - - @Override - protected long longValue() { - return value; - } - - @Override - protected double doubleValue() { - return value; - } - - @Override - protected ConfigInt newCopy(boolean ignoresFallbacks, ConfigOrigin origin) { - return new ConfigInt(origin, value, originalText); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java deleted file mode 100755 index 6e1a5073fb..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java +++ /dev/null @@ -1,53 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigValueType; - -final class ConfigLong extends ConfigNumber { - - private static final long serialVersionUID = 1L; - - final private long value; - - ConfigLong(ConfigOrigin origin, long value, String originalText) { - super(origin, originalText); - this.value = value; - } - - @Override - public ConfigValueType valueType() { - return ConfigValueType.NUMBER; - } - - @Override - public Long unwrapped() { - return value; - } - - @Override - String transformToString() { - String s = super.transformToString(); - if (s == null) - return Long.toString(value); - else - return s; - } - - @Override - protected long longValue() { - return value; - } - - @Override - protected double doubleValue() { - return value; - } - - @Override - protected ConfigLong newCopy(boolean ignoresFallbacks, ConfigOrigin origin) { - return new ConfigLong(origin, value, originalText); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java deleted file mode 100755 index 1dcd32b5b0..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java +++ /dev/null @@ -1,49 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigValueType; - -/** - * This exists because sometimes null is not the same as missing. Specifically, - * if a value is set to null we can give a better error message (indicating - * where it was set to null) in case someone asks for the value. Also, null - * overrides values set "earlier" in the search path, while missing values do - * not. - * - */ -final class ConfigNull extends AbstractConfigValue { - - private static final long serialVersionUID = 1L; - - ConfigNull(ConfigOrigin origin) { - super(origin); - } - - @Override - public ConfigValueType valueType() { - return ConfigValueType.NULL; - } - - @Override - public Object unwrapped() { - return null; - } - - @Override - String transformToString() { - return "null"; - } - - @Override - protected void render(StringBuilder sb, int indent, boolean formatted) { - sb.append("null"); - } - - @Override - protected ConfigNull newCopy(boolean ignoresFallbacks, ConfigOrigin origin) { - return new ConfigNull(origin); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java deleted file mode 100755 index 8f6996d77b..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java +++ /dev/null @@ -1,102 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigOrigin; - -abstract class ConfigNumber extends AbstractConfigValue { - - private static final long serialVersionUID = 1L; - - // This is so when we concatenate a number into a string (say it appears in - // a sentence) we always have it exactly as the person typed it into the - // config file. It's purely cosmetic; equals/hashCode don't consider this - // for example. - final protected String originalText; - - protected ConfigNumber(ConfigOrigin origin, String originalText) { - super(origin); - this.originalText = originalText; - } - - @Override - public abstract Number unwrapped(); - - @Override - String transformToString() { - return originalText; - } - - int intValueRangeChecked(String path) { - long l = longValue(); - if (l < Integer.MIN_VALUE || l > Integer.MAX_VALUE) { - throw new ConfigException.WrongType(origin(), path, "32-bit integer", - "out-of-range value " + l); - } - return (int) l; - } - - protected abstract long longValue(); - - protected abstract double doubleValue(); - - private boolean isWhole() { - long asLong = longValue(); - return asLong == doubleValue(); - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof ConfigNumber; - } - - @Override - public boolean equals(Object other) { - // note that "origin" is deliberately NOT part of equality - if (canEqual(other)) { - ConfigNumber n = (ConfigNumber) other; - if (isWhole()) { - return n.isWhole() && this.longValue() == n.longValue(); - } else { - return (!n.isWhole()) && this.doubleValue() == n.doubleValue(); - } - } else { - return false; - } - } - - @Override - public int hashCode() { - // note that "origin" is deliberately NOT part of equality - - // this matches what standard Long.hashCode and Double.hashCode - // do, though I don't think it really matters. - long asLong; - if (isWhole()) { - asLong = longValue(); - } else { - asLong = Double.doubleToLongBits(doubleValue()); - } - return (int) (asLong ^ (asLong >>> 32)); - } - - static ConfigNumber newNumber(ConfigOrigin origin, long number, - String originalText) { - if (number <= Integer.MAX_VALUE && number >= Integer.MIN_VALUE) - return new ConfigInt(origin, (int) number, originalText); - else - return new ConfigLong(origin, number, originalText); - } - - static ConfigNumber newNumber(ConfigOrigin origin, double number, - String originalText) { - long asLong = (long) number; - if (asLong == number) { - return newNumber(origin, asLong, originalText); - } else { - return new ConfigDouble(origin, number, originalText); - } - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java deleted file mode 100755 index 4bdce3b129..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigValueType; - -final class ConfigString extends AbstractConfigValue { - - private static final long serialVersionUID = 1L; - - final private String value; - - ConfigString(ConfigOrigin origin, String value) { - super(origin); - this.value = value; - } - - @Override - public ConfigValueType valueType() { - return ConfigValueType.STRING; - } - - @Override - public String unwrapped() { - return value; - } - - @Override - String transformToString() { - return value; - } - - @Override - protected void render(StringBuilder sb, int indent, boolean formatted) { - sb.append(ConfigImplUtil.renderJsonString(value)); - } - - @Override - protected ConfigString newCopy(boolean ignoresFallbacks, ConfigOrigin origin) { - return new ConfigString(origin, value); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java deleted file mode 100755 index cca0ebb577..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java +++ /dev/null @@ -1,300 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.ObjectStreamException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigResolveOptions; -import com.typesafe.config.ConfigValue; -import com.typesafe.config.ConfigValueType; - -/** - * A ConfigSubstitution represents a value with one or more substitutions in it; - * it can resolve to a value of any type, though if the substitution has more - * than one piece it always resolves to a string via value concatenation. - */ -final class ConfigSubstitution extends AbstractConfigValue implements - Unmergeable { - - private static final long serialVersionUID = 1L; - - // this is a list of String and SubstitutionExpression where the - // SubstitutionExpression has to be resolved to values, then if there's more - // than one piece everything is stringified and concatenated - final private List pieces; - // the length of any prefixes added with relativized() - final private int prefixLength; - final private boolean ignoresFallbacks; - - ConfigSubstitution(ConfigOrigin origin, List pieces) { - this(origin, pieces, 0, false); - } - - private ConfigSubstitution(ConfigOrigin origin, List pieces, - int prefixLength, boolean ignoresFallbacks) { - super(origin); - this.pieces = pieces; - this.prefixLength = prefixLength; - this.ignoresFallbacks = ignoresFallbacks; - for (Object p : pieces) { - if (p instanceof Path) - throw new RuntimeException("broken here"); - } - } - - @Override - public ConfigValueType valueType() { - throw new ConfigException.NotResolved( - "need to call resolve() on root config; tried to get value type on an unresolved substitution: " - + this); - } - - @Override - public Object unwrapped() { - throw new ConfigException.NotResolved( - "need to call resolve() on root config; tried to unwrap an unresolved substitution: " - + this); - } - - @Override - protected ConfigSubstitution newCopy(boolean ignoresFallbacks, ConfigOrigin newOrigin) { - return new ConfigSubstitution(newOrigin, pieces, prefixLength, ignoresFallbacks); - } - - @Override - protected boolean ignoresFallbacks() { - return ignoresFallbacks; - } - - @Override - protected AbstractConfigValue mergedWithTheUnmergeable(Unmergeable fallback) { - if (ignoresFallbacks) - throw new ConfigException.BugOrBroken("should not be reached"); - - // if we turn out to be an object, and the fallback also does, - // then a merge may be required; delay until we resolve. - List newStack = new ArrayList(); - newStack.add(this); - newStack.addAll(fallback.unmergedValues()); - return new ConfigDelayedMerge(AbstractConfigObject.mergeOrigins(newStack), newStack, - ((AbstractConfigValue) fallback).ignoresFallbacks()); - } - - protected AbstractConfigValue mergedLater(AbstractConfigValue fallback) { - if (ignoresFallbacks) - throw new ConfigException.BugOrBroken("should not be reached"); - - List newStack = new ArrayList(); - newStack.add(this); - newStack.add(fallback); - return new ConfigDelayedMerge(AbstractConfigObject.mergeOrigins(newStack), newStack, - fallback.ignoresFallbacks()); - } - - @Override - protected AbstractConfigValue mergedWithObject(AbstractConfigObject fallback) { - // if we turn out to be an object, and the fallback also does, - // then a merge may be required; delay until we resolve. - return mergedLater(fallback); - } - - @Override - protected AbstractConfigValue mergedWithNonObject(AbstractConfigValue fallback) { - // if the optional substitution ends up getting deleted (because it is - // not present), we'll have to use the fallback. So delay the merge. - if (pieces.size() == 1 && ((SubstitutionExpression) pieces.get(0)).optional()) - return mergedLater(fallback); - else - return super.mergedWithNonObject(fallback); - } - - @Override - public Collection unmergedValues() { - return Collections.singleton(this); - } - - List pieces() { - return pieces; - } - - // larger than anyone would ever want - private static final int MAX_DEPTH = 100; - - private ConfigValue findInObject(AbstractConfigObject root, - SubstitutionResolver resolver, /* null if we should not have refs */ - Path subst, int depth, ConfigResolveOptions options) { - if (depth > MAX_DEPTH) { - throw new ConfigException.BadValue(origin(), subst.render(), - "Substitution ${" + subst.render() - + "} is part of a cycle of substitutions"); - } - - ConfigValue result = root.peekPath(subst, resolver, depth, options); - - if (result instanceof ConfigSubstitution) { - throw new ConfigException.BugOrBroken( - "peek or peekPath returned an unresolved substitution"); - } - - return result; - } - - private ConfigValue resolve(SubstitutionResolver resolver, SubstitutionExpression subst, - int depth, ConfigResolveOptions options) { - // First we look up the full path, which means relative to the - // included file if we were not a root file - ConfigValue result = findInObject(resolver.root(), resolver, subst.path(), - depth, options); - - if (result == null) { - // Then we want to check relative to the root file. We don't - // want the prefix we were included at to be used when looking up - // env variables either. - Path unprefixed = subst.path().subPath(prefixLength); - - if (result == null && prefixLength > 0) { - result = findInObject(resolver.root(), resolver, unprefixed, depth, options); - } - - if (result == null && options.getUseSystemEnvironment()) { - result = findInObject(ConfigImpl.envVariablesAsConfigObject(), null, unprefixed, - depth, options); - } - } - - return result; - } - - private ConfigValue resolve(SubstitutionResolver resolver, int depth, - ConfigResolveOptions options) { - if (pieces.size() > 1) { - // need to concat everything into a string - StringBuilder sb = new StringBuilder(); - for (Object p : pieces) { - if (p instanceof String) { - sb.append((String) p); - } else { - SubstitutionExpression exp = (SubstitutionExpression) p; - ConfigValue v = resolve(resolver, exp, depth, options); - - if (v == null) { - if (exp.optional()) { - // append nothing to StringBuilder - } else { - throw new ConfigException.UnresolvedSubstitution(origin(), - exp.toString()); - } - } else { - switch (v.valueType()) { - case LIST: - case OBJECT: - // cannot substitute lists and objects into strings - throw new ConfigException.WrongType(v.origin(), exp.path().render(), - "not a list or object", v.valueType().name()); - default: - sb.append(((AbstractConfigValue) v).transformToString()); - } - } - } - } - return new ConfigString(origin(), sb.toString()); - } else { - if (!(pieces.get(0) instanceof SubstitutionExpression)) - throw new ConfigException.BugOrBroken( - "ConfigSubstitution should never contain a single String piece"); - SubstitutionExpression exp = (SubstitutionExpression) pieces.get(0); - ConfigValue v = resolve(resolver, exp, depth, options); - if (v == null && !exp.optional()) { - throw new ConfigException.UnresolvedSubstitution(origin(), exp.toString()); - } - return v; - } - } - - @Override - AbstractConfigValue resolveSubstitutions(SubstitutionResolver resolver, - int depth, - ConfigResolveOptions options) { - // only ConfigSubstitution adds to depth here, because the depth - // is the substitution depth not the recursion depth - AbstractConfigValue resolved = (AbstractConfigValue) resolve(resolver, - depth + 1, options); - return resolved; - } - - @Override - ResolveStatus resolveStatus() { - return ResolveStatus.UNRESOLVED; - } - - // when you graft a substitution into another object, - // you have to prefix it with the location in that object - // where you grafted it; but save prefixLength so - // system property and env variable lookups don't get - // broken. - @Override - ConfigSubstitution relativized(Path prefix) { - List newPieces = new ArrayList(); - for (Object p : pieces) { - if (p instanceof SubstitutionExpression) { - SubstitutionExpression exp = (SubstitutionExpression) p; - - newPieces.add(exp.changePath(exp.path().prepend(prefix))); - } else { - newPieces.add(p); - } - } - return new ConfigSubstitution(origin(), newPieces, prefixLength - + prefix.length(), ignoresFallbacks); - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof ConfigSubstitution; - } - - @Override - public boolean equals(Object other) { - // note that "origin" is deliberately NOT part of equality - if (other instanceof ConfigSubstitution) { - return canEqual(other) - && this.pieces.equals(((ConfigSubstitution) other).pieces); - } else { - return false; - } - } - - @Override - public int hashCode() { - // note that "origin" is deliberately NOT part of equality - return pieces.hashCode(); - } - - @Override - protected void render(StringBuilder sb, int indent, boolean formatted) { - for (Object p : pieces) { - if (p instanceof SubstitutionExpression) { - sb.append(p.toString()); - } else { - sb.append(ConfigImplUtil.renderJsonString((String) p)); - } - } - } - - // This ridiculous hack is because some JDK versions apparently can't - // serialize an array, which is used to implement ArrayList and EmptyList. - // maybe - // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6446627 - private Object writeReplace() throws ObjectStreamException { - // switch to LinkedList - return new ConfigSubstitution(origin(), new java.util.LinkedList(pieces), - prefixLength, ignoresFallbacks); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java b/akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java deleted file mode 100755 index 9a9bf5c6a9..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java +++ /dev/null @@ -1,81 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigValueType; - -/** - * Default automatic type transformations. - */ -final class DefaultTransformer { - - static AbstractConfigValue transform(AbstractConfigValue value, - ConfigValueType requested) { - if (value.valueType() == ConfigValueType.STRING) { - String s = (String) value.unwrapped(); - switch (requested) { - case NUMBER: - try { - Long v = Long.parseLong(s); - return new ConfigLong(value.origin(), v, s); - } catch (NumberFormatException e) { - // try Double - } - try { - Double v = Double.parseDouble(s); - return new ConfigDouble(value.origin(), v, s); - } catch (NumberFormatException e) { - // oh well. - } - break; - case NULL: - if (s.equals("null")) - return new ConfigNull(value.origin()); - break; - case BOOLEAN: - if (s.equals("true") || s.equals("yes") || s.equals("on")) { - return new ConfigBoolean(value.origin(), true); - } else if (s.equals("false") || s.equals("no") - || s.equals("off")) { - return new ConfigBoolean(value.origin(), false); - } - break; - case LIST: - // can't go STRING to LIST automatically - break; - case OBJECT: - // can't go STRING to OBJECT automatically - break; - case STRING: - // no-op STRING to STRING - break; - } - } else if (requested == ConfigValueType.STRING) { - // if we converted null to string here, then you wouldn't properly - // get a missing-value error if you tried to get a null value - // as a string. - switch (value.valueType()) { - case NUMBER: // FALL THROUGH - case BOOLEAN: - return new ConfigString(value.origin(), - value.transformToString()); - case NULL: - // want to be sure this throws instead of returning "null" as a - // string - break; - case OBJECT: - // no OBJECT to STRING automatically - break; - case LIST: - // no LIST to STRING automatically - break; - case STRING: - // no-op STRING to STRING - break; - } - } - - return value; - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java b/akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java deleted file mode 100755 index 764a45664c..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java +++ /dev/null @@ -1,8 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -enum FromMapMode { - KEYS_ARE_PATHS, KEYS_ARE_KEYS -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/MergeableValue.java b/akka-actor/src/main/java/com/typesafe/config/impl/MergeableValue.java deleted file mode 100755 index 38dba70cc2..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/MergeableValue.java +++ /dev/null @@ -1,9 +0,0 @@ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigMergeable; -import com.typesafe.config.ConfigValue; - -interface MergeableValue extends ConfigMergeable { - // converts a Config to its root object and a ConfigValue to itself - ConfigValue toFallbackValue(); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/OriginType.java b/akka-actor/src/main/java/com/typesafe/config/impl/OriginType.java deleted file mode 100755 index b8e990c091..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/OriginType.java +++ /dev/null @@ -1,8 +0,0 @@ -package com.typesafe.config.impl; - -enum OriginType { - GENERIC, - FILE, - URL, - RESOURCE -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java b/akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java deleted file mode 100755 index 4938603199..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java +++ /dev/null @@ -1,637 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.FilterReader; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.Reader; -import java.io.StringReader; -import java.io.UnsupportedEncodingException; -import java.net.MalformedURLException; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.URL; -import java.util.Enumeration; -import java.util.Iterator; -import java.util.Properties; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigIncludeContext; -import com.typesafe.config.ConfigObject; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigParseOptions; -import com.typesafe.config.ConfigParseable; -import com.typesafe.config.ConfigSyntax; -import com.typesafe.config.ConfigValue; - -/** - * This is public but it's only for use by the config package; DO NOT TOUCH. The - * point of this class is to avoid "propagating" each overload on - * "thing which can be parsed" through multiple interfaces. Most interfaces can - * have just one overload that takes a Parseable. Also it's used as an abstract - * "resource handle" in the ConfigIncluder interface. - */ -public abstract class Parseable implements ConfigParseable { - private ConfigIncludeContext includeContext; - private ConfigParseOptions initialOptions; - private ConfigOrigin initialOrigin; - - protected Parseable() { - - } - - private ConfigParseOptions fixupOptions(ConfigParseOptions baseOptions) { - ConfigSyntax syntax = baseOptions.getSyntax(); - if (syntax == null) { - syntax = guessSyntax(); - } - if (syntax == null) { - syntax = ConfigSyntax.CONF; - } - ConfigParseOptions modified = baseOptions.setSyntax(syntax); - - modified = modified.appendIncluder(ConfigImpl.defaultIncluder()); - - return modified; - } - - protected void postConstruct(ConfigParseOptions baseOptions) { - this.initialOptions = fixupOptions(baseOptions); - - this.includeContext = new ConfigIncludeContext() { - @Override - public ConfigParseable relativeTo(String filename) { - return Parseable.this.relativeTo(filename); - } - }; - - if (initialOptions.getOriginDescription() != null) - initialOrigin = SimpleConfigOrigin.newSimple(initialOptions.getOriginDescription()); - else - initialOrigin = createOrigin(); - } - - // the general idea is that any work should be in here, not in the - // constructor, - // so that exceptions are thrown from the public parse() function and not - // from the creation of the Parseable. Essentially this is a lazy field. - // The parser should close the reader when it's done with it. - // ALSO, IMPORTANT: if the file or URL is not found, this must throw. - // to support the "allow missing" feature. - protected abstract Reader reader() throws IOException; - - ConfigSyntax guessSyntax() { - return null; - } - - ConfigParseable relativeTo(String filename) { - return null; - } - - ConfigIncludeContext includeContext() { - return includeContext; - } - - static AbstractConfigObject forceParsedToObject(ConfigValue value) { - if (value instanceof AbstractConfigObject) { - return (AbstractConfigObject) value; - } else { - throw new ConfigException.WrongType(value.origin(), "", - "object at file root", value.valueType().name()); - } - } - - @Override - public ConfigObject parse(ConfigParseOptions baseOptions) { - return forceParsedToObject(parseValue(baseOptions)); - } - - final AbstractConfigValue parseValue(ConfigParseOptions baseOptions) { - // note that we are NOT using our "initialOptions", - // but using the ones from the passed-in options. The idea is that - // callers can get our original options and then parse with different - // ones if they want. - ConfigParseOptions options = fixupOptions(baseOptions); - - // passed-in options can override origin - ConfigOrigin origin; - if (options.getOriginDescription() != null) - origin = SimpleConfigOrigin.newSimple(options.getOriginDescription()); - else - origin = initialOrigin; - return parseValue(origin, options); - } - - final private AbstractConfigValue parseValue(ConfigOrigin origin, - ConfigParseOptions finalOptions) { - try { - return rawParseValue(origin, finalOptions); - } catch (IOException e) { - if (finalOptions.getAllowMissing()) { - return SimpleConfigObject.emptyMissing(origin); - } else { - throw new ConfigException.IO(origin, e.getMessage(), e); - } - } - } - - // this is parseValue without post-processing the IOException or handling - // options.getAllowMissing() - protected AbstractConfigValue rawParseValue(ConfigOrigin origin, ConfigParseOptions finalOptions) - throws IOException { - Reader reader = reader(); - try { - return rawParseValue(reader, origin, finalOptions); - } finally { - reader.close(); - } - } - - protected AbstractConfigValue rawParseValue(Reader reader, ConfigOrigin origin, - ConfigParseOptions finalOptions) throws IOException { - if (finalOptions.getSyntax() == ConfigSyntax.PROPERTIES) { - return PropertiesParser.parse(reader, origin); - } else { - Iterator tokens = Tokenizer.tokenize(origin, reader, finalOptions.getSyntax()); - return Parser.parse(tokens, origin, finalOptions, includeContext()); - } - } - - public ConfigObject parse() { - return forceParsedToObject(parseValue(options())); - } - - AbstractConfigValue parseValue() { - return parseValue(options()); - } - - @Override - public final ConfigOrigin origin() { - return initialOrigin; - } - - protected abstract ConfigOrigin createOrigin(); - - @Override - public ConfigParseOptions options() { - return initialOptions; - } - - @Override - public String toString() { - return getClass().getSimpleName(); - } - - private static ConfigSyntax syntaxFromExtension(String name) { - if (name.endsWith(".json")) - return ConfigSyntax.JSON; - else if (name.endsWith(".conf")) - return ConfigSyntax.CONF; - else if (name.endsWith(".properties")) - return ConfigSyntax.PROPERTIES; - else - return null; - } - - private static Reader readerFromStream(InputStream input) { - try { - // well, this is messed up. If we aren't going to close - // the passed-in InputStream then we have no way to - // close these readers. So maybe we should not have an - // InputStream version, only a Reader version. - Reader reader = new InputStreamReader(input, "UTF-8"); - return new BufferedReader(reader); - } catch (UnsupportedEncodingException e) { - throw new ConfigException.BugOrBroken( - "Java runtime does not support UTF-8", e); - } - } - - private static Reader doNotClose(Reader input) { - return new FilterReader(input) { - @Override - public void close() { - // NOTHING. - } - }; - } - - static URL relativeTo(URL url, String filename) { - // I'm guessing this completely fails on Windows, help wanted - if (new File(filename).isAbsolute()) - return null; - - try { - URI siblingURI = url.toURI(); - URI relative = new URI(filename); - - // this seems wrong, but it's documented that the last - // element of the path in siblingURI gets stripped out, - // so to get something in the same directory as - // siblingURI we just call resolve(). - URL resolved = siblingURI.resolve(relative).toURL(); - - return resolved; - } catch (MalformedURLException e) { - return null; - } catch (URISyntaxException e) { - return null; - } catch (IllegalArgumentException e) { - return null; - } - } - - static File relativeTo(File file, String filename) { - File child = new File(filename); - - if (child.isAbsolute()) - return null; - - File parent = file.getParentFile(); - - if (parent == null) - return null; - else - return new File(parent, filename); - } - - // this is a parseable that doesn't exist and just throws when you try to - // parse it - private final static class ParseableNotFound extends Parseable { - final private String what; - final private String message; - - ParseableNotFound(String what, String message, ConfigParseOptions options) { - this.what = what; - this.message = message; - postConstruct(options); - } - - @Override - protected Reader reader() throws IOException { - throw new FileNotFoundException(message); - } - - @Override - protected ConfigOrigin createOrigin() { - return SimpleConfigOrigin.newSimple(what); - } - } - - public static Parseable newNotFound(String whatNotFound, String message, - ConfigParseOptions options) { - return new ParseableNotFound(whatNotFound, message, options); - } - - private final static class ParseableReader extends Parseable { - final private Reader reader; - - ParseableReader(Reader reader, ConfigParseOptions options) { - this.reader = reader; - postConstruct(options); - } - - @Override - protected Reader reader() { - return reader; - } - - @Override - protected ConfigOrigin createOrigin() { - return SimpleConfigOrigin.newSimple("Reader"); - } - } - - /** - * note that we will never close this reader; you have to do it when parsing - * is complete. - */ - public static Parseable newReader(Reader reader, ConfigParseOptions options) { - return new ParseableReader(doNotClose(reader), options); - } - - private final static class ParseableString extends Parseable { - final private String input; - - ParseableString(String input, ConfigParseOptions options) { - this.input = input; - postConstruct(options); - } - - @Override - protected Reader reader() { - return new StringReader(input); - } - - @Override - protected ConfigOrigin createOrigin() { - return SimpleConfigOrigin.newSimple("String"); - } - } - - public static Parseable newString(String input, ConfigParseOptions options) { - return new ParseableString(input, options); - } - - private final static class ParseableURL extends Parseable { - final private URL input; - - ParseableURL(URL input, ConfigParseOptions options) { - this.input = input; - postConstruct(options); - } - - @Override - protected Reader reader() throws IOException { - InputStream stream = input.openStream(); - return readerFromStream(stream); - } - - @Override - ConfigSyntax guessSyntax() { - return syntaxFromExtension(input.getPath()); - } - - @Override - ConfigParseable relativeTo(String filename) { - URL url = relativeTo(input, filename); - if (url == null) - return null; - return newURL(url, options() - .setOriginDescription(null)); - } - - @Override - protected ConfigOrigin createOrigin() { - return SimpleConfigOrigin.newURL(input); - } - - @Override - public String toString() { - return getClass().getSimpleName() + "(" + input.toExternalForm() - + ")"; - } - } - - public static Parseable newURL(URL input, ConfigParseOptions options) { - // we want file: URLs and files to always behave the same, so switch - // to a file if it's a file: URL - if (input.getProtocol().equals("file")) { - return newFile(ConfigImplUtil.urlToFile(input), options); - } else { - return new ParseableURL(input, options); - } - } - - private final static class ParseableFile extends Parseable { - final private File input; - - ParseableFile(File input, ConfigParseOptions options) { - this.input = input; - postConstruct(options); - } - - @Override - protected Reader reader() throws IOException { - InputStream stream = new FileInputStream(input); - return readerFromStream(stream); - } - - @Override - ConfigSyntax guessSyntax() { - return syntaxFromExtension(input.getName()); - } - - @Override - ConfigParseable relativeTo(String filename) { - File sibling; - if ((new File(filename)).isAbsolute()) { - sibling = new File(filename); - } else { - // this may return null - sibling = relativeTo(input, filename); - } - if (sibling == null) - return null; - if (sibling.exists()) { - return newFile(sibling, options().setOriginDescription(null)); - } else { - // fall back to classpath; we treat the "filename" as absolute - // (don't add a package name in front), - // if it starts with "/" then remove the "/", for consistency - // with ParseableResources.relativeTo - String resource = filename; - if (filename.startsWith("/")) - resource = filename.substring(1); - return newResources(this.getClass().getClassLoader(), resource, options() - .setOriginDescription(null)); - } - } - - @Override - protected ConfigOrigin createOrigin() { - return SimpleConfigOrigin.newFile(input.getPath()); - } - - @Override - public String toString() { - return getClass().getSimpleName() + "(" + input.getPath() + ")"; - } - } - - public static Parseable newFile(File input, ConfigParseOptions options) { - return new ParseableFile(input, options); - } - - private final static class ParseableResources extends Parseable { - final private ClassLoader loader; - final private String resource; - - ParseableResources(ClassLoader loader, String resource, - ConfigParseOptions options) { - this.loader = loader; - this.resource = resource; - postConstruct(options); - } - - @Override - protected Reader reader() throws IOException { - throw new ConfigException.BugOrBroken( - "reader() should not be called on resources"); - } - - @Override - protected AbstractConfigObject rawParseValue(ConfigOrigin origin, - ConfigParseOptions finalOptions) throws IOException { - Enumeration e = loader.getResources(resource); - if (!e.hasMoreElements()) { - throw new IOException("resource not found on classpath: " + resource); - } - AbstractConfigObject merged = SimpleConfigObject.empty(origin); - while (e.hasMoreElements()) { - URL url = e.nextElement(); - - ConfigOrigin elementOrigin = ((SimpleConfigOrigin) origin).addURL(url); - - AbstractConfigValue v; - - // it's tempting to use ParseableURL here but it would be wrong - // because the wrong relativeTo() would be used for includes. - InputStream stream = url.openStream(); - try { - Reader reader = readerFromStream(stream); - stream = null; // reader now owns it - try { - // parse in "raw" mode which will throw any IOException - // from here. - v = rawParseValue(reader, elementOrigin, finalOptions); - } finally { - reader.close(); - } - } finally { - // stream is null if the reader owns it - if (stream != null) - stream.close(); - } - - merged = merged.withFallback(v); - } - - return merged; - } - - @Override - ConfigSyntax guessSyntax() { - return syntaxFromExtension(resource); - } - - static String parent(String resource) { - // the "resource" is not supposed to begin with a "/" - // because it's supposed to be the raw resource - // (ClassLoader#getResource), not the - // resource "syntax" (Class#getResource) - int i = resource.lastIndexOf('/'); - if (i < 0) { - return null; - } else { - return resource.substring(0, i); - } - } - - @Override - ConfigParseable relativeTo(String sibling) { - if (sibling.startsWith("/")) { - // if it starts with "/" then don't make it relative to - // the including resource - return newResources(loader, sibling.substring(1), - options().setOriginDescription(null)); - } else { - // here we want to build a new resource name and let - // the class loader have it, rather than getting the - // url with getResource() and relativizing to that url. - // This is needed in case the class loader is going to - // search a classpath. - String parent = parent(resource); - if (parent == null) - return newResources(loader, sibling, options().setOriginDescription(null)); - else - return newResources(loader, parent + "/" + sibling, options() - .setOriginDescription(null)); - } - } - - @Override - protected ConfigOrigin createOrigin() { - return SimpleConfigOrigin.newResource(resource); - } - - @Override - public String toString() { - return getClass().getSimpleName() + "(" + resource + "," - + loader.getClass().getSimpleName() + ")"; - } - } - - public static Parseable newResources(Class klass, String resource, - ConfigParseOptions options) { - return newResources(klass.getClassLoader(), convertResourceName(klass, resource), options); - } - - // this function is supposed to emulate the difference - // between Class.getResource and ClassLoader.getResource - // (unfortunately there doesn't seem to be public API for it). - // We're using it because the Class API is more limited, - // for example it lacks getResources(). So we want to be able to - // use ClassLoader directly. - private static String convertResourceName(Class klass, String resource) { - if (resource.startsWith("/")) { - // "absolute" resource, chop the slash - return resource.substring(1); - } else { - String className = klass.getName(); - int i = className.lastIndexOf('.'); - if (i < 0) { - // no package - return resource; - } else { - // need to be relative to the package - String packageName = className.substring(0, i); - String packagePath = packageName.replace('.', '/'); - return packagePath + "/" + resource; - } - } - } - - public static Parseable newResources(ClassLoader loader, String resource, - ConfigParseOptions options) { - return new ParseableResources(loader, resource, options); - } - - private final static class ParseableProperties extends Parseable { - final private Properties props; - - ParseableProperties(Properties props, ConfigParseOptions options) { - this.props = props; - postConstruct(options); - } - - @Override - protected Reader reader() throws IOException { - throw new ConfigException.BugOrBroken( - "reader() should not be called on props"); - } - - @Override - protected AbstractConfigObject rawParseValue(ConfigOrigin origin, - ConfigParseOptions finalOptions) { - return PropertiesParser.fromProperties(origin, props); - } - - @Override - ConfigSyntax guessSyntax() { - return ConfigSyntax.PROPERTIES; - } - - @Override - protected ConfigOrigin createOrigin() { - return SimpleConfigOrigin.newSimple("properties"); - } - - @Override - public String toString() { - return getClass().getSimpleName() + "(" + props.size() + " props)"; - } - } - - public static Parseable newProperties(Properties properties, - ConfigParseOptions options) { - return new ParseableProperties(properties, options); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Parser.java b/akka-actor/src/main/java/com/typesafe/config/impl/Parser.java deleted file mode 100755 index 1ba8535207..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Parser.java +++ /dev/null @@ -1,965 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.StringReader; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.ListIterator; -import java.util.Map; -import java.util.Stack; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigIncludeContext; -import com.typesafe.config.ConfigIncluder; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigParseOptions; -import com.typesafe.config.ConfigSyntax; -import com.typesafe.config.ConfigValueType; - -final class Parser { - - static AbstractConfigValue parse(Iterator tokens, - ConfigOrigin origin, ConfigParseOptions options, - ConfigIncludeContext includeContext) { - ParseContext context = new ParseContext(options.getSyntax(), origin, - tokens, options.getIncluder(), includeContext); - return context.parse(); - } - - static private final class TokenWithComments { - final Token token; - final List comments; - - TokenWithComments(Token token, List comments) { - this.token = token; - this.comments = comments; - } - - TokenWithComments(Token token) { - this(token, Collections. emptyList()); - } - - TokenWithComments prepend(List earlier) { - if (this.comments.isEmpty()) { - return new TokenWithComments(token, earlier); - } else { - List merged = new ArrayList(); - merged.addAll(earlier); - merged.addAll(comments); - return new TokenWithComments(token, merged); - } - } - - SimpleConfigOrigin setComments(SimpleConfigOrigin origin) { - if (comments.isEmpty()) { - return origin; - } else { - List newComments = new ArrayList(); - for (Token c : comments) { - newComments.add(Tokens.getCommentText(c)); - } - return origin.setComments(newComments); - } - } - - @Override - public String toString() { - // this ends up in user-visible error messages, so we don't want the - // comments - return token.toString(); - } - } - - static private final class ParseContext { - private int lineNumber; - final private Stack buffer; - final private Iterator tokens; - final private ConfigIncluder includer; - final private ConfigIncludeContext includeContext; - final private ConfigSyntax flavor; - final private ConfigOrigin baseOrigin; - final private LinkedList pathStack; - // this is the number of "equals" we are inside, - // used to modify the error message to reflect that - // someone may think this is .properties format. - int equalsCount; - - ParseContext(ConfigSyntax flavor, ConfigOrigin origin, - Iterator tokens, ConfigIncluder includer, - ConfigIncludeContext includeContext) { - lineNumber = 1; - buffer = new Stack(); - this.tokens = tokens; - this.flavor = flavor; - this.baseOrigin = origin; - this.includer = includer; - this.includeContext = includeContext; - this.pathStack = new LinkedList(); - this.equalsCount = 0; - } - - private void consolidateCommentBlock(Token commentToken) { - // a comment block "goes with" the following token - // unless it's separated from it by a blank line. - // we want to build a list of newline tokens followed - // by a non-newline non-comment token; with all comments - // associated with that final non-newline non-comment token. - List newlines = new ArrayList(); - List comments = new ArrayList(); - - Token previous = null; - Token next = commentToken; - while (true) { - if (Tokens.isNewline(next)) { - if (previous != null && Tokens.isNewline(previous)) { - // blank line; drop all comments to this point and - // start a new comment block - comments.clear(); - } - newlines.add(next); - } else if (Tokens.isComment(next)) { - comments.add(next); - } else { - // a non-newline non-comment token - break; - } - - previous = next; - next = tokens.next(); - } - - // put our concluding token in the queue with all the comments - // attached - buffer.push(new TokenWithComments(next, comments)); - - // now put all the newlines back in front of it - ListIterator li = newlines.listIterator(newlines.size()); - while (li.hasPrevious()) { - buffer.push(new TokenWithComments(li.previous())); - } - } - - private TokenWithComments popToken() { - if (buffer.isEmpty()) { - Token t = tokens.next(); - if (Tokens.isComment(t)) { - consolidateCommentBlock(t); - return buffer.pop(); - } else { - return new TokenWithComments(t); - } - } else { - return buffer.pop(); - } - } - - private TokenWithComments nextToken() { - TokenWithComments withComments = null; - - withComments = popToken(); - Token t = withComments.token; - - if (Tokens.isProblem(t)) { - ConfigOrigin origin = t.origin(); - String message = Tokens.getProblemMessage(t); - Throwable cause = Tokens.getProblemCause(t); - boolean suggestQuotes = Tokens.getProblemSuggestQuotes(t); - if (suggestQuotes) { - message = addQuoteSuggestion(t.toString(), message); - } else { - message = addKeyName(message); - } - throw new ConfigException.Parse(origin, message, cause); - } else { - if (flavor == ConfigSyntax.JSON) { - if (Tokens.isUnquotedText(t)) { - throw parseError(addKeyName("Token not allowed in valid JSON: '" - + Tokens.getUnquotedText(t) + "'")); - } else if (Tokens.isSubstitution(t)) { - throw parseError(addKeyName("Substitutions (${} syntax) not allowed in JSON")); - } - } - - return withComments; - } - } - - private void putBack(TokenWithComments token) { - buffer.push(token); - } - - private TokenWithComments nextTokenIgnoringNewline() { - TokenWithComments t = nextToken(); - - while (Tokens.isNewline(t.token)) { - // line number tokens have the line that was _ended_ by the - // newline, so we have to add one. - lineNumber = t.token.lineNumber() + 1; - - t = nextToken(); - } - - return t; - } - - // In arrays and objects, comma can be omitted - // as long as there's at least one newline instead. - // this skips any newlines in front of a comma, - // skips the comma, and returns true if it found - // either a newline or a comma. The iterator - // is left just after the comma or the newline. - private boolean checkElementSeparator() { - if (flavor == ConfigSyntax.JSON) { - TokenWithComments t = nextTokenIgnoringNewline(); - if (t.token == Tokens.COMMA) { - return true; - } else { - putBack(t); - return false; - } - } else { - boolean sawSeparatorOrNewline = false; - TokenWithComments t = nextToken(); - while (true) { - if (Tokens.isNewline(t.token)) { - // newline number is the line just ended, so add one - lineNumber = t.token.lineNumber() + 1; - sawSeparatorOrNewline = true; - - // we want to continue to also eat - // a comma if there is one. - } else if (t.token == Tokens.COMMA) { - return true; - } else { - // non-newline-or-comma - putBack(t); - return sawSeparatorOrNewline; - } - t = nextToken(); - } - } - } - - // merge a bunch of adjacent values into one - // value; change unquoted text into a string - // value. - private void consolidateValueTokens() { - // this trick is not done in JSON - if (flavor == ConfigSyntax.JSON) - return; - - List values = null; // create only if we have value tokens - TokenWithComments firstValueWithComments = null; - TokenWithComments t = nextTokenIgnoringNewline(); // ignore a - // newline up - // front - while (Tokens.isValue(t.token) || Tokens.isUnquotedText(t.token) - || Tokens.isSubstitution(t.token)) { - if (values == null) { - values = new ArrayList(); - firstValueWithComments = t; - } - values.add(t.token); - t = nextToken(); // but don't consolidate across a newline - } - // the last one wasn't a value token - putBack(t); - - if (values == null) - return; - - if (values.size() == 1 && Tokens.isValue(firstValueWithComments.token)) { - // a single value token requires no consolidation - putBack(firstValueWithComments); - return; - } - - // this will be a list of String and SubstitutionExpression - List minimized = new ArrayList(); - - // we have multiple value tokens or one unquoted text token; - // collapse into a string token. - StringBuilder sb = new StringBuilder(); - ConfigOrigin firstOrigin = null; - for (Token valueToken : values) { - if (Tokens.isValue(valueToken)) { - AbstractConfigValue v = Tokens.getValue(valueToken); - sb.append(v.transformToString()); - if (firstOrigin == null) - firstOrigin = v.origin(); - } else if (Tokens.isUnquotedText(valueToken)) { - String text = Tokens.getUnquotedText(valueToken); - if (firstOrigin == null) - firstOrigin = valueToken.origin(); - sb.append(text); - } else if (Tokens.isSubstitution(valueToken)) { - if (firstOrigin == null) - firstOrigin = valueToken.origin(); - - if (sb.length() > 0) { - // save string so far - minimized.add(sb.toString()); - sb.setLength(0); - } - // now save substitution - List expression = Tokens - .getSubstitutionPathExpression(valueToken); - Path path = parsePathExpression(expression.iterator(), valueToken.origin()); - boolean optional = Tokens.getSubstitutionOptional(valueToken); - - minimized.add(new SubstitutionExpression(path, optional)); - } else { - throw new ConfigException.BugOrBroken( - "should not be trying to consolidate token: " - + valueToken); - } - } - - if (sb.length() > 0) { - // save string so far - minimized.add(sb.toString()); - } - - if (minimized.isEmpty()) - throw new ConfigException.BugOrBroken( - "trying to consolidate values to nothing"); - - Token consolidated = null; - - if (minimized.size() == 1 && minimized.get(0) instanceof String) { - consolidated = Tokens.newString(firstOrigin, - (String) minimized.get(0)); - } else { - // there's some substitution to do later (post-parse step) - consolidated = Tokens.newValue(new ConfigSubstitution( - firstOrigin, minimized)); - } - - putBack(new TokenWithComments(consolidated, firstValueWithComments.comments)); - } - - private ConfigOrigin lineOrigin() { - return ((SimpleConfigOrigin) baseOrigin).setLineNumber(lineNumber); - } - - private ConfigException parseError(String message) { - return parseError(message, null); - } - - private ConfigException parseError(String message, Throwable cause) { - return new ConfigException.Parse(lineOrigin(), message, cause); - } - - - private String previousFieldName(Path lastPath) { - if (lastPath != null) { - return lastPath.render(); - } else if (pathStack.isEmpty()) - return null; - else - return pathStack.peek().render(); - } - - private String previousFieldName() { - return previousFieldName(null); - } - - private String addKeyName(String message) { - String previousFieldName = previousFieldName(); - if (previousFieldName != null) { - return "in value for key '" + previousFieldName + "': " + message; - } else { - return message; - } - } - - private String addQuoteSuggestion(String badToken, String message) { - return addQuoteSuggestion(null, equalsCount > 0, badToken, message); - } - - private String addQuoteSuggestion(Path lastPath, boolean insideEquals, String badToken, - String message) { - String previousFieldName = previousFieldName(lastPath); - - String part; - if (badToken.equals(Tokens.END.toString())) { - // EOF requires special handling for the error to make sense. - if (previousFieldName != null) - part = message + " (if you intended '" + previousFieldName - + "' to be part of a value, instead of a key, " - + "try adding double quotes around the whole value"; - else - return message; - } else { - if (previousFieldName != null) { - part = message + " (if you intended " + badToken - + " to be part of the value for '" + previousFieldName + "', " - + "try enclosing the value in double quotes"; - } else { - part = message + " (if you intended " + badToken - + " to be part of a key or string value, " - + "try enclosing the key or value in double quotes"; - } - } - - if (insideEquals) - return part - + ", or you may be able to rename the file .properties rather than .conf)"; - else - return part + ")"; - } - - private AbstractConfigValue parseValue(TokenWithComments t) { - AbstractConfigValue v; - - if (Tokens.isValue(t.token)) { - v = Tokens.getValue(t.token); - } else if (t.token == Tokens.OPEN_CURLY) { - v = parseObject(true); - } else if (t.token == Tokens.OPEN_SQUARE) { - v = parseArray(); - } else { - throw parseError(addQuoteSuggestion(t.token.toString(), - "Expecting a value but got wrong token: " + t.token)); - } - - v = v.withOrigin(t.setComments(v.origin())); - - return v; - } - - private static AbstractConfigObject createValueUnderPath(Path path, - AbstractConfigValue value) { - // for path foo.bar, we are creating - // { "foo" : { "bar" : value } } - List keys = new ArrayList(); - - String key = path.first(); - Path remaining = path.remainder(); - while (key != null) { - keys.add(key); - if (remaining == null) { - break; - } else { - key = remaining.first(); - remaining = remaining.remainder(); - } - } - - // the setComments(null) is to ensure comments are only - // on the exact leaf node they apply to. - // a comment before "foo.bar" applies to the full setting - // "foo.bar" not also to "foo" - ListIterator i = keys.listIterator(keys.size()); - String deepest = i.previous(); - AbstractConfigObject o = new SimpleConfigObject(value.origin().setComments(null), - Collections. singletonMap( - deepest, value)); - while (i.hasPrevious()) { - Map m = Collections. singletonMap( - i.previous(), o); - o = new SimpleConfigObject(value.origin().setComments(null), m); - } - - return o; - } - - private Path parseKey(TokenWithComments token) { - if (flavor == ConfigSyntax.JSON) { - if (Tokens.isValueWithType(token.token, ConfigValueType.STRING)) { - String key = (String) Tokens.getValue(token.token).unwrapped(); - return Path.newKey(key); - } else { - throw parseError(addKeyName("Expecting close brace } or a field name here, got " - + token)); - } - } else { - List expression = new ArrayList(); - TokenWithComments t = token; - while (Tokens.isValue(t.token) || Tokens.isUnquotedText(t.token)) { - expression.add(t.token); - t = nextToken(); // note: don't cross a newline - } - - if (expression.isEmpty()) { - throw parseError(addKeyName("expecting a close brace or a field name here, got " - + t)); - } - - putBack(t); // put back the token we ended with - return parsePathExpression(expression.iterator(), lineOrigin()); - } - } - - private static boolean isIncludeKeyword(Token t) { - return Tokens.isUnquotedText(t) - && Tokens.getUnquotedText(t).equals("include"); - } - - private static boolean isUnquotedWhitespace(Token t) { - if (!Tokens.isUnquotedText(t)) - return false; - - String s = Tokens.getUnquotedText(t); - - for (int i = 0; i < s.length(); ++i) { - char c = s.charAt(i); - if (!ConfigImplUtil.isWhitespace(c)) - return false; - } - return true; - } - - private void parseInclude(Map values) { - TokenWithComments t = nextTokenIgnoringNewline(); - while (isUnquotedWhitespace(t.token)) { - t = nextTokenIgnoringNewline(); - } - - if (Tokens.isValueWithType(t.token, ConfigValueType.STRING)) { - String name = (String) Tokens.getValue(t.token).unwrapped(); - AbstractConfigObject obj = (AbstractConfigObject) includer - .include(includeContext, name); - - if (!pathStack.isEmpty()) { - Path prefix = new Path(pathStack); - obj = obj.relativized(prefix); - } - - for (String key : obj.keySet()) { - AbstractConfigValue v = obj.get(key); - AbstractConfigValue existing = values.get(key); - if (existing != null) { - values.put(key, v.withFallback(existing)); - } else { - values.put(key, v); - } - } - - } else { - throw parseError("include keyword is not followed by a quoted string, but by: " - + t); - } - } - - private boolean isKeyValueSeparatorToken(Token t) { - if (flavor == ConfigSyntax.JSON) { - return t == Tokens.COLON; - } else { - return t == Tokens.COLON || t == Tokens.EQUALS; - } - } - - private AbstractConfigObject parseObject(boolean hadOpenCurly) { - // invoked just after the OPEN_CURLY (or START, if !hadOpenCurly) - Map values = new HashMap(); - ConfigOrigin objectOrigin = lineOrigin(); - boolean afterComma = false; - Path lastPath = null; - boolean lastInsideEquals = false; - - while (true) { - TokenWithComments t = nextTokenIgnoringNewline(); - if (t.token == Tokens.CLOSE_CURLY) { - if (flavor == ConfigSyntax.JSON && afterComma) { - throw parseError(addQuoteSuggestion(t.toString(), - "expecting a field name after a comma, got a close brace } instead")); - } else if (!hadOpenCurly) { - throw parseError(addQuoteSuggestion(t.toString(), - "unbalanced close brace '}' with no open brace")); - } - break; - } else if (t.token == Tokens.END && !hadOpenCurly) { - putBack(t); - break; - } else if (flavor != ConfigSyntax.JSON && isIncludeKeyword(t.token)) { - parseInclude(values); - - afterComma = false; - } else { - TokenWithComments keyToken = t; - Path path = parseKey(keyToken); - TokenWithComments afterKey = nextTokenIgnoringNewline(); - boolean insideEquals = false; - - // path must be on-stack while we parse the value - pathStack.push(path); - - TokenWithComments valueToken; - AbstractConfigValue newValue; - if (flavor == ConfigSyntax.CONF && afterKey.token == Tokens.OPEN_CURLY) { - // can omit the ':' or '=' before an object value - valueToken = afterKey; - } else { - if (!isKeyValueSeparatorToken(afterKey.token)) { - throw parseError(addQuoteSuggestion(afterKey.toString(), - "Key '" + path.render() + "' may not be followed by token: " - + afterKey)); - } - - if (afterKey.token == Tokens.EQUALS) { - insideEquals = true; - equalsCount += 1; - } - - consolidateValueTokens(); - valueToken = nextTokenIgnoringNewline(); - } - - newValue = parseValue(valueToken.prepend(keyToken.comments)); - - lastPath = pathStack.pop(); - if (insideEquals) { - equalsCount -= 1; - } - lastInsideEquals = insideEquals; - - String key = path.first(); - Path remaining = path.remainder(); - - if (remaining == null) { - AbstractConfigValue existing = values.get(key); - if (existing != null) { - // In strict JSON, dups should be an error; while in - // our custom config language, they should be merged - // if the value is an object (or substitution that - // could become an object). - - if (flavor == ConfigSyntax.JSON) { - throw parseError("JSON does not allow duplicate fields: '" - + key - + "' was already seen at " - + existing.origin().description()); - } else { - newValue = newValue.withFallback(existing); - } - } - values.put(key, newValue); - } else { - if (flavor == ConfigSyntax.JSON) { - throw new ConfigException.BugOrBroken( - "somehow got multi-element path in JSON mode"); - } - - AbstractConfigObject obj = createValueUnderPath( - remaining, newValue); - AbstractConfigValue existing = values.get(key); - if (existing != null) { - obj = obj.withFallback(existing); - } - values.put(key, obj); - } - - afterComma = false; - } - - if (checkElementSeparator()) { - // continue looping - afterComma = true; - } else { - t = nextTokenIgnoringNewline(); - if (t.token == Tokens.CLOSE_CURLY) { - if (!hadOpenCurly) { - throw parseError(addQuoteSuggestion(lastPath, lastInsideEquals, - t.toString(), "unbalanced close brace '}' with no open brace")); - } - break; - } else if (hadOpenCurly) { - throw parseError(addQuoteSuggestion(lastPath, lastInsideEquals, - t.toString(), "Expecting close brace } or a comma, got " + t)); - } else { - if (t.token == Tokens.END) { - putBack(t); - break; - } else { - throw parseError(addQuoteSuggestion(lastPath, lastInsideEquals, - t.toString(), "Expecting end of input or a comma, got " + t)); - } - } - } - } - - return new SimpleConfigObject(objectOrigin, values); - } - - private SimpleConfigList parseArray() { - // invoked just after the OPEN_SQUARE - ConfigOrigin arrayOrigin = lineOrigin(); - List values = new ArrayList(); - - consolidateValueTokens(); - - TokenWithComments t = nextTokenIgnoringNewline(); - - // special-case the first element - if (t.token == Tokens.CLOSE_SQUARE) { - return new SimpleConfigList(arrayOrigin, - Collections. emptyList()); - } else if (Tokens.isValue(t.token) || t.token == Tokens.OPEN_CURLY - || t.token == Tokens.OPEN_SQUARE) { - values.add(parseValue(t)); - } else { - throw parseError(addKeyName("List should have ] or a first element after the open [, instead had token: " - + t - + " (if you want " - + t - + " to be part of a string value, then double-quote it)")); - } - - // now remaining elements - while (true) { - // just after a value - if (checkElementSeparator()) { - // comma (or newline equivalent) consumed - } else { - t = nextTokenIgnoringNewline(); - if (t.token == Tokens.CLOSE_SQUARE) { - return new SimpleConfigList(arrayOrigin, values); - } else { - throw parseError(addKeyName("List should have ended with ] or had a comma, instead had token: " - + t - + " (if you want " - + t - + " to be part of a string value, then double-quote it)")); - } - } - - // now just after a comma - consolidateValueTokens(); - - t = nextTokenIgnoringNewline(); - if (Tokens.isValue(t.token) || t.token == Tokens.OPEN_CURLY - || t.token == Tokens.OPEN_SQUARE) { - values.add(parseValue(t)); - } else if (flavor != ConfigSyntax.JSON && t.token == Tokens.CLOSE_SQUARE) { - // we allow one trailing comma - putBack(t); - } else { - throw parseError(addKeyName("List should have had new element after a comma, instead had token: " - + t - + " (if you want the comma or " - + t - + " to be part of a string value, then double-quote it)")); - } - } - } - - AbstractConfigValue parse() { - TokenWithComments t = nextTokenIgnoringNewline(); - if (t.token == Tokens.START) { - // OK - } else { - throw new ConfigException.BugOrBroken( - "token stream did not begin with START, had " + t); - } - - t = nextTokenIgnoringNewline(); - AbstractConfigValue result = null; - if (t.token == Tokens.OPEN_CURLY || t.token == Tokens.OPEN_SQUARE) { - result = parseValue(t); - } else { - if (flavor == ConfigSyntax.JSON) { - if (t.token == Tokens.END) { - throw parseError("Empty document"); - } else { - throw parseError("Document must have an object or array at root, unexpected token: " - + t); - } - } else { - // the root object can omit the surrounding braces. - // this token should be the first field's key, or part - // of it, so put it back. - putBack(t); - result = parseObject(false); - // in this case we don't try to use commentsStack comments - // since they would all presumably apply to fields not the - // root object - } - } - - t = nextTokenIgnoringNewline(); - if (t.token == Tokens.END) { - return result; - } else { - throw parseError("Document has trailing tokens after first object or array: " - + t); - } - } - } - - static class Element { - StringBuilder sb; - // an element can be empty if it has a quoted empty string "" in it - boolean canBeEmpty; - - Element(String initial, boolean canBeEmpty) { - this.canBeEmpty = canBeEmpty; - this.sb = new StringBuilder(initial); - } - - @Override - public String toString() { - return "Element(" + sb.toString() + "," + canBeEmpty + ")"; - } - } - - private static void addPathText(List buf, boolean wasQuoted, - String newText) { - int i = wasQuoted ? -1 : newText.indexOf('.'); - Element current = buf.get(buf.size() - 1); - if (i < 0) { - // add to current path element - current.sb.append(newText); - // any empty quoted string means this element can - // now be empty. - if (wasQuoted && current.sb.length() == 0) - current.canBeEmpty = true; - } else { - // "buf" plus up to the period is an element - current.sb.append(newText.substring(0, i)); - // then start a new element - buf.add(new Element("", false)); - // recurse to consume remainder of newText - addPathText(buf, false, newText.substring(i + 1)); - } - } - - private static Path parsePathExpression(Iterator expression, - ConfigOrigin origin) { - return parsePathExpression(expression, origin, null); - } - - // originalText may be null if not available - private static Path parsePathExpression(Iterator expression, - ConfigOrigin origin, String originalText) { - // each builder in "buf" is an element in the path. - List buf = new ArrayList(); - buf.add(new Element("", false)); - - if (!expression.hasNext()) { - throw new ConfigException.BadPath(origin, originalText, - "Expecting a field name or path here, but got nothing"); - } - - while (expression.hasNext()) { - Token t = expression.next(); - if (Tokens.isValueWithType(t, ConfigValueType.STRING)) { - AbstractConfigValue v = Tokens.getValue(t); - // this is a quoted string; so any periods - // in here don't count as path separators - String s = v.transformToString(); - - addPathText(buf, true, s); - } else if (t == Tokens.END) { - // ignore this; when parsing a file, it should not happen - // since we're parsing a token list rather than the main - // token iterator, and when parsing a path expression from the - // API, it's expected to have an END. - } else { - // any periods outside of a quoted string count as - // separators - String text; - if (Tokens.isValue(t)) { - // appending a number here may add - // a period, but we _do_ count those as path - // separators, because we basically want - // "foo 3.0bar" to parse as a string even - // though there's a number in it. The fact that - // we tokenize non-string values is largely an - // implementation detail. - AbstractConfigValue v = Tokens.getValue(t); - text = v.transformToString(); - } else if (Tokens.isUnquotedText(t)) { - text = Tokens.getUnquotedText(t); - } else { - throw new ConfigException.BadPath( - origin, - originalText, - "Token not allowed in path expression: " - + t - + " (you can double-quote this token if you really want it here)"); - } - - addPathText(buf, false, text); - } - } - - PathBuilder pb = new PathBuilder(); - for (Element e : buf) { - if (e.sb.length() == 0 && !e.canBeEmpty) { - throw new ConfigException.BadPath( - origin, - originalText, - "path has a leading, trailing, or two adjacent period '.' (use quoted \"\" empty string if you want an empty element)"); - } else { - pb.appendKey(e.sb.toString()); - } - } - - return pb.result(); - } - - static ConfigOrigin apiOrigin = SimpleConfigOrigin.newSimple("path parameter"); - - static Path parsePath(String path) { - Path speculated = speculativeFastParsePath(path); - if (speculated != null) - return speculated; - - StringReader reader = new StringReader(path); - - try { - Iterator tokens = Tokenizer.tokenize(apiOrigin, reader, - ConfigSyntax.CONF); - tokens.next(); // drop START - return parsePathExpression(tokens, apiOrigin, path); - } finally { - reader.close(); - } - } - - // the idea is to see if the string has any chars that might require the - // full parser to deal with. - private static boolean hasUnsafeChars(String s) { - for (int i = 0; i < s.length(); ++i) { - char c = s.charAt(i); - if (Character.isLetter(c) || c == '.') - continue; - else - return true; - } - return false; - } - - private static void appendPathString(PathBuilder pb, String s) { - int splitAt = s.indexOf('.'); - if (splitAt < 0) { - pb.appendKey(s); - } else { - pb.appendKey(s.substring(0, splitAt)); - appendPathString(pb, s.substring(splitAt + 1)); - } - } - - // do something much faster than the full parser if - // we just have something like "foo" or "foo.bar" - private static Path speculativeFastParsePath(String path) { - String s = ConfigImplUtil.unicodeTrim(path); - if (s.isEmpty()) - return null; - if (hasUnsafeChars(s)) - return null; - if (s.startsWith(".") || s.endsWith(".") || s.contains("..")) - return null; // let the full parser throw the error - - PathBuilder pb = new PathBuilder(); - appendPathString(pb, s); - return pb.result(); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Path.java b/akka-actor/src/main/java/com/typesafe/config/impl/Path.java deleted file mode 100755 index fbbe1e0874..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Path.java +++ /dev/null @@ -1,208 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.Serializable; -import java.util.Iterator; -import java.util.List; - -import com.typesafe.config.ConfigException; - -final class Path implements Serializable { - - private static final long serialVersionUID = 1L; - - final private String first; - final private Path remainder; - - Path(String first, Path remainder) { - this.first = first; - this.remainder = remainder; - } - - Path(String... elements) { - if (elements.length == 0) - throw new ConfigException.BugOrBroken("empty path"); - this.first = elements[0]; - if (elements.length > 1) { - PathBuilder pb = new PathBuilder(); - for (int i = 1; i < elements.length; ++i) { - pb.appendKey(elements[i]); - } - this.remainder = pb.result(); - } else { - this.remainder = null; - } - } - - // append all the paths in the list together into one path - Path(List pathsToConcat) { - if (pathsToConcat.isEmpty()) - throw new ConfigException.BugOrBroken("empty path"); - - Iterator i = pathsToConcat.iterator(); - Path firstPath = i.next(); - this.first = firstPath.first; - - PathBuilder pb = new PathBuilder(); - if (firstPath.remainder != null) { - pb.appendPath(firstPath.remainder); - } - while (i.hasNext()) { - pb.appendPath(i.next()); - } - this.remainder = pb.result(); - } - - String first() { - return first; - } - - /** - * - * @return path minus the first element or null if no more elements - */ - Path remainder() { - return remainder; - } - - /** - * - * @return path minus the last element or null if we have just one element - */ - Path parent() { - if (remainder == null) - return null; - - PathBuilder pb = new PathBuilder(); - Path p = this; - while (p.remainder != null) { - pb.appendKey(p.first); - p = p.remainder; - } - return pb.result(); - } - - /** - * - * @return last element in the path - */ - String last() { - Path p = this; - while (p.remainder != null) { - p = p.remainder; - } - return p.first; - } - - Path prepend(Path toPrepend) { - PathBuilder pb = new PathBuilder(); - pb.appendPath(toPrepend); - pb.appendPath(this); - return pb.result(); - } - - int length() { - int count = 1; - Path p = remainder; - while (p != null) { - count += 1; - p = p.remainder; - } - return count; - } - - Path subPath(int removeFromFront) { - int count = removeFromFront; - Path p = this; - while (p != null && count > 0) { - count -= 1; - p = p.remainder; - } - return p; - } - - @Override - public boolean equals(Object other) { - if (other instanceof Path) { - Path that = (Path) other; - return this.first.equals(that.first) - && ConfigImplUtil.equalsHandlingNull(this.remainder, - that.remainder); - } else { - return false; - } - } - - @Override - public int hashCode() { - return 41 * (41 + first.hashCode()) - + (remainder == null ? 0 : remainder.hashCode()); - } - - // this doesn't have a very precise meaning, just to reduce - // noise from quotes in the rendered path for average cases - static boolean hasFunkyChars(String s) { - int length = s.length(); - - if (length == 0) - return false; - - // if the path starts with something that could be a number, - // we need to quote it because the number could be invalid, - // for example it could be a hyphen with no digit afterward - // or the exponent "e" notation could be mangled. - char first = s.charAt(0); - if (!(Character.isLetter(first))) - return true; - - for (int i = 1; i < length; ++i) { - char c = s.charAt(i); - - if (Character.isLetterOrDigit(c) || c == '-' || c == '_') - continue; - else - return true; - } - return false; - } - - private void appendToStringBuilder(StringBuilder sb) { - if (hasFunkyChars(first) || first.isEmpty()) - sb.append(ConfigImplUtil.renderJsonString(first)); - else - sb.append(first); - if (remainder != null) { - sb.append("."); - remainder.appendToStringBuilder(sb); - } - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("Path("); - appendToStringBuilder(sb); - sb.append(")"); - return sb.toString(); - } - - /** - * toString() is a debugging-oriented version while this is an - * error-message-oriented human-readable one. - */ - String render() { - StringBuilder sb = new StringBuilder(); - appendToStringBuilder(sb); - return sb.toString(); - } - - static Path newKey(String key) { - return new Path(key, null); - } - - static Path newPath(String path) { - return Parser.parsePath(path); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java b/akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java deleted file mode 100755 index ede6c66387..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java +++ /dev/null @@ -1,60 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.util.Stack; - -import com.typesafe.config.ConfigException; - -final class PathBuilder { - // the keys are kept "backward" (top of stack is end of path) - final private Stack keys; - private Path result; - - PathBuilder() { - keys = new Stack(); - } - - private void checkCanAppend() { - if (result != null) - throw new ConfigException.BugOrBroken( - "Adding to PathBuilder after getting result"); - } - - void appendKey(String key) { - checkCanAppend(); - - keys.push(key); - } - - void appendPath(Path path) { - checkCanAppend(); - - String first = path.first(); - Path remainder = path.remainder(); - while (true) { - keys.push(first); - if (remainder != null) { - first = remainder.first(); - remainder = remainder.remainder(); - } else { - break; - } - } - } - - Path result() { - // note: if keys is empty, we want to return null, which is a valid - // empty path - if (result == null) { - Path remainder = null; - while (!keys.isEmpty()) { - String key = keys.pop(); - remainder = new Path(key, remainder); - } - result = remainder; - } - return result; - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java b/akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java deleted file mode 100755 index 7c8c81fb07..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java +++ /dev/null @@ -1,191 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.IOException; -import java.io.Reader; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigOrigin; - -final class PropertiesParser { - static AbstractConfigObject parse(Reader reader, - ConfigOrigin origin) throws IOException { - Properties props = new Properties(); - props.load(reader); - return fromProperties(origin, props); - } - - static String lastElement(String path) { - int i = path.lastIndexOf('.'); - if (i < 0) - return path; - else - return path.substring(i + 1); - } - - static String exceptLastElement(String path) { - int i = path.lastIndexOf('.'); - if (i < 0) - return null; - else - return path.substring(0, i); - } - - static Path pathFromPropertyKey(String key) { - String last = lastElement(key); - String exceptLast = exceptLastElement(key); - Path path = new Path(last, null); - while (exceptLast != null) { - last = lastElement(exceptLast); - exceptLast = exceptLastElement(exceptLast); - path = new Path(last, path); - } - return path; - } - - static AbstractConfigObject fromProperties(ConfigOrigin origin, - Properties props) { - Map pathMap = new HashMap(); - for (Map.Entry entry : props.entrySet()) { - Object key = entry.getKey(); - if (key instanceof String) { - Path path = pathFromPropertyKey((String) key); - pathMap.put(path, entry.getValue()); - } - } - return fromPathMap(origin, pathMap, true /* from properties */); - } - - static AbstractConfigObject fromPathMap(ConfigOrigin origin, - Map pathExpressionMap) { - Map pathMap = new HashMap(); - for (Map.Entry entry : pathExpressionMap.entrySet()) { - Object keyObj = entry.getKey(); - if (!(keyObj instanceof String)) { - throw new ConfigException.BugOrBroken( - "Map has a non-string as a key, expecting a path expression as a String"); - } - Path path = Path.newPath((String) keyObj); - pathMap.put(path, entry.getValue()); - } - return fromPathMap(origin, pathMap, false /* from properties */); - } - - private static AbstractConfigObject fromPathMap(ConfigOrigin origin, - Map pathMap, boolean convertedFromProperties) { - /* - * First, build a list of paths that will have values, either string or - * object values. - */ - Set scopePaths = new HashSet(); - Set valuePaths = new HashSet(); - for (Path path : pathMap.keySet()) { - // add value's path - valuePaths.add(path); - - // all parent paths are objects - Path next = path.parent(); - while (next != null) { - scopePaths.add(next); - next = next.parent(); - } - } - - if (convertedFromProperties) { - /* - * If any string values are also objects containing other values, - * drop those string values - objects "win". - */ - valuePaths.removeAll(scopePaths); - } else { - /* If we didn't start out as properties, then this is an error. */ - for (Path path : valuePaths) { - if (scopePaths.contains(path)) { - throw new ConfigException.BugOrBroken( - "In the map, path '" - + path.render() - + "' occurs as both the parent object of a value and as a value. " - + "Because Map has no defined ordering, this is a broken situation."); - } - } - } - - /* - * Create maps for the object-valued values. - */ - Map root = new HashMap(); - Map> scopes = new HashMap>(); - - for (Path path : scopePaths) { - Map scope = new HashMap(); - scopes.put(path, scope); - } - - /* Store string values in the associated scope maps */ - for (Path path : valuePaths) { - Path parentPath = path.parent(); - Map parent = parentPath != null ? scopes - .get(parentPath) : root; - - String last = path.last(); - Object rawValue = pathMap.get(path); - AbstractConfigValue value; - if (convertedFromProperties) { - value = new ConfigString(origin, (String) rawValue); - } else { - value = ConfigImpl.fromAnyRef(pathMap.get(path), origin, - FromMapMode.KEYS_ARE_PATHS); - } - parent.put(last, value); - } - - /* - * Make a list of scope paths from longest to shortest, so children go - * before parents. - */ - List sortedScopePaths = new ArrayList(); - sortedScopePaths.addAll(scopePaths); - // sort descending by length - Collections.sort(sortedScopePaths, new Comparator() { - @Override - public int compare(Path a, Path b) { - // Path.length() is O(n) so in theory this sucks - // but in practice we can make Path precompute length - // if it ever matters. - return b.length() - a.length(); - } - }); - - /* - * Create ConfigObject for each scope map, working from children to - * parents to avoid modifying any already-created ConfigObject. This is - * where we need the sorted list. - */ - for (Path scopePath : sortedScopePaths) { - Map scope = scopes.get(scopePath); - - Path parentPath = scopePath.parent(); - Map parent = parentPath != null ? scopes - .get(parentPath) : root; - - AbstractConfigObject o = new SimpleConfigObject(origin, scope, - ResolveStatus.RESOLVED, false /* ignoresFallbacks */); - parent.put(scopePath.last(), o); - } - - // return root config object - return new SimpleConfigObject(origin, root, ResolveStatus.RESOLVED, - false /* ignoresFallbacks */); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java b/akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java deleted file mode 100755 index 8deeaf520f..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java +++ /dev/null @@ -1,26 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.util.Collection; - -/** - * Status of substitution resolution. - */ -enum ResolveStatus { - UNRESOLVED, RESOLVED; - - final static ResolveStatus fromValues( - Collection values) { - for (AbstractConfigValue v : values) { - if (v.resolveStatus() == ResolveStatus.UNRESOLVED) - return ResolveStatus.UNRESOLVED; - } - return ResolveStatus.RESOLVED; - } - - final static ResolveStatus fromBoolean(boolean resolved) { - return resolved ? ResolveStatus.RESOLVED : ResolveStatus.UNRESOLVED; - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java deleted file mode 100755 index 0ab776b9af..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java +++ /dev/null @@ -1,841 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.Serializable; -import java.util.AbstractMap; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import com.typesafe.config.Config; -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigList; -import com.typesafe.config.ConfigMergeable; -import com.typesafe.config.ConfigObject; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigResolveOptions; -import com.typesafe.config.ConfigValue; -import com.typesafe.config.ConfigValueType; - -/** - * One thing to keep in mind in the future: as Collection-like APIs are added - * here, including iterators or size() or anything, they should be consistent - * with a one-level java.util.Map from paths to non-null values. Null values are - * not "in" the map. - */ -final class SimpleConfig implements Config, MergeableValue, Serializable { - - private static final long serialVersionUID = 1L; - - final private AbstractConfigObject object; - - SimpleConfig(AbstractConfigObject object) { - this.object = object; - } - - @Override - public AbstractConfigObject root() { - return object; - } - - @Override - public ConfigOrigin origin() { - return object.origin(); - } - - @Override - public SimpleConfig resolve() { - return resolve(ConfigResolveOptions.defaults()); - } - - @Override - public SimpleConfig resolve(ConfigResolveOptions options) { - AbstractConfigValue resolved = SubstitutionResolver.resolve(object, - object, options); - if (resolved == object) - return this; - else - return new SimpleConfig((AbstractConfigObject) resolved); - } - - - @Override - public boolean hasPath(String pathExpression) { - Path path = Path.newPath(pathExpression); - ConfigValue peeked = object.peekPath(path, null, 0, null); - return peeked != null && peeked.valueType() != ConfigValueType.NULL; - } - - @Override - public boolean isEmpty() { - return object.isEmpty(); - } - - private static void findPaths(Set> entries, Path parent, - AbstractConfigObject obj) { - for (Map.Entry entry : obj.entrySet()) { - String elem = entry.getKey(); - ConfigValue v = entry.getValue(); - Path path = Path.newKey(elem); - if (parent != null) - path = path.prepend(parent); - if (v instanceof AbstractConfigObject) { - findPaths(entries, path, (AbstractConfigObject) v); - } else if (v instanceof ConfigNull) { - // nothing; nulls are conceptually not in a Config - } else { - entries.add(new AbstractMap.SimpleImmutableEntry(path.render(), v)); - } - } - } - - @Override - public Set> entrySet() { - Set> entries = new HashSet>(); - findPaths(entries, null, object); - return entries; - } - - static private AbstractConfigValue find(AbstractConfigObject self, - String pathExpression, ConfigValueType expected, String originalPath) { - Path path = Path.newPath(pathExpression); - return find(self, path, expected, originalPath); - } - - static private AbstractConfigValue findKey(AbstractConfigObject self, - String key, ConfigValueType expected, String originalPath) { - AbstractConfigValue v = self.peek(key); - if (v == null) - throw new ConfigException.Missing(originalPath); - - if (expected != null) - v = DefaultTransformer.transform(v, expected); - - if (v.valueType() == ConfigValueType.NULL) - throw new ConfigException.Null(v.origin(), originalPath, - expected != null ? expected.name() : null); - else if (expected != null && v.valueType() != expected) - throw new ConfigException.WrongType(v.origin(), originalPath, - expected.name(), v.valueType().name()); - else - return v; - } - - static private AbstractConfigValue find(AbstractConfigObject self, - Path path, ConfigValueType expected, String originalPath) { - String key = path.first(); - Path next = path.remainder(); - if (next == null) { - return findKey(self, key, expected, originalPath); - } else { - AbstractConfigObject o = (AbstractConfigObject) findKey(self, key, - ConfigValueType.OBJECT, originalPath); - assert (o != null); // missing was supposed to throw - return find(o, next, expected, originalPath); - } - } - - AbstractConfigValue find(String pathExpression, ConfigValueType expected, - String originalPath) { - return find(object, pathExpression, expected, originalPath); - } - - @Override - public AbstractConfigValue getValue(String path) { - return find(path, null, path); - } - - @Override - public boolean getBoolean(String path) { - ConfigValue v = find(path, ConfigValueType.BOOLEAN, path); - return (Boolean) v.unwrapped(); - } - - private ConfigNumber getConfigNumber(String path) { - ConfigValue v = find(path, ConfigValueType.NUMBER, path); - return (ConfigNumber) v; - } - - @Override - public Number getNumber(String path) { - return getConfigNumber(path).unwrapped(); - } - - @Override - public int getInt(String path) { - ConfigNumber n = getConfigNumber(path); - return n.intValueRangeChecked(path); - } - - @Override - public long getLong(String path) { - return getNumber(path).longValue(); - } - - @Override - public double getDouble(String path) { - return getNumber(path).doubleValue(); - } - - @Override - public String getString(String path) { - ConfigValue v = find(path, ConfigValueType.STRING, path); - return (String) v.unwrapped(); - } - - @Override - public ConfigList getList(String path) { - AbstractConfigValue v = find(path, ConfigValueType.LIST, path); - return (ConfigList) v; - } - - @Override - public AbstractConfigObject getObject(String path) { - AbstractConfigObject obj = (AbstractConfigObject) find(path, - ConfigValueType.OBJECT, path); - return obj; - } - - @Override - public SimpleConfig getConfig(String path) { - return getObject(path).toConfig(); - } - - @Override - public Object getAnyRef(String path) { - ConfigValue v = find(path, null, path); - return v.unwrapped(); - } - - @Override - public Long getBytes(String path) { - Long size = null; - try { - size = getLong(path); - } catch (ConfigException.WrongType e) { - ConfigValue v = find(path, ConfigValueType.STRING, path); - size = parseBytes((String) v.unwrapped(), - v.origin(), path); - } - return size; - } - - @Override - public Long getMilliseconds(String path) { - long ns = getNanoseconds(path); - long ms = TimeUnit.NANOSECONDS.toMillis(ns); - return ms; - } - - @Override - public Long getNanoseconds(String path) { - Long ns = null; - try { - ns = TimeUnit.MILLISECONDS.toNanos(getLong(path)); - } catch (ConfigException.WrongType e) { - ConfigValue v = find(path, ConfigValueType.STRING, path); - ns = parseDuration((String) v.unwrapped(), v.origin(), path); - } - return ns; - } - - @SuppressWarnings("unchecked") - private List getHomogeneousUnwrappedList(String path, - ConfigValueType expected) { - List l = new ArrayList(); - List list = getList(path); - for (ConfigValue cv : list) { - // variance would be nice, but stupid cast will do - AbstractConfigValue v = (AbstractConfigValue) cv; - if (expected != null) { - v = DefaultTransformer.transform(v, expected); - } - if (v.valueType() != expected) - throw new ConfigException.WrongType(v.origin(), path, - "list of " + expected.name(), "list of " - + v.valueType().name()); - l.add((T) v.unwrapped()); - } - return l; - } - - @Override - public List getBooleanList(String path) { - return getHomogeneousUnwrappedList(path, ConfigValueType.BOOLEAN); - } - - @Override - public List getNumberList(String path) { - return getHomogeneousUnwrappedList(path, ConfigValueType.NUMBER); - } - - @Override - public List getIntList(String path) { - List l = new ArrayList(); - List numbers = getHomogeneousWrappedList(path, ConfigValueType.NUMBER); - for (AbstractConfigValue v : numbers) { - l.add(((ConfigNumber) v).intValueRangeChecked(path)); - } - return l; - } - - @Override - public List getLongList(String path) { - List l = new ArrayList(); - List numbers = getNumberList(path); - for (Number n : numbers) { - l.add(n.longValue()); - } - return l; - } - - @Override - public List getDoubleList(String path) { - List l = new ArrayList(); - List numbers = getNumberList(path); - for (Number n : numbers) { - l.add(n.doubleValue()); - } - return l; - } - - @Override - public List getStringList(String path) { - return getHomogeneousUnwrappedList(path, ConfigValueType.STRING); - } - - @SuppressWarnings("unchecked") - private List getHomogeneousWrappedList( - String path, ConfigValueType expected) { - List l = new ArrayList(); - List list = getList(path); - for (ConfigValue cv : list) { - // variance would be nice, but stupid cast will do - AbstractConfigValue v = (AbstractConfigValue) cv; - if (expected != null) { - v = DefaultTransformer.transform(v, expected); - } - if (v.valueType() != expected) - throw new ConfigException.WrongType(v.origin(), path, - "list of " + expected.name(), "list of " - + v.valueType().name()); - l.add((T) v); - } - return l; - } - - @Override - public List getObjectList(String path) { - return getHomogeneousWrappedList(path, ConfigValueType.OBJECT); - } - - @Override - public List getConfigList(String path) { - List objects = getObjectList(path); - List l = new ArrayList(); - for (ConfigObject o : objects) { - l.add(o.toConfig()); - } - return l; - } - - @Override - public List getAnyRefList(String path) { - List l = new ArrayList(); - List list = getList(path); - for (ConfigValue v : list) { - l.add(v.unwrapped()); - } - return l; - } - - @Override - public List getBytesList(String path) { - List l = new ArrayList(); - List list = getList(path); - for (ConfigValue v : list) { - if (v.valueType() == ConfigValueType.NUMBER) { - l.add(((Number) v.unwrapped()).longValue()); - } else if (v.valueType() == ConfigValueType.STRING) { - String s = (String) v.unwrapped(); - Long n = parseBytes(s, v.origin(), path); - l.add(n); - } else { - throw new ConfigException.WrongType(v.origin(), path, - "memory size string or number of bytes", v.valueType() - .name()); - } - } - return l; - } - - @Override - public List getMillisecondsList(String path) { - List nanos = getNanosecondsList(path); - List l = new ArrayList(); - for (Long n : nanos) { - l.add(TimeUnit.NANOSECONDS.toMillis(n)); - } - return l; - } - - @Override - public List getNanosecondsList(String path) { - List l = new ArrayList(); - List list = getList(path); - for (ConfigValue v : list) { - if (v.valueType() == ConfigValueType.NUMBER) { - l.add(TimeUnit.MILLISECONDS.toNanos(((Number) v.unwrapped()) - .longValue())); - } else if (v.valueType() == ConfigValueType.STRING) { - String s = (String) v.unwrapped(); - Long n = parseDuration(s, v.origin(), path); - l.add(n); - } else { - throw new ConfigException.WrongType(v.origin(), path, - "duration string or number of nanoseconds", v - .valueType().name()); - } - } - return l; - } - - @Override - public AbstractConfigObject toFallbackValue() { - return object; - } - - @Override - public SimpleConfig withFallback(ConfigMergeable other) { - // this can return "this" if the withFallback doesn't need a new - // ConfigObject - return object.withFallback(other).toConfig(); - } - - @Override - public final boolean equals(Object other) { - if (other instanceof SimpleConfig) { - return object.equals(((SimpleConfig) other).object); - } else { - return false; - } - } - - @Override - public final int hashCode() { - // we do the "41*" just so our hash code won't match that of the - // underlying object. there's no real reason it can't match, but - // making it not match might catch some kinds of bug. - return 41 * object.hashCode(); - } - - @Override - public String toString() { - return "Config(" + object.toString() + ")"; - } - - private static String getUnits(String s) { - int i = s.length() - 1; - while (i >= 0) { - char c = s.charAt(i); - if (!Character.isLetter(c)) - break; - i -= 1; - } - return s.substring(i + 1); - } - - /** - * Parses a duration string. If no units are specified in the string, it is - * assumed to be in milliseconds. The returned duration is in nanoseconds. - * The purpose of this function is to implement the duration-related methods - * in the ConfigObject interface. - * - * @param input - * the string to parse - * @param originForException - * origin of the value being parsed - * @param pathForException - * path to include in exceptions - * @return duration in nanoseconds - * @throws ConfigException - * if string is invalid - */ - public static long parseDuration(String input, - ConfigOrigin originForException, String pathForException) { - String s = ConfigImplUtil.unicodeTrim(input); - String originalUnitString = getUnits(s); - String unitString = originalUnitString; - String numberString = ConfigImplUtil.unicodeTrim(s.substring(0, s.length() - - unitString.length())); - TimeUnit units = null; - - // this would be caught later anyway, but the error message - // is more helpful if we check it here. - if (numberString.length() == 0) - throw new ConfigException.BadValue(originForException, - pathForException, "No number in duration value '" + input - + "'"); - - if (unitString.length() > 2 && !unitString.endsWith("s")) - unitString = unitString + "s"; - - // note that this is deliberately case-sensitive - if (unitString.equals("") || unitString.equals("ms") - || unitString.equals("milliseconds")) { - units = TimeUnit.MILLISECONDS; - } else if (unitString.equals("us") || unitString.equals("microseconds")) { - units = TimeUnit.MICROSECONDS; - } else if (unitString.equals("ns") || unitString.equals("nanoseconds")) { - units = TimeUnit.NANOSECONDS; - } else if (unitString.equals("d") || unitString.equals("days")) { - units = TimeUnit.DAYS; - } else if (unitString.equals("h") || unitString.equals("hours")) { - units = TimeUnit.HOURS; - } else if (unitString.equals("s") || unitString.equals("seconds")) { - units = TimeUnit.SECONDS; - } else if (unitString.equals("m") || unitString.equals("minutes")) { - units = TimeUnit.MINUTES; - } else { - throw new ConfigException.BadValue(originForException, - pathForException, "Could not parse time unit '" - + originalUnitString - + "' (try ns, us, ms, s, m, d)"); - } - - try { - // if the string is purely digits, parse as an integer to avoid - // possible precision loss; - // otherwise as a double. - if (numberString.matches("[0-9]+")) { - return units.toNanos(Long.parseLong(numberString)); - } else { - long nanosInUnit = units.toNanos(1); - return (long) (Double.parseDouble(numberString) * nanosInUnit); - } - } catch (NumberFormatException e) { - throw new ConfigException.BadValue(originForException, - pathForException, "Could not parse duration number '" - + numberString + "'"); - } - } - - private static enum MemoryUnit { - BYTES("", 1024, 0), - - KILOBYTES("kilo", 1000, 1), - MEGABYTES("mega", 1000, 2), - GIGABYTES("giga", 1000, 3), - TERABYTES("tera", 1000, 4), - PETABYTES("peta", 1000, 5), - EXABYTES("exa", 1000, 6), - ZETTABYTES("zetta", 1000, 7), - YOTTABYTES("yotta", 1000, 8), - - KIBIBYTES("kibi", 1024, 1), - MEBIBYTES("mebi", 1024, 2), - GIBIBYTES("gibi", 1024, 3), - TEBIBYTES("tebi", 1024, 4), - PEBIBYTES("pebi", 1024, 5), - EXBIBYTES("exbi", 1024, 6), - ZEBIBYTES("zebi", 1024, 7), - YOBIBYTES("yobi", 1024, 8); - - final String prefix; - final int powerOf; - final int power; - final long bytes; - - MemoryUnit(String prefix, int powerOf, int power) { - this.prefix = prefix; - this.powerOf = powerOf; - this.power = power; - int i = power; - long bytes = 1; - while (i > 0) { - bytes *= powerOf; - --i; - } - this.bytes = bytes; - } - - private static Map makeUnitsMap() { - Map map = new HashMap(); - for (MemoryUnit unit : MemoryUnit.values()) { - map.put(unit.prefix + "byte", unit); - map.put(unit.prefix + "bytes", unit); - if (unit.prefix.length() == 0) { - map.put("b", unit); - map.put("B", unit); - map.put("", unit); // no unit specified means bytes - } else { - String first = unit.prefix.substring(0, 1); - String firstUpper = first.toUpperCase(); - if (unit.powerOf == 1024) { - map.put(first, unit); // 512m - map.put(firstUpper, unit); // 512M - map.put(firstUpper + "i", unit); // 512Mi - map.put(firstUpper + "iB", unit); // 512MiB - } else if (unit.powerOf == 1000) { - if (unit.power == 1) { - map.put(first + "B", unit); // 512kB - } else { - map.put(firstUpper + "B", unit); // 512MB - } - } else { - throw new RuntimeException("broken MemoryUnit enum"); - } - } - } - return map; - } - - private static Map unitsMap = makeUnitsMap(); - - static MemoryUnit parseUnit(String unit) { - return unitsMap.get(unit); - } - } - - /** - * Parses a size-in-bytes string. If no units are specified in the string, - * it is assumed to be in bytes. The returned value is in bytes. The purpose - * of this function is to implement the size-in-bytes-related methods in the - * Config interface. - * - * @param input - * the string to parse - * @param originForException - * origin of the value being parsed - * @param pathForException - * path to include in exceptions - * @return size in bytes - * @throws ConfigException - * if string is invalid - */ - public static long parseBytes(String input, ConfigOrigin originForException, - String pathForException) { - String s = ConfigImplUtil.unicodeTrim(input); - String unitString = getUnits(s); - String numberString = ConfigImplUtil.unicodeTrim(s.substring(0, - s.length() - unitString.length())); - - // this would be caught later anyway, but the error message - // is more helpful if we check it here. - if (numberString.length() == 0) - throw new ConfigException.BadValue(originForException, - pathForException, "No number in size-in-bytes value '" - + input + "'"); - - MemoryUnit units = MemoryUnit.parseUnit(unitString); - - if (units == null) { - throw new ConfigException.BadValue(originForException, pathForException, - "Could not parse size-in-bytes unit '" + unitString - + "' (try k, K, kB, KiB, kilobytes, kibibytes)"); - } - - try { - // if the string is purely digits, parse as an integer to avoid - // possible precision loss; otherwise as a double. - if (numberString.matches("[0-9]+")) { - return Long.parseLong(numberString) * units.bytes; - } else { - return (long) (Double.parseDouble(numberString) * units.bytes); - } - } catch (NumberFormatException e) { - throw new ConfigException.BadValue(originForException, pathForException, - "Could not parse size-in-bytes number '" + numberString + "'"); - } - } - - private AbstractConfigValue peekPath(Path path) { - return root().peekPath(path); - } - - private static void addProblem(List accumulator, Path path, - ConfigOrigin origin, String problem) { - accumulator.add(new ConfigException.ValidationProblem(path.render(), origin, problem)); - } - - private static String getDesc(ConfigValue refValue) { - if (refValue instanceof AbstractConfigObject) { - AbstractConfigObject obj = (AbstractConfigObject) refValue; - if (obj.isEmpty()) - return "object"; - else - return "object with keys " + obj.keySet(); - } else if (refValue instanceof SimpleConfigList) { - return "list"; - } else { - return refValue.valueType().name().toLowerCase(); - } - } - - private static void addMissing(List accumulator, - ConfigValue refValue, Path path, ConfigOrigin origin) { - addProblem(accumulator, path, origin, "No setting at '" + path.render() + "', expecting: " - + getDesc(refValue)); - } - - private static void addWrongType(List accumulator, - ConfigValue refValue, AbstractConfigValue actual, Path path) { - addProblem(accumulator, path, actual.origin(), "Wrong value type at '" + path.render() - + "', expecting: " + getDesc(refValue) + " but got: " - + getDesc(actual)); - } - - private static boolean couldBeNull(AbstractConfigValue v) { - return DefaultTransformer.transform(v, ConfigValueType.NULL) - .valueType() == ConfigValueType.NULL; - } - - private static boolean haveCompatibleTypes(ConfigValue reference, AbstractConfigValue value) { - if (couldBeNull((AbstractConfigValue) reference) || couldBeNull(value)) { - // we allow any setting to be null - return true; - } else if (reference instanceof AbstractConfigObject) { - if (value instanceof AbstractConfigObject) { - return true; - } else { - return false; - } - } else if (reference instanceof SimpleConfigList) { - if (value instanceof SimpleConfigList) { - return true; - } else { - return false; - } - } else if (reference instanceof ConfigString) { - // assume a string could be gotten as any non-collection type; - // allows things like getMilliseconds including domain-specific - // interpretations of strings - return true; - } else if (value instanceof ConfigString) { - // assume a string could be gotten as any non-collection type - return true; - } else { - if (reference.valueType() == value.valueType()) { - return true; - } else { - return false; - } - } - } - - // path is null if we're at the root - private static void checkValidObject(Path path, AbstractConfigObject reference, - AbstractConfigObject value, - List accumulator) { - for (Map.Entry entry : reference.entrySet()) { - String key = entry.getKey(); - - Path childPath; - if (path != null) - childPath = Path.newKey(key).prepend(path); - else - childPath = Path.newKey(key); - - AbstractConfigValue v = value.get(key); - if (v == null) { - addMissing(accumulator, entry.getValue(), childPath, value.origin()); - } else { - checkValid(childPath, entry.getValue(), v, accumulator); - } - } - } - - private static void checkValid(Path path, ConfigValue reference, AbstractConfigValue value, - List accumulator) { - // Unmergeable is supposed to be impossible to encounter in here - // because we check for resolve status up front. - - if (haveCompatibleTypes(reference, value)) { - if (reference instanceof AbstractConfigObject && value instanceof AbstractConfigObject) { - checkValidObject(path, (AbstractConfigObject) reference, - (AbstractConfigObject) value, accumulator); - } else if (reference instanceof SimpleConfigList && value instanceof SimpleConfigList) { - SimpleConfigList listRef = (SimpleConfigList) reference; - SimpleConfigList listValue = (SimpleConfigList) value; - if (listRef.isEmpty() || listValue.isEmpty()) { - // can't verify type, leave alone - } else { - AbstractConfigValue refElement = listRef.get(0); - for (ConfigValue elem : listValue) { - AbstractConfigValue e = (AbstractConfigValue) elem; - if (!haveCompatibleTypes(refElement, e)) { - addProblem(accumulator, path, e.origin(), "List at '" + path.render() - + "' contains wrong value type, expecting list of " - + getDesc(refElement) + " but got element of type " - + getDesc(e)); - // don't add a problem for every last array element - break; - } - } - } - } - } else { - addWrongType(accumulator, reference, value, path); - } - } - - @Override - public void checkValid(Config reference, String... restrictToPaths) { - SimpleConfig ref = (SimpleConfig) reference; - - // unresolved reference config is a bug in the caller of checkValid - if (ref.root().resolveStatus() != ResolveStatus.RESOLVED) - throw new ConfigException.BugOrBroken( - "do not call checkValid() with an unresolved reference config, call Config.resolve()"); - - // unresolved config under validation is probably a bug in something, - // but our whole goal here is to check for bugs in this config, so - // BugOrBroken is not the appropriate exception. - if (root().resolveStatus() != ResolveStatus.RESOLVED) - throw new ConfigException.NotResolved( - "config has unresolved substitutions; must call Config.resolve()"); - - // Now we know that both reference and this config are resolved - - List problems = new ArrayList(); - - if (restrictToPaths.length == 0) { - checkValidObject(null, ref.root(), root(), problems); - } else { - for (String p : restrictToPaths) { - Path path = Path.newPath(p); - AbstractConfigValue refValue = ref.peekPath(path); - if (refValue != null) { - AbstractConfigValue child = peekPath(path); - if (child != null) { - checkValid(path, refValue, child, problems); - } else { - addMissing(problems, refValue, path, origin()); - } - } - } - } - - if (!problems.isEmpty()) { - throw new ConfigException.ValidationFailed(problems); - } - } - - @Override - public SimpleConfig withOnlyPath(String pathExpression) { - Path path = Path.newPath(pathExpression); - return new SimpleConfig(root().withOnlyPath(path)); - } - - @Override - public SimpleConfig withoutPath(String pathExpression) { - Path path = Path.newPath(pathExpression); - return new SimpleConfig(root().withoutPath(path)); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java deleted file mode 100755 index 50d361f35e..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java +++ /dev/null @@ -1,383 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.ObjectStreamException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.ListIterator; - -import com.typesafe.config.ConfigList; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigResolveOptions; -import com.typesafe.config.ConfigValue; -import com.typesafe.config.ConfigValueType; - -final class SimpleConfigList extends AbstractConfigValue implements ConfigList { - - private static final long serialVersionUID = 1L; - - final private List value; - final private boolean resolved; - - SimpleConfigList(ConfigOrigin origin, List value) { - this(origin, value, ResolveStatus - .fromValues(value)); - } - - SimpleConfigList(ConfigOrigin origin, List value, - ResolveStatus status) { - super(origin); - this.value = value; - this.resolved = status == ResolveStatus.RESOLVED; - } - - @Override - public ConfigValueType valueType() { - return ConfigValueType.LIST; - } - - @Override - public List unwrapped() { - List list = new ArrayList(); - for (AbstractConfigValue v : value) { - list.add(v.unwrapped()); - } - return list; - } - - @Override - ResolveStatus resolveStatus() { - return ResolveStatus.fromBoolean(resolved); - } - - private SimpleConfigList modify(Modifier modifier, - ResolveStatus newResolveStatus) { - // lazy-create for optimization - List changed = null; - int i = 0; - for (AbstractConfigValue v : value) { - AbstractConfigValue modified = modifier.modifyChild(v); - - // lazy-create the new list if required - if (changed == null && modified != v) { - changed = new ArrayList(); - for (int j = 0; j < i; ++j) { - changed.add(value.get(j)); - } - } - - // once the new list is created, all elements - // have to go in it. if modifyChild returned - // null, we drop that element. - if (changed != null && modified != null) { - changed.add(modified); - } - - i += 1; - } - - if (changed != null) { - return new SimpleConfigList(origin(), changed, newResolveStatus); - } else { - return this; - } - } - - @Override - SimpleConfigList resolveSubstitutions(final SubstitutionResolver resolver, - final int depth, final ConfigResolveOptions options) { - if (resolved) - return this; - - return modify(new Modifier() { - @Override - public AbstractConfigValue modifyChild(AbstractConfigValue v) { - return resolver.resolve(v, depth, options); - } - - }, ResolveStatus.RESOLVED); - } - - @Override - SimpleConfigList relativized(final Path prefix) { - return modify(new Modifier() { - @Override - public AbstractConfigValue modifyChild(AbstractConfigValue v) { - return v.relativized(prefix); - } - - }, resolveStatus()); - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof SimpleConfigList; - } - - @Override - public boolean equals(Object other) { - // note that "origin" is deliberately NOT part of equality - if (other instanceof SimpleConfigList) { - // optimization to avoid unwrapped() for two ConfigList - return canEqual(other) && value.equals(((SimpleConfigList) other).value); - } else { - return false; - } - } - - @Override - public int hashCode() { - // note that "origin" is deliberately NOT part of equality - return value.hashCode(); - } - - @Override - protected void render(StringBuilder sb, int indent, boolean formatted) { - if (value.isEmpty()) { - sb.append("[]"); - } else { - sb.append("["); - if (formatted) - sb.append('\n'); - for (AbstractConfigValue v : value) { - if (formatted) { - indent(sb, indent + 1); - sb.append("# "); - sb.append(v.origin().description()); - sb.append("\n"); - - for (String comment : v.origin().comments()) { - indent(sb, indent + 1); - sb.append("# "); - sb.append(comment); - sb.append("\n"); - } - - indent(sb, indent + 1); - } - v.render(sb, indent + 1, formatted); - sb.append(","); - if (formatted) - sb.append('\n'); - } - sb.setLength(sb.length() - 1); // chop or newline - if (formatted) { - sb.setLength(sb.length() - 1); // also chop comma - sb.append('\n'); - indent(sb, indent); - } - sb.append("]"); - } - } - - @Override - public boolean contains(Object o) { - return value.contains(o); - } - - @Override - public boolean containsAll(Collection c) { - return value.containsAll(c); - } - - @Override - public AbstractConfigValue get(int index) { - return value.get(index); - } - - @Override - public int indexOf(Object o) { - return value.indexOf(o); - } - - @Override - public boolean isEmpty() { - return value.isEmpty(); - } - - @Override - public Iterator iterator() { - final Iterator i = value.iterator(); - - return new Iterator() { - @Override - public boolean hasNext() { - return i.hasNext(); - } - - @Override - public ConfigValue next() { - return i.next(); - } - - @Override - public void remove() { - throw weAreImmutable("iterator().remove"); - } - }; - } - - @Override - public int lastIndexOf(Object o) { - return value.lastIndexOf(o); - } - - private static ListIterator wrapListIterator( - final ListIterator i) { - return new ListIterator() { - @Override - public boolean hasNext() { - return i.hasNext(); - } - - @Override - public ConfigValue next() { - return i.next(); - } - - @Override - public void remove() { - throw weAreImmutable("listIterator().remove"); - } - - @Override - public void add(ConfigValue arg0) { - throw weAreImmutable("listIterator().add"); - } - - @Override - public boolean hasPrevious() { - return i.hasPrevious(); - } - - @Override - public int nextIndex() { - return i.nextIndex(); - } - - @Override - public ConfigValue previous() { - return i.previous(); - } - - @Override - public int previousIndex() { - return i.previousIndex(); - } - - @Override - public void set(ConfigValue arg0) { - throw weAreImmutable("listIterator().set"); - } - }; - } - - @Override - public ListIterator listIterator() { - return wrapListIterator(value.listIterator()); - } - - @Override - public ListIterator listIterator(int index) { - return wrapListIterator(value.listIterator(index)); - } - - @Override - public int size() { - return value.size(); - } - - @Override - public List subList(int fromIndex, int toIndex) { - List list = new ArrayList(); - // yay bloat caused by lack of type variance - for (AbstractConfigValue v : value.subList(fromIndex, toIndex)) { - list.add(v); - } - return list; - } - - @Override - public Object[] toArray() { - return value.toArray(); - } - - @Override - public T[] toArray(T[] a) { - return value.toArray(a); - } - - private static UnsupportedOperationException weAreImmutable(String method) { - return new UnsupportedOperationException( - "ConfigList is immutable, you can't call List.'" + method + "'"); - } - - @Override - public boolean add(ConfigValue e) { - throw weAreImmutable("add"); - } - - @Override - public void add(int index, ConfigValue element) { - throw weAreImmutable("add"); - } - - @Override - public boolean addAll(Collection c) { - throw weAreImmutable("addAll"); - } - - @Override - public boolean addAll(int index, Collection c) { - throw weAreImmutable("addAll"); - } - - @Override - public void clear() { - throw weAreImmutable("clear"); - } - - @Override - public boolean remove(Object o) { - throw weAreImmutable("remove"); - } - - @Override - public ConfigValue remove(int index) { - throw weAreImmutable("remove"); - } - - @Override - public boolean removeAll(Collection c) { - throw weAreImmutable("removeAll"); - } - - @Override - public boolean retainAll(Collection c) { - throw weAreImmutable("retainAll"); - } - - @Override - public ConfigValue set(int index, ConfigValue element) { - throw weAreImmutable("set"); - } - - @Override - protected SimpleConfigList newCopy(boolean ignoresFallbacks, ConfigOrigin newOrigin) { - return new SimpleConfigList(newOrigin, value); - } - - // This ridiculous hack is because some JDK versions apparently can't - // serialize an array, which is used to implement ArrayList and EmptyList. - // maybe - // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6446627 - private Object writeReplace() throws ObjectStreamException { - // switch to LinkedList - return new SimpleConfigList(origin(), new java.util.LinkedList(value), - resolveStatus()); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java deleted file mode 100755 index a138ec611e..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java +++ /dev/null @@ -1,215 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.util.AbstractMap; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigValue; - -final class SimpleConfigObject extends AbstractConfigObject { - - private static final long serialVersionUID = 1L; - - // this map should never be modified - assume immutable - final private Map value; - final private boolean resolved; - final private boolean ignoresFallbacks; - - SimpleConfigObject(ConfigOrigin origin, - Map value, ResolveStatus status, - boolean ignoresFallbacks) { - super(origin); - if (value == null) - throw new ConfigException.BugOrBroken( - "creating config object with null map"); - this.value = value; - this.resolved = status == ResolveStatus.RESOLVED; - this.ignoresFallbacks = ignoresFallbacks; - } - - SimpleConfigObject(ConfigOrigin origin, - Map value) { - this(origin, value, ResolveStatus.fromValues(value.values()), false /* ignoresFallbacks */); - } - - @Override - public SimpleConfigObject withOnlyKey(String key) { - return withOnlyPath(Path.newKey(key)); - } - - @Override - public SimpleConfigObject withoutKey(String key) { - return withoutPath(Path.newKey(key)); - } - - // gets the object with only the path if the path - // exists, otherwise null if it doesn't. this ensures - // that if we have { a : { b : 42 } } and do - // withOnlyPath("a.b.c") that we don't keep an empty - // "a" object. - @Override - protected SimpleConfigObject withOnlyPathOrNull(Path path) { - String key = path.first(); - Path next = path.remainder(); - AbstractConfigValue v = value.get(key); - - if (next != null) { - if (v != null && (v instanceof AbstractConfigObject)) { - v = ((AbstractConfigObject) v).withOnlyPathOrNull(next); - } else { - // if the path has more elements but we don't have an object, - // then the rest of the path does not exist. - v = null; - } - } - - if (v == null) { - return null; - } else { - return new SimpleConfigObject(origin(), Collections.singletonMap(key, v), - resolveStatus(), ignoresFallbacks); - } - } - - @Override - SimpleConfigObject withOnlyPath(Path path) { - SimpleConfigObject o = withOnlyPathOrNull(path); - if (o == null) { - return new SimpleConfigObject(origin(), - Collections. emptyMap(), resolveStatus(), - ignoresFallbacks); - } else { - return o; - } - } - - @Override - SimpleConfigObject withoutPath(Path path) { - String key = path.first(); - Path next = path.remainder(); - AbstractConfigValue v = value.get(key); - - if (v != null && next != null && v instanceof AbstractConfigObject) { - v = ((AbstractConfigObject) v).withoutPath(next); - Map updated = new HashMap( - value); - updated.put(key, v); - return new SimpleConfigObject(origin(), updated, resolveStatus(), ignoresFallbacks); - } else if (next != null || v == null) { - // can't descend, nothing to remove - return this; - } else { - Map smaller = new HashMap( - value.size() - 1); - for (Map.Entry old : value.entrySet()) { - if (!old.getKey().equals(key)) - smaller.put(old.getKey(), old.getValue()); - } - return new SimpleConfigObject(origin(), smaller, resolveStatus(), ignoresFallbacks); - } - } - - @Override - protected AbstractConfigValue peek(String key) { - return value.get(key); - } - - @Override - protected SimpleConfigObject newCopy(ResolveStatus newStatus, boolean newIgnoresFallbacks, - ConfigOrigin newOrigin) { - return new SimpleConfigObject(newOrigin, value, newStatus, newIgnoresFallbacks); - } - - @Override - ResolveStatus resolveStatus() { - return ResolveStatus.fromBoolean(resolved); - } - - @Override - protected boolean ignoresFallbacks() { - return ignoresFallbacks; - } - - @Override - public Map unwrapped() { - Map m = new HashMap(); - for (Map.Entry e : value.entrySet()) { - m.put(e.getKey(), e.getValue().unwrapped()); - } - return m; - } - - @Override - public boolean containsKey(Object key) { - return value.containsKey(key); - } - - @Override - public Set keySet() { - return value.keySet(); - } - - @Override - public boolean containsValue(Object v) { - return value.containsValue(v); - } - - @Override - public Set> entrySet() { - // total bloat just to work around lack of type variance - - HashSet> entries = new HashSet>(); - for (Map.Entry e : value.entrySet()) { - entries.add(new AbstractMap.SimpleImmutableEntry( - e.getKey(), e - .getValue())); - } - return entries; - } - - @Override - public boolean isEmpty() { - return value.isEmpty(); - } - - @Override - public int size() { - return value.size(); - } - - @Override - public Collection values() { - return new HashSet(value.values()); - } - - final private static String EMPTY_NAME = "empty config"; - final private static SimpleConfigObject emptyInstance = empty(SimpleConfigOrigin - .newSimple(EMPTY_NAME)); - - final static SimpleConfigObject empty() { - return emptyInstance; - } - - final static SimpleConfigObject empty(ConfigOrigin origin) { - if (origin == null) - return empty(); - else - return new SimpleConfigObject(origin, - Collections. emptyMap()); - } - - final static SimpleConfigObject emptyMissing(ConfigOrigin baseOrigin) { - return new SimpleConfigObject(SimpleConfigOrigin.newSimple( - baseOrigin.description() + " (not found)"), - Collections. emptyMap()); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java deleted file mode 100755 index 23351c1e8e..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java +++ /dev/null @@ -1,341 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.File; -import java.io.Serializable; -import java.net.MalformedURLException; -import java.net.URL; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigOrigin; - -// it would be cleaner to have a class hierarchy for various origin types, -// but was hoping this would be enough simpler to be a little messy. eh. -final class SimpleConfigOrigin implements ConfigOrigin, Serializable { - - private static final long serialVersionUID = 1L; - - final private String description; - final private int lineNumber; - final private int endLineNumber; - final private OriginType originType; - final private String urlOrNull; - final private List commentsOrNull; - - protected SimpleConfigOrigin(String description, int lineNumber, int endLineNumber, - OriginType originType, - String urlOrNull, List commentsOrNull) { - this.description = description; - this.lineNumber = lineNumber; - this.endLineNumber = endLineNumber; - this.originType = originType; - this.urlOrNull = urlOrNull; - this.commentsOrNull = commentsOrNull; - } - - static SimpleConfigOrigin newSimple(String description) { - return new SimpleConfigOrigin(description, -1, -1, OriginType.GENERIC, null, null); - } - - static SimpleConfigOrigin newFile(String filename) { - String url; - try { - url = (new File(filename)).toURI().toURL().toExternalForm(); - } catch (MalformedURLException e) { - url = null; - } - return new SimpleConfigOrigin(filename, -1, -1, OriginType.FILE, url, null); - } - - static SimpleConfigOrigin newURL(URL url) { - String u = url.toExternalForm(); - return new SimpleConfigOrigin(u, -1, -1, OriginType.URL, u, null); - } - - static SimpleConfigOrigin newResource(String resource, URL url) { - return new SimpleConfigOrigin(resource, -1, -1, OriginType.RESOURCE, - url != null ? url.toExternalForm() : null, null); - } - - static SimpleConfigOrigin newResource(String resource) { - return newResource(resource, null); - } - - SimpleConfigOrigin setLineNumber(int lineNumber) { - if (lineNumber == this.lineNumber && lineNumber == this.endLineNumber) { - return this; - } else { - return new SimpleConfigOrigin(this.description, lineNumber, lineNumber, - this.originType, this.urlOrNull, this.commentsOrNull); - } - } - - SimpleConfigOrigin addURL(URL url) { - return new SimpleConfigOrigin(this.description, this.lineNumber, this.endLineNumber, - this.originType, url != null ? url.toExternalForm() : null, this.commentsOrNull); - } - - SimpleConfigOrigin setComments(List comments) { - if (ConfigImplUtil.equalsHandlingNull(comments, this.commentsOrNull)) { - return this; - } else { - return new SimpleConfigOrigin(this.description, this.lineNumber, this.endLineNumber, - this.originType, this.urlOrNull, comments); - } - } - - @Override - public String description() { - // not putting the URL in here for files and resources, because people - // parsing "file: line" syntax would hit the ":" in the URL. - if (lineNumber < 0) { - return description; - } else if (endLineNumber == lineNumber) { - return description + ": " + lineNumber; - } else { - return description + ": " + lineNumber + "-" + endLineNumber; - } - } - - @Override - public boolean equals(Object other) { - if (other instanceof SimpleConfigOrigin) { - SimpleConfigOrigin otherOrigin = (SimpleConfigOrigin) other; - - return this.description.equals(otherOrigin.description) - && this.lineNumber == otherOrigin.lineNumber - && this.endLineNumber == otherOrigin.endLineNumber - && this.originType == otherOrigin.originType - && ConfigImplUtil.equalsHandlingNull(this.urlOrNull, otherOrigin.urlOrNull); - } else { - return false; - } - } - - @Override - public int hashCode() { - int h = 41 * (41 + description.hashCode()); - h = 41 * (h + lineNumber); - h = 41 * (h + endLineNumber); - h = 41 * (h + originType.hashCode()); - if (urlOrNull != null) - h = 41 * (h + urlOrNull.hashCode()); - return h; - } - - @Override - public String toString() { - // the url is only really useful on top of description for resources - if (originType == OriginType.RESOURCE && urlOrNull != null) { - return "ConfigOrigin(" + description + "," + urlOrNull + ")"; - } else { - return "ConfigOrigin(" + description + ")"; - } - } - - @Override - public String filename() { - if (originType == OriginType.FILE) { - return description; - } else if (urlOrNull != null) { - URL url; - try { - url = new URL(urlOrNull); - } catch (MalformedURLException e) { - return null; - } - if (url.getProtocol().equals("file")) { - return url.getFile(); - } else { - return null; - } - } else { - return null; - } - } - - @Override - public URL url() { - if (urlOrNull == null) { - return null; - } else { - try { - return new URL(urlOrNull); - } catch (MalformedURLException e) { - return null; - } - } - } - - @Override - public String resource() { - if (originType == OriginType.RESOURCE) { - return description; - } else { - return null; - } - } - - @Override - public int lineNumber() { - return lineNumber; - } - - @Override - public List comments() { - if (commentsOrNull != null) { - return commentsOrNull; - } else { - return Collections.emptyList(); - } - } - - static final String MERGE_OF_PREFIX = "merge of "; - - private static SimpleConfigOrigin mergeTwo(SimpleConfigOrigin a, SimpleConfigOrigin b) { - String mergedDesc; - int mergedStartLine; - int mergedEndLine; - List mergedComments; - - OriginType mergedType; - if (a.originType == b.originType) { - mergedType = a.originType; - } else { - mergedType = OriginType.GENERIC; - } - - // first use the "description" field which has no line numbers - // cluttering it. - String aDesc = a.description; - String bDesc = b.description; - if (aDesc.startsWith(MERGE_OF_PREFIX)) - aDesc = aDesc.substring(MERGE_OF_PREFIX.length()); - if (bDesc.startsWith(MERGE_OF_PREFIX)) - bDesc = bDesc.substring(MERGE_OF_PREFIX.length()); - - if (aDesc.equals(bDesc)) { - mergedDesc = aDesc; - - if (a.lineNumber < 0) - mergedStartLine = b.lineNumber; - else if (b.lineNumber < 0) - mergedStartLine = a.lineNumber; - else - mergedStartLine = Math.min(a.lineNumber, b.lineNumber); - - mergedEndLine = Math.max(a.endLineNumber, b.endLineNumber); - } else { - // this whole merge song-and-dance was intended to avoid this case - // whenever possible, but we've lost. Now we have to lose some - // structured information and cram into a string. - - // description() method includes line numbers, so use it instead - // of description field. - String aFull = a.description(); - String bFull = b.description(); - if (aFull.startsWith(MERGE_OF_PREFIX)) - aFull = aFull.substring(MERGE_OF_PREFIX.length()); - if (bFull.startsWith(MERGE_OF_PREFIX)) - bFull = bFull.substring(MERGE_OF_PREFIX.length()); - - mergedDesc = MERGE_OF_PREFIX + aFull + "," + bFull; - - mergedStartLine = -1; - mergedEndLine = -1; - } - - String mergedURL; - if (ConfigImplUtil.equalsHandlingNull(a.urlOrNull, b.urlOrNull)) { - mergedURL = a.urlOrNull; - } else { - mergedURL = null; - } - - if (ConfigImplUtil.equalsHandlingNull(a.commentsOrNull, b.commentsOrNull)) { - mergedComments = a.commentsOrNull; - } else { - mergedComments = new ArrayList(); - if (a.commentsOrNull != null) - mergedComments.addAll(a.commentsOrNull); - if (b.commentsOrNull != null) - mergedComments.addAll(b.commentsOrNull); - } - - return new SimpleConfigOrigin(mergedDesc, mergedStartLine, mergedEndLine, mergedType, - mergedURL, mergedComments); - } - - private static int similarity(SimpleConfigOrigin a, SimpleConfigOrigin b) { - int count = 0; - - if (a.originType == b.originType) - count += 1; - - if (a.description.equals(b.description)) { - count += 1; - - // only count these if the description field (which is the file - // or resource name) also matches. - if (a.lineNumber == b.lineNumber) - count += 1; - if (a.endLineNumber == b.endLineNumber) - count += 1; - if (ConfigImplUtil.equalsHandlingNull(a.urlOrNull, b.urlOrNull)) - count += 1; - } - - return count; - } - - // this picks the best pair to merge, because the pair has the most in - // common. we want to merge two lines in the same file rather than something - // else with one of the lines; because two lines in the same file can be - // better consolidated. - private static SimpleConfigOrigin mergeThree(SimpleConfigOrigin a, SimpleConfigOrigin b, - SimpleConfigOrigin c) { - if (similarity(a, b) >= similarity(b, c)) { - return mergeTwo(mergeTwo(a, b), c); - } else { - return mergeTwo(a, mergeTwo(b, c)); - } - } - - static ConfigOrigin mergeOrigins(Collection stack) { - if (stack.isEmpty()) { - throw new ConfigException.BugOrBroken("can't merge empty list of origins"); - } else if (stack.size() == 1) { - return stack.iterator().next(); - } else if (stack.size() == 2) { - Iterator i = stack.iterator(); - return mergeTwo((SimpleConfigOrigin) i.next(), (SimpleConfigOrigin) i.next()); - } else { - List remaining = new ArrayList(); - for (ConfigOrigin o : stack) { - remaining.add((SimpleConfigOrigin) o); - } - while (remaining.size() > 2) { - SimpleConfigOrigin c = remaining.get(remaining.size() - 1); - remaining.remove(remaining.size() - 1); - SimpleConfigOrigin b = remaining.get(remaining.size() - 1); - remaining.remove(remaining.size() - 1); - SimpleConfigOrigin a = remaining.get(remaining.size() - 1); - remaining.remove(remaining.size() - 1); - - SimpleConfigOrigin merged = mergeThree(a, b, c); - - remaining.add(merged); - } - - // should be down to either 1 or 2 - return mergeOrigins(remaining); - } - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java b/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java deleted file mode 100755 index be67073a1d..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionExpression.java +++ /dev/null @@ -1,50 +0,0 @@ -package com.typesafe.config.impl; - -import java.io.Serializable; - -final class SubstitutionExpression implements Serializable { - - private static final long serialVersionUID = 1L; - - final private Path path; - final private boolean optional; - - SubstitutionExpression(Path path, boolean optional) { - this.path = path; - this.optional = optional; - } - - Path path() { - return path; - } - - boolean optional() { - return optional; - } - - SubstitutionExpression changePath(Path newPath) { - return new SubstitutionExpression(newPath, optional); - } - - @Override - public String toString() { - return "${" + (optional ? "?" : "") + path.render() + "}"; - } - - @Override - public boolean equals(Object other) { - if (other instanceof SubstitutionExpression) { - SubstitutionExpression otherExp = (SubstitutionExpression) other; - return otherExp.path.equals(this.path) && otherExp.optional == this.optional; - } else { - return false; - } - } - - @Override - public int hashCode() { - int h = 41 * (41 + path.hashCode()); - h = 41 * (h + (optional ? 1 : 0)); - return h; - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java b/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java deleted file mode 100755 index 7bb3bf3a61..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java +++ /dev/null @@ -1,55 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.util.IdentityHashMap; -import java.util.Map; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigResolveOptions; - -/** - * This exists because we have to memoize resolved substitutions as we go - * through the config tree; otherwise we could end up creating multiple copies - * of values or whole trees of values as we follow chains of substitutions. - */ -final class SubstitutionResolver { - final private AbstractConfigObject root; - // note that we can resolve things to undefined (represented as Java null, - // rather than ConfigNull) so this map can have null values. - final private Map memos; - - SubstitutionResolver(AbstractConfigObject root) { - this.root = root; - // note: the memoization is by object identity, not object value - this.memos = new IdentityHashMap(); - } - - AbstractConfigValue resolve(AbstractConfigValue original, int depth, - ConfigResolveOptions options) { - if (memos.containsKey(original)) { - return memos.get(original); - } else { - AbstractConfigValue resolved = original.resolveSubstitutions(this, - depth, options); - if (resolved != null) { - if (resolved.resolveStatus() != ResolveStatus.RESOLVED) - throw new ConfigException.BugOrBroken( - "resolveSubstitutions() did not give us a resolved object"); - } - memos.put(original, resolved); - return resolved; - } - } - - AbstractConfigObject root() { - return this.root; - } - - static AbstractConfigValue resolve(AbstractConfigValue value, - AbstractConfigObject root, ConfigResolveOptions options) { - SubstitutionResolver resolver = new SubstitutionResolver(root); - return resolver.resolve(value, 0, options); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Token.java b/akka-actor/src/main/java/com/typesafe/config/impl/Token.java deleted file mode 100755 index 5f16d26e1d..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Token.java +++ /dev/null @@ -1,79 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigOrigin; - -class Token { - final private TokenType tokenType; - final private String debugString; - final private ConfigOrigin origin; - - Token(TokenType tokenType, ConfigOrigin origin) { - this(tokenType, origin, null); - } - - Token(TokenType tokenType, ConfigOrigin origin, String debugString) { - this.tokenType = tokenType; - this.origin = origin; - this.debugString = debugString; - } - - // this is used for singleton tokens like COMMA or OPEN_CURLY - static Token newWithoutOrigin(TokenType tokenType, String debugString) { - return new Token(tokenType, null, debugString); - } - - final TokenType tokenType() { - return tokenType; - } - - // this is final because we don't always use the origin() accessor, - // and we don't because it throws if origin is null - final ConfigOrigin origin() { - // code is only supposed to call origin() on token types that are - // expected to have an origin. - if (origin == null) - throw new ConfigException.BugOrBroken( - "tried to get origin from token that doesn't have one: " + this); - return origin; - } - - final int lineNumber() { - if (origin != null) - return origin.lineNumber(); - else - return -1; - } - - @Override - public String toString() { - if (debugString != null) - return debugString; - else - return tokenType.name(); - } - - protected boolean canEqual(Object other) { - return other instanceof Token; - } - - @Override - public boolean equals(Object other) { - if (other instanceof Token) { - // origin is deliberately left out - return canEqual(other) - && this.tokenType == ((Token) other).tokenType; - } else { - return false; - } - } - - @Override - public int hashCode() { - // origin is deliberately left out - return tokenType.hashCode(); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java b/akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java deleted file mode 100755 index 7853c09445..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java +++ /dev/null @@ -1,22 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -enum TokenType { - START, - END, - COMMA, - EQUALS, - COLON, - OPEN_CURLY, - CLOSE_CURLY, - OPEN_SQUARE, - CLOSE_SQUARE, - VALUE, - NEWLINE, - UNQUOTED_TEXT, - SUBSTITUTION, - PROBLEM, - COMMENT; -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java b/akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java deleted file mode 100755 index 2fcee8e61a..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java +++ /dev/null @@ -1,596 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.io.IOException; -import java.io.Reader; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Queue; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigSyntax; - -final class Tokenizer { - // this exception should not leave this file - private static class ProblemException extends Exception { - private static final long serialVersionUID = 1L; - - final private Token problem; - - ProblemException(Token problem) { - this.problem = problem; - } - - Token problem() { - return problem; - } - } - - private static String asString(int codepoint) { - if (codepoint == '\n') - return "newline"; - else if (codepoint == '\t') - return "tab"; - else if (codepoint == -1) - return "end of file"; - else if (Character.isISOControl(codepoint)) - return String.format("control character 0x%x", codepoint); - else - return String.format("%c", codepoint); - } - - /** - * Tokenizes a Reader. Does not close the reader; you have to arrange to do - * that after you're done with the returned iterator. - */ - static Iterator tokenize(ConfigOrigin origin, Reader input, ConfigSyntax flavor) { - return new TokenIterator(origin, input, flavor != ConfigSyntax.JSON); - } - - private static class TokenIterator implements Iterator { - - private static class WhitespaceSaver { - // has to be saved inside value concatenations - private StringBuilder whitespace; - // may need to value-concat with next value - private boolean lastTokenWasSimpleValue; - - WhitespaceSaver() { - whitespace = new StringBuilder(); - lastTokenWasSimpleValue = false; - } - - void add(int c) { - if (lastTokenWasSimpleValue) - whitespace.appendCodePoint(c); - } - - Token check(Token t, ConfigOrigin baseOrigin, int lineNumber) { - if (isSimpleValue(t)) { - return nextIsASimpleValue(baseOrigin, lineNumber); - } else { - nextIsNotASimpleValue(); - return null; - } - } - - // called if the next token is not a simple value; - // discards any whitespace we were saving between - // simple values. - private void nextIsNotASimpleValue() { - lastTokenWasSimpleValue = false; - whitespace.setLength(0); - } - - // called if the next token IS a simple value, - // so creates a whitespace token if the previous - // token also was. - private Token nextIsASimpleValue(ConfigOrigin baseOrigin, - int lineNumber) { - if (lastTokenWasSimpleValue) { - // need to save whitespace between the two so - // the parser has the option to concatenate it. - if (whitespace.length() > 0) { - Token t = Tokens.newUnquotedText( - lineOrigin(baseOrigin, lineNumber), - whitespace.toString()); - whitespace.setLength(0); // reset - return t; - } else { - // lastTokenWasSimpleValue = true still - return null; - } - } else { - lastTokenWasSimpleValue = true; - whitespace.setLength(0); - return null; - } - } - } - - final private SimpleConfigOrigin origin; - final private Reader input; - final private LinkedList buffer; - private int lineNumber; - private ConfigOrigin lineOrigin; - final private Queue tokens; - final private WhitespaceSaver whitespaceSaver; - final private boolean allowComments; - - TokenIterator(ConfigOrigin origin, Reader input, boolean allowComments) { - this.origin = (SimpleConfigOrigin) origin; - this.input = input; - this.allowComments = allowComments; - this.buffer = new LinkedList(); - lineNumber = 1; - lineOrigin = this.origin.setLineNumber(lineNumber); - tokens = new LinkedList(); - tokens.add(Tokens.START); - whitespaceSaver = new WhitespaceSaver(); - } - - - // this should ONLY be called from nextCharSkippingComments - // or when inside a quoted string, everything else should - // use nextCharSkippingComments(). - private int nextCharRaw() { - if (buffer.isEmpty()) { - try { - return input.read(); - } catch (IOException e) { - throw new ConfigException.IO(origin, "read error: " - + e.getMessage(), e); - } - } else { - int c = buffer.pop(); - return c; - } - } - - private void putBack(int c) { - if (buffer.size() > 2) { - throw new ConfigException.BugOrBroken( - "bug: putBack() three times, undesirable look-ahead"); - } - buffer.push(c); - } - - static boolean isWhitespace(int c) { - return ConfigImplUtil.isWhitespace(c); - } - - static boolean isWhitespaceNotNewline(int c) { - return c != '\n' && ConfigImplUtil.isWhitespace(c); - } - - private boolean startOfComment(int c) { - if (c == -1) { - return false; - } else { - if (allowComments) { - if (c == '#') { - return true; - } else if (c == '/') { - int maybeSecondSlash = nextCharRaw(); - // we want to predictably NOT consume any chars - putBack(maybeSecondSlash); - if (maybeSecondSlash == '/') { - return true; - } else { - return false; - } - } else { - return false; - } - } else { - return false; - } - } - } - - // get next char, skipping non-newline whitespace - private int nextCharAfterWhitespace(WhitespaceSaver saver) { - for (;;) { - int c = nextCharRaw(); - - if (c == -1) { - return -1; - } else { - if (isWhitespaceNotNewline(c)) { - saver.add(c); - continue; - } else { - return c; - } - } - } - } - - private ProblemException problem(String message) { - return problem("", message, null); - } - - private ProblemException problem(String what, String message) { - return problem(what, message, null); - } - - private ProblemException problem(String what, String message, boolean suggestQuotes) { - return problem(what, message, suggestQuotes, null); - } - - private ProblemException problem(String what, String message, Throwable cause) { - return problem(lineOrigin, what, message, cause); - } - - private ProblemException problem(String what, String message, boolean suggestQuotes, - Throwable cause) { - return problem(lineOrigin, what, message, suggestQuotes, cause); - } - - private static ProblemException problem(ConfigOrigin origin, String what, - String message, - Throwable cause) { - return problem(origin, what, message, false, cause); - } - - private static ProblemException problem(ConfigOrigin origin, String what, String message, - boolean suggestQuotes, Throwable cause) { - if (what == null || message == null) - throw new ConfigException.BugOrBroken( - "internal error, creating bad ProblemException"); - return new ProblemException(Tokens.newProblem(origin, what, message, suggestQuotes, - cause)); - } - - private static ProblemException problem(ConfigOrigin origin, String message) { - return problem(origin, "", message, null); - } - - private static ConfigOrigin lineOrigin(ConfigOrigin baseOrigin, - int lineNumber) { - return ((SimpleConfigOrigin) baseOrigin).setLineNumber(lineNumber); - } - - // ONE char has always been consumed, either the # or the first /, but - // not both slashes - private Token pullComment(int firstChar) { - if (firstChar == '/') { - int discard = nextCharRaw(); - if (discard != '/') - throw new ConfigException.BugOrBroken("called pullComment but // not seen"); - } - - StringBuilder sb = new StringBuilder(); - for (;;) { - int c = nextCharRaw(); - if (c == -1 || c == '\n') { - putBack(c); - return Tokens.newComment(lineOrigin, sb.toString()); - } else { - sb.appendCodePoint(c); - } - } - } - - // chars JSON allows a number to start with - static final String firstNumberChars = "0123456789-"; - // chars JSON allows to be part of a number - static final String numberChars = "0123456789eE+-."; - // chars that stop an unquoted string - static final String notInUnquotedText = "$\"{}[]:=,+#`^?!@*&\\"; - - // The rules here are intended to maximize convenience while - // avoiding confusion with real valid JSON. Basically anything - // that parses as JSON is treated the JSON way and otherwise - // we assume it's a string and let the parser sort it out. - private Token pullUnquotedText() { - ConfigOrigin origin = lineOrigin; - StringBuilder sb = new StringBuilder(); - int c = nextCharRaw(); - while (true) { - if (c == -1) { - break; - } else if (notInUnquotedText.indexOf(c) >= 0) { - break; - } else if (isWhitespace(c)) { - break; - } else if (startOfComment(c)) { - break; - } else { - sb.appendCodePoint(c); - } - - // we parse true/false/null tokens as such no matter - // what is after them, as long as they are at the - // start of the unquoted token. - if (sb.length() == 4) { - String s = sb.toString(); - if (s.equals("true")) - return Tokens.newBoolean(origin, true); - else if (s.equals("null")) - return Tokens.newNull(origin); - } else if (sb.length() == 5) { - String s = sb.toString(); - if (s.equals("false")) - return Tokens.newBoolean(origin, false); - } - - c = nextCharRaw(); - } - - // put back the char that ended the unquoted text - putBack(c); - - String s = sb.toString(); - return Tokens.newUnquotedText(origin, s); - } - - private Token pullNumber(int firstChar) throws ProblemException { - StringBuilder sb = new StringBuilder(); - sb.appendCodePoint(firstChar); - boolean containedDecimalOrE = false; - int c = nextCharRaw(); - while (c != -1 && numberChars.indexOf(c) >= 0) { - if (c == '.' || c == 'e' || c == 'E') - containedDecimalOrE = true; - sb.appendCodePoint(c); - c = nextCharRaw(); - } - // the last character we looked at wasn't part of the number, put it - // back - putBack(c); - String s = sb.toString(); - try { - if (containedDecimalOrE) { - // force floating point representation - return Tokens.newDouble(lineOrigin, Double.parseDouble(s), s); - } else { - // this should throw if the integer is too large for Long - return Tokens.newLong(lineOrigin, Long.parseLong(s), s); - } - } catch (NumberFormatException e) { - throw problem(s, "Invalid number: '" + s + "'", true /* suggestQuotes */, e); - } - } - - private void pullEscapeSequence(StringBuilder sb) throws ProblemException { - int escaped = nextCharRaw(); - if (escaped == -1) - throw problem("End of input but backslash in string had nothing after it"); - - switch (escaped) { - case '"': - sb.append('"'); - break; - case '\\': - sb.append('\\'); - break; - case '/': - sb.append('/'); - break; - case 'b': - sb.append('\b'); - break; - case 'f': - sb.append('\f'); - break; - case 'n': - sb.append('\n'); - break; - case 'r': - sb.append('\r'); - break; - case 't': - sb.append('\t'); - break; - case 'u': { - // kind of absurdly slow, but screw it for now - char[] a = new char[4]; - for (int i = 0; i < 4; ++i) { - int c = nextCharRaw(); - if (c == -1) - throw problem("End of input but expecting 4 hex digits for \\uXXXX escape"); - a[i] = (char) c; - } - String digits = new String(a); - try { - sb.appendCodePoint(Integer.parseInt(digits, 16)); - } catch (NumberFormatException e) { - throw problem(digits, String.format( - "Malformed hex digits after \\u escape in string: '%s'", digits), e); - } - } - break; - default: - throw problem( - asString(escaped), - String.format( - "backslash followed by '%s', this is not a valid escape sequence (quoted strings use JSON escaping, so use double-backslash \\\\ for literal backslash)", - asString(escaped))); - } - } - - private Token pullQuotedString() throws ProblemException { - // the open quote has already been consumed - StringBuilder sb = new StringBuilder(); - int c = '\0'; // value doesn't get used - do { - c = nextCharRaw(); - if (c == -1) - throw problem("End of input but string quote was still open"); - - if (c == '\\') { - pullEscapeSequence(sb); - } else if (c == '"') { - // end the loop, done! - } else if (Character.isISOControl(c)) { - throw problem(asString(c), "JSON does not allow unescaped " + asString(c) - + " in quoted strings, use a backslash escape"); - } else { - sb.appendCodePoint(c); - } - } while (c != '"'); - return Tokens.newString(lineOrigin, sb.toString()); - } - - private Token pullSubstitution() throws ProblemException { - // the initial '$' has already been consumed - ConfigOrigin origin = lineOrigin; - int c = nextCharRaw(); - if (c != '{') { - throw problem(asString(c), "'$' not followed by {, '" + asString(c) - + "' not allowed after '$'", true /* suggestQuotes */); - } - - boolean optional = false; - c = nextCharRaw(); - if (c == '?') { - optional = true; - } else { - putBack(c); - } - - WhitespaceSaver saver = new WhitespaceSaver(); - List expression = new ArrayList(); - - Token t; - do { - t = pullNextToken(saver); - - // note that we avoid validating the allowed tokens inside - // the substitution here; we even allow nested substitutions - // in the tokenizer. The parser sorts it out. - if (t == Tokens.CLOSE_CURLY) { - // end the loop, done! - break; - } else if (t == Tokens.END) { - throw problem(origin, - "Substitution ${ was not closed with a }"); - } else { - Token whitespace = saver.check(t, origin, lineNumber); - if (whitespace != null) - expression.add(whitespace); - expression.add(t); - } - } while (true); - - return Tokens.newSubstitution(origin, optional, expression); - } - - private Token pullNextToken(WhitespaceSaver saver) throws ProblemException { - int c = nextCharAfterWhitespace(saver); - if (c == -1) { - return Tokens.END; - } else if (c == '\n') { - // newline tokens have the just-ended line number - Token line = Tokens.newLine(lineOrigin); - lineNumber += 1; - lineOrigin = origin.setLineNumber(lineNumber); - return line; - } else { - Token t = null; - if (startOfComment(c)) { - t = pullComment(c); - } else { - switch (c) { - case '"': - t = pullQuotedString(); - break; - case '$': - t = pullSubstitution(); - break; - case ':': - t = Tokens.COLON; - break; - case ',': - t = Tokens.COMMA; - break; - case '=': - t = Tokens.EQUALS; - break; - case '{': - t = Tokens.OPEN_CURLY; - break; - case '}': - t = Tokens.CLOSE_CURLY; - break; - case '[': - t = Tokens.OPEN_SQUARE; - break; - case ']': - t = Tokens.CLOSE_SQUARE; - break; - } - - if (t == null) { - if (firstNumberChars.indexOf(c) >= 0) { - t = pullNumber(c); - } else if (notInUnquotedText.indexOf(c) >= 0) { - throw problem(asString(c), "Reserved character '" + asString(c) - + "' is not allowed outside quotes", true /* suggestQuotes */); - } else { - putBack(c); - t = pullUnquotedText(); - } - } - } - - if (t == null) - throw new ConfigException.BugOrBroken( - "bug: failed to generate next token"); - - return t; - } - } - - private static boolean isSimpleValue(Token t) { - if (Tokens.isSubstitution(t) || Tokens.isUnquotedText(t) - || Tokens.isValue(t)) { - return true; - } else { - return false; - } - } - - private void queueNextToken() throws ProblemException { - Token t = pullNextToken(whitespaceSaver); - Token whitespace = whitespaceSaver.check(t, origin, lineNumber); - if (whitespace != null) - tokens.add(whitespace); - - tokens.add(t); - } - - @Override - public boolean hasNext() { - return !tokens.isEmpty(); - } - - @Override - public Token next() { - Token t = tokens.remove(); - if (tokens.isEmpty() && t != Tokens.END) { - try { - queueNextToken(); - } catch (ProblemException e) { - tokens.add(e.problem()); - } - if (tokens.isEmpty()) - throw new ConfigException.BugOrBroken( - "bug: tokens queue should not be empty here"); - } - return t; - } - - @Override - public void remove() { - throw new UnsupportedOperationException( - "Does not make sense to remove items from token stream"); - } - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java b/akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java deleted file mode 100755 index 83bec62af3..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java +++ /dev/null @@ -1,413 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.util.List; - -import com.typesafe.config.ConfigException; -import com.typesafe.config.ConfigOrigin; -import com.typesafe.config.ConfigValueType; - -/* FIXME the way the subclasses of Token are private with static isFoo and accessors is kind of ridiculous. */ -final class Tokens { - static private class Value extends Token { - - final private AbstractConfigValue value; - - Value(AbstractConfigValue value) { - super(TokenType.VALUE, value.origin()); - this.value = value; - } - - AbstractConfigValue value() { - return value; - } - - @Override - public String toString() { - return "'" + value().unwrapped() + "' (" + value.valueType().name() + ")"; - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof Value; - } - - @Override - public boolean equals(Object other) { - return super.equals(other) && ((Value) other).value.equals(value); - } - - @Override - public int hashCode() { - return 41 * (41 + super.hashCode()) + value.hashCode(); - } - } - - static private class Line extends Token { - Line(ConfigOrigin origin) { - super(TokenType.NEWLINE, origin); - } - - @Override - public String toString() { - return "'\\n'@" + lineNumber(); - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof Line; - } - - @Override - public boolean equals(Object other) { - return super.equals(other) && ((Line) other).lineNumber() == lineNumber(); - } - - @Override - public int hashCode() { - return 41 * (41 + super.hashCode()) + lineNumber(); - } - } - - // This is not a Value, because it requires special processing - static private class UnquotedText extends Token { - final private String value; - - UnquotedText(ConfigOrigin origin, String s) { - super(TokenType.UNQUOTED_TEXT, origin); - this.value = s; - } - - String value() { - return value; - } - - @Override - public String toString() { - return "'" + value + "'"; - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof UnquotedText; - } - - @Override - public boolean equals(Object other) { - return super.equals(other) - && ((UnquotedText) other).value.equals(value); - } - - @Override - public int hashCode() { - return 41 * (41 + super.hashCode()) + value.hashCode(); - } - } - - static private class Problem extends Token { - final private String what; - final private String message; - final private boolean suggestQuotes; - final private Throwable cause; - - Problem(ConfigOrigin origin, String what, String message, boolean suggestQuotes, - Throwable cause) { - super(TokenType.PROBLEM, origin); - this.what = what; - this.message = message; - this.suggestQuotes = suggestQuotes; - this.cause = cause; - } - - String message() { - return message; - } - - boolean suggestQuotes() { - return suggestQuotes; - } - - Throwable cause() { - return cause; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append('\''); - sb.append(what); - sb.append('\''); - return sb.toString(); - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof Problem; - } - - @Override - public boolean equals(Object other) { - return super.equals(other) && ((Problem) other).what.equals(what) - && ((Problem) other).message.equals(message) - && ((Problem) other).suggestQuotes == suggestQuotes - && ConfigImplUtil.equalsHandlingNull(((Problem) other).cause, cause); - } - - @Override - public int hashCode() { - int h = 41 * (41 + super.hashCode()); - h = 41 * (h + what.hashCode()); - h = 41 * (h + message.hashCode()); - h = 41 * (h + Boolean.valueOf(suggestQuotes).hashCode()); - if (cause != null) - h = 41 * (h + cause.hashCode()); - return h; - } - } - - static private class Comment extends Token { - final private String text; - - Comment(ConfigOrigin origin, String text) { - super(TokenType.COMMENT, origin); - this.text = text; - } - - String text() { - return text; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("'#"); - sb.append(text); - sb.append("' (COMMENT)"); - return sb.toString(); - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof Comment; - } - - @Override - public boolean equals(Object other) { - return super.equals(other) && ((Comment) other).text.equals(text); - } - - @Override - public int hashCode() { - int h = 41 * (41 + super.hashCode()); - h = 41 * (h + text.hashCode()); - return h; - } - } - - // This is not a Value, because it requires special processing - static private class Substitution extends Token { - final private boolean optional; - final private List value; - - Substitution(ConfigOrigin origin, boolean optional, List expression) { - super(TokenType.SUBSTITUTION, origin); - this.optional = optional; - this.value = expression; - } - - boolean optional() { - return optional; - } - - List value() { - return value; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - for (Token t : value) { - sb.append(t.toString()); - } - return "'${" + sb.toString() + "}'"; - } - - @Override - protected boolean canEqual(Object other) { - return other instanceof Substitution; - } - - @Override - public boolean equals(Object other) { - return super.equals(other) - && ((Substitution) other).value.equals(value); - } - - @Override - public int hashCode() { - return 41 * (41 + super.hashCode()) + value.hashCode(); - } - } - - static boolean isValue(Token token) { - return token instanceof Value; - } - - static AbstractConfigValue getValue(Token token) { - if (token instanceof Value) { - return ((Value) token).value(); - } else { - throw new ConfigException.BugOrBroken( - "tried to get value of non-value token " + token); - } - } - - static boolean isValueWithType(Token t, ConfigValueType valueType) { - return isValue(t) && getValue(t).valueType() == valueType; - } - - static boolean isNewline(Token token) { - return token instanceof Line; - } - - static boolean isProblem(Token token) { - return token instanceof Problem; - } - - static String getProblemMessage(Token token) { - if (token instanceof Problem) { - return ((Problem) token).message(); - } else { - throw new ConfigException.BugOrBroken("tried to get problem message from " + token); - } - } - - static boolean getProblemSuggestQuotes(Token token) { - if (token instanceof Problem) { - return ((Problem) token).suggestQuotes(); - } else { - throw new ConfigException.BugOrBroken("tried to get problem suggestQuotes from " - + token); - } - } - - static Throwable getProblemCause(Token token) { - if (token instanceof Problem) { - return ((Problem) token).cause(); - } else { - throw new ConfigException.BugOrBroken("tried to get problem cause from " + token); - } - } - - static boolean isComment(Token token) { - return token instanceof Comment; - } - - static String getCommentText(Token token) { - if (token instanceof Comment) { - return ((Comment) token).text(); - } else { - throw new ConfigException.BugOrBroken("tried to get comment text from " + token); - } - } - - static boolean isUnquotedText(Token token) { - return token instanceof UnquotedText; - } - - static String getUnquotedText(Token token) { - if (token instanceof UnquotedText) { - return ((UnquotedText) token).value(); - } else { - throw new ConfigException.BugOrBroken( - "tried to get unquoted text from " + token); - } - } - - static boolean isSubstitution(Token token) { - return token instanceof Substitution; - } - - static List getSubstitutionPathExpression(Token token) { - if (token instanceof Substitution) { - return ((Substitution) token).value(); - } else { - throw new ConfigException.BugOrBroken( - "tried to get substitution from " + token); - } - } - - static boolean getSubstitutionOptional(Token token) { - if (token instanceof Substitution) { - return ((Substitution) token).optional(); - } else { - throw new ConfigException.BugOrBroken("tried to get substitution optionality from " - + token); - } - } - - final static Token START = Token.newWithoutOrigin(TokenType.START, "start of file"); - final static Token END = Token.newWithoutOrigin(TokenType.END, "end of file"); - final static Token COMMA = Token.newWithoutOrigin(TokenType.COMMA, "','"); - final static Token EQUALS = Token.newWithoutOrigin(TokenType.EQUALS, "'='"); - final static Token COLON = Token.newWithoutOrigin(TokenType.COLON, "':'"); - final static Token OPEN_CURLY = Token.newWithoutOrigin(TokenType.OPEN_CURLY, "'{'"); - final static Token CLOSE_CURLY = Token.newWithoutOrigin(TokenType.CLOSE_CURLY, "'}'"); - final static Token OPEN_SQUARE = Token.newWithoutOrigin(TokenType.OPEN_SQUARE, "'['"); - final static Token CLOSE_SQUARE = Token.newWithoutOrigin(TokenType.CLOSE_SQUARE, "']'"); - - static Token newLine(ConfigOrigin origin) { - return new Line(origin); - } - - static Token newProblem(ConfigOrigin origin, String what, String message, - boolean suggestQuotes, Throwable cause) { - return new Problem(origin, what, message, suggestQuotes, cause); - } - - static Token newComment(ConfigOrigin origin, String text) { - return new Comment(origin, text); - } - - static Token newUnquotedText(ConfigOrigin origin, String s) { - return new UnquotedText(origin, s); - } - - static Token newSubstitution(ConfigOrigin origin, boolean optional, List expression) { - return new Substitution(origin, optional, expression); - } - - static Token newValue(AbstractConfigValue value) { - return new Value(value); - } - - static Token newString(ConfigOrigin origin, String value) { - return newValue(new ConfigString(origin, value)); - } - - static Token newInt(ConfigOrigin origin, int value, String originalText) { - return newValue(ConfigNumber.newNumber(origin, value, - originalText)); - } - - static Token newDouble(ConfigOrigin origin, double value, - String originalText) { - return newValue(ConfigNumber.newNumber(origin, value, - originalText)); - } - - static Token newLong(ConfigOrigin origin, long value, String originalText) { - return newValue(ConfigNumber.newNumber(origin, value, - originalText)); - } - - static Token newNull(ConfigOrigin origin) { - return newValue(new ConfigNull(origin)); - } - - static Token newBoolean(ConfigOrigin origin, boolean value) { - return newValue(new ConfigBoolean(origin, value)); - } -} diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java b/akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java deleted file mode 100755 index 0028f2e023..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java +++ /dev/null @@ -1,16 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe Inc. - */ -package com.typesafe.config.impl; - -import java.util.Collection; - -/** - * Interface that tags a ConfigValue that is not mergeable until after - * substitutions are resolved. Basically these are special ConfigValue that - * never appear in a resolved tree, like {@link ConfigSubstitution} and - * {@link ConfigDelayedMerge}. - */ -interface Unmergeable { - Collection unmergedValues(); -} diff --git a/akka-actor/src/main/java/com/typesafe/config/package.html b/akka-actor/src/main/java/com/typesafe/config/package.html deleted file mode 100755 index 1e1c78bfb2..0000000000 --- a/akka-actor/src/main/java/com/typesafe/config/package.html +++ /dev/null @@ -1,42 +0,0 @@ - - - - - - - - -

-An API for loading and using configuration files, see the project site -for more information. -

- -

-Typically you would load configuration with a static method from {@link com.typesafe.config.ConfigFactory} and then use -it with methods in the {@link com.typesafe.config.Config} interface. -

- -

-An application can simply call {@link com.typesafe.config.ConfigFactory#load()} and place -its configuration in "application.conf" on the classpath. -If you use the default configuration from {@link com.typesafe.config.ConfigFactory#load()} -there's no need to pass a configuration to your libraries -and frameworks, as long as they all default to this same default, which they should. -

- -

-A library or framework should ship a file "reference.conf" in its jar, and allow an application to pass in a -{@link com.typesafe.config.Config} to be used for the library. If no {@link com.typesafe.config.Config} is provided, -call {@link com.typesafe.config.ConfigFactory#load()} -to get the default one. Typically a library might offer two constructors, one with a Config parameter -and one which uses {@link com.typesafe.config.ConfigFactory#load()}. -

- -

-You can find an example app and library on GitHub. -

- - - diff --git a/akka-actor/src/main/java/org/jboss/netty/akka/util/MapBackedSet.java b/akka-actor/src/main/java/org/jboss/netty/akka/util/MapBackedSet.java deleted file mode 100644 index 2bc1bc25e0..0000000000 --- a/akka-actor/src/main/java/org/jboss/netty/akka/util/MapBackedSet.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright 2009 Red Hat, Inc. - * - * Red Hat licenses this file to you under the Apache License, version 2.0 - * (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at: - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ -package org.jboss.netty.akka.util; - -import java.io.Serializable; -import java.util.AbstractSet; -import java.util.Iterator; -import java.util.Map; - -/** - * A {@link java.util.Map}-backed {@link java.util.Set}. - * - * @author The Netty Project - * @author Trustin Lee - * - * @version $Rev: 2080 $, $Date: 2010-01-26 18:04:19 +0900 (Tue, 26 Jan 2010) $ - */ -final class MapBackedSet extends AbstractSet implements Serializable { - - private static final long serialVersionUID = -6761513279741915432L; - - private final Map map; - - /** - * Creates a new instance which wraps the specified {@code map}. - */ - MapBackedSet(Map map) { - this.map = map; - } - - @Override - public int size() { - return map.size(); - } - - @Override - public boolean contains(Object o) { - return map.containsKey(o); - } - - @Override - public boolean add(E o) { - return map.put(o, Boolean.TRUE) == null; - } - - @Override - public boolean remove(Object o) { - return map.remove(o) != null; - } - - @Override - public void clear() { - map.clear(); - } - - @Override - public Iterator iterator() { - return map.keySet().iterator(); - } -} diff --git a/akka-actor/src/main/scala/akka/AkkaException.scala b/akka-actor/src/main/scala/akka/AkkaException.scala index 85de2504d3..8e49c7cb11 100644 --- a/akka-actor/src/main/scala/akka/AkkaException.scala +++ b/akka-actor/src/main/scala/akka/AkkaException.scala @@ -4,45 +4,25 @@ package akka -import akka.actor.newUuid - -object AkkaException { - - def toStringWithStackTrace(throwable: Throwable): String = throwable match { - case null ⇒ "Unknown Throwable: was 'null'" - case ae: AkkaException ⇒ ae.toLongString - case e ⇒ "%s:%s\n%s" format (e.getClass.getName, e.getMessage, stackTraceToString(e)) - } - - def stackTraceToString(throwable: Throwable): String = { - val trace = throwable.getStackTrace - val sb = new StringBuilder - for (i ← 0 until trace.length) - sb.append("\tat %s\n" format trace(i)) - sb.toString - } - -} - /** * Akka base Exception. Each Exception gets: *
    *
  • a uuid for tracking purposes
  • *
  • toString that includes exception name, message and uuid
  • - *
  • toLongString which also includes the stack trace
  • *
*/ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed -class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable { - lazy val uuid = newUuid.toString +class AkkaException(message: String, cause: Throwable) extends RuntimeException(message, cause) with Serializable { + def this(msg: String) = this(msg, null) - override lazy val toString = - "%s:%s\n[%s]".format(getClass.getName, message, uuid) + lazy val uuid: String = java.util.UUID.randomUUID().toString - lazy val toLongString = - "%s:%s\n[%s]\n%s".format(getClass.getName, message, uuid, stackTraceToString) - - def this(msg: String) = this(msg, null); - - def stackTraceToString = AkkaException.stackTraceToString(this) + override def toString(): String = uuid + super.toString() +} + +/** + * This exception is thrown when Akka detects a problem with the provided configuration + */ +class ConfigurationException(message: String, cause: Throwable) extends AkkaException(message, cause) { + def this(msg: String) = this(msg, null) } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 2499d42f10..8b9476efe9 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -7,13 +7,13 @@ package akka.actor import akka.AkkaException import scala.reflect.BeanProperty import scala.util.control.NoStackTrace -import scala.collection.immutable.Stack import java.util.regex.Pattern /** * Marker trait to show which Messages are automatically handled by Akka + * Internal use only */ -trait AutoReceivedMessage extends Serializable +private[akka] trait AutoReceivedMessage extends Serializable /** * Marker trait to indicate that a message might be potentially harmful, @@ -26,9 +26,16 @@ trait PossiblyHarmful */ trait NoSerializationVerificationNeeded -case class Failed(cause: Throwable) extends AutoReceivedMessage with PossiblyHarmful +/** + * Internal use only + */ +private[akka] case class Failed(cause: Throwable) extends AutoReceivedMessage with PossiblyHarmful abstract class PoisonPill extends AutoReceivedMessage with PossiblyHarmful + +/** + * A message all Actors will understand, that when processed will terminate the Actor permanently. + */ case object PoisonPill extends PoisonPill { /** * Java API: get the singleton instance @@ -37,6 +44,10 @@ case object PoisonPill extends PoisonPill { } abstract class Kill extends AutoReceivedMessage with PossiblyHarmful +/** + * A message all Actors will understand, that when processed will make the Actor throw an ActorKilledException, + * which will trigger supervision. + */ case object Kill extends Kill { /** * Java API: get the singleton instance @@ -44,9 +55,17 @@ case object Kill extends Kill { def getInstance = this } -case class Terminated(@BeanProperty actor: ActorRef) extends PossiblyHarmful +/** + * When Death Watch is used, the watcher will receive a Terminated(watched) message when watched is terminated. + */ +case class Terminated(@BeanProperty actor: ActorRef)(@BeanProperty val existenceConfirmed: Boolean) extends AutoReceivedMessage abstract class ReceiveTimeout extends PossiblyHarmful + +/** + * When using ActorContext.setReceiveTimeout, the singleton instance of ReceiveTimeout will be sent + * to the Actor when there hasn't been any message for that long. + */ case object ReceiveTimeout extends ReceiveTimeout { /** * Java API: get the singleton instance @@ -60,49 +79,78 @@ case object ReceiveTimeout extends ReceiveTimeout { * message is delivered by active routing of the various actors involved. */ sealed trait SelectionPath extends AutoReceivedMessage -case class SelectChildName(name: String, next: Any) extends SelectionPath -case class SelectChildPattern(pattern: Pattern, next: Any) extends SelectionPath -case class SelectParent(next: Any) extends SelectionPath -// Exceptions for Actors +/** + * Internal use only + */ +private[akka] case class SelectChildName(name: String, next: Any) extends SelectionPath + +/** + * Internal use only + */ +private[akka] case class SelectChildPattern(pattern: Pattern, next: Any) extends SelectionPath + +/** + * Internal use only + */ +private[akka] case class SelectParent(next: Any) extends SelectionPath + +/** + * IllegalActorStateException is thrown when a core invariant in the Actor implementation has been violated. + * For instance, if you try to create an Actor that doesn't extend Actor. + */ class IllegalActorStateException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) { - def this(msg: String) = this(msg, null); + def this(msg: String) = this(msg, null) } +/** + * ActorKilledException is thrown when an Actor receives the akka.actor.Kill message + */ class ActorKilledException private[akka] (message: String, cause: Throwable) extends AkkaException(message, cause) with NoStackTrace { - def this(msg: String) = this(msg, null); + def this(msg: String) = this(msg, null) } -case class InvalidActorNameException(message: String) extends AkkaException(message) +/** + * An InvalidActorNameException is thrown when you try to convert something, usually a String, to an Actor name + * which doesn't validate. + */ +class InvalidActorNameException(message: String) extends AkkaException(message) -case class ActorInitializationException private[akka] (actor: ActorRef, message: String, cause: Throwable = null) - extends AkkaException(message, cause) - with NoStackTrace { - def this(msg: String) = this(null, msg, null); -} - -class ActorTimeoutException private[akka] (message: String, cause: Throwable = null) - extends AkkaException(message, cause) { - def this(msg: String) = this(msg, null); +/** + * An ActorInitializationException is thrown when the the initialization logic for an Actor fails. + */ +class ActorInitializationException private[akka] (actor: ActorRef, message: String, cause: Throwable) + extends AkkaException(message, cause) /*with NoStackTrace*/ { + def this(msg: String) = this(null, msg, null) + def this(actor: ActorRef, msg: String) = this(actor, msg, null) } +/** + * InvalidMessageException is thrown when an invalid message is sent to an Actor. + * Technically it's only "null" which is an InvalidMessageException but who knows, + * there might be more of them in the future, or not. + */ class InvalidMessageException private[akka] (message: String, cause: Throwable = null) - extends AkkaException(message, cause) - with NoStackTrace { - def this(msg: String) = this(msg, null); + extends AkkaException(message, cause) { + def this(msg: String) = this(msg, null) } +/** + * A DeathPactException is thrown by an Actor that receives a Terminated(someActor) message + * that it doesn't handle itself, effectively crashing the Actor and escalating to the supervisor. + */ case class DeathPactException private[akka] (dead: ActorRef) extends AkkaException("Monitored actor [" + dead + "] terminated") with NoStackTrace -// must not pass InterruptedException to other threads -case class ActorInterruptedException private[akka] (cause: Throwable) - extends AkkaException(cause.getMessage, cause) - with NoStackTrace +/** + * When an InterruptedException is thrown inside an Actor, it is wrapped as an ActorInterruptedException as to + * avoid cascading interrupts to other threads than the originally interrupted one. + */ +class ActorInterruptedException private[akka] (cause: Throwable) extends AkkaException(cause.getMessage, cause) with NoStackTrace /** * This message is published to the EventStream whenever an Actor receives a message it doesn't understand @@ -115,18 +163,43 @@ case class UnhandledMessage(@BeanProperty message: Any, @BeanProperty sender: Ac */ object Status { sealed trait Status extends Serializable + + /** + * This class/message type is preferably used to indicate success of some operation performed. + */ case class Success(status: AnyRef) extends Status + + /** + * This class/message type is preferably used to indicate failure of some operation performed. + * As an example, it is used to signal failure with AskSupport is used (ask/?). + */ case class Failure(cause: Throwable) extends Status } +/** + * Mix in ActorLogging into your Actor to easily obtain a reference to a logger, which is available under the name "log". + * + * {{{ + * class MyActor extends Actor with ActorLogging { + * def receive = { + * case "pigdog" => log.info("We've got yet another pigdog on our hands") + * } + * } + * }}} + */ trait ActorLogging { this: Actor ⇒ val log = akka.event.Logging(context.system, this) } object Actor { - + /** + * Type alias representing a Receive-expression for Akka Actors. + */ type Receive = PartialFunction[Any, Unit] + /** + * emptyBehavior is a Receive-expression that matches no messages at all, ever. + */ object emptyBehavior extends Receive { def isDefinedAt(x: Any) = false def apply(x: Any) = throw new UnsupportedOperationException("Empty behavior apply()") @@ -204,18 +277,14 @@ trait Actor { */ protected[akka] implicit val context: ActorContext = { val contextStack = ActorCell.contextStack.get - - def noContextError = + if ((contextStack.isEmpty) || (contextStack.head eq null)) throw new ActorInitializationException( "\n\tYou cannot create an instance of [" + getClass.getName + "] explicitly using the constructor (new)." + "\n\tYou have to use one of the factory methods to create a new actor. Either use:" + "\n\t\t'val actor = context.actorOf(Props[MyActor])' (to create a supervised child actor from within an actor), or" + "\n\t\t'val actor = system.actorOf(Props(new MyActor(..)))' (to create a top level actor from the ActorSystem)") - - if (contextStack.isEmpty) noContextError val c = contextStack.head - if (c eq null) noContextError - ActorCell.contextStack.set(contextStack.push(null)) + ActorCell.contextStack.set(null :: contextStack) c } @@ -243,13 +312,13 @@ trait Actor { * This defines the initial actor behavior, it must return a partial function * with the actor logic. */ - protected def receive: Receive + def receive: Receive /** * User overridable definition the strategy to use for supervising * child actors. */ - def supervisorStrategy(): SupervisorStrategy = SupervisorStrategy.defaultStrategy + def supervisorStrategy: SupervisorStrategy = SupervisorStrategy.defaultStrategy /** * User overridable callback. @@ -303,42 +372,5 @@ trait Actor { case _ ⇒ context.system.eventStream.publish(UnhandledMessage(message, sender, self)) } } - - // ========================================= - // ==== INTERNAL IMPLEMENTATION DETAILS ==== - // ========================================= - - /** - * For Akka internal use only. - */ - private[akka] final def apply(msg: Any) = { - // TODO would it be more efficient to assume that most messages are matched and catch MatchError instead of using isDefinedAt? - val head = behaviorStack.head - if (head.isDefinedAt(msg)) head.apply(msg) else unhandled(msg) - } - - /** - * For Akka internal use only. - */ - private[akka] def pushBehavior(behavior: Receive): Unit = { - behaviorStack = behaviorStack.push(behavior) - } - - /** - * For Akka internal use only. - */ - private[akka] def popBehavior(): Unit = { - val original = behaviorStack - val popped = original.pop - behaviorStack = if (popped.isEmpty) original else popped - } - - /** - * For Akka internal use only. - */ - private[akka] def clearBehaviorStack(): Unit = - behaviorStack = Stack.empty[Receive].push(behaviorStack.last) - - private var behaviorStack: Stack[Receive] = Stack.empty[Receive].push(receive) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 8c68ba3315..e739ffc859 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -6,16 +6,16 @@ package akka.actor import akka.dispatch._ import scala.annotation.tailrec -import scala.collection.immutable.{ Stack, TreeMap } import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit.MILLISECONDS import akka.event.Logging.{ Debug, Warning, Error } -import akka.util.{ Duration, Helpers } import akka.japi.Procedure import java.io.{ NotSerializableException, ObjectOutputStream } import akka.serialization.SerializationExtension -import akka.util.NonFatal import akka.event.Logging.LogEventException +import collection.immutable.{ TreeSet, TreeMap } +import akka.util.{ Unsafe, Duration, Helpers, NonFatal } +import java.util.concurrent.atomic.AtomicLong //TODO: everything here for current compatibility - could be limited more @@ -134,10 +134,17 @@ trait ActorContext extends ActorRefFactory { */ def unwatch(subject: ActorRef): ActorRef + /** + * ActorContexts shouldn't be Serializable + */ final protected def writeObject(o: ObjectOutputStream): Unit = throw new NotSerializableException("ActorContext is not serializable!") } +/** + * UntypedActorContext is the UntypedActor equivalent of ActorContext, + * containing the Java API + */ trait UntypedActorContext extends ActorContext { /** @@ -161,14 +168,86 @@ trait UntypedActorContext extends ActorContext { } +/** + * INTERNAL API + */ +private[akka] trait Cell { + /** + * The “self” reference which this Cell is attached to. + */ + def self: ActorRef + /** + * The system within which this Cell lives. + */ + def system: ActorSystem + /** + * The system internals where this Cell lives. + */ + def systemImpl: ActorSystemImpl + /** + * Recursively suspend this actor and all its children. + */ + def suspend(): Unit + /** + * Recursively resume this actor and all its children. + */ + def resume(): Unit + /** + * Restart this actor (will recursively restart or stop all children). + */ + def restart(cause: Throwable): Unit + /** + * Recursively terminate this actor and all its children. + */ + def stop(): Unit + /** + * Returns “true” if the actor is locally known to be terminated, “false” if + * alive or uncertain. + */ + def isTerminated: Boolean + /** + * The supervisor of this actor. + */ + def parent: InternalActorRef + /** + * All children of this actor, including only reserved-names. + */ + def childrenRefs: ActorCell.ChildrenContainer + /** + * Enqueue a message to be sent to the actor; may or may not actually + * schedule the actor to run, depending on which type of cell it is. + */ + def tell(message: Any, sender: ActorRef): Unit + /** + * Enqueue a message to be sent to the actor; may or may not actually + * schedule the actor to run, depending on which type of cell it is. + */ + def sendSystemMessage(msg: SystemMessage): Unit + /** + * Returns true if the actor is local, i.e. if it is actually scheduled + * on a Thread in the current JVM when run. + */ + def isLocal: Boolean + /** + * If the actor isLocal, returns whether messages are currently queued, + * “false” otherwise. + */ + def hasMessages: Boolean + /** + * If the actor isLocal, returns the number of messages currently queued, + * which may be a costly operation, 0 otherwise. + */ + def numberOfMessages: Int +} + /** * Everything in here is completely Akka PRIVATE. You will not find any * supported APIs in this place. This is not the API you were looking * for! (waves hand) */ private[akka] object ActorCell { - val contextStack = new ThreadLocal[Stack[ActorContext]] { - override def initialValue = Stack[ActorContext]() + val contextStack = new ThreadLocal[List[ActorContext]] { + override def initialValue: List[ActorContext] = Nil } final val emptyCancellable: Cancellable = new Cancellable { @@ -178,7 +257,11 @@ private[akka] object ActorCell { final val emptyReceiveTimeoutData: (Long, Cancellable) = (-1, emptyCancellable) - trait SuspendReason + final val emptyBehaviorStack: List[Actor.Receive] = Nil + + final val emptyActorRefSet: Set[ActorRef] = TreeSet.empty + + sealed trait SuspendReason case object UserRequest extends SuspendReason case class Recreation(cause: Throwable) extends SuspendReason case object Termination extends SuspendReason @@ -191,10 +274,18 @@ private[akka] object ActorCell { def children: Iterable[ActorRef] def stats: Iterable[ChildRestartStats] def shallDie(actor: ActorRef): ChildrenContainer + /** + * reserve that name or throw an exception + */ + def reserve(name: String): ChildrenContainer + /** + * cancel a reservation + */ + def unreserve(name: String): ChildrenContainer } trait EmptyChildrenContainer extends ChildrenContainer { - val emptyStats = TreeMap.empty[String, ChildRestartStats] + val emptyStats = TreeMap.empty[String, ChildStats] def add(child: ActorRef): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(child.path.name, ChildRestartStats(child))) def remove(child: ActorRef): ChildrenContainer = this @@ -203,6 +294,8 @@ private[akka] object ActorCell { def children: Iterable[ActorRef] = Nil def stats: Iterable[ChildRestartStats] = Nil def shallDie(actor: ActorRef): ChildrenContainer = this + def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, ChildNameReserved)) + def unreserve(name: String): ChildrenContainer = this override def toString = "no children" } @@ -218,6 +311,8 @@ private[akka] object ActorCell { */ object TerminatedChildrenContainer extends EmptyChildrenContainer { override def add(child: ActorRef): ChildrenContainer = this + override def reserve(name: String): ChildrenContainer = + throw new IllegalStateException("cannot reserve actor name '" + name + "': already terminated") } /** @@ -226,32 +321,46 @@ private[akka] object ActorCell { * calling context.stop(child) and processing the ChildTerminated() system * message). */ - class NormalChildrenContainer(c: TreeMap[String, ChildRestartStats]) extends ChildrenContainer { + class NormalChildrenContainer(c: TreeMap[String, ChildStats]) extends ChildrenContainer { - def add(child: ActorRef): ChildrenContainer = new NormalChildrenContainer(c.updated(child.path.name, ChildRestartStats(child))) + def add(child: ActorRef): ChildrenContainer = + new NormalChildrenContainer(c.updated(child.path.name, ChildRestartStats(child))) def remove(child: ActorRef): ChildrenContainer = NormalChildrenContainer(c - child.path.name) - def getByName(name: String): Option[ChildRestartStats] = c get name - - def getByRef(actor: ActorRef): Option[ChildRestartStats] = c get actor.path.name match { - case c @ Some(crs) if (crs.child == actor) ⇒ c - case _ ⇒ None + def getByName(name: String): Option[ChildRestartStats] = c.get(name) match { + case s @ Some(_: ChildRestartStats) ⇒ s.asInstanceOf[Option[ChildRestartStats]] + case _ ⇒ None } - def children: Iterable[ActorRef] = c.values.view.map(_.child) + def getByRef(actor: ActorRef): Option[ChildRestartStats] = c.get(actor.path.name) match { + case c @ Some(crs: ChildRestartStats) if (crs.child == actor) ⇒ c.asInstanceOf[Option[ChildRestartStats]] + case _ ⇒ None + } - def stats: Iterable[ChildRestartStats] = c.values + def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) ⇒ child } + + def stats: Iterable[ChildRestartStats] = c.values.collect { case c: ChildRestartStats ⇒ c } def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest) + def reserve(name: String): ChildrenContainer = + if (c contains name) + throw new InvalidActorNameException("actor name " + name + " is not unique!") + else new NormalChildrenContainer(c.updated(name, ChildNameReserved)) + + def unreserve(name: String): ChildrenContainer = c.get(name) match { + case Some(ChildNameReserved) ⇒ NormalChildrenContainer(c - name) + case _ ⇒ this + } + override def toString = if (c.size > 20) c.size + " children" else c.mkString("children:\n ", "\n ", "") } object NormalChildrenContainer { - def apply(c: TreeMap[String, ChildRestartStats]): ChildrenContainer = + def apply(c: TreeMap[String, ChildStats]): ChildrenContainer = if (c.isEmpty) EmptyChildrenContainer else new NormalChildrenContainer(c) } @@ -266,7 +375,7 @@ private[akka] object ActorCell { * type of container, depending on whether or not children are left and whether or not * the reason was “Terminating”. */ - case class TerminatingChildrenContainer(c: TreeMap[String, ChildRestartStats], toDie: Set[ActorRef], reason: SuspendReason) + case class TerminatingChildrenContainer(c: TreeMap[String, ChildStats], toDie: Set[ActorRef], reason: SuspendReason) extends ChildrenContainer { def add(child: ActorRef): ChildrenContainer = copy(c.updated(child.path.name, ChildRestartStats(child))) @@ -280,19 +389,35 @@ private[akka] object ActorCell { else copy(c - child.path.name, t) } - def getByName(name: String): Option[ChildRestartStats] = c get name - - def getByRef(actor: ActorRef): Option[ChildRestartStats] = c get actor.path.name match { - case c @ Some(crs) if (crs.child == actor) ⇒ c - case _ ⇒ None + def getByName(name: String): Option[ChildRestartStats] = c.get(name) match { + case s @ Some(_: ChildRestartStats) ⇒ s.asInstanceOf[Option[ChildRestartStats]] + case _ ⇒ None } - def children: Iterable[ActorRef] = c.values.view.map(_.child) + def getByRef(actor: ActorRef): Option[ChildRestartStats] = c.get(actor.path.name) match { + case c @ Some(crs: ChildRestartStats) if (crs.child == actor) ⇒ c.asInstanceOf[Option[ChildRestartStats]] + case _ ⇒ None + } - def stats: Iterable[ChildRestartStats] = c.values + def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) ⇒ child } + + def stats: Iterable[ChildRestartStats] = c.values.collect { case c: ChildRestartStats ⇒ c } def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor) + def reserve(name: String): ChildrenContainer = reason match { + case Termination ⇒ throw new IllegalStateException("cannot reserve actor name '" + name + "': terminating") + case _ ⇒ + if (c contains name) + throw new InvalidActorNameException("actor name " + name + " is not unique!") + else copy(c = c.updated(name, ChildNameReserved)) + } + + def unreserve(name: String): ChildrenContainer = c.get(name) match { + case Some(ChildNameReserved) ⇒ copy(c = c - name) + case _ ⇒ this + } + override def toString = if (c.size > 20) c.size + " children" else c.mkString("children (" + toDie.size + " terminating):\n ", "\n ", "\n") + toDie @@ -306,11 +431,13 @@ private[akka] class ActorCell( val system: ActorSystemImpl, val self: InternalActorRef, val props: Props, - @volatile var parent: InternalActorRef, - /*no member*/ _receiveTimeout: Option[Duration]) extends UntypedActorContext { + @volatile var parent: InternalActorRef) extends UntypedActorContext with Cell { + import AbstractActorCell.{ mailboxOffset, childrenOffset, nextNameOffset } import ActorCell._ + final def isLocal = true + final def systemImpl = system protected final def guardian = self @@ -341,11 +468,49 @@ private[akka] class ActorCell( /** * In milliseconds */ - var receiveTimeoutData: (Long, Cancellable) = - if (_receiveTimeout.isDefined) (_receiveTimeout.get.toMillis, emptyCancellable) else emptyReceiveTimeoutData + var receiveTimeoutData: (Long, Cancellable) = emptyReceiveTimeoutData @volatile - var childrenRefs: ChildrenContainer = EmptyChildrenContainer + private var _childrenRefsDoNotCallMeDirectly: ChildrenContainer = EmptyChildrenContainer + + def childrenRefs: ChildrenContainer = Unsafe.instance.getObjectVolatile(this, childrenOffset).asInstanceOf[ChildrenContainer] + + private def swapChildrenRefs(oldChildren: ChildrenContainer, newChildren: ChildrenContainer): Boolean = + Unsafe.instance.compareAndSwapObject(this, childrenOffset, oldChildren, newChildren) + + @tailrec private def reserveChild(name: String): Boolean = { + val c = childrenRefs + swapChildrenRefs(c, c.reserve(name)) || reserveChild(name) + } + + @tailrec private def unreserveChild(name: String): Boolean = { + val c = childrenRefs + swapChildrenRefs(c, c.unreserve(name)) || unreserveChild(name) + } + + @tailrec private def addChild(ref: ActorRef): Boolean = { + val c = childrenRefs + swapChildrenRefs(c, c.add(ref)) || addChild(ref) + } + + @tailrec private def shallDie(ref: ActorRef): Boolean = { + val c = childrenRefs + swapChildrenRefs(c, c.shallDie(ref)) || shallDie(ref) + } + + @tailrec private def removeChild(ref: ActorRef): ChildrenContainer = { + val c = childrenRefs + val n = c.remove(ref) + if (swapChildrenRefs(c, n)) n + else removeChild(ref) + } + + @tailrec private def setChildrenTerminationReason(reason: SuspendReason): Boolean = { + childrenRefs match { + case c: TerminatingChildrenContainer ⇒ swapChildrenRefs(c, c.copy(reason = reason)) || setChildrenTerminationReason(reason) + case _ ⇒ false + } + } private def isTerminating = childrenRefs match { case TerminatingChildrenContainer(_, _, Termination) ⇒ true @@ -357,7 +522,7 @@ private[akka] class ActorCell( case _ ⇒ true } - private def _actorOf(props: Props, name: String): ActorRef = { + private def _actorOf(props: Props, name: String, async: Boolean): ActorRef = { if (system.settings.SerializeAllCreators && !props.creator.isInstanceOf[NoSerializationVerificationNeeded]) { val ser = SerializationExtension(system) ser.serialize(props.creator) match { @@ -368,54 +533,98 @@ private[akka] class ActorCell( } } } - // in case we are currently terminating, swallow creation requests and return EmptyLocalActorRef - if (isTerminating) provider.actorFor(self, Seq(name)) + /* + * in case we are currently terminating, fail external attachChild requests + * (internal calls cannot happen anyway because we are suspended) + */ + if (isTerminating) throw new IllegalStateException("cannot create children while terminating or terminated") else { - val actor = provider.actorOf(systemImpl, props, self, self.path / name, false, None, true) - childrenRefs = childrenRefs.add(actor) + reserveChild(name) + // this name will either be unreserved or overwritten with a real child below + val actor = + try { + provider.actorOf(systemImpl, props, self, self.path / name, + systemService = false, deploy = None, lookupDeploy = true, async = async) + } catch { + case NonFatal(e) ⇒ + unreserveChild(name) + throw e + } + addChild(actor) actor } } - def actorOf(props: Props): ActorRef = _actorOf(props, randomName()) + def actorOf(props: Props): ActorRef = _actorOf(props, randomName(), async = false) - def actorOf(props: Props, name: String): ActorRef = { + def actorOf(props: Props, name: String): ActorRef = _actorOf(props, checkName(name), async = false) + + private def checkName(name: String): String = { import ActorPath.ElementRegex name match { case null ⇒ throw new InvalidActorNameException("actor name must not be null") case "" ⇒ throw new InvalidActorNameException("actor name must not be empty") - case ElementRegex() ⇒ // this is fine + case ElementRegex() ⇒ name case _ ⇒ throw new InvalidActorNameException("illegal actor name '" + name + "', must conform to " + ElementRegex) } - childrenRefs.getByName(name) match { - case None ⇒ _actorOf(props, name) - case _ ⇒ throw new InvalidActorNameException("actor name " + name + " is not unique!") - } } + private[akka] def attachChild(props: Props, name: String): ActorRef = + _actorOf(props, checkName(name), async = true) + + private[akka] def attachChild(props: Props): ActorRef = + _actorOf(props, randomName(), async = true) + final def stop(actor: ActorRef): Unit = { - if (childrenRefs.getByRef(actor).isDefined) childrenRefs = childrenRefs.shallDie(actor) + val started = actor match { + case r: RepointableRef ⇒ r.isStarted + case _ ⇒ true + } + if (childrenRefs.getByRef(actor).isDefined && started) shallDie(actor) actor.asInstanceOf[InternalActorRef].stop() } - var currentMessage: Envelope = null - + var currentMessage: Envelope = _ var actor: Actor = _ + private var behaviorStack: List[Actor.Receive] = emptyBehaviorStack + var watching: Set[ActorRef] = emptyActorRefSet + var watchedBy: Set[ActorRef] = emptyActorRefSet - @volatile //This must be volatile since it isn't protected by the mailbox status - var mailbox: Mailbox = _ - - var nextNameSequence: Long = 0 - - //Not thread safe, so should only be used inside the actor that inhabits this ActorCell + @volatile private var _nextNameDoNotCallMeDirectly = 0L final protected def randomName(): String = { - val n = nextNameSequence - nextNameSequence = n + 1 - Helpers.base64(n) + @tailrec def inc(): Long = { + val current = Unsafe.instance.getLongVolatile(this, nextNameOffset) + if (Unsafe.instance.compareAndSwapLong(this, nextNameOffset, current, current + 1)) current + else inc() + } + Helpers.base64(inc()) } - @inline - final val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher) + @volatile private var _mailboxDoNotCallMeDirectly: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status + + /** + * INTERNAL API + * + * Returns a reference to the current mailbox + */ + @inline final def mailbox: Mailbox = Unsafe.instance.getObjectVolatile(this, mailboxOffset).asInstanceOf[Mailbox] + + /** + * INTERNAL API + * + * replaces the current mailbox using getAndSet semantics + */ + @tailrec final def swapMailbox(newMailbox: Mailbox): Mailbox = { + val oldMailbox = mailbox + if (!Unsafe.instance.compareAndSwapObject(this, mailboxOffset, oldMailbox, newMailbox)) swapMailbox(newMailbox) + else oldMailbox + } + + final def hasMessages: Boolean = mailbox.hasMessages + + final def numberOfMessages: Int = mailbox.numberOfMessages + + val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher) /** * UntypedActorContext impl @@ -424,20 +633,22 @@ private[akka] class ActorCell( final def isTerminated: Boolean = mailbox.isClosed - final def start(): Unit = { + final def start(): this.type = { + /* * Create the mailbox and enqueue the Create() message to ensure that * this is processed before anything else. */ - mailbox = dispatcher.createMailbox(this) + swapMailbox(dispatcher.createMailbox(this)) + mailbox.setActor(this) + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ mailbox.systemEnqueue(self, Create()) - // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ - parent.sendSystemMessage(akka.dispatch.Supervise(self)) - // This call is expected to start off the actor by scheduling its mailbox. dispatcher.attach(this) + + this } // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ @@ -449,16 +660,22 @@ private[akka] class ActorCell( // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ final def stop(): Unit = dispatcher.systemDispatch(this, Terminate()) - override final def watch(subject: ActorRef): ActorRef = { - // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ - dispatcher.systemDispatch(this, Link(subject)) - subject + override final def watch(subject: ActorRef): ActorRef = subject match { + case a: InternalActorRef ⇒ + if (a != self && !watching.contains(a)) { + a.sendSystemMessage(Watch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ + watching += a + } + a } - override final def unwatch(subject: ActorRef): ActorRef = { - // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ - dispatcher.systemDispatch(this, Unlink(subject)) - subject + override final def unwatch(subject: ActorRef): ActorRef = subject match { + case a: InternalActorRef ⇒ + if (a != self && watching.contains(a)) { + a.sendSystemMessage(Unwatch(a, self)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ + watching -= a + } + a } final def children: Iterable[ActorRef] = childrenRefs.children @@ -466,13 +683,13 @@ private[akka] class ActorCell( /** * Impl UntypedActorContext */ - final def getChildren(): java.lang.Iterable[ActorRef] = { - import scala.collection.JavaConverters.asJavaIterableConverter - asJavaIterableConverter(children).asJava - } + final def getChildren(): java.lang.Iterable[ActorRef] = + scala.collection.JavaConverters.asJavaIterableConverter(children).asJava - final def tell(message: Any, sender: ActorRef): Unit = - dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender)(system)) + def tell(message: Any, sender: ActorRef): Unit = + dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender, system)) + + override def sendSystemMessage(message: SystemMessage): Unit = dispatcher.systemDispatch(this, message) final def sender: ActorRef = currentMessage match { case null ⇒ system.deadLetters @@ -482,19 +699,21 @@ private[akka] class ActorCell( //This method is in charge of setting up the contextStack and create a new instance of the Actor protected def newActor(): Actor = { - val stackBefore = contextStack.get - contextStack.set(stackBefore.push(this)) + contextStack.set(this :: contextStack.get) try { - val instance = props.creator() + behaviorStack = emptyBehaviorStack + val instance = props.creator.apply() if (instance eq null) - throw ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'") + throw new ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'") + // If no becomes were issued, the actors behavior is its receive method + behaviorStack = if (behaviorStack.isEmpty) instance.receive :: behaviorStack else behaviorStack instance } finally { val stackAfter = contextStack.get if (stackAfter.nonEmpty) - contextStack.set(if (stackAfter.head eq null) stackAfter.pop.pop else stackAfter.pop) // pop null marker plus our context + contextStack.set(if (stackAfter.head eq null) stackAfter.tail.tail else stackAfter.tail) // pop null marker plus our context } } @@ -510,13 +729,12 @@ private[akka] class ActorCell( if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(created), "started (" + created + ")")) } catch { case NonFatal(i: InstantiationException) ⇒ - throw ActorInitializationException(self, + throw new ActorInitializationException(self, """exception during creation, this problem is likely to occur because the class of the Actor you tried to create is either, a non-static inner class (in which case make it a static inner class or use Props(new ...) or Props( new UntypedActorFactory ... ) or is missing an appropriate, reachable no-args constructor. - """, i) - case NonFatal(e) ⇒ - throw ActorInitializationException(self, "exception during creation", e) + """, i.getCause) + case NonFatal(e) ⇒ throw new ActorInitializationException(self, "exception during creation", e) } } @@ -534,13 +752,16 @@ private[akka] class ActorCell( } childrenRefs match { case ct: TerminatingChildrenContainer ⇒ - childrenRefs = ct.copy(reason = Recreation(cause)) + setChildrenTerminationReason(Recreation(cause)) dispatcher suspend this case _ ⇒ doRecreate(cause, failedActor) } } catch { - case NonFatal(e) ⇒ throw ActorInitializationException(self, "exception during creation", e) + case NonFatal(e) ⇒ throw new ActorInitializationException(self, "exception during creation", e match { + case i: InstantiationException ⇒ i.getCause + case other ⇒ other + }) } } @@ -548,15 +769,35 @@ private[akka] class ActorCell( def resume(): Unit = if (isNormal) dispatcher resume this - def link(subject: ActorRef): Unit = if (!isTerminating) { - if (system.deathWatch.subscribe(self, subject)) { - if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now monitoring " + subject)) + def addWatcher(watchee: ActorRef, watcher: ActorRef): Unit = { + val watcheeSelf = watchee == self + val watcherSelf = watcher == self + + if (watcheeSelf && !watcherSelf) { + if (!watchedBy.contains(watcher)) { + watchedBy += watcher + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now monitoring " + watcher)) + } + } else if (!watcheeSelf && watcherSelf) { + watch(watchee) + } else { + system.eventStream.publish(Warning(self.path.toString, clazz(actor), "BUG: illegal Watch(%s,%s) for %s".format(watchee, watcher, self))) } } - def unlink(subject: ActorRef): Unit = if (!isTerminating) { - if (system.deathWatch.unsubscribe(self, subject)) { - if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped monitoring " + subject)) + def remWatcher(watchee: ActorRef, watcher: ActorRef): Unit = { + val watcheeSelf = watchee == self + val watcherSelf = watcher == self + + if (watcheeSelf && !watcherSelf) { + if (watchedBy.contains(watcher)) { + watchedBy -= watcher + if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped monitoring " + watcher)) + } + } else if (!watcheeSelf && watcherSelf) { + unwatch(watchee) + } else { + system.eventStream.publish(Warning(self.path.toString, clazz(actor), "BUG: illegal Unwatch(%s,%s) for %s".format(watchee, watcher, self))) } } @@ -569,7 +810,7 @@ private[akka] class ActorCell( childrenRefs match { case ct: TerminatingChildrenContainer ⇒ - childrenRefs = ct.copy(reason = Termination) + setChildrenTerminationReason(Termination) // do not process normal messages while waiting for all children to terminate dispatcher suspend this if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopping")) @@ -578,21 +819,23 @@ private[akka] class ActorCell( } def supervise(child: ActorRef): Unit = if (!isTerminating) { - if (childrenRefs.getByRef(child).isEmpty) childrenRefs = childrenRefs.add(child) + if (childrenRefs.getByRef(child).isEmpty) addChild(child) + handleSupervise(child) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now supervising " + child)) } try { message match { - case Create() ⇒ create() - case Recreate(cause) ⇒ recreate(cause) - case Link(subject) ⇒ link(subject) - case Unlink(subject) ⇒ unlink(subject) - case Suspend() ⇒ suspend() - case Resume() ⇒ resume() - case Terminate() ⇒ terminate() - case Supervise(child) ⇒ supervise(child) - case ChildTerminated(child) ⇒ handleChildTerminated(child) + case Create() ⇒ create() + case Recreate(cause) ⇒ recreate(cause) + case Watch(watchee, watcher) ⇒ addWatcher(watchee, watcher) + case Unwatch(watchee, watcher) ⇒ remWatcher(watchee, watcher) + case Suspend() ⇒ suspend() + case Resume() ⇒ resume() + case Terminate() ⇒ terminate() + case Supervise(child) ⇒ supervise(child) + case ChildTerminated(child) ⇒ handleChildTerminated(child) + case NoMessage ⇒ // only here to suppress warning } } catch { case e @ (_: InterruptedException | NonFatal(_)) ⇒ handleInvokeFailure(e, "error while processing " + message) @@ -605,7 +848,7 @@ private[akka] class ActorCell( cancelReceiveTimeout() // FIXME: leave this here??? messageHandle.message match { case msg: AutoReceivedMessage ⇒ autoReceiveMessage(messageHandle) - case msg ⇒ actor(msg) + case msg ⇒ receiveMessage(msg) } currentMessage = null // reset current message after successful invocation } catch { @@ -614,22 +857,20 @@ private[akka] class ActorCell( checkReceiveTimeout // Reschedule receive timeout } - private final def handleInvokeFailure(t: Throwable, message: String): Unit = try { + final def handleInvokeFailure(t: Throwable, message: String): Unit = try { dispatcher.reportFailure(new LogEventException(Error(t, self.path.toString, clazz(actor), message), t)) // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) if (actor ne null) actor.supervisorStrategy.handleSupervisorFailing(self, children) } finally { t match { // Wrap InterruptedExceptions and rethrow - case _: InterruptedException ⇒ parent.tell(Failed(ActorInterruptedException(t)), self); throw t + case _: InterruptedException ⇒ parent.tell(Failed(new ActorInterruptedException(t)), self); throw t case _ ⇒ parent.tell(Failed(t), self) } } - def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit = { - if (discardOld) unbecome() - actor.pushBehavior(behavior) - } + def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit = + behaviorStack = behavior :: (if (discardOld && behaviorStack.nonEmpty) behaviorStack.tail else behaviorStack) /** * UntypedActorContext impl @@ -639,19 +880,23 @@ private[akka] class ActorCell( /* * UntypedActorContext impl */ - def become(behavior: Procedure[Any], discardOld: Boolean): Unit = { - def newReceive: Actor.Receive = { case msg ⇒ behavior.apply(msg) } - become(newReceive, discardOld) + def become(behavior: Procedure[Any], discardOld: Boolean): Unit = + become({ case msg ⇒ behavior.apply(msg) }: Actor.Receive, discardOld) + + def unbecome(): Unit = { + val original = behaviorStack + behaviorStack = + if (original.isEmpty || original.tail.isEmpty) actor.receive :: emptyBehaviorStack + else original.tail } - def unbecome(): Unit = actor.popBehavior() - - def autoReceiveMessage(msg: Envelope) { + def autoReceiveMessage(msg: Envelope): Unit = { if (system.settings.DebugAutoReceive) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "received AutoReceiveMessage " + msg)) msg.message match { case Failed(cause) ⇒ handleFailure(sender, cause) + case t: Terminated ⇒ watching -= t.actor; receiveMessage(t) case Kill ⇒ throw new ActorKilledException("Kill") case PoisonPill ⇒ self.stop() case SelectParent(m) ⇒ parent.tell(m, msg.sender) @@ -660,6 +905,12 @@ private[akka] class ActorCell( } } + final def receiveMessage(msg: Any): Unit = { + //FIXME replace with behaviorStack.head.applyOrElse(msg, unhandled) + "-optimize" + val head = behaviorStack.head + if (head.isDefinedAt(msg)) head.apply(msg) else actor.unhandled(msg) + } + private def doTerminate() { val a = actor try { @@ -671,11 +922,32 @@ private[akka] class ActorCell( } finally { try { parent.sendSystemMessage(ChildTerminated(self)) - system.deathWatch.publish(Terminated(self)) + + if (!watchedBy.isEmpty) { + val terminated = Terminated(self)(existenceConfirmed = true) + try { + watchedBy foreach { + watcher ⇒ + try watcher.tell(terminated, self) catch { + case NonFatal(t) ⇒ system.eventStream.publish(Error(t, self.path.toString, clazz(a), "deathwatch")) + } + } + } finally watchedBy = emptyActorRefSet + } + + if (!watching.isEmpty) { + try { + watching foreach { // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ + case watchee: InternalActorRef ⇒ try watchee.sendSystemMessage(Unwatch(watchee, self)) catch { + case NonFatal(t) ⇒ system.eventStream.publish(Error(t, self.path.toString, clazz(a), "deathwatch")) + } + } + } finally watching = emptyActorRefSet + } if (system.settings.DebugLifecycle) - system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped")) + system.eventStream.publish(Debug(self.path.toString, clazz(a), "stopped")) } finally { - if (a ne null) a.clearBehaviorStack() + behaviorStack = emptyBehaviorStack clearActorFields(a) actor = null } @@ -685,7 +957,6 @@ private[akka] class ActorCell( private def doRecreate(cause: Throwable, failedActor: Actor): Unit = try { // after all killed children have terminated, recreate the rest, then go on to start the new instance actor.supervisorStrategy.handleSupervisorRestarted(cause, self, children) - val freshActor = newActor() actor = freshActor // this must happen before postRestart has a chance to fail if (freshActor eq failedActor) setActorFields(freshActor, this, self) // If the creator returns the same instance, we need to restore our nulled out fields. @@ -699,10 +970,10 @@ private[akka] class ActorCell( dispatcher.reportFailure(new LogEventException(Error(e, self.path.toString, clazz(actor), "error while creating actor"), e)) // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) - actor.supervisorStrategy.handleSupervisorFailing(self, children) + actor.supervisorStrategy.handleSupervisorFailing(self, children) // FIXME Should this be called on actor or failedActor? clearActorFields(actor) // If this fails, we need to ensure that preRestart isn't called. } finally { - parent.tell(Failed(ActorInitializationException(self, "exception during re-creation", e)), self) + parent.tell(Failed(new ActorInitializationException(self, "exception during re-creation", e)), self) } } @@ -714,8 +985,7 @@ private[akka] class ActorCell( final def handleChildTerminated(child: ActorRef): Unit = try { childrenRefs match { case tc @ TerminatingChildrenContainer(_, _, reason) ⇒ - val n = tc.remove(child) - childrenRefs = n + val n = removeChild(child) actor.supervisorStrategy.handleChildTerminated(this, child, children) if (!n.isInstanceOf[TerminatingChildrenContainer]) reason match { case Recreation(cause) ⇒ doRecreate(cause, actor) // doRecreate since this is the continuation of "recreate" @@ -723,7 +993,7 @@ private[akka] class ActorCell( case _ ⇒ } case _ ⇒ - childrenRefs = childrenRefs.remove(child) + removeChild(child) actor.supervisorStrategy.handleChildTerminated(this, child, children) } } catch { @@ -736,6 +1006,11 @@ private[akka] class ActorCell( } } + protected def handleSupervise(child: ActorRef): Unit = child match { + case r: RepointableActorRef ⇒ r.activate() + case _ ⇒ + } + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ final def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause)) @@ -749,13 +1024,11 @@ private[akka] class ActorCell( } - final def cancelReceiveTimeout() { - //Only cancel if + final def cancelReceiveTimeout(): Unit = if (receiveTimeoutData._2 ne emptyCancellable) { receiveTimeoutData._2.cancel() receiveTimeoutData = (receiveTimeoutData._1, emptyCancellable) } - } final def clearActorFields(actorInstance: Actor): Unit = { setActorFields(actorInstance, context = null, self = system.deadLetters) diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index 4ebcec0dbb..1112b90f31 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -6,17 +6,6 @@ import scala.annotation.tailrec import java.net.MalformedURLException object ActorPath { - def split(s: String): List[String] = { - @tailrec - def rec(pos: Int, acc: List[String]): List[String] = { - val from = s.lastIndexOf('/', pos - 1) - val sub = s.substring(from + 1, pos) - val l = sub :: acc - if (from == -1) l else rec(from, l) - } - rec(s.length, Nil) - } - /** * Parse string as actor path; throws java.net.MalformedURLException if unable to do so. */ @@ -25,6 +14,11 @@ object ActorPath { case _ ⇒ throw new MalformedURLException("cannot parse as ActorPath: " + s) } + /** + * This Regular Expression is used to validate a path element (Actor Name). + * Since Actors form a tree, it is addressable using an URL, therefor an Actor Name has to conform to: + * http://www.ietf.org/rfc/rfc2396.txt + */ val ElementRegex = """[-\w:@&=+,.!~*'_;][-\w:@&=+,.!~*'$_;]*""".r } @@ -112,21 +106,21 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable { //TODO add @SerialVersionUID(1L) when SI-4804 is fixed final case class RootActorPath(address: Address, name: String = "/") extends ActorPath { - def parent: ActorPath = this + override def parent: ActorPath = this - def root: RootActorPath = this + override def root: RootActorPath = this - def /(child: String): ActorPath = new ChildActorPath(this, child) + override def /(child: String): ActorPath = new ChildActorPath(this, child) - val elements: Iterable[String] = List("") + override val elements: Iterable[String] = List("") - override val toString = address + name + override val toString: String = address + name - def toStringWithAddress(addr: Address): String = + override def toStringWithAddress(addr: Address): String = if (address.host.isDefined) address + name else addr + name - def compareTo(other: ActorPath) = other match { + override def compareTo(other: ActorPath): Int = other match { case r: RootActorPath ⇒ toString compareTo r.toString case c: ChildActorPath ⇒ 1 } @@ -136,11 +130,11 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act final class ChildActorPath(val parent: ActorPath, val name: String) extends ActorPath { if (name.indexOf('/') != -1) throw new IllegalArgumentException("/ is a path separator and is not legal in ActorPath names: [%s]" format name) - def address: Address = root.address + override def address: Address = root.address - def /(child: String): ActorPath = new ChildActorPath(this, child) + override def /(child: String): ActorPath = new ChildActorPath(this, child) - def elements: Iterable[String] = { + override def elements: Iterable[String] = { @tailrec def rec(p: ActorPath, acc: List[String]): Iterable[String] = p match { case r: RootActorPath ⇒ acc @@ -149,7 +143,7 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto rec(this, Nil) } - def root = { + override def root: RootActorPath = { @tailrec def rec(p: ActorPath): RootActorPath = p match { case r: RootActorPath ⇒ r @@ -198,7 +192,7 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto // TODO RK investigate Phil’s hash from scala.collection.mutable.HashTable.improve override def hashCode: Int = { - import scala.util.MurmurHash._ + import akka.routing.MurmurHash._ @tailrec def rec(p: ActorPath, h: Int, c: Int, k: Int): Int = p match { @@ -209,7 +203,7 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto finalizeHash(rec(this, startHash(42), startMagicA, startMagicB)) } - def compareTo(other: ActorPath) = { + override def compareTo(other: ActorPath): Int = { @tailrec def rec(left: ActorPath, right: ActorPath): Int = if (left eq right) 0 diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index b3d4ad19d1..8d42714b00 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -6,7 +6,6 @@ package akka.actor import akka.dispatch._ import akka.util._ -import scala.collection.immutable.Stack import java.lang.{ UnsupportedOperationException, IllegalStateException } import akka.serialization.{ Serialization, JavaSerializer } import akka.event.EventStream @@ -160,14 +159,28 @@ trait ScalaActorRef { ref: ActorRef ⇒ * often necessary to distinguish between local and non-local references, this * is the only method provided on the scope. */ -trait ActorRefScope { +private[akka] trait ActorRefScope { def isLocal: Boolean } -trait LocalRef extends ActorRefScope { +/** + * Refs which are statically known to be local inherit from this Scope + */ +private[akka] trait LocalRef extends ActorRefScope { final def isLocal = true } +/** + * RepointableActorRef (and potentially others) may change their locality at + * runtime, meaning that isLocal might not be stable. RepointableActorRef has + * the feature that it starts out “not fully started” (but you can send to it), + * which is why `isStarted` features here; it is not improbable that cluster + * actor refs will have the same behavior. + */ +private[akka] trait RepointableRef extends ActorRefScope { + def isStarted: Boolean +} + /** * Internal trait for assembling all the functionality needed internally on * ActorRefs. NOTE THAT THIS IS NOT A STABLE EXTERNAL INTERFACE! @@ -211,46 +224,51 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe def isLocal: Boolean } +/** + * Common trait of all actor refs which actually have a Cell, most notably + * LocalActorRef and RepointableActorRef. The former specializes the return + * type of `underlying` so that follow-up calls can use invokevirtual instead + * of invokeinterface. + */ +private[akka] abstract class ActorRefWithCell extends InternalActorRef { this: ActorRefScope ⇒ + def underlying: Cell +} + /** * This is an internal look-up failure token, not useful for anything else. */ private[akka] case object Nobody extends MinimalActorRef { - val path = new RootActorPath(Address("akka", "all-systems"), "/Nobody") - def provider = throw new UnsupportedOperationException("Nobody does not provide") + override val path: RootActorPath = new RootActorPath(Address("akka", "all-systems"), "/Nobody") + override def provider = throw new UnsupportedOperationException("Nobody does not provide") } /** * Local (serializable) ActorRef that is used when referencing the Actor on its "home" node. + * + * INTERNAL API */ private[akka] class LocalActorRef private[akka] ( _system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, - val path: ActorPath, - val systemService: Boolean = false, - _receiveTimeout: Option[Duration] = None) - extends InternalActorRef with LocalRef { + override val path: ActorPath) + extends ActorRefWithCell with LocalRef { /* - * actorCell.start() publishes actorCell & this to the dispatcher, which - * means that messages may be processed theoretically before the constructor - * ends. The JMM guarantees visibility for final fields only after the end - * of the constructor, so publish the actorCell safely by making it a - * @volatile var which is NOT TO BE WRITTEN TO. The alternative would be to - * move start() outside of the constructor, which would basically require - * us to use purely factory methods for creating LocalActorRefs. + * Safe publication of this class’s fields is guaranteed by mailbox.setActor() + * which is called indirectly from actorCell.start() (if you’re wondering why + * this is at all important, remember that under the JMM final fields are only + * frozen at the _end_ of the constructor, but we are publishing “this” before + * that is reached). */ - @volatile - private var actorCell = newActorCell(_system, this, _props, _supervisor, _receiveTimeout) + private val actorCell: ActorCell = newActorCell(_system, this, _props, _supervisor) actorCell.start() - protected def newActorCell( - system: ActorSystemImpl, - ref: InternalActorRef, - props: Props, - supervisor: InternalActorRef, - receiveTimeout: Option[Duration]): ActorCell = - new ActorCell(system, ref, props, supervisor, receiveTimeout) + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ + _supervisor.sendSystemMessage(akka.dispatch.Supervise(this)) + + protected def newActorCell(system: ActorSystemImpl, ref: InternalActorRef, props: Props, supervisor: InternalActorRef): ActorCell = + new ActorCell(system, ref, props, supervisor) protected def actorContext: ActorContext = actorCell @@ -268,21 +286,21 @@ private[akka] class LocalActorRef private[akka] ( * message sends done from the same thread after calling this method will not * be processed until resumed. */ - def suspend(): Unit = actorCell.suspend() + override def suspend(): Unit = actorCell.suspend() /** * Resumes a suspended actor. */ - def resume(): Unit = actorCell.resume() + override def resume(): Unit = actorCell.resume() /** * Shuts down the actor and its message queue */ - def stop(): Unit = actorCell.stop() + override def stop(): Unit = actorCell.stop() - def getParent: InternalActorRef = actorCell.parent + override def getParent: InternalActorRef = actorCell.parent - def provider = actorCell.provider + override def provider: ActorRefProvider = actorCell.provider /** * Method for looking up a single child beneath this actor. Override in order @@ -294,7 +312,7 @@ private[akka] class LocalActorRef private[akka] ( case None ⇒ Nobody } - def getChild(names: Iterator[String]): InternalActorRef = { + override def getChild(names: Iterator[String]): InternalActorRef = { /* * The idea is to recursively descend as far as possible with LocalActor * Refs and hand over to that “foreign” child when we encounter it. @@ -303,29 +321,29 @@ private[akka] class LocalActorRef private[akka] ( def rec(ref: InternalActorRef, name: Iterator[String]): InternalActorRef = ref match { case l: LocalActorRef ⇒ - val n = name.next() - val next = n match { + val next = name.next() match { case ".." ⇒ l.getParent case "" ⇒ l - case _ ⇒ l.getSingleChild(n) + case any ⇒ l.getSingleChild(any) } if (next == Nobody || name.isEmpty) next else rec(next, name) case _ ⇒ ref.getChild(name) } + if (names.isEmpty) this else rec(this, names) } // ========= AKKA PROTECTED FUNCTIONS ========= - protected[akka] def underlying: ActorCell = actorCell + def underlying: ActorCell = actorCell - def sendSystemMessage(message: SystemMessage) { underlying.dispatcher.systemDispatch(underlying, message) } + override def sendSystemMessage(message: SystemMessage): Unit = actorCell.sendSystemMessage(message) - def !(message: Any)(implicit sender: ActorRef = null): Unit = actorCell.tell(message, sender) + override def !(message: Any)(implicit sender: ActorRef = null): Unit = actorCell.tell(message, sender) - def restart(cause: Throwable): Unit = actorCell.restart(cause) + override def restart(cause: Throwable): Unit = actorCell.restart(cause) @throws(classOf[java.io.ObjectStreamException]) protected def writeReplace(): AnyRef = SerializedActorRef(path) @@ -333,9 +351,10 @@ private[akka] class LocalActorRef private[akka] ( /** * Memento pattern for serializing ActorRefs transparently + * INTERNAL API */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed -case class SerializedActorRef private (path: String) { +private[akka] case class SerializedActorRef private (path: String) { import akka.serialization.JavaSerializer.currentSystem @throws(classOf[java.io.ObjectStreamException]) @@ -349,7 +368,10 @@ case class SerializedActorRef private (path: String) { } } -object SerializedActorRef { +/** + * INTERNAL API + */ +private[akka] object SerializedActorRef { def apply(path: ActorPath): SerializedActorRef = { Serialization.currentTransportAddress.value match { case null ⇒ new SerializedActorRef(path.toString) @@ -360,33 +382,32 @@ object SerializedActorRef { /** * Trait for ActorRef implementations where all methods contain default stubs. + * + * INTERNAL API */ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef { - def getParent: InternalActorRef = Nobody + override def getParent: InternalActorRef = Nobody + override def getChild(names: Iterator[String]): InternalActorRef = if (names.forall(_.isEmpty)) this else Nobody - def getChild(names: Iterator[String]): InternalActorRef = { - val dropped = names.dropWhile(_.isEmpty) - if (dropped.isEmpty) this - else Nobody - } + override def suspend(): Unit = () + override def resume(): Unit = () + override def stop(): Unit = () + override def isTerminated = false - def suspend(): Unit = () - def resume(): Unit = () + override def !(message: Any)(implicit sender: ActorRef = null): Unit = () - def stop(): Unit = () - - def isTerminated = false - - def !(message: Any)(implicit sender: ActorRef = null): Unit = () - - def sendSystemMessage(message: SystemMessage): Unit = () - def restart(cause: Throwable): Unit = () + override def sendSystemMessage(message: SystemMessage): Unit = () + override def restart(cause: Throwable): Unit = () @throws(classOf[java.io.ObjectStreamException]) protected def writeReplace(): AnyRef = SerializedActorRef(path) } +/** + * When a message is sent to an Actor that is terminated before receiving the message, it will be sent as a DeadLetter + * to the ActorSystem's EventStream + */ case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) private[akka] object DeadLetterActorRef { @@ -402,30 +423,54 @@ private[akka] object DeadLetterActorRef { /** * This special dead letter reference has a name: it is that which is returned * by a local look-up which is unsuccessful. + * + * INTERNAL API */ -private[akka] class EmptyLocalActorRef( - val provider: ActorRefProvider, - val path: ActorPath, - val eventStream: EventStream) extends MinimalActorRef { +private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider, + override val path: ActorPath, + val eventStream: EventStream) extends MinimalActorRef { override def isTerminated(): Boolean = true + override def sendSystemMessage(message: SystemMessage): Unit = specialHandle(message) + override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match { - case d: DeadLetter ⇒ // do NOT form endless loops, since deadLetters will resend! - case _ ⇒ eventStream.publish(DeadLetter(message, sender, this)) + case d: DeadLetter ⇒ specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend! + case _ ⇒ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this)) + } + + protected def specialHandle(msg: Any): Boolean = msg match { + case w: Watch ⇒ + if (w.watchee == this && w.watcher != this) + w.watcher ! Terminated(w.watchee)(existenceConfirmed = false) + true + case _: Unwatch ⇒ true // Just ignore + case _ ⇒ false } } /** * Internal implementation of the dead letter destination: will publish any * received message to the eventStream, wrapped as [[akka.actor.DeadLetter]]. + * + * INTERNAL API */ -private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: ActorPath, _eventStream: EventStream) - extends EmptyLocalActorRef(_provider, _path, _eventStream) { +private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, + _path: ActorPath, + _eventStream: EventStream) extends EmptyLocalActorRef(_provider, _path, _eventStream) { override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match { - case d: DeadLetter ⇒ eventStream.publish(d) - case _ ⇒ eventStream.publish(DeadLetter(message, sender, this)) + case d: DeadLetter ⇒ if (!specialHandle(d.message)) eventStream.publish(d) + case _ ⇒ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this)) + } + + override protected def specialHandle(msg: Any): Boolean = msg match { + case w: Watch ⇒ + if (w.watchee != this && w.watcher != this) + w.watcher ! Terminated(w.watchee)(existenceConfirmed = false) + true + case w: Unwatch ⇒ true // Just ignore + case _ ⇒ false } @throws(classOf[java.io.ObjectStreamException]) @@ -434,10 +479,12 @@ private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: Actor /** * Internal implementation detail used for paths like “/temp” + * + * INTERNAL API */ private[akka] class VirtualPathContainer( - val provider: ActorRefProvider, - val path: ActorPath, + override val provider: ActorRefProvider, + override val path: ActorPath, override val getParent: InternalActorRef, val log: LoggingAdapter) extends MinimalActorRef { @@ -450,12 +497,8 @@ private[akka] class VirtualPathContainer( } } - def removeChild(name: String): Unit = { - children.remove(name) match { - case null ⇒ log.warning("{} trying to remove non-child {}", path, name) - case _ ⇒ //okay - } - } + def removeChild(name: String): Unit = + if (children.remove(name) eq null) log.warning("{} trying to remove non-child {}", path, name) def getChild(name: String): InternalActorRef = children.get(name) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 536136934a..bbb84144c5 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -8,8 +8,8 @@ import java.util.concurrent.atomic.AtomicLong import akka.dispatch._ import akka.routing._ import akka.AkkaException -import akka.util.{ Switch, Helpers } import akka.event._ +import akka.util.{ NonFatal, Switch, Helpers } /** * Interface for all ActorRef providers to implement. @@ -26,31 +26,30 @@ trait ActorRefProvider { /** * Reference to the supervisor used for all top-level user actors. */ - def guardian: InternalActorRef + def guardian: LocalActorRef /** * Reference to the supervisor used for all top-level system actors. */ - def systemGuardian: InternalActorRef + def systemGuardian: LocalActorRef /** * Dead letter destination for this provider. */ def deadLetters: ActorRef - /** - * Reference to the death watch service. - */ - def deathWatch: DeathWatch - /** * The root path for all actors within this actor system, including remote * address if enabled. */ def rootPath: ActorPath + /** + * The Settings associated with this ActorRefProvider + */ def settings: ActorSystem.Settings + //FIXME WHY IS THIS HERE? def dispatcher: MessageDispatcher /** @@ -61,8 +60,12 @@ trait ActorRefProvider { */ def init(system: ActorSystemImpl): Unit + /** + * The Deployer associated with this ActorRefProvider + */ def deployer: Deployer + //FIXME WHY IS THIS HERE? def scheduler: Scheduler /** @@ -101,7 +104,8 @@ trait ActorRefProvider { path: ActorPath, systemService: Boolean, deploy: Option[Deploy], - lookupDeploy: Boolean): InternalActorRef + lookupDeploy: Boolean, + async: Boolean): InternalActorRef /** * Create actor reference for a specified local or remote path. If no such @@ -131,6 +135,7 @@ trait ActorRefProvider { */ def terminationFuture: Future[Unit] + //FIXME I PROPOSE TO REMOVE THIS IN 2.1 - √ /** * Obtain the address which is to be used within sender references when * sending to the given other address or none if the other address cannot be @@ -141,22 +146,34 @@ trait ActorRefProvider { } /** - * Interface implemented by ActorSystem and AkkaContext, the only two places + * Interface implemented by ActorSystem and ActorContext, the only two places * from which you can get fresh actors. */ trait ActorRefFactory { - + /** + * INTERNAL USE ONLY + */ protected def systemImpl: ActorSystemImpl - + /** + * INTERNAL USE ONLY + */ protected def provider: ActorRefProvider - protected def dispatcher: MessageDispatcher + /** + * Returns the default MessageDispatcher associated with this ActorRefFactory + */ + implicit def dispatcher: MessageDispatcher /** * Father of all children created by this interface. + * + * INTERNAL USE ONLY */ protected def guardian: InternalActorRef + /** + * INTERNAL USE ONLY + */ protected def lookupRoot: InternalActorRef /** @@ -276,8 +293,6 @@ trait ActorRefFactory { def stop(actor: ActorRef): Unit } -class ActorRefProviderException(message: String) extends AkkaException(message) - /** * Internal Akka use only, used in implementation of system.actorOf. */ @@ -298,10 +313,10 @@ private[akka] case class StopChild(child: ActorRef) */ class LocalActorRefProvider( _systemName: String, - val settings: ActorSystem.Settings, + override val settings: ActorSystem.Settings, val eventStream: EventStream, - val scheduler: Scheduler, - val deployer: Deployer) extends ActorRefProvider { + override val scheduler: Scheduler, + override val deployer: Deployer) extends ActorRefProvider { // this is the constructor needed for reflectively instantiating the provider def this(_systemName: String, @@ -315,13 +330,11 @@ class LocalActorRefProvider( scheduler, new Deployer(settings, dynamicAccess)) - val rootPath: ActorPath = RootActorPath(Address("akka", _systemName)) + override val rootPath: ActorPath = RootActorPath(Address("akka", _systemName)) - val log = Logging(eventStream, "LocalActorRefProvider(" + rootPath.address + ")") + private[akka] val log: LoggingAdapter = Logging(eventStream, "LocalActorRefProvider(" + rootPath.address + ")") - val deadLetters = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream) - - val deathWatch = new LocalDeathWatch(1024) //TODO make configrable + override val deadLetters: InternalActorRef = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream) /* * generate name for temporary actor refs @@ -332,7 +345,7 @@ class LocalActorRefProvider( private val tempNode = rootPath / "temp" - def tempPath() = tempNode / tempName() + override def tempPath(): ActorPath = tempNode / tempName() /** * Top-level anchor for the supervision hierarchy of this actor system. Will @@ -348,11 +361,11 @@ class LocalActorRefProvider( def provider: ActorRefProvider = LocalActorRefProvider.this - override def stop() = stopped switchOn { + override def stop(): Unit = stopped switchOn { terminationFuture.complete(causeOfTermination.toLeft(())) } - override def isTerminated = stopped.isOn + override def isTerminated: Boolean = stopped.isOn override def !(message: Any)(implicit sender: ActorRef = null): Unit = stopped.ifOff(message match { case Failed(ex) if sender ne null ⇒ causeOfTermination = Some(ex); sender.asInstanceOf[InternalActorRef].stop() @@ -361,9 +374,9 @@ class LocalActorRefProvider( override def sendSystemMessage(message: SystemMessage): Unit = stopped ifOff { message match { - case Supervise(child) ⇒ // TODO register child in some map to keep track of it and enable shutdown after all dead - case ChildTerminated(child) ⇒ stop() - case _ ⇒ log.error(this + " received unexpected system message [" + message + "]") + case Supervise(_) ⇒ // TODO register child in some map to keep track of it and enable shutdown after all dead + case ChildTerminated(_) ⇒ stop() + case _ ⇒ log.error(this + " received unexpected system message [" + message + "]") } } } @@ -371,7 +384,7 @@ class LocalActorRefProvider( /** * Overridable supervision strategy to be used by the “/user” guardian. */ - protected def guardianSupervisionStrategy = { + protected def guardianSupervisionStrategy: SupervisorStrategy = { import akka.actor.SupervisorStrategy._ OneForOneStrategy() { case _: ActorKilledException ⇒ Stop @@ -387,12 +400,12 @@ class LocalActorRefProvider( */ private class Guardian extends Actor { - override val supervisorStrategy = guardianSupervisionStrategy + override val supervisorStrategy: SupervisorStrategy = guardianSupervisionStrategy def receive = { case Terminated(_) ⇒ context.stop(self) - case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) - case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e }) + case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case NonFatal(e) ⇒ Status.Failure(e) }) + case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case NonFatal(e) ⇒ Status.Failure(e) }) case StopChild(child) ⇒ context.stop(child); sender ! "ok" case m ⇒ deadLetters ! DeadLetter(m, sender, self) } @@ -404,12 +417,11 @@ class LocalActorRefProvider( /** * Overridable supervision strategy to be used by the “/system” guardian. */ - protected def systemGuardianSupervisionStrategy = { + protected def systemGuardianSupervisionStrategy: SupervisorStrategy = { import akka.actor.SupervisorStrategy._ OneForOneStrategy() { - case _: ActorKilledException ⇒ Stop - case _: ActorInitializationException ⇒ Stop - case _: Exception ⇒ Restart + case _: ActorKilledException | _: ActorInitializationException ⇒ Stop + case _: Exception ⇒ Restart } } @@ -420,14 +432,12 @@ class LocalActorRefProvider( */ private class SystemGuardian extends Actor { - override val supervisorStrategy = systemGuardianSupervisionStrategy + override val supervisorStrategy: SupervisorStrategy = systemGuardianSupervisionStrategy def receive = { - case Terminated(_) ⇒ - eventStream.stopDefaultLoggers() - context.stop(self) - case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) - case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e }) + case Terminated(_) ⇒ eventStream.stopDefaultLoggers(); context.stop(self) + case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case NonFatal(e) ⇒ Status.Failure(e) }) + case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case NonFatal(e) ⇒ Status.Failure(e) }) case StopChild(child) ⇒ context.stop(child); sender ! "ok" case m ⇒ deadLetters ! DeadLetter(m, sender, self) } @@ -464,27 +474,18 @@ class LocalActorRefProvider( private val guardianProps = Props(new Guardian) lazy val rootGuardian: InternalActorRef = - new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) { - object Extra { - def unapply(s: String): Option[InternalActorRef] = extraNames.get(s) - } - + new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath) { override def getParent: InternalActorRef = this - - override def getSingleChild(name: String): InternalActorRef = { - name match { - case "temp" ⇒ tempContainer - case Extra(e) ⇒ e - case _ ⇒ super.getSingleChild(name) - } + override def getSingleChild(name: String): InternalActorRef = name match { + case "temp" ⇒ tempContainer + case other ⇒ extraNames.get(other).getOrElse(super.getSingleChild(other)) } } - lazy val guardian: InternalActorRef = - actorOf(system, guardianProps, rootGuardian, rootPath / "user", true, None, false) + lazy val guardian: LocalActorRef = new LocalActorRef(system, guardianProps, rootGuardian, rootPath / "user") - lazy val systemGuardian: InternalActorRef = - actorOf(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, rootPath / "system", true, None, false) + lazy val systemGuardian: LocalActorRef = + new LocalActorRef(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, rootPath / "system") lazy val tempContainer = new VirtualPathContainer(system.provider, tempNode, rootGuardian, log) @@ -501,8 +502,8 @@ class LocalActorRefProvider( def init(_system: ActorSystemImpl) { system = _system // chain death watchers so that killing guardian stops the application - deathWatch.subscribe(systemGuardian, guardian) - deathWatch.subscribe(rootGuardian, systemGuardian) + systemGuardian.sendSystemMessage(Watch(guardian, systemGuardian)) + rootGuardian.sendSystemMessage(Watch(systemGuardian, rootGuardian)) eventStream.startDefaultLoggers(_system) } @@ -538,32 +539,20 @@ class LocalActorRefProvider( } def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, - systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean): InternalActorRef = { + systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = { props.routerConfig match { - case NoRouter ⇒ new LocalActorRef(system, props, supervisor, path, systemService) // create a local actor + case NoRouter ⇒ + if (async) new RepointableActorRef(system, props, supervisor, path).initialize() + else new LocalActorRef(system, props, supervisor, path) case router ⇒ val lookup = if (lookupDeploy) deployer.lookup(path) else None val fromProps = Iterator(props.deploy.copy(routerConfig = props.deploy.routerConfig withFallback router)) val d = fromProps ++ deploy.iterator ++ lookup.iterator reduce ((a, b) ⇒ b withFallback a) - new RoutedActorRef(system, props.withRouter(d.routerConfig), supervisor, path) + val ref = new RoutedActorRef(system, props.withRouter(d.routerConfig), supervisor, path).initialize() + if (async) ref else ref.activate() } } def getExternalAddressFor(addr: Address): Option[Address] = if (addr == rootPath.address) Some(addr) else None } -class LocalDeathWatch(val mapSize: Int) extends DeathWatch with ActorClassification { - - override def publish(event: Event): Unit = { - val monitors = dissociate(classify(event)) - if (monitors.nonEmpty) monitors.foreach(_ ! event) - } - - override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = { - if (!super.subscribe(subscriber, to)) { - subscriber ! Terminated(to) - false - } else true - } -} - diff --git a/akka-actor/src/main/scala/akka/actor/ActorSelection.scala b/akka-actor/src/main/scala/akka/actor/ActorSelection.scala index b407868270..93a26c6289 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSelection.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSelection.scala @@ -5,6 +5,10 @@ package akka.actor import java.util.regex.Pattern import akka.util.Helpers +/** + * An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors, + * allowing for broadcasting of messages to that section. + */ abstract class ActorSelection { this: ScalaActorSelection ⇒ @@ -12,11 +16,11 @@ abstract class ActorSelection { protected def path: Array[AnyRef] - def tell(msg: Any) { target ! toMessage(msg, path) } + def tell(msg: Any): Unit = target ! toMessage(msg, path) - def tell(msg: Any, sender: ActorRef) { target.tell(toMessage(msg, path), sender) } + def tell(msg: Any, sender: ActorRef): Unit = target.tell(toMessage(msg, path), sender) - // this may want to be fast ... + // FIXME make this so that "next" instead is the remaining path private def toMessage(msg: Any, path: Array[AnyRef]): Any = { var acc = msg var index = path.length - 1 @@ -32,7 +36,12 @@ abstract class ActorSelection { } } +/** + * An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors, + * allowing for broadcasting of messages to that section. + */ object ActorSelection { + //This cast is safe because the self-type of ActorSelection requires that it mixes in ScalaActorSelection implicit def toScala(sel: ActorSelection): ScalaActorSelection = sel.asInstanceOf[ScalaActorSelection] /** @@ -43,7 +52,7 @@ object ActorSelection { */ def apply(anchor: ActorRef, path: String): ActorSelection = { val elems = path.split("/+").dropWhile(_.isEmpty) - val compiled: Array[AnyRef] = elems map (x ⇒ if (x.contains("?") || x.contains("*")) Helpers.makePattern(x) else x) + val compiled: Array[AnyRef] = elems map (x ⇒ if ((x.indexOf('?') != -1) || (x.indexOf('*') != -1)) Helpers.makePattern(x) else x) new ActorSelection with ScalaActorSelection { def target = anchor def path = compiled @@ -51,6 +60,10 @@ object ActorSelection { } } +/** + * Contains the Scala API (!-method) for ActorSelections) which provides automatic tracking of the sender, + * as per the usual implicit ActorRef pattern. + */ trait ScalaActorSelection { this: ActorSelection ⇒ diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index b84057b749..261a6b3c58 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -4,38 +4,33 @@ package akka.actor -import akka.config.ConfigurationException import akka.event._ import akka.dispatch._ import akka.pattern.ask -import org.jboss.netty.akka.util.HashedWheelTimer -import java.util.concurrent.TimeUnit.MILLISECONDS -import com.typesafe.config.Config -import com.typesafe.config.ConfigFactory +import com.typesafe.config.{ Config, ConfigFactory } import scala.annotation.tailrec -import org.jboss.netty.akka.util.internal.ConcurrentIdentityHashMap import java.io.Closeable -import akka.dispatch.Await.Awaitable -import akka.dispatch.Await.CanAwait +import akka.dispatch.Await.{ Awaitable, CanAwait } import akka.util._ -import collection.immutable.Stack +import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap } import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException } +import java.util.concurrent.TimeUnit.MILLISECONDS object ActorSystem { - val Version = "2.1-SNAPSHOT" + val Version: String = "2.1-SNAPSHOT" - val EnvHome = System.getenv("AKKA_HOME") match { + val EnvHome: Option[String] = System.getenv("AKKA_HOME") match { case null | "" | "." ⇒ None case value ⇒ Some(value) } - val SystemHome = System.getProperty("akka.home") match { + val SystemHome: Option[String] = System.getProperty("akka.home") match { case null | "" ⇒ None case value ⇒ Some(value) } - val GlobalHome = SystemHome orElse EnvHome + val GlobalHome: Option[String] = SystemHome orElse EnvHome /** * Creates a new ActorSystem with the name "default", @@ -60,11 +55,15 @@ object ActorSystem { * obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader, * then tries to walk the stack to find the callers class loader, then falls back to the ClassLoader * associated with the ActorSystem class. + * + * @see The Typesafe Config Library API Documentation */ def create(name: String, config: Config): ActorSystem = apply(name, config) /** * Creates a new ActorSystem with the name "default", the specified Config, and specified ClassLoader + * + * @see The Typesafe Config Library API Documentation */ def create(name: String, config: Config, classLoader: ClassLoader): ActorSystem = apply(name, config, classLoader) @@ -94,16 +93,32 @@ object ActorSystem { * obtains the current ClassLoader by first inspecting the current threads' getContextClassLoader, * then tries to walk the stack to find the callers class loader, then falls back to the ClassLoader * associated with the ActorSystem class. + * + * @see The Typesafe Config Library API Documentation */ def apply(name: String, config: Config): ActorSystem = apply(name, config, findClassLoader()) /** * Creates a new ActorSystem with the name "default", the specified Config, and specified ClassLoader + * + * @see The Typesafe Config Library API Documentation */ def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = new ActorSystemImpl(name, config, classLoader).start() + /** + * Settings are the overall ActorSystem Settings which also provides a convenient access to the Config object. + * + * For more detailed information about the different possible configuration options, look in the Akka Documentation under "Configuration" + * + * @see The Typesafe Config Library API Documentation + */ class Settings(classLoader: ClassLoader, cfg: Config, final val name: String) { + /** + * The backing Config of this ActorSystem's Settings + * + * @see The Typesafe Config Library API Documentation + */ final val config: Config = { val config = cfg.withFallback(ConfigFactory.defaultReference(classLoader)) config.checkValid(ConfigFactory.defaultReference(classLoader), "akka") @@ -114,11 +129,9 @@ object ActorSystem { import config._ final val ConfigVersion = getString("akka.version") - final val ProviderClass = getString("akka.actor.provider") - final val CreationTimeout = Timeout(Duration(getMilliseconds("akka.actor.creation-timeout"), MILLISECONDS)) - final val ReaperInterval = Duration(getMilliseconds("akka.actor.reaper-interval"), MILLISECONDS) + final val SerializeAllMessages = getBoolean("akka.actor.serialize-messages") final val SerializeAllCreators = getBoolean("akka.actor.serialize-creators") @@ -146,13 +159,16 @@ object ActorSystem { final val JvmExitOnFatalError = getBoolean("akka.jvm-exit-on-fatal-error") if (ConfigVersion != Version) - throw new ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]") + throw new akka.ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]") + /** + * Returns the String representation of the Config that this Settings is backed by + */ override def toString: String = config.root.render } /** - * INTERNAL + * INTERNAL USE ONLY */ private[akka] def findClassLoader(): ClassLoader = { def findCaller(get: Int ⇒ Class[_]): ClassLoader = @@ -295,12 +311,14 @@ abstract class ActorSystem extends ActorRefFactory { * Default dispatcher as configured. This dispatcher is used for all actors * in the actor system which do not have a different dispatcher configured * explicitly. + * Importing this member will place the default MessageDispatcher in scope. */ - def dispatcher: MessageDispatcher + implicit def dispatcher: MessageDispatcher /** - * Register a block of code (callback) to run after all actors in this actor system have - * been stopped. Multiple code blocks may be registered by calling this method multiple times. + * Register a block of code (callback) to run after ActorSystem.shutdown has been issued and + * all actors in this actor system have been stopped. + * Multiple code blocks may be registered by calling this method multiple times. * The callbacks will be run sequentially in reverse order of registration, i.e. * last registration is run first. * @@ -311,8 +329,9 @@ abstract class ActorSystem extends ActorRefFactory { def registerOnTermination[T](code: ⇒ T): Unit /** - * Register a block of code (callback) to run after all actors in this actor system have - * been stopped. Multiple code blocks may be registered by calling this method multiple times. + * Register a block of code (callback) to run after ActorSystem.shutdown has been issued and + * all actors in this actor system have been stopped. + * Multiple code blocks may be registered by calling this method multiple times. * The callbacks will be run sequentially in reverse order of registration, i.e. * last registration is run first. * @@ -402,11 +421,6 @@ abstract class ExtendedActorSystem extends ActorSystem { */ def systemGuardian: InternalActorRef - /** - * Implementation of the mechanism which is used for watch()/unwatch(). - */ - def deathWatch: DeathWatch - /** * A ThreadFactory that can be used if the transport needs to create any Threads */ @@ -420,14 +434,21 @@ abstract class ExtendedActorSystem extends ActorSystem { * creation. */ def dynamicAccess: DynamicAccess + + /** + * For debugging: traverse actor hierarchy and make string representation. + * Careful, this may OOM on large actor systems, and it is only meant for + * helping debugging in case something already went terminally wrong. + */ + private[akka] def printTree: String } -class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Config, classLoader: ClassLoader) extends ExtendedActorSystem { +private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, classLoader: ClassLoader) extends ExtendedActorSystem { if (!name.matches("""^[a-zA-Z0-9][a-zA-Z0-9-]*$""")) throw new IllegalArgumentException( "invalid ActorSystem name [" + name + - "], must contain only word characters (i.e. [a-zA-Z_0-9] plus non-leading '-')") + "], must contain only word characters (i.e. [a-zA-Z0-9] plus non-leading '-')") import ActorSystem._ @@ -475,31 +496,13 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf def logConfiguration(): Unit = log.info(settings.toString) - protected def systemImpl = this + protected def systemImpl: ActorSystemImpl = this - private[akka] def systemActorOf(props: Props, name: String): ActorRef = { - implicit val timeout = settings.CreationTimeout - Await.result(systemGuardian ? CreateChild(props, name), timeout.duration) match { - case ref: ActorRef ⇒ ref - case ex: Exception ⇒ throw ex - } - } + private[akka] def systemActorOf(props: Props, name: String): ActorRef = systemGuardian.underlying.attachChild(props, name) - def actorOf(props: Props, name: String): ActorRef = { - implicit val timeout = settings.CreationTimeout - Await.result(guardian ? CreateChild(props, name), timeout.duration) match { - case ref: ActorRef ⇒ ref - case ex: Exception ⇒ throw ex - } - } + def actorOf(props: Props, name: String): ActorRef = guardian.underlying.attachChild(props, name) - def actorOf(props: Props): ActorRef = { - implicit val timeout = settings.CreationTimeout - Await.result(guardian ? CreateRandomNameChild(props), timeout.duration) match { - case ref: ActorRef ⇒ ref - case ex: Exception ⇒ throw ex - } - } + def actorOf(props: Props): ActorRef = guardian.underlying.attachChild(props) def stop(actor: ActorRef): Unit = { implicit val timeout = settings.CreationTimeout @@ -539,18 +542,21 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf def deadLetters: ActorRef = provider.deadLetters + //FIXME Why do we need this at all? val deadLetterQueue: MessageQueue = new MessageQueue { - def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } + def enqueue(receiver: ActorRef, envelope: Envelope): Unit = + deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) def dequeue() = null def hasMessages = false def numberOfMessages = 0 - def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = () + def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = () } - - val deadLetterMailbox: Mailbox = new Mailbox(null, deadLetterQueue) { + //FIXME Why do we need this at all? + val deadLetterMailbox: Mailbox = new Mailbox(deadLetterQueue) { becomeClosed() - def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit = deadLetters ! DeadLetter(handle, receiver, receiver) - def systemDrain(): SystemMessage = null + def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit = + deadLetters ! DeadLetter(handle, receiver, receiver) + def systemDrain(newContents: SystemMessage): SystemMessage = null def hasSystemMessages = false } @@ -561,9 +567,8 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf def terminationFuture: Future[Unit] = provider.terminationFuture def lookupRoot: InternalActorRef = provider.rootGuardian - def guardian: InternalActorRef = provider.guardian - def systemGuardian: InternalActorRef = provider.systemGuardian - def deathWatch: DeathWatch = provider.deathWatch + def guardian: LocalActorRef = provider.guardian + def systemGuardian: LocalActorRef = provider.systemGuardian def /(actorName: String): ActorPath = guardian.path / actorName def /(path: Iterable[String]): ActorPath = guardian.path / path @@ -687,10 +692,35 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf override def toString: String = lookupRoot.path.root.address.toString + override def printTree: String = { + def printNode(node: ActorRef, indent: String): String = { + node match { + case wc: ActorRefWithCell ⇒ + val cell = wc.underlying + indent + "-> " + node.path.name + " " + Logging.simpleName(node) + " " + + (cell match { + case real: ActorCell ⇒ if (real.actor ne null) real.actor.getClass else "null" + case _ ⇒ Logging.simpleName(cell) + }) + + " " + (cell.childrenRefs match { + case ActorCell.TerminatingChildrenContainer(_, toDie, reason) ⇒ + "Terminating(" + reason + ")" + + (toDie.toSeq.sorted mkString ("\n" + indent + " toDie: ", "\n" + indent + " ", "")) + case x ⇒ Logging.simpleName(x) + }) + + (if (cell.childrenRefs.children.isEmpty) "" else "\n") + + (cell.childrenRefs.children.toSeq.sorted map (printNode(_, indent + " |")) mkString ("\n")) + case _ ⇒ + indent + node.path.name + " " + Logging.simpleName(node) + } + } + printNode(actorFor("/"), "") + } + final class TerminationCallbacks extends Runnable with Awaitable[Unit] { private val lock = new ReentrantGuard - private var callbacks: Stack[Runnable] = _ //non-volatile since guarded by the lock - lock withGuard { callbacks = Stack.empty[Runnable] } + private var callbacks: List[Runnable] = _ //non-volatile since guarded by the lock + lock withGuard { callbacks = Nil } private val latch = new CountDownLatch(1) @@ -699,17 +729,17 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf case 0 ⇒ throw new RejectedExecutionException("Must be called prior to system shutdown.") case _ ⇒ lock withGuard { if (latch.getCount == 0) throw new RejectedExecutionException("Must be called prior to system shutdown.") - else callbacks = callbacks.push(callback) + else callbacks ::= callback } } } final def run(): Unit = lock withGuard { - @tailrec def runNext(c: Stack[Runnable]): Stack[Runnable] = c.headOption match { - case None ⇒ Stack.empty[Runnable] - case Some(callback) ⇒ - try callback.run() catch { case e ⇒ log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) } - runNext(c.pop) + @tailrec def runNext(c: List[Runnable]): List[Runnable] = c match { + case Nil ⇒ Nil + case callback :: rest ⇒ + try callback.run() catch { case NonFatal(e) ⇒ log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) } + runNext(rest) } try { callbacks = runNext(callbacks) } finally latch.countDown() } diff --git a/akka-actor/src/main/scala/akka/actor/Address.scala b/akka-actor/src/main/scala/akka/actor/Address.scala index 651d970885..44c12eed35 100644 --- a/akka-actor/src/main/scala/akka/actor/Address.scala +++ b/akka-actor/src/main/scala/akka/actor/Address.scala @@ -5,6 +5,7 @@ package akka.actor import java.net.URI import java.net.URISyntaxException import java.net.MalformedURLException +import annotation.tailrec /** * The address specifies the physical location under which an Actor can be @@ -20,36 +21,62 @@ final case class Address private (protocol: String, system: String, host: Option def this(protocol: String, system: String) = this(protocol, system, None, None) def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port)) + /** + * Returns the canonical String representation of this Address formatted as: + * + * ://@: + */ @transient override lazy val toString: String = { - val sb = new StringBuilder(protocol) - sb.append("://") - sb.append(system) - if (host.isDefined) { - sb.append('@') - sb.append(host.get) - } - if (port.isDefined) { - sb.append(':') - sb.append(port.get) - } + val sb = (new StringBuilder(protocol)).append("://").append(system) + + if (host.isDefined) sb.append('@').append(host.get) + if (port.isDefined) sb.append(':').append(port.get) + sb.toString } - def hostPort: String = toString.substring(protocol.length() + 3) + /** + * Returns a String representation formatted as: + * + * @: + */ + def hostPort: String = toString.substring(protocol.length + 3) } object Address { + /** + * Constructs a new Address with the specified protocol and system name + */ def apply(protocol: String, system: String) = new Address(protocol, system) + + /** + * Constructs a new Address with the specified protocol, system name, host and port + */ def apply(protocol: String, system: String, host: String, port: Int) = new Address(protocol, system, Some(host), Some(port)) } -object RelativeActorPath { +private[akka] trait PathUtils { + protected def split(s: String): List[String] = { + @tailrec + def rec(pos: Int, acc: List[String]): List[String] = { + val from = s.lastIndexOf('/', pos - 1) + val sub = s.substring(from + 1, pos) + val l = sub :: acc + if (from == -1) l else rec(from, l) + } + rec(s.length, Nil) + } +} + +object RelativeActorPath extends PathUtils { def unapply(addr: String): Option[Iterable[String]] = { try { val uri = new URI(addr) if (uri.isAbsolute) None - else Some(ActorPath.split(uri.getPath)) + else Some(split(uri.getPath)) + } catch { + case _: URISyntaxException ⇒ None } } } @@ -58,13 +85,7 @@ object RelativeActorPath { * This object serves as extractor for Scala and as address parser for Java. */ object AddressFromURIString { - def unapply(addr: String): Option[Address] = - try { - val uri = new URI(addr) - unapply(uri) - } catch { - case _: URISyntaxException ⇒ None - } + def unapply(addr: String): Option[Address] = try unapply(new URI(addr)) catch { case _: URISyntaxException ⇒ None } def unapply(uri: URI): Option[Address] = if (uri eq null) None @@ -84,7 +105,7 @@ object AddressFromURIString { */ def apply(addr: String): Address = addr match { case AddressFromURIString(address) ⇒ address - case _ ⇒ throw new MalformedURLException + case _ ⇒ throw new MalformedURLException(addr) } /** @@ -93,14 +114,17 @@ object AddressFromURIString { def parse(addr: String): Address = apply(addr) } -object ActorPathExtractor { +/** + * Given an ActorPath it returns the Address and the path elements if the path is well-formed + */ +object ActorPathExtractor extends PathUtils { def unapply(addr: String): Option[(Address, Iterable[String])] = try { val uri = new URI(addr) if (uri.getPath == null) None else AddressFromURIString.unapply(uri) match { case None ⇒ None - case Some(addr) ⇒ Some((addr, ActorPath.split(uri.getPath).drop(1))) + case Some(addr) ⇒ Some((addr, split(uri.getPath).drop(1))) } } catch { case _: URISyntaxException ⇒ None diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 2fd9538d77..30118ae03a 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -34,8 +34,19 @@ final case class Deploy( routerConfig: RouterConfig = NoRouter, scope: Scope = NoScopeGiven) { + /** + * Java API to create a Deploy with the given RouterConfig + */ def this(routing: RouterConfig) = this("", ConfigFactory.empty, routing) + + /** + * Java API to create a Deploy with the given RouterConfig with Scope + */ def this(routing: RouterConfig, scope: Scope) = this("", ConfigFactory.empty, routing, scope) + + /** + * Java API to create a Deploy with the given Scope + */ def this(scope: Scope) = this("", ConfigFactory.empty, NoRouter, scope) /** @@ -67,13 +78,9 @@ trait Scope { //TODO add @SerialVersionUID(1L) when SI-4804 is fixed abstract class LocalScope extends Scope -case object LocalScope extends LocalScope { - /** - * Java API - */ - @deprecated("use instance() method instead", "2.0.1") - def scope: Scope = this +//FIXME docs +case object LocalScope extends LocalScope { /** * Java API: get the singleton instance */ @@ -128,7 +135,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce add(d.path.split("/").drop(1), d) } - protected def parseConfig(key: String, config: Config): Option[Deploy] = { + def parseConfig(key: String, config: Config): Option[Deploy] = { val deployment = config.withFallback(default) @@ -162,5 +169,4 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce Some(Deploy(key, deployment, router, NoScopeGiven)) } - } diff --git a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala index 8d3ac68852..72ffbbe76e 100644 --- a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala +++ b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala @@ -14,7 +14,7 @@ import java.lang.reflect.InvocationTargetException * This is an internal facility and users are not expected to encounter it * unless they are extending Akka in ways which go beyond simple Extensions. */ -trait DynamicAccess { +abstract class DynamicAccess { /** * Convenience method which given a `Class[_]` object and a constructor description @@ -88,7 +88,7 @@ trait DynamicAccess { * by default. */ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAccess { - + //FIXME switch to Scala Reflection for 2.10 override def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] = try { val c = classLoader.loadClass(fqcn).asInstanceOf[Class[_ <: T]] diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index 3d1f8930c4..50c769e156 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -6,30 +6,86 @@ package akka.actor import akka.util._ import scala.collection.mutable -import akka.event.Logging import akka.routing.{ Deafen, Listen, Listeners } object FSM { + /** + * A partial function value which does not match anything and can be used to + * “reset” `whenUnhandled` and `onTermination` handlers. + * + * {{{ + * onTermination(FSM.NullFunction) + * }}} + */ object NullFunction extends PartialFunction[Any, Nothing] { def isDefinedAt(o: Any) = false def apply(o: Any) = sys.error("undefined") } + /** + * Message type which is sent directly to the subscribed actor in + * [[akka.actor.FSM.SubscribeTransitionCallback]] before sending any + * [[akka.actor.FSM.Transition]] messages. + */ case class CurrentState[S](fsmRef: ActorRef, state: S) + + /** + * Message type which is used to communicate transitions between states to + * all subscribed listeners (use [[akka.actor.FSM.SubscribeTransitionCallback]]). + */ case class Transition[S](fsmRef: ActorRef, from: S, to: S) + + /** + * Send this to an [[akka.actor.FSM]] to request first the [[akka.actor.CurrentState]] + * and then a series of [[akka.actor.Transition]] updates. Cancel the subscription + * using [[akka.actor.FSM.UnsubscribeTransitionCallback]]. + */ case class SubscribeTransitionCallBack(actorRef: ActorRef) + + /** + * Unsubscribe from [[akka.actor.FSM.Transition]] notifications which was + * effected by sending the corresponding [[akka.actor.FSM.SubscribeTransitionCallback]]. + */ case class UnsubscribeTransitionCallBack(actorRef: ActorRef) + /** + * Reason why this [[akka.actor.FSM]] is shutting down. + */ sealed trait Reason + + /** + * Default reason if calling `stop()`. + */ case object Normal extends Reason + + /** + * Reason given when someone was calling `system.stop(fsm)` from outside; + * also applies to `Stop` supervision directive. + */ case object Shutdown extends Reason + + /** + * Signifies that the [[akka.actor.FSM]] is shutting itself down because of + * an error, e.g. if the state to transition into does not exist. You can use + * this to communicate a more precise cause to the [[akka.actor.FSM$onTermination]] block. + */ case class Failure(cause: Any) extends Reason + /** + * This case object is received in case of a state timeout. + */ case object StateTimeout - case class TimeoutMarker(generation: Long) - case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit system: ActorSystem) { + /** + * Internal API + */ + private case class TimeoutMarker(generation: Long) + + /** + * Internal API + */ + private[akka] case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit system: ActorSystem) { private var ref: Option[Cancellable] = _ def schedule(actor: ActorRef, timeout: Duration) { @@ -56,8 +112,16 @@ object FSM { def unapply[S](in: (S, S)) = Some(in) } + /** + * Log Entry of the [[akka.actor.LoggingFSM]], can be obtained by calling `getLog`. + */ case class LogEntry[S, D](stateName: S, stateData: D, event: Any) + /** + * This captures all of the managed state of the [[akka.actor.FSM]]: the state + * name, the state data, possibly custom timeout, stop reason and replies + * accumulated while processing the last message. + */ case class State[S, D](stateName: S, stateData: D, timeout: Option[Duration] = None, stopReason: Option[Reason] = None, replies: List[Any] = Nil) { /** @@ -86,6 +150,9 @@ object FSM { copy(stateData = nextStateDate) } + /** + * Internal API. + */ private[akka] def withStopReason(reason: Reason): State[S, D] = { copy(stopReason = Some(reason)) } @@ -182,8 +249,19 @@ trait FSM[S, D] extends Listeners with ActorLogging { type Timeout = Option[Duration] type TransitionHandler = PartialFunction[(S, S), Unit] - // “import” so that it is visible without an import + /* + * “import” so that these are visible without an import + */ + + /** + * This extractor is just convenience for matching a (S, S) pair, including a + * reminder what the new state is. + */ val -> = FSM.-> + + /** + * This case object is received in case of a state timeout. + */ val StateTimeout = FSM.StateTimeout /** @@ -202,13 +280,9 @@ trait FSM[S, D] extends Listeners with ActorLogging { * @param stateTimeout default state timeout for this state * @param stateFunction partial function describing response to input */ - protected final def when(stateName: S, stateTimeout: Duration = null)(stateFunction: StateFunction): Unit = + final def when(stateName: S, stateTimeout: Duration = null)(stateFunction: StateFunction): Unit = register(stateName, stateFunction, Option(stateTimeout)) - @deprecated("use the more import-friendly variant taking a Duration", "2.0") - protected final def when(stateName: S, stateTimeout: Timeout)(stateFunction: StateFunction): Unit = - register(stateName, stateFunction, stateTimeout) - /** * Set initial state. Call this method from the constructor before the #initialize method. * @@ -216,9 +290,7 @@ trait FSM[S, D] extends Listeners with ActorLogging { * @param stateData initial state data * @param timeout state timeout for the initial state, overriding the default timeout for that state */ - protected final def startWith(stateName: S, - stateData: D, - timeout: Timeout = None): Unit = + final def startWith(stateName: S, stateData: D, timeout: Timeout = None): Unit = currentState = FSM.State(stateName, stateData, timeout) /** @@ -228,7 +300,7 @@ trait FSM[S, D] extends Listeners with ActorLogging { * @param nextStateName state designator for the next state * @return state transition descriptor */ - protected final def goto(nextStateName: S): State = FSM.State(nextStateName, currentState.stateData) + final def goto(nextStateName: S): State = FSM.State(nextStateName, currentState.stateData) /** * Produce "empty" transition descriptor. Return this from a state function @@ -236,29 +308,29 @@ trait FSM[S, D] extends Listeners with ActorLogging { * * @return descriptor for staying in current state */ - protected final def stay(): State = goto(currentState.stateName) // cannot directly use currentState because of the timeout field + final def stay(): State = goto(currentState.stateName) // cannot directly use currentState because of the timeout field /** * Produce change descriptor to stop this FSM actor with reason "Normal". */ - protected final def stop(): State = stop(Normal) + final def stop(): State = stop(Normal) /** * Produce change descriptor to stop this FSM actor including specified reason. */ - protected final def stop(reason: Reason): State = stop(reason, currentState.stateData) + final def stop(reason: Reason): State = stop(reason, currentState.stateData) /** * Produce change descriptor to stop this FSM actor including specified reason. */ - protected final def stop(reason: Reason, stateData: D): State = stay using stateData withStopReason (reason) + final def stop(reason: Reason, stateData: D): State = stay using stateData withStopReason (reason) - protected final class TransformHelper(func: StateFunction) { + final class TransformHelper(func: StateFunction) { def using(andThen: PartialFunction[State, State]): StateFunction = func andThen (andThen orElse { case x ⇒ x }) } - protected final def transform(func: StateFunction): TransformHelper = new TransformHelper(func) + final def transform(func: StateFunction): TransformHelper = new TransformHelper(func) /** * Schedule named timer to deliver message after given delay, possibly repeating. @@ -268,7 +340,9 @@ trait FSM[S, D] extends Listeners with ActorLogging { * @param repeat send once if false, scheduleAtFixedRate if true * @return current state descriptor */ - protected[akka] def setTimer(name: String, msg: Any, timeout: Duration, repeat: Boolean): State = { + final def setTimer(name: String, msg: Any, timeout: Duration, repeat: Boolean): State = { + if (debugEvent) + log.debug("setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg) if (timers contains name) { timers(name).cancel } @@ -282,24 +356,27 @@ trait FSM[S, D] extends Listeners with ActorLogging { * Cancel named timer, ensuring that the message is not subsequently delivered (no race). * @param name of the timer to cancel */ - protected[akka] def cancelTimer(name: String): Unit = + final def cancelTimer(name: String): Unit = { + if (debugEvent) + log.debug("canceling timer '" + name + "'") if (timers contains name) { timers(name).cancel timers -= name } + } /** * Inquire whether the named timer is still active. Returns true unless the * timer does not exist, has previously been canceled or if it was a * single-shot timer whose message was already received. */ - protected[akka] final def timerActive_?(name: String) = timers contains name + final def timerActive_?(name: String) = timers contains name /** * Set state timeout explicitly. This method can safely be used from within a * state handler. */ - protected final def setStateTimeout(state: S, timeout: Timeout): Unit = stateTimeouts(state) = timeout + final def setStateTimeout(state: S, timeout: Timeout): Unit = stateTimeouts(state) = timeout /** * Set handler which is called upon each state transition, i.e. not when @@ -326,50 +403,52 @@ trait FSM[S, D] extends Listeners with ActorLogging { * Multiple handlers may be installed, and every one of them will be * called, not only the first one matching. */ - protected final def onTransition(transitionHandler: TransitionHandler): Unit = transitionEvent :+= transitionHandler + final def onTransition(transitionHandler: TransitionHandler): Unit = transitionEvent :+= transitionHandler /** * Convenience wrapper for using a total function instead of a partial * function literal. To be used with onTransition. */ - implicit protected final def total2pf(transitionHandler: (S, S) ⇒ Unit): TransitionHandler = + implicit final def total2pf(transitionHandler: (S, S) ⇒ Unit): TransitionHandler = new TransitionHandler { def isDefinedAt(in: (S, S)) = true def apply(in: (S, S)) { transitionHandler(in._1, in._2) } } /** - * Set handler which is called upon termination of this FSM actor. + * Set handler which is called upon termination of this FSM actor. Calling + * this method again will overwrite the previous contents. */ - protected final def onTermination(terminationHandler: PartialFunction[StopEvent, Unit]): Unit = + final def onTermination(terminationHandler: PartialFunction[StopEvent, Unit]): Unit = terminateEvent = terminationHandler /** - * Set handler which is called upon reception of unhandled messages. + * Set handler which is called upon reception of unhandled messages. Calling + * this method again will overwrite the previous contents. */ - protected final def whenUnhandled(stateFunction: StateFunction): Unit = + final def whenUnhandled(stateFunction: StateFunction): Unit = handleEvent = stateFunction orElse handleEventDefault /** * Verify existence of initial state and setup timers. This should be the * last call within the constructor. */ - protected final def initialize: Unit = makeTransition(currentState) + final def initialize: Unit = makeTransition(currentState) /** * Return current state name (i.e. object of type S) */ - protected[akka] def stateName: S = currentState.stateName + final def stateName: S = currentState.stateName /** * Return current state data (i.e. object of type D) */ - protected[akka] def stateData: D = currentState.stateData + final def stateData: D = currentState.stateData /** * Return next state data (available in onTransition handlers) */ - protected[akka] def nextStateData = nextState.stateData + final def nextStateData = nextState.stateData /* * **************************************************************** @@ -377,6 +456,8 @@ trait FSM[S, D] extends Listeners with ActorLogging { * **************************************************************** */ + private[akka] def debugEvent: Boolean = false + /* * FSM State data and current timeout handling */ @@ -436,7 +517,7 @@ trait FSM[S, D] extends Listeners with ActorLogging { * Main actor receive() method * ******************************************* */ - override final protected def receive: Receive = { + override final def receive: Receive = { case TimeoutMarker(gen) ⇒ if (generation == gen) { processMsg(StateTimeout, "state timeout") @@ -524,7 +605,21 @@ trait FSM[S, D] extends Listeners with ActorLogging { } } - override def postStop(): Unit = { terminate(stay withStopReason Shutdown) } + /** + * Call `onTermination` hook; if you want to retain this behavior when + * overriding make sure to call `super.postStop()`. + * + * Please note that this method is called by default from `preRestart()`, + * so override that one if `onTermination` shall not be called during + * restart. + */ + override def postStop(): Unit = { + /* + * setting this instance’s state to terminated does no harm during restart + * since the new instance will initialize fresh using startWith() + */ + terminate(stay withStopReason Shutdown) + } private def terminate(nextState: State): Unit = { if (!currentState.stopReason.isDefined) { @@ -541,13 +636,22 @@ trait FSM[S, D] extends Listeners with ActorLogging { } } + /** + * All messages sent to the [[akka.actor.FSM]] will be wrapped inside an + * `Event`, which allows pattern matching to extract both state and data. + */ case class Event(event: Any, stateData: D) + /** + * Case class representing the state of the [[akka.actor.FSM]] whithin the + * `onTermination` block. + */ case class StopEvent(reason: Reason, currentState: S, stateData: D) } /** - * Stackable trait for FSM which adds a rolling event log. + * Stackable trait for [[akka.actor.FSM]] which adds a rolling event log and + * debug logging capabilities (analogous to [[akka.event.LoggingReceive]]). * * @since 1.2 */ @@ -557,7 +661,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒ def logDepth: Int = 0 - private val debugEvent = context.system.settings.FsmDebugEvent + private[akka] override val debugEvent = context.system.settings.FsmDebugEvent private val events = new Array[Event](logDepth) private val states = new Array[AnyRef](logDepth) @@ -574,18 +678,6 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒ } } - protected[akka] abstract override def setTimer(name: String, msg: Any, timeout: Duration, repeat: Boolean): State = { - if (debugEvent) - log.debug("setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg) - super.setTimer(name, msg, timeout, repeat) - } - - protected[akka] abstract override def cancelTimer(name: String): Unit = { - if (debugEvent) - log.debug("canceling timer '" + name + "'") - super.cancelTimer(name) - } - private[akka] abstract override def processEvent(event: Event, source: AnyRef): Unit = { if (debugEvent) { val srcstr = source match { @@ -614,6 +706,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒ /** * Retrieve current rolling log in oldest-first order. The log is filled with * each incoming event before processing by the user supplied state handler. + * The log entries are lost when this actor is restarted. */ protected def getLog: IndexedSeq[LogEntry[S, D]] = { val log = events zip states filter (_._1 ne null) map (x ⇒ LogEntry(x._2.asInstanceOf[S], x._1.stateData, x._1.event)) diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 70246bab30..76eed2eca9 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -9,8 +9,24 @@ import scala.collection.JavaConversions._ import java.lang.{ Iterable ⇒ JIterable } import akka.util.Duration -case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = 0, var restartTimeWindowStartNanos: Long = 0L) { +/** + * INTERNAL API + */ +private[akka] sealed trait ChildStats +/** + * INTERNAL API + */ +private[akka] case object ChildNameReserved extends ChildStats + +/** + * ChildRestartStats is the statistics kept by every parent Actor for every child Actor + * and is used for SupervisorStrategies to know how to deal with problems that occur for the children. + */ +case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = 0, var restartTimeWindowStartNanos: Long = 0L) + extends ChildStats { + + //FIXME How about making ChildRestartStats immutable and then move these methods into the actual supervisor strategies? def requestRestartPermission(retriesWindow: (Option[Int], Option[Int])): Boolean = retriesWindow match { case (Some(retries), _) if retries < 1 ⇒ false @@ -160,20 +176,21 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { def makeDecider(flat: Iterable[CauseDirective]): Decider = { val directives = sort(flat) - { - case x ⇒ directives find (_._1 isInstance x) map (_._2) getOrElse Escalate - } + { case x ⇒ directives collectFirst { case (c, d) if c isInstance x ⇒ d } getOrElse Escalate } } - def makeDecider(func: JDecider): Decider = { - case x ⇒ func(x) - } + /** + * Converts a Java Decider into a Scala Decider + */ + def makeDecider(func: JDecider): Decider = { case x ⇒ func(x) } /** * Sort so that subtypes always precede their supertypes, but without * obeying any order between unrelated subtypes (insert sort). + * + * INTERNAL API */ - def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] = + private[akka] def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] = (new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca) ⇒ buf.indexWhere(_._1 isAssignableFrom ca._1) match { case -1 ⇒ buf append ca @@ -184,14 +201,21 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { private[akka] def withinTimeRangeOption(withinTimeRange: Duration): Option[Duration] = if (withinTimeRange.isFinite && withinTimeRange >= Duration.Zero) Some(withinTimeRange) else None + private[akka] def maxNrOfRetriesOption(maxNrOfRetries: Int): Option[Int] = if (maxNrOfRetries < 0) None else Some(maxNrOfRetries) } +/** + * An Akka SupervisorStrategy is the policy to apply for crashing children + */ abstract class SupervisorStrategy { import SupervisorStrategy._ + /** + * Returns the Decider that is associated with this SupervisorStrategy + */ def decider: Decider /** @@ -204,21 +228,19 @@ abstract class SupervisorStrategy { */ def processFailure(context: ActorContext, restart: Boolean, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Unit - def handleSupervisorFailing(supervisor: ActorRef, children: Iterable[ActorRef]): Unit = { - if (children.nonEmpty) - children.foreach(_.asInstanceOf[InternalActorRef].suspend()) - } + //FIXME docs + def handleSupervisorFailing(supervisor: ActorRef, children: Iterable[ActorRef]): Unit = + if (children.nonEmpty) children.foreach(_.asInstanceOf[InternalActorRef].suspend()) - def handleSupervisorRestarted(cause: Throwable, supervisor: ActorRef, children: Iterable[ActorRef]): Unit = { - if (children.nonEmpty) - children.foreach(_.asInstanceOf[InternalActorRef].restart(cause)) - } + //FIXME docs + def handleSupervisorRestarted(cause: Throwable, supervisor: ActorRef, children: Iterable[ActorRef]): Unit = + if (children.nonEmpty) children.foreach(_.asInstanceOf[InternalActorRef].restart(cause)) /** * Returns whether it processed the failure or not */ def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Boolean = { - val directive = if (decider.isDefinedAt(cause)) decider(cause) else Escalate + val directive = if (decider.isDefinedAt(cause)) decider(cause) else Escalate //FIXME applyOrElse in Scala 2.10 directive match { case Resume ⇒ child.asInstanceOf[InternalActorRef].resume(); true case Restart ⇒ processFailure(context, true, child, cause, stats, children); true @@ -242,6 +264,8 @@ abstract class SupervisorStrategy { case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider) extends SupervisorStrategy { + import SupervisorStrategy._ + def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider) = this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(decider)) @@ -256,9 +280,7 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration * every call to requestRestartPermission, assuming that strategies are shared * across actors and thus this field does not take up much space */ - private val retriesWindow = ( - SupervisorStrategy.maxNrOfRetriesOption(maxNrOfRetries), - SupervisorStrategy.withinTimeRangeOption(withinTimeRange).map(_.toMillis.toInt)) + private val retriesWindow = (maxNrOfRetriesOption(maxNrOfRetries), withinTimeRangeOption(withinTimeRange).map(_.toMillis.toInt)) def handleChildTerminated(context: ActorContext, child: ActorRef, children: Iterable[ActorRef]): Unit = {} diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index a90dd6f579..07af4213fc 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -21,8 +21,7 @@ import java.nio.channels.{ import scala.collection.mutable import scala.annotation.tailrec import scala.collection.generic.CanBuildFrom -import com.eaio.uuid.UUID - +import java.util.UUID /** * IO messages and iteratees. * @@ -31,7 +30,7 @@ import com.eaio.uuid.UUID */ object IO { - final class DivergentIterateeException extends Exception("Iteratees should not return a continuation when receiving EOF") + final class DivergentIterateeException extends IllegalStateException("Iteratees should not return a continuation when receiving EOF") /** * An immutable handle to a Java NIO Channel. Contains a reference to the @@ -65,14 +64,14 @@ object IO { * A [[akka.actor.IO.Handle]] to a ReadableByteChannel. */ sealed trait ReadHandle extends Handle with Product { - override def asReadable = this + override def asReadable: ReadHandle = this } /** * A [[akka.actor.IO.Handle]] to a WritableByteChannel. */ sealed trait WriteHandle extends Handle with Product { - override def asWritable = this + override def asWritable: WriteHandle = this /** * Sends a request to the [[akka.actor.IOManager]] to write to the @@ -89,16 +88,16 @@ object IO { * created by [[akka.actor.IOManager]].connect() and * [[akka.actor.IO.ServerHandle]].accept(). */ - case class SocketHandle(owner: ActorRef, ioManager: ActorRef, uuid: UUID = new UUID()) extends ReadHandle with WriteHandle { - override def asSocket = this + case class SocketHandle(owner: ActorRef, ioManager: ActorRef, uuid: UUID = UUID.randomUUID()) extends ReadHandle with WriteHandle { + override def asSocket: SocketHandle = this } /** * A [[akka.actor.IO.Handle]] to a ServerSocketChannel. Instances are * normally created by [[akka.actor.IOManager]].listen(). */ - case class ServerHandle(owner: ActorRef, ioManager: ActorRef, uuid: UUID = new UUID()) extends Handle { - override def asServer = this + case class ServerHandle(owner: ActorRef, ioManager: ActorRef, uuid: UUID = UUID.randomUUID()) extends Handle { + override def asServer: ServerHandle = this /** * Sends a request to the [[akka.actor.IOManager]] to accept an incoming @@ -321,16 +320,18 @@ object IO { } object Chunk { - val empty = Chunk(ByteString.empty) + val empty: Chunk = new Chunk(ByteString.empty) } /** * Part of an [[akka.actor.IO.Input]] stream that contains a chunk of bytes. */ case class Chunk(bytes: ByteString) extends Input { - def ++(that: Input) = that match { - case Chunk(more) ⇒ Chunk(bytes ++ more) - case _: EOF ⇒ that + final override def ++(that: Input): Input = that match { + case Chunk(more) if more.isEmpty ⇒ this + case c: Chunk if bytes.isEmpty ⇒ c + case Chunk(more) ⇒ Chunk(bytes ++ more) + case _: EOF ⇒ that } } @@ -343,7 +344,7 @@ object IO { * Iteratee.recover() in order to handle it properly. */ case class EOF(cause: Option[Exception]) extends Input { - def ++(that: Input) = that + final override def ++(that: Input): Input = that } object Iteratee { @@ -353,7 +354,15 @@ object IO { * inferred as an Iteratee and not as a Done. */ def apply[A](value: A): Iteratee[A] = Done(value) + + /** + * Returns Iteratee.unit + */ def apply(): Iteratee[Unit] = unit + + /** + * The single value representing Done(()) + */ val unit: Iteratee[Unit] = Done(()) } @@ -446,6 +455,7 @@ object IO { */ final case class Cont[+A](f: Input ⇒ (Iteratee[A], Input), error: Option[Exception] = None) extends Iteratee[A] + //FIXME general description of what an IterateeRef is and how it is used, potentially with link to docs object IterateeRef { /** @@ -478,13 +488,14 @@ object IO { * 'refFactory' is used to provide the default value for new keys. */ class Map[K, V] private (refFactory: ⇒ IterateeRef[V], underlying: mutable.Map[K, IterateeRef[V]] = mutable.Map.empty[K, IterateeRef[V]]) extends mutable.Map[K, IterateeRef[V]] { - def get(key: K) = Some(underlying.getOrElseUpdate(key, refFactory)) - def iterator = underlying.iterator - def +=(kv: (K, IterateeRef[V])) = { underlying += kv; this } - def -=(key: K) = { underlying -= key; this } + override def get(key: K) = Some(underlying.getOrElseUpdate(key, refFactory)) + override def iterator = underlying.iterator + override def +=(kv: (K, IterateeRef[V])) = { underlying += kv; this } + override def -=(key: K) = { underlying -= key; this } override def empty = new Map[K, V](refFactory) } + //FIXME general description of what an Map is and how it is used, potentially with link to docs object Map { /** * Uses a factory to create the initial IterateeRef for each new key. @@ -501,7 +512,6 @@ object IO { */ def async[K]()(implicit executor: ExecutionContext): IterateeRef.Map[K, Unit] = new Map(IterateeRef.async()) } - } /** @@ -511,8 +521,11 @@ object IO { * for details. */ trait IterateeRef[A] { + //FIXME Add docs def flatMap(f: A ⇒ Iteratee[A]): Unit + //FIXME Add docs def map(f: A ⇒ A): Unit + //FIXME Add docs def apply(input: Input): Unit } @@ -529,12 +542,16 @@ object IO { */ final class IterateeRefSync[A](initial: Iteratee[A]) extends IterateeRef[A] { private var _value: (Iteratee[A], Input) = (initial, Chunk.empty) - def flatMap(f: A ⇒ Iteratee[A]): Unit = _value = _value match { + override def flatMap(f: A ⇒ Iteratee[A]): Unit = _value = _value match { case (iter, chunk @ Chunk(bytes)) if bytes.nonEmpty ⇒ (iter flatMap f)(chunk) case (iter, input) ⇒ (iter flatMap f, input) } - def map(f: A ⇒ A): Unit = _value = (_value._1 map f, _value._2) - def apply(input: Input): Unit = _value = _value._1(_value._2 ++ input) + override def map(f: A ⇒ A): Unit = _value = (_value._1 map f, _value._2) + override def apply(input: Input): Unit = _value = _value._1(_value._2 ++ input) + + /** + * Returns the current value of this IterateeRefSync + */ def value: (Iteratee[A], Input) = _value } @@ -554,12 +571,16 @@ object IO { */ final class IterateeRefAsync[A](initial: Iteratee[A])(implicit executor: ExecutionContext) extends IterateeRef[A] { private var _value: Future[(Iteratee[A], Input)] = Future((initial, Chunk.empty)) - def flatMap(f: A ⇒ Iteratee[A]): Unit = _value = _value map { + override def flatMap(f: A ⇒ Iteratee[A]): Unit = _value = _value map { case (iter, chunk @ Chunk(bytes)) if bytes.nonEmpty ⇒ (iter flatMap f)(chunk) case (iter, input) ⇒ (iter flatMap f, input) } - def map(f: A ⇒ A): Unit = _value = _value map (v ⇒ (v._1 map f, v._2)) - def apply(input: Input): Unit = _value = _value map (v ⇒ v._1(v._2 ++ input)) + override def map(f: A ⇒ A): Unit = _value = _value map (v ⇒ (v._1 map f, v._2)) + override def apply(input: Input): Unit = _value = _value map (v ⇒ v._1(v._2 ++ input)) + + /** + * Returns a Future which will hold the future value of this IterateeRefAsync + */ def future: Future[(Iteratee[A], Input)] = _value } @@ -703,10 +724,9 @@ object IO { /** * An Iteratee that continually repeats an Iteratee. * - * TODO: Should terminate on EOF + * FIXME TODO: Should terminate on EOF */ - def repeat(iter: Iteratee[Unit]): Iteratee[Unit] = - iter flatMap (_ ⇒ repeat(iter)) + def repeat(iter: Iteratee[Unit]): Iteratee[Unit] = iter flatMap (_ ⇒ repeat(iter)) /** * An Iteratee that applies an Iteratee to each element of a Traversable @@ -781,7 +801,7 @@ object IO { * An IOManager does not need to be manually stopped when not in use as it will * automatically enter an idle state when it has no channels to manage. */ -final class IOManager private (system: ActorSystem) extends Extension { +final class IOManager private (system: ActorSystem) extends Extension { //FIXME how about taking an ActorContext /** * A reference to the [[akka.actor.IOManagerActor]] that performs the actual * IO. It communicates with other actors using subclasses of @@ -862,9 +882,10 @@ final class IOManager private (system: ActorSystem) extends Extension { } +//FIXME add docs object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider { - override def lookup = this - override def createExtension(system: ExtendedActorSystem) = new IOManager(system) + override def lookup: IOManager.type = this + override def createExtension(system: ExtendedActorSystem): IOManager = new IOManager(system) } /** @@ -875,7 +896,7 @@ object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider { final class IOManagerActor extends Actor with ActorLogging { import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT } - private val bufferSize = 8192 // TODO: make buffer size configurable + private val bufferSize = 8192 // FIXME TODO: make configurable private type ReadChannel = ReadableByteChannel with SelectableChannel private type WriteChannel = WritableByteChannel with SelectableChannel @@ -898,7 +919,7 @@ final class IOManagerActor extends Actor with ActorLogging { private var lastSelect = 0 /** force a select when lastSelect reaches this amount */ - private val selectAt = 100 + private val selectAt = 100 // FIXME TODO: make configurable /** true while the selector is open and channels.nonEmpty */ private var running = false @@ -948,9 +969,7 @@ final class IOManagerActor extends Actor with ActorLogging { lastSelect = 0 } - private def forwardFailure(f: ⇒ Unit): Unit = { - try { f } catch { case NonFatal(e) ⇒ sender ! Status.Failure(e) } - } + private def forwardFailure(f: ⇒ Unit): Unit = try f catch { case NonFatal(e) ⇒ sender ! Status.Failure(e) } private def setSocketOptions(socket: java.net.Socket, options: Seq[IO.SocketOption]) { options foreach { @@ -968,7 +987,7 @@ final class IOManagerActor extends Actor with ActorLogging { } } - protected def receive = { + def receive = { case Select ⇒ select() if (running) self ! Select @@ -986,7 +1005,7 @@ final class IOManagerActor extends Actor with ActorLogging { forwardFailure(sock.setPerformancePreferences(connTime, latency, bandwidth)) } - channel.socket bind (address, 1000) // TODO: make backlog configurable + channel.socket bind (address, 1000) // FIXME TODO: make backlog configurable channels update (server, channel) channel register (selector, OP_ACCEPT, server) server.owner ! IO.Listening(server, channel.socket.getLocalSocketAddress()) @@ -1049,29 +1068,13 @@ final class IOManagerActor extends Actor with ActorLogging { private def process(key: SelectionKey) { val handle = key.attachment.asInstanceOf[IO.Handle] try { - if (key.isConnectable) key.channel match { - case channel: SocketChannel ⇒ connect(handle.asSocket, channel) - } - if (key.isAcceptable) key.channel match { - case channel: ServerSocketChannel ⇒ accept(handle.asServer, channel) - } - if (key.isReadable) key.channel match { - case channel: ReadChannel ⇒ read(handle.asReadable, channel) - } - if (key.isWritable) key.channel match { - case channel: WriteChannel ⇒ - try { - write(handle.asWritable, channel) - } catch { - case e: IOException ⇒ - // ignore, let it fail on read to ensure nothing left in read buffer. - } - } + if (key.isConnectable) key.channel match { case channel: SocketChannel ⇒ connect(handle.asSocket, channel) } + if (key.isAcceptable) key.channel match { case channel: ServerSocketChannel ⇒ accept(handle.asServer, channel) } + if (key.isReadable) key.channel match { case channel: ReadChannel ⇒ read(handle.asReadable, channel) } + if (key.isWritable) key.channel match { case channel: WriteChannel ⇒ try write(handle.asWritable, channel) catch { case e: IOException ⇒ } } // ignore, let it fail on read to ensure nothing left in read buffer. } catch { - case e: ClassCastException ⇒ cleanup(handle, Some(e)) - case e: CancelledKeyException ⇒ cleanup(handle, Some(e)) - case e: IOException ⇒ cleanup(handle, Some(e)) - case e: ActorInitializationException ⇒ cleanup(handle, Some(e)) + case e @ (_: ClassCastException | _: CancelledKeyException | _: IOException | _: ActorInitializationException) ⇒ + cleanup(handle, Some(e.asInstanceOf[Exception])) //Scala patmat is broken } } @@ -1090,9 +1093,6 @@ final class IOManagerActor extends Actor with ActorLogging { } } - private def setOps(handle: IO.Handle, ops: Int): Unit = - channels(handle) keyFor selector interestOps ops - private def addOps(handle: IO.Handle, ops: Int) { val key = channels(handle) keyFor selector val cur = key.interestOps @@ -1158,9 +1158,9 @@ final class IOManagerActor extends Actor with ActorLogging { } } } - } +//FIXME is this public API? final class WriteBuffer(bufferSize: Int) { private val _queue = new java.util.ArrayDeque[ByteString] private val _buffer = ByteBuffer.allocate(bufferSize) @@ -1182,9 +1182,9 @@ final class WriteBuffer(bufferSize: Int) { this } - def length = _length + def length: Int = _length - def isEmpty = _length == 0 + def isEmpty: Boolean = _length == 0 def write(channel: WritableByteChannel with SelectableChannel): Int = { @tailrec diff --git a/akka-actor/src/main/scala/akka/actor/Props.scala b/akka-actor/src/main/scala/akka/actor/Props.scala index 3751898c5c..f48bbe9573 100644 --- a/akka-actor/src/main/scala/akka/actor/Props.scala +++ b/akka-actor/src/main/scala/akka/actor/Props.scala @@ -6,7 +6,6 @@ package akka.actor import akka.dispatch._ import akka.japi.Creator -import collection.immutable.Stack import akka.routing._ /** @@ -18,12 +17,24 @@ import akka.routing._ */ object Props { + /** + * The defaultCreator, simply throws an UnsupportedOperationException when applied, which is used when creating a Props + */ final val defaultCreator: () ⇒ Actor = () ⇒ throw new UnsupportedOperationException("No actor creator specified!") + /** + * The defaultRoutedProps is NoRouter which is used when creating a Props + */ final val defaultRoutedProps: RouterConfig = NoRouter + /** + * The default Deploy instance which is used when creating a Props + */ final val defaultDeploy = Deploy() + /** + * A Props instance whose creator will create an actor that doesn't respond to any message + */ final val empty = new Props(() ⇒ new Actor { def receive = Actor.emptyBehavior }) /** @@ -31,26 +42,20 @@ object Props { */ final val default = new Props() - /** - * 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. * * Scala API. */ - def apply[T <: Actor: ClassManifest]: Props = + def apply[T <: Actor: ClassManifest](): Props = default.withCreator(implicitly[ClassManifest[T]].erasure.asInstanceOf[Class[_ <: Actor]]) /** * 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) + def apply(actorClass: Class[_ <: Actor]): Props = default.withCreator(actorClass) /** * Returns a Props that has default values except for "creator" which will be a function that creates an instance @@ -58,24 +63,26 @@ object Props { * * Scala API. */ - def apply(creator: ⇒ Actor): Props = - default.withCreator(creator) + def apply(creator: ⇒ Actor): Props = default.withCreator(creator) /** * 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: Creator[_ <: Actor]): Props = - default.withCreator(creator.create) + def apply(creator: Creator[_ <: Actor]): Props = default.withCreator(creator.create) - def apply(behavior: ActorContext ⇒ Actor.Receive): Props = - apply(new Actor { def receive = behavior(context) }) + /** + * Returns a new Props whose creator will instantiate an Actor that has the behavior specified + */ + def apply(behavior: ActorContext ⇒ Actor.Receive): Props = apply(new Actor { def receive = behavior(context) }) } /** * Props is a ActorRef configuration object, that is thread safe and fully sharable. * Used when creating new actors through; ActorSystem.actorOf and ActorContext.actorOf. * + * In case of providing code which creates the actual Actor instance, that must not return the same instance multiple times. + * * Examples on Scala API: * {{{ * val props = Props[MyActor] @@ -127,13 +134,15 @@ case class Props( * Java API. */ def this(actorClass: Class[_ <: Actor]) = this( - creator = () ⇒ actorClass.newInstance, + creator = FromClassCreator(actorClass), dispatcher = Dispatchers.DefaultDispatcherId, routerConfig = Props.defaultRoutedProps) /** * Returns a new Props with the specified creator set. * + * The creator must not return the same instance multiple times. + * * Scala API. */ def withCreator(c: ⇒ Actor): Props = copy(creator = () ⇒ c) @@ -141,6 +150,8 @@ case class Props( /** * Returns a new Props with the specified creator set. * + * The creator must not return the same instance multiple times. + * * Java API. */ def withCreator(c: Creator[Actor]): Props = copy(creator = () ⇒ c.create) @@ -150,7 +161,7 @@ case class Props( * * Java API. */ - def withCreator(c: Class[_ <: Actor]): Props = copy(creator = () ⇒ c.newInstance) + def withCreator(c: Class[_ <: Actor]): Props = copy(creator = FromClassCreator(c)) /** * Returns a new Props with the specified dispatcher set. @@ -166,4 +177,18 @@ case class Props( * Returns a new Props with the specified deployment configuration. */ def withDeploy(d: Deploy): Props = copy(deploy = d) + +} + +/** + * Used when creating an Actor from a class. Special Function0 to be + * able to optimize serialization. + */ +private[akka] case class FromClassCreator(clazz: Class[_ <: Actor]) extends Function0[Actor] { + def apply(): Actor = try clazz.newInstance catch { + case iae: IllegalAccessException ⇒ + val ctor = clazz.getDeclaredConstructor() + ctor.setAccessible(true) + ctor.newInstance() + } } diff --git a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala new file mode 100644 index 0000000000..ad9a7cb0c4 --- /dev/null +++ b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala @@ -0,0 +1,214 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.actor + +import akka.util.Unsafe +import scala.annotation.tailrec +import akka.dispatch.SystemMessage +import akka.dispatch.Mailbox +import akka.dispatch.Terminate +import akka.dispatch.Envelope +import akka.dispatch.Supervise +import akka.dispatch.Create +import akka.dispatch.MessageDispatcher +import java.util.concurrent.locks.ReentrantLock +import akka.event.Logging.Warning +import scala.collection.mutable.Queue + +/** + * This actor ref starts out with some dummy cell (by default just enqueuing + * messages into vectors protected by ReentrantLock), it must be initialize()’d + * before it can be sent to, and it will be activate()’d by its supervisor in + * response to the Supervise() message, which will replace the contained Cell + * with a fully functional one, transfer all messages from dummy to real queue + * and swap out the cell ref. + */ +private[akka] class RepointableActorRef( + val system: ActorSystemImpl, + val props: Props, + val supervisor: InternalActorRef, + val path: ActorPath) + extends ActorRefWithCell with RepointableRef { + + import AbstractActorRef.cellOffset + + @volatile private var _cellDoNotCallMeDirectly: Cell = _ + + def underlying: Cell = Unsafe.instance.getObjectVolatile(this, cellOffset).asInstanceOf[Cell] + + @tailrec final def swapCell(next: Cell): Cell = { + val old = underlying + if (Unsafe.instance.compareAndSwapObject(this, cellOffset, old, next)) old else swapCell(next) + } + + /** + * Initialize: make a dummy cell which holds just a mailbox, then tell our + * supervisor that we exist so that he can create the real Cell in + * handleSupervise(). + * + * Call twice on your own peril! + * + * This is protected so that others can have different initialization. + */ + def initialize(): this.type = { + swapCell(new UnstartedCell(system, this, props, supervisor)) + supervisor.sendSystemMessage(Supervise(this)) + this + } + + /** + * This method is supposed to be called by the supervisor in handleSupervise() + * to replace the UnstartedCell with the real one. It assumes no concurrent + * modification of the `underlying` field, though it is safe to send messages + * at any time. + */ + def activate(): this.type = { + underlying match { + case u: UnstartedCell ⇒ u.replaceWith(newCell()) + case _ ⇒ // this happens routinely for things which were created async=false + } + this + } + + /** + * This is called by activate() to obtain the cell which is to replace the + * unstarted cell. The cell must be fully functional. + */ + def newCell(): Cell = new ActorCell(system, this, props, supervisor).start() + + def suspend(): Unit = underlying.suspend() + + def resume(): Unit = underlying.resume() + + def stop(): Unit = underlying.stop() + + def restart(cause: Throwable): Unit = underlying.restart(cause) + + def isStarted: Boolean = !underlying.isInstanceOf[UnstartedCell] + + def isTerminated: Boolean = underlying.isTerminated + + def provider: ActorRefProvider = system.provider + + def isLocal: Boolean = underlying.isLocal + + def getParent: InternalActorRef = underlying.parent + + def getChild(name: Iterator[String]): InternalActorRef = + if (name.hasNext) { + name.next match { + case ".." ⇒ getParent.getChild(name) + case "" ⇒ getChild(name) + case other ⇒ + underlying.childrenRefs.getByName(other) match { + case Some(crs) ⇒ crs.child.asInstanceOf[InternalActorRef].getChild(name) + case None ⇒ Nobody + } + } + } else this + + def !(message: Any)(implicit sender: ActorRef = null) = underlying.tell(message, sender) + + def sendSystemMessage(message: SystemMessage) = underlying.sendSystemMessage(message) + + @throws(classOf[java.io.ObjectStreamException]) + protected def writeReplace(): AnyRef = SerializedActorRef(path) +} + +private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, val self: RepointableActorRef, val props: Props, val supervisor: InternalActorRef) + extends Cell { + + /* + * This lock protects all accesses to this cell’s queues. It also ensures + * safe switching to the started ActorCell. + */ + val lock = new ReentrantLock + + // use Envelope to keep on-send checks in the same place + val queue: Queue[Envelope] = Queue() + val systemQueue: Queue[SystemMessage] = Queue() + + def replaceWith(cell: Cell): Unit = { + lock.lock() + try { + /* + * The CallingThreadDispatcher nicely dives under the ReentrantLock and + * breaks things by enqueueing into stale queues from within the message + * processing which happens in-line for sendSystemMessage() and tell(). + * Since this is the only possible way to f*ck things up within this + * lock, double-tap (well, N-tap, really); concurrent modification is + * still not possible because we’re the only thread accessing the queues. + */ + var interrupted = false + while (systemQueue.nonEmpty || queue.nonEmpty) { + while (systemQueue.nonEmpty) { + val msg = systemQueue.dequeue() + try cell.sendSystemMessage(msg) + catch { + case _: InterruptedException ⇒ interrupted = true + } + } + if (queue.nonEmpty) { + val envelope = queue.dequeue() + try cell.tell(envelope.message, envelope.sender) + catch { + case _: InterruptedException ⇒ interrupted = true + } + } + } + if (interrupted) throw new InterruptedException + } finally try + self.swapCell(cell) + finally + lock.unlock() + } + + def system: ActorSystem = systemImpl + def suspend(): Unit = {} + def resume(): Unit = {} + def restart(cause: Throwable): Unit = {} + def stop(): Unit = sendSystemMessage(Terminate()) + def isTerminated: Boolean = false + def parent: InternalActorRef = supervisor + def childrenRefs: ActorCell.ChildrenContainer = ActorCell.EmptyChildrenContainer + def tell(message: Any, sender: ActorRef): Unit = { + lock.lock() + try { + if (self.underlying eq this) queue enqueue Envelope(message, sender, system) + else self.underlying.tell(message, sender) + } finally { + lock.unlock() + } + } + def sendSystemMessage(msg: SystemMessage): Unit = { + lock.lock() + try { + if (self.underlying eq this) systemQueue enqueue msg + else self.underlying.sendSystemMessage(msg) + } finally { + lock.unlock() + } + } + def isLocal = true + def hasMessages: Boolean = { + lock.lock() + try { + if (self.underlying eq this) !queue.isEmpty + else self.underlying.hasMessages + } finally { + lock.unlock() + } + } + def numberOfMessages: Int = { + lock.lock() + try { + if (self.underlying eq this) queue.size + else self.underlying.numberOfMessages + } finally { + lock.unlock() + } + } + +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index 827e511308..e1d36ab95d 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -1,14 +1,17 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor import akka.util.Duration -import org.jboss.netty.akka.util.{ TimerTask, HashedWheelTimer, Timeout ⇒ HWTimeout } +import akka.util.internal.{ TimerTask, HashedWheelTimer, Timeout ⇒ HWTimeout, Timer } import akka.event.LoggingAdapter import akka.dispatch.MessageDispatcher import java.io.Closeable +import java.util.concurrent.atomic.AtomicReference +import scala.annotation.tailrec +import akka.util.internal._ //#scheduler /** @@ -119,7 +122,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter, dispatcher: ⇒ MessageDispatcher) extends Scheduler with Closeable { - def schedule(initialDelay: Duration, delay: Duration, receiver: ActorRef, message: Any): Cancellable = { + override def schedule(initialDelay: Duration, delay: Duration, receiver: ActorRef, message: Any): Cancellable = { val continuousCancellable = new ContinuousCancellable continuousCancellable.init( hashedWheelTimer.newTimeout( @@ -134,7 +137,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, initialDelay)) } - def schedule(initialDelay: Duration, delay: Duration)(f: ⇒ Unit): Cancellable = { + override def schedule(initialDelay: Duration, delay: Duration)(f: ⇒ Unit): Cancellable = { val continuousCancellable = new ContinuousCancellable continuousCancellable.init( hashedWheelTimer.newTimeout( @@ -148,7 +151,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, initialDelay)) } - def schedule(initialDelay: Duration, delay: Duration, runnable: Runnable): Cancellable = { + override def schedule(initialDelay: Duration, delay: Duration, runnable: Runnable): Cancellable = { val continuousCancellable = new ContinuousCancellable continuousCancellable.init( hashedWheelTimer.newTimeout( @@ -161,7 +164,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, initialDelay)) } - def scheduleOnce(delay: Duration, runnable: Runnable): Cancellable = + override def scheduleOnce(delay: Duration, runnable: Runnable): Cancellable = new DefaultCancellable( hashedWheelTimer.newTimeout( new TimerTask() { @@ -169,7 +172,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, }, delay)) - def scheduleOnce(delay: Duration, receiver: ActorRef, message: Any): Cancellable = + override def scheduleOnce(delay: Duration, receiver: ActorRef, message: Any): Cancellable = new DefaultCancellable( hashedWheelTimer.newTimeout( new TimerTask { @@ -177,7 +180,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, }, delay)) - def scheduleOnce(delay: Duration)(f: ⇒ Unit): Cancellable = + override def scheduleOnce(delay: Duration)(f: ⇒ Unit): Cancellable = new DefaultCancellable( hashedWheelTimer.newTimeout( new TimerTask with Runnable { @@ -188,11 +191,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, private trait ContinuousScheduling { this: TimerTask ⇒ def scheduleNext(timeout: HWTimeout, delay: Duration, delegator: ContinuousCancellable) { - try { - delegator.swap(timeout.getTimer.newTimeout(this, delay)) - } catch { - case _: IllegalStateException ⇒ // stop recurring if timer is stopped - } + try delegator.swap(timeout.getTimer.newTimeout(this, delay)) catch { case _: IllegalStateException ⇒ } // stop recurring if timer is stopped } } @@ -203,54 +202,50 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, } } - def close() = { + override def close(): Unit = { import scala.collection.JavaConverters._ hashedWheelTimer.stop().asScala foreach execDirectly } } +private[akka] object ContinuousCancellable { + val initial: HWTimeout = new HWTimeout { + override def getTimer: Timer = null + override def getTask: TimerTask = null + override def isExpired: Boolean = false + override def isCancelled: Boolean = false + override def cancel: Unit = () + } + + val cancelled: HWTimeout = new HWTimeout { + override def getTimer: Timer = null + override def getTask: TimerTask = null + override def isExpired: Boolean = false + override def isCancelled: Boolean = true + override def cancel: Unit = () + } +} /** * Wrapper of a [[org.jboss.netty.akka.util.Timeout]] that delegates all * methods. Needed to be able to cancel continuous tasks, * since they create new Timeout for each tick. */ -private[akka] class ContinuousCancellable extends Cancellable { - @volatile - private var delegate: HWTimeout = _ - @volatile - private var cancelled = false - +private[akka] class ContinuousCancellable extends AtomicReference[HWTimeout](ContinuousCancellable.initial) with Cancellable { private[akka] def init(initialTimeout: HWTimeout): this.type = { - delegate = initialTimeout + compareAndSet(ContinuousCancellable.initial, initialTimeout) this } - private[akka] def swap(newTimeout: HWTimeout): Unit = { - val wasCancelled = isCancelled - delegate = newTimeout - if (wasCancelled || isCancelled) cancel() + @tailrec private[akka] final def swap(newTimeout: HWTimeout): Unit = get match { + case some if some.isCancelled ⇒ try cancel() finally newTimeout.cancel() + case some ⇒ if (!compareAndSet(some, newTimeout)) swap(newTimeout) } - def isCancelled(): Boolean = { - // delegate is initially null, but this object will not be exposed to the world until after init - cancelled || delegate.isCancelled() - } - - def cancel(): Unit = { - // the underlying Timeout will not become cancelled once the task has been started to run, - // therefore we keep a flag here to make sure that rescheduling doesn't occur when cancelled - cancelled = true - // delegate is initially null, but this object will not be exposed to the world until after init - delegate.cancel() - } + def isCancelled(): Boolean = get().isCancelled() + def cancel(): Unit = getAndSet(ContinuousCancellable.cancelled).cancel() } -class DefaultCancellable(val timeout: HWTimeout) extends Cancellable { - def cancel() { - timeout.cancel() - } - - def isCancelled: Boolean = { - timeout.isCancelled - } +private[akka] class DefaultCancellable(timeout: HWTimeout) extends AtomicReference[HWTimeout](timeout) with Cancellable { + override def cancel(): Unit = getAndSet(ContinuousCancellable.cancelled).cancel() + override def isCancelled: Boolean = get().isCancelled } diff --git a/akka-actor/src/main/scala/akka/actor/Stash.scala b/akka-actor/src/main/scala/akka/actor/Stash.scala index 6672945522..386bc0f070 100644 --- a/akka-actor/src/main/scala/akka/actor/Stash.scala +++ b/akka-actor/src/main/scala/akka/actor/Stash.scala @@ -56,7 +56,7 @@ trait Stash { /* The capacity of the stash. Configured in the actor's dispatcher config. */ - private val capacity = { + private val capacity: Int = { val dispatcher = context.system.settings.config.getConfig(context.props.dispatcher) val config = dispatcher.withFallback(context.system.settings.config.getConfig("akka.actor.default-dispatcher")) config.getInt("stash-capacity") @@ -125,4 +125,7 @@ An (unbounded) deque-based mailbox can be configured as follows: } +/** + * Is thrown when the size of the Stash exceeds the capacity of the Stash + */ class StashOverflowException(message: String, cause: Throwable = null) extends AkkaException(message, cause) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 4d85542d36..1933015e88 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -6,21 +6,28 @@ package akka.actor import akka.japi.{ Creator, Option ⇒ JOption } import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy } -import akka.util.{ Timeout, NonFatal } +import akka.util.{ Timeout, NonFatal, Duration } import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar } import akka.dispatch._ import java.util.concurrent.TimeoutException import java.util.concurrent.TimeUnit.MILLISECONDS -import java.lang.IllegalStateException -import akka.util.Duration import akka.actor.TypedActor.TypedActorInvocationHandler -import akka.serialization.{ JavaSerializer, Serialization, SerializationExtension } +import akka.serialization.{ JavaSerializer, SerializationExtension } import java.io.ObjectStreamException +/** + * A TypedActorFactory is something that can created TypedActor instances. + */ trait TypedActorFactory { + /** + * Underlying dependency is to be able to create normal Actors + */ protected def actorFactory: ActorRefFactory + /** + * Underlying dependency to a TypedActorExtension, which can either be contextual or ActorSystem "global" + */ protected def typedActor: TypedActorExtension /** @@ -80,6 +87,9 @@ trait TypedActorFactory { } +/** + * This represents the TypedActor Akka Extension, access to the functionality is done through a given ActorSystem. + */ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvider { override def get(system: ActorSystem): TypedActorExtension = super.get(system) @@ -145,8 +155,10 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi /** * Represents the serialized form of a MethodCall, uses readResolve and writeReplace to marshall the call + * + * INTERNAL USE ONLY */ - case class SerializedMethodCall(ownerType: Class[_], methodName: String, parameterTypes: Array[Class[_]], serializedParameters: Array[(Int, Class[_], Array[Byte])]) { + private[akka] case class SerializedMethodCall(ownerType: Class[_], methodName: String, parameterTypes: Array[Class[_]], serializedParameters: Array[(Int, Class[_], Array[Byte])]) { //TODO implement writeObject and readObject to serialize //TODO Possible optimization is to special encode the parameter-types to conserve space @@ -213,6 +225,8 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi /** * Implementation of TypedActor as an Actor + * + * INTERNAL USE ONLY */ private[akka] class TypedActor[R <: AnyRef, T <: R](val proxyVar: AtomVar[R], createInstance: ⇒ T) extends Actor { val me = try { @@ -371,6 +385,9 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi def postRestart(reason: Throwable): Unit } + /** + * INTERNAL USE ONLY + */ private[akka] class TypedActorInvocationHandler(@transient val extension: TypedActorExtension, @transient val actorVar: AtomVar[ActorRef], @transient val timeout: Timeout) extends InvocationHandler with Serializable { def actor = actorVar.get @throws(classOf[Throwable]) @@ -396,6 +413,9 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi @throws(classOf[ObjectStreamException]) private def writeReplace(): AnyRef = SerializedTypedActorInvocationHandler(actor, timeout.duration) } + /** + * INTERNAL USE ONLY + */ private[akka] case class SerializedTypedActorInvocationHandler(val actor: ActorRef, val timeout: Duration) { @throws(classOf[ObjectStreamException]) private def readResolve(): AnyRef = JavaSerializer.currentSystem.value match { case null ⇒ throw new IllegalStateException("SerializedTypedActorInvocationHandler.readResolve requires that JavaSerializer.currentSystem.value is set to a non-null value") @@ -569,12 +589,16 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( def withoutInterface(interface: Class[_ >: T]): TypedProps[T] = this.copy(interfaces = interfaces diff TypedProps.extractInterfaces(interface)) - import akka.actor.{ Props ⇒ ActorProps } - def actorProps(): ActorProps = - if (dispatcher == ActorProps().dispatcher) ActorProps() - else ActorProps(dispatcher = dispatcher) + /** + * Returns the akka.actor.Props representation of this TypedProps + */ + def actorProps(): Props = if (dispatcher == Props.default.dispatcher) Props.default else Props(dispatcher = dispatcher) } +/** + * ContextualTypedActorFactory allows TypedActors to create children, effectively forming the same Actor Supervision Hierarchies + * as normal Actors can. + */ case class ContextualTypedActorFactory(typedActor: TypedActorExtension, actorFactory: ActorContext) extends TypedActorFactory { override def getActorRefFor(proxy: AnyRef): ActorRef = typedActor.getActorRefFor(proxy) override def isTypedActor(proxyOrNot: AnyRef): Boolean = typedActor.isTypedActor(proxyOrNot) @@ -607,7 +631,9 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory def isTypedActor(proxyOrNot: AnyRef): Boolean = invocationHandlerFor(proxyOrNot) ne null // Private API - + /** + * INTERNAL USE ONLY + */ private[akka] def createActorRefProxy[R <: AnyRef, T <: R](props: TypedProps[T], proxyVar: AtomVar[R], actorRef: ⇒ ActorRef): R = { //Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling val actorVar = new AtomVar[ActorRef](null) @@ -631,6 +657,9 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory } } + /** + * INTERNAL USE ONLY + */ private[akka] def invocationHandlerFor(typedActor_? : AnyRef): TypedActorInvocationHandler = if ((typedActor_? ne null) && Proxy.isProxyClass(typedActor_?.getClass)) typedActor_? match { case null ⇒ null diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index a5ebeb851c..015d8fb9e3 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -93,11 +93,17 @@ import akka.japi.{ Creator } abstract class UntypedActor extends Actor { /** - * To be implemented by concrete UntypedActor. Defines the message handler. + * To be implemented by concrete UntypedActor, this defines the behavior of the + * UntypedActor. */ @throws(classOf[Exception]) def onReceive(message: Any): Unit + /** + * Returns this UntypedActor's UntypedActorContext + * The UntypedActorContext is not thread safe so do not expose it outside of the + * UntypedActor. + */ def getContext(): UntypedActorContext = context.asInstanceOf[UntypedActorContext] /** @@ -116,7 +122,7 @@ abstract class UntypedActor extends Actor { * User overridable definition the strategy to use for supervising * child actors. */ - override def supervisorStrategy(): SupervisorStrategy = super.supervisorStrategy() + override def supervisorStrategy: SupervisorStrategy = super.supervisorStrategy /** * User overridable callback. @@ -150,9 +156,7 @@ abstract class UntypedActor extends Actor { */ override def postRestart(reason: Throwable): Unit = super.postRestart(reason) - final protected def receive = { - case msg ⇒ onReceive(msg) - } + final def receive = { case msg ⇒ onReceive(msg) } } /** diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala index 9ec5348fee..3bf56b8bc4 100644 --- a/akka-actor/src/main/scala/akka/actor/package.scala +++ b/akka-actor/src/main/scala/akka/actor/package.scala @@ -7,24 +7,4 @@ package akka package object actor { implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef] implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef] - - type Uuid = com.eaio.uuid.UUID - - def newUuid(): Uuid = new Uuid() - - def uuidFrom(time: Long, clockSeqAndNode: Long): Uuid = new Uuid(time, clockSeqAndNode) - - def uuidFrom(uuid: String): Uuid = new Uuid(uuid) - - def simpleName(obj: AnyRef): String = { - val n = obj.getClass.getName - val i = n.lastIndexOf('.') - n.substring(i + 1) - } - - def simpleName(clazz: Class[_]): String = { - val n = clazz.getName - val i = n.lastIndexOf('.') - n.substring(i + 1) - } } diff --git a/akka-actor/src/main/scala/akka/config/ConfigurationException.scala b/akka-actor/src/main/scala/akka/config/ConfigurationException.scala deleted file mode 100644 index ba0a3a2234..0000000000 --- a/akka-actor/src/main/scala/akka/config/ConfigurationException.scala +++ /dev/null @@ -1,15 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.config - -import akka.AkkaException - -class ConfigurationException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { - def this(msg: String) = this(msg, null); -} - -class ModuleNotAvailableException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { - def this(msg: String) = this(msg, null); -} diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index db5c71167b..546373c33f 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -16,8 +16,10 @@ import akka.event.Logging.LogEventException import akka.jsr166y.{ ForkJoinTask, ForkJoinPool } import akka.util.{ Unsafe, Duration, NonFatal, Index } -final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorSystem) { - if (message.isInstanceOf[AnyRef]) { +final case class Envelope private (val message: Any, val sender: ActorRef) + +object Envelope { + def apply(message: Any, sender: ActorRef, system: ActorSystem): Envelope = { val msg = message.asInstanceOf[AnyRef] if (msg eq null) throw new InvalidMessageException("Message is null") if (system.settings.SerializeAllMessages && !msg.isInstanceOf[NoSerializationVerificationNeeded]) { @@ -30,10 +32,14 @@ final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorS } } } + new Envelope(message, sender) } } -object SystemMessage { +/** + * INTERNAL API + */ +private[akka] object SystemMessage { @tailrec final def size(list: SystemMessage, acc: Int = 0): Int = { if (list eq null) acc else size(list.next, acc + 1) @@ -59,33 +65,61 @@ object SystemMessage { * system messages is handled in a single thread only and not ever passed around, * hence no further synchronization is needed. * + * INTERNAL API + * * ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ */ -sealed trait SystemMessage extends PossiblyHarmful { +private[akka] sealed trait SystemMessage extends PossiblyHarmful { @transient var next: SystemMessage = _ } -case class Create() extends SystemMessage // send to self from Dispatcher.register -case class Recreate(cause: Throwable) extends SystemMessage // sent to self from ActorCell.restart -case class Suspend() extends SystemMessage // sent to self from ActorCell.suspend -case class Resume() extends SystemMessage // sent to self from ActorCell.resume -case class Terminate() extends SystemMessage // sent to self from ActorCell.stop -case class Supervise(child: ActorRef) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start -case class ChildTerminated(child: ActorRef) extends SystemMessage // sent to supervisor from ActorCell.doTerminate -case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.watch -case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.unwatch + +/** + * INTERNAL API + */ +private[akka] case class Create() extends SystemMessage // send to self from Dispatcher.register +/** + * INTERNAL API + */ +private[akka] case class Recreate(cause: Throwable) extends SystemMessage // sent to self from ActorCell.restart +/** + * INTERNAL API + */ +private[akka] case class Suspend() extends SystemMessage // sent to self from ActorCell.suspend +/** + * INTERNAL API + */ +private[akka] case class Resume() extends SystemMessage // sent to self from ActorCell.resume +/** + * INTERNAL API + */ +private[akka] case class Terminate() extends SystemMessage // sent to self from ActorCell.stop +/** + * INTERNAL API + */ +private[akka] case class Supervise(child: ActorRef) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start +/** + * INTERNAL API + */ +private[akka] case class ChildTerminated(child: ActorRef) extends SystemMessage // sent to supervisor from ActorCell.doTerminate +/** + * INTERNAL API + */ +private[akka] case class Watch(watchee: ActorRef, watcher: ActorRef) extends SystemMessage // sent to establish a DeathWatch +/** + * INTERNAL API + */ +private[akka] case class Unwatch(watchee: ActorRef, watcher: ActorRef) extends SystemMessage // sent to tear down a DeathWatch +/** + * INTERNAL API + */ +private[akka] case object NoMessage extends SystemMessage // switched into the mailbox to signal termination final case class TaskInvocation(eventStream: EventStream, runnable: Runnable, cleanup: () ⇒ Unit) extends Runnable { - def run() { - try { - runnable.run() - } catch { - case NonFatal(e) ⇒ - eventStream.publish(Error(e, "TaskInvocation", this.getClass, e.getMessage)) - } finally { - cleanup() - } - } + def run(): Unit = + try runnable.run() catch { + case NonFatal(e) ⇒ eventStream.publish(Error(e, "TaskInvocation", this.getClass, e.getMessage)) + } finally cleanup() } /** @@ -170,10 +204,16 @@ trait ExecutionContext { def reportFailure(t: Throwable): Unit } +/** + * INTERNAL API + */ private[akka] trait LoadMetrics { self: Executor ⇒ def atFullThrottle(): Boolean } +/** + * INTERNAL API + */ private[akka] object MessageDispatcher { val UNSCHEDULED = 0 //WARNING DO NOT CHANGE THE VALUE OF THIS: It relies on the faster init of 0 in AbstractMessageDispatcher val SCHEDULED = 1 @@ -181,7 +221,7 @@ private[akka] object MessageDispatcher { // dispatcher debugging helper using println (see below) // since this is a compile-time constant, scalac will elide code behind if (MessageDispatcher.debug) (RK checked with 2.9.1) - final val debug = false + final val debug = false // Deliberately without type ascription to make it a compile-time constant lazy val actors = new Index[MessageDispatcher, ActorRef](16, _ compareTo _) def printActors: Unit = if (debug) { for { @@ -191,8 +231,8 @@ private[akka] object MessageDispatcher { } { val status = if (a.isTerminated) " (terminated)" else " (alive)" val messages = a match { - case l: LocalActorRef ⇒ " " + l.underlying.mailbox.numberOfMessages + " messages" - case _ ⇒ " " + a.getClass + case r: ActorRefWithCell ⇒ " " + r.underlying.numberOfMessages + " messages" + case _ ⇒ " " + a.getClass } val parent = a match { case i: InternalActorRef ⇒ ", parent: " + i.getParent @@ -228,7 +268,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext /** * Creates and returns a mailbox for the given actor. */ - protected[akka] def createMailbox(actor: ActorCell): Mailbox + protected[akka] def createMailbox(actor: Cell): Mailbox //FIXME should this really be private[akka]? /** * Identifier of this dispatcher, corresponds to the full key @@ -255,7 +295,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext ifSensibleToDoSoThenScheduleShutdown() } - final def execute(runnable: Runnable) { + final def execute(runnable: Runnable): Unit = { val invocation = TaskInvocation(eventStream, runnable, taskCleanup) addInhabitants(+1) try { @@ -277,16 +317,14 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext case 0 ⇒ shutdownSchedule match { case UNSCHEDULED ⇒ - if (updateShutdownSchedule(UNSCHEDULED, SCHEDULED)) { - scheduleShutdownAction() - () - } else ifSensibleToDoSoThenScheduleShutdown() + if (updateShutdownSchedule(UNSCHEDULED, SCHEDULED)) scheduleShutdownAction() + else ifSensibleToDoSoThenScheduleShutdown() case SCHEDULED ⇒ if (updateShutdownSchedule(SCHEDULED, RESCHEDULED)) () else ifSensibleToDoSoThenScheduleShutdown() - case RESCHEDULED ⇒ () + case RESCHEDULED ⇒ } - case _ ⇒ () + case _ ⇒ } private def scheduleShutdownAction(): Unit = { @@ -300,6 +338,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext /** * If you override it, you must call it. But only ever once. See "attach" for only invocation. + * + * INTERNAL API */ protected[akka] def register(actor: ActorCell) { if (debug) actors.put(this, actor.self) @@ -308,13 +348,14 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext /** * If you override it, you must call it. But only ever once. See "detach" for the only invocation + * + * INTERNAL API */ protected[akka] def unregister(actor: ActorCell) { if (debug) actors.remove(this, actor.self) addInhabitants(-1) - val mailBox = actor.mailbox + val mailBox = actor.swapMailbox(deadLetterMailbox) mailBox.becomeClosed() // FIXME reschedule in tell if possible race with cleanUp is detected in order to properly clean up - actor.mailbox = deadLetterMailbox mailBox.cleanUp() } @@ -322,7 +363,6 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext @tailrec final def run() { shutdownSchedule match { - case UNSCHEDULED ⇒ () case SCHEDULED ⇒ try { if (inhabitants == 0) shutdown() //Warning, racy @@ -332,6 +372,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext case RESCHEDULED ⇒ if (updateShutdownSchedule(RESCHEDULED, SCHEDULED)) scheduleShutdownAction() else run() + case UNSCHEDULED ⇒ } } } @@ -340,6 +381,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext * When the dispatcher no longer has any actors registered, how long will it wait until it shuts itself down, * defaulting to your akka configs "akka.actor.default-dispatcher.shutdown-timeout" or default specified in * reference.conf + * + * INTERNAL API */ protected[akka] def shutdownTimeout: Duration @@ -362,36 +405,59 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext } /** - * Will be called when the dispatcher is to queue an invocation for execution + * Will be called when the dispatcher is to queue an invocation for execution + * + * INTERNAL API */ protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage) /** - * Will be called when the dispatcher is to queue an invocation for execution + * Will be called when the dispatcher is to queue an invocation for execution + * + * INTERNAL API */ protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) /** * Suggest to register the provided mailbox for execution + * + * INTERNAL API */ protected[akka] def registerForExecution(mbox: Mailbox, hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean // TODO check whether this should not actually be a property of the mailbox + /** + * INTERNAL API + */ protected[akka] def throughput: Int + + /** + * INTERNAL API + */ protected[akka] def throughputDeadlineTime: Duration - @inline - protected[akka] final val isThroughputDeadlineTimeDefined = throughputDeadlineTime.toMillis > 0 + /** + * INTERNAL API + */ + @inline protected[akka] final val isThroughputDeadlineTimeDefined = throughputDeadlineTime.toMillis > 0 + /** + * INTERNAL API + */ protected[akka] def executeTask(invocation: TaskInvocation) /** * Called one time every time an actor is detached from this dispatcher and this dispatcher has no actors left attached * Must be idempotent + * + * INTERNAL API */ protected[akka] def shutdown(): Unit } +/** + * An ExecutorServiceConfigurator is a class that given some prerequisites and a configuration can create instances of ExecutorService + */ abstract class ExecutorServiceConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceFactoryProvider /** diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index e95f54b88b..5b8c5209b0 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -9,6 +9,7 @@ import annotation.tailrec import akka.util.{ Duration, Helpers } import java.util.{ Comparator, Iterator } import java.util.concurrent.{ Executor, LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet } +import akka.actor.ActorSystemImpl /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -35,14 +36,36 @@ class BalancingDispatcher( attemptTeamWork: Boolean) extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) { - val team = new ConcurrentSkipListSet[ActorCell]( + /** + * INTERNAL USE ONLY + */ + private[akka] val team = new ConcurrentSkipListSet[ActorCell]( Helpers.identityHashComparator(new Comparator[ActorCell] { def compare(l: ActorCell, r: ActorCell) = l.self.path compareTo r.self.path })) - val messageQueue: MessageQueue = mailboxType.create(None) + /** + * INTERNAL USE ONLY + */ + private[akka] val messageQueue: MessageQueue = mailboxType.create(None, None) - protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor, messageQueue) + private class SharingMailbox(val system: ActorSystemImpl, _messageQueue: MessageQueue) + extends Mailbox(_messageQueue) with DefaultSystemMessageQueue { + override def cleanUp(): Unit = { + val dlq = system.deadLetterMailbox + //Don't call the original implementation of this since it scraps all messages, and we don't want to do that + var message = systemDrain(NoMessage) + while (message ne null) { + // message must be “virgin” before being able to systemEnqueue again + val next = message.next + message.next = null + dlq.systemEnqueue(system.deadLetters, message) + message = next + } + } + } + + protected[akka] override def createMailbox(actor: akka.actor.Cell): Mailbox = new SharingMailbox(actor.systemImpl, messageQueue) protected[akka] override def register(actor: ActorCell): Unit = { super.register(actor) @@ -64,7 +87,7 @@ class BalancingDispatcher( @tailrec def scheduleOne(i: Iterator[ActorCell] = team.iterator): Unit = if (messageQueue.hasMessages && i.hasNext - && (executorService.get().executor match { + && (executorService.executor match { case lm: LoadMetrics ⇒ lm.atFullThrottle == false case other ⇒ true }) @@ -74,22 +97,3 @@ class BalancingDispatcher( scheduleOne() } } - -class SharingMailbox(_actor: ActorCell, _messageQueue: MessageQueue) - extends Mailbox(_actor, _messageQueue) with DefaultSystemMessageQueue { - - override def cleanUp(): Unit = { - //Don't call the original implementation of this since it scraps all messages, and we don't want to do that - if (hasSystemMessages) { - val dlq = actor.systemImpl.deadLetterMailbox - var message = systemDrain() - while (message ne null) { - // message must be “virgin” before being able to systemEnqueue again - val next = message.next - message.next = null - dlq.systemEnqueue(actor.self, message) - message = next - } - } - } -} diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index fbffd08d7e..d382cc5ecc 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -9,6 +9,7 @@ import java.util.concurrent.atomic.AtomicReference import akka.actor.ActorCell import akka.util.Duration import java.util.concurrent._ +import akka.event.Logging /** * The event-based ``Dispatcher`` binds a set of Actors to a thread pool backed up by a @@ -32,31 +33,44 @@ class Dispatcher( val shutdownTimeout: Duration) extends MessageDispatcher(_prerequisites) { - protected val executorServiceFactory: ExecutorServiceFactory = - executorServiceFactoryProvider.createExecutorServiceFactory(id, prerequisites.threadFactory) + private class LazyExecutorServiceDelegate(factory: ExecutorServiceFactory) extends ExecutorServiceDelegate { + lazy val executor: ExecutorService = factory.createExecutorService + def copy(): LazyExecutorServiceDelegate = new LazyExecutorServiceDelegate(factory) + } - protected val executorService = new AtomicReference[ExecutorServiceDelegate]( - new ExecutorServiceDelegate { lazy val executor = executorServiceFactory.createExecutorService }) + @volatile private var executorServiceDelegate: LazyExecutorServiceDelegate = + new LazyExecutorServiceDelegate(executorServiceFactoryProvider.createExecutorServiceFactory(id, prerequisites.threadFactory)) - protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { + protected final def executorService: ExecutorServiceDelegate = executorServiceDelegate + + /** + * INTERNAL USE ONLY + */ + protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope): Unit = { val mbox = receiver.mailbox mbox.enqueue(receiver.self, invocation) registerForExecution(mbox, true, false) } - protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage) = { + /** + * INTERNAL USE ONLY + */ + protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage): Unit = { val mbox = receiver.mailbox mbox.systemEnqueue(receiver.self, invocation) registerForExecution(mbox, false, true) } + /** + * INTERNAL USE ONLY + */ protected[akka] def executeTask(invocation: TaskInvocation) { try { - executorService.get() execute invocation + executorService execute invocation } catch { case e: RejectedExecutionException ⇒ try { - executorService.get() execute invocation + executorService execute invocation } catch { case e2: RejectedExecutionException ⇒ prerequisites.eventStream.publish(Error(e, getClass.getName, getClass, "executeTask was rejected twice!")) @@ -65,26 +79,40 @@ class Dispatcher( } } - protected[akka] def createMailbox(actor: ActorCell): Mailbox = new Mailbox(actor, mailboxType.create(Some(actor))) with DefaultSystemMessageQueue + /** + * INTERNAL USE ONLY + */ + protected[akka] def createMailbox(actor: akka.actor.Cell): Mailbox = + new Mailbox(mailboxType.create(Some(actor.self), Some(actor.system))) with DefaultSystemMessageQueue - protected[akka] def shutdown: Unit = - Option(executorService.getAndSet(new ExecutorServiceDelegate { - lazy val executor = executorServiceFactory.createExecutorService - })) foreach { _.shutdown() } + /** + * INTERNAL USE ONLY + */ + protected[akka] def shutdown: Unit = { + val newDelegate = executorServiceDelegate.copy() // Doesn't matter which one we copy + val es = synchronized { // FIXME getAndSet using ARFU or Unsafe + val service = executorServiceDelegate + executorServiceDelegate = newDelegate // just a quick getAndSet + service + } + es.shutdown() + } /** * Returns if it was registered + * + * INTERNAL USE ONLY */ protected[akka] override def registerForExecution(mbox: Mailbox, hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = { if (mbox.canBeScheduledForExecution(hasMessageHint, hasSystemMessageHint)) { //This needs to be here to ensure thread safety and no races if (mbox.setAsScheduled()) { try { - executorService.get() execute mbox + executorService execute mbox true } catch { case e: RejectedExecutionException ⇒ try { - executorService.get() execute mbox + executorService execute mbox true } catch { //Retry once case e: RejectedExecutionException ⇒ @@ -97,7 +125,7 @@ class Dispatcher( } else false } - override val toString = getClass.getSimpleName + "[" + id + "]" + override val toString: String = Logging.simpleName(this) + "[" + id + "]" } object PriorityGenerator { diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index 93d44e007d..e148129bce 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -5,17 +5,15 @@ package akka.dispatch import java.util.concurrent.{ ConcurrentHashMap, TimeUnit, ThreadFactory } - -import scala.collection.JavaConverters.mapAsJavaMapConverter - import com.typesafe.config.{ ConfigFactory, Config } - -import Dispatchers.DefaultDispatcherId import akka.actor.{ Scheduler, DynamicAccess, ActorSystem } import akka.event.Logging.Warning import akka.event.EventStream import akka.util.Duration +/** + * DispatcherPrerequisites represents useful contextual pieces when constructing a MessageDispatcher + */ trait DispatcherPrerequisites { def threadFactory: ThreadFactory def eventStream: EventStream @@ -25,7 +23,10 @@ trait DispatcherPrerequisites { def settings: ActorSystem.Settings } -case class DefaultDispatcherPrerequisites( +/** + * INTERNAL USE ONLY + */ +private[akka] case class DefaultDispatcherPrerequisites( val threadFactory: ThreadFactory, val eventStream: EventStream, val deadLetterMailbox: Mailbox, @@ -96,6 +97,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc } } + //INTERNAL API private def config(id: String): Config = { import scala.collection.JavaConverters._ def simpleName = id.substring(id.lastIndexOf('.') + 1) @@ -105,12 +107,13 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc .withFallback(defaultDispatcherConfig) } + //INTERNAL API private def idConfig(id: String): Config = { import scala.collection.JavaConverters._ ConfigFactory.parseMap(Map("id" -> id).asJava) } - /* + /** * Creates a dispatcher from a Config. Internal test purpose only. * * ex: from(config.getConfig(id)) @@ -119,18 +122,22 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc * * Throws: IllegalArgumentException if the value of "type" is not valid * IllegalArgumentException if it cannot create the MessageDispatcherConfigurator + * + * INTERNAL USE ONLY */ - private[akka] def from(cfg: Config): MessageDispatcher = { - configuratorFrom(cfg).dispatcher() - } + private[akka] def from(cfg: Config): MessageDispatcher = configuratorFrom(cfg).dispatcher() - /* + private[akka] def isBalancingDispatcher(id: String): Boolean = settings.config.hasPath(id) && config(id).getString("type") == "BalancingDispatcher" + + /** * Creates a MessageDispatcherConfigurator from a Config. * * The Config must also contain a `id` property, which is the identifier of the dispatcher. * * Throws: IllegalArgumentException if the value of "type" is not valid * IllegalArgumentException if it cannot create the MessageDispatcherConfigurator + * + * INTERNAL USE ONLY */ private def configuratorFrom(cfg: Config): MessageDispatcherConfigurator = { if (!cfg.hasPath("id")) throw new IllegalArgumentException("Missing dispatcher 'id' property in config: " + cfg.root.render) @@ -208,7 +215,7 @@ class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherP class PinnedDispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends MessageDispatcherConfigurator(config, prerequisites) { - val threadPoolConfig: ThreadPoolConfig = configureExecutor() match { + private val threadPoolConfig: ThreadPoolConfig = configureExecutor() match { case e: ThreadPoolExecutorConfigurator ⇒ e.threadPoolConfig case other ⇒ prerequisites.eventStream.publish( diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 54ec2d08b4..e3c7f8348c 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -68,6 +68,7 @@ object Await { * WARNING: Blocking operation, use with caution. * * @throws [[java.util.concurrent.TimeoutException]] if times out + * @throws [[java.lang.Throwable]] (throws clause is Exception due to Java) if there was a problem * @return The returned value as returned by Awaitable.result */ @throws(classOf[Exception]) diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 2e3a98e8d9..36b386cef1 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -6,17 +6,17 @@ package akka.dispatch import akka.AkkaException import java.util.{ Comparator, PriorityQueue, Queue, Deque } import akka.util._ -import akka.actor.{ ActorCell, ActorRef } +import akka.actor.{ ActorCell, ActorRef, Cell } import java.util.concurrent._ import annotation.tailrec import akka.event.Logging.Error -import akka.actor.ActorContext import com.typesafe.config.Config -import akka.actor.ActorSystem +import akka.actor._ -class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause) - -object Mailbox { +/** + * INTERNAL API + */ +private[akka] object Mailbox { type Status = Int @@ -25,27 +25,49 @@ object Mailbox { */ // primary status: only first three - final val Open = 0 // _status is not initialized in AbstractMailbox, so default must be zero! - final val Suspended = 1 - final val Closed = 2 + final val Open = 0 // _status is not initialized in AbstractMailbox, so default must be zero! Deliberately without type ascription to make it a compile-time constant + final val Suspended = 1 // Deliberately without type ascription to make it a compile-time constant + final val Closed = 2 // Deliberately without type ascription to make it a compile-time constant // secondary status: Scheduled bit may be added to Open/Suspended - final val Scheduled = 4 + final val Scheduled = 4 // Deliberately without type ascription to make it a compile-time constant // mailbox debugging helper using println (see below) // since this is a compile-time constant, scalac will elide code behind if (Mailbox.debug) (RK checked with 2.9.1) - final val debug = false + final val debug = false // Deliberately without type ascription to make it a compile-time constant } /** * Mailbox and InternalMailbox is separated in two classes because ActorCell is needed for implementation, * but can't be exposed to user defined mailbox subclasses. * + * INTERNAL API */ -private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: MessageQueue) +private[akka] abstract class Mailbox(val messageQueue: MessageQueue) extends SystemMessageQueue with Runnable { import Mailbox._ + /* + * This is needed for actually executing the mailbox, i.e. invoking the + * ActorCell. There are situations (e.g. RepointableActorRef) where a Mailbox + * is constructed but we know that we will not execute it, in which case this + * will be null. It must be a var to support switching into an “active” + * mailbox, should the owning ActorRef turn local. + * + * ANOTHER THING, IMPORTANT: + * + * actorCell.start() publishes actorCell & self to the dispatcher, which + * means that messages may be processed theoretically before self’s constructor + * ends. The JMM guarantees visibility for final fields only after the end + * of the constructor, so safe publication requires that THIS WRITE BELOW + * stay as it is. + */ + @volatile + var actor: ActorCell = _ + def setActor(cell: ActorCell): Unit = actor = cell + + def dispatcher: MessageDispatcher = actor.dispatcher + /** * Try to enqueue the message to this queue, or throw an exception. */ @@ -162,6 +184,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes */ protected final def systemQueueGet: SystemMessage = Unsafe.instance.getObjectVolatile(this, AbstractMailbox.systemMessageOffset).asInstanceOf[SystemMessage] + protected final def systemQueuePut(_old: SystemMessage, _new: SystemMessage): Boolean = Unsafe.instance.compareAndSwapObject(this, AbstractMailbox.systemMessageOffset, _old, _new) @@ -200,26 +223,50 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes } } + /** + * Will at least try to process all queued system messages: in case of + * failure simply drop and go on to the next, because there is nothing to + * restart here (failure is in ActorCell somewhere …). In case the mailbox + * becomes closed (because of processing a Terminate message), dump all + * already dequeued message to deadLetters. + */ final def processAllSystemMessages() { - var nextMessage = systemDrain() - try { - while ((nextMessage ne null) && !isClosed) { - if (debug) println(actor.self + " processing system message " + nextMessage + " with " + actor.childrenRefs) - actor systemInvoke nextMessage - nextMessage = nextMessage.next - // don’t ever execute normal message when system message present! - if (nextMessage eq null) nextMessage = systemDrain() + var failure: Throwable = null + var nextMessage = systemDrain(null) + while ((nextMessage ne null) && !isClosed) { + val msg = nextMessage + nextMessage = nextMessage.next + msg.next = null + if (debug) println(actor.self + " processing system message " + msg + " with " + actor.childrenRefs) + try { + actor systemInvoke msg + } catch { + case NonFatal(e) ⇒ + if (failure eq null) failure = e + actor.system.eventStream.publish(Error(e, actor.self.path.toString, this.getClass, "exception during processing system message " + msg + ": " + e.getMessage)) } - } catch { - case NonFatal(e) ⇒ - actor.system.eventStream.publish(Error(e, actor.self.path.toString, this.getClass, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!")) - throw e + // don’t ever execute normal message when system message present! + if ((nextMessage eq null) && !isClosed) nextMessage = systemDrain(null) } + /* + * if we closed the mailbox, we must dump the remaining system messages + * to deadLetters (this is essential for DeathWatch) + */ + val dlm = actor.systemImpl.deadLetterMailbox + while (nextMessage ne null) { + val msg = nextMessage + nextMessage = nextMessage.next + msg.next = null + try dlm.systemEnqueue(actor.self, msg) + catch { + case NonFatal(e) ⇒ actor.system.eventStream.publish( + Error(e, actor.self.path.toString, this.getClass, "error while enqueuing " + msg + " to deadLetters: " + e.getMessage)) + } + } + // if something happened while processing, fail this actor (most probable: exception in supervisorStrategy) + if (failure ne null) actor.handleInvokeFailure(failure, failure.getMessage) } - @inline - final def dispatcher: MessageDispatcher = actor.dispatcher - /** * Overridable callback to clean up the mailbox, * called when an actor is unregistered. @@ -228,22 +275,25 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes protected[dispatch] def cleanUp(): Unit = if (actor ne null) { // actor is null for the deadLetterMailbox val dlm = actor.systemImpl.deadLetterMailbox - if (hasSystemMessages) { - var message = systemDrain() - while (message ne null) { - // message must be “virgin” before being able to systemEnqueue again - val next = message.next - message.next = null - dlm.systemEnqueue(actor.self, message) - message = next - } + var message = systemDrain(NoMessage) + while (message ne null) { + // message must be “virgin” before being able to systemEnqueue again + val next = message.next + message.next = null + dlm.systemEnqueue(actor.self, message) + message = next } if (messageQueue ne null) // needed for CallingThreadDispatcher, which never calls Mailbox.run() - messageQueue.cleanUp(actor, actor.systemImpl.deadLetterQueue) + messageQueue.cleanUp(actor.self, actor.systemImpl.deadLetterQueue) } } +/** + * A MessageQueue is one of the core components in forming an Akka Mailbox. + * The MessageQueue is where the normal messages that are sent to Actors will be enqueued (and subsequently dequeued) + * It needs to atleast support N producers and 1 consumer thread-safely. + */ trait MessageQueue { /** * Try to enqueue the message to this queue, or throw an exception. @@ -273,11 +323,11 @@ trait MessageQueue { * which is passed in. The owner of this MessageQueue is passed in if * available (e.g. for creating DeadLetters()), “/deadletters” otherwise. */ - def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit + def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit } /** - * Internal mailbox implementation detail. + * INTERNAL USE ONLY */ private[akka] trait SystemMessageQueue { /** @@ -288,130 +338,166 @@ private[akka] trait SystemMessageQueue { /** * Dequeue all messages from system queue and return them as single-linked list. */ - def systemDrain(): SystemMessage + def systemDrain(newContents: SystemMessage): SystemMessage def hasSystemMessages: Boolean } /** - * Internal mailbox implementation detail. + * INTERNAL USE ONLY */ private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒ @tailrec final def systemEnqueue(receiver: ActorRef, message: SystemMessage): Unit = { assert(message.next eq null) - if (Mailbox.debug) println(actor.self + " having enqueued " + message) + if (Mailbox.debug) println(receiver + " having enqueued " + message) val head = systemQueueGet - /* - * this write is safely published by the compareAndSet contained within - * systemQueuePut; “Intra-Thread Semantics” on page 12 of the JSR133 spec - * guarantees that “head” uses the value obtained from systemQueueGet above. - * Hence, SystemMessage.next does not need to be volatile. - */ - message.next = head - if (!systemQueuePut(head, message)) { - message.next = null - systemEnqueue(receiver, message) + if (head == NoMessage) { + if (actor ne null) actor.systemImpl.deadLetterMailbox.systemEnqueue(receiver, message) + } else { + /* + * this write is safely published by the compareAndSet contained within + * systemQueuePut; “Intra-Thread Semantics” on page 12 of the JSR133 spec + * guarantees that “head” uses the value obtained from systemQueueGet above. + * Hence, SystemMessage.next does not need to be volatile. + */ + message.next = head + if (!systemQueuePut(head, message)) { + message.next = null + systemEnqueue(receiver, message) + } } } @tailrec - final def systemDrain(): SystemMessage = { + final def systemDrain(newContents: SystemMessage): SystemMessage = { val head = systemQueueGet - if (systemQueuePut(head, null)) SystemMessage.reverse(head) else systemDrain() + if (systemQueuePut(head, newContents)) SystemMessage.reverse(head) else systemDrain(newContents) } def hasSystemMessages: Boolean = systemQueueGet ne null + } +/** + * A QueueBasedMessageQueue is a MessageQueue backed by a java.util.Queue + */ trait QueueBasedMessageQueue extends MessageQueue { def queue: Queue[Envelope] def numberOfMessages = queue.size def hasMessages = !queue.isEmpty - def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = { + def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = { if (hasMessages) { var envelope = dequeue while (envelope ne null) { - deadLetters.enqueue(owner.self, envelope) + deadLetters.enqueue(owner, envelope) envelope = dequeue } } } } +/** + * UnboundedMessageQueueSemantics adds unbounded semantics to a QueueBasedMessageQueue, + * i.e. a non-blocking enqueue and dequeue. + */ trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue { def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle def dequeue(): Envelope = queue.poll() } +/** + * BoundedMessageQueueSemantics adds bounded semantics to a QueueBasedMessageQueue, + * i.e. blocking enqueue with timeout + */ trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue { def pushTimeOut: Duration override def queue: BlockingQueue[Envelope] - def enqueue(receiver: ActorRef, handle: Envelope) { + def enqueue(receiver: ActorRef, handle: Envelope): Unit = if (pushTimeOut.length > 0) { - queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || { - throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver) - } + if (!queue.offer(handle, pushTimeOut.length, pushTimeOut.unit)) + receiver.asInstanceOf[InternalActorRef].provider.deadLetters ! DeadLetter(handle.message, handle.sender, receiver) } else queue put handle - } def dequeue(): Envelope = queue.poll() } +/** + * DequeBasedMessageQueue refines QueueBasedMessageQueue to be backed by a java.util.Deque + */ trait DequeBasedMessageQueue extends QueueBasedMessageQueue { def queue: Deque[Envelope] def enqueueFirst(receiver: ActorRef, handle: Envelope): Unit } +/** + * UnboundedDequeBasedMessageQueueSemantics adds unbounded semantics to a DequeBasedMessageQueue, + * i.e. a non-blocking enqueue and dequeue. + */ trait UnboundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue { def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle def enqueueFirst(receiver: ActorRef, handle: Envelope): Unit = queue addFirst handle def dequeue(): Envelope = queue.poll() } +/** + * BoundedMessageQueueSemantics adds bounded semantics to a DequeBasedMessageQueue, + * i.e. blocking enqueue with timeout + */ trait BoundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue { def pushTimeOut: Duration override def queue: BlockingDeque[Envelope] def enqueue(receiver: ActorRef, handle: Envelope): Unit = - if (pushTimeOut.length > 0) - queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || { - throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver) - } - else queue put handle + if (pushTimeOut.length > 0) { + if (!queue.offer(handle, pushTimeOut.length, pushTimeOut.unit)) + receiver.asInstanceOf[InternalActorRef].provider.deadLetters ! DeadLetter(handle.message, handle.sender, receiver) + } else queue put handle def enqueueFirst(receiver: ActorRef, handle: Envelope): Unit = - if (pushTimeOut.length > 0) - queue.offerFirst(handle, pushTimeOut.length, pushTimeOut.unit) || { - throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + receiver) - } - else queue putFirst handle + if (pushTimeOut.length > 0) { + if (!queue.offerFirst(handle, pushTimeOut.length, pushTimeOut.unit)) + receiver.asInstanceOf[InternalActorRef].provider.deadLetters ! DeadLetter(handle.message, handle.sender, receiver) + } else queue putFirst handle def dequeue(): Envelope = queue.poll() } /** - * Mailbox configuration. + * MailboxType is a factory to create MessageQueues for an optionally + * provided ActorContext. + * + * Possibly Important Notice + * + * When implementing a custom mailbox type, be aware that there is special + * semantics attached to `system.actorOf()` in that sending to the returned + * ActorRef may—for a short period of time—enqueue the messages first in a + * dummy queue. Top-level actors are created in two steps, and only after the + * guardian actor has performed that second step will all previously sent + * messages be transferred from the dummy queue into the real mailbox. */ trait MailboxType { - def create(owner: Option[ActorContext]): MessageQueue + def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue } /** - * It's a case class for Java (new UnboundedMailbox) + * UnboundedMailbox is the default unbounded MailboxType used by Akka Actors. */ case class UnboundedMailbox() extends MailboxType { def this(settings: ActorSystem.Settings, config: Config) = this() - final override def create(owner: Option[ActorContext]): MessageQueue = + final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new ConcurrentLinkedQueue[Envelope]() with QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final def queue: Queue[Envelope] = this } } +/** + * BoundedMailbox is the default bounded MailboxType used by Akka Actors. + */ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType { def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"), @@ -420,7 +506,7 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative") if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null") - final override def create(owner: Option[ActorContext]): MessageQueue = + final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new LinkedBlockingQueue[Envelope](capacity) with QueueBasedMessageQueue with BoundedMessageQueueSemantics { final def queue: BlockingQueue[Envelope] = this final val pushTimeOut = BoundedMailbox.this.pushTimeOut @@ -428,40 +514,49 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat } /** - * Extend me to provide the comparator + * UnboundedPriorityMailbox is an unbounded mailbox that allows for priorization of its contents. + * Extend this class and provide the Comparator in the constructor. */ -class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType { - final override def create(owner: Option[ActorContext]): MessageQueue = - new PriorityBlockingQueue[Envelope](11, cmp) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics { +class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope], final val initialCapacity: Int) extends MailboxType { + def this(cmp: Comparator[Envelope]) = this(cmp, 11) + final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = + new PriorityBlockingQueue[Envelope](initialCapacity, cmp) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final def queue: Queue[Envelope] = this } } /** - * Extend me to provide the comparator + * BoundedPriorityMailbox is a bounded mailbox that allows for priorization of its contents. + * Extend this class and provide the Comparator in the constructor. */ class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType { if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative") if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null") - final override def create(owner: Option[ActorContext]): MessageQueue = + final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) with QueueBasedMessageQueue with BoundedMessageQueueSemantics { final def queue: BlockingQueue[Envelope] = this final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut } } +/** + * UnboundedDequeBasedMailbox is an unbounded MailboxType, backed by a Deque. + */ case class UnboundedDequeBasedMailbox() extends MailboxType { def this(settings: ActorSystem.Settings, config: Config) = this() - final override def create(owner: Option[ActorContext]): MessageQueue = + final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new LinkedBlockingDeque[Envelope]() with DequeBasedMessageQueue with UnboundedDequeBasedMessageQueueSemantics { final val queue = this } } +/** + * BoundedDequeBasedMailbox is an bounded MailboxType, backed by a Deque. + */ case class BoundedDequeBasedMailbox( final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType { def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"), @@ -470,7 +565,7 @@ case class BoundedDequeBasedMailbox( final val capacity: Int, final val pushTime if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedDequeBasedMailbox can not be negative") if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedDequeBasedMailbox can not be null") - final override def create(owner: Option[ActorContext]): MessageQueue = + final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new LinkedBlockingDeque[Envelope](capacity) with DequeBasedMessageQueue with BoundedDequeBasedMessageQueueSemantics { final val queue = this final val pushTimeOut = BoundedDequeBasedMailbox.this.pushTimeOut diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index b6fd432296..963299debc 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -29,30 +29,20 @@ object ThreadPoolConfig { val defaultTimeout: Duration = Duration(60000L, TimeUnit.MILLISECONDS) val defaultRejectionPolicy: RejectedExecutionHandler = new SaneRejectedExecutionHandler() - def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int = { - import scala.math.{ min, max } - min(max((Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt, floor), ceiling) - } + def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int = + math.min(math.max((Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt, floor), ceiling) - def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory = - () ⇒ new ArrayBlockingQueue[Runnable](capacity, fair) + def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory = () ⇒ new ArrayBlockingQueue[Runnable](capacity, fair) - def synchronousQueue(fair: Boolean): QueueFactory = - () ⇒ new SynchronousQueue[Runnable](fair) + def synchronousQueue(fair: Boolean): QueueFactory = () ⇒ new SynchronousQueue[Runnable](fair) - def linkedBlockingQueue(): QueueFactory = - () ⇒ new LinkedBlockingQueue[Runnable]() + def linkedBlockingQueue(): QueueFactory = () ⇒ new LinkedBlockingQueue[Runnable]() - def linkedBlockingQueue(capacity: Int): QueueFactory = - () ⇒ new LinkedBlockingQueue[Runnable](capacity) + def linkedBlockingQueue(capacity: Int): QueueFactory = () ⇒ new LinkedBlockingQueue[Runnable](capacity) - def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory = - () ⇒ queue + def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory = () ⇒ queue - def reusableQueue(queueFactory: QueueFactory): QueueFactory = { - val queue = queueFactory() - () ⇒ queue - } + def reusableQueue(queueFactory: QueueFactory): QueueFactory = reusableQueue(queueFactory()) } /** @@ -157,7 +147,8 @@ case class ThreadPoolConfigBuilder(config: ThreadPoolConfig) { def setQueueFactory(newQueueFactory: QueueFactory): ThreadPoolConfigBuilder = this.copy(config = config.copy(queueFactory = newQueueFactory)) - def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder = fs.foldLeft(this)((c, f) ⇒ f.map(_(c)).getOrElse(c)) + def configure(fs: Option[Function[ThreadPoolConfigBuilder, ThreadPoolConfigBuilder]]*): ThreadPoolConfigBuilder = + fs.foldLeft(this)((c, f) ⇒ f.map(_(c)).getOrElse(c)) } object MonitorableThreadFactory { diff --git a/akka-actor/src/main/scala/akka/event/DeathWatch.scala b/akka-actor/src/main/scala/akka/event/DeathWatch.scala deleted file mode 100644 index 7469f6609f..0000000000 --- a/akka-actor/src/main/scala/akka/event/DeathWatch.scala +++ /dev/null @@ -1,19 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.event - -import akka.actor._ - -/** - * The contract of DeathWatch is not properly expressed using the type system - * Whenever there is a publish, all listeners to the Terminated Actor should be atomically removed - * A failed subscribe should also only mean that the Classifier (ActorRef) that is listened to is already shut down - * See LocalDeathWatch for semantics - */ -trait DeathWatch extends ActorEventBus with ActorClassifier { - type Event = Terminated - - protected final def classify(event: Event): Classifier = event.actor -} diff --git a/akka-actor/src/main/scala/akka/event/EventBus.scala b/akka-actor/src/main/scala/akka/event/EventBus.scala index 2dd22b3b54..cad7351bbb 100644 --- a/akka-actor/src/main/scala/akka/event/EventBus.scala +++ b/akka-actor/src/main/scala/akka/event/EventBus.scala @@ -182,10 +182,9 @@ trait SubchannelClassification { this: EventBus ⇒ */ trait ScanningClassification { self: EventBus ⇒ protected final val subscribers = new ConcurrentSkipListSet[(Classifier, Subscriber)](new Comparator[(Classifier, Subscriber)] { - def compare(a: (Classifier, Subscriber), b: (Classifier, Subscriber)): Int = { - val cM = compareClassifiers(a._1, b._1) - if (cM != 0) cM - else compareSubscribers(a._2, b._2) + def compare(a: (Classifier, Subscriber), b: (Classifier, Subscriber)): Int = compareClassifiers(a._1, b._1) match { + case 0 ⇒ compareSubscribers(a._2, b._2) + case other ⇒ other } }) @@ -238,7 +237,7 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒ import java.util.concurrent.ConcurrentHashMap import scala.annotation.tailrec private val empty = TreeSet.empty[ActorRef] - protected val mappings = new ConcurrentHashMap[ActorRef, TreeSet[ActorRef]](mapSize) + private val mappings = new ConcurrentHashMap[ActorRef, TreeSet[ActorRef]](mapSize) @tailrec protected final def associate(monitored: ActorRef, monitor: ActorRef): Boolean = { @@ -320,12 +319,22 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒ */ protected def mapSize: Int - def publish(event: Event): Unit = { - val receivers = mappings.get(classify(event)) - if (receivers ne null) receivers foreach { _ ! event } + def publish(event: Event): Unit = mappings.get(classify(event)) match { + case null ⇒ () + case some ⇒ some foreach { _ ! event } } - def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber) - def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = dissociate(from, subscriber) - def unsubscribe(subscriber: Subscriber): Unit = dissociate(subscriber) + def subscribe(subscriber: Subscriber, to: Classifier): Boolean = + if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null") + else if (to eq null) throw new IllegalArgumentException("Classifier is null") + else associate(to, subscriber) + + def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean = + if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null") + else if (from eq null) throw new IllegalArgumentException("Classifier is null") + else dissociate(from, subscriber) + + def unsubscribe(subscriber: Subscriber): Unit = + if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null") + else dissociate(subscriber) } diff --git a/akka-actor/src/main/scala/akka/event/EventStream.scala b/akka-actor/src/main/scala/akka/event/EventStream.scala index 27f0c71515..2cc9bf8c2b 100644 --- a/akka-actor/src/main/scala/akka/event/EventStream.scala +++ b/akka-actor/src/main/scala/akka/event/EventStream.scala @@ -3,7 +3,8 @@ */ package akka.event -import akka.actor.{ ActorRef, ActorSystem, simpleName } +import akka.actor.{ ActorRef, ActorSystem } +import akka.event.Logging.simpleName import akka.util.Subclassification object EventStream { @@ -38,17 +39,20 @@ class EventStream(private val debug: Boolean = false) extends LoggingBus with Su } override def subscribe(subscriber: ActorRef, channel: Class[_]): Boolean = { + if (subscriber eq null) throw new IllegalArgumentException("subscriber is null") if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "subscribing " + subscriber + " to channel " + channel)) super.subscribe(subscriber, channel) } override def unsubscribe(subscriber: ActorRef, channel: Class[_]): Boolean = { + if (subscriber eq null) throw new IllegalArgumentException("subscriber is null") val ret = super.unsubscribe(subscriber, channel) if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "unsubscribing " + subscriber + " from channel " + channel)) ret } override def unsubscribe(subscriber: ActorRef) { + if (subscriber eq null) throw new IllegalArgumentException("subscriber is null") super.unsubscribe(subscriber) if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "unsubscribing " + subscriber + " from all channels")) } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index bf4fc7996d..0777d9aef1 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -4,12 +4,10 @@ package akka.event import akka.actor._ -import akka.AkkaException +import akka.{ ConfigurationException, AkkaException } import akka.actor.ActorSystem.Settings -import akka.config.ConfigurationException -import akka.util.ReentrantGuard +import akka.util.{ Timeout, ReentrantGuard } import akka.util.duration._ -import akka.util.Timeout import java.util.concurrent.atomic.AtomicInteger import scala.util.control.NoStackTrace import java.util.concurrent.TimeoutException @@ -31,7 +29,7 @@ trait LoggingBus extends ActorEventBus { import Logging._ - private val guard = new ReentrantGuard + private val guard = new ReentrantGuard //Switch to ReentrantReadWrite private var loggers = Seq.empty[ActorRef] private var _logLevel: LogLevel = _ @@ -99,7 +97,7 @@ trait LoggingBus extends ActorEventBus { val myloggers = for { loggerName ← defaultLoggers - if loggerName != StandardOutLoggerName + if loggerName != StandardOutLogger.getClass.getName } yield { try { system.dynamicAccess.getClassFor[Actor](loggerName) match { @@ -131,7 +129,7 @@ trait LoggingBus extends ActorEventBus { case _: InvalidActorNameException ⇒ // ignore if it is already running } publish(Debug(logName, this.getClass, "Default Loggers started")) - if (!(defaultLoggers contains StandardOutLoggerName)) { + if (!(defaultLoggers contains StandardOutLogger.getClass.getName)) { unsubscribe(StandardOutLogger) } } catch { @@ -165,6 +163,9 @@ trait LoggingBus extends ActorEventBus { publish(Debug(simpleName(this), this.getClass, "all default loggers stopped")) } + /** + * INTERNAL API + */ private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel, logName: String): ActorRef = { val name = "log" + Extension(system).id() + "-" + simpleName(clazz) val actor = system.systemActorOf(Props(clazz), name) @@ -275,9 +276,9 @@ object LogSource { // this one unfortunately does not work as implicit, because existential types have some weird behavior val fromClass: LogSource[Class[_]] = new LogSource[Class[_]] { - def genString(c: Class[_]) = simpleName(c) - override def genString(c: Class[_], system: ActorSystem) = simpleName(c) + "(" + system + ")" - override def getClazz(c: Class[_]) = c + def genString(c: Class[_]): String = Logging.simpleName(c) + override def genString(c: Class[_], system: ActorSystem): String = genString(c) + "(" + system + ")" + override def getClazz(c: Class[_]): Class[_] = c } implicit def fromAnyClass[T]: LogSource[Class[T]] = fromClass.asInstanceOf[LogSource[Class[T]]] @@ -310,7 +311,7 @@ object LogSource { case a: Actor ⇒ apply(a) case a: ActorRef ⇒ apply(a) case s: String ⇒ apply(s) - case x ⇒ (simpleName(x), x.getClass) + case x ⇒ (Logging.simpleName(x), x.getClass) } /** @@ -324,7 +325,7 @@ object LogSource { case a: Actor ⇒ apply(a) case a: ActorRef ⇒ apply(a) case s: String ⇒ apply(s) - case x ⇒ (simpleName(x) + "(" + system + ")", x.getClass) + case x ⇒ (Logging.simpleName(x) + "(" + system + ")", x.getClass) } } @@ -363,9 +364,33 @@ object LogSource { */ object Logging { - object Extension extends ExtensionKey[LogExt] + /** + * Returns a 'safe' getSimpleName for the provided object's Class + * @param obj + * @return the simple name of the given object's Class + */ + def simpleName(obj: AnyRef): String = simpleName(obj.getClass) - class LogExt(system: ExtendedActorSystem) extends Extension { + /** + * Returns a 'safe' getSimpleName for the provided Class + * @param obj + * @return the simple name of the given Class + */ + def simpleName(clazz: Class[_]): String = { + val n = clazz.getName + val i = n.lastIndexOf('.') + n.substring(i + 1) + } + + /** + * INTERNAL API + */ + private[akka] object Extension extends ExtensionKey[LogExt] + + /** + * INTERNAL API + */ + private[akka] class LogExt(system: ExtendedActorSystem) extends Extension { private val loggerId = new AtomicInteger def id() = loggerId.incrementAndGet() } @@ -425,12 +450,6 @@ object Logging { // these type ascriptions/casts are necessary to avoid CCEs during construction while retaining correct type val AllLogLevels = Seq(ErrorLevel: AnyRef, WarningLevel, InfoLevel, DebugLevel).asInstanceOf[Seq[LogLevel]] - val errorFormat = "[ERROR] [%s] [%s] [%s] %s\n%s".intern - val errorFormatWithoutCause = "[ERROR] [%s] [%s] [%s] %s".intern - val warningFormat = "[WARN] [%s] [%s] [%s] %s".intern - val infoFormat = "[INFO] [%s] [%s] [%s] %s".intern - val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern - /** * Obtain LoggingAdapter for the given actor system and source object. This * will use the system’s event stream and include the system’s address in the @@ -511,7 +530,7 @@ object Logging { * Artificial exception injected into Error events if no Throwable is * supplied; used for getting a stack dump of error locations. */ - class EventHandlerException extends AkkaException + class EventHandlerException extends AkkaException("") /** * Exception that wraps a LogEvent. @@ -618,27 +637,34 @@ object Logging { // weird return type due to binary compatibility def loggerInitialized(): LoggerInitialized.type = LoggerInitialized + /** + * LoggerInitializationException is thrown to indicate that there was a problem initializing a logger + * @param msg + */ class LoggerInitializationException(msg: String) extends AkkaException(msg) trait StdOutLogger { import java.text.SimpleDateFormat import java.util.Date - val dateFormat = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss.S") + private val dateFormat = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss.SSS") + private val errorFormat = "[ERROR] [%s] [%s] [%s] %s%s".intern + private val errorFormatWithoutCause = "[ERROR] [%s] [%s] [%s] %s".intern + private val warningFormat = "[WARN] [%s] [%s] [%s] %s".intern + private val infoFormat = "[INFO] [%s] [%s] [%s] %s".intern + private val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern - def timestamp = dateFormat.format(new Date) + def timestamp(): String = synchronized { dateFormat.format(new Date) } // SDF isn't threadsafe - def print(event: Any) { - event match { - case e: Error ⇒ error(e) - case e: Warning ⇒ warning(e) - case e: Info ⇒ info(e) - case e: Debug ⇒ debug(e) - case e ⇒ warning(Warning(simpleName(this), this.getClass, "received unexpected event of class " + e.getClass + ": " + e)) - } + def print(event: Any): Unit = event match { + case e: Error ⇒ error(e) + case e: Warning ⇒ warning(e) + case e: Info ⇒ info(e) + case e: Debug ⇒ debug(e) + case e ⇒ warning(Warning(simpleName(this), this.getClass, "received unexpected event of class " + e.getClass + ": " + e)) } - def error(event: Error) = { + def error(event: Error): Unit = { val f = if (event.cause == Error.NoCause) errorFormatWithoutCause else errorFormat println(f.format( timestamp, @@ -648,21 +674,21 @@ object Logging { stackTraceFor(event.cause))) } - def warning(event: Warning) = + def warning(event: Warning): Unit = println(warningFormat.format( timestamp, event.thread.getName, event.logSource, event.message)) - def info(event: Info) = + def info(event: Info): Unit = println(infoFormat.format( timestamp, event.thread.getName, event.logSource, event.message)) - def debug(event: Debug) = + def debug(event: Debug): Unit = println(debugFormat.format( timestamp, event.thread.getName, @@ -683,8 +709,8 @@ object Logging { override val toString = "StandardOutLogger" override def !(message: Any)(implicit sender: ActorRef = null): Unit = print(message) } + val StandardOutLogger = new StandardOutLogger - val StandardOutLoggerName = StandardOutLogger.getClass.getName /** * Actor wrapper around the standard output logger. If @@ -703,9 +729,11 @@ object Logging { */ def stackTraceFor(e: Throwable): String = e match { case null | Error.NoCause ⇒ "" + case _: NoStackTrace ⇒ " (" + e.getClass.getName + ")" case other ⇒ val sw = new java.io.StringWriter val pw = new java.io.PrintWriter(sw) + pw.append('\n') other.printStackTrace(pw) sw.toString } @@ -746,51 +774,51 @@ trait LoggingAdapter { * These actually implement the passing on of the messages to be logged. * Will not be called if is...Enabled returned false. */ - protected def notifyError(message: String) - protected def notifyError(cause: Throwable, message: String) - protected def notifyWarning(message: String) - protected def notifyInfo(message: String) - protected def notifyDebug(message: String) + protected def notifyError(message: String): Unit + protected def notifyError(cause: Throwable, message: String): Unit + protected def notifyWarning(message: String): Unit + protected def notifyInfo(message: String): Unit + protected def notifyDebug(message: String): Unit /* * The rest is just the widening of the API for the user's convenience. */ - def error(cause: Throwable, message: String) { if (isErrorEnabled) notifyError(cause, message) } - def error(cause: Throwable, template: String, arg1: Any) { if (isErrorEnabled) notifyError(cause, format1(template, arg1)) } - def error(cause: Throwable, template: String, arg1: Any, arg2: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2)) } - def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3)) } - def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3, arg4)) } + def error(cause: Throwable, message: String): Unit = { if (isErrorEnabled) notifyError(cause, message) } + def error(cause: Throwable, template: String, arg1: Any): Unit = { if (isErrorEnabled) notifyError(cause, format1(template, arg1)) } + def error(cause: Throwable, template: String, arg1: Any, arg2: Any): Unit = { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2)) } + def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3)) } + def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3, arg4)) } - def error(message: String) { if (isErrorEnabled) notifyError(message) } - def error(template: String, arg1: Any) { if (isErrorEnabled) notifyError(format1(template, arg1)) } - def error(template: String, arg1: Any, arg2: Any) { if (isErrorEnabled) notifyError(format(template, arg1, arg2)) } - def error(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3)) } - def error(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3, arg4)) } + def error(message: String): Unit = { if (isErrorEnabled) notifyError(message) } + def error(template: String, arg1: Any): Unit = { if (isErrorEnabled) notifyError(format1(template, arg1)) } + def error(template: String, arg1: Any, arg2: Any): Unit = { if (isErrorEnabled) notifyError(format(template, arg1, arg2)) } + def error(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3)) } + def error(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3, arg4)) } - def warning(message: String) { if (isWarningEnabled) notifyWarning(message) } - def warning(template: String, arg1: Any) { if (isWarningEnabled) notifyWarning(format1(template, arg1)) } - def warning(template: String, arg1: Any, arg2: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2)) } - def warning(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3)) } - def warning(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3, arg4)) } + def warning(message: String): Unit = { if (isWarningEnabled) notifyWarning(message) } + def warning(template: String, arg1: Any): Unit = { if (isWarningEnabled) notifyWarning(format1(template, arg1)) } + def warning(template: String, arg1: Any, arg2: Any): Unit = { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2)) } + def warning(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3)) } + def warning(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3, arg4)) } def info(message: String) { if (isInfoEnabled) notifyInfo(message) } - def info(template: String, arg1: Any) { if (isInfoEnabled) notifyInfo(format1(template, arg1)) } - def info(template: String, arg1: Any, arg2: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2)) } - def info(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3)) } - def info(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3, arg4)) } + def info(template: String, arg1: Any): Unit = { if (isInfoEnabled) notifyInfo(format1(template, arg1)) } + def info(template: String, arg1: Any, arg2: Any): Unit = { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2)) } + def info(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3)) } + def info(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3, arg4)) } def debug(message: String) { if (isDebugEnabled) notifyDebug(message) } - def debug(template: String, arg1: Any) { if (isDebugEnabled) notifyDebug(format1(template, arg1)) } - def debug(template: String, arg1: Any, arg2: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2)) } - def debug(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3)) } - def debug(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3, arg4)) } + def debug(template: String, arg1: Any): Unit = { if (isDebugEnabled) notifyDebug(format1(template, arg1)) } + def debug(template: String, arg1: Any, arg2: Any): Unit = { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2)) } + def debug(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3)) } + def debug(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3, arg4)) } def log(level: Logging.LogLevel, message: String) { if (isEnabled(level)) notifyLog(level, message) } - def log(level: Logging.LogLevel, template: String, arg1: Any) { if (isEnabled(level)) notifyLog(level, format1(template, arg1)) } - def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2)) } - def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3)) } - def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3, arg4)) } + def log(level: Logging.LogLevel, template: String, arg1: Any): Unit = { if (isEnabled(level)) notifyLog(level, format1(template, arg1)) } + def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any): Unit = { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2)) } + def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3)) } + def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3, arg4)) } final def isEnabled(level: Logging.LogLevel): Boolean = level match { case Logging.ErrorLevel ⇒ isErrorEnabled @@ -806,14 +834,14 @@ trait LoggingAdapter { case Logging.DebugLevel ⇒ if (isDebugEnabled) notifyDebug(message) } - private def format1(t: String, arg: Any) = arg match { + private def format1(t: String, arg: Any): String = arg match { case a: Array[_] if !a.getClass.getComponentType.isPrimitive ⇒ format(t, a: _*) case a: Array[_] ⇒ format(t, (a map (_.asInstanceOf[AnyRef]): _*)) case x ⇒ format(t, x) } - def format(t: String, arg: Any*) = { - val sb = new StringBuilder + def format(t: String, arg: Any*): String = { + val sb = new StringBuilder //FIXME add some decent size hint here var p = 0 var rest = t while (p < arg.length) { @@ -823,17 +851,15 @@ trait LoggingAdapter { rest = "" p = arg.length } else { - sb.append(rest.substring(0, index)) - sb.append(arg(p)) + sb.append(rest.substring(0, index)).append(arg(p)) rest = rest.substring(index + 2) p += 1 } } - sb.append(rest) - sb.toString + sb.append(rest).toString } } - +//FIXME DOCUMENT class BusLogging(val bus: LoggingBus, val logSource: String, val logClass: Class[_]) extends LoggingAdapter { import Logging._ @@ -843,14 +869,22 @@ class BusLogging(val bus: LoggingBus, val logSource: String, val logClass: Class def isInfoEnabled = bus.logLevel >= InfoLevel def isDebugEnabled = bus.logLevel >= DebugLevel - protected def notifyError(message: String) { bus.publish(Error(logSource, logClass, message)) } - - protected def notifyError(cause: Throwable, message: String) { bus.publish(Error(cause, logSource, logClass, message)) } - - protected def notifyWarning(message: String) { bus.publish(Warning(logSource, logClass, message)) } - - protected def notifyInfo(message: String) { bus.publish(Info(logSource, logClass, message)) } - - protected def notifyDebug(message: String) { bus.publish(Debug(logSource, logClass, message)) } - + protected def notifyError(message: String): Unit = bus.publish(Error(logSource, logClass, message)) + protected def notifyError(cause: Throwable, message: String): Unit = bus.publish(Error(cause, logSource, logClass, message)) + protected def notifyWarning(message: String): Unit = bus.publish(Warning(logSource, logClass, message)) + protected def notifyInfo(message: String): Unit = bus.publish(Info(logSource, logClass, message)) + protected def notifyDebug(message: String): Unit = bus.publish(Debug(logSource, logClass, message)) +} + +private[akka] object NoLogging extends LoggingAdapter { + def isErrorEnabled = false + def isWarningEnabled = false + def isInfoEnabled = false + def isDebugEnabled = false + + protected def notifyError(message: String): Unit = () + protected def notifyError(cause: Throwable, message: String): Unit = () + protected def notifyWarning(message: String): Unit = () + protected def notifyInfo(message: String): Unit = () + protected def notifyDebug(message: String): Unit = () } diff --git a/akka-actor/src/main/scala/akka/event/LoggingReceive.scala b/akka-actor/src/main/scala/akka/event/LoggingReceive.scala index 452b2b6b19..337815eed1 100644 --- a/akka-actor/src/main/scala/akka/event/LoggingReceive.scala +++ b/akka-actor/src/main/scala/akka/event/LoggingReceive.scala @@ -26,9 +26,7 @@ object LoggingReceive { */ def apply(r: Receive)(implicit context: ActorContext): Receive = r match { case _: LoggingReceive ⇒ r - case _ ⇒ - if (context.system.settings.AddLoggingReceive) new LoggingReceive(None, r) - else r + case _ ⇒ if (context.system.settings.AddLoggingReceive) new LoggingReceive(None, r) else r } } @@ -37,7 +35,7 @@ object LoggingReceive { * @param source the log source, if not defined the actor of the context will be used */ class LoggingReceive(source: Option[AnyRef], r: Receive)(implicit context: ActorContext) extends Receive { - def isDefinedAt(o: Any) = { + def isDefinedAt(o: Any): Boolean = { val handled = r.isDefinedAt(o) val (str, clazz) = LogSource.fromAnyRef(source getOrElse context.asInstanceOf[ActorCell].actor) context.system.eventStream.publish(Debug(str, clazz, "received " + (if (handled) "handled" else "unhandled") + " message " + o)) diff --git a/akka-actor/src/main/scala/akka/experimental.scala b/akka-actor/src/main/scala/akka/experimental.scala deleted file mode 100644 index aef3cb5c85..0000000000 --- a/akka-actor/src/main/scala/akka/experimental.scala +++ /dev/null @@ -1,19 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka - -import annotation.target._ - -/** - * This annotation marks a feature which is not yet considered stable and may - * change or be removed in a future release. - * - * @since 1.2 - */ -@getter -@setter -@beanGetter -@beanSetter -final class experimental(since: String) extends annotation.StaticAnnotation diff --git a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala index 47ce667759..b0db141aee 100644 --- a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala +++ b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala @@ -24,34 +24,23 @@ trait Function2[T1, T2, R] { * A Procedure is like a Function, but it doesn't produce a return value. */ trait Procedure[T] { - def apply(param: T) -} - -/** - * A Procedure is like a Function, but it doesn't produce a return value. - */ -trait Procedure2[T1, T2] { - def apply(param: T1, param2: T2) -} - -/** - * An executable piece of code that takes no parameters and doesn't return any value. - */ -trait SideEffect { - def apply() + def apply(param: T): Unit } /** * An executable piece of code that takes no parameters and doesn't return any value. */ trait Effect { - def apply() + def apply(): Unit } /** * A constructor/factory, takes no parameters but creates a new value of type T every call. */ trait Creator[T] { + /** + * This method must return a different instance upon every call. + */ def create(): T } @@ -67,9 +56,9 @@ sealed abstract class Option[A] extends java.lang.Iterable[A] { def get: A def isEmpty: Boolean - def isDefined = !isEmpty + def isDefined: Boolean = !isEmpty def asScala: scala.Option[A] - def iterator = if (isEmpty) Iterator.empty else Iterator.single(get) + def iterator: java.util.Iterator[A] = if (isEmpty) Iterator.empty else Iterator.single(get) } object Option { @@ -102,18 +91,18 @@ object Option { * A. */ final case class Some[A](v: A) extends Option[A] { - def get = v - def isEmpty = false - def asScala = scala.Some(v) + def get: A = v + def isEmpty: Boolean = false + def asScala: scala.Some[A] = scala.Some(v) } /** * This case object represents non-existent values. */ private case object None extends Option[Nothing] { - def get = throw new NoSuchElementException("None.get") - def isEmpty = true - def asScala = scala.None + def get: Nothing = throw new NoSuchElementException("None.get") + def isEmpty: Boolean = true + def asScala: scala.None.type = scala.None } implicit def java2ScalaOption[A](o: Option[A]): scala.Option[A] = o.asScala diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala index ef4217039d..c66fa4178d 100644 --- a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala @@ -4,12 +4,10 @@ package akka.pattern import java.util.concurrent.TimeoutException -import akka.dispatch.{ Promise, Terminate, SystemMessage, Future } -import akka.event.DeathWatch -import akka.util.Timeout import annotation.tailrec -import akka.util.Unsafe import akka.actor._ +import akka.dispatch._ +import akka.util.{ NonFatal, Timeout, Unsafe } /** * This is what is used to complete a Future that is returned from an ask/? call, @@ -46,7 +44,7 @@ trait AskSupport { * Sends a message asynchronously and returns a [[akka.dispatch.Future]] * holding the eventual reply message; this means that the target actor * needs to send the result to the `sender` reference provided. The Future - * will be completed with an [[akka.actor.AskTimeoutException]] after the + * will be completed with an [[akka.pattern.AskTimeoutException]] after the * given timeout has expired; this is independent from any timeout applied * while awaiting a result for this future (i.e. in * `Await.result(..., timeout)`). @@ -96,7 +94,7 @@ trait AskSupport { * Sends a message asynchronously and returns a [[akka.dispatch.Future]] * holding the eventual reply message; this means that the target actor * needs to send the result to the `sender` reference provided. The Future - * will be completed with an [[akka.actor.AskTimeoutException]] after the + * will be completed with an [[akka.pattern.AskTimeoutException]] after the * given timeout has expired; this is independent from any timeout applied * while awaiting a result for this future (i.e. in * `Await.result(..., timeout)`). @@ -126,7 +124,7 @@ trait AskSupport { * Sends a message asynchronously and returns a [[akka.dispatch.Future]] * holding the eventual reply message; this means that the target actor * needs to send the result to the `sender` reference provided. The Future - * will be completed with an [[akka.actor.AskTimeoutException]] after the + * will be completed with an [[akka.pattern.AskTimeoutException]] after the * given timeout has expired; this is independent from any timeout applied * while awaiting a result for this future (i.e. in * `Await.result(..., timeout)`). @@ -157,11 +155,14 @@ trait AskSupport { /** * Akka private optimized representation of the temporary actor spawned to * receive the reply to an "ask" operation. + * + * INTERNAL API */ private[akka] final class PromiseActorRef private (val provider: ActorRefProvider, val result: Promise[Any]) extends MinimalActorRef { import PromiseActorRef._ import AbstractPromiseActorRef.stateOffset + import AbstractPromiseActorRef.watchedByOffset /** * As an optimization for the common (local) case we only register this PromiseActorRef @@ -178,18 +179,45 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide @volatile private[this] var _stateDoNotCallMeDirectly: AnyRef = _ - @inline - private def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset) + @volatile + private[this] var _watchedByDoNotCallMeDirectly: Set[ActorRef] = ActorCell.emptyActorRefSet @inline - private def updateState(oldState: AnyRef, newState: AnyRef): Boolean = + private[this] def watchedBy: Set[ActorRef] = Unsafe.instance.getObjectVolatile(this, watchedByOffset).asInstanceOf[Set[ActorRef]] + + @inline + private[this] def updateWatchedBy(oldWatchedBy: Set[ActorRef], newWatchedBy: Set[ActorRef]): Boolean = + Unsafe.instance.compareAndSwapObject(this, watchedByOffset, oldWatchedBy, newWatchedBy) + + @tailrec // Returns false if the Promise is already completed + private[this] final def addWatcher(watcher: ActorRef): Boolean = watchedBy match { + case null ⇒ false + case other ⇒ updateWatchedBy(other, other + watcher) || addWatcher(watcher) + } + + @tailrec + private[this] final def remWatcher(watcher: ActorRef): Unit = watchedBy match { + case null ⇒ () + case other ⇒ if (!updateWatchedBy(other, other - watcher)) remWatcher(watcher) + } + + @tailrec + private[this] final def clearWatchers(): Set[ActorRef] = watchedBy match { + case null ⇒ ActorCell.emptyActorRefSet + case other ⇒ if (!updateWatchedBy(other, null)) clearWatchers() else other + } + + @inline + private[this] def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset) + + @inline + private[this] def updateState(oldState: AnyRef, newState: AnyRef): Boolean = Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState) @inline - private def setState(newState: AnyRef): Unit = - Unsafe.instance.putObjectVolatile(this, stateOffset, newState) + private[this] def setState(newState: AnyRef): Unit = Unsafe.instance.putObjectVolatile(this, stateOffset, newState) - override def getParent = provider.tempContainer + override def getParent: InternalActorRef = provider.tempContainer /** * Contract of this method: @@ -218,51 +246,56 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide override def !(message: Any)(implicit sender: ActorRef = null): Unit = state match { case Stopped | _: StoppedWithPath ⇒ provider.deadLetters ! message - case _ ⇒ - val completedJustNow = result.tryComplete { - message match { - case Status.Success(r) ⇒ Right(r) - case Status.Failure(f) ⇒ Left(f) - case other ⇒ Right(other) - } + case _ ⇒ if (!(result.tryComplete { + message match { + case Status.Success(r) ⇒ Right(r) + case Status.Failure(f) ⇒ Left(f) + case other ⇒ Right(other) } - if (!completedJustNow) provider.deadLetters ! message + })) provider.deadLetters ! message } override def sendSystemMessage(message: SystemMessage): Unit = message match { case _: Terminate ⇒ stop() - case _ ⇒ + case Watch(watchee, watcher) ⇒ + if (watchee == this && watcher != this) { + if (!addWatcher(watcher)) watcher ! Terminated(watchee)(existenceConfirmed = true) + } else System.err.println("BUG: illegal Watch(%s,%s) for %s".format(watchee, watcher, this)) + case Unwatch(watchee, watcher) ⇒ + if (watchee == this && watcher != this) remWatcher(watcher) + else System.err.println("BUG: illegal Unwatch(%s,%s) for %s".format(watchee, watcher, this)) + case _ ⇒ } - override def isTerminated = state match { + override def isTerminated: Boolean = state match { case Stopped | _: StoppedWithPath ⇒ true case _ ⇒ false } @tailrec override def stop(): Unit = { - def ensurePromiseCompleted(): Unit = + def ensureCompleted(): Unit = { if (!result.isCompleted) result.tryComplete(Left(new ActorKilledException("Stopped"))) + val watchers = clearWatchers() + if (!watchers.isEmpty) { + val termination = Terminated(this)(existenceConfirmed = true) + watchers foreach { w ⇒ try w.tell(termination, this) catch { case NonFatal(t) ⇒ /* FIXME LOG THIS */ } } + } + } state match { - case null ⇒ - // if path was never queried nobody can possibly be watching us, so we don't have to publish termination either - if (updateState(null, Stopped)) ensurePromiseCompleted() - else stop() + case null ⇒ // if path was never queried nobody can possibly be watching us, so we don't have to publish termination either + if (updateState(null, Stopped)) ensureCompleted() else stop() case p: ActorPath ⇒ - if (updateState(p, StoppedWithPath(p))) { - try { - ensurePromiseCompleted() - provider.deathWatch.publish(Terminated(this)) - } finally { - provider.unregisterTempActor(p) - } - } else stop() - case Stopped | _: StoppedWithPath ⇒ + if (updateState(p, StoppedWithPath(p))) { try ensureCompleted() finally provider.unregisterTempActor(p) } else stop() + case Stopped | _: StoppedWithPath ⇒ // already stopped case Registering ⇒ stop() // spin until registration is completed before stopping } } } +/** + * INTERNAL API + */ private[akka] object PromiseActorRef { private case object Registering private case object Stopped @@ -272,9 +305,7 @@ private[akka] object PromiseActorRef { val result = Promise[Any]()(provider.dispatcher) val a = new PromiseActorRef(provider, result) val f = provider.scheduler.scheduleOnce(timeout.duration) { result.tryComplete(Left(new AskTimeoutException("Timed out"))) } - result onComplete { _ ⇒ - try a.stop() finally f.cancel() - } + result onComplete { _ ⇒ try a.stop() finally f.cancel() } a } } \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala b/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala new file mode 100644 index 0000000000..ac8fd1c5ed --- /dev/null +++ b/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala @@ -0,0 +1,560 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.pattern + +import java.util.concurrent.atomic.{ AtomicInteger, AtomicLong, AtomicBoolean } +import akka.AkkaException +import akka.actor.Scheduler +import akka.dispatch.{ Future, ExecutionContext, Await, Promise } +import akka.util.{ Deadline, Duration, NonFatal, Unsafe } +import akka.util.duration._ +import util.control.NoStackTrace +import java.util.concurrent.{ Callable, CopyOnWriteArrayList } + +/** + * Companion object providing factory methods for Circuit Breaker which runs callbacks in caller's thread + */ +object CircuitBreaker { + + /** + * Synchronous execution context to run in caller's thread - used by companion object factory methods + */ + private[CircuitBreaker] val syncExecutionContext = new ExecutionContext { + def execute(runnable: Runnable): Unit = runnable.run() + + def reportFailure(t: Throwable): Unit = () + } + + /** + * Callbacks run in caller's thread when using withSyncCircuitBreaker, and in same ExecutionContext as the passed + * in Future when using withCircuitBreaker. To use another ExecutionContext for the callbacks you can specify the + * executor in the constructor. + * + * @param scheduler Reference to Akka scheduler + * @param maxFailures Maximum number of failures before opening the circuit + * @param callTimeout [[akka.util.Duration]] of time after which to consider a call a failure + * @param resetTimeout [[akka.util.Duration]] of time after which to attempt to close the circuit + */ + def apply(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration): CircuitBreaker = + new CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration)(syncExecutionContext) + + /** + * Java API alias for apply + * + * @param scheduler Reference to Akka scheduler + * @param maxFailures Maximum number of failures before opening the circuit + * @param callTimeout [[akka.util.Duration]] of time after which to consider a call a failure + * @param resetTimeout [[akka.util.Duration]] of time after which to attempt to close the circuit + */ + def create(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration): CircuitBreaker = + apply(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration) +} + +/** + * Provides circuit breaker functionality to provide stability when working with "dangerous" operations, e.g. calls to + * remote systems + * + * Transitions through three states: + * - In *Closed* state, calls pass through until the `maxFailures` count is reached. This causes the circuit breaker + * to open. Both exceptions and calls exceeding `callTimeout` are considered failures. + * - In *Open* state, calls fail-fast with an exception. After `resetTimeout`, circuit breaker transitions to + * half-open state. + * - In *Half-Open* state, the first call will be allowed through, if it succeeds the circuit breaker will reset to + * closed state. If it fails, the circuit breaker will re-open to open state. All calls beyond the first that + * execute while the first is running will fail-fast with an exception. + * + * + * @param scheduler Reference to Akka scheduler + * @param maxFailures Maximum number of failures before opening the circuit + * @param callTimeout [[akka.util.Duration]] of time after which to consider a call a failure + * @param resetTimeout [[akka.util.Duration]] of time after which to attempt to close the circuit + * @param executor [[akka.dispatch.ExecutionContext]] used for execution of state transition listeners + */ +class CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration)(implicit executor: ExecutionContext) extends AbstractCircuitBreaker { + + def this(executor: ExecutionContext, scheduler: Scheduler, maxFailures: Int, callTimeout: Duration, resetTimeout: Duration) = { + this(scheduler, maxFailures, callTimeout, resetTimeout)(executor) + } + + /** + * Holds reference to current state of CircuitBreaker - *access only via helper methods* + */ + @volatile + private[this] var _currentStateDoNotCallMeDirectly: State = Closed + + /** + * Helper method for access to underlying state via Unsafe + * + * @param oldState Previous state on transition + * @param newState Next state on transition + * @return Whether the previous state matched correctly + */ + @inline + private[this] def swapState(oldState: State, newState: State): Boolean = + Unsafe.instance.compareAndSwapObject(this, AbstractCircuitBreaker.stateOffset, oldState, newState) + + /** + * Helper method for accessing underlying state via Unsafe + * + * @return Reference to current state + */ + @inline + private[this] def currentState: State = + Unsafe.instance.getObjectVolatile(this, AbstractCircuitBreaker.stateOffset).asInstanceOf[State] + + /** + * Wraps invocations of asynchronous calls that need to be protected + * + * @param body Call needing protected + * @tparam T return type from call + * @return [[akka.dispatch.Future]] containing the call result + */ + def withCircuitBreaker[T](body: ⇒ Future[T]): Future[T] = { + currentState.invoke(body) + } + + /** + * Java API for withCircuitBreaker + * + * @param body Call needing protected + * @tparam T return type from call + * @return [[akka.dispatch.Future]] containing the call result + */ + def callWithCircuitBreaker[T](body: Callable[Future[T]]): Future[T] = { + withCircuitBreaker(body.call) + } + + /** + * Wraps invocations of synchronous calls that need to be protected + * + * Calls are run in caller's thread + * + * @param body Call needing protected + * @tparam T return type from call + * @return The result of the call + */ + def withSyncCircuitBreaker[T](body: ⇒ T): T = { + Await.result(withCircuitBreaker( + { + try + Promise.successful(body)(CircuitBreaker.syncExecutionContext) + catch { + case NonFatal(t) ⇒ Promise.failed(t)(CircuitBreaker.syncExecutionContext) + } + }), callTimeout) + } + + /** + * Java API for withSyncCircuitBreaker + * + * @param body Call needing protected + * @tparam T return type from call + * @return The result of the call + */ + + def callWithSyncCircuitBreaker[T](body: Callable[T]): T = { + withSyncCircuitBreaker(body.call) + } + + /** + * Adds a callback to execute when circuit breaker opens + * + * The callback is run in the [[akka.dispatch.ExecutionContext]] supplied in the constructor. + * + * @param callback Handler to be invoked on state change + * @tparam T Type supplied to assist with type inference, otherwise ignored by implementation + * @return CircuitBreaker for fluent usage + */ + def onOpen[T](callback: ⇒ T): CircuitBreaker = { + Open.addListener(() ⇒ callback) + this + } + + /** + * Java API for onOpen + * + * @param callback Handler to be invoked on state change + * @tparam T Type supplied to assist with type inference, otherwise ignored by implementation + * @return CircuitBreaker for fluent usage + */ + def onOpen[T](callback: Callable[T]): CircuitBreaker = { + onOpen(callback.call) + } + + /** + * Adds a callback to execute when circuit breaker transitions to half-open + * + * The callback is run in the [[akka.dispatch.ExecutionContext]] supplied in the constructor. + * + * @param callback Handler to be invoked on state change + * @tparam T Type supplied to assist with type inference, otherwise ignored by implementation + * @return CircuitBreaker for fluent usage + */ + def onHalfOpen[T](callback: ⇒ T): CircuitBreaker = { + HalfOpen.addListener(() ⇒ callback) + this + } + + /** + * JavaAPI for onHalfOpen + * + * @param callback Handler to be invoked on state change + * @tparam T Type supplied to assist with type inference, otherwise ignored by implementation + * @return CircuitBreaker for fluent usage + */ + def onHalfOpen[T](callback: Callable[T]): CircuitBreaker = { + onHalfOpen(callback.call) + } + + /** + * Adds a callback to execute when circuit breaker state closes + * + * The callback is run in the [[akka.dispatch.ExecutionContext]] supplied in the constructor. + * + * @param callback Handler to be invoked on state change + * @tparam T Type supplied to assist with type inference, otherwise ignored by implementation + * @return CircuitBreaker for fluent usage + */ + def onClose[T](callback: ⇒ T): CircuitBreaker = { + Closed.addListener(() ⇒ callback) + this + } + + /** + * JavaAPI for onClose + * + * @param callback Handler to be invoked on state change + * @tparam T Type supplied to assist with type inference, otherwise ignored by implementation + * @return CircuitBreaker for fluent usage + */ + def onClose[T](callback: Callable[T]): CircuitBreaker = { + onClose(callback.call) + } + + /** + * Retrieves current failure count. + * + * @return count + */ + private[akka] def currentFailureCount: Int = Closed.get + + /** + * Implements consistent transition between states + * + * @param fromState State being transitioning from + * @param toState State being transitioning from + * @throws IllegalStateException if an invalid transition is attempted + */ + private def transition(fromState: State, toState: State): Unit = { + if (swapState(fromState, toState)) + toState.enter() + else + throw new IllegalStateException("Illegal transition attempted from: " + fromState + " to " + toState) + } + + /** + * Trips breaker to an open state. This is valid from Closed or Half-Open states. + * + * @param fromState State we're coming from (Closed or Half-Open) + */ + private def tripBreaker(fromState: State): Unit = { + transition(fromState, Open) + } + + /** + * Resets breaker to a closed state. This is valid from an Half-Open state only. + * + */ + private def resetBreaker(): Unit = { + transition(HalfOpen, Closed) + } + + /** + * Attempts to reset breaker by transitioning to a half-open state. This is valid from an Open state only. + * + */ + private def attemptReset(): Unit = { + transition(Open, HalfOpen) + } + + /** + * Internal state abstraction + */ + private sealed trait State { + private val listeners = new CopyOnWriteArrayList[() ⇒ _] + + /** + * Add a listener function which is invoked on state entry + * + * @param listener listener implementation + * @tparam T return type of listener, not used - but supplied for type inference purposes + */ + def addListener[T](listener: () ⇒ T) { + listeners add listener + } + + /** + * Test for whether listeners exist + * + * @return whether listeners exist + */ + private def hasListeners: Boolean = !listeners.isEmpty + + /** + * Notifies the listeners of the transition event via a Future executed in implicit parameter ExecutionContext + * + * @return Promise which executes listener in supplied [[akka.dispatch.ExecutionContext]] + */ + protected def notifyTransitionListeners() { + if (hasListeners) { + val iterator = listeners.iterator + while (iterator.hasNext) { + val listener = iterator.next + //FIXME per @viktorklang: it's a bit wasteful to create Futures for one-offs, just use EC.execute instead + Future(listener()) + } + } + } + + /** + * Shared implementation of call across all states. Thrown exception or execution of the call beyond the allowed + * call timeout is counted as a failed call, otherwise a successful call + * + * @param body Implementation of the call + * @tparam T Return type of the call's implementation + * @return Future containing the result of the call + */ + def callThrough[T](body: ⇒ Future[T]): Future[T] = { + val deadline = callTimeout.fromNow + val bodyFuture = try body catch { + case NonFatal(t) ⇒ Promise.failed(t) + } + bodyFuture onFailure { + case _ ⇒ callFails() + } onSuccess { + case _ ⇒ + if (deadline.isOverdue()) callFails() + else callSucceeds() + } + } + + /** + * Abstract entry point for all states + * + * @param body Implementation of the call that needs protected + * @tparam T Return type of protected call + * @return Future containing result of protected call + */ + def invoke[T](body: ⇒ Future[T]): Future[T] + + /** + * Invoked when call succeeds + * + */ + def callSucceeds(): Unit + + /** + * Invoked when call fails + * + */ + def callFails(): Unit + + /** + * Invoked on the transitioned-to state during transition. Notifies listeners after invoking subclass template + * method _enter + * + */ + final def enter(): Unit = { + _enter() + notifyTransitionListeners() + } + + /** + * Template method for concrete traits + * + */ + def _enter(): Unit + } + + /** + * Concrete implementation of Closed state + */ + private object Closed extends AtomicInteger with State { + + /** + * Implementation of invoke, which simply attempts the call + * + * @param body Implementation of the call that needs protected + * @tparam T Return type of protected call + * @return Future containing result of protected call + */ + override def invoke[T](body: ⇒ Future[T]): Future[T] = { + callThrough(body) + } + + /** + * On successful call, the failure count is reset to 0 + * + * @return + */ + override def callSucceeds(): Unit = { set(0) } + + /** + * On failed call, the failure count is incremented. The count is checked against the configured maxFailures, and + * the breaker is tripped if we have reached maxFailures. + * + * @return + */ + override def callFails(): Unit = { + if (incrementAndGet() == maxFailures) tripBreaker(Closed) + } + + /** + * On entry of this state, failure count is reset. + * + * @return + */ + override def _enter(): Unit = { + set(0) + } + + /** + * Override for more descriptive toString + * + * @return + */ + override def toString: String = { + "Closed with failure count = " + get() + } + } + + /** + * Concrete implementation of half-open state + */ + private object HalfOpen extends AtomicBoolean(true) with State { + + /** + * Allows a single call through, during which all other callers fail-fast. If the call fails, the breaker reopens. + * If the call succeeds the breaker closes. + * + * @param body Implementation of the call that needs protected + * @tparam T Return type of protected call + * @return Future containing result of protected call + */ + override def invoke[T](body: ⇒ Future[T]): Future[T] = { + if (compareAndSet(true, false)) + callThrough(body) + else + Promise.failed[T](new CircuitBreakerOpenException(Duration.Zero)) + } + + /** + * Reset breaker on successful call. + * + * @return + */ + override def callSucceeds(): Unit = { resetBreaker() } + + /** + * Reopen breaker on failed call. + * + * @return + */ + override def callFails(): Unit = { tripBreaker(HalfOpen) } + + /** + * On entry, guard should be reset for that first call to get in + * + * @return + */ + override def _enter(): Unit = { + set(true) + } + + /** + * Override for more descriptive toString + * + * @return + */ + override def toString: String = { + "Half-Open currently testing call for success = " + get() + } + } + + /** + * Concrete implementation of Open state + */ + private object Open extends AtomicLong with State { + + /** + * Fail-fast on any invocation + * + * @param body Implementation of the call that needs protected + * @tparam T Return type of protected call + * @return Future containing result of protected call + */ + override def invoke[T](body: ⇒ Future[T]): Future[T] = { + Promise.failed[T](new CircuitBreakerOpenException(remainingTimeout().timeLeft)) + } + + /** + * Calculate remaining timeout to inform the caller in case a backoff algorithm is useful + * + * @return [[akka.util.Deadline]] to when the breaker will attempt a reset by transitioning to half-open + */ + private def remainingTimeout(): Deadline = get match { + case 0L ⇒ Deadline.now + case t ⇒ (t.millis + resetTimeout).fromNow + } + + /** + * No-op for open, calls are never executed so cannot succeed or fail + * + * @return + */ + override def callSucceeds(): Unit = {} + + /** + * No-op for open, calls are never executed so cannot succeed or fail + * + * @return + */ + override def callFails(): Unit = {} + + /** + * On entering this state, schedule an attempted reset via [[akka.actor.Scheduler]] and store the entry time to + * calculate remaining time before attempted reset. + * + * @return + */ + override def _enter(): Unit = { + set(System.currentTimeMillis) + scheduler.scheduleOnce(resetTimeout) { + attemptReset() + } + } + + /** + * Override for more descriptive toString + * + * @return + */ + override def toString: String = { + "Open" + } + } + +} + +/** + * Exception thrown when Circuit Breaker is open. + * + * @param remainingDuration Stores remaining time before attempting a reset. Zero duration means the breaker is + * currently in half-open state. + * @param message Defaults to "Circuit Breaker is open; calls are failing fast" + */ +class CircuitBreakerOpenException( + val remainingDuration: Duration, + message: String = "Circuit Breaker is open; calls are failing fast") + extends AkkaException(message) with NoStackTrace diff --git a/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala b/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala index d6fbd31c1e..91293cb0d1 100644 --- a/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala @@ -4,9 +4,9 @@ package akka.pattern -import akka.actor.{ ActorRef, Actor, ActorSystem, Props, PoisonPill, Terminated, ReceiveTimeout, ActorTimeoutException } -import akka.dispatch.{ Promise, Future } -import akka.util.Duration +import akka.actor._ +import akka.util.{ Timeout, Duration } +import akka.dispatch.{ Unwatch, Watch, Promise, Future } trait GracefulStopSupport { /** @@ -14,34 +14,42 @@ trait GracefulStopSupport { * existing messages of the target actor has been processed and the actor has been * terminated. * - * Useful when you need to wait for termination or compose ordered termination of several actors. + * Useful when you need to wait for termination or compose ordered termination of several actors, + * which should only be done outside of the ActorSystem as blocking inside Actors is discouraged. + * + * IMPORTANT NOTICE: the actor being terminated and its supervisor + * being informed of the availability of the deceased actor’s name are two + * distinct operations, which do not obey any reliable ordering. Especially + * the following will NOT work: + * + * {{{ + * def receive = { + * case msg => + * Await.result(gracefulStop(someChild, timeout), timeout) + * context.actorOf(Props(...), "someChild") // assuming that that was someChild’s name, this will NOT work + * } + * }}} * * If the target actor isn't terminated within the timeout the [[akka.dispatch.Future]] - * is completed with failure [[akka.actor.ActorTimeoutException]]. + * is completed with failure [[akka.pattern.AskTimeoutException]]. */ def gracefulStop(target: ActorRef, timeout: Duration)(implicit system: ActorSystem): Future[Boolean] = { - if (target.isTerminated) { - Promise.successful(true) - } else { - val result = Promise[Boolean]() - system.actorOf(Props(new Actor { - // Terminated will be received when target has been stopped - context watch target - target ! PoisonPill - // ReceiveTimeout will be received if nothing else is received within the timeout - context setReceiveTimeout timeout - - def receive = { - case Terminated(a) if a == target ⇒ - result success true - context stop self - case ReceiveTimeout ⇒ - result failure new ActorTimeoutException( - "Failed to stop [%s] within [%s]".format(target.path, context.receiveTimeout)) - context stop self + if (target.isTerminated) Promise.successful(true) + else system match { + case e: ExtendedActorSystem ⇒ + val internalTarget = target.asInstanceOf[InternalActorRef] + val ref = PromiseActorRef(e.provider, Timeout(timeout)) + internalTarget.sendSystemMessage(Watch(target, ref)) + ref.result onComplete { // Just making sure we're not leaking here + case Right(Terminated(`target`)) ⇒ () + case _ ⇒ internalTarget.sendSystemMessage(Unwatch(target, ref)) } - })) - result + target ! PoisonPill + ref.result map { + case Terminated(`target`) ⇒ true + case _ ⇒ false + } + case s ⇒ throw new IllegalArgumentException("Unknown ActorSystem implementation: '" + s + "'") } } } \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/pattern/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala index b58e9a8fc1..853b46e318 100644 --- a/akka-actor/src/main/scala/akka/pattern/Patterns.scala +++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala @@ -18,7 +18,7 @@ object Patterns { * Sends a message asynchronously and returns a [[akka.dispatch.Future]] * holding the eventual reply message; this means that the target actor * needs to send the result to the `sender` reference provided. The Future - * will be completed with an [[akka.actor.AskTimeoutException]] after the + * will be completed with an [[akka.pattern.AskTimeoutException]] after the * given timeout has expired; this is independent from any timeout applied * while awaiting a result for this future (i.e. in * `Await.result(..., timeout)`). @@ -49,7 +49,7 @@ object Patterns { * Sends a message asynchronously and returns a [[akka.dispatch.Future]] * holding the eventual reply message; this means that the target actor * needs to send the result to the `sender` reference provided. The Future - * will be completed with an [[akka.actor.AskTimeoutException]] after the + * will be completed with an [[akka.pattern.AskTimeoutException]] after the * given timeout has expired; this is independent from any timeout applied * while awaiting a result for this future (i.e. in * `Await.result(..., timeout)`). @@ -100,7 +100,7 @@ object Patterns { * Useful when you need to wait for termination or compose ordered termination of several actors. * * If the target actor isn't terminated within the timeout the [[akka.dispatch.Future]] - * is completed with failure [[akka.actor.ActorTimeoutException]]. + * is completed with failure [[akka.pattern.AskTimeoutException]]. */ def gracefulStop(target: ActorRef, timeout: Duration, system: ActorSystem): Future[java.lang.Boolean] = scalaGracefulStop(target, timeout)(system).asInstanceOf[Future[java.lang.Boolean]] diff --git a/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala b/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala index 3136a2342d..9029c1f78b 100644 --- a/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala +++ b/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala @@ -10,10 +10,8 @@ import akka.actor._ * An Iterable that also contains a version. */ trait VersionedIterable[A] { - val version: Long - + def version: Long def iterable: Iterable[A] - def apply(): Iterable[A] = iterable } @@ -42,7 +40,7 @@ trait ConnectionManager { /** * Shuts the connection manager down, which stops all managed actors */ - def shutdown() + def shutdown(): Unit /** * Returns a VersionedIterator containing all connected ActorRefs at some moment in time. Since there is @@ -59,5 +57,5 @@ trait ConnectionManager { * * @param ref the dead */ - def remove(deadRef: ActorRef) + def remove(deadRef: ActorRef): Unit } diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index 130db2be3e..afa321d07d 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -23,7 +23,7 @@ class ConsistentHash[T](nodes: Seq[T], replicas: Int) { nodes.foreach(this += _) - def +=(node: T) { + def +=(node: T): Unit = { cluster += node (1 to replicas) foreach { replica ⇒ val key = hashFor((node + ":" + replica).getBytes("UTF-8")) @@ -32,7 +32,7 @@ class ConsistentHash[T](nodes: Seq[T], replicas: Int) { } } - def -=(node: T) { + def -=(node: T): Unit = { cluster -= node (1 to replicas) foreach { replica ⇒ val key = hashFor((node + ":" + replica).getBytes("UTF-8")) @@ -96,7 +96,7 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T private var hashvalue = h /** Begin a new hash using the same seed. */ - def reset() { + def reset(): Unit = { h = startHash(seed) c = hiddenMagicA k = hiddenMagicB @@ -104,7 +104,7 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T } /** Incorporate the hash value of one item. */ - def apply(t: T) { + def apply(t: T): Unit = { h = extendHash(h, t.##, c, k) c = nextMagicA(c) k = nextMagicB(k) @@ -112,7 +112,7 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T } /** Incorporate a known hash value. */ - def append(i: Int) { + def append(i: Int): Unit = { h = extendHash(h, i, c, k) c = nextMagicA(c) k = nextMagicB(k) @@ -120,14 +120,15 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T } /** Retrieve the hash value */ - def hash = { + def hash: Int = { if (!hashed) { hashvalue = finalizeHash(h) hashed = true } hashvalue } - override def hashCode = hash + + override def hashCode: Int = hash } /** @@ -143,35 +144,35 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T object MurmurHash { // Magic values used for MurmurHash's 32 bit hash. // Don't change these without consulting a hashing expert! - final private val visibleMagic = 0x971e137b - final private val hiddenMagicA = 0x95543787 - final private val hiddenMagicB = 0x2ad7eb25 - final private val visibleMixer = 0x52dce729 - final private val hiddenMixerA = 0x7b7d159c - final private val hiddenMixerB = 0x6bce6396 - final private val finalMixer1 = 0x85ebca6b - final private val finalMixer2 = 0xc2b2ae35 + final private val visibleMagic: Int = 0x971e137b + final private val hiddenMagicA: Int = 0x95543787 + final private val hiddenMagicB: Int = 0x2ad7eb25 + final private val visibleMixer: Int = 0x52dce729 + final private val hiddenMixerA: Int = 0x7b7d159c + final private val hiddenMixerB: Int = 0x6bce6396 + final private val finalMixer1: Int = 0x85ebca6b + final private val finalMixer2: Int = 0xc2b2ae35 // Arbitrary values used for hashing certain classes - final private val seedString = 0xf7ca7fd2 - final private val seedArray = 0x3c074a61 + final private val seedString: Int = 0xf7ca7fd2 + final private val seedArray: Int = 0x3c074a61 /** The first 23 magic integers from the first stream are stored here */ - val storedMagicA = + val storedMagicA: Array[Int] = Iterator.iterate(hiddenMagicA)(nextMagicA).take(23).toArray /** The first 23 magic integers from the second stream are stored here */ - val storedMagicB = + val storedMagicB: Array[Int] = Iterator.iterate(hiddenMagicB)(nextMagicB).take(23).toArray /** Begin a new hash with a seed value. */ - def startHash(seed: Int) = seed ^ visibleMagic + def startHash(seed: Int): Int = seed ^ visibleMagic /** The initial magic integers in the first stream. */ - def startMagicA = hiddenMagicA + def startMagicA: Int = hiddenMagicA /** The initial magic integer in the second stream. */ - def startMagicB = hiddenMagicB + def startMagicB: Int = hiddenMagicB /** * Incorporates a new value into an existing hash. @@ -182,18 +183,17 @@ object MurmurHash { * @param magicB a magic integer from a different stream * @return the updated hash value */ - def extendHash(hash: Int, value: Int, magicA: Int, magicB: Int) = { + def extendHash(hash: Int, value: Int, magicA: Int, magicB: Int): Int = (hash ^ rotl(value * magicA, 11) * magicB) * 3 + visibleMixer - } /** Given a magic integer from the first stream, compute the next */ - def nextMagicA(magicA: Int) = magicA * 5 + hiddenMixerA + def nextMagicA(magicA: Int): Int = magicA * 5 + hiddenMixerA /** Given a magic integer from the second stream, compute the next */ - def nextMagicB(magicB: Int) = magicB * 5 + hiddenMixerB + def nextMagicB(magicB: Int): Int = magicB * 5 + hiddenMixerB /** Once all hashes have been incorporated, this performs a final mixing */ - def finalizeHash(hash: Int) = { + def finalizeHash(hash: Int): Int = { var i = (hash ^ (hash >>> 16)) i *= finalMixer1 i ^= (i >>> 13) @@ -203,7 +203,7 @@ object MurmurHash { } /** Compute a high-quality hash of an array */ - def arrayHash[@specialized T](a: Array[T]) = { + def arrayHash[@specialized T](a: Array[T]): Int = { var h = startHash(a.length * seedArray) var c = hiddenMagicA var k = hiddenMagicB @@ -218,7 +218,7 @@ object MurmurHash { } /** Compute a high-quality hash of a string */ - def stringHash(s: String) = { + def stringHash(s: String): Int = { var h = startHash(s.length * seedString) var c = hiddenMagicA var k = hiddenMagicB @@ -239,7 +239,7 @@ object MurmurHash { * where the order of appearance of elements does not matter. * This is useful for hashing sets, for example. */ - def symmetricHash[T](xs: TraversableOnce[T], seed: Int) = { + def symmetricHash[T](xs: TraversableOnce[T], seed: Int): Int = { var a, b, n = 0 var c = 1 xs.foreach(i ⇒ { diff --git a/akka-actor/src/main/scala/akka/routing/Listeners.scala b/akka-actor/src/main/scala/akka/routing/Listeners.scala index 39fbf6355d..5ac02e2945 100644 --- a/akka-actor/src/main/scala/akka/routing/Listeners.scala +++ b/akka-actor/src/main/scala/akka/routing/Listeners.scala @@ -5,8 +5,7 @@ package akka.routing import akka.actor.{ Actor, ActorRef } -import java.util.concurrent.ConcurrentSkipListSet -import scala.collection.JavaConversions._ +import java.util.{ Set, TreeSet } sealed trait ListenerMessage case class Listen(listener: ActorRef) extends ListenerMessage @@ -25,13 +24,29 @@ case class WithListeners(f: (ActorRef) ⇒ Unit) extends ListenerMessage * Send WithListeners(fun) to traverse the current listeners. */ trait Listeners { self: Actor ⇒ - protected val listeners = new ConcurrentSkipListSet[ActorRef] + protected val listeners: Set[ActorRef] = new TreeSet[ActorRef] + /** + * Chain this into the receive function. + * + * {{ def receive = listenerManagement orElse … }} + */ protected def listenerManagement: Actor.Receive = { - case Listen(l) ⇒ listeners add l - case Deafen(l) ⇒ listeners remove l - case WithListeners(f) ⇒ listeners foreach f + case Listen(l) ⇒ listeners add l + case Deafen(l) ⇒ listeners remove l + case WithListeners(f) ⇒ + val i = listeners.iterator + while (i.hasNext) f(i.next) } - protected def gossip(msg: Any) = listeners foreach (_ ! msg) + /** + * Sends the supplied message to all current listeners using the provided sender as sender. + * + * @param msg + * @param sender + */ + protected def gossip(msg: Any)(implicit sender: ActorRef = null): Unit = { + val i = listeners.iterator + while (i.hasNext) i.next ! msg + } } diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index fdf14a5b96..cb0f5ee09b 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -6,14 +6,12 @@ package akka.routing import akka.actor._ import akka.util.Duration import akka.util.duration._ -import akka.config.ConfigurationException +import akka.ConfigurationException import akka.pattern.pipe -import akka.pattern.AskSupport import com.typesafe.config.Config import scala.collection.JavaConversions.iterableAsScalaIterable import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean } import java.util.concurrent.TimeUnit -import java.util.concurrent.locks.ReentrantLock import akka.jsr166y.ThreadLocalRandom import akka.util.Unsafe import akka.dispatch.Dispatchers @@ -25,36 +23,27 @@ import scala.runtime.ScalaRunTime * send a message to on (or more) of these actors. */ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath) - extends LocalActorRef( - _system, - _props.copy(creator = () ⇒ _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher), - _supervisor, - _path) { + extends RepointableActorRef(_system, _props, _supervisor, _path) { - /* - * CAUTION: RoutedActorRef is PROBLEMATIC - * ====================================== - * - * We are constructing/assembling the children outside of the scope of the - * Router actor, inserting them in its childrenRef list, which is not at all - * synchronized. This is done exactly once at start-up, all other accesses - * are done from the Router actor. This means that the only thing which is - * really hairy is making sure that the Router does not touch its childrenRefs - * before we are done with them: lock the monitor of the actor cell (hence the - * override of newActorCell) and use that to block the Router constructor for - * as long as it takes to setup the RoutedActorRef itself. - */ - override def newActorCell( - system: ActorSystemImpl, - ref: InternalActorRef, - props: Props, - supervisor: InternalActorRef, - receiveTimeout: Option[Duration]): ActorCell = - { - val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout) - Unsafe.instance.monitorEnter(cell) - cell - } + // verify that a BalancingDispatcher is not used with a Router + if (_props.routerConfig != NoRouter && _system.dispatchers.isBalancingDispatcher(_props.routerConfig.routerDispatcher)) { + throw new ConfigurationException( + "Configuration for " + this + + " is invalid - you can not use a 'BalancingDispatcher' as a Router's dispatcher, you can however use it for the routees.") + } + + _props.routerConfig.verifyConfig() + + override def newCell(): Cell = new RoutedActorCell(system, this, props, supervisor) + +} + +private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActorRef, _props: Props, _supervisor: InternalActorRef) + extends ActorCell( + _system, + _ref, + _props.copy(creator = () ⇒ _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher), + _supervisor) { private[akka] val routerConfig = _props.routerConfig private[akka] val routeeProps = _props.copy(routerConfig = NoRouter) @@ -69,31 +58,28 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup private var _routeeProvider: RouteeProvider = _ def routeeProvider = _routeeProvider - val route = - try { - _routeeProvider = routerConfig.createRouteeProvider(actorContext) - val r = routerConfig.createRoute(routeeProps, routeeProvider) - // initial resize, before message send - routerConfig.resizer foreach { r ⇒ - if (r.isTimeForResize(resizeCounter.getAndIncrement())) - r.resize(routeeProps, routeeProvider) - } - r - } finally { - assert(Thread.holdsLock(actorContext)) - Unsafe.instance.monitorExit(actorContext) // unblock Router’s constructor + val route = { + _routeeProvider = routerConfig.createRouteeProvider(this) + val r = routerConfig.createRoute(routeeProps, routeeProvider) + // initial resize, before message send + routerConfig.resizer foreach { r ⇒ + if (r.isTimeForResize(resizeCounter.getAndIncrement())) + r.resize(routeeProps, routeeProvider) } + r + } if (routerConfig.resizer.isEmpty && _routees.isEmpty) throw new ActorInitializationException("router " + routerConfig + " did not register routees!") + start() + /* * end of construction */ def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match { - case _: AutoReceivedMessage ⇒ Destination(this, this) :: Nil - case Terminated(_) ⇒ Destination(this, this) :: Nil + case _: AutoReceivedMessage ⇒ Destination(self, self) :: Nil case CurrentRoutees ⇒ sender ! RouterRoutees(_routees) Nil @@ -111,7 +97,7 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]): Unit = { _routees = _routees ++ newRoutees // subscribe to Terminated messages for all route destinations, to be handled by Router actor - newRoutees foreach underlying.watch + newRoutees foreach watch } /** @@ -122,13 +108,13 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup */ private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]): Unit = { _routees = _routees diff abandonedRoutees - abandonedRoutees foreach underlying.unwatch + abandonedRoutees foreach unwatch } - override def !(message: Any)(implicit sender: ActorRef = null): Unit = { + override def tell(message: Any, sender: ActorRef): Unit = { resize() - val s = if (sender eq null) underlying.system.deadLetters else sender + val s = if (sender eq null) system.deadLetters else sender val msg = message match { case Broadcast(m) ⇒ m @@ -136,15 +122,18 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup } applyRoute(s, message) match { - case Destination(_, x) :: Nil if x eq this ⇒ super.!(message)(s) - case refs ⇒ refs foreach (p ⇒ p.recipient.!(msg)(p.sender)) + case Destination(_, x) :: Nil if x == self ⇒ super.tell(message, s) + case refs ⇒ + refs foreach (p ⇒ + if (p.recipient == self) super.tell(msg, p.sender) + else p.recipient.!(msg)(p.sender)) } } def resize(): Unit = { for (r ← routerConfig.resizer) { if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeInProgress.compareAndSet(false, true)) - super.!(Router.Resize) + super.tell(Router.Resize, self) } } } @@ -171,7 +160,7 @@ trait RouterConfig { def createRoute(routeeProps: Props, routeeProvider: RouteeProvider): Route - def createRouteeProvider(context: ActorContext) = new RouteeProvider(context, resizer) + def createRouteeProvider(context: ActorContext): RouteeProvider = new RouteeProvider(context, resizer) def createActor(): Router = new Router { override def supervisorStrategy: SupervisorStrategy = RouterConfig.this.supervisorStrategy @@ -192,7 +181,8 @@ trait RouterConfig { */ def withFallback(other: RouterConfig): RouterConfig = this - protected def toAll(sender: ActorRef, routees: Iterable[ActorRef]): Iterable[Destination] = routees.map(Destination(sender, _)) + protected def toAll(sender: ActorRef, routees: Iterable[ActorRef]): Iterable[Destination] = + routees.map(Destination(sender, _)) /** * Routers with dynamically resizable number of routees return the [[akka.routing.Resizer]] @@ -200,6 +190,11 @@ trait RouterConfig { */ def resizer: Option[Resizer] = None + /** + * Check that everything is there which is needed. Called in constructor of RoutedActorRef to fail early. + */ + def verifyConfig(): Unit = {} + } /** @@ -215,9 +210,7 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) { * Not thread safe, but intended to be called from protected points, such as * `RouterConfig.createRoute` and `Resizer.resize`. */ - def registerRoutees(routees: IndexedSeq[ActorRef]): Unit = { - routedRef.addRoutees(routees) - } + def registerRoutees(routees: IndexedSeq[ActorRef]): Unit = routedCell.addRoutees(routees) /** * Adds the routees to the router. @@ -237,9 +230,7 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) { * Not thread safe, but intended to be called from protected points, such as * `Resizer.resize`. */ - def unregisterRoutees(routees: IndexedSeq[ActorRef]): Unit = { - routedRef.removeRoutees(routees) - } + def unregisterRoutees(routees: IndexedSeq[ActorRef]): Unit = routedCell.removeRoutees(routees) def createRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): IndexedSeq[ActorRef] = (nrOfInstances, routees) match { @@ -250,19 +241,15 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) { case (_, xs) ⇒ xs.map(context.actorFor(_))(scala.collection.breakOut) } - def createAndRegisterRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): Unit = { - if (resizer.isEmpty) { - registerRoutees(createRoutees(props, nrOfInstances, routees)) - } - } + def createAndRegisterRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): Unit = + if (resizer.isEmpty) registerRoutees(createRoutees(props, nrOfInstances, routees)) /** * All routees of the router */ - def routees: IndexedSeq[ActorRef] = routedRef.routees - - private def routedRef = context.self.asInstanceOf[RoutedActorRef] + def routees: IndexedSeq[ActorRef] = routedCell.routees + private def routedCell = context.asInstanceOf[RoutedActorCell] } /** @@ -294,19 +281,16 @@ trait CustomRoute { */ trait Router extends Actor { - // make sure that we synchronize properly to get the childrenRefs into our CPU cache - val ref = context.synchronized { - self match { - case x: RoutedActorRef ⇒ x - case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef") - } + val ref = context match { + case x: RoutedActorCell ⇒ x + case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef, not in " + context.getClass) } final def receive = ({ case Router.Resize ⇒ - try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider)) - finally assert(ref.resizeInProgress.getAndSet(false)) + val ab = ref.resizeInProgress + if (ab.get) try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider)) finally ab.set(false) case Terminated(child) ⇒ ref.removeRoutees(IndexedSeq(child)) @@ -321,6 +305,9 @@ trait Router extends Actor { } } +/** + * INTERNAL API + */ private object Router { case object Resize @@ -374,9 +361,9 @@ case class Destination(sender: ActorRef, recipient: ActorRef) //TODO add @SerialVersionUID(1L) when SI-4804 is fixed abstract class NoRouter extends RouterConfig case object NoRouter extends NoRouter { - def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null + def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null // FIXME, null, really?? def routerDispatcher: String = "" - def supervisorStrategy = null + def supervisorStrategy = null // FIXME null, really?? override def withFallback(other: RouterConfig): RouterConfig = other /** @@ -406,52 +393,24 @@ case object FromConfig extends FromConfig { //TODO add @SerialVersionUID(1L) when SI-4804 is fixed class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig - with Product - with Serializable - with Equals { + with Serializable { def this() = this(Dispatchers.DefaultDispatcherId) - def createRoute(props: Props, routeeProvider: RouteeProvider): Route = - throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)") + override def verifyConfig(): Unit = + throw new ConfigurationException("router needs external configuration from file (e.g. application.conf)") + + def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null def supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy - - // open-coded case class to preserve binary compatibility, all deprecated for 2.1 - @deprecated("FromConfig does not make sense as case class", "2.0.1") - override def productPrefix = "FromConfig" - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - def productArity = 1 - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - def productElement(x: Int) = x match { - case 0 ⇒ routerDispatcher - case _ ⇒ throw new IndexOutOfBoundsException(x.toString) - } - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - def copy(d: String = Dispatchers.DefaultDispatcherId): FromConfig = new FromConfig(d) - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - def canEqual(o: Any) = o.isInstanceOf[FromConfig] - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - override def hashCode = ScalaRunTime._hashCode(this) - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - override def toString = "FromConfig(" + routerDispatcher + ")" - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - override def equals(other: Any): Boolean = other match { - case FromConfig(x) ⇒ x == routerDispatcher - case _ ⇒ false - } - } object RoundRobinRouter { - def apply(routees: Iterable[ActorRef]) = new RoundRobinRouter(routees = routees map (_.path.toString)) + /** + * Creates a new RoundRobinRouter, routing to the specified routees + */ + def apply(routees: Iterable[ActorRef]): RoundRobinRouter = + new RoundRobinRouter(routees = routees map (_.path.toString)) /** * Java API to create router with the supplied 'routees' actors. @@ -512,9 +471,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int) = { - this(nrOfInstances = nr) - } + def this(nr: Int) = this(nrOfInstances = nr) /** * Constructor that sets the routees to be used. @@ -522,9 +479,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = { - this(routees = iterableAsScalaIterable(routeePaths)) - } + def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -535,13 +490,13 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = /** * Java API for setting routerDispatcher */ - def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + def withDispatcher(dispatcherId: String): RoundRobinRouter = copy(routerDispatcher = dispatcherId) /** * Java API for setting the supervisor strategy to be used for the “head” * Router actor. */ - def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) + def withSupervisorStrategy(strategy: SupervisorStrategy): RoundRobinRouter = copy(supervisorStrategy = strategy) } trait RoundRobinLike { this: RouterConfig ⇒ @@ -571,7 +526,10 @@ trait RoundRobinLike { this: RouterConfig ⇒ } object RandomRouter { - def apply(routees: Iterable[ActorRef]) = new RandomRouter(routees = routees map (_.path.toString)) + /** + * Creates a new RandomRouter, routing to the specified routees + */ + def apply(routees: Iterable[ActorRef]): RandomRouter = new RandomRouter(routees = routees map (_.path.toString)) /** * Java API to create router with the supplied 'routees' actors. @@ -632,9 +590,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int) = { - this(nrOfInstances = nr) - } + def this(nr: Int) = this(nrOfInstances = nr) /** * Constructor that sets the routees to be used. @@ -642,9 +598,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = { - this(routees = iterableAsScalaIterable(routeePaths)) - } + def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -655,13 +609,13 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, /** * Java API for setting routerDispatcher */ - def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + def withDispatcher(dispatcherId: String): RandomRouter = copy(routerDispatcher = dispatcherId) /** * Java API for setting the supervisor strategy to be used for the “head” * Router actor. */ - def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) + def withSupervisorStrategy(strategy: SupervisorStrategy): RandomRouter = copy(supervisorStrategy = strategy) } trait RandomLike { this: RouterConfig ⇒ @@ -688,7 +642,11 @@ trait RandomLike { this: RouterConfig ⇒ } object SmallestMailboxRouter { - def apply(routees: Iterable[ActorRef]) = new SmallestMailboxRouter(routees = routees map (_.path.toString)) + /** + * Creates a new SmallestMailboxRouter, routing to the specified routees + */ + def apply(routees: Iterable[ActorRef]): SmallestMailboxRouter = + new SmallestMailboxRouter(routees = routees map (_.path.toString)) /** * Java API to create router with the supplied 'routees' actors. @@ -758,9 +716,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int) = { - this(nrOfInstances = nr) - } + def this(nr: Int) = this(nrOfInstances = nr) /** * Constructor that sets the routees to be used. @@ -768,9 +724,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = { - this(routees = iterableAsScalaIterable(routeePaths)) - } + def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -781,19 +735,16 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin /** * Java API for setting routerDispatcher */ - def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + def withDispatcher(dispatcherId: String): SmallestMailboxRouter = copy(routerDispatcher = dispatcherId) /** * Java API for setting the supervisor strategy to be used for the “head” * Router actor. */ - def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) + def withSupervisorStrategy(strategy: SupervisorStrategy): SmallestMailboxRouter = copy(supervisorStrategy = strategy) } trait SmallestMailboxLike { this: RouterConfig ⇒ - - import java.security.SecureRandom - def nrOfInstances: Int def routees: Iterable[String] @@ -805,9 +756,11 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ * routers based on mailbox and actor internal state. */ protected def isProcessingMessage(a: ActorRef): Boolean = a match { - case x: LocalActorRef ⇒ - val cell = x.underlying - cell.mailbox.isScheduled && cell.currentMessage != null + case x: ActorRefWithCell ⇒ + x.underlying match { + case cell: ActorCell ⇒ cell.mailbox.isScheduled && cell.currentMessage != null + case _ ⇒ false + } case _ ⇒ false } @@ -819,8 +772,8 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ * routers based on mailbox and actor internal state. */ protected def hasMessages(a: ActorRef): Boolean = a match { - case x: LocalActorRef ⇒ x.underlying.mailbox.hasMessages - case _ ⇒ false + case x: ActorRefWithCell ⇒ x.underlying.hasMessages + case _ ⇒ false } /** @@ -830,8 +783,12 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ * routers based on mailbox and actor internal state. */ protected def isSuspended(a: ActorRef): Boolean = a match { - case x: LocalActorRef ⇒ x.underlying.mailbox.isSuspended - case _ ⇒ false + case x: ActorRefWithCell ⇒ + x.underlying match { + case cell: ActorCell ⇒ cell.mailbox.isSuspended + case _ ⇒ true + } + case _ ⇒ false } /** @@ -841,8 +798,8 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ * routers based on mailbox and actor internal state. */ protected def numberOfMessages(a: ActorRef): Int = a match { - case x: LocalActorRef ⇒ x.underlying.mailbox.numberOfMessages - case _ ⇒ 0 + case x: ActorRefWithCell ⇒ x.underlying.numberOfMessages + case _ ⇒ 0 } def createRoute(props: Props, routeeProvider: RouteeProvider): Route = { @@ -895,7 +852,10 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ } object BroadcastRouter { - def apply(routees: Iterable[ActorRef]) = new BroadcastRouter(routees = routees map (_.path.toString)) + /** + * Creates a new BroadcastRouter, routing to the specified routees + */ + def apply(routees: Iterable[ActorRef]): BroadcastRouter = new BroadcastRouter(routees = routees map (_.path.toString)) /** * Java API to create router with the supplied 'routees' actors. @@ -956,9 +916,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int) = { - this(nrOfInstances = nr) - } + def this(nr: Int) = this(nrOfInstances = nr) /** * Constructor that sets the routees to be used. @@ -966,9 +924,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = { - this(routees = iterableAsScalaIterable(routeePaths)) - } + def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -979,13 +935,13 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N /** * Java API for setting routerDispatcher */ - def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + def withDispatcher(dispatcherId: String): BroadcastRouter = copy(routerDispatcher = dispatcherId) /** * Java API for setting the supervisor strategy to be used for the “head” * Router actor. */ - def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) + def withSupervisorStrategy(strategy: SupervisorStrategy): BroadcastRouter = copy(supervisorStrategy = strategy) } trait BroadcastLike { this: RouterConfig ⇒ @@ -1004,7 +960,11 @@ trait BroadcastLike { this: RouterConfig ⇒ } object ScatterGatherFirstCompletedRouter { - def apply(routees: Iterable[ActorRef], within: Duration) = new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within) + /** + * Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration + */ + def apply(routees: Iterable[ActorRef], within: Duration): ScatterGatherFirstCompletedRouter = + new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within) /** * Java API to create router with the supplied 'routees' actors. @@ -1071,9 +1031,7 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int, w: Duration) = { - this(nrOfInstances = nr, within = w) - } + def this(nr: Int, w: Duration) = this(nrOfInstances = nr, within = w) /** * Constructor that sets the routees to be used. @@ -1081,9 +1039,8 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String], w: Duration) = { + def this(routeePaths: java.lang.Iterable[String], w: Duration) = this(routees = iterableAsScalaIterable(routeePaths), within = w) - } /** * Constructor that sets the resizer to be used. @@ -1152,10 +1109,14 @@ trait Resizer { * This method is invoked only in the context of the Router actor in order to safely * create/stop children. */ - def resize(props: Props, routeeProvider: RouteeProvider) + def resize(props: Props, routeeProvider: RouteeProvider): Unit } case object DefaultResizer { + + /** + * Creates a new DefaultResizer from the given configuration + */ def apply(resizerConfig: Config): DefaultResizer = DefaultResizer( lowerBound = resizerConfig.getInt("lower-bound"), @@ -1168,6 +1129,7 @@ case object DefaultResizer { messagesPerResize = resizerConfig.getInt("messages-per-resize")) } +//FIXME DOCUMENT ME case class DefaultResizer( /** * The fewest number of routees the router should ever have. @@ -1242,7 +1204,7 @@ case class DefaultResizer( def isTimeForResize(messageCounter: Long): Boolean = (messageCounter % messagesPerResize == 0) - def resize(props: Props, routeeProvider: RouteeProvider) { + def resize(props: Props, routeeProvider: RouteeProvider): Unit = { val currentRoutees = routeeProvider.routees val requestedCapacity = capacity(currentRoutees) @@ -1260,7 +1222,7 @@ case class DefaultResizer( * Give concurrent messages a chance to be placed in mailbox before * sending PoisonPill. */ - protected def delayedStop(scheduler: Scheduler, abandon: IndexedSeq[ActorRef]) { + protected def delayedStop(scheduler: Scheduler, abandon: IndexedSeq[ActorRef]): Unit = { if (abandon.nonEmpty) { if (stopDelay <= Duration.Zero) { abandon foreach (_ ! PoisonPill) @@ -1309,12 +1271,20 @@ case class DefaultResizer( */ def pressure(routees: IndexedSeq[ActorRef]): Int = { routees count { - case a: LocalActorRef ⇒ - val cell = a.underlying - pressureThreshold match { - case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.hasMessages - case i if i < 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null - case threshold ⇒ cell.mailbox.numberOfMessages >= threshold + case a: ActorRefWithCell ⇒ + a.underlying match { + case cell: ActorCell ⇒ + pressureThreshold match { + case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.hasMessages + case i if i < 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null + case threshold ⇒ cell.mailbox.numberOfMessages >= threshold + } + case cell ⇒ + pressureThreshold match { + case 1 ⇒ cell.hasMessages + case i if i < 1 ⇒ true // unstarted cells are always busy, for example + case threshold ⇒ cell.numberOfMessages >= threshold + } } case x ⇒ false @@ -1329,9 +1299,7 @@ case class DefaultResizer( * @param capacity current number of routees * @return proposed change in the capacity */ - def filter(pressure: Int, capacity: Int): Int = { - rampup(pressure, capacity) + backoff(pressure, capacity) - } + def filter(pressure: Int, capacity: Int): Int = rampup(pressure, capacity) + backoff(pressure, capacity) /** * Computes a proposed positive (or zero) capacity delta using diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index ce0f56a238..7355e4f7fb 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -14,8 +14,6 @@ import akka.util.NonFatal import scala.collection.mutable.ArrayBuffer import java.io.NotSerializableException -case class NoSerializerFoundException(m: String) extends AkkaException(m) - object Serialization { /** @@ -85,7 +83,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { /** * Returns the Serializer configured for the given object, returns the NullSerializer if it's null. * - * @throws akka.config.ConfigurationException if no `serialization-bindings` is configured for the + * @throws akka.ConfigurationException if no `serialization-bindings` is configured for the * class of the object */ def findSerializerFor(o: AnyRef): Serializer = o match { @@ -120,9 +118,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { possibilities(0)._2 } serializerMap.putIfAbsent(clazz, ser) match { - case null ⇒ - log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName) - ser + case null ⇒ log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName); ser case some ⇒ some } case ser ⇒ ser @@ -140,10 +136,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer) * By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer */ - private val serializers: Map[String, Serializer] = { - for ((k: String, v: String) ← settings.Serializers) - yield k -> serializerOf(v).fold(throw _, identity) - } + private val serializers: Map[String, Serializer] = + for ((k: String, v: String) ← settings.Serializers) yield k -> serializerOf(v).fold(throw _, identity) /** * bindings is a Seq of tuple representing the mapping from Class to Serializer. diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index 5696201f62..f6300ca998 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -6,7 +6,6 @@ package akka.serialization import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream } import akka.util.ClassLoaderObjectInputStream -import akka.actor.DynamicAccess import akka.actor.ExtendedActorSystem import scala.util.DynamicVariable diff --git a/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala b/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala index 7eb90b8ef0..c7c8308de0 100644 --- a/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala +++ b/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala @@ -8,6 +8,12 @@ import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.{ TimeUnit, BlockingQueue } import java.util.{ AbstractQueue, Queue, Collection, Iterator } +/** + * BoundedBlockingQueue wraps any Queue and turns the result into a BlockingQueue with a limited capacity + * @param maxCapacity - the maximum capacity of this Queue, needs to be > 0 + * @param backing - the backing Queue + * @tparam E - The type of the contents of this Queue + */ class BoundedBlockingQueue[E <: AnyRef]( val maxCapacity: Int, private val backing: Queue[E]) extends AbstractQueue[E] with BlockingQueue[E] { @@ -22,7 +28,7 @@ class BoundedBlockingQueue[E <: AnyRef]( require(maxCapacity > 0) } - protected val lock = new ReentrantLock(false) + protected val lock = new ReentrantLock(false) // TODO might want to switch to ReentrantReadWriteLock private val notEmpty = lock.newCondition() private val notFull = lock.newCondition() diff --git a/akka-actor/src/main/scala/akka/util/ByteString.scala b/akka-actor/src/main/scala/akka/util/ByteString.scala index b0dc19d49f..c2a690aa26 100644 --- a/akka-actor/src/main/scala/akka/util/ByteString.scala +++ b/akka-actor/src/main/scala/akka/util/ByteString.scala @@ -55,14 +55,14 @@ object ByteString { def newBuilder: ByteStringBuilder = new ByteStringBuilder - implicit def canBuildFrom: CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] = + implicit val canBuildFrom: CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] = new CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] { - def apply(from: TraversableOnce[Byte]) = newBuilder - def apply() = newBuilder + def apply(ignore: TraversableOnce[Byte]): ByteStringBuilder = newBuilder + def apply(): ByteStringBuilder = newBuilder } private[akka] object ByteString1C { - def apply(bytes: Array[Byte]) = new ByteString1C(bytes) + def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes) } /** @@ -181,9 +181,9 @@ object ByteString { // 0: both empty, 1: 2nd empty, 2: 1st empty, 3: neither empty def compare(b1: ByteString, b2: ByteString): Int = - if (b1.length == 0) - if (b2.length == 0) 0 else 2 - else if (b2.length == 0) 1 else 3 + if (b1.isEmpty) + if (b2.isEmpty) 0 else 2 + else if (b2.isEmpty) 1 else 3 } diff --git a/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala b/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala index 3ad55d69eb..ab2514861e 100644 --- a/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala +++ b/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala @@ -6,6 +6,13 @@ package akka.util import java.io.{ InputStream, ObjectInputStream, ObjectStreamClass } +/** + * ClassLoaderObjectInputStream tries to utilize the provided ClassLoader to load Classes and falls + * back to ObjectInputStreams resolver. + * + * @param classLoader - the ClassLoader which is to be used primarily + * @param is - the InputStream that is wrapped + */ class ClassLoaderObjectInputStream(classLoader: ClassLoader, is: InputStream) extends ObjectInputStream(is) { override protected def resolveClass(objectStreamClass: ObjectStreamClass): Class[_] = try Class.forName(objectStreamClass.getName, false, classLoader) catch { diff --git a/akka-actor/src/main/scala/akka/util/Convert.scala b/akka-actor/src/main/scala/akka/util/Convert.scala index a805b17fb2..3fead7aef7 100644 --- a/akka-actor/src/main/scala/akka/util/Convert.scala +++ b/akka-actor/src/main/scala/akka/util/Convert.scala @@ -3,7 +3,7 @@ */ package akka.util - +//FIXME DOCS! object Convert { def intToBytes(value: Int): Array[Byte] = { diff --git a/akka-actor/src/main/scala/akka/util/Crypt.scala b/akka-actor/src/main/scala/akka/util/Crypt.scala index 7dd678e748..280cd90768 100644 --- a/akka-actor/src/main/scala/akka/util/Crypt.scala +++ b/akka-actor/src/main/scala/akka/util/Crypt.scala @@ -5,7 +5,7 @@ package akka.util import java.security.{ MessageDigest, SecureRandom } - +//FIXME DOCS object Crypt { val hex = "0123456789ABCDEF" val lineSeparator = System.getProperty("line.separator") @@ -32,7 +32,7 @@ object Crypt { } def hexify(bytes: Array[Byte]): String = { - val builder = new StringBuilder + val builder = new StringBuilder(bytes.length * 2) bytes.foreach { byte ⇒ builder.append(hex.charAt((byte & 0xF0) >> 4)).append(hex.charAt(byte & 0xF)) } builder.toString } diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index a213fe1869..3a1c2e80c8 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -9,16 +9,22 @@ import TimeUnit._ import java.lang.{ Double ⇒ JDouble } //TODO add @SerialVersionUID(1L) when SI-4804 is fixed -case class Deadline private (time: Duration) { +case class Deadline private (time: Duration) extends Ordered[Deadline] { def +(other: Duration): Deadline = copy(time = time + other) def -(other: Duration): Deadline = copy(time = time - other) def -(other: Deadline): Duration = time - other.time def timeLeft: Duration = this - Deadline.now def hasTimeLeft(): Boolean = !isOverdue() //Code reuse FTW def isOverdue(): Boolean = (time.toNanos - System.nanoTime()) < 0 + def compare(that: Deadline) = this.time compare that.time } + object Deadline { def now: Deadline = Deadline(Duration(System.nanoTime, NANOSECONDS)) + + implicit object DeadlineIsOrdered extends Ordering[Deadline] { + def compare(a: Deadline, b: Deadline) = a compare b + } } object Duration { @@ -110,6 +116,7 @@ object Duration { } val Zero: FiniteDuration = new FiniteDuration(0, NANOSECONDS) + val Undefined: Duration = new Duration with Infinite { override def toString = "Duration.Undefined" override def equals(other: Any) = other.asInstanceOf[AnyRef] eq this @@ -166,8 +173,8 @@ object Duration { * including itself. */ val Inf: Duration = new Duration with Infinite { - override def toString = "Duration.Inf" - def compare(other: Duration) = if (other eq this) 0 else 1 + override def toString: String = "Duration.Inf" + def compare(other: Duration): Int = if (other eq this) 0 else 1 def unary_- : Duration = MinusInf } @@ -177,7 +184,7 @@ object Duration { */ val MinusInf: Duration = new Duration with Infinite { override def toString = "Duration.MinusInf" - def compare(other: Duration) = if (other eq this) 0 else -1 + def compare(other: Duration): Int = if (other eq this) 0 else -1 def unary_- : Duration = Inf } @@ -188,7 +195,7 @@ object Duration { def parse(s: String): Duration = unapply(s).get implicit object DurationIsOrdered extends Ordering[Duration] { - def compare(a: Duration, b: Duration) = a compare b + def compare(a: Duration, b: Duration): Int = a compare b } } @@ -263,17 +270,17 @@ abstract class Duration extends Serializable with Ordered[Duration] { def fromNow: Deadline = Deadline.now + this // Java API - def lt(other: Duration) = this < other - def lteq(other: Duration) = this <= other - def gt(other: Duration) = this > other - def gteq(other: Duration) = this >= other - def plus(other: Duration) = this + other - def minus(other: Duration) = this - other - def mul(factor: Double) = this * factor - def div(factor: Double) = this / factor - def div(other: Duration) = this / other - def neg() = -this - def isFinite() = finite_? + def lt(other: Duration): Boolean = this < other + def lteq(other: Duration): Boolean = this <= other + def gt(other: Duration): Boolean = this > other + def gteq(other: Duration): Boolean = this >= other + def plus(other: Duration): Duration = this + other + def minus(other: Duration): Duration = this - other + def mul(factor: Double): Duration = this * factor + def div(factor: Double): Duration = this / factor + def div(other: Duration): Double = this / other + def neg(): Duration = -this + def isFinite(): Boolean = finite_? } object FiniteDuration { @@ -349,31 +356,19 @@ class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration { else c } - def +(other: Duration) = { - if (!other.finite_?) { - other - } else { - fromNanos(add(toNanos, other.toNanos)) - } - } + def +(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, other.toNanos)) - def -(other: Duration) = { - if (!other.finite_?) { - other - } else { - fromNanos(add(toNanos, -other.toNanos)) - } - } + def -(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, -other.toNanos)) - def *(factor: Double) = fromNanos(long2double(toNanos) * factor) + def *(factor: Double): FiniteDuration = fromNanos(long2double(toNanos) * factor) - def /(factor: Double) = fromNanos(long2double(toNanos) / factor) + def /(factor: Double): FiniteDuration = fromNanos(long2double(toNanos) / factor) - def /(other: Duration) = if (other.finite_?) long2double(toNanos) / other.toNanos else 0 + def /(other: Duration): Double = if (other.finite_?) long2double(toNanos) / other.toNanos else 0 - def unary_- = Duration(-length, unit) + def unary_- : FiniteDuration = Duration(-length, unit) - def finite_? = true + def finite_? : Boolean = true override def equals(other: Any) = (other.asInstanceOf[AnyRef] eq this) || other.isInstanceOf[FiniteDuration] && @@ -385,178 +380,74 @@ class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration { } } -class DurationInt(n: Int) { +private[akka] trait DurationOps { import duration.Classifier + protected def from(timeUnit: TimeUnit): FiniteDuration + def nanoseconds: FiniteDuration = from(NANOSECONDS) + def nanos: FiniteDuration = from(NANOSECONDS) + def nanosecond: FiniteDuration = from(NANOSECONDS) + def nano: FiniteDuration = from(NANOSECONDS) - def nanoseconds = Duration(n, NANOSECONDS) - def nanos = Duration(n, NANOSECONDS) - def nanosecond = Duration(n, NANOSECONDS) - def nano = Duration(n, NANOSECONDS) + def microseconds: FiniteDuration = from(MICROSECONDS) + def micros: FiniteDuration = from(MICROSECONDS) + def microsecond: FiniteDuration = from(MICROSECONDS) + def micro: FiniteDuration = from(MICROSECONDS) - def microseconds = Duration(n, MICROSECONDS) - def micros = Duration(n, MICROSECONDS) - def microsecond = Duration(n, MICROSECONDS) - def micro = Duration(n, MICROSECONDS) + def milliseconds: FiniteDuration = from(MILLISECONDS) + def millis: FiniteDuration = from(MILLISECONDS) + def millisecond: FiniteDuration = from(MILLISECONDS) + def milli: FiniteDuration = from(MILLISECONDS) - def milliseconds = Duration(n, MILLISECONDS) - def millis = Duration(n, MILLISECONDS) - def millisecond = Duration(n, MILLISECONDS) - def milli = Duration(n, MILLISECONDS) + def seconds: FiniteDuration = from(SECONDS) + def second: FiniteDuration = from(SECONDS) - def seconds = Duration(n, SECONDS) - def second = Duration(n, SECONDS) + def minutes: FiniteDuration = from(MINUTES) + def minute: FiniteDuration = from(MINUTES) - def minutes = Duration(n, MINUTES) - def minute = Duration(n, MINUTES) + def hours: FiniteDuration = from(HOURS) + def hour: FiniteDuration = from(HOURS) - def hours = Duration(n, HOURS) - def hour = Duration(n, HOURS) + def days: FiniteDuration = from(DAYS) + def day: FiniteDuration = from(DAYS) - def days = Duration(n, DAYS) - def day = Duration(n, DAYS) + def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS)) + def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS)) + def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS)) + def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS)) - def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) + def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS)) + def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS)) + def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS)) + def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS)) - def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) + def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS)) + def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS)) + def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS)) + def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS)) - def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) + def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS)) + def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS)) - def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS)) - def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS)) + def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MINUTES)) + def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MINUTES)) - def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES)) - def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES)) + def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(HOURS)) + def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(HOURS)) - def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS)) - def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS)) - - def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS)) - def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS)) + def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS)) + def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS)) } -class DurationLong(n: Long) { - import duration.Classifier - - def nanoseconds = Duration(n, NANOSECONDS) - def nanos = Duration(n, NANOSECONDS) - def nanosecond = Duration(n, NANOSECONDS) - def nano = Duration(n, NANOSECONDS) - - def microseconds = Duration(n, MICROSECONDS) - def micros = Duration(n, MICROSECONDS) - def microsecond = Duration(n, MICROSECONDS) - def micro = Duration(n, MICROSECONDS) - - def milliseconds = Duration(n, MILLISECONDS) - def millis = Duration(n, MILLISECONDS) - def millisecond = Duration(n, MILLISECONDS) - def milli = Duration(n, MILLISECONDS) - - def seconds = Duration(n, SECONDS) - def second = Duration(n, SECONDS) - - def minutes = Duration(n, MINUTES) - def minute = Duration(n, MINUTES) - - def hours = Duration(n, HOURS) - def hour = Duration(n, HOURS) - - def days = Duration(n, DAYS) - def day = Duration(n, DAYS) - - def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - - def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - - def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - - def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS)) - def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS)) - - def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES)) - def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES)) - - def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS)) - def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS)) - - def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS)) - def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS)) +class DurationInt(n: Int) extends DurationOps { + override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit) } -class DurationDouble(d: Double) { - import duration.Classifier +class DurationLong(n: Long) extends DurationOps { + override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit) +} - def nanoseconds = Duration(d, NANOSECONDS) - def nanos = Duration(d, NANOSECONDS) - def nanosecond = Duration(d, NANOSECONDS) - def nano = Duration(d, NANOSECONDS) - - def microseconds = Duration(d, MICROSECONDS) - def micros = Duration(d, MICROSECONDS) - def microsecond = Duration(d, MICROSECONDS) - def micro = Duration(d, MICROSECONDS) - - def milliseconds = Duration(d, MILLISECONDS) - def millis = Duration(d, MILLISECONDS) - def millisecond = Duration(d, MILLISECONDS) - def milli = Duration(d, MILLISECONDS) - - def seconds = Duration(d, SECONDS) - def second = Duration(d, SECONDS) - - def minutes = Duration(d, MINUTES) - def minute = Duration(d, MINUTES) - - def hours = Duration(d, HOURS) - def hour = Duration(d, HOURS) - - def days = Duration(d, DAYS) - def day = Duration(d, DAYS) - - def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS)) - def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS)) - def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS)) - def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS)) - - def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS)) - def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS)) - def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS)) - def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS)) - - def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS)) - def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS)) - def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS)) - def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS)) - - def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, SECONDS)) - def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, SECONDS)) - - def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MINUTES)) - def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MINUTES)) - - def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, HOURS)) - def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, HOURS)) - - def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS)) - def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS)) +class DurationDouble(d: Double) extends DurationOps { + override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(d, timeUnit) } //TODO add @SerialVersionUID(1L) when SI-4804 is fixed @@ -565,24 +456,27 @@ case class Timeout(duration: Duration) { def this(length: Long, unit: TimeUnit) = this(Duration(length, unit)) } +/** + * A Timeout is a wrapper on top of Duration to be more specific about what the duration means. + */ object Timeout { /** * A timeout with zero duration, will cause most requests to always timeout. */ - val zero = new Timeout(Duration.Zero) + val zero: Timeout = new Timeout(Duration.Zero) /** * A Timeout with infinite duration. Will never timeout. Use extreme caution with this * as it may cause memory leaks, blocked threads, or may not even be supported by * the receiver, which would result in an exception. */ - val never = new Timeout(Duration.Inf) + val never: Timeout = new Timeout(Duration.Inf) - def apply(timeout: Long) = new Timeout(timeout) - def apply(length: Long, unit: TimeUnit) = new Timeout(length, unit) + def apply(timeout: Long): Timeout = new Timeout(timeout) + def apply(length: Long, unit: TimeUnit): Timeout = new Timeout(length, unit) - implicit def durationToTimeout(duration: Duration) = new Timeout(duration) - implicit def intToTimeout(timeout: Int) = new Timeout(timeout) - implicit def longToTimeout(timeout: Long) = new Timeout(timeout) + implicit def durationToTimeout(duration: Duration): Timeout = new Timeout(duration) + implicit def intToTimeout(timeout: Int): Timeout = new Timeout(timeout) + implicit def longToTimeout(timeout: Long): Timeout = new Timeout(timeout) } diff --git a/akka-actor/src/main/scala/akka/util/Helpers.scala b/akka-actor/src/main/scala/akka/util/Helpers.scala index 25cb279f2e..a3618359ac 100644 --- a/akka-actor/src/main/scala/akka/util/Helpers.scala +++ b/akka-actor/src/main/scala/akka/util/Helpers.scala @@ -45,18 +45,13 @@ object Helpers { else base64(next, sb) } - def ignore[E: Manifest](body: ⇒ Unit) { - try { - body - } catch { - case e if manifest[E].erasure.isAssignableFrom(e.getClass) ⇒ () - } - } + //FIXME docs + def ignore[E: Manifest](body: ⇒ Unit): Unit = + try body catch { case e if manifest[E].erasure.isAssignableFrom(e.getClass) ⇒ () } - def withPrintStackTraceOnError(body: ⇒ Unit) { - try { - body - } catch { + //FIXME docs + def withPrintStackTraceOnError(body: ⇒ Unit): Unit = { + try body catch { case e: Throwable ⇒ val sw = new java.io.StringWriter() var root = e diff --git a/akka-actor/src/main/scala/akka/util/Index.scala b/akka-actor/src/main/scala/akka/util/Index.scala index 1153c9e045..3289ed8f13 100644 --- a/akka-actor/src/main/scala/akka/util/Index.scala +++ b/akka-actor/src/main/scala/akka/util/Index.scala @@ -91,7 +91,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { /** * Applies the supplied function to all keys and their values */ - def foreach(fun: (K, V) ⇒ Unit) { + def foreach(fun: (K, V) ⇒ Unit): Unit = { import scala.collection.JavaConversions._ container.entrySet foreach { e ⇒ e.getValue.foreach(fun(e.getKey, _)) } } @@ -112,7 +112,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { /** * Returns the key set. */ - def keys = scala.collection.JavaConversions.collectionAsScalaIterable(container.keySet) + def keys: Iterable[K] = scala.collection.JavaConversions.collectionAsScalaIterable(container.keySet) /** * Disassociates the value of type V from the key of type K diff --git a/akka-actor/src/main/scala/akka/util/LockUtil.scala b/akka-actor/src/main/scala/akka/util/LockUtil.scala index c3295d4b52..c465002b70 100644 --- a/akka-actor/src/main/scala/akka/util/LockUtil.scala +++ b/akka-actor/src/main/scala/akka/util/LockUtil.scala @@ -7,17 +7,12 @@ package akka.util import java.util.concurrent.locks.{ ReentrantLock } import java.util.concurrent.atomic.{ AtomicBoolean } -final class ReentrantGuard { - final val lock = new ReentrantLock +final class ReentrantGuard extends ReentrantLock { @inline final def withGuard[T](body: ⇒ T): T = { - lock.lock - try { - body - } finally { - lock.unlock - } + lock() + try body finally unlock() } } @@ -25,13 +20,11 @@ final class ReentrantGuard { * An atomic switch that can be either on or off */ class Switch(startAsOn: Boolean = false) { - private val switch = new AtomicBoolean(startAsOn) + private val switch = new AtomicBoolean(startAsOn) // FIXME switch to AQS protected def transcend(from: Boolean, action: ⇒ Unit): Boolean = synchronized { if (switch.compareAndSet(from, !from)) { - try { - action - } catch { + try action catch { case e ⇒ switch.compareAndSet(!from, from) // revert status throw e @@ -67,18 +60,12 @@ class Switch(startAsOn: Boolean = false) { /** * Executes the provided action and returns its value if the switch is IMMEDIATELY on (i.e. no lock involved) */ - def ifOnYield[T](action: ⇒ T): Option[T] = { - if (switch.get) Some(action) - else None - } + def ifOnYield[T](action: ⇒ T): Option[T] = if (switch.get) Some(action) else None /** * Executes the provided action and returns its value if the switch is IMMEDIATELY off (i.e. no lock involved) */ - def ifOffYield[T](action: ⇒ T): Option[T] = { - if (!switch.get) Some(action) - else None - } + def ifOffYield[T](action: ⇒ T): Option[T] = if (!switch.get) Some(action) else None /** * Executes the provided action and returns if the action was executed or not, if the switch is IMMEDIATELY on (i.e. no lock involved) @@ -104,19 +91,13 @@ class Switch(startAsOn: Boolean = false) { * Executes the provided action and returns its value if the switch is on, waiting for any pending changes to happen before (locking) * Be careful of longrunning or blocking within the provided action as it can lead to deadlocks or bad performance */ - def whileOnYield[T](action: ⇒ T): Option[T] = synchronized { - if (switch.get) Some(action) - else None - } + def whileOnYield[T](action: ⇒ T): Option[T] = synchronized { if (switch.get) Some(action) else None } /** * Executes the provided action and returns its value if the switch is off, waiting for any pending changes to happen before (locking) * Be careful of longrunning or blocking within the provided action as it can lead to deadlocks or bad performance */ - def whileOffYield[T](action: ⇒ T): Option[T] = synchronized { - if (!switch.get) Some(action) - else None - } + def whileOffYield[T](action: ⇒ T): Option[T] = synchronized { if (!switch.get) Some(action) else None } /** * Executes the provided action and returns if the action was executed or not, if the switch is on, waiting for any pending changes to happen before (locking) @@ -144,22 +125,20 @@ class Switch(startAsOn: Boolean = false) { * Executes the provided callbacks depending on if the switch is either on or off waiting for any pending changes to happen before (locking) * Be careful of longrunning or blocking within the provided action as it can lead to deadlocks or bad performance */ - def fold[T](on: ⇒ T)(off: ⇒ T) = synchronized { - if (switch.get) on else off - } + def fold[T](on: ⇒ T)(off: ⇒ T): T = synchronized { if (switch.get) on else off } /** * Executes the given code while holding this switch’s lock, i.e. protected from concurrent modification of the switch status. */ - def locked[T](code: ⇒ T) = synchronized { code } + def locked[T](code: ⇒ T): T = synchronized { code } /** * Returns whether the switch is IMMEDIATELY on (no locking) */ - def isOn = switch.get + def isOn: Boolean = switch.get /** * Returns whether the switch is IMMEDDIATELY off (no locking) */ - def isOff = !isOn + def isOff: Boolean = !isOn } diff --git a/akka-actor/src/main/scala/akka/util/Reflect.scala b/akka-actor/src/main/scala/akka/util/Reflect.scala index 25c56a983f..3a46edeab1 100644 --- a/akka-actor/src/main/scala/akka/util/Reflect.scala +++ b/akka-actor/src/main/scala/akka/util/Reflect.scala @@ -6,8 +6,10 @@ package akka.util /** * Collection of internal reflection utilities which may or may not be * available (most services specific to HotSpot, but fails gracefully). + * + * INTERNAL API */ -object Reflect { +private[akka] object Reflect { /** * This optionally holds a function which looks N levels above itself diff --git a/akka-actor/src/main/scala/akka/util/Unsafe.java b/akka-actor/src/main/scala/akka/util/Unsafe.java index 608cb3d46e..ace3c1baac 100644 --- a/akka-actor/src/main/scala/akka/util/Unsafe.java +++ b/akka-actor/src/main/scala/akka/util/Unsafe.java @@ -7,6 +7,9 @@ package akka.util; import java.lang.reflect.Field; +/** + * INTERNAL API + */ public final class Unsafe { public final static sun.misc.Unsafe instance; static { diff --git a/akka-actor/src/main/scala/akka/util/cps/package.scala b/akka-actor/src/main/scala/akka/util/cps/package.scala index 198c2beacd..a1b4bc39eb 100644 --- a/akka-actor/src/main/scala/akka/util/cps/package.scala +++ b/akka-actor/src/main/scala/akka/util/cps/package.scala @@ -7,6 +7,7 @@ package akka.util import scala.util.continuations._ import akka.dispatch.MessageDispatcher +//FIXME Needs docs package object cps { def matchC[A, B, C, D](in: A)(pf: PartialFunction[A, B @cpsParam[C, D]]): B @cpsParam[C, D] = pf(in) diff --git a/akka-actor/src/main/scala/akka/util/duration/package.scala b/akka-actor/src/main/scala/akka/util/duration/package.scala index 7f14a0be48..6a7d28a6e6 100644 --- a/akka-actor/src/main/scala/akka/util/duration/package.scala +++ b/akka-actor/src/main/scala/akka/util/duration/package.scala @@ -5,7 +5,7 @@ package akka.util import java.util.concurrent.TimeUnit - +//FIXME Needs docs package object duration { trait Classifier[C] { type R @@ -15,38 +15,32 @@ package object duration { object span implicit object spanConvert extends Classifier[span.type] { type R = FiniteDuration - def convert(d: FiniteDuration) = d + def convert(d: FiniteDuration): FiniteDuration = d } object fromNow implicit object fromNowConvert extends Classifier[fromNow.type] { type R = Deadline - def convert(d: FiniteDuration) = Deadline.now + d + def convert(d: FiniteDuration): Deadline = Deadline.now + d } - implicit def intToDurationInt(n: Int) = new DurationInt(n) - implicit def longToDurationLong(n: Long) = new DurationLong(n) - implicit def doubleToDurationDouble(d: Double) = new DurationDouble(d) + implicit def intToDurationInt(n: Int): DurationInt = new DurationInt(n) + implicit def longToDurationLong(n: Long): DurationLong = new DurationLong(n) + implicit def doubleToDurationDouble(d: Double): DurationDouble = new DurationDouble(d) - implicit def pairIntToDuration(p: (Int, TimeUnit)) = Duration(p._1, p._2) - implicit def pairLongToDuration(p: (Long, TimeUnit)) = Duration(p._1, p._2) - implicit def durationToPair(d: Duration) = (d.length, d.unit) + implicit def pairIntToDuration(p: (Int, TimeUnit)): FiniteDuration = Duration(p._1, p._2) + implicit def pairLongToDuration(p: (Long, TimeUnit)): FiniteDuration = Duration(p._1, p._2) + implicit def durationToPair(d: Duration): (Long, TimeUnit) = (d.length, d.unit) /* * avoid reflection based invocation by using non-duck type */ - class IntMult(i: Int) { - def *(d: Duration) = d * i - } - implicit def intMult(i: Int) = new IntMult(i) + class IntMult(i: Int) { def *(d: Duration): Duration = d * i } + implicit def intMult(i: Int): IntMult = new IntMult(i) - class LongMult(l: Long) { - def *(d: Duration) = d * l - } - implicit def longMult(l: Long) = new LongMult(l) + class LongMult(l: Long) { def *(d: Duration): Duration = d * l } + implicit def longMult(l: Long): LongMult = new LongMult(l) - class DoubleMult(f: Double) { - def *(d: Duration) = d * f - } - implicit def doubleMult(f: Double) = new DoubleMult(f) + class DoubleMult(f: Double) { def *(d: Duration): Duration = d * f } + implicit def doubleMult(f: Double): DoubleMult = new DoubleMult(f) } diff --git a/akka-agent/src/main/scala/akka/agent/Agent.scala b/akka-agent/src/main/scala/akka/agent/Agent.scala index af551d00c8..ea3d8719cd 100644 --- a/akka-agent/src/main/scala/akka/agent/Agent.scala +++ b/akka-agent/src/main/scala/akka/agent/Agent.scala @@ -96,18 +96,18 @@ object Agent { * }}} */ class Agent[T](initialValue: T, system: ActorSystem) { - private[akka] val ref = Ref(initialValue) - private[akka] val updater = system.actorOf(Props(new AgentUpdater(this))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow? + private val ref = Ref(initialValue) + private val updater = system.actorOf(Props(new AgentUpdater(this, ref))).asInstanceOf[InternalActorRef] //TODO can we avoid this somehow? /** * Read the internal state of the agent. */ - def get() = ref.single.get + def get(): T = ref.single.get /** * Read the internal state of the agent. */ - def apply() = get + def apply(): T = get /** * Dispatch a function to update the internal state. @@ -154,7 +154,7 @@ class Agent[T](initialValue: T, system: ActorSystem) { def sendOff(f: T ⇒ T): Unit = { send((value: T) ⇒ { suspend() - val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher("akka.agent.send-off-dispatcher")) + val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this, ref)).withDispatcher("akka.agent.send-off-dispatcher")) threadBased ! Update(f) value }) @@ -171,7 +171,7 @@ class Agent[T](initialValue: T, system: ActorSystem) { val result = Promise[T]()(system.dispatcher) send((value: T) ⇒ { suspend() - val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher("akka.agent.alter-off-dispatcher")) + val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this, ref)).withDispatcher("akka.agent.alter-off-dispatcher")) result completeWith ask(threadBased, Alter(f))(timeout).asInstanceOf[Future[T]] value }) @@ -209,18 +209,18 @@ class Agent[T](initialValue: T, system: ActorSystem) { /** * Suspends processing of `send` actions for the agent. */ - def suspend() = updater.suspend() + def suspend(): Unit = updater.suspend() /** * Resumes processing of `send` actions for the agent. */ - def resume() = updater.resume() + def resume(): Unit = updater.resume() /** * Closes the agents and makes it eligible for garbage collection. * A closed agent cannot accept any `send` actions. */ - def close() = updater.stop() + def close(): Unit = updater.stop() // --------------------------------------------- // Support for Java API Functions and Procedures @@ -281,8 +281,10 @@ class Agent[T](initialValue: T, system: ActorSystem) { /** * Agent updater actor. Used internally for `send` actions. + * + * INTERNAL API */ -class AgentUpdater[T](agent: Agent[T]) extends Actor { +private[akka] class AgentUpdater[T](agent: Agent[T], ref: Ref[T]) extends Actor { def receive = { case u: Update[_] ⇒ update(u.function.asInstanceOf[T ⇒ T]) case a: Alter[_] ⇒ sender ! update(a.function.asInstanceOf[T ⇒ T]) @@ -290,13 +292,15 @@ class AgentUpdater[T](agent: Agent[T]) extends Actor { case _ ⇒ } - def update(function: T ⇒ T): T = agent.ref.single.transformAndGet(function) + def update(function: T ⇒ T): T = ref.single.transformAndGet(function) } /** * Thread-based agent updater actor. Used internally for `sendOff` actions. + * + * INTERNAL API */ -class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor { +private[akka] class ThreadBasedAgentUpdater[T](agent: Agent[T], ref: Ref[T]) extends Actor { def receive = { case u: Update[_] ⇒ try { update(u.function.asInstanceOf[T ⇒ T]) @@ -313,5 +317,5 @@ class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor { case _ ⇒ context.stop(self) } - def update(function: T ⇒ T): T = agent.ref.single.transformAndGet(function) + def update(function: T ⇒ T): T = ref.single.transformAndGet(function) } diff --git a/akka-camel/src/main/scala/akka/camel/Activation.scala b/akka-camel/src/main/scala/akka/camel/Activation.scala index 56d116dca8..d01c990136 100644 --- a/akka-camel/src/main/scala/akka/camel/Activation.scala +++ b/akka-camel/src/main/scala/akka/camel/Activation.scala @@ -18,9 +18,9 @@ import akka.pattern._ trait Activation { import akka.dispatch.Await - def system: ActorSystem + def system: ActorSystem //FIXME Why is this here, what's it needed for and who should use it? - private val activationTracker = system.actorOf(Props[ActivationTracker], "camelActivationTracker") + private val activationTracker = system.actorOf(Props[ActivationTracker], "camelActivationTracker") //FIXME Why is this also top level? /** * Awaits for endpoint to be activated. It blocks until the endpoint is registered in camel context or timeout expires. @@ -29,13 +29,10 @@ trait Activation { * @throws akka.camel.ActivationTimeoutException if endpoint is not activated within timeout. * @return the activated ActorRef */ - def awaitActivation(endpoint: ActorRef, timeout: Duration): ActorRef = { - try { - Await.result(activationFutureFor(endpoint, timeout), timeout) - } catch { + def awaitActivation(endpoint: ActorRef, timeout: Duration): ActorRef = + try Await.result(activationFutureFor(endpoint, timeout), timeout) catch { case e: TimeoutException ⇒ throw new ActivationTimeoutException(endpoint, timeout) } - } /** * Awaits for endpoint to be de-activated. It is blocking until endpoint is unregistered in camel context or timeout expires. @@ -43,37 +40,32 @@ trait Activation { * @param timeout the timeout for the wait * @throws akka.camel.DeActivationTimeoutException if endpoint is not de-activated within timeout. */ - def awaitDeactivation(endpoint: ActorRef, timeout: Duration) { - try { - Await.result(deactivationFutureFor(endpoint, timeout), timeout) - } catch { + def awaitDeactivation(endpoint: ActorRef, timeout: Duration): Unit = + try Await.result(deactivationFutureFor(endpoint, timeout), timeout) catch { case e: TimeoutException ⇒ throw new DeActivationTimeoutException(endpoint, timeout) } - } /** * Similar to `awaitActivation` but returns a future instead. * @param endpoint the endpoint to be activated * @param timeout the timeout for the Future */ - def activationFutureFor(endpoint: ActorRef, timeout: Duration): Future[ActorRef] = { + def activationFutureFor(endpoint: ActorRef, timeout: Duration): Future[ActorRef] = (activationTracker.ask(AwaitActivation(endpoint))(Timeout(timeout))).map[ActorRef] { case EndpointActivated(_) ⇒ endpoint case EndpointFailedToActivate(_, cause) ⇒ throw cause } - } /** * Similar to awaitDeactivation but returns a future instead. * @param endpoint the endpoint to be deactivated * @param timeout the timeout of the Future */ - def deactivationFutureFor(endpoint: ActorRef, timeout: Duration): Future[Unit] = { + def deactivationFutureFor(endpoint: ActorRef, timeout: Duration): Future[Unit] = (activationTracker.ask(AwaitDeActivation(endpoint))(Timeout(timeout))).map[Unit] { case EndpointDeActivated(_) ⇒ () case EndpointFailedToDeActivate(_, cause) ⇒ throw cause } - } } /** @@ -82,7 +74,7 @@ trait Activation { * @param timeout the timeout */ class DeActivationTimeoutException(endpoint: ActorRef, timeout: Duration) extends TimeoutException { - override def getMessage = "Timed out after %s, while waiting for de-activation of %s" format (timeout, endpoint.path) + override def getMessage: String = "Timed out after %s, while waiting for de-activation of %s" format (timeout, endpoint.path) } /** @@ -91,5 +83,5 @@ class DeActivationTimeoutException(endpoint: ActorRef, timeout: Duration) extend * @param timeout the timeout */ class ActivationTimeoutException(endpoint: ActorRef, timeout: Duration) extends TimeoutException { - override def getMessage = "Timed out after %s, while waiting for activation of %s" format (timeout, endpoint.path) + override def getMessage: String = "Timed out after %s, while waiting for activation of %s" format (timeout, endpoint.path) } \ No newline at end of file diff --git a/akka-camel/src/main/scala/akka/camel/ActorNotRegisteredException.scala b/akka-camel/src/main/scala/akka/camel/ActorNotRegisteredException.scala index a468eeace5..7a303e47b3 100644 --- a/akka-camel/src/main/scala/akka/camel/ActorNotRegisteredException.scala +++ b/akka-camel/src/main/scala/akka/camel/ActorNotRegisteredException.scala @@ -6,5 +6,5 @@ package akka.camel * @author Martin Krasser */ class ActorNotRegisteredException(uri: String) extends RuntimeException { - override def getMessage = "Actor [%s] doesn't exist" format uri + override def getMessage: String = "Actor [%s] doesn't exist" format uri } diff --git a/akka-camel/src/main/scala/akka/camel/ActorRouteDefinition.scala b/akka-camel/src/main/scala/akka/camel/ActorRouteDefinition.scala index f5175b90eb..6286edad87 100644 --- a/akka-camel/src/main/scala/akka/camel/ActorRouteDefinition.scala +++ b/akka-camel/src/main/scala/akka/camel/ActorRouteDefinition.scala @@ -29,7 +29,8 @@ class ActorRouteDefinition(definition: ProcessorDefinition[_]) { * @param actorRef the consumer with a default configuration. * @return the path to the actor, as a camel uri String */ - def to(actorRef: ActorRef) = definition.to(ActorEndpointPath(actorRef).toCamelPath()) + def to(actorRef: ActorRef) = //FIXME What is the return type of this? + definition.to(ActorEndpointPath(actorRef).toCamelPath()) /** * Sends the message to an ActorRef endpoint @@ -37,6 +38,7 @@ class ActorRouteDefinition(definition: ProcessorDefinition[_]) { * @param consumerConfig the configuration for the consumer * @return the path to the actor, as a camel uri String */ - def to(actorRef: ActorRef, consumerConfig: ConsumerConfig) = definition.to(ActorEndpointPath(actorRef).toCamelPath(consumerConfig)) + def to(actorRef: ActorRef, consumerConfig: ConsumerConfig) = //FIXME What is the return type of this? + definition.to(ActorEndpointPath(actorRef).toCamelPath(consumerConfig)) } diff --git a/akka-camel/src/main/scala/akka/camel/Camel.scala b/akka-camel/src/main/scala/akka/camel/Camel.scala index 4e96f038e5..72252212cf 100644 --- a/akka-camel/src/main/scala/akka/camel/Camel.scala +++ b/akka-camel/src/main/scala/akka/camel/Camel.scala @@ -50,13 +50,13 @@ object CamelExtension extends ExtensionId[Camel] with ExtensionIdProvider { /** * Creates a new instance of Camel and makes sure it gets stopped when the actor system is shutdown. */ - def createExtension(system: ExtendedActorSystem) = { + override def createExtension(system: ExtendedActorSystem): Camel = { val camel = new DefaultCamel(system).start system.registerOnTermination(camel.shutdown()) camel } - def lookup(): ExtensionId[Camel] = CamelExtension + override def lookup(): ExtensionId[Camel] = CamelExtension override def get(system: ActorSystem): Camel = super.get(system) } \ No newline at end of file diff --git a/akka-camel/src/main/scala/akka/camel/CamelMessage.scala b/akka-camel/src/main/scala/akka/camel/CamelMessage.scala index 2ea046b856..cb4121189d 100644 --- a/akka-camel/src/main/scala/akka/camel/CamelMessage.scala +++ b/akka-camel/src/main/scala/akka/camel/CamelMessage.scala @@ -21,12 +21,12 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) { def this(body: Any, headers: JMap[String, Any]) = this(body, headers.toMap) //for Java - override def toString = "CamelMessage(%s, %s)" format (body, headers) + override def toString: String = "CamelMessage(%s, %s)" format (body, headers) /** * Returns those headers from this message whose name is contained in names. */ - def headers(names: Set[String]): Map[String, Any] = headers.filterKeys(names contains _) + def headers(names: Set[String]): Map[String, Any] = headers filterKeys names /** * Returns those headers from this message whose name is contained in names. @@ -75,7 +75,7 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) { /** * Creates a CamelMessage with a given body. */ - def withBody(body: Any) = CamelMessage(body, this.headers) + def withBody(body: Any): CamelMessage = CamelMessage(body, this.headers) /** * Creates a new CamelMessage with given headers. @@ -119,9 +119,9 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) { * Creates a new CamelMessage where the header with given headerName is removed from * the existing headers. */ - def withoutHeader(headerName: String) = copy(this.body, this.headers - headerName) + def withoutHeader(headerName: String): CamelMessage = copy(this.body, this.headers - headerName) - def copyContentTo(to: JCamelMessage) = { + def copyContentTo(to: JCamelMessage): Unit = { to.setBody(this.body) for ((name, value) ← this.headers) to.getHeaders.put(name, value.asInstanceOf[AnyRef]) } @@ -145,8 +145,7 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) { * Java API * */ - def getBodyAs[T](clazz: Class[T], camelContext: CamelContext): T = - camelContext.getTypeConverter.mandatoryConvertTo[T](clazz, body) + def getBodyAs[T](clazz: Class[T], camelContext: CamelContext): T = camelContext.getTypeConverter.mandatoryConvertTo[T](clazz, body) /** * Creates a CamelMessage with current body converted to type T. @@ -184,7 +183,7 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) { *

* Java API */ - def getHeaderAs[T](name: String, clazz: Class[T], camelContext: CamelContext) = headerAs[T](name)(Manifest.classType(clazz), camelContext).get + def getHeaderAs[T](name: String, clazz: Class[T], camelContext: CamelContext): T = headerAs[T](name)(Manifest.classType(clazz), camelContext).get } @@ -201,7 +200,7 @@ object CamelMessage { * so that it can be correlated with an asynchronous response. Messages send to Consumer * actors have this header already set. */ - val MessageExchangeId = "MessageExchangeId".intern + val MessageExchangeId = "MessageExchangeId" //Deliberately without type ascription to make it a constant /** * Creates a canonical form of the given message msg. If msg of type @@ -244,5 +243,7 @@ case object Ack { * message or Exchange.getOut message, depending on the exchange pattern. * */ -class AkkaCamelException private[akka] (cause: Throwable, val headers: Map[String, Any] = Map.empty) - extends AkkaException(cause.getMessage, cause) +class AkkaCamelException private[akka] (cause: Throwable, val headers: Map[String, Any]) + extends AkkaException(cause.getMessage, cause) { + def this(cause: Throwable) = this(cause, Map.empty) +} diff --git a/akka-camel/src/main/scala/akka/camel/Consumer.scala b/akka-camel/src/main/scala/akka/camel/Consumer.scala index 1d21ffbec7..0351ce39cb 100644 --- a/akka-camel/src/main/scala/akka/camel/Consumer.scala +++ b/akka-camel/src/main/scala/akka/camel/Consumer.scala @@ -31,7 +31,7 @@ trait ConsumerConfig { /** * How long the actor should wait for activation before it fails. */ - def activationTimeout: Duration = 10 seconds + def activationTimeout: Duration = 10 seconds // FIXME Should be configured in reference.conf /** * When endpoint is out-capable (can produce responses) replyTimeout is the maximum time @@ -39,14 +39,14 @@ trait ConsumerConfig { * This setting is used for out-capable, in-only, manually acknowledged communication. * When the blocking is set to Blocking replyTimeout is ignored. */ - def replyTimeout: Duration = 1 minute + def replyTimeout: Duration = 1 minute // FIXME Should be configured in reference.conf /** * Determines whether one-way communications between an endpoint and this consumer actor * should be auto-acknowledged or application-acknowledged. * This flag has only effect when exchange is in-only. */ - def autoack: Boolean = true + def autoack: Boolean = true // FIXME Should be configured in reference.conf /** * The route definition handler for creating a custom route to this consumer instance. diff --git a/akka-camel/src/main/scala/akka/camel/Producer.scala b/akka-camel/src/main/scala/akka/camel/Producer.scala index 80537fda12..5a7262a133 100644 --- a/akka-camel/src/main/scala/akka/camel/Producer.scala +++ b/akka-camel/src/main/scala/akka/camel/Producer.scala @@ -6,8 +6,9 @@ package akka.camel import akka.actor.Actor import internal.CamelExchangeAdapter -import org.apache.camel.{ Exchange, ExchangePattern, AsyncCallback } import akka.actor.Status.Failure +import org.apache.camel.{ Endpoint, Exchange, ExchangePattern, AsyncCallback } +import org.apache.camel.processor.SendProcessor /** * Support trait for producing messages to Camel endpoints. @@ -15,19 +16,19 @@ import akka.actor.Status.Failure * @author Martin Krasser */ trait ProducerSupport { this: Actor ⇒ - protected[this] implicit def camel = CamelExtension(context.system) + protected[this] implicit def camel = CamelExtension(context.system) // FIXME This is duplicated from Consumer, create a common base-trait? /** * camelContext implicit is useful when using advanced methods of CamelMessage. */ - protected[this] implicit def camelContext = camel.context + protected[this] implicit def camelContext = camel.context // FIXME This is duplicated from Consumer, create a common base-trait? - protected[this] lazy val (endpoint, processor) = camel.registerProducer(self, endpointUri) + protected[this] lazy val (endpoint: Endpoint, processor: SendProcessor) = camel.registerProducer(self, endpointUri) /** * CamelMessage headers to copy by default from request message to response-message. */ - private val headersToCopyDefault = Set(CamelMessage.MessageExchangeId) + private val headersToCopyDefault: Set[String] = Set(CamelMessage.MessageExchangeId) /** * If set to false (default), this producer expects a response message from the Camel endpoint. @@ -64,20 +65,21 @@ trait ProducerSupport { this: Actor ⇒ * @param pattern exchange pattern */ protected def produce(msg: Any, pattern: ExchangePattern): Unit = { - implicit def toExchangeAdapter(exchange: Exchange): CamelExchangeAdapter = new CamelExchangeAdapter(exchange) + // Need copies of sender reference here since the callback could be done + // later by another thread. + val producer = self + val originalSender = sender val cmsg = CamelMessage.canonicalize(msg) - val exchange = endpoint.createExchange(pattern) - exchange.setRequest(cmsg) - processor.process(exchange, new AsyncCallback { - val producer = self - // Need copies of sender reference here since the callback could be done - // later by another thread. - val originalSender = sender + val xchg = new CamelExchangeAdapter(endpoint.createExchange(pattern)) + + xchg.setRequest(cmsg) + + processor.process(xchg.exchange, new AsyncCallback { // Ignoring doneSync, sending back async uniformly. def done(doneSync: Boolean): Unit = producer.tell( - if (exchange.isFailed) exchange.toFailureResult(cmsg.headers(headersToCopy)) - else MessageResult(exchange.toResponseMessage(cmsg.headers(headersToCopy))), originalSender) + if (xchg.exchange.isFailed) xchg.toFailureResult(cmsg.headers(headersToCopy)) + else MessageResult(xchg.toResponseMessage(cmsg.headers(headersToCopy))), originalSender) }) } @@ -94,9 +96,7 @@ trait ProducerSupport { this: Actor ⇒ val e = new AkkaCamelException(res.cause, res.headers) routeResponse(Failure(e)) throw e - case msg ⇒ - val exchangePattern = if (oneway) ExchangePattern.InOnly else ExchangePattern.InOut - produce(transformOutgoingMessage(msg), exchangePattern) + case msg ⇒ produce(transformOutgoingMessage(msg), if (oneway) ExchangePattern.InOnly else ExchangePattern.InOut) } /** @@ -134,7 +134,7 @@ trait Producer extends ProducerSupport { this: Actor ⇒ * Default implementation of Actor.receive. Any messages received by this actors * will be produced to the endpoint specified by endpointUri. */ - protected def receive = produce + def receive: Actor.Receive = produce } /** @@ -153,6 +153,6 @@ private case class FailureResult(cause: Throwable, headers: Map[String, Any] = M * @author Martin Krasser */ trait Oneway extends Producer { this: Actor ⇒ - override def oneway = true + override def oneway: Boolean = true } diff --git a/akka-camel/src/main/scala/akka/camel/internal/ActivationMessage.scala b/akka-camel/src/main/scala/akka/camel/internal/ActivationMessage.scala index b8c3f42a47..bdd915ff70 100644 --- a/akka-camel/src/main/scala/akka/camel/internal/ActivationMessage.scala +++ b/akka-camel/src/main/scala/akka/camel/internal/ActivationMessage.scala @@ -20,7 +20,7 @@ private[camel] abstract class ActivationMessage(val actor: ActorRef) * */ private[camel] object ActivationMessage { - def unapply(msg: ActivationMessage): Option[ActorRef] = Some(msg.actor) + def unapply(msg: ActivationMessage): Option[ActorRef] = Option(msg.actor) } /** diff --git a/akka-camel/src/main/scala/akka/camel/internal/ActivationTracker.scala b/akka-camel/src/main/scala/akka/camel/internal/ActivationTracker.scala index 0b93460be0..f5a87eff25 100644 --- a/akka-camel/src/main/scala/akka/camel/internal/ActivationTracker.scala +++ b/akka-camel/src/main/scala/akka/camel/internal/ActivationTracker.scala @@ -96,17 +96,15 @@ private[akka] final class ActivationTracker extends Actor with ActorLogging { /** * Subscribes self to messages of type ActivationMessage */ - override def preStart() { - context.system.eventStream.subscribe(self, classOf[ActivationMessage]) - } + override def preStart(): Unit = context.system.eventStream.subscribe(self, classOf[ActivationMessage]) override def receive = { case msg @ ActivationMessage(ref) ⇒ - val state = activations.getOrElseUpdate(ref, new ActivationStateMachine) - (state.receive orElse logStateWarning(ref))(msg) + (activations.getOrElseUpdate(ref, new ActivationStateMachine).receive orElse logStateWarning(ref))(msg) } - private[this] def logStateWarning(actorRef: ActorRef): Receive = { case msg ⇒ log.warning("Message [{}] not expected in current state of actor [{}]", msg, actorRef) } + private[this] def logStateWarning(actorRef: ActorRef): Receive = + { case msg ⇒ log.warning("Message [{}] not expected in current state of actor [{}]", msg, actorRef) } } /** diff --git a/akka-camel/src/main/scala/akka/camel/internal/CamelExchangeAdapter.scala b/akka-camel/src/main/scala/akka/camel/internal/CamelExchangeAdapter.scala index 1f2d80e6df..5de9eb447d 100644 --- a/akka-camel/src/main/scala/akka/camel/internal/CamelExchangeAdapter.scala +++ b/akka-camel/src/main/scala/akka/camel/internal/CamelExchangeAdapter.scala @@ -16,34 +16,34 @@ import akka.camel.{ FailureResult, AkkaCamelException, CamelMessage } * * @author Martin Krasser */ -private[camel] class CamelExchangeAdapter(exchange: Exchange) { +private[camel] class CamelExchangeAdapter(val exchange: Exchange) { /** * Returns the exchange id */ - def getExchangeId = exchange.getExchangeId + def getExchangeId: String = exchange.getExchangeId /** * Returns if the exchange is out capable. */ - def isOutCapable = exchange.getPattern.isOutCapable + def isOutCapable: Boolean = exchange.getPattern.isOutCapable /** * Sets Exchange.getIn from the given CamelMessage object. */ - def setRequest(msg: CamelMessage) { msg.copyContentTo(request) } + def setRequest(msg: CamelMessage): Unit = msg.copyContentTo(request) /** * Depending on the exchange pattern, sets Exchange.getIn or Exchange.getOut from the given * CamelMessage object. If the exchange is out-capable then the Exchange.getOut is set, otherwise * Exchange.getIn. */ - def setResponse(msg: CamelMessage) { msg.copyContentTo(response) } + def setResponse(msg: CamelMessage): Unit = msg.copyContentTo(response) /** * Sets Exchange.getException from the given FailureResult message. Headers of the FailureResult message * are ignored. */ - def setFailure(msg: FailureResult) { exchange.setException(msg.cause) } + def setFailure(msg: FailureResult): Unit = exchange.setException(msg.cause) /** * Creates an immutable CamelMessage object from Exchange.getIn so it can be used with Actors. @@ -120,7 +120,7 @@ private[camel] class CamelExchangeAdapter(exchange: Exchange) { */ def toResponseMessage(headers: Map[String, Any]): CamelMessage = CamelMessage.from(response, headers) - private def request = exchange.getIn + private def request: JCamelMessage = exchange.getIn private def response: JCamelMessage = ExchangeHelper.getResultMessage(exchange) diff --git a/akka-camel/src/main/scala/akka/camel/internal/DefaultCamel.scala b/akka-camel/src/main/scala/akka/camel/internal/DefaultCamel.scala index 1754bb0073..2ac35fdec2 100644 --- a/akka-camel/src/main/scala/akka/camel/internal/DefaultCamel.scala +++ b/akka-camel/src/main/scala/akka/camel/internal/DefaultCamel.scala @@ -2,12 +2,12 @@ package akka.camel.internal import akka.actor.ActorSystem import component.{ DurationTypeConverter, ActorComponent } -import org.apache.camel.CamelContext import org.apache.camel.impl.DefaultCamelContext import scala.Predef._ import akka.event.Logging import akka.camel.Camel import akka.util.{ NonFatal, Duration } +import org.apache.camel.{ ProducerTemplate, CamelContext } /** * For internal use only. @@ -33,14 +33,14 @@ private[camel] class DefaultCamel(val system: ActorSystem) extends Camel { ctx } - lazy val template = context.createProducerTemplate() + lazy val template: ProducerTemplate = context.createProducerTemplate() /** * Starts camel and underlying camel context and template. * Only the creator of Camel should start and stop it. * @see akka.camel.DefaultCamel#stop() */ - def start = { + def start(): this.type = { context.start() try template.start() catch { case NonFatal(e) ⇒ context.stop(); throw e } log.debug("Started CamelContext[{}] for ActorSystem[{}]", context.getName, system.name) @@ -54,9 +54,9 @@ private[camel] class DefaultCamel(val system: ActorSystem) extends Camel { * * @see akka.camel.DefaultCamel#start() */ - def shutdown() { + def shutdown(): Unit = { try context.stop() finally { - try { template.stop() } catch { case NonFatal(e) ⇒ log.debug("Swallowing non-fatal exception [{}] on stopping Camel producer template", e) } + try template.stop() catch { case NonFatal(e) ⇒ log.debug("Swallowing non-fatal exception [{}] on stopping Camel producer template", e) } } log.debug("Stopped CamelContext[{}] for ActorSystem[{}]", context.getName, system.name) } diff --git a/akka-camel/src/main/scala/akka/camel/internal/ProducerRegistry.scala b/akka-camel/src/main/scala/akka/camel/internal/ProducerRegistry.scala index 03d130efe2..d338dbfdea 100644 --- a/akka-camel/src/main/scala/akka/camel/internal/ProducerRegistry.scala +++ b/akka-camel/src/main/scala/akka/camel/internal/ProducerRegistry.scala @@ -11,6 +11,8 @@ import akka.util.NonFatal * Watches the end of life of Producers. * Removes a Producer from the ProducerRegistry when it is Terminated, * which in turn stops the SendProcessor. + * + * INTERNAL API */ private class ProducerWatcher(registry: ProducerRegistry) extends Actor { override def receive = { @@ -19,6 +21,9 @@ private class ProducerWatcher(registry: ProducerRegistry) extends Actor { } } +/** + * INTERNAL API + */ private case class RegisterProducer(actorRef: ActorRef) /** @@ -27,14 +32,11 @@ private case class RegisterProducer(actorRef: ActorRef) * Every Producer needs an Endpoint and a SendProcessor * to produce messages over an Exchange. */ -private[camel] trait ProducerRegistry { - this: Camel ⇒ +private[camel] trait ProducerRegistry { this: Camel ⇒ private val camelObjects = new ConcurrentHashMap[ActorRef, (Endpoint, SendProcessor)]() - private val watcher = system.actorOf(Props(new ProducerWatcher(this))) + private val watcher = system.actorOf(Props(new ProducerWatcher(this))) //FIXME should this really be top level? - private def registerWatch(actorRef: ActorRef) { - watcher ! RegisterProducer(actorRef) - } + private def registerWatch(actorRef: ActorRef): Unit = watcher ! RegisterProducer(actorRef) /** * For internal use only. @@ -77,7 +79,7 @@ private[camel] trait ProducerRegistry { case NonFatal(e) ⇒ { system.eventStream.publish(EndpointFailedToActivate(actorRef, e)) // can't return null to the producer actor, so blow up actor in initialization. - throw e + throw e //FIXME I'm not a huge fan of log-rethrow, either log or rethrow } } } diff --git a/akka-camel/src/main/scala/akka/camel/internal/component/ActorComponent.scala b/akka-camel/src/main/scala/akka/camel/internal/component/ActorComponent.scala index 7ec5919dc9..d52f74f2f6 100644 --- a/akka-camel/src/main/scala/akka/camel/internal/component/ActorComponent.scala +++ b/akka-camel/src/main/scala/akka/camel/internal/component/ActorComponent.scala @@ -35,10 +35,8 @@ private[camel] class ActorComponent(camel: Camel) extends DefaultComponent { /** * @see org.apache.camel.Component */ - def createEndpoint(uri: String, remaining: String, parameters: JMap[String, Object]): ActorEndpoint = { - val path = ActorEndpointPath.fromCamelPath(remaining) - new ActorEndpoint(uri, this, path, camel) - } + def createEndpoint(uri: String, remaining: String, parameters: JMap[String, Object]): ActorEndpoint = + new ActorEndpoint(uri, this, ActorEndpointPath.fromCamelPath(remaining), camel) } /** @@ -92,7 +90,7 @@ private[camel] class ActorEndpoint(uri: String, private[camel] trait ActorEndpointConfig { def path: ActorEndpointPath - @BeanProperty var replyTimeout: Duration = 1 minute + @BeanProperty var replyTimeout: Duration = 1 minute // FIXME default should be in config, not code /** * Whether to auto-acknowledge one-way message exchanges with (untyped) actors. This is @@ -117,7 +115,7 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex * Calls the asynchronous version of the method and waits for the result (blocking). * @param exchange the exchange to process */ - def process(exchange: Exchange) { processExchangeAdapter(new CamelExchangeAdapter(exchange)) } + def process(exchange: Exchange): Unit = processExchangeAdapter(new CamelExchangeAdapter(exchange)) /** * Processes the message exchange. the caller supports having the exchange asynchronously processed. @@ -129,13 +127,15 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex * The callback should therefore be careful of starting recursive loop. * @return (doneSync) true to continue execute synchronously, false to continue being executed asynchronously */ - def process(exchange: Exchange, callback: AsyncCallback): Boolean = { processExchangeAdapter(new CamelExchangeAdapter(exchange), callback) } + def process(exchange: Exchange, callback: AsyncCallback): Boolean = processExchangeAdapter(new CamelExchangeAdapter(exchange), callback) /** * For internal use only. Processes the [[akka.camel.internal.CamelExchangeAdapter]] * @param exchange the [[akka.camel.internal.CamelExchangeAdapter]] + * + * WARNING UNBOUNDED BLOCKING AWAITS */ - private[camel] def processExchangeAdapter(exchange: CamelExchangeAdapter) { + private[camel] def processExchangeAdapter(exchange: CamelExchangeAdapter): Unit = { val isDone = new CountDownLatch(1) processExchangeAdapter(exchange, new AsyncCallback { def done(doneSync: Boolean) { isDone.countDown() } }) isDone.await() // this should never wait forever as the process(exchange, callback) method guarantees that. @@ -151,10 +151,10 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex private[camel] def processExchangeAdapter(exchange: CamelExchangeAdapter, callback: AsyncCallback): Boolean = { // these notify methods are just a syntax sugar - def notifyDoneSynchronously[A](a: A = null) = callback.done(true) - def notifyDoneAsynchronously[A](a: A = null) = callback.done(false) + def notifyDoneSynchronously[A](a: A = null): Unit = callback.done(true) + def notifyDoneAsynchronously[A](a: A = null): Unit = callback.done(false) - def message = messageFor(exchange) + def message: CamelMessage = messageFor(exchange) if (exchange.isOutCapable) { //InOut sendAsync(message, onComplete = forwardResponseTo(exchange) andThen notifyDoneAsynchronously) @@ -186,42 +186,41 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex private def sendAsync(message: CamelMessage, onComplete: PartialFunction[Either[Throwable, Any], Unit]): Boolean = { try { - val actor = actorFor(endpoint.path) - val future = actor.ask(message)(new Timeout(endpoint.replyTimeout)) - future.onComplete(onComplete) + actorFor(endpoint.path).ask(message)(Timeout(endpoint.replyTimeout)).onComplete(onComplete) } catch { case NonFatal(e) ⇒ onComplete(Left(e)) } false // Done async } - private def fireAndForget(message: CamelMessage, exchange: CamelExchangeAdapter) { - try { - actorFor(endpoint.path) ! message - } catch { - case e ⇒ exchange.setFailure(new FailureResult(e)) - } - } + private def fireAndForget(message: CamelMessage, exchange: CamelExchangeAdapter): Unit = + try { actorFor(endpoint.path) ! message } catch { case NonFatal(e) ⇒ exchange.setFailure(new FailureResult(e)) } private[this] def actorFor(path: ActorEndpointPath): ActorRef = path.findActorIn(camel.system) getOrElse (throw new ActorNotRegisteredException(path.actorPath)) private[this] def messageFor(exchange: CamelExchangeAdapter) = exchange.toRequestMessage(Map(CamelMessage.MessageExchangeId -> exchange.getExchangeId)) - } /** * For internal use only. Converts Strings to [[akka.util.Duration]]s */ private[camel] object DurationTypeConverter extends TypeConverter { - def convertTo[T](`type`: Class[T], value: AnyRef) = { - Duration(value.toString).asInstanceOf[T] + override def convertTo[T](`type`: Class[T], value: AnyRef): T = `type`.cast(try { + val d = Duration(value.toString) + if (`type`.isInstance(d)) d else null + } catch { + case NonFatal(_) ⇒ null + }) + + def convertTo[T](`type`: Class[T], exchange: Exchange, value: AnyRef): T = convertTo(`type`, value) + def mandatoryConvertTo[T](`type`: Class[T], value: AnyRef): T = convertTo(`type`, value) match { + case null ⇒ throw new NoTypeConversionAvailableException(value, `type`) + case some ⇒ some } - def convertTo[T](`type`: Class[T], exchange: Exchange, value: AnyRef) = convertTo(`type`, value) - def mandatoryConvertTo[T](`type`: Class[T], value: AnyRef) = convertTo(`type`, value) - def mandatoryConvertTo[T](`type`: Class[T], exchange: Exchange, value: AnyRef) = convertTo(`type`, value) - def toString(duration: Duration) = duration.toNanos + " nanos" + def mandatoryConvertTo[T](`type`: Class[T], exchange: Exchange, value: AnyRef): T = mandatoryConvertTo(`type`, value) + def toString(duration: Duration): String = duration.toNanos + " nanos" } /** @@ -243,15 +242,15 @@ private[camel] case class ActorEndpointPath private (actorPath: String) { * For internal use only. Companion of `ActorEndpointPath` */ private[camel] object ActorEndpointPath { - private val consumerConfig = new ConsumerConfig {} + private val consumerConfig: ConsumerConfig = new ConsumerConfig {} - def apply(actorRef: ActorRef) = new ActorEndpointPath(actorRef.path.toString) + def apply(actorRef: ActorRef): ActorEndpointPath = new ActorEndpointPath(actorRef.path.toString) /** * Creates an [[akka.camel.internal.component.ActorEndpointPath]] from the remaining part of the endpoint URI (the part after the scheme, without the parameters of the URI). * Expects the remaining part of the URI (the actor path) in a format: path:%s */ - def fromCamelPath(camelPath: String) = camelPath match { + def fromCamelPath(camelPath: String): ActorEndpointPath = camelPath match { case id if id startsWith "path:" ⇒ new ActorEndpointPath(id substring 5) case _ ⇒ throw new IllegalArgumentException("Invalid path: [%s] - should be path:" format camelPath) } diff --git a/akka-camel/src/main/scala/akka/camel/javaapi/UntypedConsumer.scala b/akka-camel/src/main/scala/akka/camel/javaapi/UntypedConsumer.scala index 56f11831d0..a4671583bb 100644 --- a/akka-camel/src/main/scala/akka/camel/javaapi/UntypedConsumer.scala +++ b/akka-camel/src/main/scala/akka/camel/javaapi/UntypedConsumer.scala @@ -13,7 +13,7 @@ import org.apache.camel.{ ProducerTemplate, CamelContext } * class is meant to be used from Java. */ abstract class UntypedConsumerActor extends UntypedActor with Consumer { - final def endpointUri = getEndpointUri + final def endpointUri: String = getEndpointUri /** * Returns the Camel endpoint URI to consume messages from. diff --git a/akka-camel/src/main/scala/akka/camel/javaapi/UntypedProducerActor.scala b/akka-camel/src/main/scala/akka/camel/javaapi/UntypedProducerActor.scala index c4d0a9c1a0..f44daf0725 100644 --- a/akka-camel/src/main/scala/akka/camel/javaapi/UntypedProducerActor.scala +++ b/akka-camel/src/main/scala/akka/camel/javaapi/UntypedProducerActor.scala @@ -40,16 +40,14 @@ abstract class UntypedProducerActor extends UntypedActor with ProducerSupport { final override def transformResponse(msg: Any): AnyRef = onTransformResponse(msg.asInstanceOf[AnyRef]) final override def routeResponse(msg: Any): Unit = onRouteResponse(msg.asInstanceOf[AnyRef]) - final override def endpointUri = getEndpointUri + final override def endpointUri: String = getEndpointUri - final override def oneway = isOneway + final override def oneway: Boolean = isOneway /** * Default implementation of UntypedActor.onReceive */ - def onReceive(message: Any) { - produce(message) - } + def onReceive(message: Any): Unit = produce(message) /** * Returns the Camel endpoint URI to produce messages to. @@ -61,7 +59,7 @@ abstract class UntypedProducerActor extends UntypedActor with ProducerSupport { * If set to true, this producer communicates with the Camel endpoint with an in-only message * exchange pattern (fire and forget). */ - def isOneway() = super.oneway + def isOneway(): Boolean = super.oneway /** * Returns the CamelContext. diff --git a/akka-camel/src/main/scala/akka/camelexamples/ExamplesSupport.scala b/akka-camel/src/main/scala/akka/camelexamples/ExamplesSupport.scala index 3e35b8c7c9..df5b0e5508 100644 --- a/akka-camel/src/main/scala/akka/camelexamples/ExamplesSupport.scala +++ b/akka-camel/src/main/scala/akka/camelexamples/ExamplesSupport.scala @@ -19,7 +19,7 @@ private[camelexamples] class SysOutConsumer extends Consumer { override def activationTimeout = 10 seconds def endpointUri = "file://data/input/CamelConsumer" - protected def receive = { + def receive = { case msg: CamelMessage ⇒ { printf("Received '%s'\n", msg.bodyAs[String]) } @@ -30,12 +30,12 @@ private[camelexamples] class TroubleMaker extends Consumer { def endpointUri = "WRONG URI" println("Trying to instantiate conumer with uri: " + endpointUri) - protected def receive = { case _ ⇒ } + def receive = { case _ ⇒ } } private[camelexamples] class SysOutActor(implicit camel: Camel) extends Actor { implicit val camelContext = camel.context - protected def receive = { + def receive = { case msg: CamelMessage ⇒ { printf("Received '%s'\n", msg.bodyAs[String]) } diff --git a/akka-camel/src/main/scala/akka/camelexamples/_2_SupervisedConsumers.scala b/akka-camel/src/main/scala/akka/camelexamples/_2_SupervisedConsumers.scala index 5d321b28db..cdf46f012f 100644 --- a/akka-camel/src/main/scala/akka/camelexamples/_2_SupervisedConsumers.scala +++ b/akka-camel/src/main/scala/akka/camelexamples/_2_SupervisedConsumers.scala @@ -14,7 +14,7 @@ private[camelexamples] object SupervisedConsumersExample extends App { system.actorOf(Props(new Actor { context.watch(context.actorOf(Props[EndpointManager])) - protected def receive = { + def receive = { case Terminated(ref) ⇒ system.shutdown() } })) @@ -30,7 +30,7 @@ private[camelexamples] class EndpointManager extends Actor { watch(actorOf(Props[SysOutConsumer])) watch(actorOf(Props[TroubleMaker])) - protected def receive = { + def receive = { case Terminated(ref) ⇒ { printf("Hey! One of the endpoints has died: %s. I am doing sepuku...\n", ref) self ! PoisonPill diff --git a/akka-camel/src/main/scala/akka/package.scala b/akka-camel/src/main/scala/akka/package.scala index 436d2fc1b3..d3e60ae24f 100644 --- a/akka-camel/src/main/scala/akka/package.scala +++ b/akka-camel/src/main/scala/akka/package.scala @@ -7,5 +7,12 @@ package akka import org.apache.camel.model.ProcessorDefinition package object camel { + /** + * To allow using Actors with the Camel Route DSL: + * + * {{{ + * from("file://data/input/CamelConsumer").to(actor) + * }}} + */ implicit def toActorRouteDefinition(definition: ProcessorDefinition[_]) = new ActorRouteDefinition(definition) } \ No newline at end of file diff --git a/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala b/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala index cef098b8fe..a7e5b9e4cb 100644 --- a/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala +++ b/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala @@ -244,7 +244,7 @@ object ProducerFeatureTest { } class TestResponder extends Actor { - protected def receive = { + def receive = { case msg: CamelMessage ⇒ msg.body match { case "fail" ⇒ context.sender ! akka.actor.Status.Failure(new AkkaCamelException(new Exception("failure"), msg.headers)) case _ ⇒ @@ -256,7 +256,7 @@ object ProducerFeatureTest { } class ReplyingForwardTarget extends Actor { - protected def receive = { + def receive = { case msg: CamelMessage ⇒ context.sender ! (msg.addHeader("test" -> "result")) case msg: akka.actor.Status.Failure ⇒ diff --git a/akka-camel/src/test/scala/akka/camel/internal/component/ActorProducerTest.scala b/akka-camel/src/test/scala/akka/camel/internal/component/ActorProducerTest.scala index 8146b17399..a0e153fd54 100644 --- a/akka-camel/src/test/scala/akka/camel/internal/component/ActorProducerTest.scala +++ b/akka-camel/src/test/scala/akka/camel/internal/component/ActorProducerTest.scala @@ -334,7 +334,7 @@ trait ActorProducerFixture extends MockitoSugar with BeforeAndAfterAll with Befo } def echoActor = system.actorOf(Props(new Actor { - protected def receive = { + def receive = { case msg ⇒ sender ! "received " + msg } })) diff --git a/akka-camel/src/test/scala/akka/camel/internal/component/DurationConverterTest.scala b/akka-camel/src/test/scala/akka/camel/internal/component/DurationConverterTest.scala index 3787a9f46f..53729a0b6f 100644 --- a/akka-camel/src/test/scala/akka/camel/internal/component/DurationConverterTest.scala +++ b/akka-camel/src/test/scala/akka/camel/internal/component/DurationConverterTest.scala @@ -8,8 +8,9 @@ import org.scalatest.matchers.MustMatchers import akka.util.duration._ import akka.util.Duration import org.scalatest.WordSpec +import org.apache.camel.NoTypeConversionAvailableException -class DurationConverterTest extends WordSpec with MustMatchers { +class DurationConverterSpec extends WordSpec with MustMatchers { import DurationTypeConverter._ "DurationTypeConverter must convert '10 nanos'" in { @@ -21,15 +22,19 @@ class DurationConverterTest extends WordSpec with MustMatchers { } "DurationTypeConverter must throw if invalid format" in { - intercept[Exception] { - convertTo(classOf[Duration], "abc nanos") must be(10 nanos) - } + convertTo(classOf[Duration], "abc nanos") must be === null + + intercept[NoTypeConversionAvailableException] { + mandatoryConvertTo(classOf[Duration], "abc nanos") must be(10 nanos) + }.getValue must be === "abc nanos" } - "DurationTypeConverter must throw if doesn't end with nanos" in { - intercept[Exception] { - convertTo(classOf[Duration], "10233") must be(10 nanos) - } + "DurationTypeConverter must throw if doesn't end with time unit" in { + convertTo(classOf[Duration], "10233") must be === null + + intercept[NoTypeConversionAvailableException] { + mandatoryConvertTo(classOf[Duration], "10233") must be(10 nanos) + }.getValue must be === "10233" } } diff --git a/akka-cluster/src/main/resources/reference.conf b/akka-cluster/src/main/resources/reference.conf index 7dd511e34a..b60b91ec43 100644 --- a/akka-cluster/src/main/resources/reference.conf +++ b/akka-cluster/src/main/resources/reference.conf @@ -8,11 +8,23 @@ akka { cluster { - # node to join - the full URI defined by a string on the form of "akka://system@hostname:port" - # leave as empty string if the node should be a singleton cluster - node-to-join = "" + # Initial contact points of the cluster. Nodes to join at startup if auto-join = on. + # The seed nodes also play the role of deputy nodes (the nodes responsible + # for breaking network partitions). + # Comma separated full URIs defined by a string on the form of "akka://system@hostname:port" + # Leave as empty if the node should be a singleton cluster. + seed-nodes = [] - # should the 'leader' in the cluster be allowed to automatically mark unreachable nodes as DOWN? + # how long to wait for one of the seed nodes to reply to initial join request + seed-node-timeout = 5s + + # Automatic join the seed-nodes at startup. + # If seed-nodes is empty it will join itself and become a single node cluster. + auto-join = on + + # Should the 'leader' in the cluster be allowed to automatically mark unreachable nodes as DOWN? + # Using auto-down implies that two separate clusters will automatically be formed in case of + # network partition. auto-down = on # the number of gossip daemon actors @@ -25,15 +37,26 @@ akka { periodic-tasks-initial-delay = 1s # how often should the node send out gossip information? - gossip-frequency = 1s + gossip-interval = 1s + + # how often should the node send out heartbeats? + heartbeat-interval = 1s # how often should the leader perform maintenance tasks? - leader-actions-frequency = 1s + leader-actions-interval = 1s # how often should the node move nodes, marked as unreachable by the failure detector, out of the membership ring? - unreachable-nodes-reaper-frequency = 1s + unreachable-nodes-reaper-interval = 1s + + # A joining node stops sending heartbeats to the node to join if it hasn't become member + # of the cluster within this deadline. + join-timeout = 60s + + # Gossip to random node with newer or older state information, if any with some + # this probability. Otherwise Gossip to any random live node. + # Probability value is between 0.0 and 1.0. 0.0 means never, 1.0 means always. + gossip-different-view-probability = 0.8 - # accrual failure detection config failure-detector { # defines the failure detector threshold @@ -41,9 +64,34 @@ akka { # a quick detection in the event of a real crash. Conversely, a high # threshold generates fewer mistakes but needs more time to detect # actual crashes - threshold = 8 + threshold = 8.0 + + # Minimum standard deviation to use for the normal distribution in + # AccrualFailureDetector. Too low standard deviation might result in + # too much sensitivity for sudden, but normal, deviations in heartbeat + # inter arrival times. + min-std-deviation = 100 ms + + # Number of potentially lost/delayed heartbeats that will be + # accepted before considering it to be an anomaly. + # It is a factor of heartbeat-interval. + # This margin is important to be able to survive sudden, occasional, + # pauses in heartbeat arrivals, due to for example garbage collect or + # network drop. + acceptable-heartbeat-pause = 3s + + implementation-class = "akka.cluster.AccrualFailureDetector" max-sample-size = 1000 } + + # If the tick-duration of the default scheduler is longer than the tick-duration + # configured here a dedicated scheduler will be used for periodic tasks of the cluster, + # otherwise the default scheduler is used. + # See akka.scheduler settings for more details about the HashedWheelTimer. + scheduler { + tick-duration = 33ms + ticks-per-wheel = 512 + } } } diff --git a/akka-cluster/src/main/scala/akka/cluster/AccrualFailureDetector.scala b/akka-cluster/src/main/scala/akka/cluster/AccrualFailureDetector.scala index 3caece392c..f1c761dec7 100644 --- a/akka-cluster/src/main/scala/akka/cluster/AccrualFailureDetector.scala +++ b/akka-cluster/src/main/scala/akka/cluster/AccrualFailureDetector.scala @@ -4,39 +4,101 @@ package akka.cluster -import akka.actor.{ ActorSystem, Address } +import akka.actor.{ ActorSystem, Address, ExtendedActorSystem } +import akka.remote.RemoteActorRefProvider import akka.event.Logging - import scala.collection.immutable.Map import scala.annotation.tailrec - import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.TimeUnit.NANOSECONDS +import akka.util.Duration +import akka.util.duration._ +object AccrualFailureDetector { + private def realClock: () ⇒ Long = () ⇒ NANOSECONDS.toMillis(System.nanoTime) +} /** * Implementation of 'The Phi Accrual Failure Detector' by Hayashibara et al. as defined in their paper: * [http://ddg.jaist.ac.jp/pub/HDY+04.pdf] - *

- * A low threshold is prone to generate many wrong suspicions but ensures a quick detection in the event - * of a real crash. Conversely, a high threshold generates fewer mistakes but needs more time to detect - * actual crashes - *

- * Default threshold is 8, but can be configured in the Akka config. + * + * The suspicion level of failure is given by a value called φ (phi). + * The basic idea of the φ failure detector is to express the value of φ on a scale that + * is dynamically adjusted to reflect current network conditions. A configurable + * threshold is used to decide if φ is considered to be a failure. + * + * The value of φ is calculated as: + * + * {{{ + * φ = -log10(1 - F(timeSinceLastHeartbeat) + * }}} + * where F is the cumulative distribution function of a normal distribution with mean + * and standard deviation estimated from historical heartbeat inter-arrival times. + * + * + * @param system Belongs to the [[akka.actor.ActorSystem]]. Used for logging. + * + * @param threshold A low threshold is prone to generate many wrong suspicions but ensures a quick detection in the event + * of a real crash. Conversely, a high threshold generates fewer mistakes but needs more time to detect + * actual crashes + * + * @param maxSampleSize Number of samples to use for calculation of mean and standard deviation of + * inter-arrival times. + * + * @param minStdDeviation Minimum standard deviation to use for the normal distribution used when calculating phi. + * Too low standard deviation might result in too much sensitivity for sudden, but normal, deviations + * in heartbeat inter arrival times. + * + * @param acceptableHeartbeatPause Duration corresponding to number of potentially lost/delayed + * heartbeats that will be accepted before considering it to be an anomaly. + * This margin is important to be able to survive sudden, occasional, pauses in heartbeat + * arrivals, due to for example garbage collect or network drop. + * + * @param firstHeartbeatEstimate Bootstrap the stats with heartbeats that corresponds to + * to this duration, with a with rather high standard deviation (since environment is unknown + * in the beginning) + * + * @param clock The clock, returning current time in milliseconds, but can be faked for testing + * purposes. It is only used for measuring intervals (duration). + * */ class AccrualFailureDetector( - system: ActorSystem, - address: Address, - val threshold: Int = 8, - val maxSampleSize: Int = 1000, - val timeMachine: () ⇒ Long = System.currentTimeMillis) { + val system: ActorSystem, + val threshold: Double, + val maxSampleSize: Int, + val minStdDeviation: Duration, + val acceptableHeartbeatPause: Duration, + val firstHeartbeatEstimate: Duration, + val clock: () ⇒ Long = AccrualFailureDetector.realClock) extends FailureDetector { - private final val PhiFactor = 1.0 / math.log(10.0) + import AccrualFailureDetector._ + + /** + * Constructor that picks configuration from the settings. + */ + def this( + system: ActorSystem, + settings: ClusterSettings) = + this( + system, + threshold = settings.FailureDetectorThreshold, + maxSampleSize = settings.FailureDetectorMaxSampleSize, + minStdDeviation = settings.FailureDetectorMinStdDeviation, + acceptableHeartbeatPause = settings.FailureDetectorAcceptableHeartbeatPause, + firstHeartbeatEstimate = settings.HeartbeatInterval, + clock = AccrualFailureDetector.realClock) private val log = Logging(system, "FailureDetector") - /** - * Holds the failure statistics for a specific node Address. - */ - private case class FailureStats(mean: Double = 0.0D, variance: Double = 0.0D, deviation: Double = 0.0D) + // guess statistics for first heartbeat, + // important so that connections with only one heartbeat becomes unavailable + private val firstHeartbeat: HeartbeatHistory = { + // bootstrap with 2 entries with rather high standard deviation + val mean = firstHeartbeatEstimate.toMillis + val stdDeviation = mean / 4 + HeartbeatHistory(maxSampleSize) :+ (mean - stdDeviation) :+ (mean + stdDeviation) + } + + private val acceptableHeartbeatPauseMillis = acceptableHeartbeatPause.toMillis /** * Implement using optimistic lockless concurrency, all state is represented @@ -44,10 +106,8 @@ class AccrualFailureDetector( */ private case class State( version: Long = 0L, - failureStats: Map[Address, FailureStats] = Map.empty[Address, FailureStats], - intervalHistory: Map[Address, IndexedSeq[Long]] = Map.empty[Address, IndexedSeq[Long]], - timestamps: Map[Address, Long] = Map.empty[Address, Long], - explicitRemovals: Set[Address] = Set.empty[Address]) + history: Map[Address, HeartbeatHistory] = Map.empty, + timestamps: Map[Address, Long] = Map.empty[Address, Long]) private val state = new AtomicReference[State](State()) @@ -62,123 +122,167 @@ class AccrualFailureDetector( */ @tailrec final def heartbeat(connection: Address) { - log.debug("Node [{}] - Heartbeat from connection [{}] ", address, connection) + log.debug("Heartbeat from connection [{}] ", connection) + val timestamp = clock() val oldState = state.get - val latestTimestamp = oldState.timestamps.get(connection) - if (latestTimestamp.isEmpty) { - // this is heartbeat from a new connection - // add starter records for this new connection - val newState = oldState copy ( - version = oldState.version + 1, - failureStats = oldState.failureStats + (connection -> FailureStats()), - intervalHistory = oldState.intervalHistory + (connection -> IndexedSeq.empty[Long]), - timestamps = oldState.timestamps + (connection -> timeMachine()), - explicitRemovals = oldState.explicitRemovals - connection) - - // if we won the race then update else try again - if (!state.compareAndSet(oldState, newState)) heartbeat(connection) // recur - - } else { - // this is a known connection - val timestamp = timeMachine() - val interval = timestamp - latestTimestamp.get - - val newIntervalsForConnection = (oldState.intervalHistory.get(connection) match { - case Some(history) if history.size >= maxSampleSize ⇒ - // reached max history, drop first interval - history drop 1 - case Some(history) ⇒ history - case _ ⇒ IndexedSeq.empty[Long] - }) :+ interval - - val newFailureStats = - if (newIntervalsForConnection.size > 1) { - - val newMean: Double = newIntervalsForConnection.sum / newIntervalsForConnection.size.toDouble - - val oldConnectionFailureStats = oldState.failureStats.get(connection).getOrElse { - throw new IllegalStateException("Can't calculate new failure statistics due to missing heartbeat history") - } - - val deviationSum = - newIntervalsForConnection - .map(_.toDouble) - .foldLeft(0.0D)((x, y) ⇒ x + (y - newMean)) - - val newVariance: Double = deviationSum / newIntervalsForConnection.size.toDouble - val newDeviation: Double = math.sqrt(newVariance) - - val newFailureStats = oldConnectionFailureStats copy (mean = newMean, deviation = newDeviation, variance = newVariance) - oldState.failureStats + (connection -> newFailureStats) - - } else { - oldState.failureStats - } - - val newState = oldState copy (version = oldState.version + 1, - failureStats = newFailureStats, - intervalHistory = oldState.intervalHistory + (connection -> newIntervalsForConnection), - timestamps = oldState.timestamps + (connection -> timestamp), // record new timestamp, - explicitRemovals = oldState.explicitRemovals - connection) - - // if we won the race then update else try again - if (!state.compareAndSet(oldState, newState)) heartbeat(connection) // recur + val newHistory = oldState.timestamps.get(connection) match { + case None ⇒ + // this is heartbeat from a new connection + // add starter records for this new connection + firstHeartbeat + case Some(latestTimestamp) ⇒ + // this is a known connection + val interval = timestamp - latestTimestamp + oldState.history(connection) :+ interval } + + val newState = oldState copy (version = oldState.version + 1, + history = oldState.history + (connection -> newHistory), + timestamps = oldState.timestamps + (connection -> timestamp)) // record new timestamp + + // if we won the race then update else try again + if (!state.compareAndSet(oldState, newState)) heartbeat(connection) // recur } /** - * Calculates how likely it is that the connection has failed. - *

+ * The suspicion level of the accrual failure detector. + * * If a connection does not have any records in failure detector then it is - * considered dead. This is true either if the heartbeat have not started - * yet or the connection have been explicitly removed. - *

- * Implementations of 'Cumulative Distribution Function' for Exponential Distribution. - * For a discussion on the math read [https://issues.apache.org/jira/browse/CASSANDRA-2597]. + * considered healthy. */ def phi(connection: Address): Double = { val oldState = state.get val oldTimestamp = oldState.timestamps.get(connection) - val phi = - // if connection has been removed explicitly - if (oldState.explicitRemovals.contains(connection)) Double.MaxValue - else if (oldTimestamp.isEmpty) 0.0D // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections - else { - val timestampDiff = timeMachine() - oldTimestamp.get + if (oldTimestamp.isEmpty) 0.0 // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections + else { + val timeDiff = clock() - oldTimestamp.get - val mean = oldState.failureStats.get(connection) match { - case Some(FailureStats(mean, _, _)) ⇒ mean - case _ ⇒ throw new IllegalStateException("Can't calculate Failure Detector Phi value for a node that have no heartbeat history") - } + val history = oldState.history(connection) + val mean = history.mean + val stdDeviation = ensureValidStdDeviation(history.stdDeviation) - if (mean == 0.0D) 0.0D - else PhiFactor * timestampDiff / mean - } + val φ = phi(timeDiff, mean + acceptableHeartbeatPauseMillis, stdDeviation) - // only log if PHI value is starting to get interesting - if (phi > 0.0D) log.debug("Node [{}] - Phi value [{}] and threshold [{}] for connection [{}] ", address, phi, threshold, connection) - phi + // FIXME change to debug log level, when failure detector is stable + if (φ > 1.0) log.info("Phi value [{}] for connection [{}], after [{} ms], based on [{}]", + φ, connection, timeDiff, "N(" + mean + ", " + stdDeviation + ")") + + φ + } + } + + private[cluster] def phi(timeDiff: Long, mean: Double, stdDeviation: Double): Double = { + val cdf = cumulativeDistributionFunction(timeDiff, mean, stdDeviation) + -math.log10(1.0 - cdf) + } + + private val minStdDeviationMillis = minStdDeviation.toMillis + + private def ensureValidStdDeviation(stdDeviation: Double): Double = math.max(stdDeviation, minStdDeviationMillis) + + /** + * Cumulative distribution function for N(mean, stdDeviation) normal distribution. + * This is an approximation defined in β Mathematics Handbook. + */ + private[cluster] def cumulativeDistributionFunction(x: Double, mean: Double, stdDeviation: Double): Double = { + val y = (x - mean) / stdDeviation + // Cumulative distribution function for N(0, 1) + 1.0 / (1.0 + math.exp(-y * (1.5976 + 0.070566 * y * y))) } /** * Removes the heartbeat management for a connection. */ @tailrec - final def remove(connection: Address) { + final def remove(connection: Address): Unit = { + log.debug("Remove connection [{}] ", connection) val oldState = state.get - if (oldState.failureStats.contains(connection)) { + if (oldState.history.contains(connection)) { val newState = oldState copy (version = oldState.version + 1, - failureStats = oldState.failureStats - connection, - intervalHistory = oldState.intervalHistory - connection, - timestamps = oldState.timestamps - connection, - explicitRemovals = oldState.explicitRemovals + connection) + history = oldState.history - connection, + timestamps = oldState.timestamps - connection) // if we won the race then update else try again if (!state.compareAndSet(oldState, newState)) remove(connection) // recur } } + + def reset(): Unit = { + @tailrec + def doReset(): Unit = { + val oldState = state.get + val newState = oldState.copy(version = oldState.version + 1, history = Map.empty, timestamps = Map.empty) + // if we won the race then update else try again + if (!state.compareAndSet(oldState, newState)) doReset() // recur + } + log.debug("Resetting failure detector") + doReset() + } } + +private[cluster] object HeartbeatHistory { + + /** + * Create an empty HeartbeatHistory, without any history. + * Can only be used as starting point for appending intervals. + * The stats (mean, variance, stdDeviation) are not defined for + * for empty HeartbeatHistory, i.e. throws AritmeticException. + */ + def apply(maxSampleSize: Int): HeartbeatHistory = HeartbeatHistory( + maxSampleSize = maxSampleSize, + intervals = IndexedSeq.empty, + intervalSum = 0L, + squaredIntervalSum = 0L) + +} + +/** + * Holds the heartbeat statistics for a specific node Address. + * It is capped by the number of samples specified in `maxSampleSize`. + * + * The stats (mean, variance, stdDeviation) are not defined for + * for empty HeartbeatHistory, i.e. throws AritmeticException. + */ +private[cluster] case class HeartbeatHistory private ( + maxSampleSize: Int, + intervals: IndexedSeq[Long], + intervalSum: Long, + squaredIntervalSum: Long) { + + if (maxSampleSize < 1) + throw new IllegalArgumentException("maxSampleSize must be >= 1, got [%s]" format maxSampleSize) + if (intervalSum < 0L) + throw new IllegalArgumentException("intervalSum must be >= 0, got [%s]" format intervalSum) + if (squaredIntervalSum < 0L) + throw new IllegalArgumentException("squaredIntervalSum must be >= 0, got [%s]" format squaredIntervalSum) + + def mean: Double = intervalSum.toDouble / intervals.size + + def variance: Double = (squaredIntervalSum.toDouble / intervals.size) - (mean * mean) + + def stdDeviation: Double = math.sqrt(variance) + + @tailrec + final def :+(interval: Long): HeartbeatHistory = { + if (intervals.size < maxSampleSize) + HeartbeatHistory( + maxSampleSize, + intervals = intervals :+ interval, + intervalSum = intervalSum + interval, + squaredIntervalSum = squaredIntervalSum + pow2(interval)) + else + dropOldest :+ interval // recur + } + + private def dropOldest: HeartbeatHistory = HeartbeatHistory( + maxSampleSize, + intervals = intervals drop 1, + intervalSum = intervalSum - intervals.head, + squaredIntervalSum = squaredIntervalSum - pow2(intervals.head)) + + private def pow2(x: Long) = x * x +} \ No newline at end of file diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index e3429cfdb3..44c646ebe8 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -6,27 +6,29 @@ package akka.cluster import akka.actor._ import akka.actor.Status._ +import akka.ConfigurationException +import akka.dispatch.Await +import akka.dispatch.MonitorableThreadFactory +import akka.event.Logging +import akka.jsr166y.ThreadLocalRandom +import akka.pattern._ import akka.remote._ import akka.routing._ -import akka.event.Logging -import akka.dispatch.Await -import akka.pattern.ask import akka.util._ import akka.util.duration._ -import akka.config.ConfigurationException - -import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean } -import java.util.concurrent.TimeUnit._ -import java.util.concurrent.TimeoutException -import java.security.SecureRandom - -import java.lang.management.ManagementFactory -import javax.management._ - -import scala.collection.immutable.{ Map, SortedSet } -import scala.annotation.tailrec - +import akka.util.internal.HashedWheelTimer import com.google.protobuf.ByteString +import java.io.Closeable +import java.lang.management.ManagementFactory +import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean } +import java.util.concurrent.TimeoutException +import java.util.concurrent.TimeUnit._ +import javax.management._ +import MemberStatus._ +import scala.annotation.tailrec +import scala.collection.immutable.{ Map, SortedSet } +import scala.collection.GenTraversableOnce +import java.util.concurrent.atomic.AtomicLong /** * Interface for membership change listener. @@ -44,24 +46,39 @@ trait MetaDataChangeListener { /** * Base trait for all cluster messages. All ClusterMessage's are serializable. + * + * FIXME Protobuf all ClusterMessages */ sealed trait ClusterMessage extends Serializable /** * Cluster commands sent by the USER. */ -object ClusterAction { +object ClusterUserAction { /** - * Command to join the cluster. Sent when a node (reprsesented by 'address') + * Command to join the cluster. Sent when a node (represented by 'address') * wants to join another node (the receiver). */ case class Join(address: Address) extends ClusterMessage /** - * Command to set a node to Up (from Joining). + * Start message of the process to join one of the seed nodes. + * The node sends `InitJoin` to all seed nodes, which replies + * with `InitJoinAck`. The first reply is used others are discarded. + * The node sends `Join` command to the seed node that replied first. */ - case class Up(address: Address) extends ClusterMessage + case object JoinSeedNode extends ClusterMessage + + /** + * @see JoinSeedNode + */ + case object InitJoin extends ClusterMessage + + /** + * @see JoinSeedNode + */ + case class InitJoinAck(address: Address) extends ClusterMessage /** * Command to leave the cluster. @@ -72,20 +89,33 @@ object ClusterAction { * Command to mark node as temporary down. */ case class Down(address: Address) extends ClusterMessage +} + +/** + * Cluster commands sent by the LEADER. + */ +object ClusterLeaderAction { /** + * INTERNAL API. + * * Command to mark a node to be removed from the cluster immediately. + * Can only be sent by the leader. */ - case class Exit(address: Address) extends ClusterMessage + private[cluster] case class Exit(address: Address) extends ClusterMessage /** + * INTERNAL API. + * * Command to remove a node from the cluster immediately. */ - case class Remove(address: Address) extends ClusterMessage + private[cluster] case class Remove(address: Address) extends ClusterMessage } /** * Represents the address and the current status of a cluster member node. + * + * Note: `hashCode` and `equals` are solely based on the underlying `Address`, not its `MemberStatus`. */ class Member(val address: Address, val status: MemberStatus) extends ClusterMessage { override def hashCode = address.## @@ -95,12 +125,28 @@ class Member(val address: Address, val status: MemberStatus) extends ClusterMess } /** - * Factory and Utility module for Member instances. + * Module with factory and ordering methods for Member instances. */ object Member { - import MemberStatus._ - implicit val ordering = Ordering.fromLessThan[Member](_.address.toString < _.address.toString) + /** + * `Address` ordering type class, sorts addresses by host and port. + */ + implicit val addressOrdering: Ordering[Address] = Ordering.fromLessThan[Address] { (a, b) ⇒ + if (a.host != b.host) a.host.getOrElse("").compareTo(b.host.getOrElse("")) < 0 + else if (a.port != b.port) a.port.getOrElse(0) < b.port.getOrElse(0) + else false + } + + /** + * `Member` ordering type class, sorts members by host and port with the exception that + * it puts all members that are in MemberStatus.EXITING last. + */ + implicit val ordering: Ordering[Member] = Ordering.fromLessThan[Member] { (a, b) ⇒ + if (a.status == Exiting && b.status != Exiting) false + else if (a.status != Exiting && b.status == Exiting) true + else addressOrdering.compare(a.address, b.address) < 0 + } def apply(address: Address, status: MemberStatus): Member = new Member(address, status) @@ -109,6 +155,15 @@ object Member { case _ ⇒ None } + def pickHighestPriority(a: Set[Member], b: Set[Member]): Set[Member] = { + // group all members by Address => Seq[Member] + val groupedByAddress = (a.toSeq ++ b.toSeq).groupBy(_.address) + // pick highest MemberStatus + (Set.empty[Member] /: groupedByAddress) { + case (acc, (_, members)) ⇒ acc + members.reduceLeft(highestPriorityOf) + } + } + /** * Picks the Member with the highest "priority" MemberStatus. */ @@ -121,24 +176,45 @@ object Member { case (_, Exiting) ⇒ m2 case (Leaving, _) ⇒ m1 case (_, Leaving) ⇒ m2 - case (Up, Joining) ⇒ m1 - case (Joining, Up) ⇒ m2 + case (Up, Joining) ⇒ m2 + case (Joining, Up) ⇒ m1 case (Joining, Joining) ⇒ m1 case (Up, Up) ⇒ m1 } + + // FIXME Workaround for https://issues.scala-lang.org/browse/SI-5986 + // SortedSet + and ++ operators replaces existing element + // Use these :+ and :++ operators for the Gossip members + implicit def sortedSetWorkaround(sortedSet: SortedSet[Member]): SortedSetWorkaround = new SortedSetWorkaround(sortedSet) + class SortedSetWorkaround(sortedSet: SortedSet[Member]) { + implicit def :+(elem: Member): SortedSet[Member] = { + if (sortedSet.contains(elem)) sortedSet + else sortedSet + elem + } + + implicit def :++(elems: GenTraversableOnce[Member]): SortedSet[Member] = + sortedSet ++ (elems.toSet diff sortedSet) + } } /** * Envelope adding a sender address to the gossip. */ -case class GossipEnvelope(sender: Member, gossip: Gossip) extends ClusterMessage +case class GossipEnvelope(from: Address, gossip: Gossip) extends ClusterMessage /** * Defines the current status of a cluster member node * * Can be one of: Joining, Up, Leaving, Exiting and Down. */ -sealed trait MemberStatus extends ClusterMessage +sealed trait MemberStatus extends ClusterMessage { + + /** + * Using the same notion for 'unavailable' as 'non-convergence': DOWN + */ + def isUnavailable: Boolean = this == Down +} + object MemberStatus { case object Joining extends MemberStatus case object Up extends MemberStatus @@ -146,23 +222,17 @@ object MemberStatus { case object Exiting extends MemberStatus case object Down extends MemberStatus case object Removed extends MemberStatus - - def isUnavailable(status: MemberStatus): Boolean = { - status == MemberStatus.Down || - status == MemberStatus.Exiting || - status == MemberStatus.Removed || - status == MemberStatus.Leaving - } } /** * Represents the overview of the cluster, holds the cluster convergence table and set with unreachable nodes. */ case class GossipOverview( - seen: Map[Address, VectorClock] = Map.empty[Address, VectorClock], - unreachable: Set[Member] = Set.empty[Member]) { + seen: Map[Address, VectorClock] = Map.empty, + unreachable: Set[Member] = Set.empty) { - // FIXME document when nodes are put in 'unreachable' set and removed from 'members' + def isNonDownUnreachable(address: Address): Boolean = + unreachable.exists { m ⇒ m.address == address && m.status != Down } override def toString = "GossipOverview(seen = [" + seen.mkString(", ") + @@ -170,30 +240,89 @@ case class GossipOverview( "])" } +object Gossip { + val emptyMembers: SortedSet[Member] = SortedSet.empty + +} + /** - * Represents the state of the cluster; cluster ring membership, ring convergence, meta data - all versioned by a vector clock. + * Represents the state of the cluster; cluster ring membership, ring convergence, meta data - + * all versioned by a vector clock. + * + * When a node is joining the `Member`, with status `Joining`, is added to `members`. + * If the joining node was downed it is moved from `overview.unreachable` (status `Down`) + * to `members` (status `Joining`). It cannot rejoin if not first downed. + * + * When convergence is reached the leader change status of `members` from `Joining` + * to `Up`. + * + * When failure detector consider a node as unavailable it will be moved from + * `members` to `overview.unreachable`. + * + * When a node is downed, either manually or automatically, its status is changed to `Down`. + * It is also removed from `overview.seen` table. The node will reside as `Down` in the + * `overview.unreachable` set until joining again and it will then go through the normal + * joining procedure. + * + * When a `Gossip` is received the version (vector clock) is used to determine if the + * received `Gossip` is newer or older than the current local `Gossip`. The received `Gossip` + * and local `Gossip` is merged in case of conflicting version, i.e. vector clocks without + * same history. When merged the seen table is cleared. + * + * When a node is told by the user to leave the cluster the leader will move it to `Leaving` + * and then rebalance and repartition the cluster and start hand-off by migrating the actors + * from the leaving node to the new partitions. Once this process is complete the leader will + * move the node to the `Exiting` state and once a convergence is complete move the node to + * `Removed` by removing it from the `members` set and sending a `Removed` command to the + * removed node telling it to shut itself down. */ case class Gossip( overview: GossipOverview = GossipOverview(), - members: SortedSet[Member], // sorted set of members with their status, sorted by name - meta: Map[String, Array[Byte]] = Map.empty[String, Array[Byte]], + members: SortedSet[Member], // sorted set of members with their status, sorted by address + meta: Map[String, Array[Byte]] = Map.empty, version: VectorClock = VectorClock()) // vector clock version extends ClusterMessage // is a serializable cluster message with Versioned[Gossip] { - /** - * Increments the version for this 'Node'. - */ - def +(node: VectorClock.Node): Gossip = copy(version = version + node) + // FIXME can be disabled as optimization + assertInvariants + + private def assertInvariants: Unit = { + val unreachableAndLive = members.intersect(overview.unreachable) + if (unreachableAndLive.nonEmpty) + throw new IllegalArgumentException("Same nodes in both members and unreachable is not allowed, got [%s]" + format unreachableAndLive.mkString(", ")) + + val allowedLiveMemberStatuses: Set[MemberStatus] = Set(Joining, Up, Leaving, Exiting) + def hasNotAllowedLiveMemberStatus(m: Member) = !allowedLiveMemberStatuses.contains(m.status) + if (members exists hasNotAllowedLiveMemberStatus) + throw new IllegalArgumentException("Live members must have status [%s], got [%s]" + format (allowedLiveMemberStatuses.mkString(", "), + (members filter hasNotAllowedLiveMemberStatus).mkString(", "))) + + val seenButNotMember = overview.seen.keySet -- members.map(_.address) -- overview.unreachable.map(_.address) + if (seenButNotMember.nonEmpty) + throw new IllegalArgumentException("Nodes not part of cluster have marked the Gossip as seen, got [%s]" + format seenButNotMember.mkString(", ")) - def +(member: Member): Gossip = { - if (members contains member) this - else this copy (members = members + member) } /** - * Marks the gossip as seen by this node (remoteAddress) by updating the address entry in the 'gossip.overview.seen' - * Map with the VectorClock for the new gossip. + * Increments the version for this 'Node'. + */ + def :+(node: VectorClock.Node): Gossip = copy(version = version :+ node) + + /** + * Adds a member to the member node ring. + */ + def :+(member: Member): Gossip = { + if (members contains member) this + else this copy (members = members :+ member) + } + + /** + * Marks the gossip as seen by this node (address) by updating the address entry in the 'gossip.overview.seen' + * Map with the VectorClock (version) for the new gossip. */ def seen(address: Address): Gossip = { if (overview.seen.contains(address) && overview.seen(address) == version) this @@ -209,29 +338,20 @@ case class Gossip( // 1. merge vector clocks val mergedVClock = this.version merge that.version - // 2. group all members by Address => Vector[Member] - var membersGroupedByAddress = Map.empty[Address, Vector[Member]] - (this.members ++ that.members) foreach { m ⇒ - val ms = membersGroupedByAddress.get(m.address).getOrElse(Vector.empty[Member]) - membersGroupedByAddress += (m.address -> (ms :+ m)) - } - - // 3. merge members by selecting the single Member with highest MemberStatus out of the Member groups - val mergedMembers = - SortedSet.empty[Member] ++ - membersGroupedByAddress.values.foldLeft(Vector.empty[Member]) { (acc, members) ⇒ - acc :+ members.reduceLeft(Member.highestPriorityOf(_, _)) - } - - // 4. merge meta-data + // 2. merge meta-data val mergedMeta = this.meta ++ that.meta - // 5. merge gossip overview - val mergedOverview = GossipOverview( - this.overview.seen ++ that.overview.seen, - this.overview.unreachable ++ that.overview.unreachable) + // 3. merge unreachable by selecting the single Member with highest MemberStatus out of the Member groups + val mergedUnreachable = Member.pickHighestPriority(this.overview.unreachable, that.overview.unreachable) - Gossip(mergedOverview, mergedMembers, mergedMeta, mergedVClock) + // 4. merge members by selecting the single Member with highest MemberStatus out of the Member groups, + // and exclude unreachable + val mergedMembers = Gossip.emptyMembers :++ Member.pickHighestPriority(this.members, that.members).filterNot(mergedUnreachable.contains) + + // 5. fresh seen table + val mergedSeen = Map.empty[Address, VectorClock] + + Gossip(GossipOverview(mergedSeen, mergedUnreachable), mergedMembers, mergedMeta, mergedVClock) } override def toString = @@ -244,52 +364,83 @@ case class Gossip( } /** + * Sent at regular intervals for failure detection. + */ +case class Heartbeat(from: Address) extends ClusterMessage + +/** + * INTERNAL API. + * * Manages routing of the different cluster commands. * Instantiated as a single instance for each Cluster - e.g. commands are serialized to Cluster message after message. */ -final class ClusterCommandDaemon extends Actor { - import ClusterAction._ +private[cluster] final class ClusterCommandDaemon(cluster: Cluster) extends Actor { + import ClusterUserAction._ + import ClusterLeaderAction._ - val cluster = Cluster(context.system) val log = Logging(context.system, this) def receive = { - case Join(address) ⇒ cluster.joining(address) - case Up(address) ⇒ cluster.up(address) - case Down(address) ⇒ cluster.downing(address) - case Leave(address) ⇒ cluster.leaving(address) - case Exit(address) ⇒ cluster.exiting(address) - case Remove(address) ⇒ cluster.removing(address) + case JoinSeedNode ⇒ joinSeedNode() + case InitJoin ⇒ sender ! InitJoinAck(cluster.selfAddress) + case InitJoinAck(address) ⇒ cluster.join(address) + case Join(address) ⇒ cluster.joining(address) + case Down(address) ⇒ cluster.downing(address) + case Leave(address) ⇒ cluster.leaving(address) + case Exit(address) ⇒ cluster.exiting(address) + case Remove(address) ⇒ cluster.removing(address) + case Failure(e: AskTimeoutException) ⇒ joinSeedNodeTimeout() } + def joinSeedNode(): Unit = { + val seedRoutees = for (address ← cluster.seedNodes; if address != cluster.selfAddress) + yield self.path.toStringWithAddress(address) + if (seedRoutees.isEmpty) { + cluster join cluster.selfAddress + } else { + implicit val within = Timeout(cluster.clusterSettings.SeedNodeTimeout) + val seedRouter = context.actorOf( + Props.empty.withRouter(ScatterGatherFirstCompletedRouter( + routees = seedRoutees, within = within.duration))) + seedRouter ? InitJoin pipeTo self + seedRouter ! PoisonPill + } + } + + def joinSeedNodeTimeout(): Unit = cluster join cluster.selfAddress + override def unhandled(unknown: Any) = log.error("Illegal command [{}]", unknown) } /** + * INTERNAL API. + * * Pooled and routed with N number of configurable instances. * Concurrent access to Cluster. */ -final class ClusterGossipDaemon extends Actor { +private[cluster] final class ClusterGossipDaemon(cluster: Cluster) extends Actor { val log = Logging(context.system, this) - val cluster = Cluster(context.system) def receive = { - case GossipEnvelope(sender, gossip) ⇒ cluster.receive(sender, gossip) + case Heartbeat(from) ⇒ cluster.receiveHeartbeat(from) + case GossipEnvelope(from, gossip) ⇒ cluster.receiveGossip(from, gossip) } override def unhandled(unknown: Any) = log.error("[/system/cluster/gossip] can not respond to messages - received [{}]", unknown) } /** + * INTERNAL API. + * * Supervisor managing the different Cluster daemons. */ -final class ClusterDaemonSupervisor extends Actor { +private[cluster] final class ClusterDaemonSupervisor(cluster: Cluster) extends Actor { val log = Logging(context.system, this) - val cluster = Cluster(context.system) - private val commands = context.actorOf(Props[ClusterCommandDaemon], "commands") + private val commands = context.actorOf(Props(new ClusterCommandDaemon(cluster)), "commands") private val gossip = context.actorOf( - Props[ClusterGossipDaemon].withRouter(RoundRobinRouter(cluster.clusterSettings.NrOfGossipDaemons)), "gossip") + Props(new ClusterGossipDaemon(cluster)).withRouter( + RoundRobinRouter(cluster.clusterSettings.NrOfGossipDaemons)), "gossip") def receive = Actor.emptyBehavior @@ -308,7 +459,19 @@ object Cluster extends ExtensionId[Cluster] with ExtensionIdProvider { override def lookup = Cluster - override def createExtension(system: ExtendedActorSystem): Cluster = new Cluster(system) + override def createExtension(system: ExtendedActorSystem): Cluster = { + val clusterSettings = new ClusterSettings(system.settings.config, system.name) + + val failureDetector = { + import clusterSettings.{ FailureDetectorImplementationClass ⇒ fqcn } + system.dynamicAccess.createInstanceFor[FailureDetector]( + fqcn, Seq(classOf[ActorSystem] -> system, classOf[ClusterSettings] -> clusterSettings)).fold( + e ⇒ throw new ConfigurationException("Could not create custom failure detector [" + fqcn + "] due to:" + e.toString), + identity) + } + + new Cluster(system, failureDetector) + } } /** @@ -322,27 +485,22 @@ trait ClusterNodeMBean { def isSingleton: Boolean def isConvergence: Boolean def isAvailable: Boolean + def isRunning: Boolean def join(address: String) def leave(address: String) def down(address: String) - def remove(address: String) - - def shutdown() } /** * This module is responsible for Gossiping cluster information. The abstraction maintains the list of live * and dead members. Periodically i.e. every 1 second this module chooses a random member and initiates a round - * of Gossip with it. Whenever it gets gossip updates it updates the Failure Detector with the liveness - * information. + * of Gossip with it. *

- * During each of these runs the member initiates gossip exchange according to following rules (as defined in the - * Cassandra documentation [http://wiki.apache.org/cassandra/ArchitectureGossip]: + * During each of these runs the member initiates gossip exchange according to following rules: *

  *   1) Gossip to random live member (if any)
- *   2) Gossip to random unreachable member with certain probability depending on number of unreachable and live members
- *   3) If the member gossiped to at (1) was not deputy, or the number of live members is less than number of deputy list,
+ *   2) If the member gossiped to at (1) was not deputy, or the number of live members is less than number of deputy list,
  *       gossip to random deputy with certain probability depending on number of unreachable, deputy and live members.
  * 
* @@ -351,7 +509,7 @@ trait ClusterNodeMBean { * if (Cluster(system).isLeader) { ... } * }}} */ -class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ +class Cluster(system: ExtendedActorSystem, val failureDetector: FailureDetector) extends Extension { clusterNode ⇒ /** * Represents the state for this Cluster. Implemented using optimistic lockless concurrency. @@ -359,7 +517,8 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ */ private case class State( latestGossip: Gossip, - memberMembershipChangeListeners: Set[MembershipChangeListener] = Set.empty[MembershipChangeListener]) + joinInProgress: Map[Address, Deadline] = Map.empty, + memberMembershipChangeListeners: Set[MembershipChangeListener] = Set.empty) if (!system.provider.isInstanceOf[RemoteActorRefProvider]) throw new ConfigurationException("ActorSystem[" + system + "] needs to have a 'RemoteActorRefProvider' enabled in the configuration") @@ -368,84 +527,133 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ val remoteSettings = new RemoteSettings(system.settings.config, system.name) val clusterSettings = new ClusterSettings(system.settings.config, system.name) + import clusterSettings._ - val remoteAddress = remote.transport.address - val failureDetector = new AccrualFailureDetector( - system, remoteAddress, clusterSettings.FailureDetectorThreshold, clusterSettings.FailureDetectorMaxSampleSize) + val selfAddress = remote.transport.address + private val selfHeartbeat = Heartbeat(selfAddress) - private val vclockNode = VectorClock.Node(remoteAddress.toString) - - private val periodicTasksInitialDelay = clusterSettings.PeriodicTasksInitialDelay - private val gossipFrequency = clusterSettings.GossipFrequency - private val leaderActionsFrequency = clusterSettings.LeaderActionsFrequency - private val unreachableNodesReaperFrequency = clusterSettings.UnreachableNodesReaperFrequency + private val vclockNode = VectorClock.Node(selfAddress.toString) implicit private val defaultTimeout = Timeout(remoteSettings.RemoteSystemDaemonAckTimeout) - private val autoDown = clusterSettings.AutoDown - private val nrOfDeputyNodes = clusterSettings.NrOfDeputyNodes - private val nrOfGossipDaemons = clusterSettings.NrOfGossipDaemons - private val nodeToJoin: Option[Address] = clusterSettings.NodeToJoin filter (_ != remoteAddress) - private val serialization = remote.serialization - private val isRunning = new AtomicBoolean(true) + private val _isRunning = new AtomicBoolean(true) private val log = Logging(system, "Node") - private val random = SecureRandom.getInstance("SHA1PRNG") private val mBeanServer = ManagementFactory.getPlatformMBeanServer private val clusterMBeanName = new ObjectName("akka:type=Cluster") - log.info("Cluster Node [{}] - is starting up...", remoteAddress) + log.info("Cluster Node [{}] - is starting up...", selfAddress) - // create superisor for daemons under path "/system/cluster" + // create supervisor for daemons under path "/system/cluster" private val clusterDaemons = { - val createChild = CreateChild(Props[ClusterDaemonSupervisor], "cluster") + val createChild = CreateChild(Props(new ClusterDaemonSupervisor(this)), "cluster") Await.result(system.systemGuardian ? createChild, defaultTimeout.duration) match { case a: ActorRef ⇒ a case e: Exception ⇒ throw e } } - private val state = { - val member = Member(remoteAddress, MemberStatus.Joining) - val gossip = Gossip(members = SortedSet.empty[Member] + member) + vclockNode // add me as member and update my vector clock - new AtomicReference[State](State(gossip)) + private def createCleanState: State = { + // note that self is not initially member, + // and the Gossip is not versioned for this 'Node' yet + State(Gossip(members = Gossip.emptyMembers)) } - // try to join the node defined in the 'akka.cluster.node-to-join' option - autoJoin() + private val state = new AtomicReference[State](createCleanState) + + // try to join one of the nodes defined in the 'akka.cluster.seed-nodes' + if (AutoJoin) joinSeedNode() // ======================================================== // ===================== WORK DAEMONS ===================== // ======================================================== - // start periodic gossip to random nodes in cluster - private val gossipCanceller = system.scheduler.schedule(periodicTasksInitialDelay, gossipFrequency) { - gossip() + private val clusterScheduler: Scheduler with Closeable = { + if (system.settings.SchedulerTickDuration > SchedulerTickDuration) { + log.info("Using a dedicated scheduler for cluster. Default scheduler can be used if configured " + + "with 'akka.scheduler.tick-duration' [{} ms] <= 'akka.cluster.scheduler.tick-duration' [{} ms].", + system.settings.SchedulerTickDuration.toMillis, SchedulerTickDuration.toMillis) + val threadFactory = system.threadFactory match { + case tf: MonitorableThreadFactory ⇒ tf.copy(name = tf.name + "-cluster-scheduler") + case tf ⇒ tf + } + val hwt = new HashedWheelTimer(log, + threadFactory, + SchedulerTickDuration, SchedulerTicksPerWheel) + new DefaultScheduler(hwt, log, system.dispatcher) + } else { + // delegate to system.scheduler, but don't close + val systemScheduler = system.scheduler + new Scheduler with Closeable { + // we are using system.scheduler, which we are not responsible for closing + def close(): Unit = () + def schedule(initialDelay: Duration, frequency: Duration, receiver: ActorRef, message: Any): Cancellable = + systemScheduler.schedule(initialDelay, frequency, receiver, message) + def schedule(initialDelay: Duration, frequency: Duration)(f: ⇒ Unit): Cancellable = + systemScheduler.schedule(initialDelay, frequency)(f) + def schedule(initialDelay: Duration, frequency: Duration, runnable: Runnable): Cancellable = + systemScheduler.schedule(initialDelay, frequency, runnable) + def scheduleOnce(delay: Duration, runnable: Runnable): Cancellable = + systemScheduler.scheduleOnce(delay, runnable) + def scheduleOnce(delay: Duration, receiver: ActorRef, message: Any): Cancellable = + systemScheduler.scheduleOnce(delay, receiver, message) + def scheduleOnce(delay: Duration)(f: ⇒ Unit): Cancellable = + systemScheduler.scheduleOnce(delay)(f) + } + } } + // start periodic gossip to random nodes in cluster + private val gossipTask = + FixedRateTask(clusterScheduler, PeriodicTasksInitialDelay.max(GossipInterval), GossipInterval) { + gossip() + } + + // start periodic heartbeat to all nodes in cluster + private val heartbeatTask = + FixedRateTask(clusterScheduler, PeriodicTasksInitialDelay.max(HeartbeatInterval), HeartbeatInterval) { + heartbeat() + } + // start periodic cluster failure detector reaping (moving nodes condemned by the failure detector to unreachable list) - private val failureDetectorReaperCanceller = system.scheduler.schedule(periodicTasksInitialDelay, unreachableNodesReaperFrequency) { - reapUnreachableMembers() - } + private val failureDetectorReaperTask = + FixedRateTask(clusterScheduler, PeriodicTasksInitialDelay.max(UnreachableNodesReaperInterval), UnreachableNodesReaperInterval) { + reapUnreachableMembers() + } // start periodic leader action management (only applies for the current leader) - private val leaderActionsCanceller = system.scheduler.schedule(periodicTasksInitialDelay, leaderActionsFrequency) { - leaderActions() - } + private val leaderActionsTask = + FixedRateTask(clusterScheduler, PeriodicTasksInitialDelay.max(LeaderActionsInterval), LeaderActionsInterval) { + leaderActions() + } createMBean() - log.info("Cluster Node [{}] - has started up successfully", remoteAddress) + system.registerOnTermination(shutdown()) + + log.info("Cluster Node [{}] - has started up successfully", selfAddress) // ====================================================== // ===================== PUBLIC API ===================== // ====================================================== - def self: Member = latestGossip.members - .find(_.address == remoteAddress) - .getOrElse(throw new IllegalStateException("Can't find 'this' Member (" + remoteAddress + ") in the cluster membership ring")) + def self: Member = { + val gossip = latestGossip + gossip.members + .find(_.address == selfAddress) + .getOrElse { + gossip.overview.unreachable + .find(_.address == selfAddress) + .getOrElse(throw new IllegalStateException("Can't find 'this' Member [" + selfAddress + "] in the cluster membership ring or in the unreachable set")) + } + } + + /** + * Returns true if the cluster node is up and running, false if it is shut down. + */ + def isRunning: Boolean = _isRunning.get /** * Latest gossip. @@ -453,16 +661,23 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ def latestGossip: Gossip = state.get.latestGossip /** - * Member status for this node. + * Member status for this node (`MemberStatus`). + * + * NOTE: If the node has been removed from the cluster (and shut down) then it's status is set to the 'REMOVED' tombstone state + * and is no longer present in the node ring or any other part of the gossiping state. However in order to maintain the + * model and the semantics the user would expect, this method will in this situation return `MemberStatus.Removed`. */ - def status: MemberStatus = self.status + def status: MemberStatus = { + if (isRunning) self.status + else MemberStatus.Removed + } /** * Is this node the leader? */ def isLeader: Boolean = { val members = latestGossip.members - !members.isEmpty && (remoteAddress == members.head.address) + members.nonEmpty && (selfAddress == members.head.address) } /** @@ -478,7 +693,7 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ /** * Checks if we have a cluster convergence. * - * @returns Some(convergedGossip) if convergence have been reached and None if not + * @return Some(convergedGossip) if convergence have been reached and None if not */ def convergence: Option[Gossip] = convergence(latestGossip) @@ -488,28 +703,16 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ def isAvailable: Boolean = !isUnavailable(state.get) /** - * Shuts down all connections to other members, the cluster daemon and the periodic gossip and cleanup tasks. + * Make it possible to override/configure seedNodes from tests without + * specifying in config. Addresses are unknown before startup time. */ - def shutdown() { - if (isRunning.compareAndSet(true, false)) { - log.info("Cluster Node [{}] - Shutting down cluster Node and cluster daemons...", remoteAddress) - gossipCanceller.cancel() - failureDetectorReaperCanceller.cancel() - leaderActionsCanceller.cancel() - system.stop(clusterDaemons) - try { - mBeanServer.unregisterMBean(clusterMBeanName) - } catch { - case e: InstanceNotFoundException ⇒ // ignore - we are running multiple cluster nodes in the same JVM (probably for testing) - } - } - } + def seedNodes: IndexedSeq[Address] = SeedNodes /** * Registers a listener to subscribe to cluster membership changes. */ @tailrec - final def registerListener(listener: MembershipChangeListener) { + final def registerListener(listener: MembershipChangeListener): Unit = { val localState = state.get val newListeners = localState.memberMembershipChangeListeners + listener val newState = localState copy (memberMembershipChangeListeners = newListeners) @@ -520,7 +723,7 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ * Unsubscribes to cluster membership changes. */ @tailrec - final def unregisterListener(listener: MembershipChangeListener) { + final def unregisterListener(listener: MembershipChangeListener): Unit = { val localState = state.get val newListeners = localState.memberMembershipChangeListeners - listener val newState = localState copy (memberMembershipChangeListeners = newListeners) @@ -531,32 +734,35 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ * Try to join this cluster node with the node specified by 'address'. * A 'Join(thisNodeAddress)' command is sent to the node to join. */ - def join(address: Address) { - val connection = clusterCommandConnectionFor(address) - val command = ClusterAction.Join(remoteAddress) - log.info("Cluster Node [{}] - Trying to send JOIN to [{}] through connection [{}]", remoteAddress, address, connection) - connection ! command + @tailrec + final def join(address: Address): Unit = { + val localState = state.get + // wipe our state since a node that joins a cluster must be empty + val newState = createCleanState copy (joinInProgress = Map.empty + (address -> (Deadline.now + JoinTimeout)), + memberMembershipChangeListeners = localState.memberMembershipChangeListeners) + // wipe the failure detector since we are starting fresh and shouldn't care about the past + failureDetector.reset() + if (!state.compareAndSet(localState, newState)) join(address) // recur + else { + val connection = clusterCommandConnectionFor(address) + val command = ClusterUserAction.Join(selfAddress) + log.info("Cluster Node [{}] - Trying to send JOIN to [{}] through connection [{}]", selfAddress, address, connection) + connection ! command + } } /** * Send command to issue state transition to LEAVING for the node specified by 'address'. */ - def leave(address: Address) { - clusterCommandDaemon ! ClusterAction.Leave(address) + def leave(address: Address): Unit = { + clusterCommandDaemon ! ClusterUserAction.Leave(address) } /** - * Send command to issue state transition to from DOWN to EXITING for the node specified by 'address'. + * Send command to DOWN the node specified by 'address'. */ - def down(address: Address) { - clusterCommandDaemon ! ClusterAction.Down(address) - } - - /** - * Send command to issue state transition to REMOVED for the node specified by 'address'. - */ - def remove(address: Address) { - clusterCommandDaemon ! ClusterAction.Remove(address) + def down(address: Address): Unit = { + clusterCommandDaemon ! ClusterUserAction.Down(address) } // ======================================================== @@ -564,77 +770,148 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ // ======================================================== /** - * State transition to JOINING. - * New node joining. + * INTERNAL API. + * + * Shuts down all connections to other members, the cluster daemon and the periodic gossip and cleanup tasks. + * + * Should not called by the user. The user can issue a LEAVE command which will tell the node + * to go through graceful handoff process `LEAVE -> EXITING -> REMOVED -> SHUTDOWN`. + */ + private[cluster] def shutdown(): Unit = { + if (_isRunning.compareAndSet(true, false)) { + log.info("Cluster Node [{}] - Shutting down cluster Node and cluster daemons...", selfAddress) + + // cancel the periodic tasks, note that otherwise they will be run when scheduler is shutdown + gossipTask.cancel() + heartbeatTask.cancel() + failureDetectorReaperTask.cancel() + leaderActionsTask.cancel() + clusterScheduler.close() + + // FIXME isTerminated check can be removed when ticket #2221 is fixed + // now it prevents logging if system is shutdown (or in progress of shutdown) + if (!clusterDaemons.isTerminated) + system.stop(clusterDaemons) + + try { + mBeanServer.unregisterMBean(clusterMBeanName) + } catch { + case e: InstanceNotFoundException ⇒ // ignore - we are running multiple cluster nodes in the same JVM (probably for testing) + } + log.info("Cluster Node [{}] - Cluster node successfully shut down", selfAddress) + } + } + + /** + * INTERNAL API. + * + * State transition to JOINING - new node joining. */ @tailrec - private[cluster] final def joining(node: Address) { - log.info("Cluster Node [{}] - Node [{}] is JOINING", remoteAddress, node) - + private[cluster] final def joining(node: Address): Unit = { val localState = state.get val localGossip = localState.latestGossip val localMembers = localGossip.members - val localOverview = localGossip.overview - val localUnreachableMembers = localOverview.unreachable + val localUnreachable = localGossip.overview.unreachable - // remove the node from the 'unreachable' set in case it is a DOWN node that is rejoining cluster - val newUnreachableMembers = localUnreachableMembers filterNot { _.address == node } - val newOverview = localOverview copy (unreachable = newUnreachableMembers) + val alreadyMember = localMembers.exists(_.address == node) + val isUnreachable = localGossip.overview.isNonDownUnreachable(node) - val newMembers = localMembers + Member(node, MemberStatus.Joining) // add joining node as Joining - val newGossip = localGossip copy (overview = newOverview, members = newMembers) + if (!alreadyMember && !isUnreachable) { - val versionedGossip = newGossip + vclockNode - val seenVersionedGossip = versionedGossip seen remoteAddress + // remove the node from the 'unreachable' set in case it is a DOWN node that is rejoining cluster + val (rejoiningMember, newUnreachableMembers) = localUnreachable partition { _.address == node } + val newOverview = localGossip.overview copy (unreachable = newUnreachableMembers) - val newState = localState copy (latestGossip = seenVersionedGossip) + // remove the node from the failure detector if it is a DOWN node that is rejoining cluster + if (rejoiningMember.nonEmpty) failureDetector.remove(node) - if (!state.compareAndSet(localState, newState)) joining(node) // recur if we failed update - else { - failureDetector heartbeat node // update heartbeat in failure detector - if (convergence(newState.latestGossip).isDefined) { - newState.memberMembershipChangeListeners foreach { _ notify newMembers } + // add joining node as Joining + // add self in case someone else joins before self has joined (Set discards duplicates) + val newMembers = localMembers :+ Member(node, Joining) :+ Member(selfAddress, Joining) + val newGossip = localGossip copy (overview = newOverview, members = newMembers) + + val versionedGossip = newGossip :+ vclockNode + val seenVersionedGossip = versionedGossip seen selfAddress + + val newState = localState copy (latestGossip = seenVersionedGossip) + + if (!state.compareAndSet(localState, newState)) joining(node) // recur if we failed update + else { + log.info("Cluster Node [{}] - Node [{}] is JOINING", selfAddress, node) + // treat join as initial heartbeat, so that it becomes unavailable if nothing more happens + if (node != selfAddress) { + failureDetector heartbeat node + gossipTo(node) + } + notifyMembershipChangeListeners(localState, newState) } } } /** - * State transition to UP. - */ - private[cluster] final def up(address: Address) { - log.info("Cluster Node [{}] - Marking node [{}] as UP", remoteAddress, address) - } - - /** + * INTERNAL API. + * * State transition to LEAVING. */ + @tailrec private[cluster] final def leaving(address: Address) { - log.info("Cluster Node [{}] - Marking node [{}] as LEAVING", remoteAddress, address) + val localState = state.get + val localGossip = localState.latestGossip + if (localGossip.members.exists(_.address == address)) { // only try to update if the node is available (in the member ring) + val newMembers = localGossip.members map { member ⇒ if (member.address == address) Member(address, Leaving) else member } // mark node as LEAVING + val newGossip = localGossip copy (members = newMembers) + + val versionedGossip = newGossip :+ vclockNode + val seenVersionedGossip = versionedGossip seen selfAddress + + val newState = localState copy (latestGossip = seenVersionedGossip) + + if (!state.compareAndSet(localState, newState)) leaving(address) // recur if we failed update + else { + log.info("Cluster Node [{}] - Marked address [{}] as LEAVING", selfAddress, address) + notifyMembershipChangeListeners(localState, newState) + } + } } /** + * INTERNAL API. + * * State transition to EXITING. */ - private[cluster] final def exiting(address: Address) { - log.info("Cluster Node [{}] - Marking node [{}] as EXITING", remoteAddress, address) + private[cluster] final def exiting(address: Address): Unit = { + log.info("Cluster Node [{}] - Marked node [{}] as EXITING", selfAddress, address) + // FIXME implement when we implement hand-off } /** + * INTERNAL API. + * * State transition to REMOVED. + * + * This method is for now only called after the LEADER have sent a Removed message - telling the node + * to shut down himself. + * + * In the future we might change this to allow the USER to send a Removed(address) message telling an + * arbitrary node to be moved direcly from UP -> REMOVED. */ - private[cluster] final def removing(address: Address) { - log.info("Cluster Node [{}] - Marking node [{}] as REMOVED", remoteAddress, address) + private[cluster] final def removing(address: Address): Unit = { + log.info("Cluster Node [{}] - Node has been REMOVED by the leader - shutting down...", selfAddress) + shutdown() } /** - * The node to DOWN is removed from the 'members' set and put in the 'unreachable' set (if not alread there) - * and its status is set to DOWN. The node is alo removed from the 'seen' table. + * INTERNAL API. + * + * The node to DOWN is removed from the 'members' set and put in the 'unreachable' set (if not already there) + * and its status is set to DOWN. The node is also removed from the 'seen' table. * * The node will reside as DOWN in the 'unreachable' set until an explicit command JOIN command is sent directly * to this node and it will then go through the normal JOINING procedure. */ @tailrec - final private[cluster] def downing(address: Address) { + final private[cluster] def downing(address: Address): Unit = { val localState = state.get val localGossip = localState.latestGossip val localMembers = localGossip.members @@ -643,213 +920,237 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ val localUnreachableMembers = localOverview.unreachable // 1. check if the node to DOWN is in the 'members' set - var downedMember: Option[Member] = None - val newMembers = - localMembers - .map { member ⇒ - if (member.address == address) { - log.info("Cluster Node [{}] - Marking node [{}] as DOWN", remoteAddress, member.address) - val newMember = member copy (status = MemberStatus.Down) - downedMember = Some(newMember) - newMember - } else member - } - .filter(_.status != MemberStatus.Down) + val downedMember: Option[Member] = localMembers.collectFirst { + case m if m.address == address ⇒ m.copy(status = Down) + } + val newMembers = downedMember match { + case Some(m) ⇒ + log.info("Cluster Node [{}] - Marking node [{}] as DOWN", selfAddress, m.address) + localMembers - m + case None ⇒ localMembers + } // 2. check if the node to DOWN is in the 'unreachable' set val newUnreachableMembers = - localUnreachableMembers - .filter(_.status != MemberStatus.Down) // no need to DOWN members already DOWN - .map { member ⇒ - if (member.address == address) { - log.info("Cluster Node [{}] - Marking unreachable node [{}] as DOWN", remoteAddress, member.address) - member copy (status = MemberStatus.Down) - } else member - } + localUnreachableMembers.map { member ⇒ + // no need to DOWN members already DOWN + if (member.address == address && member.status != Down) { + log.info("Cluster Node [{}] - Marking unreachable node [{}] as DOWN", selfAddress, member.address) + member copy (status = Down) + } else member + } // 3. add the newly DOWNED members from the 'members' (in step 1.) to the 'newUnreachableMembers' set. - val newUnreachablePlusNewlyDownedMembers = downedMember match { - case Some(member) ⇒ newUnreachableMembers + member - case None ⇒ newUnreachableMembers - } + val newUnreachablePlusNewlyDownedMembers = newUnreachableMembers ++ downedMember // 4. remove nodes marked as DOWN from the 'seen' table - val newSeen = newUnreachablePlusNewlyDownedMembers.foldLeft(localSeen) { (currentSeen, member) ⇒ - currentSeen - member.address + val newSeen = localSeen -- newUnreachablePlusNewlyDownedMembers.collect { + case m if m.status == Down ⇒ m.address } - val newOverview = localOverview copy (seen = newSeen, unreachable = newUnreachablePlusNewlyDownedMembers) // update gossip overview + // update gossip overview + val newOverview = localOverview copy (seen = newSeen, unreachable = newUnreachablePlusNewlyDownedMembers) val newGossip = localGossip copy (overview = newOverview, members = newMembers) // update gossip - val versionedGossip = newGossip + vclockNode - val newState = localState copy (latestGossip = versionedGossip seen remoteAddress) + val versionedGossip = newGossip :+ vclockNode + val newState = localState copy (latestGossip = versionedGossip seen selfAddress) if (!state.compareAndSet(localState, newState)) downing(address) // recur if we fail the update else { - if (convergence(newState.latestGossip).isDefined) { - newState.memberMembershipChangeListeners foreach { _ notify newState.latestGossip.members } - } + notifyMembershipChangeListeners(localState, newState) } } + // Can be removed when gossip has been optimized + private val _receivedGossipCount = new AtomicLong /** + * INTERNAL API. + */ + private[cluster] def receivedGossipCount: Long = _receivedGossipCount.get + + /** + * INTERNAL API. + * * Receive new gossip. */ @tailrec - final private[cluster] def receive(sender: Member, remoteGossip: Gossip) { + final private[cluster] def receiveGossip(from: Address, remoteGossip: Gossip): Unit = { val localState = state.get val localGossip = localState.latestGossip - val winningGossip = - if (remoteGossip.version <> localGossip.version) { - // concurrent - val mergedGossip = remoteGossip merge localGossip - val versionedMergedGossip = mergedGossip + vclockNode + if (!localGossip.overview.isNonDownUnreachable(from)) { - log.debug( - "Can't establish a causal relationship between \"remote\" gossip [{}] and \"local\" gossip [{}] - merging them into [{}]", - remoteGossip, localGossip, versionedMergedGossip) + val winningGossip = + if (remoteGossip.version <> localGossip.version) { + // concurrent + val mergedGossip = remoteGossip merge localGossip + val versionedMergedGossip = mergedGossip :+ vclockNode - versionedMergedGossip + versionedMergedGossip - } else if (remoteGossip.version < localGossip.version) { - // local gossip is newer - localGossip + } else if (remoteGossip.version < localGossip.version) { + // local gossip is newer + localGossip - } else { - // remote gossip is newer - remoteGossip + } else { + // remote gossip is newer + remoteGossip + } + + val newJoinInProgress = + if (localState.joinInProgress.isEmpty) localState.joinInProgress + else localState.joinInProgress -- + winningGossip.members.map(_.address) -- + winningGossip.overview.unreachable.map(_.address) + + val newState = localState copy ( + latestGossip = winningGossip seen selfAddress, + joinInProgress = newJoinInProgress) + + // for all new joining nodes we optimistically remove them from the failure detector, since if we wait until + // we have won the CAS, then the node might be picked up by the reapUnreachableMembers task and moved to + // unreachable before we can remove the node from the failure detector + (newState.latestGossip.members -- localState.latestGossip.members).filter(_.status == Joining).foreach { + case node ⇒ failureDetector.remove(node.address) } - val newState = localState copy (latestGossip = winningGossip seen remoteAddress) + // if we won the race then update else try again + if (!state.compareAndSet(localState, newState)) receiveGossip(from, remoteGossip) // recur if we fail the update + else { + log.debug("Cluster Node [{}] - Receiving gossip from [{}]", selfAddress, from) - // if we won the race then update else try again - if (!state.compareAndSet(localState, newState)) receive(sender, remoteGossip) // recur if we fail the update - else { - log.debug("Cluster Node [{}] - Receiving gossip from [{}]", remoteAddress, sender.address) + if ((winningGossip ne localGossip) && (winningGossip ne remoteGossip)) + log.debug( + """Couldn't establish a causal relationship between "remote" gossip and "local" gossip - Remote[{}] - Local[{}] - merged them into [{}]""", + remoteGossip, localGossip, winningGossip) - failureDetector heartbeat sender.address // update heartbeat in failure detector + _receivedGossipCount.incrementAndGet() + notifyMembershipChangeListeners(localState, newState) - if (convergence(newState.latestGossip).isDefined) { - newState.memberMembershipChangeListeners foreach { _ notify newState.latestGossip.members } + if ((winningGossip ne remoteGossip) || (newState.latestGossip ne remoteGossip)) { + // send back gossip to sender when sender had different view, i.e. merge, or sender had + // older or sender had newer + gossipTo(from) + } } } } /** - * Joins the pre-configured contact point and retrieves current gossip state. + * INTERNAL API. */ - private def autoJoin() = nodeToJoin foreach { address ⇒ - val connection = clusterCommandConnectionFor(address) - val command = ClusterAction.Join(remoteAddress) - log.info("Cluster Node [{}] - Sending [{}] to [{}] through connection [{}]", remoteAddress, command, address, connection) - connection ! command - } + private[cluster] def receiveHeartbeat(from: Address): Unit = failureDetector heartbeat from /** - * Switches the member status. + * Joins the pre-configured contact points. + */ + private def joinSeedNode(): Unit = clusterCommandDaemon ! ClusterUserAction.JoinSeedNode + + /** + * INTERNAL API. * - * @param newStatus the new member status - * @param oldState the state to change the member status in - * @return the updated new state with the new member status - */ - private def switchMemberStatusTo(newStatus: MemberStatus, state: State): State = { - log.info("Cluster Node [{}] - Switching membership status to [{}]", remoteAddress, newStatus) - - val localSelf = self - - val localGossip = state.latestGossip - val localMembers = localGossip.members - - // change my state into a "new" self - val newSelf = localSelf copy (status = newStatus) - - // change my state in 'gossip.members' - val newMembersSet = localMembers map { member ⇒ - if (member.address == remoteAddress) newSelf - else member - } - - // ugly crap to work around bug in scala colletions ('val ss: SortedSet[Member] = SortedSet.empty[Member] ++ aSet' does not compile) - val newMembersSortedSet = SortedSet[Member](newMembersSet.toList: _*) - val newGossip = localGossip copy (members = newMembersSortedSet) - - // version my changes - val versionedGossip = newGossip + vclockNode - val seenVersionedGossip = versionedGossip seen remoteAddress - - state copy (latestGossip = seenVersionedGossip) - } - - /** * Gossips latest gossip to an address. */ - private def gossipTo(address: Address) { + private[cluster] def gossipTo(address: Address): Unit = { val connection = clusterGossipConnectionFor(address) - log.debug("Cluster Node [{}] - Gossiping to [{}]", remoteAddress, connection) - connection ! GossipEnvelope(self, latestGossip) + log.debug("Cluster Node [{}] - Gossiping to [{}]", selfAddress, connection) + connection ! GossipEnvelope(selfAddress, latestGossip) } /** * Gossips latest gossip to a random member in the set of members passed in as argument. * - * @return 'true' if it gossiped to a "deputy" member. + * @return the used [[akka.actor.Address] if any */ - private def gossipToRandomNodeOf(addresses: Iterable[Address]): Boolean = { - if (addresses.isEmpty) false - else { - val peers = addresses filter (_ != remoteAddress) // filter out myself - val peer = selectRandomNode(peers) - gossipTo(peer) - deputyNodes exists (peer == _) + private def gossipToRandomNodeOf(addresses: IndexedSeq[Address]): Option[Address] = { + log.debug("Cluster Node [{}] - Selecting random node to gossip to [{}]", selfAddress, addresses.mkString(", ")) + val peers = addresses filterNot (_ == selfAddress) // filter out myself + val peer = selectRandomNode(peers) + peer foreach gossipTo + peer + } + + /** + * INTERNAL API. + */ + private[cluster] def gossipToDeputyProbablity(membersSize: Int, unreachableSize: Int, nrOfDeputyNodes: Int): Double = { + if (nrOfDeputyNodes > membersSize) 1.0 + else if (nrOfDeputyNodes == 0) 0.0 + else (membersSize + unreachableSize) match { + case 0 ⇒ 0.0 + case sum ⇒ (nrOfDeputyNodes + unreachableSize).toDouble / sum } } /** + * INTERNAL API. + * * Initates a new round of gossip. */ - private def gossip() { + private[cluster] def gossip(): Unit = { val localState = state.get - val localGossip = localState.latestGossip - val localMembers = localGossip.members + + log.debug("Cluster Node [{}] - Initiating new round of gossip", selfAddress) if (!isSingletonCluster(localState) && isAvailable(localState)) { - // only gossip if we are a non-singleton cluster and available - - log.debug("Cluster Node [{}] - Initiating new round of gossip", remoteAddress) - val localGossip = localState.latestGossip - val localMembers = localGossip.members + // important to not accidentally use `map` of the SortedSet, since the original order is not preserved + val localMembers = localGossip.members.toIndexedSeq val localMembersSize = localMembers.size + val localMemberAddresses = localMembers map { _.address } - val localUnreachableMembers = localGossip.overview.unreachable + val localUnreachableMembers = localGossip.overview.unreachable.toIndexedSeq val localUnreachableSize = localUnreachableMembers.size - // 1. gossip to alive members - val gossipedToDeputy = gossipToRandomNodeOf(localMembers map { _.address }) - - // 2. gossip to unreachable members - if (localUnreachableSize > 0) { - val probability: Double = localUnreachableSize / (localMembersSize + 1) - if (random.nextDouble() < probability) gossipToRandomNodeOf(localUnreachableMembers.map(_.address)) + // 1. gossip to a random alive member with preference to a member + // with older or newer gossip version + val nodesWithdifferentView = { + val localMemberAddressesSet = localGossip.members map { _.address } + for { + (address, version) ← localGossip.overview.seen + if localMemberAddressesSet contains address + if version != localGossip.version + } yield address } + val gossipedToAlive = + if (nodesWithdifferentView.nonEmpty && ThreadLocalRandom.current.nextDouble() < GossipDifferentViewProbability) + gossipToRandomNodeOf(nodesWithdifferentView.toIndexedSeq) + else + gossipToRandomNodeOf(localMemberAddresses) - // 3. gossip to a deputy nodes for facilitating partition healing - val deputies = deputyNodes - if ((!gossipedToDeputy || localMembersSize < 1) && !deputies.isEmpty) { - if (localMembersSize == 0) gossipToRandomNodeOf(deputies) - else { - val probability = 1.0 / localMembersSize + localUnreachableSize - if (random.nextDouble() <= probability) gossipToRandomNodeOf(deputies) - } + // 2. gossip to a deputy nodes for facilitating partition healing + val deputies = deputyNodes(localMemberAddresses) + val alreadyGossipedToDeputy = gossipedToAlive.map(deputies.contains(_)).getOrElse(false) + if ((!alreadyGossipedToDeputy || localMembersSize < seedNodes.size) && deputies.nonEmpty) { + val probability = gossipToDeputyProbablity(localMembersSize, localUnreachableSize, seedNodes.size) + if (ThreadLocalRandom.current.nextDouble() < probability) + gossipToRandomNodeOf(deputies) } } } /** + * INTERNAL API. + */ + private[cluster] def heartbeat(): Unit = { + removeOverdueJoinInProgress() + val localState = state.get + + val beatTo = localState.latestGossip.members.toSeq.map(_.address) ++ localState.joinInProgress.keys + + for (address ← beatTo; if address != selfAddress) { + val connection = clusterGossipConnectionFor(address) + log.debug("Cluster Node [{}] - Heartbeat to [{}]", selfAddress, connection) + connection ! selfHeartbeat + } + } + + /** + * INTERNAL API. + * * Reaps the unreachable members (moves them to the 'unreachable' list in the cluster overview) according to the failure detector's verdict. */ @tailrec - final private def reapUnreachableMembers() { + final private[cluster] def reapUnreachableMembers(): Unit = { val localState = state.get if (!isSingletonCluster(localState) && isAvailable(localState)) { @@ -857,126 +1158,230 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ val localGossip = localState.latestGossip val localOverview = localGossip.overview - val localSeen = localOverview.seen val localMembers = localGossip.members val localUnreachableMembers = localGossip.overview.unreachable val newlyDetectedUnreachableMembers = localMembers filterNot { member ⇒ failureDetector.isAvailable(member.address) } - if (!newlyDetectedUnreachableMembers.isEmpty) { // we have newly detected members marked as unavailable + if (newlyDetectedUnreachableMembers.nonEmpty) { // we have newly detected members marked as unavailable - val newMembers = localMembers diff newlyDetectedUnreachableMembers - val newUnreachableMembers: Set[Member] = localUnreachableMembers ++ newlyDetectedUnreachableMembers + val newMembers = localMembers -- newlyDetectedUnreachableMembers + val newUnreachableMembers = localUnreachableMembers ++ newlyDetectedUnreachableMembers val newOverview = localOverview copy (unreachable = newUnreachableMembers) val newGossip = localGossip copy (overview = newOverview, members = newMembers) // updating vclock and 'seen' table - val versionedGossip = newGossip + vclockNode - val seenVersionedGossip = versionedGossip seen remoteAddress + val versionedGossip = newGossip :+ vclockNode + val seenVersionedGossip = versionedGossip seen selfAddress val newState = localState copy (latestGossip = seenVersionedGossip) // if we won the race then update else try again if (!state.compareAndSet(localState, newState)) reapUnreachableMembers() // recur else { - log.info("Cluster Node [{}] - Marking node(s) as UNREACHABLE [{}]", remoteAddress, newlyDetectedUnreachableMembers.mkString(", ")) + log.info("Cluster Node [{}] - Marking node(s) as UNREACHABLE [{}]", selfAddress, newlyDetectedUnreachableMembers.mkString(", ")) - if (convergence(newState.latestGossip).isDefined) { - newState.memberMembershipChangeListeners foreach { _ notify newMembers } - } + notifyMembershipChangeListeners(localState, newState) } } } } /** + * INTERNAL API. + * + * Removes overdue joinInProgress from State. + */ + @tailrec + final private[cluster] def removeOverdueJoinInProgress(): Unit = { + val localState = state.get + val overdueJoins = localState.joinInProgress collect { + case (address, deadline) if deadline.isOverdue ⇒ address + } + if (overdueJoins.nonEmpty) { + val newState = localState copy (joinInProgress = localState.joinInProgress -- overdueJoins) + if (!state.compareAndSet(localState, newState)) removeOverdueJoinInProgress() // recur + } + } + + /** + * INTERNAL API. + * * Runs periodic leader actions, such as auto-downing unreachable nodes, assigning partitions etc. */ @tailrec - final private def leaderActions() { + final private[cluster] def leaderActions(): Unit = { val localState = state.get val localGossip = localState.latestGossip val localMembers = localGossip.members - val isLeader = !localMembers.isEmpty && (remoteAddress == localMembers.head.address) + val isLeader = localMembers.nonEmpty && (selfAddress == localMembers.head.address) if (isLeader && isAvailable(localState)) { // only run the leader actions if we are the LEADER and available val localOverview = localGossip.overview val localSeen = localOverview.seen - val localUnreachableMembers = localGossip.overview.unreachable + val localUnreachableMembers = localOverview.unreachable + val hasPartionHandoffCompletedSuccessfully: Boolean = { + // FIXME implement partion handoff and a check if it is completed - now just returns TRUE - e.g. has completed successfully + true + } // Leader actions are as follows: - // 1. Move JOINING => UP - // 2. Move EXITING => REMOVED - // 3. Move UNREACHABLE => DOWN (auto-downing by leader) - // 4. Updating the vclock version for the changes - // 5. Updating the 'seen' table + // 1. Move EXITING => REMOVED -- When all nodes have seen that the node is EXITING (convergence) - remove the nodes from the node ring and seen table + // 2. Move JOINING => UP -- When a node joins the cluster + // 3. Move LEAVING => EXITING -- When all partition handoff has completed + // 4. Move UNREACHABLE => DOWN -- When the node is in the UNREACHABLE set it can be auto-down by leader + // 5. Store away all stuff needed for the side-effecting processing in 10. + // 6. Updating the vclock version for the changes + // 7. Updating the 'seen' table + // 8. Try to update the state with the new gossip + // 9. If failure - retry + // 10. If success - run all the side-effecting processing - var hasChangedState = false - val newGossip = + val ( + newGossip: Gossip, + hasChangedState: Boolean, + upMembers, + exitingMembers, + removedMembers, + unreachableButNotDownedMembers) = if (convergence(localGossip).isDefined) { // we have convergence - so we can't have unreachable nodes + // transform the node member ring - filterNot/map/map val newMembers = - localMembers map { member ⇒ - // 1. Move JOINING => UP - if (member.status == MemberStatus.Joining) { - log.info("Cluster Node [{}] - Leader is moving node [{}] from JOINING to UP", remoteAddress, member.address) - hasChangedState = true - member copy (status = MemberStatus.Up) - } else member + localMembers filterNot { member ⇒ + // ---------------------- + // 1. Move EXITING => REMOVED - e.g. remove the nodes from the 'members' set/node ring and seen table + // ---------------------- + member.status == MemberStatus.Exiting + } map { member ⇒ - // 2. Move EXITING => REMOVED - if (member.status == MemberStatus.Exiting) { - log.info("Cluster Node [{}] - Leader is moving node [{}] from EXITING to REMOVED", remoteAddress, member.address) - hasChangedState = true - member copy (status = MemberStatus.Removed) - } else member + // ---------------------- + // 2. Move JOINING => UP (once all nodes have seen that this node is JOINING e.g. we have a convergence) + // ---------------------- + if (member.status == Joining) member copy (status = Up) + else member + + } map { member ⇒ + // ---------------------- + // 3. Move LEAVING => EXITING (once we have a convergence on LEAVING *and* if we have a successful partition handoff) + // ---------------------- + if (member.status == Leaving && hasPartionHandoffCompletedSuccessfully) member copy (status = Exiting) + else member } - localGossip copy (members = newMembers) // update gossip - } else if (autoDown) { - // we don't have convergence - so we might have unreachable nodes - // if 'auto-down' is turned on, then try to auto-down any unreachable nodes + // ---------------------- + // 5. Store away all stuff needed for the side-effecting processing in 10. + // ---------------------- - // 3. Move UNREACHABLE => DOWN (auto-downing by leader) - val newUnreachableMembers = - localUnreachableMembers - .filter(_.status != MemberStatus.Down) // no need to DOWN members already DOWN - .map { member ⇒ - log.info("Cluster Node [{}] - Leader is marking unreachable node [{}] as DOWN", remoteAddress, member.address) - hasChangedState = true - member copy (status = MemberStatus.Down) - } + // Check for the need to do side-effecting on successful state change + // Repeat the checking for transitions between JOINING -> UP, LEAVING -> EXITING, EXITING -> REMOVED + // to check for state-changes and to store away removed and exiting members for later notification + // 1. check for state-changes to update + // 2. store away removed and exiting members so we can separate the pure state changes (that can be retried on collision) and the side-effecting message sending + val (removedMembers, newMembers1) = localMembers partition (_.status == Exiting) - // removing nodes marked as DOWN from the 'seen' table - val newSeen = localUnreachableMembers.foldLeft(localSeen)((currentSeen, member) ⇒ currentSeen - member.address) + val (upMembers, newMembers2) = newMembers1 partition (_.status == Joining) + + val (exitingMembers, newMembers3) = newMembers2 partition (_.status == Leaving && hasPartionHandoffCompletedSuccessfully) + + val hasChangedState = removedMembers.nonEmpty || upMembers.nonEmpty || exitingMembers.nonEmpty + + // removing REMOVED nodes from the 'seen' table + val newSeen = localSeen -- removedMembers.map(_.address) + + // removing REMOVED nodes from the 'unreachable' set + val newUnreachableMembers = localUnreachableMembers -- removedMembers val newOverview = localOverview copy (seen = newSeen, unreachable = newUnreachableMembers) // update gossip overview - localGossip copy (overview = newOverview) // update gossip + val newGossip = localGossip copy (members = newMembers, overview = newOverview) // update gossip - } else localGossip + (newGossip, hasChangedState, upMembers, exitingMembers, removedMembers, Set.empty[Member]) + + } else if (AutoDown) { + // we don't have convergence - so we might have unreachable nodes + + // if 'auto-down' is turned on, then try to auto-down any unreachable nodes + val newUnreachableMembers = localUnreachableMembers.map { member ⇒ + // ---------------------- + // 5. Move UNREACHABLE => DOWN (auto-downing by leader) + // ---------------------- + if (member.status == Down) member // no need to DOWN members already DOWN + else member copy (status = Down) + } + + // Check for the need to do side-effecting on successful state change + val (unreachableButNotDownedMembers, _) = localUnreachableMembers partition (_.status != Down) + + // removing nodes marked as DOWN from the 'seen' table + val newSeen = localSeen -- newUnreachableMembers.collect { case m if m.status == Down ⇒ m.address } + + val newOverview = localOverview copy (seen = newSeen, unreachable = newUnreachableMembers) // update gossip overview + val newGossip = localGossip copy (overview = newOverview) // update gossip + + (newGossip, unreachableButNotDownedMembers.nonEmpty, Set.empty[Member], Set.empty[Member], Set.empty[Member], unreachableButNotDownedMembers) + + } else (localGossip, false, Set.empty[Member], Set.empty[Member], Set.empty[Member], Set.empty[Member]) if (hasChangedState) { // we have a change of state - version it and try to update + // ---------------------- + // 6. Updating the vclock version for the changes + // ---------------------- + val versionedGossip = newGossip :+ vclockNode - // 4. Updating the vclock version for the changes - val versionedGossip = newGossip + vclockNode - - // 5. Updating the 'seen' table - val seenVersionedGossip = versionedGossip seen remoteAddress + // ---------------------- + // 7. Updating the 'seen' table + // Unless the leader (this node) is part of the removed members, i.e. the leader have moved himself from EXITING -> REMOVED + // ---------------------- + val seenVersionedGossip = + if (removedMembers.exists(_.address == selfAddress)) versionedGossip + else versionedGossip seen selfAddress val newState = localState copy (latestGossip = seenVersionedGossip) - // if we won the race then update else try again - if (!state.compareAndSet(localState, newState)) leaderActions() // recur - else { - if (convergence(newState.latestGossip).isDefined) { - newState.memberMembershipChangeListeners map { _ notify newGossip.members } + // ---------------------- + // 8. Try to update the state with the new gossip + // ---------------------- + if (!state.compareAndSet(localState, newState)) { + + // ---------------------- + // 9. Failure - retry + // ---------------------- + leaderActions() // recur + + } else { + // ---------------------- + // 10. Success - run all the side-effecting processing + // ---------------------- + + // log the move of members from joining to up + upMembers foreach { member ⇒ log.info("Cluster Node [{}] - Leader is moving node [{}] from JOINING to UP", selfAddress, member.address) } + + // tell all removed members to remove and shut down themselves + removedMembers foreach { member ⇒ + val address = member.address + log.info("Cluster Node [{}] - Leader is moving node [{}] from EXITING to REMOVED - and removing node from node ring", selfAddress, address) + clusterCommandConnectionFor(address) ! ClusterLeaderAction.Remove(address) } + + // tell all exiting members to exit + exitingMembers foreach { member ⇒ + val address = member.address + log.info("Cluster Node [{}] - Leader is moving node [{}] from LEAVING to EXITING", selfAddress, address) + clusterCommandConnectionFor(address) ! ClusterLeaderAction.Exit(address) // FIXME should use ? to await completion of handoff? + } + + // log the auto-downing of the unreachable nodes + unreachableButNotDownedMembers foreach { member ⇒ + log.info("Cluster Node [{}] - Leader is marking unreachable node [{}] as DOWN", selfAddress, member.address) + } + + notifyMembershipChangeListeners(localState, newState) } } } @@ -991,39 +1396,59 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ private def convergence(gossip: Gossip): Option[Gossip] = { val overview = gossip.overview val unreachable = overview.unreachable + val seen = overview.seen // First check that: - // 1. we don't have any members that are unreachable (unreachable.isEmpty == true), or + // 1. we don't have any members that are unreachable, or // 2. all unreachable members in the set have status DOWN // Else we can't continue to check for convergence // When that is done we check that all the entries in the 'seen' table have the same vector clock version - if (unreachable.isEmpty || !unreachable.exists(m ⇒ (m.status != MemberStatus.Down) && (m.status != MemberStatus.Removed))) { - val seen = gossip.overview.seen - val views = Set.empty[VectorClock] ++ seen.values + // and that all members exists in seen table + val hasUnreachable = unreachable.nonEmpty && unreachable.exists { _.status != Down } + val allMembersInSeen = gossip.members.forall(m ⇒ seen.contains(m.address)) - if (views.size == 1) { - log.debug("Cluster Node [{}] - Cluster convergence reached", remoteAddress) + if (hasUnreachable) { + log.debug("Cluster Node [{}] - No cluster convergence, due to unreachable nodes [{}].", selfAddress, unreachable) + None + } else if (!allMembersInSeen) { + log.debug("Cluster Node [{}] - No cluster convergence, due to members not in seen table [{}].", selfAddress, + gossip.members.map(_.address) -- seen.keySet) + None + } else { + + val views = seen.values.toSet.size + + if (views == 1) { + log.debug("Cluster Node [{}] - Cluster convergence reached: [{}]", selfAddress, gossip.members.mkString(", ")) Some(gossip) - } else None - } else None + } else { + log.debug("Cluster Node [{}] - No cluster convergence, since not all nodes have seen the same state yet. [{} of {}]", + selfAddress, views, seen.values.size) + None + } + } } private def isAvailable(state: State): Boolean = !isUnavailable(state) private def isUnavailable(state: State): Boolean = { val localGossip = state.latestGossip - val localOverview = localGossip.overview - val localMembers = localGossip.members - val localUnreachableMembers = localOverview.unreachable - val isUnreachable = localUnreachableMembers exists { _.address == remoteAddress } - val hasUnavailableMemberStatus = localMembers exists { m ⇒ (m == self) && MemberStatus.isUnavailable(m.status) } + val isUnreachable = localGossip.overview.unreachable exists { _.address == selfAddress } + val hasUnavailableMemberStatus = localGossip.members exists { m ⇒ (m == self) && m.status.isUnavailable } isUnreachable || hasUnavailableMemberStatus } + private def notifyMembershipChangeListeners(oldState: State, newState: State): Unit = { + val oldMembersStatus = oldState.latestGossip.members.map(m ⇒ (m.address, m.status)) + val newMembersStatus = newState.latestGossip.members.map(m ⇒ (m.address, m.status)) + if (newMembersStatus != oldMembersStatus) + newState.memberMembershipChangeListeners foreach { _ notify newState.latestGossip.members } + } + /** * Looks up and returns the local cluster command connection. */ - private def clusterCommandDaemon = system.actorFor(RootActorPath(remoteAddress) / "system" / "cluster" / "commands") + private def clusterCommandDaemon = system.actorFor(RootActorPath(selfAddress) / "system" / "cluster" / "commands") /** * Looks up and returns the remote cluster command connection for the specific address. @@ -1036,11 +1461,17 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ private def clusterGossipConnectionFor(address: Address): ActorRef = system.actorFor(RootActorPath(address) / "system" / "cluster" / "gossip") /** - * Gets an Iterable with the addresses of a all the 'deputy' nodes - excluding this node if part of the group. + * Gets the addresses of a all the 'deputy' nodes - excluding this node if part of the group. */ - private def deputyNodes: Iterable[Address] = state.get.latestGossip.members.toIterable map (_.address) drop 1 take nrOfDeputyNodes filter (_ != remoteAddress) + private def deputyNodes(addresses: IndexedSeq[Address]): IndexedSeq[Address] = + addresses filterNot (_ == selfAddress) intersect seedNodes - private def selectRandomNode(addresses: Iterable[Address]): Address = addresses.toSeq(random nextInt addresses.size) + /** + * INTERNAL API. + */ + private[cluster] def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] = + if (addresses.isEmpty) None + else Some(addresses(ThreadLocalRandom.current nextInt addresses.size)) private def isSingletonCluster(currentState: State): Boolean = currentState.latestGossip.members.size == 1 @@ -1067,8 +1498,8 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ val unreachable = gossip.overview.unreachable val metaData = gossip.meta "\nMembers:\n\t" + gossip.members.mkString("\n\t") + - { if (!unreachable.isEmpty) "\nUnreachable:\n\t" + unreachable.mkString("\n\t") else "" } + - { if (!metaData.isEmpty) "\nMeta Data:\t" + metaData.toString else "" } + { if (unreachable.nonEmpty) "\nUnreachable:\n\t" + unreachable.mkString("\n\t") else "" } + + { if (metaData.nonEmpty) "\nMeta Data:\t" + metaData.toString else "" } } def getMemberStatus: String = clusterNode.status.toString @@ -1081,6 +1512,8 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ def isAvailable: Boolean = clusterNode.isAvailable + def isRunning: Boolean = clusterNode.isRunning + // JMX commands def join(address: String) = clusterNode.join(AddressFromURIString(address)) @@ -1088,12 +1521,8 @@ class Cluster(system: ExtendedActorSystem) extends Extension { clusterNode ⇒ def leave(address: String) = clusterNode.leave(AddressFromURIString(address)) def down(address: String) = clusterNode.down(AddressFromURIString(address)) - - def remove(address: String) = clusterNode.remove(AddressFromURIString(address)) - - def shutdown() = clusterNode.shutdown() } - log.info("Cluster Node [{}] - registering cluster JMX MBean [{}]", remoteAddress, clusterMBeanName) + log.info("Cluster Node [{}] - registering cluster JMX MBean [{}]", selfAddress, clusterMBeanName) try { mBeanServer.registerMBean(mbean, clusterMBeanName) } catch { diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala index a24c75b436..6e4cbc4e60 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala @@ -6,24 +6,37 @@ package akka.cluster import com.typesafe.config.Config import akka.util.Duration import java.util.concurrent.TimeUnit.MILLISECONDS -import akka.config.ConfigurationException +import akka.ConfigurationException import scala.collection.JavaConverters._ import akka.actor.Address import akka.actor.AddressFromURIString class ClusterSettings(val config: Config, val systemName: String) { import config._ - val FailureDetectorThreshold = getInt("akka.cluster.failure-detector.threshold") - val FailureDetectorMaxSampleSize = getInt("akka.cluster.failure-detector.max-sample-size") - val NodeToJoin: Option[Address] = getString("akka.cluster.node-to-join") match { - case "" ⇒ None - case AddressFromURIString(addr) ⇒ Some(addr) - } - val PeriodicTasksInitialDelay = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS) - val GossipFrequency = Duration(getMilliseconds("akka.cluster.gossip-frequency"), MILLISECONDS) - val LeaderActionsFrequency = Duration(getMilliseconds("akka.cluster.leader-actions-frequency"), MILLISECONDS) - val UnreachableNodesReaperFrequency = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-frequency"), MILLISECONDS) - val NrOfGossipDaemons = getInt("akka.cluster.nr-of-gossip-daemons") - val NrOfDeputyNodes = getInt("akka.cluster.nr-of-deputy-nodes") - val AutoDown = getBoolean("akka.cluster.auto-down") + + final val FailureDetectorThreshold = getDouble("akka.cluster.failure-detector.threshold") + final val FailureDetectorMaxSampleSize = getInt("akka.cluster.failure-detector.max-sample-size") + final val FailureDetectorImplementationClass = getString("akka.cluster.failure-detector.implementation-class") + final val FailureDetectorMinStdDeviation: Duration = + Duration(getMilliseconds("akka.cluster.failure-detector.min-std-deviation"), MILLISECONDS) + final val FailureDetectorAcceptableHeartbeatPause: Duration = + Duration(getMilliseconds("akka.cluster.failure-detector.acceptable-heartbeat-pause"), MILLISECONDS) + + final val SeedNodes: IndexedSeq[Address] = getStringList("akka.cluster.seed-nodes").asScala.map { + case AddressFromURIString(addr) ⇒ addr + }.toIndexedSeq + final val SeedNodeTimeout: Duration = Duration(getMilliseconds("akka.cluster.seed-node-timeout"), MILLISECONDS) + final val PeriodicTasksInitialDelay: Duration = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS) + final val GossipInterval: Duration = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS) + final val HeartbeatInterval: Duration = Duration(getMilliseconds("akka.cluster.heartbeat-interval"), MILLISECONDS) + final val LeaderActionsInterval: Duration = Duration(getMilliseconds("akka.cluster.leader-actions-interval"), MILLISECONDS) + final val UnreachableNodesReaperInterval: Duration = Duration(getMilliseconds("akka.cluster.unreachable-nodes-reaper-interval"), MILLISECONDS) + final val NrOfGossipDaemons: Int = getInt("akka.cluster.nr-of-gossip-daemons") + final val NrOfDeputyNodes: Int = getInt("akka.cluster.nr-of-deputy-nodes") + final val AutoJoin: Boolean = getBoolean("akka.cluster.auto-join") + final val AutoDown: Boolean = getBoolean("akka.cluster.auto-down") + final val JoinTimeout: Duration = Duration(getMilliseconds("akka.cluster.join-timeout"), MILLISECONDS) + final val GossipDifferentViewProbability: Double = getDouble("akka.cluster.gossip-different-view-probability") + final val SchedulerTickDuration: Duration = Duration(getMilliseconds("akka.cluster.scheduler.tick-duration"), MILLISECONDS) + final val SchedulerTicksPerWheel: Int = getInt("akka.cluster.scheduler.ticks-per-wheel") } diff --git a/akka-cluster/src/main/scala/akka/cluster/FailureDetector.scala b/akka-cluster/src/main/scala/akka/cluster/FailureDetector.scala new file mode 100644 index 0000000000..1aa926c5e5 --- /dev/null +++ b/akka-cluster/src/main/scala/akka/cluster/FailureDetector.scala @@ -0,0 +1,33 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import akka.actor.Address + +/** + * Interface for Akka failure detectors. + */ +trait FailureDetector { + + /** + * Returns true if the connection is considered to be up and healthy and returns false otherwise. + */ + def isAvailable(connection: Address): Boolean + + /** + * Records a heartbeat for a connection. + */ + def heartbeat(connection: Address): Unit + + /** + * Removes the heartbeat management for a connection. + */ + def remove(connection: Address): Unit + + /** + * Removes all connections and starts over. + */ + def reset(): Unit +} diff --git a/akka-cluster/src/main/scala/akka/cluster/FixedRateTask.scala b/akka-cluster/src/main/scala/akka/cluster/FixedRateTask.scala new file mode 100644 index 0000000000..25ef058465 --- /dev/null +++ b/akka-cluster/src/main/scala/akka/cluster/FixedRateTask.scala @@ -0,0 +1,51 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.AtomicLong + +import akka.actor.Scheduler +import akka.util.Duration + +/** + * INTERNAL API + */ +private[akka] object FixedRateTask { + def apply(scheduler: Scheduler, initalDelay: Duration, delay: Duration)(f: ⇒ Unit): FixedRateTask = { + new FixedRateTask(scheduler, initalDelay, delay, new Runnable { def run(): Unit = f }) + } +} + +/** + * INTERNAL API + * + * Task to be scheduled periodically at a fixed rate, compensating, on average, + * for inaccuracy in scheduler. It will start when constructed, using the + * initialDelay. + */ +private[akka] class FixedRateTask(scheduler: Scheduler, initalDelay: Duration, delay: Duration, task: Runnable) extends Runnable { + + private val delayNanos = delay.toNanos + private val cancelled = new AtomicBoolean(false) + private val counter = new AtomicLong(0L) + private val startTime = System.nanoTime + initalDelay.toNanos + scheduler.scheduleOnce(initalDelay, this) + + def cancel(): Unit = cancelled.set(true) + + override final def run(): Unit = if (!cancelled.get) try { + task.run() + } finally if (!cancelled.get) { + val nextTime = startTime + delayNanos * counter.incrementAndGet + // it's ok to schedule with negative duration, will run asap + val nextDelay = Duration(nextTime - System.nanoTime, TimeUnit.NANOSECONDS) + try { + scheduler.scheduleOnce(nextDelay, this) + } catch { case e: IllegalStateException ⇒ /* will happen when scheduler is closed, nothing wrong */ } + } + +} diff --git a/akka-cluster/src/main/scala/akka/cluster/VectorClock.scala b/akka-cluster/src/main/scala/akka/cluster/VectorClock.scala index 82c1b9881d..ed6724058f 100644 --- a/akka-cluster/src/main/scala/akka/cluster/VectorClock.scala +++ b/akka-cluster/src/main/scala/akka/cluster/VectorClock.scala @@ -19,7 +19,7 @@ class VectorClockException(message: String) extends AkkaException(message) */ trait Versioned[T] { def version: VectorClock - def +(node: VectorClock.Node): T + def :+(node: VectorClock.Node): T } /** @@ -142,7 +142,7 @@ case class VectorClock( /** * Increment the version for the node passed as argument. Returns a new VectorClock. */ - def +(node: Node): VectorClock = copy(versions = versions + (node -> Timestamp())) + def :+(node: Node): VectorClock = copy(versions = versions + (node -> Timestamp())) /** * Returns true if this and that are concurrent else false. diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUnreachableSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUnreachableSpec.scala new file mode 100644 index 0000000000..8112aeab25 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUnreachableSpec.scala @@ -0,0 +1,69 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.actor.Address + +object ClientDowningNodeThatIsUnreachableMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + + commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode1 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy +class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode2 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy +class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode3 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy +class ClientDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode4 extends ClientDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy + +class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode1 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy +class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode2 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy +class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode3 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy +class ClientDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode4 extends ClientDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy + +abstract class ClientDowningNodeThatIsUnreachableSpec + extends MultiNodeSpec(ClientDowningNodeThatIsUnreachableMultiJvmSpec) + with MultiNodeClusterSpec { + + import ClientDowningNodeThatIsUnreachableMultiJvmSpec._ + + "Client of a 4 node cluster" must { + + "be able to DOWN a node that is UNREACHABLE (killed)" taggedAs LongRunningTest in { + val thirdAddress = address(third) + awaitClusterUp(first, second, third, fourth) + + runOn(first) { + // kill 'third' node + testConductor.shutdown(third, 0) + markNodeAsUnavailable(thirdAddress) + + // mark 'third' node as DOWN + cluster.down(thirdAddress) + enterBarrier("down-third-node") + + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) + cluster.latestGossip.members.exists(_.address == thirdAddress) must be(false) + } + + runOn(third) { + enterBarrier("down-third-node") + } + + runOn(second, fourth) { + enterBarrier("down-third-node") + + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) + } + + enterBarrier("await-completion") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUpSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUpSpec.scala new file mode 100644 index 0000000000..4c65e85054 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUpSpec.scala @@ -0,0 +1,67 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.actor.Address + +object ClientDowningNodeThatIsUpMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + + commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy +class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy +class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy +class ClientDowningNodeThatIsUpWithFailureDetectorPuppetMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec with FailureDetectorPuppetStrategy + +class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode1 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy +class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode2 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy +class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode3 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy +class ClientDowningNodeThatIsUpWithAccrualFailureDetectorMultiJvmNode4 extends ClientDowningNodeThatIsUpSpec with AccrualFailureDetectorStrategy + +abstract class ClientDowningNodeThatIsUpSpec + extends MultiNodeSpec(ClientDowningNodeThatIsUpMultiJvmSpec) + with MultiNodeClusterSpec { + + import ClientDowningNodeThatIsUpMultiJvmSpec._ + + "Client of a 4 node cluster" must { + + "be able to DOWN a node that is UP (healthy and available)" taggedAs LongRunningTest in { + val thirdAddress = address(third) + awaitClusterUp(first, second, third, fourth) + + runOn(first) { + // mark 'third' node as DOWN + cluster.down(thirdAddress) + enterBarrier("down-third-node") + + markNodeAsUnavailable(thirdAddress) + + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) + cluster.latestGossip.members.exists(_.address == thirdAddress) must be(false) + } + + runOn(third) { + enterBarrier("down-third-node") + } + + runOn(second, fourth) { + enterBarrier("down-third-node") + + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) + } + + enterBarrier("await-completion") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterAccrualFailureDetectorSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterAccrualFailureDetectorSpec.scala new file mode 100644 index 0000000000..d5d41b52aa --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterAccrualFailureDetectorSpec.scala @@ -0,0 +1,63 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.util.duration._ +import akka.testkit._ + +object ClusterAccrualFailureDetectorMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig(debugConfig(on = false). + withFallback(ConfigFactory.parseString("akka.cluster.failure-detector.threshold = 4")). + withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class ClusterAccrualFailureDetectorMultiJvmNode1 extends ClusterAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy +class ClusterAccrualFailureDetectorMultiJvmNode2 extends ClusterAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy +class ClusterAccrualFailureDetectorMultiJvmNode3 extends ClusterAccrualFailureDetectorSpec with AccrualFailureDetectorStrategy + +abstract class ClusterAccrualFailureDetectorSpec + extends MultiNodeSpec(ClusterAccrualFailureDetectorMultiJvmSpec) + with MultiNodeClusterSpec { + + import ClusterAccrualFailureDetectorMultiJvmSpec._ + + "A heartbeat driven Failure Detector" must { + + "receive heartbeats so that all member nodes in the cluster are marked 'available'" taggedAs LongRunningTest in { + awaitClusterUp(first, second, third) + + 5.seconds.dilated.sleep // let them heartbeat + cluster.failureDetector.isAvailable(first) must be(true) + cluster.failureDetector.isAvailable(second) must be(true) + cluster.failureDetector.isAvailable(third) must be(true) + + enterBarrier("after-1") + } + + "mark node as 'unavailable' if a node in the cluster is shut down (and its heartbeats stops)" taggedAs LongRunningTest in { + runOn(first) { + testConductor.shutdown(third, 0) + } + + enterBarrier("third-shutdown") + + runOn(first, second) { + // remaning nodes should detect failure... + awaitCond(!cluster.failureDetector.isAvailable(third), 15.seconds) + // other connections still ok + cluster.failureDetector.isAvailable(first) must be(true) + cluster.failureDetector.isAvailable(second) must be(true) + } + + enterBarrier("after-2") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ConvergenceSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ConvergenceSpec.scala new file mode 100644 index 0000000000..6d92a6f094 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ConvergenceSpec.scala @@ -0,0 +1,116 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ +import akka.actor.Address + +object ConvergenceMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + + commonConfig(debugConfig(on = false). + withFallback(ConfigFactory.parseString("akka.cluster.failure-detector.threshold = 4")). + withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class ConvergenceWithFailureDetectorPuppetMultiJvmNode1 extends ConvergenceSpec with FailureDetectorPuppetStrategy +class ConvergenceWithFailureDetectorPuppetMultiJvmNode2 extends ConvergenceSpec with FailureDetectorPuppetStrategy +class ConvergenceWithFailureDetectorPuppetMultiJvmNode3 extends ConvergenceSpec with FailureDetectorPuppetStrategy +class ConvergenceWithFailureDetectorPuppetMultiJvmNode4 extends ConvergenceSpec with FailureDetectorPuppetStrategy + +class ConvergenceWithAccrualFailureDetectorMultiJvmNode1 extends ConvergenceSpec with AccrualFailureDetectorStrategy +class ConvergenceWithAccrualFailureDetectorMultiJvmNode2 extends ConvergenceSpec with AccrualFailureDetectorStrategy +class ConvergenceWithAccrualFailureDetectorMultiJvmNode3 extends ConvergenceSpec with AccrualFailureDetectorStrategy +class ConvergenceWithAccrualFailureDetectorMultiJvmNode4 extends ConvergenceSpec with AccrualFailureDetectorStrategy + +abstract class ConvergenceSpec + extends MultiNodeSpec(ConvergenceMultiJvmSpec) + with MultiNodeClusterSpec { + + import ConvergenceMultiJvmSpec._ + + "A cluster of 3 members" must { + + "reach initial convergence" taggedAs LongRunningTest in { + awaitClusterUp(first, second, third) + + runOn(fourth) { + // doesn't join immediately + } + + enterBarrier("after-1") + } + + "not reach convergence while any nodes are unreachable" taggedAs LongRunningTest in { + val thirdAddress = address(third) + enterBarrier("before-shutdown") + + runOn(first) { + // kill 'third' node + testConductor.shutdown(third, 0) + markNodeAsUnavailable(thirdAddress) + } + + runOn(first, second) { + + within(28 seconds) { + // third becomes unreachable + awaitCond(cluster.latestGossip.overview.unreachable.size == 1) + awaitCond(cluster.latestGossip.members.size == 2) + awaitCond(cluster.latestGossip.members.forall(_.status == MemberStatus.Up)) + awaitSeenSameState(first, second) + // still one unreachable + cluster.latestGossip.overview.unreachable.size must be(1) + cluster.latestGossip.overview.unreachable.head.address must be(thirdAddress) + // and therefore no convergence + cluster.convergence.isDefined must be(false) + + } + } + + enterBarrier("after-2") + } + + "not move a new joining node to Up while there is no convergence" taggedAs LongRunningTest in { + runOn(fourth) { + // try to join + cluster.join(first) + } + + def memberStatus(address: Address): Option[MemberStatus] = + cluster.latestGossip.members.collectFirst { case m if m.address == address ⇒ m.status } + + def assertNotMovedUp: Unit = { + within(20 seconds) { + awaitCond(cluster.latestGossip.members.size == 3) + awaitSeenSameState(first, second, fourth) + memberStatus(first) must be(Some(MemberStatus.Up)) + memberStatus(second) must be(Some(MemberStatus.Up)) + // leader is not allowed to move the new node to Up + memberStatus(fourth) must be(Some(MemberStatus.Joining)) + // still no convergence + cluster.convergence.isDefined must be(false) + } + } + + runOn(first, second, fourth) { + for (n ← 1 to 5) { + log.debug("assertNotMovedUp#" + n) + assertNotMovedUp + // wait and then check again + 1.second.dilated.sleep + } + } + + enterBarrier("after-3") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/FailureDetectorStrategy.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/FailureDetectorStrategy.scala new file mode 100644 index 0000000000..86e03f9457 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/FailureDetectorStrategy.scala @@ -0,0 +1,61 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import akka.actor.Address +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ + +/** + * Base trait for all failure detector strategies. + */ +trait FailureDetectorStrategy { + + /** + * Get or create the FailureDetector to be used in the cluster node. + * To be defined by subclass. + */ + def failureDetector: FailureDetector + + /** + * Marks a node as available in the failure detector. + * To be defined by subclass. + */ + def markNodeAsAvailable(address: Address): Unit + + /** + * Marks a node as unavailable in the failure detector. + * To be defined by subclass. + */ + def markNodeAsUnavailable(address: Address): Unit +} + +/** + * Defines a FailureDetectorPuppet-based FailureDetectorStrategy. + */ +trait FailureDetectorPuppetStrategy extends FailureDetectorStrategy { self: MultiNodeSpec ⇒ + + /** + * The puppet instance. Separated from 'failureDetector' field so we don't have to cast when using the puppet specific methods. + */ + private val puppet = new FailureDetectorPuppet(system) + + override def failureDetector: FailureDetector = puppet + + override def markNodeAsAvailable(address: Address): Unit = puppet markNodeAsAvailable address + + override def markNodeAsUnavailable(address: Address): Unit = puppet markNodeAsUnavailable address +} + +/** + * Defines a AccrualFailureDetector-based FailureDetectorStrategy. + */ +trait AccrualFailureDetectorStrategy extends FailureDetectorStrategy { self: MultiNodeSpec ⇒ + + override val failureDetector: FailureDetector = new AccrualFailureDetector(system, new ClusterSettings(system.settings.config, system.name)) + + override def markNodeAsAvailable(address: Address): Unit = () + + override def markNodeAsUnavailable(address: Address): Unit = () +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/GossipMembershipMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/GossipMembershipMultiJvmSpec.scala deleted file mode 100644 index c380d3e5eb..0000000000 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/GossipMembershipMultiJvmSpec.scala +++ /dev/null @@ -1,134 +0,0 @@ -// package akka.cluster - -// import akka.actor.Actor -// import akka.remote._ -// import akka.routing._ -// import akka.routing.Routing.Broadcast - -// object GossipMembershipMultiJvmSpec { -// val NrOfNodes = 4 -// class SomeActor extends Actor with Serializable { -// def receive = { -// case "hit" ⇒ sender ! system.nodename -// case "end" ⇒ self.stop() -// } -// } - -// import com.typesafe.config.ConfigFactory -// val commonConfig = ConfigFactory.parseString(""" -// akka { -// loglevel = "WARNING" -// cluster { -// seed-nodes = ["localhost:9991"] -// } -// remote.server.hostname = "localhost" -// }""") - -// val node1Config = ConfigFactory.parseString(""" -// akka { -// remote.server.port = "9991" -// cluster.nodename = "node1" -// }""") withFallback commonConfig - -// val node2Config = ConfigFactory.parseString(""" -// akka { -// remote.server.port = "9992" -// cluster.nodename = "node2" -// }""") withFallback commonConfig - -// val node3Config = ConfigFactory.parseString(""" -// akka { -// remote.server.port = "9993" -// cluster.nodename = "node3" -// }""") withFallback commonConfig - -// val node4Config = ConfigFactory.parseString(""" -// akka { -// remote.server.port = "9994" -// cluster.nodename = "node4" -// }""") withFallback commonConfig -// } - -// class GossipMembershipMultiJvmNode1 extends AkkaRemoteSpec(GossipMembershipMultiJvmSpec.node1Config) { -// import GossipMembershipMultiJvmSpec._ -// val nodes = NrOfNodes -// "A cluster" must { -// "allow new node to join and should reach convergence with new membership table" in { - -// barrier("setup") -// remote.start() - -// barrier("start") -// val actor = system.actorOf(Props[SomeActor]("service-hello") -// actor.isInstanceOf[RoutedActorRef] must be(true) - -// val connectionCount = NrOfNodes - 1 -// val iterationCount = 10 - -// var replies = Map( -// "node1" -> 0, -// "node2" -> 0, -// "node3" -> 0) - -// for (i ← 0 until iterationCount) { -// for (k ← 0 until connectionCount) { -// val nodeName = (actor ? "hit").as[String].getOrElse(fail("No id returned by actor")) -// replies = replies + (nodeName -> (replies(nodeName) + 1)) -// } -// } - -// barrier("broadcast-end") -// actor ! Broadcast("end") - -// barrier("end") -// replies.values foreach { _ must be > (0) } - -// barrier("done") -// } -// } -// } - -// class GossipMembershipMultiJvmNode2 extends AkkaRemoteSpec(GossipMembershipMultiJvmSpec.node2Config) { -// import GossipMembershipMultiJvmSpec._ -// val nodes = NrOfNodes -// "___" must { -// "___" in { -// barrier("setup") -// remote.start() -// barrier("start") -// barrier("broadcast-end") -// barrier("end") -// barrier("done") -// } -// } -// } - -// class GossipMembershipMultiJvmNode3 extends AkkaRemoteSpec(GossipMembershipMultiJvmSpec.node3Config) { -// import GossipMembershipMultiJvmSpec._ -// val nodes = NrOfNodes -// "___" must { -// "___" in { -// barrier("setup") -// remote.start() -// barrier("start") -// barrier("broadcast-end") -// barrier("end") -// barrier("done") -// } -// } -// } - -// class GossipMembershipMultiJvmNode4 extends AkkaRemoteSpec(GossipMembershipMultiJvmSpec.node4Config) { -// import GossipMembershipMultiJvmSpec._ -// val nodes = NrOfNodes -// "___" must { -// "___" in { -// barrier("setup") -// remote.start() -// barrier("start") -// barrier("broadcast-end") -// barrier("end") -// barrier("done") -// } -// } -// } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinInProgressSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinInProgressSpec.scala new file mode 100644 index 0000000000..256b7d563d --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinInProgressSpec.scala @@ -0,0 +1,65 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfter +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ +import akka.util.Deadline + +object JoinInProgressMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + commonConfig( + debugConfig(on = false) + .withFallback(ConfigFactory.parseString(""" + akka.cluster { + # simulate delay in gossip by turning it off + gossip-interval = 300 s + failure-detector { + threshold = 4 + acceptable-heartbeat-pause = 1 second + } + }""") // increase the leader action task interval + .withFallback(MultiNodeClusterSpec.clusterConfig))) +} + +class JoinInProgressMultiJvmNode1 extends JoinInProgressSpec with AccrualFailureDetectorStrategy +class JoinInProgressMultiJvmNode2 extends JoinInProgressSpec with AccrualFailureDetectorStrategy + +abstract class JoinInProgressSpec + extends MultiNodeSpec(JoinInProgressMultiJvmSpec) + with MultiNodeClusterSpec { + + import JoinInProgressMultiJvmSpec._ + + "A cluster node" must { + "send heartbeats immediately when joining to avoid false failure detection due to delayed gossip" taggedAs LongRunningTest in { + + runOn(first) { + startClusterNode() + } + + enterBarrier("first-started") + + runOn(second) { + cluster.join(first) + } + + runOn(first) { + val until = Deadline.now + 5.seconds + while (!until.isOverdue) { + 200.millis.sleep + cluster.failureDetector.isAvailable(second) must be(true) + } + } + + enterBarrier("after") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinSeedNodeSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinSeedNodeSpec.scala new file mode 100644 index 0000000000..20dec26a45 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinSeedNodeSpec.scala @@ -0,0 +1,65 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfter +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ + +object JoinSeedNodeMultiJvmSpec extends MultiNodeConfig { + val seed1 = role("seed1") + val seed2 = role("seed2") + val ordinary1 = role("ordinary1") + val ordinary2 = role("ordinary2") + + commonConfig(debugConfig(on = false). + withFallback(ConfigFactory.parseString("akka.cluster.auto-join = on")). + withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class JoinSeedNodeMultiJvmNode1 extends JoinSeedNodeSpec with FailureDetectorPuppetStrategy +class JoinSeedNodeMultiJvmNode2 extends JoinSeedNodeSpec with FailureDetectorPuppetStrategy +class JoinSeedNodeMultiJvmNode3 extends JoinSeedNodeSpec with FailureDetectorPuppetStrategy +class JoinSeedNodeMultiJvmNode4 extends JoinSeedNodeSpec with FailureDetectorPuppetStrategy + +abstract class JoinSeedNodeSpec + extends MultiNodeSpec(JoinSeedNodeMultiJvmSpec) + with MultiNodeClusterSpec { + + import JoinSeedNodeMultiJvmSpec._ + + override def seedNodes = IndexedSeq(seed1, seed2) + + "A cluster with configured seed nodes" must { + "start the seed nodes sequentially" taggedAs LongRunningTest in { + runOn(seed1) { + startClusterNode() + } + enterBarrier("seed1-started") + + runOn(seed2) { + startClusterNode() + } + enterBarrier("seed2-started") + + runOn(seed1, seed2) { + awaitUpConvergence(2) + } + enterBarrier("after-1") + } + + "join the seed nodes at startup" taggedAs LongRunningTest in { + + startClusterNode() + enterBarrier("all-started") + + awaitUpConvergence(4) + + enterBarrier("after-2") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinTwoClustersSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinTwoClustersSpec.scala new file mode 100644 index 0000000000..d34a48f48e --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/JoinTwoClustersSpec.scala @@ -0,0 +1,90 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ + +object JoinTwoClustersMultiJvmSpec extends MultiNodeConfig { + val a1 = role("a1") + val a2 = role("a2") + val b1 = role("b1") + val b2 = role("b2") + val c1 = role("c1") + val c2 = role("c2") + + commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class JoinTwoClustersMultiJvmNode1 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy +class JoinTwoClustersMultiJvmNode2 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy +class JoinTwoClustersMultiJvmNode3 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy +class JoinTwoClustersMultiJvmNode4 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy +class JoinTwoClustersMultiJvmNode5 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy +class JoinTwoClustersMultiJvmNode6 extends JoinTwoClustersSpec with FailureDetectorPuppetStrategy + +abstract class JoinTwoClustersSpec + extends MultiNodeSpec(JoinTwoClustersMultiJvmSpec) + with MultiNodeClusterSpec { + + import JoinTwoClustersMultiJvmSpec._ + + "Three different clusters (A, B and C)" must { + + "be able to 'elect' a single leader after joining (A -> B)" taggedAs LongRunningTest in { + // make sure that the node-to-join is started before other join + runOn(a1, b1, c1) { + startClusterNode() + } + enterBarrier("first-started") + + runOn(a1, a2) { + cluster.join(a1) + } + runOn(b1, b2) { + cluster.join(b1) + } + runOn(c1, c2) { + cluster.join(c1) + } + + awaitUpConvergence(numberOfMembers = 2) + + assertLeader(a1, a2) + assertLeader(b1, b2) + assertLeader(c1, c2) + + enterBarrier("two-members") + + runOn(b2) { + cluster.join(a1) + } + + runOn(a1, a2, b1, b2) { + awaitUpConvergence(numberOfMembers = 4) + } + + assertLeader(a1, a2, b1, b2) + assertLeader(c1, c2) + + enterBarrier("four-members") + } + + "be able to 'elect' a single leader after joining (C -> A + B)" taggedAs LongRunningTest in { + + runOn(b2) { + cluster.join(c1) + } + + awaitUpConvergence(numberOfMembers = 6) + + assertLeader(a1, a2, b1, b2, c1, c2) + + enterBarrier("six-members") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala new file mode 100644 index 0000000000..e3dc7719c1 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala @@ -0,0 +1,293 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ +import akka.actor.ActorSystem +import akka.util.Deadline +import java.util.concurrent.TimeoutException +import scala.collection.immutable.SortedSet +import akka.dispatch.Await +import akka.util.Duration +import java.util.concurrent.TimeUnit +import akka.remote.testconductor.RoleName + +object LargeClusterMultiJvmSpec extends MultiNodeConfig { + // each jvm simulates a datacenter with many nodes + val firstDatacenter = role("first-datacenter") + val secondDatacenter = role("second-datacenter") + val thirdDatacenter = role("third-datacenter") + val fourthDatacenter = role("fourth-datacenter") + val fifthDatacenter = role("fifth-datacenter") + + // Note that this test uses default configuration, + // not MultiNodeClusterSpec.clusterConfig + commonConfig(ConfigFactory.parseString(""" + # Number of ActorSystems in each jvm, can be specified as + # system property when running real tests. Many nodes + # will take long time and consume many threads. + # 10 => 50 nodes is possible to run on one machine. + akka.test.large-cluster-spec.nodes-per-datacenter = 2 + akka.cluster { + gossip-interval = 500 ms + auto-join = off + failure-detector.threshold = 4 + } + akka.loglevel = INFO + akka.actor.default-dispatcher.fork-join-executor.parallelism-max = 2 + akka.scheduler.tick-duration = 33 ms + akka.remote.netty.execution-pool-size = 0 + + # don't use testconductor transport in this test, especially not + # when using use-dispatcher-for-io + akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" + + # Using a separate dispatcher for netty io doesn't reduce number + # of needed threads + # akka.remote.netty.use-dispatcher-for-io=akka.test.io-dispatcher + # akka.test.io-dispatcher.fork-join-executor { + # parallelism-min = 100 + # parallelism-max = 100 + # } + """)) +} + +class LargeClusterMultiJvmNode1 extends LargeClusterSpec with AccrualFailureDetectorStrategy +class LargeClusterMultiJvmNode2 extends LargeClusterSpec with AccrualFailureDetectorStrategy +class LargeClusterMultiJvmNode3 extends LargeClusterSpec with AccrualFailureDetectorStrategy +class LargeClusterMultiJvmNode4 extends LargeClusterSpec with AccrualFailureDetectorStrategy +class LargeClusterMultiJvmNode5 extends LargeClusterSpec with AccrualFailureDetectorStrategy + +abstract class LargeClusterSpec + extends MultiNodeSpec(LargeClusterMultiJvmSpec) + with MultiNodeClusterSpec { + + import LargeClusterMultiJvmSpec._ + + var systems: IndexedSeq[ActorSystem] = IndexedSeq(system) + val nodesPerDatacenter = system.settings.config.getInt( + "akka.test.large-cluster-spec.nodes-per-datacenter") + + /** + * Since we start some ActorSystems/Clusters outside of the + * MultiNodeClusterSpec control we can't use use the mechanism + * defined in MultiNodeClusterSpec to inject failure detector etc. + * Use ordinary Cluster extension with default AccrualFailureDetector. + */ + override def cluster: Cluster = Cluster(system) + + override def atTermination(): Unit = { + systems foreach { _.shutdown } + val shutdownTimeout = 20.seconds + val deadline = Deadline.now + shutdownTimeout + systems.foreach { sys ⇒ + if (sys.isTerminated) + () // already done + else if (deadline.isOverdue) + sys.log.warning("Failed to shutdown [{}] within [{}]", sys.name, shutdownTimeout) + else { + try sys.awaitTermination(deadline.timeLeft) catch { + case _: TimeoutException ⇒ sys.log.warning("Failed to shutdown [{}] within [{}]", sys.name, shutdownTimeout) + } + } + } + } + + def startupSystems(): Unit = { + // one system is already started by the multi-node test + for (n ← 2 to nodesPerDatacenter) + systems :+= ActorSystem(myself.name + "-" + n, system.settings.config) + + // Initialize the Cluster extensions, i.e. startup the clusters + systems foreach { Cluster(_) } + } + + def expectedMaxDuration(totalNodes: Int): Duration = + 5.seconds + (2.seconds * totalNodes) + + def joinAll(from: RoleName, to: RoleName, totalNodes: Int, runOnRoles: RoleName*): Unit = { + val joiningClusters = systems.map(Cluster(_)).toSet + join(joiningClusters, from, to, totalNodes, runOnRoles: _*) + } + + def join(joiningClusterNodes: Set[Cluster], from: RoleName, to: RoleName, totalNodes: Int, runOnRoles: RoleName*): Unit = { + runOnRoles must contain(from) + runOnRoles must contain(to) + + runOn(runOnRoles: _*) { + systems.size must be(nodesPerDatacenter) // make sure it is initialized + + val clusterNodes = ifNode(from)(joiningClusterNodes)(systems.map(Cluster(_)).toSet) + val startGossipCounts = Map.empty[Cluster, Long] ++ + clusterNodes.map(c ⇒ (c -> c.receivedGossipCount)) + def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c) + val startTime = System.nanoTime + def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms" + + val latch = TestLatch(clusterNodes.size) + clusterNodes foreach { c ⇒ + c.registerListener(new MembershipChangeListener { + override def notify(members: SortedSet[Member]): Unit = { + if (!latch.isOpen && members.size == totalNodes && members.forall(_.status == MemberStatus.Up)) { + log.debug("All [{}] nodes Up in [{}], it took [{}], received [{}] gossip messages", + totalNodes, c.selfAddress, tookMillis, gossipCount(c)) + latch.countDown() + } + } + }) + } + + runOn(from) { + clusterNodes foreach { _ join to } + } + + Await.ready(latch, remaining) + + awaitCond(clusterNodes.forall(_.convergence.isDefined)) + val counts = clusterNodes.map(gossipCount(_)) + val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / clusterNodes.size, counts.min, counts.max) + log.info("Convergence of [{}] nodes reached, it took [{}], received [{}] gossip messages per node", + totalNodes, tookMillis, formattedStats) + + } + } + + "A large cluster" must { + + "join all nodes in first-datacenter to first-datacenter" taggedAs LongRunningTest in { + runOn(firstDatacenter) { + startupSystems() + startClusterNode() + } + enterBarrier("first-datacenter-started") + + val totalNodes = nodesPerDatacenter + within(expectedMaxDuration(totalNodes)) { + joinAll(from = firstDatacenter, to = firstDatacenter, totalNodes, runOnRoles = firstDatacenter) + enterBarrier("first-datacenter-joined") + } + } + + "join all nodes in second-datacenter to first-datacenter" taggedAs LongRunningTest in { + runOn(secondDatacenter) { + startupSystems() + } + enterBarrier("second-datacenter-started") + + val totalNodes = nodesPerDatacenter * 2 + within(expectedMaxDuration(totalNodes)) { + joinAll(from = secondDatacenter, to = firstDatacenter, totalNodes, runOnRoles = firstDatacenter, secondDatacenter) + enterBarrier("second-datacenter-joined") + } + } + + "join all nodes in third-datacenter to first-datacenter" taggedAs LongRunningTest in { + runOn(thirdDatacenter) { + startupSystems() + } + enterBarrier("third-datacenter-started") + + val totalNodes = nodesPerDatacenter * 3 + within(expectedMaxDuration(totalNodes)) { + joinAll(from = thirdDatacenter, to = firstDatacenter, totalNodes, + runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter) + enterBarrier("third-datacenter-joined") + } + } + + "join all nodes in fourth-datacenter to first-datacenter" taggedAs LongRunningTest in { + runOn(fourthDatacenter) { + startupSystems() + } + enterBarrier("fourth-datacenter-started") + + val totalNodes = nodesPerDatacenter * 4 + within(expectedMaxDuration(totalNodes)) { + joinAll(from = fourthDatacenter, to = firstDatacenter, totalNodes, + runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter) + enterBarrier("fourth-datacenter-joined") + } + } + + "join nodes one by one from fifth-datacenter to first-datacenter" taggedAs LongRunningTest in { + runOn(fifthDatacenter) { + startupSystems() + } + enterBarrier("fifth-datacenter-started") + + // enough to join a few one-by-one (takes too long time otherwise) + val (bulk, oneByOne) = systems.splitAt(systems.size - 3) + + if (bulk.nonEmpty) { + val totalNodes = nodesPerDatacenter * 4 + bulk.size + within(expectedMaxDuration(totalNodes)) { + val joiningClusters = ifNode(fifthDatacenter)(bulk.map(Cluster(_)).toSet)(Set.empty) + join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes, + runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) + enterBarrier("fifth-datacenter-joined-" + bulk.size) + } + } + + for (i ← 0 until oneByOne.size) { + val totalNodes = nodesPerDatacenter * 4 + bulk.size + i + 1 + within(expectedMaxDuration(totalNodes)) { + val joiningClusters = ifNode(fifthDatacenter)(Set(Cluster(oneByOne(i))))(Set.empty) + join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes, + runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) + enterBarrier("fifth-datacenter-joined-" + (bulk.size + i)) + } + } + } + + // FIXME sometimes this fails, FD marks nodes from other than second-datacenter as unavailable + "detect failure and auto-down crashed nodes in second-datacenter" taggedAs LongRunningTest ignore { + val unreachableNodes = nodesPerDatacenter + val liveNodes = nodesPerDatacenter * 4 + + within(20.seconds + expectedMaxDuration(liveNodes)) { + val startGossipCounts = Map.empty[Cluster, Long] ++ + systems.map(sys ⇒ (Cluster(sys) -> Cluster(sys).receivedGossipCount)) + def gossipCount(c: Cluster): Long = c.receivedGossipCount - startGossipCounts(c) + val startTime = System.nanoTime + def tookMillis: String = TimeUnit.NANOSECONDS.toMillis(System.nanoTime - startTime) + " ms" + + val latch = TestLatch(nodesPerDatacenter) + systems foreach { sys ⇒ + Cluster(sys).registerListener(new MembershipChangeListener { + override def notify(members: SortedSet[Member]): Unit = { + if (!latch.isOpen && members.size == liveNodes && Cluster(sys).latestGossip.overview.unreachable.size == unreachableNodes) { + log.info("Detected [{}] unreachable nodes in [{}], it took [{}], received [{}] gossip messages", + unreachableNodes, Cluster(sys).selfAddress, tookMillis, gossipCount(Cluster(sys))) + latch.countDown() + } + } + }) + } + + runOn(firstDatacenter) { + testConductor.shutdown(secondDatacenter, 0) + } + + enterBarrier("second-datacenter-shutdown") + + runOn(firstDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) { + Await.ready(latch, remaining) + awaitCond(systems.forall(Cluster(_).convergence.isDefined)) + val counts = systems.map(sys ⇒ gossipCount(Cluster(sys))) + val formattedStats = "mean=%s min=%s max=%s".format(counts.sum / nodesPerDatacenter, counts.min, counts.max) + log.info("Convergence of [{}] nodes reached after failure, it took [{}], received [{}] gossip messages per node", + liveNodes, tookMillis, formattedStats) + } + + enterBarrier("after-6") + } + + } + + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala new file mode 100644 index 0000000000..d1640be511 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala @@ -0,0 +1,102 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.actor._ +import akka.util.duration._ + +object LeaderDowningNodeThatIsUnreachableMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + + commonConfig(debugConfig(on = false). + withFallback(ConfigFactory.parseString("akka.cluster.auto-down = on")). + withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode1 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy +class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode2 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy +class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode3 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy +class LeaderDowningNodeThatIsUnreachableWithFailureDetectorPuppetMultiJvmNode4 extends LeaderDowningNodeThatIsUnreachableSpec with FailureDetectorPuppetStrategy + +class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode1 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy +class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode2 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy +class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode3 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy +class LeaderDowningNodeThatIsUnreachableWithAccrualFailureDetectorMultiJvmNode4 extends LeaderDowningNodeThatIsUnreachableSpec with AccrualFailureDetectorStrategy + +abstract class LeaderDowningNodeThatIsUnreachableSpec + extends MultiNodeSpec(LeaderDowningNodeThatIsUnreachableMultiJvmSpec) + with MultiNodeClusterSpec { + + import LeaderDowningNodeThatIsUnreachableMultiJvmSpec._ + + "The Leader in a 4 node cluster" must { + + "be able to DOWN a 'last' node that is UNREACHABLE" taggedAs LongRunningTest in { + awaitClusterUp(first, second, third, fourth) + + val fourthAddress = address(fourth) + runOn(first) { + // kill 'fourth' node + testConductor.shutdown(fourth, 0) + enterBarrier("down-fourth-node") + + // mark the node as unreachable in the failure detector + markNodeAsUnavailable(fourthAddress) + + // --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE --- + + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds) + } + + runOn(fourth) { + enterBarrier("down-fourth-node") + } + + runOn(second, third) { + enterBarrier("down-fourth-node") + + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds) + } + + enterBarrier("await-completion-1") + } + + "be able to DOWN a 'middle' node that is UNREACHABLE" taggedAs LongRunningTest in { + val secondAddress = address(second) + + enterBarrier("before-down-second-node") + runOn(first) { + // kill 'second' node + testConductor.shutdown(second, 0) + enterBarrier("down-second-node") + + // mark the node as unreachable in the failure detector + markNodeAsUnavailable(secondAddress) + + // --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE --- + + awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds) + } + + runOn(second) { + enterBarrier("down-second-node") + } + + runOn(third) { + enterBarrier("down-second-node") + + awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30 seconds) + } + + enterBarrier("await-completion-2") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala new file mode 100644 index 0000000000..9ed8f27ad4 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala @@ -0,0 +1,108 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ + +object LeaderElectionMultiJvmSpec extends MultiNodeConfig { + val controller = role("controller") + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + + commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class LeaderElectionWithFailureDetectorPuppetMultiJvmNode1 extends LeaderElectionSpec with FailureDetectorPuppetStrategy +class LeaderElectionWithFailureDetectorPuppetMultiJvmNode2 extends LeaderElectionSpec with FailureDetectorPuppetStrategy +class LeaderElectionWithFailureDetectorPuppetMultiJvmNode3 extends LeaderElectionSpec with FailureDetectorPuppetStrategy +class LeaderElectionWithFailureDetectorPuppetMultiJvmNode4 extends LeaderElectionSpec with FailureDetectorPuppetStrategy +class LeaderElectionWithFailureDetectorPuppetMultiJvmNode5 extends LeaderElectionSpec with FailureDetectorPuppetStrategy + +class LeaderElectionWithAccrualFailureDetectorMultiJvmNode1 extends LeaderElectionSpec with AccrualFailureDetectorStrategy +class LeaderElectionWithAccrualFailureDetectorMultiJvmNode2 extends LeaderElectionSpec with AccrualFailureDetectorStrategy +class LeaderElectionWithAccrualFailureDetectorMultiJvmNode3 extends LeaderElectionSpec with AccrualFailureDetectorStrategy +class LeaderElectionWithAccrualFailureDetectorMultiJvmNode4 extends LeaderElectionSpec with AccrualFailureDetectorStrategy +class LeaderElectionWithAccrualFailureDetectorMultiJvmNode5 extends LeaderElectionSpec with AccrualFailureDetectorStrategy + +abstract class LeaderElectionSpec + extends MultiNodeSpec(LeaderElectionMultiJvmSpec) + with MultiNodeClusterSpec { + + import LeaderElectionMultiJvmSpec._ + + // sorted in the order used by the cluster + lazy val sortedRoles = Seq(first, second, third, fourth).sorted + + "A cluster of four nodes" must { + + "be able to 'elect' a single leader" taggedAs LongRunningTest in { + awaitClusterUp(first, second, third, fourth) + + if (myself != controller) { + cluster.isLeader must be(myself == sortedRoles.head) + assertLeaderIn(sortedRoles) + } + + enterBarrier("after-1") + } + + def shutdownLeaderAndVerifyNewLeader(alreadyShutdown: Int): Unit = { + val currentRoles = sortedRoles.drop(alreadyShutdown) + currentRoles.size must be >= (2) + val leader = currentRoles.head + val aUser = currentRoles.last + val remainingRoles = currentRoles.tail + + myself match { + + case `controller` ⇒ + val leaderAddress = address(leader) + enterBarrier("before-shutdown") + testConductor.shutdown(leader, 0) + enterBarrier("after-shutdown", "after-down", "completed") + markNodeAsUnavailable(leaderAddress) + + case `leader` ⇒ + enterBarrier("before-shutdown", "after-shutdown") + // this node will be shutdown by the controller and doesn't participate in more barriers + + case `aUser` ⇒ + val leaderAddress = address(leader) + enterBarrier("before-shutdown", "after-shutdown") + // user marks the shutdown leader as DOWN + cluster.down(leaderAddress) + enterBarrier("after-down", "completed") + markNodeAsUnavailable(leaderAddress) + + case _ if remainingRoles.contains(myself) ⇒ + // remaining cluster nodes, not shutdown + enterBarrier("before-shutdown", "after-shutdown", "after-down") + + awaitUpConvergence(currentRoles.size - 1) + val nextExpectedLeader = remainingRoles.head + cluster.isLeader must be(myself == nextExpectedLeader) + assertLeaderIn(remainingRoles) + + enterBarrier("completed") + + } + } + + "be able to 're-elect' a single leader after leader has left" taggedAs LongRunningTest in { + shutdownLeaderAndVerifyNewLeader(alreadyShutdown = 0) + enterBarrier("after-2") + } + + "be able to 're-elect' a single leader after leader has left (again)" taggedAs LongRunningTest in { + shutdownLeaderAndVerifyNewLeader(alreadyShutdown = 1) + enterBarrier("after-3") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderLeavingSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderLeavingSpec.scala new file mode 100644 index 0000000000..54154b6973 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderLeavingSpec.scala @@ -0,0 +1,85 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import scala.collection.immutable.SortedSet +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ + +object LeaderLeavingMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig( + debugConfig(on = false) + .withFallback(ConfigFactory.parseString(""" + akka.cluster { + leader-actions-interval = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state + unreachable-nodes-reaper-interval = 30 s + }""") + .withFallback(MultiNodeClusterSpec.clusterConfig))) +} + +class LeaderLeavingMultiJvmNode1 extends LeaderLeavingSpec with FailureDetectorPuppetStrategy +class LeaderLeavingMultiJvmNode2 extends LeaderLeavingSpec with FailureDetectorPuppetStrategy +class LeaderLeavingMultiJvmNode3 extends LeaderLeavingSpec with FailureDetectorPuppetStrategy + +abstract class LeaderLeavingSpec + extends MultiNodeSpec(LeaderLeavingMultiJvmSpec) + with MultiNodeClusterSpec { + + import LeaderLeavingMultiJvmSpec._ + + val leaderHandoffWaitingTime = 30.seconds.dilated + + "A LEADER that is LEAVING" must { + + "be moved to LEAVING, then to EXITING, then to REMOVED, then be shut down and then a new LEADER should be elected" taggedAs LongRunningTest in { + + awaitClusterUp(first, second, third) + + val oldLeaderAddress = cluster.leader + + if (cluster.isLeader) { + + cluster.leave(oldLeaderAddress) + enterBarrier("leader-left") + + // verify that a NEW LEADER have taken over + awaitCond(!cluster.isLeader) + + // verify that the LEADER is shut down + awaitCond(!cluster.isRunning, 30.seconds.dilated) + + // verify that the LEADER is REMOVED + awaitCond(cluster.status == MemberStatus.Removed) + + } else { + + enterBarrier("leader-left") + + // verify that the LEADER is LEAVING + awaitCond(cluster.latestGossip.members.exists(m ⇒ m.status == MemberStatus.Leaving && m.address == oldLeaderAddress), leaderHandoffWaitingTime) // wait on LEAVING + + // verify that the LEADER is EXITING + awaitCond(cluster.latestGossip.members.exists(m ⇒ m.status == MemberStatus.Exiting && m.address == oldLeaderAddress), leaderHandoffWaitingTime) // wait on EXITING + + // verify that the LEADER is no longer part of the 'members' set + awaitCond(cluster.latestGossip.members.forall(_.address != oldLeaderAddress), leaderHandoffWaitingTime) + + // verify that the LEADER is not part of the 'unreachable' set + awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != oldLeaderAddress), leaderHandoffWaitingTime) + + // verify that we have a new LEADER + awaitCond(cluster.leader != oldLeaderAddress, leaderHandoffWaitingTime) + } + + enterBarrier("finished") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerExitingSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerExitingSpec.scala new file mode 100644 index 0000000000..ee74584953 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerExitingSpec.scala @@ -0,0 +1,69 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import scala.collection.immutable.SortedSet +import org.scalatest.BeforeAndAfter +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ + +object MembershipChangeListenerExitingMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig( + debugConfig(on = false) + .withFallback(ConfigFactory.parseString(""" + akka.cluster { + leader-actions-interval = 5 s # increase the leader action task interval + unreachable-nodes-reaper-interval = 300 s # turn "off" reaping to unreachable node set + } + """) + .withFallback(MultiNodeClusterSpec.clusterConfig))) +} + +class MembershipChangeListenerExitingMultiJvmNode1 extends MembershipChangeListenerExitingSpec with FailureDetectorPuppetStrategy +class MembershipChangeListenerExitingMultiJvmNode2 extends MembershipChangeListenerExitingSpec with FailureDetectorPuppetStrategy +class MembershipChangeListenerExitingMultiJvmNode3 extends MembershipChangeListenerExitingSpec with FailureDetectorPuppetStrategy + +abstract class MembershipChangeListenerExitingSpec + extends MultiNodeSpec(MembershipChangeListenerExitingMultiJvmSpec) + with MultiNodeClusterSpec { + + import MembershipChangeListenerExitingMultiJvmSpec._ + + "A registered MembershipChangeListener" must { + "be notified when new node is EXITING" taggedAs LongRunningTest in { + + awaitClusterUp(first, second, third) + + runOn(first) { + enterBarrier("registered-listener") + cluster.leave(second) + } + + runOn(second) { + enterBarrier("registered-listener") + } + + runOn(third) { + val exitingLatch = TestLatch() + cluster.registerListener(new MembershipChangeListener { + def notify(members: SortedSet[Member]) { + if (members.size == 3 && members.exists(m ⇒ m.address == address(second) && m.status == MemberStatus.Exiting)) + exitingLatch.countDown() + } + }) + enterBarrier("registered-listener") + exitingLatch.await + } + + enterBarrier("finished") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerJoinSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerJoinSpec.scala new file mode 100644 index 0000000000..eacec24109 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerJoinSpec.scala @@ -0,0 +1,60 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import scala.collection.immutable.SortedSet +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfter +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ + +object MembershipChangeListenerJoinMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + commonConfig( + debugConfig(on = false) + .withFallback(ConfigFactory.parseString("akka.cluster.leader-actions-interval = 5 s") // increase the leader action task interval to allow time checking for JOIN before leader moves it to UP + .withFallback(MultiNodeClusterSpec.clusterConfig))) +} + +class MembershipChangeListenerJoinMultiJvmNode1 extends MembershipChangeListenerJoinSpec with FailureDetectorPuppetStrategy +class MembershipChangeListenerJoinMultiJvmNode2 extends MembershipChangeListenerJoinSpec with FailureDetectorPuppetStrategy + +abstract class MembershipChangeListenerJoinSpec + extends MultiNodeSpec(MembershipChangeListenerJoinMultiJvmSpec) + with MultiNodeClusterSpec { + + import MembershipChangeListenerJoinMultiJvmSpec._ + + "A registered MembershipChangeListener" must { + "be notified when new node is JOINING" taggedAs LongRunningTest in { + + runOn(first) { + val joinLatch = TestLatch() + val expectedAddresses = Set(first, second) map address + cluster.registerListener(new MembershipChangeListener { + def notify(members: SortedSet[Member]) { + if (members.map(_.address) == expectedAddresses && members.exists(_.status == MemberStatus.Joining)) + joinLatch.countDown() + } + }) + enterBarrier("registered-listener") + + joinLatch.await + } + + runOn(second) { + enterBarrier("registered-listener") + cluster.join(first) + } + + awaitUpConvergence(2) + + enterBarrier("after") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerLeavingSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerLeavingSpec.scala new file mode 100644 index 0000000000..e6430314d4 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerLeavingSpec.scala @@ -0,0 +1,69 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import scala.collection.immutable.SortedSet +import org.scalatest.BeforeAndAfter +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.actor.Address + +object MembershipChangeListenerLeavingMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig( + debugConfig(on = false) + .withFallback(ConfigFactory.parseString(""" + akka.cluster.leader-actions-interval = 5 s + akka.cluster.unreachable-nodes-reaper-interval = 300 s # turn "off" + """)) + .withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class MembershipChangeListenerLeavingMultiJvmNode1 extends MembershipChangeListenerLeavingSpec with FailureDetectorPuppetStrategy +class MembershipChangeListenerLeavingMultiJvmNode2 extends MembershipChangeListenerLeavingSpec with FailureDetectorPuppetStrategy +class MembershipChangeListenerLeavingMultiJvmNode3 extends MembershipChangeListenerLeavingSpec with FailureDetectorPuppetStrategy + +abstract class MembershipChangeListenerLeavingSpec + extends MultiNodeSpec(MembershipChangeListenerLeavingMultiJvmSpec) + with MultiNodeClusterSpec { + + import MembershipChangeListenerLeavingMultiJvmSpec._ + + "A registered MembershipChangeListener" must { + "be notified when new node is LEAVING" taggedAs LongRunningTest in { + + awaitClusterUp(first, second, third) + + runOn(first) { + enterBarrier("registered-listener") + cluster.leave(second) + } + + runOn(second) { + enterBarrier("registered-listener") + } + + runOn(third) { + val latch = TestLatch() + val expectedAddresses = Set(first, second, third) map address + cluster.registerListener(new MembershipChangeListener { + def notify(members: SortedSet[Member]) { + if (members.map(_.address) == expectedAddresses && + members.exists(m ⇒ m.address == address(second) && m.status == MemberStatus.Leaving)) + latch.countDown() + } + }) + enterBarrier("registered-listener") + latch.await + } + + enterBarrier("finished") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerUpSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerUpSpec.scala new file mode 100644 index 0000000000..5638399b59 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MembershipChangeListenerUpSpec.scala @@ -0,0 +1,78 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import scala.collection.immutable.SortedSet +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ + +object MembershipChangeListenerUpMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class MembershipChangeListenerUpMultiJvmNode1 extends MembershipChangeListenerUpSpec with FailureDetectorPuppetStrategy +class MembershipChangeListenerUpMultiJvmNode2 extends MembershipChangeListenerUpSpec with FailureDetectorPuppetStrategy +class MembershipChangeListenerUpMultiJvmNode3 extends MembershipChangeListenerUpSpec with FailureDetectorPuppetStrategy + +abstract class MembershipChangeListenerUpSpec + extends MultiNodeSpec(MembershipChangeListenerUpMultiJvmSpec) + with MultiNodeClusterSpec { + + import MembershipChangeListenerUpMultiJvmSpec._ + + "A set of connected cluster systems" must { + + "(when two nodes) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in { + + awaitClusterUp(first) + + runOn(first, second) { + val latch = TestLatch() + val expectedAddresses = Set(first, second) map address + cluster.registerListener(new MembershipChangeListener { + def notify(members: SortedSet[Member]) { + if (members.map(_.address) == expectedAddresses && members.forall(_.status == MemberStatus.Up)) + latch.countDown() + } + }) + enterBarrier("listener-1-registered") + cluster.join(first) + latch.await + } + + runOn(third) { + enterBarrier("listener-1-registered") + } + + enterBarrier("after-1") + } + + "(when three nodes) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in { + + val latch = TestLatch() + val expectedAddresses = Set(first, second, third) map address + cluster.registerListener(new MembershipChangeListener { + def notify(members: SortedSet[Member]) { + if (members.map(_.address) == expectedAddresses && members.forall(_.status == MemberStatus.Up)) + latch.countDown() + } + }) + enterBarrier("listener-2-registered") + + runOn(third) { + cluster.join(first) + } + + latch.await + + enterBarrier("after-2") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala new file mode 100644 index 0000000000..3264c661b0 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala @@ -0,0 +1,216 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.Config +import com.typesafe.config.ConfigFactory +import akka.actor.{ Address, ExtendedActorSystem } +import akka.remote.testconductor.RoleName +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ +import akka.util.Duration +import org.scalatest.Suite +import org.scalatest.TestFailedException +import java.util.concurrent.ConcurrentHashMap +import akka.actor.ActorPath +import akka.actor.RootActorPath + +object MultiNodeClusterSpec { + def clusterConfig: Config = ConfigFactory.parseString(""" + akka.cluster { + auto-join = off + auto-down = off + gossip-interval = 200 ms + heartbeat-interval = 400 ms + leader-actions-interval = 200 ms + unreachable-nodes-reaper-interval = 200 ms + periodic-tasks-initial-delay = 300 ms + } + akka.test { + single-expect-default = 5 s + } + """) +} + +trait MultiNodeClusterSpec extends FailureDetectorStrategy with Suite { self: MultiNodeSpec ⇒ + + override def initialParticipants = roles.size + + private val cachedAddresses = new ConcurrentHashMap[RoleName, Address] + + /** + * Lookup the Address for the role. + * + * Implicit conversion from RoleName to Address. + * + * It is cached, which has the implication that stopping + * and then restarting a role (jvm) with another address is not + * supported. + */ + implicit def address(role: RoleName): Address = { + cachedAddresses.get(role) match { + case null ⇒ + val address = node(role).address + cachedAddresses.put(role, address) + address + case address ⇒ address + } + } + + // Cluster tests are written so that if previous step (test method) failed + // it will most likely not be possible to run next step. This ensures + // fail fast of steps after the first failure. + private var failed = false + override protected def withFixture(test: NoArgTest): Unit = try { + if (failed) { + val e = new TestFailedException("Previous step failed", 0) + // short stack trace + e.setStackTrace(e.getStackTrace.take(1)) + throw e + } + super.withFixture(test) + } catch { + case t ⇒ + failed = true + throw t + } + + /** + * Make it possible to override/configure seedNodes from tests without + * specifying in config. Addresses are unknown before startup time. + */ + protected def seedNodes: IndexedSeq[RoleName] = IndexedSeq.empty + + /** + * The cluster node instance. Needs to be lazily created. + */ + private lazy val clusterNode = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector) { + override def seedNodes: IndexedSeq[Address] = { + val testSeedNodes = MultiNodeClusterSpec.this.seedNodes + if (testSeedNodes.isEmpty) super.seedNodes + else testSeedNodes map address + } + } + + /** + * Get the cluster node to use. + */ + def cluster: Cluster = clusterNode + + /** + * Use this method for the initial startup of the cluster node. + */ + def startClusterNode(): Unit = { + if (cluster.latestGossip.members.isEmpty) { + cluster join myself + awaitCond(cluster.latestGossip.members.exists(_.address == address(myself))) + } else + cluster.self + } + + /** + * Initialize the cluster with the specified member + * nodes (roles). First node will be started first + * and others will join the first. + */ + def startCluster(roles: RoleName*): Unit = awaitStartCluster(false, roles.toSeq) + + /** + * Initialize the cluster of the specified member + * nodes (roles) and wait until all joined and `Up`. + * First node will be started first and others will join + * the first. + */ + def awaitClusterUp(roles: RoleName*): Unit = { + awaitStartCluster(true, roles.toSeq) + } + + private def awaitStartCluster(upConvergence: Boolean = true, roles: Seq[RoleName]): Unit = { + runOn(roles.head) { + // make sure that the node-to-join is started before other join + startClusterNode() + } + enterBarrier(roles.head.name + "-started") + if (roles.tail.contains(myself)) { + cluster.join(roles.head) + } + if (upConvergence && roles.contains(myself)) { + awaitUpConvergence(numberOfMembers = roles.length) + } + enterBarrier(roles.map(_.name).mkString("-") + "-joined") + } + + /** + * Assert that the member addresses match the expected addresses in the + * sort order used by the cluster. + */ + def assertMembers(gotMembers: Iterable[Member], expectedAddresses: Address*): Unit = { + import Member.addressOrdering + val members = gotMembers.toIndexedSeq + members.size must be(expectedAddresses.length) + expectedAddresses.sorted.zipWithIndex.foreach { case (a, i) ⇒ members(i).address must be(a) } + } + + def assertLeader(nodesInCluster: RoleName*): Unit = if (nodesInCluster.contains(myself)) { + assertLeaderIn(nodesInCluster) + } + + /** + * Assert that the cluster has elected the correct leader + * out of all nodes in the cluster. First + * member in the cluster ring is expected leader. + */ + def assertLeaderIn(nodesInCluster: Seq[RoleName]): Unit = if (nodesInCluster.contains(myself)) { + nodesInCluster.length must not be (0) + val expectedLeader = roleOfLeader(nodesInCluster) + cluster.isLeader must be(ifNode(expectedLeader)(true)(false)) + cluster.status must (be(MemberStatus.Up) or be(MemberStatus.Leaving)) + } + + /** + * Wait until the expected number of members has status Up and convergence has been reached. + * Also asserts that nodes in the 'canNotBePartOfMemberRing' are *not* part of the cluster ring. + */ + def awaitUpConvergence( + numberOfMembers: Int, + canNotBePartOfMemberRing: Seq[Address] = Seq.empty[Address], + timeout: Duration = 20.seconds): Unit = { + within(timeout) { + awaitCond(cluster.latestGossip.members.size == numberOfMembers) + awaitCond(cluster.latestGossip.members.forall(_.status == MemberStatus.Up)) + awaitCond(cluster.convergence.isDefined) + if (!canNotBePartOfMemberRing.isEmpty) // don't run this on an empty set + awaitCond( + canNotBePartOfMemberRing forall (address ⇒ !(cluster.latestGossip.members exists (_.address == address)))) + } + } + + /** + * Wait until the specified nodes have seen the same gossip overview. + */ + def awaitSeenSameState(addresses: Address*): Unit = { + awaitCond { + val seen = cluster.latestGossip.overview.seen + val seenVectorClocks = addresses.flatMap(seen.get(_)) + seenVectorClocks.size == addresses.size && seenVectorClocks.toSet.size == 1 + } + } + + def roleOfLeader(nodesInCluster: Seq[RoleName] = roles): RoleName = { + nodesInCluster.length must not be (0) + nodesInCluster.sorted.head + } + + /** + * Sort the roles in the order used by the cluster. + */ + implicit val clusterOrdering: Ordering[RoleName] = new Ordering[RoleName] { + import Member.addressOrdering + def compare(x: RoleName, y: RoleName) = addressOrdering.compare(address(x), address(y)) + } + + def roleName(addr: Address): Option[RoleName] = roles.find(address(_) == addr) + +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeJoinSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeJoinSpec.scala new file mode 100644 index 0000000000..50656a6a9d --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeJoinSpec.scala @@ -0,0 +1,50 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfter +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ + +object NodeJoinMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + commonConfig( + debugConfig(on = false) + .withFallback(ConfigFactory.parseString("akka.cluster.leader-actions-interval = 5 s") // increase the leader action task interval + .withFallback(MultiNodeClusterSpec.clusterConfig))) +} + +class NodeJoinMultiJvmNode1 extends NodeJoinSpec with FailureDetectorPuppetStrategy +class NodeJoinMultiJvmNode2 extends NodeJoinSpec with FailureDetectorPuppetStrategy + +abstract class NodeJoinSpec + extends MultiNodeSpec(NodeJoinMultiJvmSpec) + with MultiNodeClusterSpec { + + import NodeJoinMultiJvmSpec._ + + "A cluster node" must { + "join another cluster and get status JOINING - when sending a 'Join' command" taggedAs LongRunningTest in { + + runOn(first) { + startClusterNode() + } + + enterBarrier("first-started") + + runOn(second) { + cluster.join(first) + } + + awaitCond(cluster.latestGossip.members.exists { member ⇒ member.address == address(second) && member.status == MemberStatus.Joining }) + + enterBarrier("after") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingAndBeingRemovedSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingAndBeingRemovedSpec.scala new file mode 100644 index 0000000000..64f52c4549 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingAndBeingRemovedSpec.scala @@ -0,0 +1,61 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import scala.collection.immutable.SortedSet +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ + +object NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode1 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy +class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode2 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy +class NodeLeavingAndExitingAndBeingRemovedMultiJvmNode3 extends NodeLeavingAndExitingAndBeingRemovedSpec with FailureDetectorPuppetStrategy + +abstract class NodeLeavingAndExitingAndBeingRemovedSpec + extends MultiNodeSpec(NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec) + with MultiNodeClusterSpec { + + import NodeLeavingAndExitingAndBeingRemovedMultiJvmSpec._ + + val reaperWaitingTime = 30.seconds.dilated + + "A node that is LEAVING a non-singleton cluster" must { + + "eventually set to REMOVED by the reaper, and removed from membership ring and seen table" taggedAs LongRunningTest in { + + awaitClusterUp(first, second, third) + + runOn(first) { + cluster.leave(second) + } + enterBarrier("second-left") + + runOn(first, third) { + // verify that the 'second' node is no longer part of the 'members' set + awaitCond(cluster.latestGossip.members.forall(_.address != address(second)), reaperWaitingTime) + + // verify that the 'second' node is not part of the 'unreachable' set + awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != address(second)), reaperWaitingTime) + } + + runOn(second) { + // verify that the second node is shut down and has status REMOVED + awaitCond(!cluster.isRunning, reaperWaitingTime) + awaitCond(cluster.status == MemberStatus.Removed, reaperWaitingTime) + } + + enterBarrier("finished") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingSpec.scala new file mode 100644 index 0000000000..5f9efb0b47 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingAndExitingSpec.scala @@ -0,0 +1,70 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import scala.collection.immutable.SortedSet +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ + +object NodeLeavingAndExitingMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig( + debugConfig(on = false) + .withFallback(ConfigFactory.parseString(""" + akka.cluster { + leader-actions-interval = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state + unreachable-nodes-reaper-interval = 300 s # turn "off" + } + """) + .withFallback(MultiNodeClusterSpec.clusterConfig))) +} + +class NodeLeavingAndExitingMultiJvmNode1 extends NodeLeavingAndExitingSpec with FailureDetectorPuppetStrategy +class NodeLeavingAndExitingMultiJvmNode2 extends NodeLeavingAndExitingSpec with FailureDetectorPuppetStrategy +class NodeLeavingAndExitingMultiJvmNode3 extends NodeLeavingAndExitingSpec with FailureDetectorPuppetStrategy + +abstract class NodeLeavingAndExitingSpec + extends MultiNodeSpec(NodeLeavingAndExitingMultiJvmSpec) + with MultiNodeClusterSpec { + + import NodeLeavingAndExitingMultiJvmSpec._ + + "A node that is LEAVING a non-singleton cluster" must { + + "be moved to EXITING by the leader" taggedAs LongRunningTest in { + + awaitClusterUp(first, second, third) + + runOn(first) { + cluster.leave(second) + } + enterBarrier("second-left") + + runOn(first, third) { + + // 1. Verify that 'second' node is set to LEAVING + // We have set the 'leader-actions-interval' to 5 seconds to make sure that we get a + // chance to test the LEAVING state before the leader moves the node to EXITING + awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Leaving)) // wait on LEAVING + val hasLeft = cluster.latestGossip.members.find(_.status == MemberStatus.Leaving) // verify node that left + hasLeft must be('defined) + hasLeft.get.address must be(address(second)) + + // 2. Verify that 'second' node is set to EXITING + awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Exiting)) // wait on EXITING + val hasExited = cluster.latestGossip.members.find(_.status == MemberStatus.Exiting) // verify node that exited + hasExited must be('defined) + hasExited.get.address must be(address(second)) + } + + enterBarrier("finished") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingSpec.scala new file mode 100644 index 0000000000..9ece38aae8 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeLeavingSpec.scala @@ -0,0 +1,55 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import scala.collection.immutable.SortedSet +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ + +object NodeLeavingMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig( + debugConfig(on = false) + .withFallback(ConfigFactory.parseString("akka.cluster.unreachable-nodes-reaper-frequency = 30 s")) + .withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class NodeLeavingMultiJvmNode1 extends NodeLeavingSpec with FailureDetectorPuppetStrategy +class NodeLeavingMultiJvmNode2 extends NodeLeavingSpec with FailureDetectorPuppetStrategy +class NodeLeavingMultiJvmNode3 extends NodeLeavingSpec with FailureDetectorPuppetStrategy + +abstract class NodeLeavingSpec + extends MultiNodeSpec(NodeLeavingMultiJvmSpec) + with MultiNodeClusterSpec { + + import NodeLeavingMultiJvmSpec._ + + "A node that is LEAVING a non-singleton cluster" must { + + "be marked as LEAVING in the converged membership table" taggedAs LongRunningTest in { + + awaitClusterUp(first, second, third) + + runOn(first) { + cluster.leave(second) + } + enterBarrier("second-left") + + runOn(first, third) { + awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Leaving)) + + val hasLeft = cluster.latestGossip.members.find(_.status == MemberStatus.Leaving) + hasLeft must be('defined) + hasLeft.get.address must be(address(second)) + } + + enterBarrier("finished") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeMembershipSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeMembershipSpec.scala new file mode 100644 index 0000000000..364edca08b --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeMembershipSpec.scala @@ -0,0 +1,68 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ + +object NodeMembershipMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class NodeMembershipMultiJvmNode1 extends NodeMembershipSpec with FailureDetectorPuppetStrategy +class NodeMembershipMultiJvmNode2 extends NodeMembershipSpec with FailureDetectorPuppetStrategy +class NodeMembershipMultiJvmNode3 extends NodeMembershipSpec with FailureDetectorPuppetStrategy + +abstract class NodeMembershipSpec + extends MultiNodeSpec(NodeMembershipMultiJvmSpec) + with MultiNodeClusterSpec { + + import NodeMembershipMultiJvmSpec._ + + "A set of connected cluster systems" must { + + "(when two nodes) start gossiping to each other so that both nodes gets the same gossip info" taggedAs LongRunningTest in { + + // make sure that the node-to-join is started before other join + runOn(first) { + startClusterNode() + } + enterBarrier("first-started") + + runOn(first, second) { + cluster.join(first) + awaitCond(cluster.latestGossip.members.size == 2) + assertMembers(cluster.latestGossip.members, first, second) + awaitCond { + cluster.latestGossip.members.forall(_.status == MemberStatus.Up) + } + awaitCond(cluster.convergence.isDefined) + } + + enterBarrier("after-1") + } + + "(when three nodes) start gossiping to each other so that all nodes gets the same gossip info" taggedAs LongRunningTest in { + + runOn(third) { + cluster.join(first) + } + + awaitCond(cluster.latestGossip.members.size == 3) + assertMembers(cluster.latestGossip.members, first, second, third) + awaitCond { + cluster.latestGossip.members.forall(_.status == MemberStatus.Up) + } + awaitCond(cluster.convergence.isDefined) + + enterBarrier("after-2") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeUpSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeUpSpec.scala new file mode 100644 index 0000000000..3da6b2715a --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/NodeUpSpec.scala @@ -0,0 +1,65 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfter +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ +import scala.collection.immutable.SortedSet +import java.util.concurrent.atomic.AtomicReference + +object NodeUpMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class NodeUpMultiJvmNode1 extends NodeUpSpec with FailureDetectorPuppetStrategy +class NodeUpMultiJvmNode2 extends NodeUpSpec with FailureDetectorPuppetStrategy + +abstract class NodeUpSpec + extends MultiNodeSpec(NodeUpMultiJvmSpec) + with MultiNodeClusterSpec { + + import NodeUpMultiJvmSpec._ + + "A cluster node that is joining another cluster" must { + "be moved to UP by the leader after a convergence" taggedAs LongRunningTest in { + + awaitClusterUp(first, second) + + enterBarrier("after-1") + } + + "be unaffected when joining again" taggedAs LongRunningTest in { + + val unexpected = new AtomicReference[SortedSet[Member]] + cluster.registerListener(new MembershipChangeListener { + def notify(members: SortedSet[Member]) { + if (members.size != 2 || members.exists(_.status != MemberStatus.Up)) + unexpected.set(members) + } + }) + enterBarrier("listener-registered") + + runOn(second) { + cluster.join(first) + } + enterBarrier("joined-again") + + // let it run for a while to make sure that nothing bad happens + for (n ← 1 to 20) { + 100.millis.dilated.sleep() + unexpected.get must be(null) + cluster.latestGossip.members.forall(_.status == MemberStatus.Up) must be(true) + } + + enterBarrier("after-2") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala new file mode 100644 index 0000000000..3c35e95333 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala @@ -0,0 +1,73 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ + +object SingletonClusterMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + + commonConfig(debugConfig(on = false). + withFallback(ConfigFactory.parseString(""" + akka.cluster { + auto-join = on + auto-down = on + failure-detector.threshold = 4 + } + """)). + withFallback(MultiNodeClusterSpec.clusterConfig)) + +} + +class SingletonClusterWithFailureDetectorPuppetMultiJvmNode1 extends SingletonClusterSpec with FailureDetectorPuppetStrategy +class SingletonClusterWithFailureDetectorPuppetMultiJvmNode2 extends SingletonClusterSpec with FailureDetectorPuppetStrategy + +class SingletonClusterWithAccrualFailureDetectorMultiJvmNode1 extends SingletonClusterSpec with AccrualFailureDetectorStrategy +class SingletonClusterWithAccrualFailureDetectorMultiJvmNode2 extends SingletonClusterSpec with AccrualFailureDetectorStrategy + +abstract class SingletonClusterSpec + extends MultiNodeSpec(SingletonClusterMultiJvmSpec) + with MultiNodeClusterSpec { + + import SingletonClusterMultiJvmSpec._ + + "A cluster of 2 nodes" must { + + "become singleton cluster when started with 'auto-join=on' and 'seed-nodes=[]'" taggedAs LongRunningTest in { + startClusterNode() + awaitUpConvergence(1) + cluster.isSingletonCluster must be(true) + + enterBarrier("after-1") + } + + "not be singleton cluster when joined with other node" taggedAs LongRunningTest in { + awaitClusterUp(first, second) + cluster.isSingletonCluster must be(false) + assertLeader(first, second) + + enterBarrier("after-2") + } + + "become singleton cluster when one node is shutdown" taggedAs LongRunningTest in { + runOn(first) { + val secondAddress = address(second) + testConductor.shutdown(second, 0) + + markNodeAsUnavailable(secondAddress) + + awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds) + cluster.isSingletonCluster must be(true) + assertLeader(first) + } + + enterBarrier("after-3") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala new file mode 100644 index 0000000000..24e94f715d --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala @@ -0,0 +1,111 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ +import akka.actor.Address +import akka.remote.testconductor.Direction + +object SplitBrainMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + val fifth = role("fifth") + + commonConfig(debugConfig(on = false). + withFallback(ConfigFactory.parseString(""" + akka.cluster { + auto-down = on + failure-detector.threshold = 4 + }""")). + withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class SplitBrainWithFailureDetectorPuppetMultiJvmNode1 extends SplitBrainSpec with FailureDetectorPuppetStrategy +class SplitBrainWithFailureDetectorPuppetMultiJvmNode2 extends SplitBrainSpec with FailureDetectorPuppetStrategy +class SplitBrainWithFailureDetectorPuppetMultiJvmNode3 extends SplitBrainSpec with FailureDetectorPuppetStrategy +class SplitBrainWithFailureDetectorPuppetMultiJvmNode4 extends SplitBrainSpec with FailureDetectorPuppetStrategy +class SplitBrainWithFailureDetectorPuppetMultiJvmNode5 extends SplitBrainSpec with FailureDetectorPuppetStrategy + +class SplitBrainWithAccrualFailureDetectorMultiJvmNode1 extends SplitBrainSpec with AccrualFailureDetectorStrategy +class SplitBrainWithAccrualFailureDetectorMultiJvmNode2 extends SplitBrainSpec with AccrualFailureDetectorStrategy +class SplitBrainWithAccrualFailureDetectorMultiJvmNode3 extends SplitBrainSpec with AccrualFailureDetectorStrategy +class SplitBrainWithAccrualFailureDetectorMultiJvmNode4 extends SplitBrainSpec with AccrualFailureDetectorStrategy +class SplitBrainWithAccrualFailureDetectorMultiJvmNode5 extends SplitBrainSpec with AccrualFailureDetectorStrategy + +abstract class SplitBrainSpec + extends MultiNodeSpec(SplitBrainMultiJvmSpec) + with MultiNodeClusterSpec { + + import SplitBrainMultiJvmSpec._ + + val side1 = IndexedSeq(first, second) + val side2 = IndexedSeq(third, fourth, fifth) + + "A cluster of 5 members" must { + + "reach initial convergence" taggedAs LongRunningTest in { + awaitClusterUp(first, second, third, fourth, fifth) + + enterBarrier("after-1") + } + + "detect network partition and mark nodes on other side as unreachable" taggedAs LongRunningTest in { + val thirdAddress = address(third) + enterBarrier("before-split") + + runOn(first) { + // split the cluster in two parts (first, second) / (third, fourth, fifth) + for (role1 ← side1; role2 ← side2) { + testConductor.blackhole(role1, role2, Direction.Both).await + } + } + enterBarrier("after-split") + + runOn(side1.last) { + for (role ← side2) markNodeAsUnavailable(role) + } + runOn(side2.last) { + for (role ← side1) markNodeAsUnavailable(role) + } + + runOn(side1: _*) { + awaitCond(cluster.latestGossip.overview.unreachable.map(_.address) == (side2.toSet map address), 20 seconds) + } + runOn(side2: _*) { + awaitCond(cluster.latestGossip.overview.unreachable.map(_.address) == (side1.toSet map address), 20 seconds) + } + + enterBarrier("after-2") + } + + "auto-down the other nodes and form new cluster with potentially new leader" taggedAs LongRunningTest in { + + runOn(side1: _*) { + // auto-down = on + awaitCond(cluster.latestGossip.overview.unreachable.forall(m ⇒ m.status == MemberStatus.Down), 15 seconds) + cluster.latestGossip.overview.unreachable.map(_.address) must be(side2.toSet map address) + awaitUpConvergence(side1.size, side2 map address) + assertLeader(side1: _*) + } + + runOn(side2: _*) { + // auto-down = on + awaitCond(cluster.latestGossip.overview.unreachable.forall(m ⇒ m.status == MemberStatus.Down), 15 seconds) + cluster.latestGossip.overview.unreachable.map(_.address) must be(side1.toSet map address) + awaitUpConvergence(side2.size, side1 map address) + assertLeader(side2: _*) + } + + enterBarrier("after-3") + } + + } + +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala new file mode 100644 index 0000000000..ddacf668e0 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala @@ -0,0 +1,77 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfter +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.util.duration._ +import java.util.concurrent.atomic.AtomicReference +import scala.collection.immutable.SortedSet + +object SunnyWeatherMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + val fifth = role("fifth") + + // Note that this test uses default configuration, + // not MultiNodeClusterSpec.clusterConfig + commonConfig(ConfigFactory.parseString(""" + akka.cluster { + auto-join = off + } + akka.loglevel = INFO + """)) +} + +class SunnyWeatherMultiJvmNode1 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy +class SunnyWeatherMultiJvmNode2 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy +class SunnyWeatherMultiJvmNode3 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy +class SunnyWeatherMultiJvmNode4 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy +class SunnyWeatherMultiJvmNode5 extends SunnyWeatherSpec with AccrualFailureDetectorStrategy + +abstract class SunnyWeatherSpec + extends MultiNodeSpec(SunnyWeatherMultiJvmSpec) + with MultiNodeClusterSpec { + + import SunnyWeatherMultiJvmSpec._ + + "A normal cluster" must { + "be healthy" taggedAs LongRunningTest in { + + // start some + awaitClusterUp(first, second, third) + runOn(first, second, third) { + log.info("3 joined") + } + + // add a few more + awaitClusterUp(roles: _*) + log.info("5 joined") + + val unexpected = new AtomicReference[SortedSet[Member]] + cluster.registerListener(new MembershipChangeListener { + def notify(members: SortedSet[Member]) { + // we don't expected any changes to the cluster + unexpected.set(members) + } + }) + + for (n ← 1 to 30) { + enterBarrier("period-" + n) + unexpected.get must be(null) + awaitUpConvergence(roles.size) + assertLeaderIn(roles) + if (n % 5 == 0) log.info("Passed period [{}]", n) + 1.seconds.sleep + } + + enterBarrier("after") + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/TransitionSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/TransitionSpec.scala new file mode 100644 index 0000000000..c4e43b9abf --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/TransitionSpec.scala @@ -0,0 +1,261 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import com.typesafe.config.ConfigFactory +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import akka.actor.Address +import akka.remote.testconductor.RoleName +import MemberStatus._ + +object TransitionMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig(debugConfig(on = false). + withFallback(ConfigFactory.parseString("akka.cluster.periodic-tasks-initial-delay = 300 s # turn off all periodic tasks")). + withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class TransitionMultiJvmNode1 extends TransitionSpec with FailureDetectorPuppetStrategy +class TransitionMultiJvmNode2 extends TransitionSpec with FailureDetectorPuppetStrategy +class TransitionMultiJvmNode3 extends TransitionSpec with FailureDetectorPuppetStrategy + +abstract class TransitionSpec + extends MultiNodeSpec(TransitionMultiJvmSpec) + with MultiNodeClusterSpec { + + import TransitionMultiJvmSpec._ + + // sorted in the order used by the cluster + def leader(roles: RoleName*) = roles.sorted.head + def nonLeader(roles: RoleName*) = roles.toSeq.sorted.tail + + def memberStatus(address: Address): MemberStatus = { + val statusOption = (cluster.latestGossip.members ++ cluster.latestGossip.overview.unreachable).collectFirst { + case m if m.address == address ⇒ m.status + } + statusOption must not be (None) + statusOption.get + } + + def memberAddresses: Set[Address] = cluster.latestGossip.members.map(_.address) + + def members: Set[RoleName] = memberAddresses.flatMap(roleName(_)) + + def seenLatestGossip: Set[RoleName] = { + val gossip = cluster.latestGossip + gossip.overview.seen.collect { + case (address, v) if v == gossip.version ⇒ roleName(address) + }.flatten.toSet + } + + def awaitSeen(addresses: Address*): Unit = awaitCond { + (seenLatestGossip map address) == addresses.toSet + } + + def awaitMembers(addresses: Address*): Unit = awaitCond { + memberAddresses == addresses.toSet + } + + def awaitMemberStatus(address: Address, status: MemberStatus): Unit = awaitCond { + memberStatus(address) == status + } + + // DSL sugar for `role1 gossipTo role2` + implicit def roleExtras(role: RoleName): RoleWrapper = new RoleWrapper(role) + var gossipBarrierCounter = 0 + class RoleWrapper(fromRole: RoleName) { + def gossipTo(toRole: RoleName): Unit = { + gossipBarrierCounter += 1 + runOn(toRole) { + val g = cluster.latestGossip + enterBarrier("before-gossip-" + gossipBarrierCounter) + awaitCond(cluster.latestGossip != g) // received gossip + // gossip chat will synchronize the views + awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty) + enterBarrier("after-gossip-" + gossipBarrierCounter) + } + runOn(fromRole) { + enterBarrier("before-gossip-" + gossipBarrierCounter) + cluster.gossipTo(toRole) // send gossip + // gossip chat will synchronize the views + awaitCond((Set(fromRole, toRole) -- seenLatestGossip).isEmpty) + enterBarrier("after-gossip-" + gossipBarrierCounter) + } + runOn(roles.filterNot(r ⇒ r == fromRole || r == toRole): _*) { + enterBarrier("before-gossip-" + gossipBarrierCounter) + enterBarrier("after-gossip-" + gossipBarrierCounter) + } + } + } + + "A Cluster" must { + + "start nodes as singleton clusters" taggedAs LongRunningTest in { + + runOn(first) { + startClusterNode() + cluster.isSingletonCluster must be(true) + cluster.status must be(Joining) + cluster.convergence.isDefined must be(true) + cluster.leaderActions() + cluster.status must be(Up) + } + + enterBarrier("after-1") + } + + "perform correct transitions when second joining first" taggedAs LongRunningTest in { + + runOn(second) { + cluster.join(first) + } + runOn(first, second) { + // gossip chat from the join will synchronize the views + awaitMembers(first, second) + memberStatus(first) must be(Up) + memberStatus(second) must be(Joining) + awaitCond(seenLatestGossip == Set(first, second)) + cluster.convergence.isDefined must be(true) + } + enterBarrier("convergence-joining-2") + + runOn(leader(first, second)) { + cluster.leaderActions() + memberStatus(first) must be(Up) + memberStatus(second) must be(Up) + } + enterBarrier("leader-actions-2") + + leader(first, second) gossipTo nonLeader(first, second).head + runOn(first, second) { + // gossip chat will synchronize the views + awaitCond(memberStatus(second) == Up) + seenLatestGossip must be(Set(first, second)) + memberStatus(first) must be(Up) + cluster.convergence.isDefined must be(true) + } + + enterBarrier("after-2") + } + + "perform correct transitions when third joins second" taggedAs LongRunningTest in { + + runOn(third) { + cluster.join(second) + } + runOn(second, third) { + // gossip chat from the join will synchronize the views + awaitMembers(first, second, third) + memberStatus(third) must be(Joining) + awaitCond(seenLatestGossip == Set(second, third)) + cluster.convergence.isDefined must be(false) + } + enterBarrier("third-joined-second") + + second gossipTo first + runOn(first, second) { + // gossip chat will synchronize the views + awaitMembers(first, second, third) + memberStatus(third) must be(Joining) + awaitCond(memberStatus(second) == Up) + seenLatestGossip must be(Set(first, second, third)) + cluster.convergence.isDefined must be(true) + } + + first gossipTo third + runOn(first, second, third) { + members must be(Set(first, second, third)) + memberStatus(first) must be(Up) + memberStatus(second) must be(Up) + memberStatus(third) must be(Joining) + seenLatestGossip must be(Set(first, second, third)) + cluster.convergence.isDefined must be(true) + } + + enterBarrier("convergence-joining-3") + + runOn(leader(first, second, third)) { + cluster.leaderActions() + memberStatus(first) must be(Up) + memberStatus(second) must be(Up) + memberStatus(third) must be(Up) + } + enterBarrier("leader-actions-3") + + // leader gossipTo first non-leader + leader(first, second, third) gossipTo nonLeader(first, second, third).head + runOn(nonLeader(first, second, third).head) { + memberStatus(third) must be(Up) + seenLatestGossip must be(Set(leader(first, second, third), myself)) + cluster.convergence.isDefined must be(false) + } + + // first non-leader gossipTo the other non-leader + nonLeader(first, second, third).head gossipTo nonLeader(first, second, third).tail.head + runOn(nonLeader(first, second, third).head) { + cluster.gossipTo(nonLeader(first, second, third).tail.head) + } + runOn(nonLeader(first, second, third).tail.head) { + memberStatus(third) must be(Up) + seenLatestGossip must be(Set(first, second, third)) + cluster.convergence.isDefined must be(true) + } + + // first non-leader gossipTo the leader + nonLeader(first, second, third).head gossipTo leader(first, second, third) + runOn(first, second, third) { + memberStatus(first) must be(Up) + memberStatus(second) must be(Up) + memberStatus(third) must be(Up) + seenLatestGossip must be(Set(first, second, third)) + cluster.convergence.isDefined must be(true) + } + + enterBarrier("after-3") + } + + "perform correct transitions when second becomes unavailble" taggedAs LongRunningTest in { + runOn(third) { + markNodeAsUnavailable(second) + cluster.reapUnreachableMembers() + cluster.latestGossip.overview.unreachable must contain(Member(second, Up)) + seenLatestGossip must be(Set(third)) + } + + enterBarrier("after-second-unavailble") + + third gossipTo first + + runOn(first, third) { + cluster.latestGossip.overview.unreachable must contain(Member(second, Up)) + cluster.convergence.isDefined must be(false) + } + + runOn(first) { + cluster.down(second) + awaitMemberStatus(second, Down) + } + + enterBarrier("after-second-down") + + first gossipTo third + + runOn(first, third) { + cluster.latestGossip.overview.unreachable must contain(Member(second, Down)) + memberStatus(second) must be(Down) + seenLatestGossip must be(Set(first, third)) + cluster.convergence.isDefined must be(true) + } + + enterBarrier("after-6") + } + + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeRejoinsClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeRejoinsClusterSpec.scala new file mode 100644 index 0000000000..34f8605af1 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeRejoinsClusterSpec.scala @@ -0,0 +1,137 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster + +import org.scalatest.BeforeAndAfter +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ +import com.typesafe.config.ConfigFactory +import akka.actor.Address +import akka.remote.testconductor.{RoleName, Direction} +import akka.util.duration._ + +object UnreachableNodeRejoinsClusterMultiJvmSpec extends MultiNodeConfig { + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + + commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig)) +} + +class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy +class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy +class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy +class UnreachableNodeRejoinsClusterWithFailureDetectorPuppetMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with FailureDetectorPuppetStrategy + + +class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode1 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy +class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode2 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy +class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode3 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy +class UnreachableNodeRejoinsClusterWithAccrualFailureDetectorMultiJvmNode4 extends UnreachableNodeRejoinsClusterSpec with AccrualFailureDetectorStrategy + +abstract class UnreachableNodeRejoinsClusterSpec + extends MultiNodeSpec(UnreachableNodeRejoinsClusterMultiJvmSpec) + with MultiNodeClusterSpec { + import UnreachableNodeRejoinsClusterMultiJvmSpec._ + + def allBut(role: RoleName, roles: Seq[RoleName] = roles): Seq[RoleName] = { + roles.filterNot(_ == role) + } + + + lazy val sortedRoles = roles.sorted + lazy val master = sortedRoles(0) + lazy val victim = sortedRoles(1) + + var endBarrierNumber = 0 + def endBarrier: Unit = { + endBarrierNumber += 1 + enterBarrier("after_" + endBarrierNumber) + } + + "A cluster of " + roles.size + " members" must { + + "reach initial convergence" taggedAs LongRunningTest in { + awaitClusterUp(roles:_*) + endBarrier + } + + "mark a node as UNREACHABLE when we pull the network" taggedAs LongRunningTest in { + runOn(first) { + // pull network for victim node from all nodes + allBut(victim).foreach { roleName => + testConductor.blackhole(victim, roleName, Direction.Both).await + } + } + + enterBarrier("unplug_victim") + + val allButVictim = allBut(victim, sortedRoles) + runOn(victim) { + allButVictim.foreach(markNodeAsUnavailable(_)) + within(30 seconds) { + // victim becomes all alone + awaitCond({ val gossip = cluster.latestGossip + gossip.overview.unreachable.size == (roles.size - 1) && + gossip.members.size == 1 && + gossip.members.forall(_.status == MemberStatus.Up) }) + cluster.latestGossip.overview.unreachable.map(_.address) must be((allButVictim map address).toSet) + cluster.convergence.isDefined must be(false) + } + } + + runOn(allButVictim:_*) { + markNodeAsUnavailable(victim) + within(30 seconds) { + // victim becomes unreachable + awaitCond({ val gossip = cluster.latestGossip + gossip.overview.unreachable.size == 1 && + gossip.members.size == (roles.size - 1) && + gossip.members.forall(_.status == MemberStatus.Up) }) + awaitSeenSameState(allButVictim map address:_*) + // still one unreachable + cluster.latestGossip.overview.unreachable.size must be(1) + cluster.latestGossip.overview.unreachable.head.address must be(node(victim).address) + // and therefore no convergence + cluster.convergence.isDefined must be(false) + } + } + + endBarrier + } + + "mark the node as DOWN" taggedAs LongRunningTest in { + runOn(master) { + cluster down victim + } + + runOn(allBut(victim):_*) { + awaitUpConvergence(roles.size - 1, Seq(victim)) + } + + endBarrier + } + + "allow node to REJOIN when the network is plugged back in" taggedAs LongRunningTest in { + runOn(first) { + // put the network back in + allBut(victim).foreach { roleName => + testConductor.passThrough(victim, roleName, Direction.Both).await + } + } + + enterBarrier("plug_in_victim") + + runOn(victim) { + cluster join master + } + + awaitUpConvergence(roles.size) + + endBarrier + } + } +} diff --git a/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala index 173ce799f8..df69a52e19 100644 --- a/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala @@ -6,6 +6,9 @@ package akka.cluster import akka.actor.Address import akka.testkit.{ LongRunningTest, AkkaSpec } +import scala.collection.immutable.TreeMap +import akka.util.duration._ +import akka.util.Duration @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class AccrualFailureDetectorSpec extends AkkaSpec(""" @@ -17,7 +20,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec(""" val conn = Address("akka", "", "localhost", 2552) val conn2 = Address("akka", "", "localhost", 2553) - def fakeTimeGenerator(timeIntervals: List[Long]): () ⇒ Long = { + def fakeTimeGenerator(timeIntervals: Seq[Long]): () ⇒ Long = { var times = timeIntervals.tail.foldLeft(List[Long](timeIntervals.head))((acc, c) ⇒ acc ::: List[Long](acc.last + c)) def timeGenerator(): Long = { val currentTime = times.head @@ -27,120 +30,186 @@ class AccrualFailureDetectorSpec extends AkkaSpec(""" timeGenerator } - "return phi value of 0.0D on startup for each address" in { - val fd = new AccrualFailureDetector(system, conn) - fd.phi(conn) must be(0.0D) - fd.phi(conn2) must be(0.0D) + val defaultFakeTimeIntervals = Vector.fill(20)(1000L) + def createFailureDetector( + threshold: Double = 8.0, + maxSampleSize: Int = 1000, + minStdDeviation: Duration = 10.millis, + acceptableLostDuration: Duration = Duration.Zero, + firstHeartbeatEstimate: Duration = 1.second, + clock: () ⇒ Long = fakeTimeGenerator(defaultFakeTimeIntervals)): AccrualFailureDetector = + new AccrualFailureDetector(system, + threshold, + maxSampleSize, + minStdDeviation, + acceptableLostDuration, + firstHeartbeatEstimate = firstHeartbeatEstimate, + clock = clock) + + "use good enough cumulative distribution function" in { + val fd = createFailureDetector() + fd.cumulativeDistributionFunction(0.0, 0, 1) must be(0.5 plusOrMinus (0.001)) + fd.cumulativeDistributionFunction(0.6, 0, 1) must be(0.7257 plusOrMinus (0.001)) + fd.cumulativeDistributionFunction(1.5, 0, 1) must be(0.9332 plusOrMinus (0.001)) + fd.cumulativeDistributionFunction(2.0, 0, 1) must be(0.97725 plusOrMinus (0.01)) + fd.cumulativeDistributionFunction(2.5, 0, 1) must be(0.9379 plusOrMinus (0.1)) + fd.cumulativeDistributionFunction(3.5, 0, 1) must be(0.99977 plusOrMinus (0.1)) + fd.cumulativeDistributionFunction(4.0, 0, 1) must be(0.99997 plusOrMinus (0.1)) + + for (x :: y :: Nil ← (0.0 to 4.0 by 0.1).toList.sliding(2)) { + fd.cumulativeDistributionFunction(x, 0, 1) must be < ( + fd.cumulativeDistributionFunction(y, 0, 1)) + } + + fd.cumulativeDistributionFunction(2.2, 2.0, 0.3) must be(0.7475 plusOrMinus (0.001)) + } + + "return realistic phi values" in { + val fd = createFailureDetector() + val test = TreeMap(0 -> 0.0, 500 -> 0.1, 1000 -> 0.3, 1200 -> 1.6, 1400 -> 4.7, 1600 -> 10.8, 1700 -> 15.3) + for ((timeDiff, expectedPhi) ← test) { + fd.phi(timeDiff = timeDiff, mean = 1000.0, stdDeviation = 100.0) must be(expectedPhi plusOrMinus (0.1)) + } + + // larger stdDeviation results => lower phi + fd.phi(timeDiff = 1100, mean = 1000.0, stdDeviation = 500.0) must be < ( + fd.phi(timeDiff = 1100, mean = 1000.0, stdDeviation = 100.0)) + } + + "return phi value of 0.0 on startup for each address, when no heartbeats" in { + val fd = createFailureDetector() + fd.phi(conn) must be(0.0) + fd.phi(conn2) must be(0.0) + } + + "return phi based on guess when only one heartbeat" in { + val timeInterval = List[Long](0, 1000, 1000, 1000, 1000) + val fd = createFailureDetector(firstHeartbeatEstimate = 1.seconds, + clock = fakeTimeGenerator(timeInterval)) + + fd.heartbeat(conn) + fd.phi(conn) must be(0.3 plusOrMinus 0.2) + fd.phi(conn) must be(4.5 plusOrMinus 0.3) + fd.phi(conn) must be > (15.0) + } + + "return phi value using first interval after second heartbeat" in { + val timeInterval = List[Long](0, 100, 100, 100) + val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval)) + + fd.heartbeat(conn) + fd.phi(conn) must be > (0.0) + fd.heartbeat(conn) + fd.phi(conn) must be > (0.0) } "mark node as available after a series of successful heartbeats" in { val timeInterval = List[Long](0, 1000, 100, 100) - val ft = fakeTimeGenerator(timeInterval) - - val fd = new AccrualFailureDetector(system, conn, timeMachine = ft) + val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval)) fd.heartbeat(conn) - fd.heartbeat(conn) - fd.heartbeat(conn) fd.isAvailable(conn) must be(true) } - "mark node as dead after explicit removal of connection" in { + "mark node as available after explicit removal of connection" in { val timeInterval = List[Long](0, 1000, 100, 100, 100) - val ft = fakeTimeGenerator(timeInterval) - - val fd = new AccrualFailureDetector(system, conn, timeMachine = ft) + val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval)) fd.heartbeat(conn) - fd.heartbeat(conn) - fd.heartbeat(conn) - fd.isAvailable(conn) must be(true) - fd.remove(conn) - fd.isAvailable(conn) must be(false) + fd.isAvailable(conn) must be(true) } "mark node as available after explicit removal of connection and receiving heartbeat again" in { val timeInterval = List[Long](0, 1000, 100, 1100, 1100, 1100, 1100, 1100, 100) - val ft = fakeTimeGenerator(timeInterval) - - val fd = new AccrualFailureDetector(system, conn, timeMachine = ft) + val fd = createFailureDetector(clock = fakeTimeGenerator(timeInterval)) fd.heartbeat(conn) //0 fd.heartbeat(conn) //1000 - fd.heartbeat(conn) //1100 fd.isAvailable(conn) must be(true) //2200 fd.remove(conn) - fd.isAvailable(conn) must be(false) //3300 + fd.isAvailable(conn) must be(true) //3300 // it receives heartbeat from an explicitly removed node fd.heartbeat(conn) //4400 - fd.heartbeat(conn) //5500 - fd.heartbeat(conn) //6600 fd.isAvailable(conn) must be(true) //6700 } "mark node as dead if heartbeat are missed" in { - val timeInterval = List[Long](0, 1000, 100, 100, 5000) + val timeInterval = List[Long](0, 1000, 100, 100, 7000) val ft = fakeTimeGenerator(timeInterval) - - val fd = new AccrualFailureDetector(system, conn, threshold = 3, timeMachine = ft) + val fd = createFailureDetector(threshold = 3, clock = fakeTimeGenerator(timeInterval)) fd.heartbeat(conn) //0 - fd.heartbeat(conn) //1000 - fd.heartbeat(conn) //1100 fd.isAvailable(conn) must be(true) //1200 - - fd.isAvailable(conn) must be(false) //6200 + fd.isAvailable(conn) must be(false) //8200 } "mark node as available if it starts heartbeat again after being marked dead due to detection of failure" in { - val timeInterval = List[Long](0, 1000, 100, 1100, 5000, 100, 1000, 100, 100) - val ft = fakeTimeGenerator(timeInterval) - - val fd = new AccrualFailureDetector(system, conn, threshold = 3, timeMachine = ft) + val timeInterval = List[Long](0, 1000, 100, 1100, 7000, 100, 1000, 100, 100) + val fd = createFailureDetector(threshold = 3, clock = fakeTimeGenerator(timeInterval)) fd.heartbeat(conn) //0 - fd.heartbeat(conn) //1000 - fd.heartbeat(conn) //1100 - fd.isAvailable(conn) must be(true) //1200 + fd.isAvailable(conn) must be(false) //8200 + fd.heartbeat(conn) //8300 + fd.heartbeat(conn) //9300 + fd.heartbeat(conn) //9400 - fd.isAvailable(conn) must be(false) //6200 + fd.isAvailable(conn) must be(true) //9500 + } - fd.heartbeat(conn) //6300 + "accept some configured missing heartbeats" in { + val timeInterval = List[Long](0, 1000, 1000, 1000, 4000, 1000, 1000) + val fd = createFailureDetector(acceptableLostDuration = 3.seconds, clock = fakeTimeGenerator(timeInterval)) - fd.heartbeat(conn) //7300 + fd.heartbeat(conn) + fd.heartbeat(conn) + fd.heartbeat(conn) + fd.heartbeat(conn) + fd.isAvailable(conn) must be(true) + fd.heartbeat(conn) + fd.isAvailable(conn) must be(true) + } - fd.heartbeat(conn) //7400 + "fail after configured acceptable missing heartbeats" in { + val timeInterval = List[Long](0, 1000, 1000, 1000, 1000, 1000, 500, 500, 5000) + val fd = createFailureDetector(acceptableLostDuration = 3.seconds, clock = fakeTimeGenerator(timeInterval)) - fd.isAvailable(conn) must be(true) //7500 + fd.heartbeat(conn) + fd.heartbeat(conn) + fd.heartbeat(conn) + fd.heartbeat(conn) + fd.heartbeat(conn) + fd.heartbeat(conn) + fd.isAvailable(conn) must be(true) + fd.heartbeat(conn) + fd.isAvailable(conn) must be(false) } "use maxSampleSize heartbeats" in { val timeInterval = List[Long](0, 100, 100, 100, 100, 600, 1000, 1000, 1000, 1000, 1000) - val ft = fakeTimeGenerator(timeInterval) - val fd = new AccrualFailureDetector(system, conn, maxSampleSize = 3, timeMachine = ft) + val fd = createFailureDetector(maxSampleSize = 3, clock = fakeTimeGenerator(timeInterval)) // 100 ms interval fd.heartbeat(conn) //0 @@ -156,5 +225,35 @@ class AccrualFailureDetectorSpec extends AkkaSpec(""" val phi2 = fd.phi(conn) //5000 phi2 must be(phi1.plusOrMinus(0.001)) } + + } + + "Statistics for heartbeats" must { + + "calculate correct mean and variance" in { + val samples = Seq(100, 200, 125, 340, 130) + val stats = (HeartbeatHistory(maxSampleSize = 20) /: samples) { (stats, value) ⇒ stats :+ value } + stats.mean must be(179.0 plusOrMinus 0.00001) + stats.variance must be(7584.0 plusOrMinus 0.00001) + } + + "have 0.0 variance for one sample" in { + (HeartbeatHistory(600) :+ 1000L).variance must be(0.0 plusOrMinus 0.00001) + } + + "be capped by the specified maxSampleSize" in { + val history3 = HeartbeatHistory(maxSampleSize = 3) :+ 100 :+ 110 :+ 90 + history3.mean must be(100.0 plusOrMinus 0.00001) + history3.variance must be(66.6666667 plusOrMinus 0.00001) + + val history4 = history3 :+ 140 + history4.mean must be(113.333333 plusOrMinus 0.00001) + history4.variance must be(422.222222 plusOrMinus 0.00001) + + val history5 = history4 :+ 80 + history5.mean must be(103.333333 plusOrMinus 0.00001) + history5.variance must be(688.88888889 plusOrMinus 0.00001) + + } } } diff --git a/akka-cluster/src/test/scala/akka/cluster/ClientDowningSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClientDowningSpec.scala deleted file mode 100644 index 0e7b0ed330..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/ClientDowningSpec.scala +++ /dev/null @@ -1,145 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.cluster - -import akka.testkit._ -import akka.dispatch._ -import akka.actor._ -import akka.remote._ -import akka.util.duration._ - -import com.typesafe.config._ - -import java.net.InetSocketAddress - -class ClientDowningSpec extends ClusterSpec("akka.cluster.auto-down = off") with ImplicitSender { - val portPrefix = 1 - - var node1: Cluster = _ - var node2: Cluster = _ - var node3: Cluster = _ - var node4: Cluster = _ - - var system1: ActorSystemImpl = _ - var system2: ActorSystemImpl = _ - var system3: ActorSystemImpl = _ - var system4: ActorSystemImpl = _ - - try { - "Client of a 4 node cluster" must { - - // ======= NODE 1 ======== - system1 = ActorSystem("system1", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d550 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote1 = system1.provider.asInstanceOf[RemoteActorRefProvider] - node1 = Cluster(system1) - val fd1 = node1.failureDetector - val address1 = node1.remoteAddress - - // ======= NODE 2 ======== - system2 = ActorSystem("system2", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d551 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote2 = system2.provider.asInstanceOf[RemoteActorRefProvider] - node2 = Cluster(system2) - val fd2 = node2.failureDetector - val address2 = node2.remoteAddress - - // ======= NODE 3 ======== - system3 = ActorSystem("system3", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d552 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote3 = system3.provider.asInstanceOf[RemoteActorRefProvider] - node3 = Cluster(system3) - val fd3 = node3.failureDetector - val address3 = node3.remoteAddress - - // ======= NODE 4 ======== - system4 = ActorSystem("system4", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d553 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote4 = system4.provider.asInstanceOf[RemoteActorRefProvider] - node4 = Cluster(system4) - val fd4 = node4.failureDetector - val address4 = node4.remoteAddress - - "be able to DOWN a node that is UP" taggedAs LongRunningTest in { - println("Give the system time to converge...") - awaitConvergence(node1 :: node2 :: node3 :: node4 :: Nil) - - node3.shutdown() - system3.shutdown() - - // client marks node3 as DOWN - node1.down(address3) - - println("Give the system time to converge...") - Thread.sleep(10.seconds.dilated.toMillis) - awaitConvergence(node1 :: node2 :: node4 :: Nil) - - node1.latestGossip.members.size must be(3) - node1.latestGossip.members.exists(_.address == address3) must be(false) - } - - "be able to DOWN a node that is UNREACHABLE" taggedAs LongRunningTest in { - node4.shutdown() - system4.shutdown() - - // clien marks node4 as DOWN - node2.down(address4) - - println("Give the system time to converge...") - Thread.sleep(10.seconds.dilated.toMillis) - awaitConvergence(node1 :: node2 :: Nil) - - node1.latestGossip.members.size must be(2) - node1.latestGossip.members.exists(_.address == address4) must be(false) - node1.latestGossip.members.exists(_.address == address3) must be(false) - } - } - } catch { - case e: Exception ⇒ - e.printStackTrace - fail(e.toString) - } - - override def atTermination() { - if (node1 ne null) node1.shutdown() - if (system1 ne null) system1.shutdown() - - if (node2 ne null) node2.shutdown() - if (system2 ne null) system2.shutdown() - - if (node3 ne null) node3.shutdown() - if (system3 ne null) system3.shutdown() - - if (node4 ne null) node4.shutdown() - if (system4 ne null) system4.shutdown() - } -} diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala index 45b0a35521..07671c6164 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala @@ -16,16 +16,25 @@ class ClusterConfigSpec extends AkkaSpec { "be able to parse generic cluster config elements" in { val settings = new ClusterSettings(system.settings.config, system.name) import settings._ - FailureDetectorThreshold must be(8) + FailureDetectorThreshold must be(8.0 plusOrMinus 0.0001) FailureDetectorMaxSampleSize must be(1000) - NodeToJoin must be(None) + FailureDetectorImplementationClass must be(classOf[AccrualFailureDetector].getName) + FailureDetectorMinStdDeviation must be(100 millis) + FailureDetectorAcceptableHeartbeatPause must be(3 seconds) + SeedNodes must be(Seq.empty[String]) + SeedNodeTimeout must be(5 seconds) PeriodicTasksInitialDelay must be(1 seconds) - GossipFrequency must be(1 second) - LeaderActionsFrequency must be(1 second) - UnreachableNodesReaperFrequency must be(1 second) + GossipInterval must be(1 second) + HeartbeatInterval must be(1 second) + LeaderActionsInterval must be(1 second) + UnreachableNodesReaperInterval must be(1 second) + JoinTimeout must be(60 seconds) NrOfGossipDaemons must be(4) - NrOfDeputyNodes must be(3) + AutoJoin must be(true) AutoDown must be(true) + GossipDifferentViewProbability must be(0.8 plusOrMinus 0.0001) + SchedulerTickDuration must be(33 millis) + SchedulerTicksPerWheel must be(512) } } } diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala index 38017ad00c..68731b89b2 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterSpec.scala @@ -1,69 +1,193 @@ /** - * Copyright (C) 2009-2012 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ + package akka.cluster -import akka.actor.ActorSystem -import akka.util._ -import akka.util.duration._ - import akka.testkit.AkkaSpec -import akka.testkit.TestEvent._ -import akka.testkit.EventFilter - -import com.typesafe.config.{ Config, ConfigFactory } +import akka.util.duration._ +import akka.util.Duration +import akka.actor.ExtendedActorSystem +import akka.actor.Address +import java.util.concurrent.atomic.AtomicInteger +import org.scalatest.BeforeAndAfter +import akka.remote.RemoteActorRefProvider object ClusterSpec { - val testConf: Config = ConfigFactory.parseString(""" - akka { - event-handlers = ["akka.testkit.TestEventListener"] - loglevel = "WARNING" - stdout-loglevel = "WARNING" - actor { - default-dispatcher { - executor = "fork-join-executor" - fork-join-executor { - parallelism-min = 8 - parallelism-factor = 2.0 - parallelism-max = 8 - } - } - } - remote.netty.hostname = localhost - cluster { - failure-detector.threshold = 3 - auto-down = on - } + val config = """ + akka.cluster { + auto-join = off + auto-down = off + periodic-tasks-initial-delay = 120 seconds // turn off scheduled tasks } - """) + akka.actor.provider = "akka.remote.RemoteActorRefProvider" + akka.remote.netty.port = 0 + # akka.loglevel = DEBUG + """ + + case class GossipTo(address: Address) } -abstract class ClusterSpec(_system: ActorSystem) extends AkkaSpec(_system) { - case class PortPrefix(port: Int) { - def withPortPrefix: Int = (portPrefix.toString + port.toString).toInt - } +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class ClusterSpec extends AkkaSpec(ClusterSpec.config) with BeforeAndAfter { + import ClusterSpec._ - implicit def intToPortPrefix(port: Int) = PortPrefix(port) + val selfAddress = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.address + val addresses = IndexedSeq( + selfAddress, + Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 1), + Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 2), + Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 3), + Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 4), + Address("akka", system.name, selfAddress.host.get, selfAddress.port.get + 5)) - def portPrefix: Int + val deterministicRandom = new AtomicInteger - def this(config: Config) = this(ActorSystem(AkkaSpec.getCallerName, config.withFallback(ClusterSpec.testConf))) + val failureDetector = new FailureDetectorPuppet(system) - def this(s: String) = this(ConfigFactory.parseString(s)) + val cluster = new Cluster(system.asInstanceOf[ExtendedActorSystem], failureDetector) { - def this() = this(ActorSystem(AkkaSpec.getCallerName, ClusterSpec.testConf)) + // 3 deputy nodes (addresses index 1, 2, 3) + override def seedNodes = addresses.slice(1, 4) - def awaitConvergence(nodes: Iterable[Cluster], maxWaitTime: Duration = 60 seconds) { - val deadline = maxWaitTime.fromNow - while (nodes map (_.convergence.isDefined) exists (_ == false)) { - if (deadline.isOverdue) throw new IllegalStateException("Convergence could no be reached within " + maxWaitTime) - Thread.sleep(1000) + override def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] = { + if (addresses.isEmpty) None + else Some(addresses.toSeq(deterministicRandom.getAndIncrement % addresses.size)) } - nodes foreach { n ⇒ println("Converged: " + n.self + " == " + n.convergence.isDefined) } + + override def gossipTo(address: Address): Unit = { + if (address == self.address) { + super.gossipTo(address) + } + // represent the gossip with a message to be used in asserts + testActor ! GossipTo(address) + } + + @volatile + var _gossipToDeputyProbablity = 0.0 + + override def gossipToDeputyProbablity(membersSize: Int, unreachableSize: Int, deputySize: Int): Double = { + if (_gossipToDeputyProbablity < 0.0) super.gossipToDeputyProbablity(membersSize, unreachableSize, deputySize) + else _gossipToDeputyProbablity + } + } - override def atStartup { - system.eventStream.publish(Mute(EventFilter[java.net.ConnectException]())) - system.eventStream.publish(Mute(EventFilter[java.nio.channels.ClosedChannelException]())) + def memberStatus(address: Address): Option[MemberStatus] = + cluster.latestGossip.members.collectFirst { case m if m.address == address ⇒ m.status } + + before { + cluster._gossipToDeputyProbablity = 0.0 + addresses foreach failureDetector.remove + deterministicRandom.set(0) + } + + "A Cluster" must { + + "use the address of the remote transport" in { + cluster.selfAddress must be(selfAddress) + } + + "initially become singleton cluster when joining itself and reach convergence" in { + cluster.isSingletonCluster must be(false) // auto-join = off + cluster.join(selfAddress) + awaitCond(cluster.isSingletonCluster) + cluster.self.address must be(selfAddress) + cluster.latestGossip.members.map(_.address) must be(Set(selfAddress)) + memberStatus(selfAddress) must be(Some(MemberStatus.Joining)) + cluster.convergence.isDefined must be(true) + cluster.leaderActions() + memberStatus(selfAddress) must be(Some(MemberStatus.Up)) + } + + "accept a joining node" in { + cluster.joining(addresses(1)) + cluster.latestGossip.members.map(_.address) must be(Set(selfAddress, addresses(1))) + memberStatus(addresses(1)) must be(Some(MemberStatus.Joining)) + cluster.convergence.isDefined must be(false) + expectMsg(GossipTo(addresses(1))) + } + + "accept a few more joining nodes" in { + for (a ← addresses.drop(2)) { + cluster.joining(a) + memberStatus(a) must be(Some(MemberStatus.Joining)) + expectMsg(GossipTo(a)) + } + cluster.latestGossip.members.map(_.address) must be(addresses.toSet) + } + + "order members by host and port" in { + // note the importance of using toSeq before map, otherwise it will not preserve the order + cluster.latestGossip.members.toSeq.map(_.address) must be(addresses.toSeq) + } + + "gossip to random live node" in { + cluster.gossip() + cluster.gossip() + cluster.gossip() + cluster.gossip() + + expectMsg(GossipTo(addresses(1))) + expectMsg(GossipTo(addresses(2))) + expectMsg(GossipTo(addresses(3))) + expectMsg(GossipTo(addresses(4))) + + expectNoMsg(1 second) + } + + "use certain probability for gossiping to deputy node depending on the number of unreachable and live nodes" in { + cluster._gossipToDeputyProbablity = -1.0 // use real impl + cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(9, 1, 2)) + cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 2)) + cluster.gossipToDeputyProbablity(10, 1, 2) must be < (cluster.gossipToDeputyProbablity(10, 2, 3)) + cluster.gossipToDeputyProbablity(10, 5, 5) must be < (cluster.gossipToDeputyProbablity(10, 9, 5)) + cluster.gossipToDeputyProbablity(0, 10, 0) must be <= (1.0) + cluster.gossipToDeputyProbablity(1, 10, 1) must be <= (1.0) + cluster.gossipToDeputyProbablity(10, 0, 0) must be(0.0 plusOrMinus (0.0001)) + cluster.gossipToDeputyProbablity(0, 0, 0) must be(0.0 plusOrMinus (0.0001)) + cluster.gossipToDeputyProbablity(4, 0, 4) must be(1.0 plusOrMinus (0.0001)) + cluster.gossipToDeputyProbablity(3, 7, 4) must be(1.0 plusOrMinus (0.0001)) + } + + "gossip to duputy node" in { + cluster._gossipToDeputyProbablity = 1.0 // always + + // we have configured 3 deputy nodes (seedNodes) + cluster.gossip() // 1 is deputy + cluster.gossip() // 2 is deputy + cluster.gossip() // 3 is deputy + cluster.gossip() // 4 is not deputy, and therefore a deputy is also used + + expectMsg(GossipTo(addresses(1))) + expectMsg(GossipTo(addresses(2))) + expectMsg(GossipTo(addresses(3))) + expectMsg(GossipTo(addresses(4))) + // and the extra gossip to deputy + expectMsgAnyOf(GossipTo(addresses(1)), GossipTo(addresses(2)), GossipTo(addresses(3))) + + expectNoMsg(1 second) + + } + + "gossip to random deputy node if number of live nodes is less than number of deputy nodes" in { + cluster._gossipToDeputyProbablity = -1.0 // real impl + // 0 and 2 still alive + val dead = Set(addresses(1), addresses(3), addresses(4), addresses(5)) + dead foreach failureDetector.markNodeAsUnavailable + + cluster.reapUnreachableMembers() + cluster.latestGossip.overview.unreachable.map(_.address) must be(dead) + + for (n ← 1 to 20) { + cluster.gossip() + expectMsg(GossipTo(addresses(2))) // the only available + // and always to one of the 3 deputies + expectMsgAnyOf(GossipTo(addresses(1)), GossipTo(addresses(2)), GossipTo(addresses(3))) + } + + expectNoMsg(1 second) + + } } } diff --git a/akka-cluster/src/test/scala/akka/cluster/FailureDetectorPuppet.scala b/akka-cluster/src/test/scala/akka/cluster/FailureDetectorPuppet.scala new file mode 100644 index 0000000000..9ddc9942b0 --- /dev/null +++ b/akka-cluster/src/test/scala/akka/cluster/FailureDetectorPuppet.scala @@ -0,0 +1,65 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import akka.actor.{ Address, ActorSystem } +import akka.event.{ Logging, LogSource } + +/** + * User controllable "puppet" failure detector. + */ +class FailureDetectorPuppet(system: ActorSystem, settings: ClusterSettings) extends FailureDetector { + import java.util.concurrent.ConcurrentHashMap + + def this(system: ActorSystem) = this(system, new ClusterSettings(system.settings.config, system.name)) + + trait Status + object Up extends Status + object Down extends Status + + implicit private val logSource: LogSource[AnyRef] = new LogSource[AnyRef] { + def genString(o: AnyRef): String = o.getClass.getName + override def getClazz(o: AnyRef): Class[_] = o.getClass + } + + private val log = Logging(system, this) + + private val connections = new ConcurrentHashMap[Address, Status] + + def markNodeAsUnavailable(connection: Address): this.type = { + connections.put(connection, Down) + this + } + + def markNodeAsAvailable(connection: Address): this.type = { + connections.put(connection, Up) + this + } + + def isAvailable(connection: Address): Boolean = connections.get(connection) match { + case null ⇒ + log.debug("Adding cluster node [{}]", connection) + connections.put(connection, Up) + true + case Up ⇒ + log.debug("isAvailable: Cluster node IS NOT available [{}]", connection) + true + case Down ⇒ + log.debug("isAvailable: Cluster node IS available [{}]", connection) + false + } + + def heartbeat(connection: Address): Unit = log.debug("Heart beat from cluster node[{}]", connection) + + def remove(connection: Address): Unit = { + log.debug("Removing cluster node [{}]", connection) + connections.remove(connection) + } + + def reset(): Unit = { + log.debug("Resetting failure detector") + connections.clear() + } +} diff --git a/akka-cluster/src/test/scala/akka/cluster/FixedRateTaskSpec.scala b/akka-cluster/src/test/scala/akka/cluster/FixedRateTaskSpec.scala new file mode 100644 index 0000000000..d259a5310b --- /dev/null +++ b/akka-cluster/src/test/scala/akka/cluster/FixedRateTaskSpec.scala @@ -0,0 +1,43 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import akka.testkit.AkkaSpec +import akka.util.duration._ +import akka.testkit.TimingTest +import akka.testkit.TestLatch +import akka.dispatch.Await + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class FixedRateTaskSpec extends AkkaSpec { + + "Task scheduled at fixed rate" must { + "adjust for scheduler inaccuracy" taggedAs TimingTest in { + val startTime = System.nanoTime + val n = 33 + val latch = new TestLatch(n) + FixedRateTask(system.scheduler, 150.millis, 150.millis) { + latch.countDown() + } + Await.ready(latch, 6.seconds) + val rate = n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis + rate must be(6.66 plusOrMinus (0.4)) + } + + "compensate for long running task" taggedAs TimingTest in { + val startTime = System.nanoTime + val n = 22 + val latch = new TestLatch(n) + FixedRateTask(system.scheduler, 225.millis, 225.millis) { + 80.millis.sleep() + latch.countDown() + } + Await.ready(latch, 6.seconds) + val rate = n * 1000.0 / (System.nanoTime - startTime).nanos.toMillis + rate must be(4.4 plusOrMinus (0.3)) + } + } +} + diff --git a/akka-cluster/src/test/scala/akka/cluster/GossipSpec.scala b/akka-cluster/src/test/scala/akka/cluster/GossipSpec.scala new file mode 100644 index 0000000000..8020010655 --- /dev/null +++ b/akka-cluster/src/test/scala/akka/cluster/GossipSpec.scala @@ -0,0 +1,104 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import org.scalatest.WordSpec +import org.scalatest.matchers.MustMatchers +import akka.actor.Address +import scala.collection.immutable.SortedSet + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class GossipSpec extends WordSpec with MustMatchers { + + import MemberStatus._ + + val a1 = Member(Address("akka", "sys", "a", 2552), Up) + val a2 = Member(Address("akka", "sys", "a", 2552), Joining) + val b1 = Member(Address("akka", "sys", "b", 2552), Up) + val b2 = Member(Address("akka", "sys", "b", 2552), Removed) + val c1 = Member(Address("akka", "sys", "c", 2552), Leaving) + val c2 = Member(Address("akka", "sys", "c", 2552), Up) + val d1 = Member(Address("akka", "sys", "d", 2552), Leaving) + val d2 = Member(Address("akka", "sys", "d", 2552), Removed) + val e1 = Member(Address("akka", "sys", "e", 2552), Joining) + val e2 = Member(Address("akka", "sys", "e", 2552), Up) + + "A Gossip" must { + + "merge members by status priority" in { + + val g1 = Gossip(members = SortedSet(a1, c1, e1)) + val g2 = Gossip(members = SortedSet(a2, c2, e2)) + + val merged1 = g1 merge g2 + merged1.members must be(SortedSet(a2, c1, e1)) + merged1.members.toSeq.map(_.status) must be(Seq(Joining, Leaving, Joining)) + + val merged2 = g2 merge g1 + merged2.members must be(SortedSet(a2, c1, e1)) + merged2.members.toSeq.map(_.status) must be(Seq(Joining, Leaving, Joining)) + + } + + "merge unreachable by status priority" in { + + val g1 = Gossip(members = Gossip.emptyMembers, overview = GossipOverview(unreachable = Set(a1, b1, c1, d1))) + val g2 = Gossip(members = Gossip.emptyMembers, overview = GossipOverview(unreachable = Set(a2, b2, c2, d2))) + + val merged1 = g1 merge g2 + merged1.overview.unreachable must be(Set(a2, b2, c1, d2)) + merged1.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Joining, Removed, Leaving, Removed)) + + val merged2 = g2 merge g1 + merged2.overview.unreachable must be(Set(a2, b2, c1, d2)) + merged2.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Joining, Removed, Leaving, Removed)) + + } + + "merge by excluding unreachable from members" in { + val g1 = Gossip(members = SortedSet(a1, b1), overview = GossipOverview(unreachable = Set(c1, d1))) + val g2 = Gossip(members = SortedSet(a2, c2), overview = GossipOverview(unreachable = Set(b2, d2))) + + val merged1 = g1 merge g2 + merged1.members must be(SortedSet(a2)) + merged1.members.toSeq.map(_.status) must be(Seq(Joining)) + merged1.overview.unreachable must be(Set(b2, c1, d2)) + merged1.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Removed, Leaving, Removed)) + + val merged2 = g2 merge g1 + merged2.members must be(SortedSet(a2)) + merged2.members.toSeq.map(_.status) must be(Seq(Joining)) + merged2.overview.unreachable must be(Set(b2, c1, d2)) + merged2.overview.unreachable.toSeq.sorted.map(_.status) must be(Seq(Removed, Leaving, Removed)) + + } + + "start with fresh seen table after merge" in { + val g1 = Gossip(members = SortedSet(a1, e1)).seen(a1.address).seen(a1.address) + val g2 = Gossip(members = SortedSet(a2, e2)).seen(e2.address).seen(e2.address) + + val merged1 = g1 merge g2 + merged1.overview.seen.isEmpty must be(true) + + val merged2 = g2 merge g1 + merged2.overview.seen.isEmpty must be(true) + + } + + "not have node in both members and unreachable" in intercept[IllegalArgumentException] { + Gossip(members = SortedSet(a1, b1), overview = GossipOverview(unreachable = Set(b2))) + } + + "not have live members with wrong status" in intercept[IllegalArgumentException] { + // b2 is Removed + Gossip(members = SortedSet(a2, b2)) + } + + "not have non cluster members in seen table" in intercept[IllegalArgumentException] { + Gossip(members = SortedSet(a1, e1)).seen(a1.address).seen(e1.address).seen(b1.address) + } + + } +} diff --git a/akka-cluster/src/test/scala/akka/cluster/GossipingAccrualFailureDetectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/GossipingAccrualFailureDetectorSpec.scala deleted file mode 100644 index 029b0b221d..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/GossipingAccrualFailureDetectorSpec.scala +++ /dev/null @@ -1,113 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ -package akka.cluster - -import akka.testkit._ -import akka.dispatch._ -import akka.actor._ -import akka.remote._ -import akka.util.duration._ - -import com.typesafe.config._ - -import java.net.InetSocketAddress - -class GossipingAccrualFailureDetectorSpec extends ClusterSpec with ImplicitSender { - val portPrefix = 2 - - var node1: Cluster = _ - var node2: Cluster = _ - var node3: Cluster = _ - - var system1: ActorSystemImpl = _ - var system2: ActorSystemImpl = _ - var system3: ActorSystemImpl = _ - - try { - "A Gossip-driven Failure Detector" must { - - // ======= NODE 1 ======== - system1 = ActorSystem("system1", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port=%d550 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote1 = system1.provider.asInstanceOf[RemoteActorRefProvider] - node1 = Cluster(system1) - val fd1 = node1.failureDetector - val address1 = node1.remoteAddress - - // ======= NODE 2 ======== - system2 = ActorSystem("system2", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port=%d551 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote2 = system2.provider.asInstanceOf[RemoteActorRefProvider] - node2 = Cluster(system2) - val fd2 = node2.failureDetector - val address2 = node2.remoteAddress - - // ======= NODE 3 ======== - system3 = ActorSystem("system3", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port=%d552 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote3 = system3.provider.asInstanceOf[RemoteActorRefProvider] - node3 = Cluster(system3) - val fd3 = node3.failureDetector - val address3 = node3.remoteAddress - - "receive gossip heartbeats so that all healthy systems in the cluster are marked 'available'" taggedAs LongRunningTest in { - println("Let the systems gossip for a while...") - Thread.sleep(30.seconds.dilated.toMillis) // let them gossip for 30 seconds - fd1.isAvailable(address2) must be(true) - fd1.isAvailable(address3) must be(true) - fd2.isAvailable(address1) must be(true) - fd2.isAvailable(address3) must be(true) - fd3.isAvailable(address1) must be(true) - fd3.isAvailable(address2) must be(true) - } - - "mark system as 'unavailable' if a system in the cluster is shut down (and its heartbeats stops)" taggedAs LongRunningTest in { - // shut down system3 - node3.shutdown() - system3.shutdown() - println("Give the remaning systems time to detect failure...") - Thread.sleep(30.seconds.dilated.toMillis) // give them 30 seconds to detect failure of system3 - fd1.isAvailable(address2) must be(true) - fd1.isAvailable(address3) must be(false) - fd2.isAvailable(address1) must be(true) - fd2.isAvailable(address3) must be(false) - } - } - } catch { - case e: Exception ⇒ - e.printStackTrace - fail(e.toString) - } - - override def atTermination() { - if (node1 ne null) node1.shutdown() - if (system1 ne null) system1.shutdown() - - if (node2 ne null) node2.shutdown() - if (system2 ne null) system2.shutdown() - - if (node3 ne null) node3.shutdown() - if (system3 ne null) system3.shutdown() - } -} diff --git a/akka-cluster/src/test/scala/akka/cluster/JoinTwoClustersSpec.scala b/akka-cluster/src/test/scala/akka/cluster/JoinTwoClustersSpec.scala deleted file mode 100644 index ff12cb6c60..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/JoinTwoClustersSpec.scala +++ /dev/null @@ -1,179 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.cluster - -import akka.testkit._ -import akka.dispatch._ -import akka.actor._ -import akka.remote._ -import akka.util.duration._ - -import com.typesafe.config._ - -import java.net.InetSocketAddress - -class JoinTwoClustersSpec extends ClusterSpec("akka.cluster.failure-detector.threshold = 5") with ImplicitSender { - val portPrefix = 3 - - var node1: Cluster = _ - var node2: Cluster = _ - var node3: Cluster = _ - var node4: Cluster = _ - var node5: Cluster = _ - var node6: Cluster = _ - - var system1: ActorSystemImpl = _ - var system2: ActorSystemImpl = _ - var system3: ActorSystemImpl = _ - var system4: ActorSystemImpl = _ - var system5: ActorSystemImpl = _ - var system6: ActorSystemImpl = _ - - try { - "Three different clusters (A, B and C)" must { - - // ======= NODE 1 ======== - system1 = ActorSystem("system1", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d551 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - node1 = Cluster(system1) - - // ======= NODE 2 ======== - system2 = ActorSystem("system2", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d552 - cluster.node-to-join = "akka://system1@localhost:%d551" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - node2 = Cluster(system2) - - // ======= NODE 3 ======== - system3 = ActorSystem("system3", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d553 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - node3 = Cluster(system3) - - // ======= NODE 4 ======== - system4 = ActorSystem("system4", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d554 - cluster.node-to-join = "akka://system3@localhost:%d553" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - node4 = Cluster(system4) - - // ======= NODE 5 ======== - system5 = ActorSystem("system5", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d555 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - node5 = Cluster(system5) - - // ======= NODE 6 ======== - system6 = ActorSystem("system6", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d556 - cluster.node-to-join = "akka://system5@localhost:%d555" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - node6 = Cluster(system6) - - "be able to 'elect' a single leader after joining (A -> B)" taggedAs LongRunningTest in { - - println("Give the system time to converge...") - awaitConvergence(node1 :: node2 :: node3 :: node4 :: node5 :: node6 :: Nil) - - // check leader - node1.isLeader must be(true) - node2.isLeader must be(false) - node3.isLeader must be(true) - node4.isLeader must be(false) - node5.isLeader must be(true) - node6.isLeader must be(false) - - // join - node4.join(node1.remoteAddress) - //node1.scheduleNodeJoin(node4.remoteAddress) - - println("Give the system time to converge...") - Thread.sleep(10.seconds.dilated.toMillis) - awaitConvergence(node1 :: node2 :: node3 :: node4 :: node5 :: node6 :: Nil) - - // check leader - node1.isLeader must be(true) - node2.isLeader must be(false) - node3.isLeader must be(false) - node4.isLeader must be(false) - node5.isLeader must be(true) - node6.isLeader must be(false) - } - - "be able to 'elect' a single leader after joining (C -> A + B)" taggedAs LongRunningTest in { - // join - node4.join(node5.remoteAddress) - //node5.scheduleNodeJoin(node4.remoteAddress) - - println("Give the system time to converge...") - Thread.sleep(10.seconds.dilated.toMillis) - awaitConvergence(node1 :: node2 :: node3 :: node4 :: node5 :: node6 :: Nil) - - // check leader - node1.isLeader must be(true) - node2.isLeader must be(false) - node3.isLeader must be(false) - node4.isLeader must be(false) - node5.isLeader must be(false) - node6.isLeader must be(false) - } - } - } catch { - case e: Exception ⇒ - e.printStackTrace - fail(e.toString) - } - - override def atTermination() { - if (node1 ne null) node1.shutdown() - if (system1 ne null) system1.shutdown() - - if (node2 ne null) node2.shutdown() - if (system2 ne null) system2.shutdown() - - if (node3 ne null) node3.shutdown() - if (system3 ne null) system3.shutdown() - - if (node4 ne null) node4.shutdown() - if (system4 ne null) system4.shutdown() - - if (node5 ne null) node5.shutdown() - if (system5 ne null) system5.shutdown() - - if (node6 ne null) node6.shutdown() - if (system6 ne null) system6.shutdown() - } -} diff --git a/akka-cluster/src/test/scala/akka/cluster/LeaderDowningSpec.scala b/akka-cluster/src/test/scala/akka/cluster/LeaderDowningSpec.scala deleted file mode 100644 index 15e6cec838..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/LeaderDowningSpec.scala +++ /dev/null @@ -1,144 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.cluster - -import akka.testkit._ -import akka.dispatch._ -import akka.actor._ -import akka.remote._ -import akka.util.duration._ - -import com.typesafe.config._ - -import java.net.InetSocketAddress - -class LeaderDowningSpec extends ClusterSpec with ImplicitSender { - val portPrefix = 4 - - var node1: Cluster = _ - var node2: Cluster = _ - var node3: Cluster = _ - var node4: Cluster = _ - - var system1: ActorSystemImpl = _ - var system2: ActorSystemImpl = _ - var system3: ActorSystemImpl = _ - var system4: ActorSystemImpl = _ - - try { - "The Leader in a 4 node cluster" must { - - // ======= NODE 1 ======== - system1 = ActorSystem("system1", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d550 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote1 = system1.provider.asInstanceOf[RemoteActorRefProvider] - node1 = Cluster(system1) - val fd1 = node1.failureDetector - val address1 = node1.remoteAddress - - // ======= NODE 2 ======== - system2 = ActorSystem("system2", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d551 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote2 = system2.provider.asInstanceOf[RemoteActorRefProvider] - node2 = Cluster(system2) - val fd2 = node2.failureDetector - val address2 = node2.remoteAddress - - // ======= NODE 3 ======== - system3 = ActorSystem("system3", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d552 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote3 = system3.provider.asInstanceOf[RemoteActorRefProvider] - node3 = Cluster(system3) - val fd3 = node3.failureDetector - val address3 = node3.remoteAddress - - // ======= NODE 4 ======== - system4 = ActorSystem("system4", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d553 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote4 = system4.provider.asInstanceOf[RemoteActorRefProvider] - node4 = Cluster(system4) - val fd4 = node4.failureDetector - val address4 = node4.remoteAddress - - "be able to DOWN a (last) node that is UNREACHABLE" taggedAs LongRunningTest in { - - println("Give the system time to converge...") - awaitConvergence(node1 :: node2 :: node3 :: node4 :: Nil) - - // shut down system4 - node4.shutdown() - system4.shutdown() - - // wait for convergence - e.g. the leader to auto-down the failed node - println("Give the system time to converge...") - Thread.sleep(30.seconds.dilated.toMillis) - awaitConvergence(node1 :: node2 :: node3 :: Nil) - - node1.latestGossip.members.size must be(3) - node1.latestGossip.members.exists(_.address == address4) must be(false) - } - - "be able to DOWN a (middle) node that is UNREACHABLE" taggedAs LongRunningTest in { - // shut down system4 - node2.shutdown() - system2.shutdown() - - // wait for convergence - e.g. the leader to auto-down the failed node - println("Give the system time to converge...") - Thread.sleep(30.seconds.dilated.toMillis) - awaitConvergence(node1 :: node3 :: Nil) - - node1.latestGossip.members.size must be(2) - node1.latestGossip.members.exists(_.address == address4) must be(false) - node1.latestGossip.members.exists(_.address == address2) must be(false) - } - } - } catch { - case e: Exception ⇒ - e.printStackTrace - fail(e.toString) - } - - override def atTermination() { - if (node1 ne null) node1.shutdown() - if (system1 ne null) system1.shutdown() - - if (node2 ne null) node2.shutdown() - if (system2 ne null) system2.shutdown() - - if (node3 ne null) node3.shutdown() - if (system3 ne null) system3.shutdown() - - if (node4 ne null) node4.shutdown() - if (system4 ne null) system4.shutdown() - } -} diff --git a/akka-cluster/src/test/scala/akka/cluster/LeaderElectionSpec.scala b/akka-cluster/src/test/scala/akka/cluster/LeaderElectionSpec.scala deleted file mode 100644 index c262fad8c3..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/LeaderElectionSpec.scala +++ /dev/null @@ -1,131 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.cluster - -import akka.testkit._ -import akka.dispatch._ -import akka.actor._ -import akka.remote._ -import akka.util.duration._ - -import com.typesafe.config._ - -import java.net.InetSocketAddress - -class LeaderElectionSpec extends ClusterSpec with ImplicitSender { - val portPrefix = 5 - - var node1: Cluster = _ - var node2: Cluster = _ - var node3: Cluster = _ - - var system1: ActorSystemImpl = _ - var system2: ActorSystemImpl = _ - var system3: ActorSystemImpl = _ - - try { - "A cluster of three nodes" must { - - // ======= NODE 1 ======== - system1 = ActorSystem("system1", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d550 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - node1 = Cluster(system1) - val address1 = node1.remoteAddress - - // ======= NODE 2 ======== - system2 = ActorSystem("system2", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d551 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - node2 = Cluster(system2) - val address2 = node2.remoteAddress - - // ======= NODE 3 ======== - system3 = ActorSystem("system3", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d552 - cluster.node-to-join = "akka://system1@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - node3 = Cluster(system3) - val address3 = node3.remoteAddress - - "be able to 'elect' a single leader" taggedAs LongRunningTest in { - - println("Give the system time to converge...") - awaitConvergence(node1 :: node2 :: node3 :: Nil) - - // check leader - node1.isLeader must be(true) - node2.isLeader must be(false) - node3.isLeader must be(false) - } - - "be able to 're-elect' a single leader after leader has left" taggedAs LongRunningTest in { - - // shut down system1 - the leader - node1.shutdown() - system1.shutdown() - - // user marks node1 as DOWN - node2.down(address1) - - println("Give the system time to converge...") - Thread.sleep(10.seconds.dilated.toMillis) - awaitConvergence(node2 :: node3 :: Nil) - - // check leader - node2.isLeader must be(true) - node3.isLeader must be(false) - } - - "be able to 're-elect' a single leader after leader has left (again, leaving a single node)" taggedAs LongRunningTest in { - - // shut down system1 - the leader - node2.shutdown() - system2.shutdown() - - // user marks node2 as DOWN - node3.down(address2) - - println("Give the system time to converge...") - Thread.sleep(10.seconds.dilated.toMillis) - awaitConvergence(node3 :: Nil) - - // check leader - node3.isLeader must be(true) - } - } - } catch { - case e: Exception ⇒ - e.printStackTrace - fail(e.toString) - } - - override def atTermination() { - if (node1 ne null) node1.shutdown() - if (system1 ne null) system1.shutdown() - - if (node2 ne null) node2.shutdown() - if (system2 ne null) system2.shutdown() - - if (node3 ne null) node3.shutdown() - if (system3 ne null) system3.shutdown() - } -} diff --git a/akka-cluster/src/test/scala/akka/cluster/MemberOrderingSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MemberOrderingSpec.scala new file mode 100644 index 0000000000..d8687312da --- /dev/null +++ b/akka-cluster/src/test/scala/akka/cluster/MemberOrderingSpec.scala @@ -0,0 +1,138 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import akka.actor.{ Address, AddressFromURIString } +import java.net.InetSocketAddress +import org.scalatest.matchers.MustMatchers +import org.scalatest.WordSpec +import scala.collection.immutable.SortedSet +import scala.util.Random + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class MemberOrderingSpec extends WordSpec with MustMatchers { + import Member.ordering + import Member.addressOrdering + import MemberStatus._ + + "An Ordering[Member]" must { + + "order non-exiting members by host:port" in { + val members = SortedSet.empty[Member] + + Member(AddressFromURIString("akka://sys@darkstar:1112"), Up) + + Member(AddressFromURIString("akka://sys@darkstar:1113"), Joining) + + Member(AddressFromURIString("akka://sys@darkstar:1111"), Up) + + val seq = members.toSeq + seq.size must equal(3) + seq(0) must equal(Member(AddressFromURIString("akka://sys@darkstar:1111"), Up)) + seq(1) must equal(Member(AddressFromURIString("akka://sys@darkstar:1112"), Up)) + seq(2) must equal(Member(AddressFromURIString("akka://sys@darkstar:1113"), Joining)) + } + + "order exiting members by last" in { + val members = SortedSet.empty[Member] + + Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting) + + Member(AddressFromURIString("akka://sys@darkstar:1113"), Up) + + Member(AddressFromURIString("akka://sys@darkstar:1111"), Joining) + + val seq = members.toSeq + seq.size must equal(3) + seq(0) must equal(Member(AddressFromURIString("akka://sys@darkstar:1111"), Joining)) + seq(1) must equal(Member(AddressFromURIString("akka://sys@darkstar:1113"), Up)) + seq(2) must equal(Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting)) + } + + "order multiple exiting members by last but internally by host:port" in { + val members = SortedSet.empty[Member] + + Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting) + + Member(AddressFromURIString("akka://sys@darkstar:1113"), Leaving) + + Member(AddressFromURIString("akka://sys@darkstar:1111"), Up) + + Member(AddressFromURIString("akka://sys@darkstar:1110"), Exiting) + + val seq = members.toSeq + seq.size must equal(4) + seq(0) must equal(Member(AddressFromURIString("akka://sys@darkstar:1111"), Up)) + seq(1) must equal(Member(AddressFromURIString("akka://sys@darkstar:1113"), Leaving)) + seq(2) must equal(Member(AddressFromURIString("akka://sys@darkstar:1110"), Exiting)) + seq(3) must equal(Member(AddressFromURIString("akka://sys@darkstar:1112"), Exiting)) + } + + "be sorted by address correctly" in { + import Member.ordering + // sorting should be done on host and port, only + val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up) + val m2 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up) + val m3 = Member(Address("cluster", "sys2", "host2", 8000), MemberStatus.Up) + val m4 = Member(Address("cluster", "sys2", "host2", 9000), MemberStatus.Up) + val m5 = Member(Address("cluster", "sys1", "host2", 10000), MemberStatus.Up) + + val expected = IndexedSeq(m1, m2, m3, m4, m5) + val shuffled = Random.shuffle(expected) + shuffled.sorted must be(expected) + (SortedSet.empty[Member] ++ shuffled).toIndexedSeq must be(expected) + } + + "have stable equals and hashCode" in { + val m1 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Joining) + val m2 = Member(Address("akka", "sys1", "host1", 9000), MemberStatus.Up) + val m3 = Member(Address("akka", "sys1", "host1", 10000), MemberStatus.Up) + + m1 must be(m2) + m1.hashCode must be(m2.hashCode) + + m3 must not be (m2) + m3 must not be (m1) + } + } + + "An Ordering[Address]" must { + + "order addresses by port" in { + val addresses = SortedSet.empty[Address] + + AddressFromURIString("akka://sys@darkstar:1112") + + AddressFromURIString("akka://sys@darkstar:1113") + + AddressFromURIString("akka://sys@darkstar:1110") + + AddressFromURIString("akka://sys@darkstar:1111") + + val seq = addresses.toSeq + seq.size must equal(4) + seq(0) must equal(AddressFromURIString("akka://sys@darkstar:1110")) + seq(1) must equal(AddressFromURIString("akka://sys@darkstar:1111")) + seq(2) must equal(AddressFromURIString("akka://sys@darkstar:1112")) + seq(3) must equal(AddressFromURIString("akka://sys@darkstar:1113")) + } + + "order addresses by hostname" in { + val addresses = SortedSet.empty[Address] + + AddressFromURIString("akka://sys@darkstar2:1110") + + AddressFromURIString("akka://sys@darkstar1:1110") + + AddressFromURIString("akka://sys@darkstar3:1110") + + AddressFromURIString("akka://sys@darkstar0:1110") + + val seq = addresses.toSeq + seq.size must equal(4) + seq(0) must equal(AddressFromURIString("akka://sys@darkstar0:1110")) + seq(1) must equal(AddressFromURIString("akka://sys@darkstar1:1110")) + seq(2) must equal(AddressFromURIString("akka://sys@darkstar2:1110")) + seq(3) must equal(AddressFromURIString("akka://sys@darkstar3:1110")) + } + + "order addresses by hostname and port" in { + val addresses = SortedSet.empty[Address] + + AddressFromURIString("akka://sys@darkstar2:1110") + + AddressFromURIString("akka://sys@darkstar0:1111") + + AddressFromURIString("akka://sys@darkstar2:1111") + + AddressFromURIString("akka://sys@darkstar0:1110") + + val seq = addresses.toSeq + seq.size must equal(4) + seq(0) must equal(AddressFromURIString("akka://sys@darkstar0:1110")) + seq(1) must equal(AddressFromURIString("akka://sys@darkstar0:1111")) + seq(2) must equal(AddressFromURIString("akka://sys@darkstar2:1110")) + seq(3) must equal(AddressFromURIString("akka://sys@darkstar2:1111")) + } + } +} diff --git a/akka-cluster/src/test/scala/akka/cluster/MembershipChangeListenerSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MembershipChangeListenerSpec.scala deleted file mode 100644 index 17a7c6ed7a..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/MembershipChangeListenerSpec.scala +++ /dev/null @@ -1,129 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ -package akka.cluster - -import akka.testkit._ -import akka.dispatch._ -import akka.actor._ -import akka.remote._ -import akka.util.duration._ - -import java.net.InetSocketAddress -import java.util.concurrent.{ CountDownLatch, TimeUnit } - -import scala.collection.immutable.SortedSet - -import com.typesafe.config._ - -class MembershipChangeListenerSpec extends ClusterSpec with ImplicitSender { - val portPrefix = 6 - - var node0: Cluster = _ - var node1: Cluster = _ - var node2: Cluster = _ - - var system0: ActorSystemImpl = _ - var system1: ActorSystemImpl = _ - var system2: ActorSystemImpl = _ - - try { - "A set of connected cluster systems" must { - "(when two systems) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in { - system0 = ActorSystem("system0", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d550 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote0 = system0.provider.asInstanceOf[RemoteActorRefProvider] - node0 = Cluster(system0) - - system1 = ActorSystem("system1", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port=%d551 - cluster.node-to-join = "akka://system0@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote1 = system1.provider.asInstanceOf[RemoteActorRefProvider] - node1 = Cluster(system1) - - val latch = new CountDownLatch(2) - - node0.registerListener(new MembershipChangeListener { - def notify(members: SortedSet[Member]) { - latch.countDown() - } - }) - node1.registerListener(new MembershipChangeListener { - def notify(members: SortedSet[Member]) { - latch.countDown() - } - }) - - latch.await(10.seconds.dilated.toMillis, TimeUnit.MILLISECONDS) - - Thread.sleep(10.seconds.dilated.toMillis) - - // check cluster convergence - node0.convergence must be('defined) - node1.convergence must be('defined) - } - - "(when three systems) after cluster convergence updates the membership table then all MembershipChangeListeners should be triggered" taggedAs LongRunningTest in { - - // ======= NODE 2 ======== - system2 = ActorSystem("system2", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port=%d552 - cluster.node-to-join = "akka://system0@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote2 = system2.provider.asInstanceOf[RemoteActorRefProvider] - node2 = Cluster(system2) - - val latch = new CountDownLatch(3) - node0.registerListener(new MembershipChangeListener { - def notify(members: SortedSet[Member]) { - latch.countDown() - } - }) - node1.registerListener(new MembershipChangeListener { - def notify(members: SortedSet[Member]) { - latch.countDown() - } - }) - node2.registerListener(new MembershipChangeListener { - def notify(members: SortedSet[Member]) { - latch.countDown() - } - }) - - latch.await(30.seconds.dilated.toMillis, TimeUnit.MILLISECONDS) - } - } - } catch { - case e: Exception ⇒ - e.printStackTrace - fail(e.toString) - } - - override def atTermination() { - if (node0 ne null) node0.shutdown() - if (system0 ne null) system0.shutdown() - - if (node1 ne null) node1.shutdown() - if (system1 ne null) system1.shutdown() - - if (node2 ne null) node2.shutdown() - if (system2 ne null) system2.shutdown() - } -} diff --git a/akka-cluster/src/test/scala/akka/cluster/NodeMembershipSpec.scala b/akka-cluster/src/test/scala/akka/cluster/NodeMembershipSpec.scala deleted file mode 100644 index 923a3267ca..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/NodeMembershipSpec.scala +++ /dev/null @@ -1,136 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ -package akka.cluster - -import java.net.InetSocketAddress - -import akka.testkit._ -import akka.dispatch._ -import akka.actor._ -import akka.remote._ -import akka.util.duration._ - -import com.typesafe.config._ - -class NodeMembershipSpec extends ClusterSpec with ImplicitSender { - val portPrefix = 7 - - var node0: Cluster = _ - var node1: Cluster = _ - var node2: Cluster = _ - - var system0: ActorSystemImpl = _ - var system1: ActorSystemImpl = _ - var system2: ActorSystemImpl = _ - - try { - "A set of connected cluster systems" must { - "(when two systems) start gossiping to each other so that both systems gets the same gossip info" taggedAs LongRunningTest in { - - // ======= NODE 0 ======== - system0 = ActorSystem("system0", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d550 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote0 = system0.provider.asInstanceOf[RemoteActorRefProvider] - node0 = Cluster(system0) - - // ======= NODE 1 ======== - system1 = ActorSystem("system1", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d551 - cluster.node-to-join = "akka://system0@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote1 = system1.provider.asInstanceOf[RemoteActorRefProvider] - node1 = Cluster(system1) - - // check cluster convergence - awaitConvergence(node0 :: node1 :: Nil) - - val members0 = node0.latestGossip.members.toArray - members0.size must be(2) - members0(0).address.port.get must be(550.withPortPrefix) - members0(0).status must be(MemberStatus.Up) - members0(1).address.port.get must be(551.withPortPrefix) - members0(1).status must be(MemberStatus.Up) - - val members1 = node1.latestGossip.members.toArray - members1.size must be(2) - members1(0).address.port.get must be(550.withPortPrefix) - members1(0).status must be(MemberStatus.Up) - members1(1).address.port.get must be(551.withPortPrefix) - members1(1).status must be(MemberStatus.Up) - } - - "(when three systems) start gossiping to each other so that both systems gets the same gossip info" taggedAs LongRunningTest ignore { - - // ======= NODE 2 ======== - system2 = ActorSystem("system2", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port = %d552 - cluster.node-to-join = "akka://system0@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote2 = system2.provider.asInstanceOf[RemoteActorRefProvider] - node2 = Cluster(system2) - - awaitConvergence(node0 :: node1 :: node2 :: Nil) - - val members0 = node0.latestGossip.members.toArray - val version = node0.latestGossip.version - members0.size must be(3) - members0(0).address.port.get must be(550.withPortPrefix) - members0(0).status must be(MemberStatus.Up) - members0(1).address.port.get must be(551.withPortPrefix) - members0(1).status must be(MemberStatus.Up) - members0(2).address.port.get must be(552.withPortPrefix) - members0(2).status must be(MemberStatus.Up) - - val members1 = node1.latestGossip.members.toArray - members1.size must be(3) - members1(0).address.port.get must be(550.withPortPrefix) - members1(0).status must be(MemberStatus.Up) - members1(1).address.port.get must be(551.withPortPrefix) - members1(1).status must be(MemberStatus.Up) - members1(2).address.port.get must be(552.withPortPrefix) - members1(2).status must be(MemberStatus.Up) - - val members2 = node2.latestGossip.members.toArray - members2.size must be(3) - members2(0).address.port.get must be(550.withPortPrefix) - members2(0).status must be(MemberStatus.Up) - members2(1).address.port.get must be(551.withPortPrefix) - members2(1).status must be(MemberStatus.Up) - members2(2).address.port.get must be(552.withPortPrefix) - members2(2).status must be(MemberStatus.Up) - } - } - } catch { - case e: Exception ⇒ - e.printStackTrace - fail(e.toString) - } - - override def atTermination() { - if (node0 ne null) node0.shutdown() - if (system0 ne null) system0.shutdown() - - if (node1 ne null) node1.shutdown() - if (system1 ne null) system1.shutdown() - - if (node2 ne null) node2.shutdown() - if (system2 ne null) system2.shutdown() - } -} diff --git a/akka-cluster/src/test/scala/akka/cluster/NodeStartupSpec.scala b/akka-cluster/src/test/scala/akka/cluster/NodeStartupSpec.scala deleted file mode 100644 index 33f069e3bb..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/NodeStartupSpec.scala +++ /dev/null @@ -1,84 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ -package akka.cluster - -import java.net.InetSocketAddress - -import akka.testkit._ -import akka.dispatch._ -import akka.actor._ -import akka.remote._ -import akka.util.duration._ - -import com.typesafe.config._ - -class NodeStartupSpec extends ClusterSpec with ImplicitSender { - val portPrefix = 8 - - var node0: Cluster = _ - var node1: Cluster = _ - var system0: ActorSystemImpl = _ - var system1: ActorSystemImpl = _ - - try { - "A first cluster node with a 'node-to-join' config set to empty string (singleton cluster)" must { - system0 = ActorSystem("system0", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port=%d550 - }""".format(portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote0 = system0.provider.asInstanceOf[RemoteActorRefProvider] - node0 = Cluster(system0) - - "be a singleton cluster when started up" taggedAs LongRunningTest in { - Thread.sleep(1.seconds.dilated.toMillis) - node0.isSingletonCluster must be(true) - } - - "be in 'Joining' phase when started up" taggedAs LongRunningTest in { - val members = node0.latestGossip.members - val joiningMember = members find (_.address.port.get == 550.withPortPrefix) - joiningMember must be('defined) - joiningMember.get.status must be(MemberStatus.Joining) - } - } - - "A second cluster node with a 'node-to-join' config defined" must { - "join the other node cluster when sending a Join command" taggedAs LongRunningTest in { - system1 = ActorSystem("system1", ConfigFactory - .parseString(""" - akka { - actor.provider = "akka.remote.RemoteActorRefProvider" - remote.netty.port=%d551 - cluster.node-to-join = "akka://system0@localhost:%d550" - }""".format(portPrefix, portPrefix)) - .withFallback(system.settings.config)) - .asInstanceOf[ActorSystemImpl] - val remote1 = system1.provider.asInstanceOf[RemoteActorRefProvider] - node1 = Cluster(system1) - - Thread.sleep(10.seconds.dilated.toMillis) // give enough time for node1 to JOIN node0 and leader to move him to UP - val members = node0.latestGossip.members - val joiningMember = members find (_.address.port.get == 551.withPortPrefix) - joiningMember must be('defined) - joiningMember.get.status must be(MemberStatus.Up) - } - } - } catch { - case e: Exception ⇒ - e.printStackTrace - fail(e.toString) - } - - override def atTermination() { - if (node0 ne null) node0.shutdown() - if (system0 ne null) system0.shutdown() - - if (node1 ne null) node1.shutdown() - if (system1 ne null) system1.shutdown() - } -} diff --git a/akka-cluster/src/test/scala/akka/cluster/VectorClockSpec.scala b/akka-cluster/src/test/scala/akka/cluster/VectorClockSpec.scala index de1142b668..19ad9410c4 100644 --- a/akka-cluster/src/test/scala/akka/cluster/VectorClockSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/VectorClockSpec.scala @@ -27,67 +27,67 @@ class VectorClockSpec extends AkkaSpec { "pass misc comparison test 1" in { val clock1_1 = VectorClock() - val clock2_1 = clock1_1 + Node("1") - val clock3_1 = clock2_1 + Node("2") - val clock4_1 = clock3_1 + Node("1") + val clock2_1 = clock1_1 :+ Node("1") + val clock3_1 = clock2_1 :+ Node("2") + val clock4_1 = clock3_1 :+ Node("1") val clock1_2 = VectorClock() - val clock2_2 = clock1_2 + Node("1") - val clock3_2 = clock2_2 + Node("2") - val clock4_2 = clock3_2 + Node("1") + val clock2_2 = clock1_2 :+ Node("1") + val clock3_2 = clock2_2 :+ Node("2") + val clock4_2 = clock3_2 :+ Node("1") clock4_1 <> clock4_2 must be(false) } "pass misc comparison test 2" in { val clock1_1 = VectorClock() - val clock2_1 = clock1_1 + Node("1") - val clock3_1 = clock2_1 + Node("2") - val clock4_1 = clock3_1 + Node("1") + val clock2_1 = clock1_1 :+ Node("1") + val clock3_1 = clock2_1 :+ Node("2") + val clock4_1 = clock3_1 :+ Node("1") val clock1_2 = VectorClock() - val clock2_2 = clock1_2 + Node("1") - val clock3_2 = clock2_2 + Node("2") - val clock4_2 = clock3_2 + Node("1") - val clock5_2 = clock4_2 + Node("3") + val clock2_2 = clock1_2 :+ Node("1") + val clock3_2 = clock2_2 :+ Node("2") + val clock4_2 = clock3_2 :+ Node("1") + val clock5_2 = clock4_2 :+ Node("3") clock4_1 < clock5_2 must be(true) } "pass misc comparison test 3" in { var clock1_1 = VectorClock() - val clock2_1 = clock1_1 + Node("1") + val clock2_1 = clock1_1 :+ Node("1") val clock1_2 = VectorClock() - val clock2_2 = clock1_2 + Node("2") + val clock2_2 = clock1_2 :+ Node("2") clock2_1 <> clock2_2 must be(true) } "pass misc comparison test 4" in { val clock1_3 = VectorClock() - val clock2_3 = clock1_3 + Node("1") - val clock3_3 = clock2_3 + Node("2") - val clock4_3 = clock3_3 + Node("1") + val clock2_3 = clock1_3 :+ Node("1") + val clock3_3 = clock2_3 :+ Node("2") + val clock4_3 = clock3_3 :+ Node("1") val clock1_4 = VectorClock() - val clock2_4 = clock1_4 + Node("1") - val clock3_4 = clock2_4 + Node("1") - val clock4_4 = clock3_4 + Node("3") + val clock2_4 = clock1_4 :+ Node("1") + val clock3_4 = clock2_4 :+ Node("1") + val clock4_4 = clock3_4 :+ Node("3") clock4_3 <> clock4_4 must be(true) } "pass misc comparison test 5" in { val clock1_1 = VectorClock() - val clock2_1 = clock1_1 + Node("2") - val clock3_1 = clock2_1 + Node("2") + val clock2_1 = clock1_1 :+ Node("2") + val clock3_1 = clock2_1 :+ Node("2") val clock1_2 = VectorClock() - val clock2_2 = clock1_2 + Node("1") - val clock3_2 = clock2_2 + Node("2") - val clock4_2 = clock3_2 + Node("2") - val clock5_2 = clock4_2 + Node("3") + val clock2_2 = clock1_2 :+ Node("1") + val clock3_2 = clock2_2 :+ Node("2") + val clock4_2 = clock3_2 :+ Node("2") + val clock5_2 = clock4_2 :+ Node("3") clock3_1 < clock5_2 must be(true) clock5_2 > clock3_1 must be(true) @@ -95,12 +95,12 @@ class VectorClockSpec extends AkkaSpec { "pass misc comparison test 6" in { val clock1_1 = VectorClock() - val clock2_1 = clock1_1 + Node("1") - val clock3_1 = clock2_1 + Node("2") + val clock2_1 = clock1_1 :+ Node("1") + val clock3_1 = clock2_1 :+ Node("2") val clock1_2 = VectorClock() - val clock2_2 = clock1_2 + Node("1") - val clock3_2 = clock2_2 + Node("1") + val clock2_2 = clock1_2 :+ Node("1") + val clock3_2 = clock2_2 :+ Node("1") clock3_1 <> clock3_2 must be(true) clock3_2 <> clock3_1 must be(true) @@ -108,14 +108,14 @@ class VectorClockSpec extends AkkaSpec { "pass misc comparison test 7" in { val clock1_1 = VectorClock() - val clock2_1 = clock1_1 + Node("1") - val clock3_1 = clock2_1 + Node("2") - val clock4_1 = clock3_1 + Node("2") - val clock5_1 = clock4_1 + Node("3") + val clock2_1 = clock1_1 :+ Node("1") + val clock3_1 = clock2_1 :+ Node("2") + val clock4_1 = clock3_1 :+ Node("2") + val clock5_1 = clock4_1 :+ Node("3") val clock1_2 = VectorClock() - val clock2_2 = clock1_2 + Node("2") - val clock3_2 = clock2_2 + Node("2") + val clock2_2 = clock1_2 :+ Node("2") + val clock3_2 = clock2_2 :+ Node("2") clock5_1 <> clock3_2 must be(true) clock3_2 <> clock5_1 must be(true) @@ -127,14 +127,14 @@ class VectorClockSpec extends AkkaSpec { val node3 = Node("3") val clock1_1 = VectorClock() - val clock2_1 = clock1_1 + node1 - val clock3_1 = clock2_1 + node2 - val clock4_1 = clock3_1 + node2 - val clock5_1 = clock4_1 + node3 + val clock2_1 = clock1_1 :+ node1 + val clock3_1 = clock2_1 :+ node2 + val clock4_1 = clock3_1 :+ node2 + val clock5_1 = clock4_1 :+ node3 val clock1_2 = VectorClock() - val clock2_2 = clock1_2 + node2 - val clock3_2 = clock2_2 + node2 + val clock2_2 = clock1_2 :+ node2 + val clock3_2 = clock2_2 :+ node2 val merged1 = clock3_2 merge clock5_1 merged1.versions.size must be(3) @@ -164,14 +164,14 @@ class VectorClockSpec extends AkkaSpec { val node4 = Node("4") val clock1_1 = VectorClock() - val clock2_1 = clock1_1 + node1 - val clock3_1 = clock2_1 + node2 - val clock4_1 = clock3_1 + node2 - val clock5_1 = clock4_1 + node3 + val clock2_1 = clock1_1 :+ node1 + val clock3_1 = clock2_1 :+ node2 + val clock4_1 = clock3_1 :+ node2 + val clock5_1 = clock4_1 :+ node3 val clock1_2 = VectorClock() - val clock2_2 = clock1_2 + node4 - val clock3_2 = clock2_2 + node4 + val clock2_2 = clock1_2 :+ node4 + val clock3_2 = clock2_2 :+ node4 val merged1 = clock3_2 merge clock5_1 merged1.versions.size must be(4) @@ -204,8 +204,8 @@ class VectorClockSpec extends AkkaSpec { val v1 = VectorClock() val v2 = VectorClock() - val vv1 = v1 + node1 - val vv2 = v2 + node2 + val vv1 = v1 :+ node1 + val vv2 = v2 :+ node2 (vv1 > v1) must equal(true) (vv2 > v2) must equal(true) @@ -225,12 +225,12 @@ class VectorClockSpec extends AkkaSpec { val a = VectorClock() val b = VectorClock() - val a1 = a + node1 - val b1 = b + node2 + val a1 = a :+ node1 + val b1 = b :+ node2 - var a2 = a1 + node1 + var a2 = a1 :+ node1 var c = a2.merge(b1) - var c1 = c + node3 + var c1 = c :+ node3 (c1 > a2) must equal(true) (c1 > b1) must equal(true) @@ -239,7 +239,7 @@ class VectorClockSpec extends AkkaSpec { "An instance of Versioned" must { class TestVersioned(val version: VectorClock = VectorClock()) extends Versioned[TestVersioned] { - def +(node: Node): TestVersioned = new TestVersioned(version + node) + def :+(node: Node): TestVersioned = new TestVersioned(version :+ node) } import Versioned.latestVersionOf @@ -251,67 +251,67 @@ class VectorClockSpec extends AkkaSpec { "happen before an identical versioned with a single additional event" in { val versioned1_1 = new TestVersioned() - val versioned2_1 = versioned1_1 + Node("1") - val versioned3_1 = versioned2_1 + Node("2") - val versioned4_1 = versioned3_1 + Node("1") + val versioned2_1 = versioned1_1 :+ Node("1") + val versioned3_1 = versioned2_1 :+ Node("2") + val versioned4_1 = versioned3_1 :+ Node("1") val versioned1_2 = new TestVersioned() - val versioned2_2 = versioned1_2 + Node("1") - val versioned3_2 = versioned2_2 + Node("2") - val versioned4_2 = versioned3_2 + Node("1") - val versioned5_2 = versioned4_2 + Node("3") + val versioned2_2 = versioned1_2 :+ Node("1") + val versioned3_2 = versioned2_2 :+ Node("2") + val versioned4_2 = versioned3_2 :+ Node("1") + val versioned5_2 = versioned4_2 :+ Node("3") latestVersionOf[TestVersioned](versioned4_1, versioned5_2) must be(versioned5_2) } "pass misc comparison test 1" in { var versioned1_1 = new TestVersioned() - val versioned2_1 = versioned1_1 + Node("1") + val versioned2_1 = versioned1_1 :+ Node("1") val versioned1_2 = new TestVersioned() - val versioned2_2 = versioned1_2 + Node("2") + val versioned2_2 = versioned1_2 :+ Node("2") latestVersionOf[TestVersioned](versioned2_1, versioned2_2) must be(versioned2_2) } "pass misc comparison test 2" in { val versioned1_3 = new TestVersioned() - val versioned2_3 = versioned1_3 + Node("1") - val versioned3_3 = versioned2_3 + Node("2") - val versioned4_3 = versioned3_3 + Node("1") + val versioned2_3 = versioned1_3 :+ Node("1") + val versioned3_3 = versioned2_3 :+ Node("2") + val versioned4_3 = versioned3_3 :+ Node("1") val versioned1_4 = new TestVersioned() - val versioned2_4 = versioned1_4 + Node("1") - val versioned3_4 = versioned2_4 + Node("1") - val versioned4_4 = versioned3_4 + Node("3") + val versioned2_4 = versioned1_4 :+ Node("1") + val versioned3_4 = versioned2_4 :+ Node("1") + val versioned4_4 = versioned3_4 :+ Node("3") latestVersionOf[TestVersioned](versioned4_3, versioned4_4) must be(versioned4_4) } "pass misc comparison test 3" in { val versioned1_1 = new TestVersioned() - val versioned2_1 = versioned1_1 + Node("2") - val versioned3_1 = versioned2_1 + Node("2") + val versioned2_1 = versioned1_1 :+ Node("2") + val versioned3_1 = versioned2_1 :+ Node("2") val versioned1_2 = new TestVersioned() - val versioned2_2 = versioned1_2 + Node("1") - val versioned3_2 = versioned2_2 + Node("2") - val versioned4_2 = versioned3_2 + Node("2") - val versioned5_2 = versioned4_2 + Node("3") + val versioned2_2 = versioned1_2 :+ Node("1") + val versioned3_2 = versioned2_2 :+ Node("2") + val versioned4_2 = versioned3_2 :+ Node("2") + val versioned5_2 = versioned4_2 :+ Node("3") latestVersionOf[TestVersioned](versioned3_1, versioned5_2) must be(versioned5_2) } "pass misc comparison test 4" in { val versioned1_1 = new TestVersioned() - val versioned2_1 = versioned1_1 + Node("1") - val versioned3_1 = versioned2_1 + Node("2") - val versioned4_1 = versioned3_1 + Node("2") - val versioned5_1 = versioned4_1 + Node("3") + val versioned2_1 = versioned1_1 :+ Node("1") + val versioned3_1 = versioned2_1 :+ Node("2") + val versioned4_1 = versioned3_1 :+ Node("2") + val versioned5_1 = versioned4_1 :+ Node("3") val versioned1_2 = new TestVersioned() - val versioned2_2 = versioned1_2 + Node("2") - val versioned3_2 = versioned2_2 + Node("2") + val versioned2_2 = versioned1_2 :+ Node("2") + val versioned3_2 = versioned2_2 :+ Node("2") latestVersionOf[TestVersioned](versioned5_1, versioned3_2) must be(versioned3_2) } diff --git a/akka-docs/Makefile b/akka-docs/Makefile index 3c0041537d..c78d5ba317 100644 --- a/akka-docs/Makefile +++ b/akka-docs/Makefile @@ -34,6 +34,7 @@ endif help: @echo "Please use \`make ' where is one of" @echo " pygments to locally install the custom pygments styles" + @echo " epub to make an epub" @echo " html to make standalone HTML files" @echo " singlehtml to make a single large HTML file" @echo " latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter" @@ -53,6 +54,11 @@ pygments: $(LOCALPACKAGES): $(MAKE) pygments +epub: $(LOCALPACKAGES) + $(SPHINXBUILD) $(SPHINXFLAGS) -b epub $(ALLSPHINXOPTS) $(BUILDDIR)/epub + @echo + @echo "Build finished. The epub file is in $(BUILDDIR)/epub." + html: $(LOCALPACKAGES) $(SPHINXBUILD) $(SPHINXFLAGS) -b html $(ALLSPHINXOPTS) $(BUILDDIR)/html @echo diff --git a/akka-docs/_sphinx/themes/akka/static/docs.css b/akka-docs/_sphinx/themes/akka/static/docs.css index 7b6d3dbf52..3d37718c68 100644 --- a/akka-docs/_sphinx/themes/akka/static/docs.css +++ b/akka-docs/_sphinx/themes/akka/static/docs.css @@ -90,6 +90,42 @@ strong {color: #1d3c52; } box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25); } +.warning { + background-image: none; + background-color: #fdf5d9; + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + padding: 14px; + border-color: #ffffc4; + -webkit-box-shadow: none; + -moz-box-shadow: none; + box-shadow: none; + margin-bottom: 18px; + position: relative; + padding: 7px 15px; + color: #404040; + background-repeat: repeat-x; + background-image: -khtml-gradient(linear, left top, left bottom, from(#ffffc4), to(#ffff00)); + background-image: -moz-linear-gradient(top, #ffffc4, #ffff00); + background-image: -ms-linear-gradient(top, #ffffc4, #ffff00); + background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #ffffc4), color-stop(100%, #ffff00)); + background-image: -webkit-linear-gradient(top, #ffffc4, #ffff00); + background-image: -o-linear-gradient(top, #ffffc4, #ffff00); + background-image: linear-gradient(top, #ffffc4, #ffff00); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffc4', endColorstr='#ffff00', GradientType=0); + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + border-color: #dff69a #ffff00 #E4C652; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + text-shadow: 0 1px 0 rgba(255, 255, 255, 0.5); + border-width: 1px; + border-style: solid; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25); + -moz-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25); +} + .admonition p.admonition-title { color: rgba(0, 0, 0, 0.6); text-shadow: 0 1px 0 rgba(255, 255, 255, .7); diff --git a/akka-docs/additional/code/osgi/Activator.scala b/akka-docs/additional/code/osgi/Activator.scala new file mode 100644 index 0000000000..4f432452c3 --- /dev/null +++ b/akka-docs/additional/code/osgi/Activator.scala @@ -0,0 +1,25 @@ +package docs.osgi + +case object SomeMessage + +class SomeActor extends akka.actor.Actor { + def receive = { case SomeMessage ⇒ } +} + +//#Activator +import akka.actor.{ Props, ActorSystem } +import org.osgi.framework.BundleContext +import akka.osgi.ActorSystemActivator + +class Activator extends ActorSystemActivator { + + def configure(context: BundleContext, system: ActorSystem) { + // optionally register the ActorSystem in the OSGi Service Registry + registerService(context, system) + + val someActor = system.actorOf(Props[SomeActor], name = "someName") + someActor ! SomeMessage + } + +} +//#Activator \ No newline at end of file diff --git a/akka-docs/additional/code/osgi/blueprint.xml b/akka-docs/additional/code/osgi/blueprint.xml new file mode 100644 index 0000000000..8fcedb990c --- /dev/null +++ b/akka-docs/additional/code/osgi/blueprint.xml @@ -0,0 +1,14 @@ + + + + + + + + some.config { + key=value + } + + + diff --git a/akka-docs/additional/index.rst b/akka-docs/additional/index.rst index b3c89356c9..284586d59d 100644 --- a/akka-docs/additional/index.rst +++ b/akka-docs/additional/index.rst @@ -6,3 +6,4 @@ Additional Information recipes language-bindings + osgi diff --git a/akka-docs/additional/osgi.rst b/akka-docs/additional/osgi.rst new file mode 100644 index 0000000000..3bedc8c7dd --- /dev/null +++ b/akka-docs/additional/osgi.rst @@ -0,0 +1,27 @@ +Akka in OSGi +============ + +Configuring the OSGi Framework +------------------------------ + +To use Akka in an OSGi environment, the ``org.osgi.framework.bootdelegation`` +property must be set to always delegate the ``sun.misc`` package to the boot classloader +instead of resolving it through the normal OSGi class space. + + +Activator +--------- + +To bootstrap Akka inside an OSGi environment, you can use the akka.osgi.AkkaSystemActivator class +to conveniently set up the ActorSystem. + +.. includecode:: code/osgi/Activator.scala#Activator + + +Blueprint +--------- + +For the Apache Aries Blueprint implementation, there's also a namespace handler available. The namespace URI +is http://akka.io/xmlns/blueprint/v1.0.0 and it can be used to set up an ActorSystem. + +.. includecode:: code/osgi/blueprint.xml diff --git a/akka-docs/cluster/cluster.rst b/akka-docs/cluster/cluster.rst index 231830cecb..cbad3ef690 100644 --- a/akka-docs/cluster/cluster.rst +++ b/akka-docs/cluster/cluster.rst @@ -5,7 +5,7 @@ Cluster Specification ###################### -.. note:: *This document describes the new clustering coming in Akka 2.1 (not 2.0)* +.. note:: *This document describes the new clustering coming in Akka Coltrane and is not available in the latest stable release)* Intro ===== @@ -81,16 +81,6 @@ can later explicitly send a ``Join`` message to another node to form a N-node cluster. It is also possible to link multiple N-node clusters by ``joining`` them. -Singleton Cluster ------------------ - -If a node does not have a preconfigured contact point to join in the Akka -configuration, then it is considered a singleton cluster (single node cluster) -and will automatically transition from ``joining`` to ``up``. Singleton clusters -can later explicitly send a ``Join`` message to another node to form a N-node -cluster. It is also possible to link multiple N-node clusters by ``joining`` them. - - Gossip ------ @@ -148,7 +138,7 @@ implementation of `The Phi Accrual Failure Detector`_ by Hayashibara et al. An accrual failure detector decouple monitoring and interpretation. That makes them applicable to a wider area of scenarios and more adequate to build generic failure detection services. The idea is that it is keeping a history of failure -statistics, calculated from heartbeats received from the gossip protocol, and is +statistics, calculated from heartbeats received from other nodes, and is trying to do educated guesses by taking multiple factors, and how they accumulate over time, into account in order to come up with a better guess if a specific node is up or down. Rather than just answering "yes" or "no" to the @@ -173,8 +163,8 @@ After gossip convergence a ``leader`` for the cluster can be determined. There i ``leader`` election process, the ``leader`` can always be recognised deterministically by any node whenever there is gossip convergence. The ``leader`` is simply the first node in sorted order that is able to take the leadership role, where the only -allowed member states for a ``leader`` are ``up`` or ``leaving`` (see below for more -information about member states). +allowed member states for a ``leader`` are ``up``, ``leaving`` or ``exiting`` (see +below for more information about member states). The role of the ``leader`` is to shift members in and out of the cluster, changing ``joining`` members to the ``up`` state or ``exiting`` members to the @@ -193,14 +183,20 @@ according to the Failure Detector is considered unreachable. This means setting the unreachable node status to ``down`` automatically. +Seed Nodes +^^^^^^^^^^ + +The seed nodes are configured contact points for inital join of the cluster. +When a new node is started started it sends a message to all seed nodes and +then sends join command to the one that answers first. + +It is possible to turn off automatic join. + Deputy Nodes ^^^^^^^^^^^^ -After gossip convergence a set of ``deputy`` nodes for the cluster can be -determined. As with the ``leader``, there is no ``deputy`` election process, -the deputies can always be recognised deterministically by any node whenever there -is gossip convergence. The list of ``deputy`` nodes is simply the N - 1 number -of nodes (e.g. starting with the first node after the ``leader``) in sorted order. +The deputy nodes are the live members of the configured seed nodes. +It is preferred to use deputy nodes in different racks/data centers. The nodes defined as ``deputy`` nodes are just regular member nodes whose only "special role" is to help breaking logical partitions as seen in the gossip @@ -223,7 +219,7 @@ nodes involved in a gossip exchange. Periodically, the default is every 1 second, each node chooses another random node to initiate a round of gossip with. The choice of node is random but can -also include extra gossiping for unreachable nodes, ``deputy`` nodes, and nodes with +also include extra gossiping for ``deputy`` nodes, and nodes with either newer or older state versions. The gossip overview contains the current state version for all nodes and also a @@ -236,18 +232,14 @@ breaking logical partitions as seen in the gossip algorithm defined below. During each round of gossip exchange the following process is used: -1. Gossip to random live node (if any) +1. Gossip to random node with newer or older state information, if any, based on the + current gossip overview, with some probability. Otherwise Gossip to any random + live node. -2. Gossip to random unreachable node with certain probability depending on the - number of unreachable and live nodes - -3. If the node gossiped to at (1) was not a ``deputy`` node, or the number of live +2. If the node gossiped to at (1) was not a ``deputy`` node, or the number of live nodes is less than number of ``deputy`` nodes, gossip to random ``deputy`` node with certain probability depending on number of unreachable, ``deputy``, and live nodes. -4. Gossip to random node with newer or older state information, based on the - current gossip overview, with some probability (?) - The gossiper only sends the gossip overview to the chosen node. The recipient of the gossip can use the gossip overview to determine whether: @@ -311,10 +303,6 @@ handoff has completed then the node will change to the ``exiting`` state. Once all nodes have seen the exiting state (convergence) the ``leader`` will remove the node from the cluster, marking it as ``removed``. -A node can also be removed forcefully by moving it directly to the ``removed`` -state using the ``remove`` action. The cluster will rebalance based on the new -cluster membership. - If a node is unreachable then gossip convergence is not possible and therefore any ``leader`` actions are also not possible (for instance, allowing a node to become a part of the cluster, or changing actor distribution). To be able to @@ -323,11 +311,12 @@ unreachable node is experiencing only transient difficulties then it can be explicitly marked as ``down`` using the ``down`` user action. When this node comes back up and begins gossiping it will automatically go through the joining process again. If the unreachable node will be permanently down then it can be -removed from the cluster directly with the ``remove`` user action. The cluster -can also *auto-down* a node using the accrual failure detector. +removed from the cluster directly by shutting the actor system down or killing it +through an external ``SIGKILL`` signal, invocation of ``System.exit(status)`` or +similar. The cluster can, through the leader, also *auto-down* a node. -This means that nodes can join and leave the cluster at any point in time, -e.g. provide cluster elasticity. +This means that nodes can join and leave the cluster at any point in time, i.e. +provide cluster elasticity. State Diagram for the Member States @@ -348,12 +337,12 @@ Member States - **leaving** / **exiting** states during graceful removal -- **removed** - tombstone state (no longer a member) - - **down** marked as down/offline/unreachable +- **removed** + tombstone state (no longer a member) + User Actions ^^^^^^^^^^^^ @@ -368,9 +357,6 @@ User Actions - **down** mark a node as temporarily down -- **remove** - remove a node from the cluster immediately - Leader Actions ^^^^^^^^^^^^^^ diff --git a/akka-docs/common/circuitbreaker.rst b/akka-docs/common/circuitbreaker.rst new file mode 100644 index 0000000000..bd13927c8e --- /dev/null +++ b/akka-docs/common/circuitbreaker.rst @@ -0,0 +1,130 @@ +.. _circuit-breaker: + +############### +Circuit Breaker +############### + +================== +Why are they used? +================== +A circuit breaker is used to provide stability and prevent cascading failures in distributed +systems. These should be used in conjunction with judicious timeouts at the interfaces between +remote systems to prevent the failure of a single component from bringing down all components. + +As an example, we have a web application interacting with a remote third party web service. +Let's say the third party has oversold their capacity and their database melts down under load. +Assume that the database fails in such a way that it takes a very long time to hand back an +error to the third party web service. This in turn makes calls fail after a long period of +time. Back to our web application, the users have noticed that their form submissions take +much longer seeming to hang. Well the users do what they know to do which is use the refresh +button, adding more requests to their already running requests. This eventually causes the +failure of the web application due to resource exhaustion. This will affect all users, even +those who are not using functionality dependent on this third party web service. + +Introducing circuit breakers on the web service call would cause the requests to begin to +fail-fast, letting the user know that something is wrong and that they need not refresh +their request. This also confines the failure behavior to only those users that are using +functionality dependent on the third party, other users are no longer affected as there is no +resource exhaustion. Circuit breakers can also allow savvy developers to mark portions of +the site that use the functionality unavailable, or perhaps show some cached content as +appropriate while the breaker is open. + +The Akka library provides an implementation of a circuit breaker called +:class:`akka.pattern.CircuitBreaker` which has the behavior described below. + +================= +What do they do? +================= +* During normal operation, a circuit breaker is in the `Closed` state: + * Exceptions or calls exceeding the configured `callTimeout` increment a failure counter + * Successes reset the failure count to zero + * When the failure counter reaches a `maxFailures` count, the breaker is tripped into `Open` state +* While in `Open` state: + * All calls fail-fast with a :class:`CircuitBreakerOpenException` + * After the configured `resetTimeout`, the circuit breaker enters a `Half-Open` state +* In `Half-Open` state: + * The first call attempted is allowed through without failing fast + * All other calls fail-fast with an exception just as in `Open` state + * If the first call succeeds, the breaker is reset back to `Closed` state + * If the first call fails, the breaker is tripped again into the `Open` state for another full `resetTimeout` +* State transition listeners: + * Callbacks can be provided for every state entry via `onOpen`, `onClose`, and `onHalfOpen` + * These are executed in the :class:`ExecutionContext` provided. + +.. graphviz:: + + digraph circuit_breaker { + rankdir = "LR"; + size = "6,5"; + graph [ bgcolor = "transparent" ] + node [ fontname = "Helvetica", + fontsize = 14, + shape = circle, + color = white, + style = filled ]; + edge [ fontname = "Helvetica", fontsize = 12 ] + Closed [ fillcolor = green2 ]; + "Half-Open" [fillcolor = yellow2 ]; + Open [ fillcolor = red2 ]; + Closed -> Closed [ label = "Success" ]; + "Half-Open" -> Open [ label = "Trip Breaker" ]; + "Half-Open" -> Closed [ label = "Reset Breaker" ]; + Closed -> Open [ label = "Trip Breaker" ]; + Open -> Open [ label = "Calls failing fast" ]; + Open -> "Half-Open" [ label = "Attempt Reset" ]; + } + +======== +Examples +======== + +-------------- +Initialization +-------------- + +Here's how a :class:`CircuitBreaker` would be configured for: + * 5 maximum failures + * a call timeout of 10 seconds + * a reset timeout of 1 minute + +^^^^^^^ +Scala +^^^^^^^ + +.. includecode:: code/docs/circuitbreaker/CircuitBreakerDocSpec.scala + :include: imports1,circuit-breaker-initialization + +^^^^^^^ +Java +^^^^^^^ + +.. includecode:: code/docs/circuitbreaker/DangerousJavaActor.java + :include: imports1,circuit-breaker-initialization + +--------------- +Call Protection +--------------- + +Here's how the :class:`CircuitBreaker` would be used to protect an asynchronous +call as well as a synchronous one: + +^^^^^^^ +Scala +^^^^^^^ + +.. includecode:: code/docs/circuitbreaker/CircuitBreakerDocSpec.scala + :include: circuit-breaker-usage + +^^^^^^ +Java +^^^^^^ + +.. includecode:: code/docs/circuitbreaker/DangerousJavaActor.java + :include: circuit-breaker-usage + +.. note:: + + Using the :class:`CircuitBreaker` companion object's `apply` or `create` methods + will return a :class:`CircuitBreaker` where callbacks are executed in the caller's thread. + This can be useful if the asynchronous :class:`Future` behavior is unnecessary, for + example invoking a synchronous-only API. diff --git a/akka-docs/common/code/docs/circuitbreaker/CircuitBreakerDocSpec.scala b/akka-docs/common/code/docs/circuitbreaker/CircuitBreakerDocSpec.scala new file mode 100644 index 0000000000..c4603017e3 --- /dev/null +++ b/akka-docs/common/code/docs/circuitbreaker/CircuitBreakerDocSpec.scala @@ -0,0 +1,43 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package docs.circuitbreaker + +//#imports1 +import akka.util.duration._ // small d is important here +import akka.pattern.CircuitBreaker +import akka.actor.Actor +import akka.dispatch.Future +import akka.event.Logging + +//#imports1 + +class CircuitBreakerDocSpec {} + +//#circuit-breaker-initialization +class DangerousActor extends Actor { + + val log = Logging(context.system, this) + implicit val executionContext = context.dispatcher + val breaker = + new CircuitBreaker(context.system.scheduler, 5, 10.seconds, 1.minute) + .onOpen(notifyMeOnOpen) + + def notifyMeOnOpen = + log.warning("My CircuitBreaker is now open, and will not close for one minute") + //#circuit-breaker-initialization + + //#circuit-breaker-usage + def dangerousCall: String = "This really isn't that dangerous of a call after all" + + def receive = { + case "is my middle name" ⇒ + sender ! breaker.withCircuitBreaker(Future(dangerousCall)) + case "block for me" ⇒ + sender ! breaker.withSyncCircuitBreaker(dangerousCall) + } + //#circuit-breaker-usage + +} + diff --git a/akka-docs/common/code/docs/circuitbreaker/DangerousJavaActor.java b/akka-docs/common/code/docs/circuitbreaker/DangerousJavaActor.java new file mode 100644 index 0000000000..1562338e04 --- /dev/null +++ b/akka-docs/common/code/docs/circuitbreaker/DangerousJavaActor.java @@ -0,0 +1,83 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package docs.circuitbreaker; + +//#imports1 + +import akka.actor.UntypedActor; +import akka.dispatch.Future; +import akka.event.LoggingAdapter; +import akka.util.Duration; +import akka.pattern.CircuitBreaker; +import akka.event.Logging; + +import static akka.dispatch.Futures.future; + +import java.util.concurrent.Callable; + +//#imports1 + +//#circuit-breaker-initialization +public class DangerousJavaActor extends UntypedActor { + + private final CircuitBreaker breaker; + private final LoggingAdapter log = Logging.getLogger(getContext().system(), this); + + public DangerousJavaActor() { + this.breaker = new CircuitBreaker( + getContext().dispatcher(), getContext().system().scheduler(), + 5, Duration.parse("10s"), Duration.parse("1m")) + .onOpen(new Callable() { + public Object call() throws Exception { + notifyMeOnOpen(); + return null; + } + }); + } + + public void notifyMeOnOpen() { + log.warning("My CircuitBreaker is now open, and will not close for one minute"); + } +//#circuit-breaker-initialization + + //#circuit-breaker-usage + public String dangerousCall() { + return "This really isn't that dangerous of a call after all"; + } + + @Override + public void onReceive(Object message) { + if (message instanceof String) { + String m = (String) message; + if ("is my middle name".equals(m)) { + final Future f = future( + new Callable() { + public String call() { + return dangerousCall(); + } + }, getContext().dispatcher()); + + getSender().tell(breaker + .callWithCircuitBreaker( + new Callable>() { + public Future call() throws Exception { + return f; + } + })); + } + if ("block for me".equals(m)) { + getSender().tell(breaker + .callWithSyncCircuitBreaker( + new Callable() { + @Override + public String call() throws Exception { + return dangerousCall(); + } + })); + } + } + } +//#circuit-breaker-usage + +} \ No newline at end of file diff --git a/akka-docs/common/index.rst b/akka-docs/common/index.rst index 4e19d1a1aa..de9c7016fc 100644 --- a/akka-docs/common/index.rst +++ b/akka-docs/common/index.rst @@ -5,3 +5,4 @@ Common utilities :maxdepth: 2 duration + circuitbreaker diff --git a/akka-docs/conf.py b/akka-docs/conf.py index 2e66d8b56c..77b7c80be0 100644 --- a/akka-docs/conf.py +++ b/akka-docs/conf.py @@ -8,7 +8,7 @@ import sys, os # -- General configuration ----------------------------------------------------- sys.path.append(os.path.abspath('_sphinx/exts')) -extensions = ['sphinx.ext.todo', 'includecode'] +extensions = ['sphinx.ext.todo', 'includecode', 'sphinx.ext.graphviz'] templates_path = ['_templates'] source_suffix = '.rst' @@ -52,6 +52,14 @@ html_context = { 'include_analytics': 'online' in tags } +# -- Options for EPUB output --------------------------------------------------- +epub_author = "Typesafe Inc" +epub_language = "en" +epub_publisher = epub_author +epub_identifier = "http://doc.akka.io/docs/akka/snapshot/" +epub_scheme = "URL" +epub_cover = ("_sphinx/static/akka.png", "") + # -- Options for LaTeX output -------------------------------------------------- def setup(app): diff --git a/akka-docs/dev/multi-jvm-testing.rst b/akka-docs/dev/multi-jvm-testing.rst index 16271c5390..0ba053d2a4 100644 --- a/akka-docs/dev/multi-jvm-testing.rst +++ b/akka-docs/dev/multi-jvm-testing.rst @@ -334,3 +334,10 @@ same machine at the same time. The machines that are used for testing (slaves) should have ssh access to the outside world and be able to talk to each other with the internal addresses given. On the master machine ssh client is required. Obviosly git and sbt should be installed on both master and slave machines. + +The Test Conductor Extension +============================ + +The Test Conductor Extension is aimed at enhancing the multi JVM and multi node testing facilities. + +.. image:: ../images/akka-remote-testconductor.png diff --git a/akka-docs/general/ActorPath.png b/akka-docs/general/ActorPath.png index 08ecb64284..988a2807dd 100644 Binary files a/akka-docs/general/ActorPath.png and b/akka-docs/general/ActorPath.png differ diff --git a/akka-docs/general/actor-systems.rst b/akka-docs/general/actor-systems.rst index 2051f2d845..1b7d6a7759 100644 --- a/akka-docs/general/actor-systems.rst +++ b/akka-docs/general/actor-systems.rst @@ -14,6 +14,11 @@ which means that we need not concern ourselves with their emotional state or moral issues). The result can then serve as a mental scaffolding for building the software implementation. +.. note:: + + An ActorSystem is a heavyweight structure that will allocate 1…N Threads, + so create one per logical application. + Hierarchical Structure ---------------------- diff --git a/akka-docs/general/code/akka/docs/config/ConfigDoc.java b/akka-docs/general/code/docs/config/ConfigDoc.java similarity index 97% rename from akka-docs/general/code/akka/docs/config/ConfigDoc.java rename to akka-docs/general/code/docs/config/ConfigDoc.java index 69d856947f..ee6393fb1a 100644 --- a/akka-docs/general/code/akka/docs/config/ConfigDoc.java +++ b/akka-docs/general/code/docs/config/ConfigDoc.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.config; +package docs.config; import akka.actor.ActorSystem; import com.typesafe.config.*; diff --git a/akka-docs/general/code/akka/docs/config/ConfigDocSpec.scala b/akka-docs/general/code/docs/config/ConfigDocSpec.scala similarity index 97% rename from akka-docs/general/code/akka/docs/config/ConfigDocSpec.scala rename to akka-docs/general/code/docs/config/ConfigDocSpec.scala index 3b7cb10ed2..643116e14f 100644 --- a/akka-docs/general/code/akka/docs/config/ConfigDocSpec.scala +++ b/akka-docs/general/code/docs/config/ConfigDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.config +package docs.config import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers diff --git a/akka-docs/general/configuration.rst b/akka-docs/general/configuration.rst index a0dc9dd49a..1f3f051614 100644 --- a/akka-docs/general/configuration.rst +++ b/akka-docs/general/configuration.rst @@ -65,7 +65,7 @@ When using JarJar, OneJar, Assembly or any jar-bundler Akka's configuration approach relies heavily on the notion of every module/jar having its own reference.conf file, all of these will be discovered by the configuration and loaded. Unfortunately this also means - that if you put merge multiple jars into the same jar, you need to merge all the + that if you put/merge multiple jars into the same jar, you need to merge all the reference.confs as well. Otherwise all defaults will be lost and Akka will not function. Custom application.conf @@ -256,7 +256,7 @@ result:: You may also specify and parse the configuration programmatically in other ways when instantiating the ``ActorSystem``. -.. includecode:: code/akka/docs/config/ConfigDocSpec.scala +.. includecode:: code/docs/config/ConfigDocSpec.scala :include: imports,custom-config Reading configuration from a custom location @@ -301,7 +301,7 @@ you could put a config string in code using You can also combine your custom config with the usual config, that might look like: -.. includecode:: code/akka/docs/config/ConfigDoc.java +.. includecode:: code/docs/config/ConfigDoc.java :include: java-custom-config When working with ``Config`` objects, keep in mind that there are diff --git a/akka-docs/general/message-send-semantics.rst b/akka-docs/general/message-send-semantics.rst index d9488d1f2b..41eb727358 100644 --- a/akka-docs/general/message-send-semantics.rst +++ b/akka-docs/general/message-send-semantics.rst @@ -48,14 +48,14 @@ At-most-once Actual transports may provide stronger semantics, but at-most-once is the semantics you should expect. -The alternatives would be once-and-only-once, which is extremely costly, +The alternatives would be once-and-only-once, which is extremely costly, or at-least-once which essentially requires idempotency of message processing, which is a user-level concern. Ordering is preserved on a per-sender basis ------------------------------------------- -Actor ``A1` sends messages ``M1``, ``M2``, ``M3`` to ``A2`` +Actor ``A1`` sends messages ``M1``, ``M2``, ``M3`` to ``A2`` Actor ``A3`` sends messages ``M4``, ``M5``, ``M6`` to ``A2`` This means that: @@ -66,4 +66,4 @@ This means that: 5) ``A2`` can see messages from ``A1`` interleaved with messages from ``A3`` 6) Since there is no guaranteed delivery, none, some or all of the messages may arrive to ``A2`` -.. _Erlang documentation: http://www.erlang.org/faq/academic.html \ No newline at end of file +.. _Erlang documentation: http://www.erlang.org/faq/academic.html diff --git a/akka-docs/general/supervision.rst b/akka-docs/general/supervision.rst index fef3a585dd..c1bc684ce4 100644 --- a/akka-docs/general/supervision.rst +++ b/akka-docs/general/supervision.rst @@ -55,6 +55,8 @@ actors cannot be orphaned or attached to supervisors from the outside, which might otherwise catch them unawares. In addition, this yields a natural and clean shutdown procedure for (sub-trees of) actor applications. +.. _supervision-restart: + What Restarting Means --------------------- diff --git a/akka-docs/images/akka-remote-testconductor.png b/akka-docs/images/akka-remote-testconductor.png new file mode 100644 index 0000000000..b213538326 Binary files /dev/null and b/akka-docs/images/akka-remote-testconductor.png differ diff --git a/akka-docs/intro/deployment-scenarios.rst b/akka-docs/intro/deployment-scenarios.rst index b2d0334c7d..fc3b38cbd2 100644 --- a/akka-docs/intro/deployment-scenarios.rst +++ b/akka-docs/intro/deployment-scenarios.rst @@ -13,7 +13,7 @@ Akka can be used in different ways: be put into ``WEB-INF/lib`` - As a stand alone application by instantiating ActorSystem in a main class or - using the :ref:`microkernel` + using the :ref:`microkernel-scala` / :ref:`microkernel-java` Using Akka as library @@ -27,5 +27,6 @@ modules to the stack. Using Akka as a stand alone microkernel ---------------------------------------- -Akka can also be run as a stand-alone microkernel. See :ref:`microkernel` for +Akka can also be run as a stand-alone microkernel. See +:ref:`microkernel-scala` / :ref:`microkernel-java` for more information. diff --git a/akka-docs/intro/getting-started.rst b/akka-docs/intro/getting-started.rst index b3bdbf70f3..f96bb42b9d 100644 --- a/akka-docs/intro/getting-started.rst +++ b/akka-docs/intro/getting-started.rst @@ -31,7 +31,7 @@ Modules Akka is very modular and has many JARs for containing different features. -- ``akka-actor-2.1-SNAPSHOT.jar`` -- Classic Actors, Typed Actors, IO Actor etc. Has ZERO dependencies. +- ``akka-actor-2.1-SNAPSHOT.jar`` -- Classic Actors, Typed Actors, IO Actor etc. - ``akka-remote-2.1-SNAPSHOT.jar`` -- Remote Actors - ``akka-testkit-2.1-SNAPSHOT.jar`` -- Toolkit for testing Actor systems - ``akka-kernel-2.1-SNAPSHOT.jar`` -- Akka microkernel for running a bare-bones mini application server @@ -43,8 +43,7 @@ Akka is very modular and has many JARs for containing different features. - ``akka--mailbox-2.1-SNAPSHOT.jar`` -- Akka durable mailboxes How to see the JARs dependencies of each Akka module is described in the -:ref:`dependencies` section. Worth noting is that ``akka-actor`` has zero -external dependencies (apart from the ``scala-library.jar`` JAR). +:ref:`dependencies` section. Using a release distribution ---------------------------- @@ -67,7 +66,8 @@ The Akka distribution includes the microkernel. To run the microkernel put your application jar in the ``deploy`` directory and use the scripts in the ``bin`` directory. -More information is available in the documentation of the :ref:`microkernel`. +More information is available in the documentation of the +:ref:`microkernel-scala` / :ref:`microkernel-java`. Using a build tool ------------------ @@ -136,12 +136,17 @@ SBT installation instructions on `https://github.com/harrah/xsbt/wiki/Setup `_ to generate Eclipse project. +Setup SBT project and then use `sbteclipse `_ to generate a Eclipse project. Using Akka with IntelliJ IDEA ----------------------------- -Setup SBT project and then use `sbt-idea `_ to generate IntelliJ IDEA project. +Setup SBT project and then use `sbt-idea `_ to generate a IntelliJ IDEA project. + +Using Akka with NetBeans +------------------------ + +Setup SBT project and then use `sbt-netbeans-plugin `_ to generate a NetBeans project. Build from sources ------------------ diff --git a/akka-docs/java/agents.rst b/akka-docs/java/agents.rst index 3fcc886d17..0ba7dd90ce 100644 --- a/akka-docs/java/agents.rst +++ b/akka-docs/java/agents.rst @@ -40,17 +40,17 @@ application. An ``ActorSystem`` is required to create the underlying Actors. See Here is an example of creating an Agent: -.. includecode:: code/akka/docs/agent/AgentDocTest.java +.. includecode:: code/docs/agent/AgentDocTest.java :include: import-system,import-agent :language: java -.. includecode:: code/akka/docs/agent/AgentDocTest.java#create +.. includecode:: code/docs/agent/AgentDocTest.java#create :language: java An Agent will be running until you invoke ``close`` on it. Then it will be eligible for garbage collection (unless you hold on to it in some way). -.. includecode:: code/akka/docs/agent/AgentDocTest.java#close +.. includecode:: code/docs/agent/AgentDocTest.java#close :language: java @@ -65,10 +65,10 @@ the update will be applied but dispatches to an Agent from a single thread will occur in order. You apply a value or a function by invoking the ``send`` function. -.. includecode:: code/akka/docs/agent/AgentDocTest.java#import-function +.. includecode:: code/docs/agent/AgentDocTest.java#import-function :language: java -.. includecode:: code/akka/docs/agent/AgentDocTest.java#send +.. includecode:: code/docs/agent/AgentDocTest.java#send :language: java You can also dispatch a function to update the internal state but on its own @@ -77,7 +77,7 @@ long-running or blocking operations. You do this with the ``sendOff`` method. Dispatches using either ``sendOff`` or ``send`` will still be executed in order. -.. includecode:: code/akka/docs/agent/AgentDocTest.java#send-off +.. includecode:: code/docs/agent/AgentDocTest.java#send-off :language: java @@ -87,7 +87,7 @@ Reading an Agent's value Agents can be dereferenced (you can get an Agent's value) by calling the get method: -.. includecode:: code/akka/docs/agent/AgentDocTest.java#read-get +.. includecode:: code/docs/agent/AgentDocTest.java#read-get :language: java Reading an Agent's current value does not involve any message passing and @@ -101,8 +101,8 @@ Awaiting an Agent's value It is also possible to read the value after all currently queued sends have completed. You can do this with ``await``: -.. includecode:: code/akka/docs/agent/AgentDocTest.java#import-timeout +.. includecode:: code/docs/agent/AgentDocTest.java#import-timeout :language: java -.. includecode:: code/akka/docs/agent/AgentDocTest.java#read-await +.. includecode:: code/docs/agent/AgentDocTest.java#read-await :language: java diff --git a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java deleted file mode 100644 index aa24c92249..0000000000 --- a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java +++ /dev/null @@ -1,109 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ -package akka.docs.serialization; - -import org.junit.Test; -import static org.junit.Assert.*; -//#imports -import akka.actor.*; -import akka.serialization.*; -import com.typesafe.config.*; - -//#imports - -public class SerializationDocTestBase { - //#my-own-serializer - public static class MyOwnSerializer extends JSerializer { - - // This is whether "fromBinary" requires a "clazz" or not - @Override public boolean includeManifest() { - return false; - } - - // Pick a unique identifier for your Serializer, - // you've got a couple of billions to choose from, - // 0 - 16 is reserved by Akka itself - @Override public int identifier() { - return 1234567; - } - - // "toBinary" serializes the given object to an Array of Bytes - @Override public byte[] toBinary(Object obj) { - // Put the code that serializes the object here - //#... - return new byte[0]; - //#... - } - - // "fromBinary" deserializes the given array, - // using the type hint (if any, see "includeManifest" above) - @Override public Object fromBinaryJava(byte[] bytes, - Class clazz) { - // Put your code that deserializes here - //#... - return null; - //#... - } - } -//#my-own-serializer - - @Test public void serializeActorRefs() { - final ActorSystem theActorSystem = - ActorSystem.create("whatever"); - final ActorRef theActorRef = - theActorSystem.deadLetters(); // Of course this should be you - - //#actorref-serializer - // Serialize - // (beneath toBinary) - final Address transportAddress = - Serialization.currentTransportAddress().value(); - String identifier; - - // If there is no transportAddress, - // it means that either this Serializer isn't called - // within a piece of code that sets it, - // so either you need to supply your own, - // or simply use the local path. - if (transportAddress == null) identifier = theActorRef.path().toString(); - else identifier = theActorRef.path().toStringWithAddress(transportAddress); - // Then just serialize the identifier however you like - - - // Deserialize - // (beneath fromBinary) - final ActorRef deserializedActorRef = theActorSystem.actorFor(identifier); - // Then just use the ActorRef - //#actorref-serializer - theActorSystem.shutdown(); - } - - - @Test public void demonstrateTheProgrammaticAPI() { - //#programmatic - ActorSystem system = ActorSystem.create("example"); - - // Get the Serialization Extension - Serialization serialization = SerializationExtension.get(system); - - // Have something to serialize - String original = "woohoo"; - - // Find the Serializer for it - Serializer serializer = serialization.findSerializerFor(original); - - // Turn it into bytes - byte[] bytes = serializer.toBinary(original); - - // Turn it back into an object, - // the nulls are for the class manifest and for the classloader - String back = (String)serializer.fromBinary(bytes); - - // Voilá! - assertEquals(original, back); - - //#programmatic - system.shutdown(); - } -} diff --git a/akka-docs/java/code/akka/docs/actor/FSMDocTest.scala b/akka-docs/java/code/docs/actor/FSMDocTest.scala similarity index 87% rename from akka-docs/java/code/akka/docs/actor/FSMDocTest.scala rename to akka-docs/java/code/docs/actor/FSMDocTest.scala index 11bb542808..7077365d6d 100644 --- a/akka-docs/java/code/akka/docs/actor/FSMDocTest.scala +++ b/akka-docs/java/code/docs/actor/FSMDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java b/akka-docs/java/code/docs/actor/FSMDocTestBase.java similarity index 99% rename from akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java rename to akka-docs/java/code/docs/actor/FSMDocTestBase.java index aeaca63f92..9064833cb0 100644 --- a/akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java +++ b/akka-docs/java/code/docs/actor/FSMDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; //#imports-data import java.util.ArrayList; diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTest.scala b/akka-docs/java/code/docs/actor/FaultHandlingTest.scala similarity index 88% rename from akka-docs/java/code/akka/docs/actor/FaultHandlingTest.scala rename to akka-docs/java/code/docs/actor/FaultHandlingTest.scala index 03802d6968..9b6fad0609 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTest.scala +++ b/akka-docs/java/code/docs/actor/FaultHandlingTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor import org.scalatest.junit.JUnitSuite class FaultHandlingTest extends FaultHandlingTestBase with JUnitSuite diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/docs/actor/FaultHandlingTestBase.java similarity index 98% rename from akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java rename to akka-docs/java/code/docs/actor/FaultHandlingTestBase.java index dc2ce9bae7..2d40071fe8 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/docs/actor/FaultHandlingTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; //#testkit import akka.actor.ActorRef; @@ -182,7 +182,7 @@ public class FaultHandlingTestBase { final TestProbe probe = new TestProbe(system); probe.watch(child); child.tell(new IllegalArgumentException()); - probe.expectMsg(new Terminated(child)); + probe.expectMsg(new Terminated(child, true)); //#stop //#escalate-kill @@ -190,7 +190,7 @@ public class FaultHandlingTestBase { probe.watch(child); assert Await.result(ask(child, "get", 5000), timeout).equals(0); child.tell(new Exception()); - probe.expectMsg(new Terminated(child)); + probe.expectMsg(new Terminated(child, true)); //#escalate-kill //#escalate-restart diff --git a/akka-docs/java/code/akka/docs/actor/FirstUntypedActor.java b/akka-docs/java/code/docs/actor/FirstUntypedActor.java similarity index 95% rename from akka-docs/java/code/akka/docs/actor/FirstUntypedActor.java rename to akka-docs/java/code/docs/actor/FirstUntypedActor.java index 6cfbe75b99..fa5d3d35a0 100644 --- a/akka-docs/java/code/akka/docs/actor/FirstUntypedActor.java +++ b/akka-docs/java/code/docs/actor/FirstUntypedActor.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; import akka.actor.ActorRef; import akka.actor.Props; diff --git a/akka-docs/java/code/akka/docs/actor/ImmutableMessage.java b/akka-docs/java/code/docs/actor/ImmutableMessage.java similarity index 96% rename from akka-docs/java/code/akka/docs/actor/ImmutableMessage.java rename to akka-docs/java/code/docs/actor/ImmutableMessage.java index 41bc4eb0e5..60e72ecfb5 100644 --- a/akka-docs/java/code/akka/docs/actor/ImmutableMessage.java +++ b/akka-docs/java/code/docs/actor/ImmutableMessage.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; import java.util.ArrayList; import java.util.Collections; diff --git a/akka-docs/java/code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java b/akka-docs/java/code/docs/actor/MyReceivedTimeoutUntypedActor.java similarity index 96% rename from akka-docs/java/code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java rename to akka-docs/java/code/docs/actor/MyReceivedTimeoutUntypedActor.java index 97742d9bd1..025d634b09 100644 --- a/akka-docs/java/code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java +++ b/akka-docs/java/code/docs/actor/MyReceivedTimeoutUntypedActor.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; //#receive-timeout import akka.actor.ReceiveTimeout; diff --git a/akka-docs/java/code/akka/docs/actor/MyUntypedActor.java b/akka-docs/java/code/docs/actor/MyUntypedActor.java similarity index 95% rename from akka-docs/java/code/akka/docs/actor/MyUntypedActor.java rename to akka-docs/java/code/docs/actor/MyUntypedActor.java index 93a817ef2c..f31fc402c7 100644 --- a/akka-docs/java/code/akka/docs/actor/MyUntypedActor.java +++ b/akka-docs/java/code/docs/actor/MyUntypedActor.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; //#my-untyped-actor import akka.actor.UntypedActor; diff --git a/akka-docs/java/code/akka/docs/actor/SchedulerDocTest.scala b/akka-docs/java/code/docs/actor/SchedulerDocTest.scala similarity index 88% rename from akka-docs/java/code/akka/docs/actor/SchedulerDocTest.scala rename to akka-docs/java/code/docs/actor/SchedulerDocTest.scala index ecad03213e..9e6b4c9613 100644 --- a/akka-docs/java/code/akka/docs/actor/SchedulerDocTest.scala +++ b/akka-docs/java/code/docs/actor/SchedulerDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/actor/SchedulerDocTestBase.java b/akka-docs/java/code/docs/actor/SchedulerDocTestBase.java similarity index 98% rename from akka-docs/java/code/akka/docs/actor/SchedulerDocTestBase.java rename to akka-docs/java/code/docs/actor/SchedulerDocTestBase.java index 7a58da0f5e..d7e8fa644f 100644 --- a/akka-docs/java/code/akka/docs/actor/SchedulerDocTestBase.java +++ b/akka-docs/java/code/docs/actor/SchedulerDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; //#imports1 import akka.actor.Props; diff --git a/akka-docs/java/code/akka/docs/actor/TypedActorDocTest.scala b/akka-docs/java/code/docs/actor/TypedActorDocTest.scala similarity index 88% rename from akka-docs/java/code/akka/docs/actor/TypedActorDocTest.scala rename to akka-docs/java/code/docs/actor/TypedActorDocTest.scala index 476d570b4a..0d9796ca56 100644 --- a/akka-docs/java/code/akka/docs/actor/TypedActorDocTest.scala +++ b/akka-docs/java/code/docs/actor/TypedActorDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/actor/TypedActorDocTestBase.java b/akka-docs/java/code/docs/actor/TypedActorDocTestBase.java similarity index 89% rename from akka-docs/java/code/akka/docs/actor/TypedActorDocTestBase.java rename to akka-docs/java/code/docs/actor/TypedActorDocTestBase.java index 30db92ee0f..fdd677c78b 100644 --- a/akka-docs/java/code/akka/docs/actor/TypedActorDocTestBase.java +++ b/akka-docs/java/code/docs/actor/TypedActorDocTestBase.java @@ -1,10 +1,11 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; //#imports +import akka.actor.TypedActor; import akka.dispatch.*; import akka.actor.*; import akka.japi.*; @@ -151,6 +152,21 @@ public class TypedActorDocTestBase { } } + @Test public void createHierarchies() { + try { + //#typed-actor-hierarchy + Squarer childSquarer = + TypedActor.get(TypedActor.context()). + typedActorOf( + new TypedProps(Squarer.class, SquarerImpl.class) + ); + //Use "childSquarer" as a Squarer + //#typed-actor-hierarchy + } catch (Exception e) { + //dun care + } + } + @Test public void proxyAnyActorRef() { try { //#typed-actor-remote diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTest.scala b/akka-docs/java/code/docs/actor/UntypedActorDocTest.scala similarity index 88% rename from akka-docs/java/code/akka/docs/actor/UntypedActorDocTest.scala rename to akka-docs/java/code/docs/actor/UntypedActorDocTest.scala index e341914c8c..8047b94df9 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTest.scala +++ b/akka-docs/java/code/docs/actor/UntypedActorDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java b/akka-docs/java/code/docs/actor/UntypedActorDocTestBase.java similarity index 98% rename from akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java rename to akka-docs/java/code/docs/actor/UntypedActorDocTestBase.java index 65ff37c10e..c82ce30661 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/java/code/docs/actor/UntypedActorDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; //#imports import akka.actor.ActorRef; @@ -36,7 +36,7 @@ import static akka.pattern.Patterns.gracefulStop; import akka.dispatch.Future; import akka.dispatch.Await; import akka.util.Duration; -import akka.actor.ActorTimeoutException; +import akka.pattern.AskTimeoutException; //#import-gracefulStop //#import-askPipe @@ -207,7 +207,7 @@ public class UntypedActorDocTestBase { Future stopped = gracefulStop(actorRef, Duration.create(5, TimeUnit.SECONDS), system); Await.result(stopped, Duration.create(6, TimeUnit.SECONDS)); // the actor has been stopped - } catch (ActorTimeoutException e) { + } catch (AskTimeoutException e) { // the actor wasn't stopped within 5 seconds } //#gracefulStop diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorSwapper.java b/akka-docs/java/code/docs/actor/UntypedActorSwapper.java similarity index 93% rename from akka-docs/java/code/akka/docs/actor/UntypedActorSwapper.java rename to akka-docs/java/code/docs/actor/UntypedActorSwapper.java index b2fb98c305..985c75bfd7 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorSwapper.java +++ b/akka-docs/java/code/docs/actor/UntypedActorSwapper.java @@ -1,9 +1,9 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor; +package docs.actor; -import static akka.docs.actor.UntypedActorSwapper.Swap.SWAP; +import static docs.actor.UntypedActorSwapper.Swap.SWAP; import akka.actor.ActorRef; import akka.actor.Props; import akka.actor.ActorSystem; diff --git a/akka-docs/java/code/akka/docs/actor/japi/FaultHandlingDocSample.java b/akka-docs/java/code/docs/actor/japi/FaultHandlingDocSample.java similarity index 97% rename from akka-docs/java/code/akka/docs/actor/japi/FaultHandlingDocSample.java rename to akka-docs/java/code/docs/actor/japi/FaultHandlingDocSample.java index 4ba8358174..1213ab5949 100644 --- a/akka-docs/java/code/akka/docs/actor/japi/FaultHandlingDocSample.java +++ b/akka-docs/java/code/docs/actor/japi/FaultHandlingDocSample.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor.japi; +package docs.actor.japi; //#all //#imports @@ -26,10 +26,10 @@ import static akka.actor.SupervisorStrategy.*; import static akka.pattern.Patterns.ask; import static akka.pattern.Patterns.pipe; -import static akka.docs.actor.japi.FaultHandlingDocSample.WorkerApi.*; -import static akka.docs.actor.japi.FaultHandlingDocSample.CounterServiceApi.*; -import static akka.docs.actor.japi.FaultHandlingDocSample.CounterApi.*; -import static akka.docs.actor.japi.FaultHandlingDocSample.StorageApi.*; +import static docs.actor.japi.FaultHandlingDocSample.WorkerApi.*; +import static docs.actor.japi.FaultHandlingDocSample.CounterServiceApi.*; +import static docs.actor.japi.FaultHandlingDocSample.CounterApi.*; +import static docs.actor.japi.FaultHandlingDocSample.StorageApi.*; //#imports diff --git a/akka-docs/java/code/akka/docs/agent/AgentDocJavaSpec.scala b/akka-docs/java/code/docs/agent/AgentDocJavaSpec.scala similarity index 78% rename from akka-docs/java/code/akka/docs/agent/AgentDocJavaSpec.scala rename to akka-docs/java/code/docs/agent/AgentDocJavaSpec.scala index c3c0c296ed..566a439c62 100644 --- a/akka-docs/java/code/akka/docs/agent/AgentDocJavaSpec.scala +++ b/akka-docs/java/code/docs/agent/AgentDocJavaSpec.scala @@ -1,10 +1,10 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.agent +package docs.agent import org.scalatest.junit.JUnitWrapperSuite class AgentDocJavaSpec extends JUnitWrapperSuite( - "akka.docs.agent.AgentDocTest", + "docs.agent.AgentDocTest", Thread.currentThread.getContextClassLoader) \ No newline at end of file diff --git a/akka-docs/java/code/akka/docs/agent/AgentDocTest.java b/akka-docs/java/code/docs/agent/AgentDocTest.java similarity index 98% rename from akka-docs/java/code/akka/docs/agent/AgentDocTest.java rename to akka-docs/java/code/docs/agent/AgentDocTest.java index 553d64eee5..0da96ebfc9 100644 --- a/akka-docs/java/code/akka/docs/agent/AgentDocTest.java +++ b/akka-docs/java/code/docs/agent/AgentDocTest.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.agent; +package docs.agent; import static org.junit.Assert.*; diff --git a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTest.scala b/akka-docs/java/code/docs/dispatcher/DispatcherDocTest.scala similarity index 86% rename from akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTest.scala rename to akka-docs/java/code/docs/dispatcher/DispatcherDocTest.scala index 8216c36757..62c9e37051 100644 --- a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTest.scala +++ b/akka-docs/java/code/docs/dispatcher/DispatcherDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.dispatcher +package docs.dispatcher import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java b/akka-docs/java/code/docs/dispatcher/DispatcherDocTestBase.java similarity index 71% rename from akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java rename to akka-docs/java/code/docs/dispatcher/DispatcherDocTestBase.java index f080dd52b9..ca5569657e 100644 --- a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java +++ b/akka-docs/java/code/docs/dispatcher/DispatcherDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.dispatcher; +package docs.dispatcher; //#imports import akka.actor.*; @@ -24,6 +24,15 @@ import com.typesafe.config.Config; //#imports-prio-mailbox +//#imports-custom +import akka.dispatch.Envelope; +import akka.dispatch.MessageQueue; +import akka.dispatch.MailboxType; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + +//#imports-custom + import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -32,8 +41,8 @@ import static org.junit.Assert.*; import com.typesafe.config.ConfigFactory; -import akka.docs.actor.MyUntypedActor; -import akka.docs.actor.UntypedActorDocTestBase.MyActor; +import docs.actor.MyUntypedActor; +import docs.actor.UntypedActorDocTestBase.MyActor; import akka.testkit.AkkaSpec; public class DispatcherDocTestBase { @@ -136,4 +145,32 @@ public class DispatcherDocTestBase { } } //#prio-mailbox + + //#mailbox-implementation-example + class MyUnboundedMailbox implements MailboxType { + + // This constructor signature must exist, it will be called by Akka + public MyUnboundedMailbox(ActorSystem.Settings settings, Config config) { + // put your initialization code here + } + + // The create method is called to create the MessageQueue + public MessageQueue create(Option owner, Option system) { + return new MessageQueue() { + private final Queue queue = new ConcurrentLinkedQueue(); + + // these must be implemented; queue used as example + public void enqueue(ActorRef receiver, Envelope handle) { queue.offer(handle); } + public Envelope dequeue() { return queue.poll(); } + public int numberOfMessages() { return queue.size(); } + public boolean hasMessages() { return !queue.isEmpty(); } + public void cleanUp(ActorRef owner, MessageQueue deadLetters) { + for (Envelope handle: queue) { + deadLetters.enqueue(owner, handle); + } + } + }; + } + } + //#mailbox-implementation-example } diff --git a/akka-docs/java/code/akka/docs/event/LoggingDocTest.scala b/akka-docs/java/code/docs/event/LoggingDocTest.scala similarity index 88% rename from akka-docs/java/code/akka/docs/event/LoggingDocTest.scala rename to akka-docs/java/code/docs/event/LoggingDocTest.scala index ee44f502a4..1d7f34827f 100644 --- a/akka-docs/java/code/akka/docs/event/LoggingDocTest.scala +++ b/akka-docs/java/code/docs/event/LoggingDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.event +package docs.event import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java b/akka-docs/java/code/docs/event/LoggingDocTestBase.java similarity index 99% rename from akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java rename to akka-docs/java/code/docs/event/LoggingDocTestBase.java index 8f7b63d8a8..77e46b3f92 100644 --- a/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java +++ b/akka-docs/java/code/docs/event/LoggingDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.event; +package docs.event; //#imports import akka.event.Logging; diff --git a/akka-docs/java/code/akka/docs/extension/ExtensionDocTest.scala b/akka-docs/java/code/docs/extension/ExtensionDocTest.scala similarity index 86% rename from akka-docs/java/code/akka/docs/extension/ExtensionDocTest.scala rename to akka-docs/java/code/docs/extension/ExtensionDocTest.scala index 7b1b43b6ca..f22e300cfc 100644 --- a/akka-docs/java/code/akka/docs/extension/ExtensionDocTest.scala +++ b/akka-docs/java/code/docs/extension/ExtensionDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.extension +package docs.extension import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java b/akka-docs/java/code/docs/extension/ExtensionDocTestBase.java similarity index 98% rename from akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java rename to akka-docs/java/code/docs/extension/ExtensionDocTestBase.java index 11dfe4c198..7623d1cc0a 100644 --- a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java +++ b/akka-docs/java/code/docs/extension/ExtensionDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.extension; +package docs.extension; //#imports import akka.actor.*; diff --git a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTest.scala b/akka-docs/java/code/docs/extension/SettingsExtensionDocTest.scala similarity index 87% rename from akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTest.scala rename to akka-docs/java/code/docs/extension/SettingsExtensionDocTest.scala index 0979c00d4f..60289bfdca 100644 --- a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTest.scala +++ b/akka-docs/java/code/docs/extension/SettingsExtensionDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.extension +package docs.extension import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java b/akka-docs/java/code/docs/extension/SettingsExtensionDocTestBase.java similarity index 98% rename from akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java rename to akka-docs/java/code/docs/extension/SettingsExtensionDocTestBase.java index 9aef290ecb..265c91b206 100644 --- a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java +++ b/akka-docs/java/code/docs/extension/SettingsExtensionDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.extension; +package docs.extension; //#imports import akka.actor.Extension; diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTest.scala b/akka-docs/java/code/docs/future/FutureDocTest.scala similarity index 87% rename from akka-docs/java/code/akka/docs/future/FutureDocTest.scala rename to akka-docs/java/code/docs/future/FutureDocTest.scala index 8716beeced..fef5f3d967 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTest.scala +++ b/akka-docs/java/code/docs/future/FutureDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.future +package docs.future import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/docs/future/FutureDocTestBase.java similarity index 99% rename from akka-docs/java/code/akka/docs/future/FutureDocTestBase.java rename to akka-docs/java/code/docs/future/FutureDocTestBase.java index d8e59f5d30..2fe2220223 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/docs/future/FutureDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.future; +package docs.future; //#imports1 import akka.dispatch.*; diff --git a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTest.scala b/akka-docs/java/code/docs/jrouting/CustomRouterDocTest.scala similarity index 80% rename from akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTest.scala rename to akka-docs/java/code/docs/jrouting/CustomRouterDocTest.scala index 48e323c634..d11b07f22a 100644 --- a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTest.scala +++ b/akka-docs/java/code/docs/jrouting/CustomRouterDocTest.scala @@ -1,4 +1,4 @@ -package akka.docs.jrouting; +package docs.jrouting; import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java b/akka-docs/java/code/docs/jrouting/CustomRouterDocTestBase.java similarity index 95% rename from akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java rename to akka-docs/java/code/docs/jrouting/CustomRouterDocTestBase.java index dc4d140ec3..74e7759b62 100644 --- a/akka-docs/java/code/akka/docs/jrouting/CustomRouterDocTestBase.java +++ b/akka-docs/java/code/docs/jrouting/CustomRouterDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.jrouting; +package docs.jrouting; import java.util.List; import java.util.Arrays; @@ -22,9 +22,9 @@ import akka.testkit.AkkaSpec; import com.typesafe.config.ConfigFactory; import static akka.pattern.Patterns.ask; -import static akka.docs.jrouting.CustomRouterDocTestBase.DemocratActor; -import static akka.docs.jrouting.CustomRouterDocTestBase.RepublicanActor; -import static akka.docs.jrouting.CustomRouterDocTestBase.Message.*; +import static docs.jrouting.CustomRouterDocTestBase.DemocratActor; +import static docs.jrouting.CustomRouterDocTestBase.RepublicanActor; +import static docs.jrouting.CustomRouterDocTestBase.Message.*; public class CustomRouterDocTestBase { diff --git a/akka-docs/java/code/akka/docs/jrouting/FibonacciActor.java b/akka-docs/java/code/docs/jrouting/FibonacciActor.java similarity index 97% rename from akka-docs/java/code/akka/docs/jrouting/FibonacciActor.java rename to akka-docs/java/code/docs/jrouting/FibonacciActor.java index 8e426cf8fe..e316f27bce 100644 --- a/akka-docs/java/code/akka/docs/jrouting/FibonacciActor.java +++ b/akka-docs/java/code/docs/jrouting/FibonacciActor.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.jrouting; +package docs.jrouting; import java.io.Serializable; diff --git a/akka-docs/java/code/akka/docs/jrouting/ParentActor.java b/akka-docs/java/code/docs/jrouting/ParentActor.java similarity index 98% rename from akka-docs/java/code/akka/docs/jrouting/ParentActor.java rename to akka-docs/java/code/docs/jrouting/ParentActor.java index 32a33b3a1b..ada9e92138 100644 --- a/akka-docs/java/code/akka/docs/jrouting/ParentActor.java +++ b/akka-docs/java/code/docs/jrouting/ParentActor.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.jrouting; +package docs.jrouting; import akka.routing.ScatterGatherFirstCompletedRouter; import akka.routing.BroadcastRouter; diff --git a/akka-docs/java/code/akka/docs/jrouting/PrintlnActor.java b/akka-docs/java/code/docs/jrouting/PrintlnActor.java similarity index 92% rename from akka-docs/java/code/akka/docs/jrouting/PrintlnActor.java rename to akka-docs/java/code/docs/jrouting/PrintlnActor.java index d6ad652ebe..adf56fe863 100644 --- a/akka-docs/java/code/akka/docs/jrouting/PrintlnActor.java +++ b/akka-docs/java/code/docs/jrouting/PrintlnActor.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.jrouting; +package docs.jrouting; import akka.actor.UntypedActor; diff --git a/akka-docs/java/code/akka/docs/jrouting/RouterViaConfigExample.java b/akka-docs/java/code/docs/jrouting/RouterViaConfigExample.java similarity index 98% rename from akka-docs/java/code/akka/docs/jrouting/RouterViaConfigExample.java rename to akka-docs/java/code/docs/jrouting/RouterViaConfigExample.java index 61b9a573d7..1505766196 100644 --- a/akka-docs/java/code/akka/docs/jrouting/RouterViaConfigExample.java +++ b/akka-docs/java/code/docs/jrouting/RouterViaConfigExample.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.jrouting; +package docs.jrouting; import akka.routing.FromConfig; import akka.actor.ActorRef; diff --git a/akka-docs/java/code/akka/docs/jrouting/RouterViaProgramExample.java b/akka-docs/java/code/docs/jrouting/RouterViaProgramExample.java similarity index 95% rename from akka-docs/java/code/akka/docs/jrouting/RouterViaProgramExample.java rename to akka-docs/java/code/docs/jrouting/RouterViaProgramExample.java index 44984c3ec7..72843b44c6 100644 --- a/akka-docs/java/code/akka/docs/jrouting/RouterViaProgramExample.java +++ b/akka-docs/java/code/docs/jrouting/RouterViaProgramExample.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.jrouting; +package docs.jrouting; import akka.routing.RoundRobinRouter; import akka.routing.DefaultResizer; @@ -55,7 +55,7 @@ public class RouterViaProgramExample { ActorRef actor2 = system.actorOf(new Props(ExampleActor.class)); ActorRef actor3 = system.actorOf(new Props(ExampleActor.class)); Iterable routees = Arrays.asList(new ActorRef[] { actor1, actor2, actor3 }); - ActorRef router2 = system.actorOf(new Props(ExampleActor.class).withRouter(RoundRobinRouter.create(routees))); + ActorRef router2 = system.actorOf(new Props().withRouter(RoundRobinRouter.create(routees))); //#programmaticRoutingRoutees for (int i = 1; i <= 6; i++) { router2.tell(new ExampleActor.Message(i)); diff --git a/akka-docs/java/code/docs/pattern/JavaTemplate.java b/akka-docs/java/code/docs/pattern/JavaTemplate.java new file mode 100644 index 0000000000..7e6fd175fb --- /dev/null +++ b/akka-docs/java/code/docs/pattern/JavaTemplate.java @@ -0,0 +1,18 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package docs.pattern; + +// this part will not appear in the docs + +//#all-of-it +class JavaTemplate { + public JavaTemplate() { + System.out.println("Hello, Template!"); + } + //#uninteresting-stuff + // don’t show this plumbimg + //#uninteresting-stuff +} +//#all-of-it diff --git a/akka-docs/java/code/akka/docs/remoting/RemoteActorExample.java b/akka-docs/java/code/docs/remoting/RemoteActorExample.java similarity index 95% rename from akka-docs/java/code/akka/docs/remoting/RemoteActorExample.java rename to akka-docs/java/code/docs/remoting/RemoteActorExample.java index f7686e744a..3ca25bd153 100644 --- a/akka-docs/java/code/akka/docs/remoting/RemoteActorExample.java +++ b/akka-docs/java/code/docs/remoting/RemoteActorExample.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.remoting; +package docs.remoting; import akka.actor.ActorRef; import akka.actor.UntypedActor; diff --git a/akka-docs/java/code/akka/docs/remoting/RemoteDeploymentDocTest.scala b/akka-docs/java/code/docs/remoting/RemoteDeploymentDocTest.scala similarity index 87% rename from akka-docs/java/code/akka/docs/remoting/RemoteDeploymentDocTest.scala rename to akka-docs/java/code/docs/remoting/RemoteDeploymentDocTest.scala index 9290b7c897..4ac46c4504 100644 --- a/akka-docs/java/code/akka/docs/remoting/RemoteDeploymentDocTest.scala +++ b/akka-docs/java/code/docs/remoting/RemoteDeploymentDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.remoting +package docs.remoting import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/remoting/RemoteDeploymentDocTestBase.java b/akka-docs/java/code/docs/remoting/RemoteDeploymentDocTestBase.java similarity index 97% rename from akka-docs/java/code/akka/docs/remoting/RemoteDeploymentDocTestBase.java rename to akka-docs/java/code/docs/remoting/RemoteDeploymentDocTestBase.java index b105e2b42a..cfb12ac7c4 100644 --- a/akka-docs/java/code/akka/docs/remoting/RemoteDeploymentDocTestBase.java +++ b/akka-docs/java/code/docs/remoting/RemoteDeploymentDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.remoting; +package docs.remoting; import org.junit.AfterClass; import org.junit.BeforeClass; diff --git a/akka-docs/java/code/akka/docs/serialization/SerializationDocTest.scala b/akka-docs/java/code/docs/serialization/SerializationDocTest.scala similarity index 85% rename from akka-docs/java/code/akka/docs/serialization/SerializationDocTest.scala rename to akka-docs/java/code/docs/serialization/SerializationDocTest.scala index 26685dea80..ffac606928 100644 --- a/akka-docs/java/code/akka/docs/serialization/SerializationDocTest.scala +++ b/akka-docs/java/code/docs/serialization/SerializationDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.serialization +package docs.serialization import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/docs/serialization/SerializationDocTestBase.java b/akka-docs/java/code/docs/serialization/SerializationDocTestBase.java new file mode 100644 index 0000000000..5d27e4f37f --- /dev/null +++ b/akka-docs/java/code/docs/serialization/SerializationDocTestBase.java @@ -0,0 +1,196 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package docs.serialization; + +import org.junit.Test; +import static org.junit.Assert.*; +//#imports +import akka.actor.*; +import akka.remote.RemoteActorRefProvider; +import akka.serialization.*; +import com.typesafe.config.*; + +//#imports + +public class SerializationDocTestBase { + //#my-own-serializer + public static class MyOwnSerializer extends JSerializer { + + // This is whether "fromBinary" requires a "clazz" or not + @Override public boolean includeManifest() { + return false; + } + + // Pick a unique identifier for your Serializer, + // you've got a couple of billions to choose from, + // 0 - 16 is reserved by Akka itself + @Override public int identifier() { + return 1234567; + } + + // "toBinary" serializes the given object to an Array of Bytes + @Override public byte[] toBinary(Object obj) { + // Put the code that serializes the object here + //#... + return new byte[0]; + //#... + } + + // "fromBinary" deserializes the given array, + // using the type hint (if any, see "includeManifest" above) + @Override public Object fromBinaryJava(byte[] bytes, + Class clazz) { + // Put your code that deserializes here + //#... + return null; + //#... + } + } +//#my-own-serializer + + @Test public void serializeActorRefs() { + final ActorSystem theActorSystem = + ActorSystem.create("whatever"); + final ActorRef theActorRef = + theActorSystem.deadLetters(); // Of course this should be you + + //#actorref-serializer + // Serialize + // (beneath toBinary) + final Address transportAddress = + Serialization.currentTransportAddress().value(); + String identifier; + + // If there is no transportAddress, + // it means that either this Serializer isn't called + // within a piece of code that sets it, + // so either you need to supply your own, + // or simply use the local path. + if (transportAddress == null) identifier = theActorRef.path().toString(); + else identifier = theActorRef.path().toStringWithAddress(transportAddress); + // Then just serialize the identifier however you like + + + // Deserialize + // (beneath fromBinary) + final ActorRef deserializedActorRef = theActorSystem.actorFor(identifier); + // Then just use the ActorRef + //#actorref-serializer + theActorSystem.shutdown(); + } + + //#external-address + public static class ExternalAddressExt implements Extension { + private final ExtendedActorSystem system; + + public ExternalAddressExt(ExtendedActorSystem system) { + this.system = system; + } + + public Address getAddressFor(Address remoteAddress) { + final scala.Option
optAddr = system.provider() + .getExternalAddressFor(remoteAddress); + if (optAddr.isDefined()) { + return optAddr.get(); + } else { + throw new UnsupportedOperationException( + "cannot send to remote address " + remoteAddress); + } + } + } + + public static class ExternalAddress extends + AbstractExtensionId implements ExtensionIdProvider { + public static final ExternalAddress ID = new ExternalAddress(); + + public ExternalAddress lookup() { + return ID; + } + + public ExternalAddressExt createExtension(ExtendedActorSystem system) { + return new ExternalAddressExt(system); + } + } + + //#external-address + + public void demonstrateExternalAddress() { + // this is not meant to be run, only to be compiled + final ActorSystem system = ActorSystem.create(); + final Address remoteAddr = new Address("", ""); + // #external-address + final Address addr = ExternalAddress.ID.get(system).getAddressFor(remoteAddr); + // #external-address + } + + //#external-address-default + public static class DefaultAddressExt implements Extension { + private final ExtendedActorSystem system; + + public DefaultAddressExt(ExtendedActorSystem system) { + this.system = system; + } + + public Address getAddress() { + final ActorRefProvider provider = system.provider(); + if (provider instanceof RemoteActorRefProvider) { + return ((RemoteActorRefProvider) provider).transport().address(); + } else { + throw new UnsupportedOperationException("need RemoteActorRefProvider"); + } + } + } + + public static class DefaultAddress extends + AbstractExtensionId implements ExtensionIdProvider { + public static final DefaultAddress ID = new DefaultAddress(); + + public DefaultAddress lookup() { + return ID; + } + + public DefaultAddressExt createExtension(ExtendedActorSystem system) { + return new DefaultAddressExt(system); + } + } + + //#external-address-default + + public void demonstrateDefaultAddress() { + // this is not meant to be run, only to be compiled + final ActorSystem system = ActorSystem.create(); + final Address remoteAddr = new Address("", ""); + // #external-address-default + final Address addr = DefaultAddress.ID.get(system).getAddress(); + // #external-address-default + } + + @Test + public void demonstrateTheProgrammaticAPI() { + // #programmatic + ActorSystem system = ActorSystem.create("example"); + + // Get the Serialization Extension + Serialization serialization = SerializationExtension.get(system); + + // Have something to serialize + String original = "woohoo"; + + // Find the Serializer for it + Serializer serializer = serialization.findSerializerFor(original); + + // Turn it into bytes + byte[] bytes = serializer.toBinary(original); + + // Turn it back into an object, + // the nulls are for the class manifest and for the classloader + String back = (String) serializer.fromBinary(bytes); + + // Voilá! + assertEquals(original, back); + + // #programmatic + system.shutdown(); + } +} diff --git a/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java b/akka-docs/java/code/docs/transactor/CoordinatedCounter.java similarity index 97% rename from akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java rename to akka-docs/java/code/docs/transactor/CoordinatedCounter.java index dd7f119005..4bd679f1eb 100644 --- a/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java +++ b/akka-docs/java/code/docs/transactor/CoordinatedCounter.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.transactor; +package docs.transactor; //#class import akka.actor.*; diff --git a/akka-docs/java/code/akka/docs/transactor/Coordinator.java b/akka-docs/java/code/docs/transactor/Coordinator.java similarity index 96% rename from akka-docs/java/code/akka/docs/transactor/Coordinator.java rename to akka-docs/java/code/docs/transactor/Coordinator.java index f1f04761cd..644eb4312e 100644 --- a/akka-docs/java/code/akka/docs/transactor/Coordinator.java +++ b/akka-docs/java/code/docs/transactor/Coordinator.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.transactor; +package docs.transactor; import akka.actor.*; import akka.transactor.*; diff --git a/akka-docs/java/code/akka/docs/transactor/Counter.java b/akka-docs/java/code/docs/transactor/Counter.java similarity index 95% rename from akka-docs/java/code/akka/docs/transactor/Counter.java rename to akka-docs/java/code/docs/transactor/Counter.java index ea2291afeb..06092c5db0 100644 --- a/akka-docs/java/code/akka/docs/transactor/Counter.java +++ b/akka-docs/java/code/docs/transactor/Counter.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.transactor; +package docs.transactor; //#class import akka.transactor.*; diff --git a/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java b/akka-docs/java/code/docs/transactor/FriendlyCounter.java similarity index 97% rename from akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java rename to akka-docs/java/code/docs/transactor/FriendlyCounter.java index 18f2137ea4..f24c044750 100644 --- a/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java +++ b/akka-docs/java/code/docs/transactor/FriendlyCounter.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.transactor; +package docs.transactor; //#class import akka.actor.*; diff --git a/akka-docs/java/code/akka/docs/transactor/Increment.java b/akka-docs/java/code/docs/transactor/Increment.java similarity index 93% rename from akka-docs/java/code/akka/docs/transactor/Increment.java rename to akka-docs/java/code/docs/transactor/Increment.java index 1d789c99e2..3794ce631d 100644 --- a/akka-docs/java/code/akka/docs/transactor/Increment.java +++ b/akka-docs/java/code/docs/transactor/Increment.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.transactor; +package docs.transactor; //#class import akka.actor.ActorRef; diff --git a/akka-docs/java/code/akka/docs/transactor/Message.java b/akka-docs/java/code/docs/transactor/Message.java similarity index 77% rename from akka-docs/java/code/akka/docs/transactor/Message.java rename to akka-docs/java/code/docs/transactor/Message.java index 6a8da72070..0f1edfc51f 100644 --- a/akka-docs/java/code/akka/docs/transactor/Message.java +++ b/akka-docs/java/code/docs/transactor/Message.java @@ -2,6 +2,6 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.transactor; +package docs.transactor; public class Message {} diff --git a/akka-docs/java/code/akka/docs/transactor/TransactorDocJavaSpec.scala b/akka-docs/java/code/docs/transactor/TransactorDocJavaSpec.scala similarity index 75% rename from akka-docs/java/code/akka/docs/transactor/TransactorDocJavaSpec.scala rename to akka-docs/java/code/docs/transactor/TransactorDocJavaSpec.scala index 4c61a156e8..6293b2effa 100644 --- a/akka-docs/java/code/akka/docs/transactor/TransactorDocJavaSpec.scala +++ b/akka-docs/java/code/docs/transactor/TransactorDocJavaSpec.scala @@ -2,10 +2,10 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.transactor +package docs.transactor import org.scalatest.junit.JUnitWrapperSuite class TransactorDocJavaSpec extends JUnitWrapperSuite( - "akka.docs.transactor.TransactorDocTest", + "docs.transactor.TransactorDocTest", Thread.currentThread.getContextClassLoader) \ No newline at end of file diff --git a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java b/akka-docs/java/code/docs/transactor/TransactorDocTest.java similarity index 99% rename from akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java rename to akka-docs/java/code/docs/transactor/TransactorDocTest.java index bb1d38651b..4eaaa0bb31 100644 --- a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java +++ b/akka-docs/java/code/docs/transactor/TransactorDocTest.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.transactor; +package docs.transactor; import static org.junit.Assert.*; import org.junit.Test; diff --git a/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTest.scala b/akka-docs/java/code/docs/zeromq/ZeromqDocTest.scala similarity index 87% rename from akka-docs/java/code/akka/docs/zeromq/ZeromqDocTest.scala rename to akka-docs/java/code/docs/zeromq/ZeromqDocTest.scala index a9747959e3..c5e6f224da 100644 --- a/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTest.scala +++ b/akka-docs/java/code/docs/zeromq/ZeromqDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.zeromq +package docs.zeromq import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java b/akka-docs/java/code/docs/zeromq/ZeromqDocTestBase.java similarity index 99% rename from akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java rename to akka-docs/java/code/docs/zeromq/ZeromqDocTestBase.java index ee8252a6ad..1a311c9529 100644 --- a/akka-docs/java/code/akka/docs/zeromq/ZeromqDocTestBase.java +++ b/akka-docs/java/code/docs/zeromq/ZeromqDocTestBase.java @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.zeromq; +package docs.zeromq; //#pub-socket import akka.zeromq.Bind; diff --git a/akka-docs/java/dispatchers.rst b/akka-docs/java/dispatchers.rst index 6ef0d44d7e..577740d78c 100644 --- a/akka-docs/java/dispatchers.rst +++ b/akka-docs/java/dispatchers.rst @@ -18,7 +18,7 @@ Setting the dispatcher for an Actor So in case you want to give your ``Actor`` a different dispatcher than the default, you need to do two things, of which the first is: -.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#defining-dispatcher +.. includecode:: ../java/code/docs/dispatcher/DispatcherDocTestBase.java#defining-dispatcher .. note:: The "dispatcherId" you specify in withDispatcher is in fact a path into your configuration. @@ -27,11 +27,11 @@ So in case you want to give your ``Actor`` a different dispatcher than the defau And then you just need to configure that dispatcher in your configuration: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config And here's another example that uses the "thread-pool-executor": -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config For more options, see the default-dispatcher section of the :ref:`configuration`. @@ -70,7 +70,9 @@ There are 4 different types of message dispatchers: * BalancingDispatcher - - This is an executor based event driven dispatcher that will try to redistribute work from busy actors to idle actors. + - This is an executor based event driven dispatcher that will try to redistribute work from busy actors to idle actors. + + - All the actors share a single Mailbox that they get their messages from. - It is assumed that all actors using the same instance of this dispatcher can process all messages that have been sent to one of the actors; i.e. the actors belong to a pool of actors, and to the client there is no guarantee about which actor instance actually processes a given message. @@ -85,9 +87,11 @@ There are 4 different types of message dispatchers: "thread-pool-executor" or the FQCN of an ``akka.dispatcher.ExecutorServiceConfigurator`` + - Note that you can **not** use a ``BalancingDispatcher`` as a **Router Dispatcher**. (You can however use it for the **Routees**) + * CallingThreadDispatcher - - This dispatcher runs invocations on the current thread only. This dispatcher does not create any new threads, + - This dispatcher runs invocations on the current thread only. This dispatcher does not create any new threads, but it can be used from different threads concurrently for the same actor. See :ref:`TestCallingThreadDispatcherRef` for details and restrictions. @@ -104,11 +108,11 @@ More dispatcher configuration examples Configuring a ``PinnedDispatcher``: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-pinned-dispatcher-config +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#my-pinned-dispatcher-config And then using it: -.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#defining-pinned-dispatcher +.. includecode:: ../java/code/docs/dispatcher/DispatcherDocTestBase.java#defining-pinned-dispatcher Mailboxes --------- @@ -160,15 +164,15 @@ Mailbox configuration examples How to create a PriorityMailbox: -.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#prio-mailbox +.. includecode:: ../java/code/docs/dispatcher/DispatcherDocTestBase.java#prio-mailbox And then add it to the configuration: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher-config +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher-config And then an example on how you would use it: -.. includecode:: ../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java#prio-dispatcher +.. includecode:: ../java/code/docs/dispatcher/DispatcherDocTestBase.java#prio-dispatcher .. note:: @@ -179,3 +183,46 @@ And then an example on how you would use it: the configuration which describes the dispatcher using this mailbox type; the mailbox type will be instantiated once for each dispatcher using it. +Creating your own Mailbox type +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +An example is worth a thousand quacks: + +.. includecode:: code/docs/dispatcher/DispatcherDocTestBase.java#imports-custom + +.. includecode:: code/docs/dispatcher/DispatcherDocTestBase.java#mailbox-implementation-example + +And then you just specify the FQCN of your MailboxType as the value of the "mailbox-type" in the dispatcher configuration. + +.. note:: + + Make sure to include a constructor which takes + ``akka.actor.ActorSystem.Settings`` and ``com.typesafe.config.Config`` + arguments, as this constructor is invoked reflectively to construct your + mailbox type. The config passed in as second argument is that section from + the configuration which describes the dispatcher using this mailbox type; the + mailbox type will be instantiated once for each dispatcher using it. + + +Special Semantics of ``system.actorOf`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +In order to make ``system.actorOf`` both synchronous and non-blocking while +keeping the return type :class:`ActorRef` (and the semantics that the returned +ref is fully functional), special handling takes place for this case. Behind +the scenes, a hollow kind of actor reference is constructed, which is sent to +the system’s guardian actor who actually creates the actor and its context and +puts those inside the reference. Until that has happened, messages sent to the +:class:`ActorRef` will be queued locally, and only upon swapping the real +filling in will they be transferred into the real mailbox. Thus, + +.. code-block:: scala + + final Props props = ... + // this actor uses MyCustomMailbox, which is assumed to be a singleton + system.actorOf(props.withDispatcher("myCustomMailbox").tell("bang"); + assert(MyCustomMailbox.getInstance().getLastEnqueued().equals("bang")); + +will probably fail; you will have to allow for some time to pass and retry the +check à la :meth:`TestKit.awaitCond`. + diff --git a/akka-docs/java/event-bus.rst b/akka-docs/java/event-bus.rst index 542b94e821..faecd1d209 100644 --- a/akka-docs/java/event-bus.rst +++ b/akka-docs/java/event-bus.rst @@ -158,12 +158,12 @@ Classification`_ which enables registering to related sets of channels (as is used for :class:`RemoteLifeCycleMessage`). The following example demonstrates how a simple subscription works. Given a simple actor: -.. includecode:: code/akka/docs/event/LoggingDocTestBase.java#imports-deadletter -.. includecode:: code/akka/docs/event/LoggingDocTestBase.java#deadletter-actor +.. includecode:: code/docs/event/LoggingDocTestBase.java#imports-deadletter +.. includecode:: code/docs/event/LoggingDocTestBase.java#deadletter-actor it can be subscribed like this: -.. includecode:: code/akka/docs/event/LoggingDocTestBase.java#deadletters +.. includecode:: code/docs/event/LoggingDocTestBase.java#deadletters Default Handlers ---------------- diff --git a/akka-docs/java/extending-akka.rst b/akka-docs/java/extending-akka.rst index 6e8bfca4cb..c90fd40858 100644 --- a/akka-docs/java/extending-akka.rst +++ b/akka-docs/java/extending-akka.rst @@ -25,22 +25,22 @@ So let's create a sample extension that just lets us count the number of times s First, we define what our ``Extension`` should do: -.. includecode:: code/akka/docs/extension/ExtensionDocTestBase.java +.. includecode:: code/docs/extension/ExtensionDocTestBase.java :include: imports,extension Then we need to create an ``ExtensionId`` for our extension so we can grab ahold of it. -.. includecode:: code/akka/docs/extension/ExtensionDocTestBase.java +.. includecode:: code/docs/extension/ExtensionDocTestBase.java :include: imports,extensionid Wicked! Now all we need to do is to actually use it: -.. includecode:: code/akka/docs/extension/ExtensionDocTestBase.java +.. includecode:: code/docs/extension/ExtensionDocTestBase.java :include: extension-usage Or from inside of an Akka Actor: -.. includecode:: code/akka/docs/extension/ExtensionDocTestBase.java +.. includecode:: code/docs/extension/ExtensionDocTestBase.java :include: extension-usage-actor That's all there is to it! @@ -54,7 +54,7 @@ in the "akka.extensions" section of the config you provide to your ``ActorSystem :: akka { - extensions = ["akka.docs.extension.ExtensionDocTestBase.CountExtension"] + extensions = ["docs.extension.ExtensionDocTestBase.CountExtension"] } Applicability @@ -72,17 +72,17 @@ The :ref:`configuration` can be used for application specific settings. A good p Sample configuration: -.. includecode:: ../scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala +.. includecode:: ../scala/code/docs/extension/SettingsExtensionDocSpec.scala :include: config The ``Extension``: -.. includecode:: code/akka/docs/extension/SettingsExtensionDocTestBase.java +.. includecode:: code/docs/extension/SettingsExtensionDocTestBase.java :include: imports,extension,extensionid Use it: -.. includecode:: code/akka/docs/extension/SettingsExtensionDocTestBase.java +.. includecode:: code/docs/extension/SettingsExtensionDocTestBase.java :include: extension-usage-actor diff --git a/akka-docs/java/fault-tolerance-sample.rst b/akka-docs/java/fault-tolerance-sample.rst index 8e379c5fcc..749cf7ef95 100644 --- a/akka-docs/java/fault-tolerance-sample.rst +++ b/akka-docs/java/fault-tolerance-sample.rst @@ -43,11 +43,11 @@ Step Description 9, 10, 11 and tells the ``Counter`` that there is no ``Storage``. 12 The ``CounterService`` schedules a ``Reconnect`` message to itself. 13, 14 When it receives the ``Reconnect`` message it creates a new ``Storage`` ... -15, 16 and tells the the ``Counter`` to use the new ``Storage`` +15, 16 and tells the ``Counter`` to use the new ``Storage`` =========== ================================================================================== Full Source Code of the Fault Tolerance Sample (Java) ------------------------------------------------------ -.. includecode:: code/akka/docs/actor/japi/FaultHandlingDocSample.java#all +.. includecode:: code/docs/actor/japi/FaultHandlingDocSample.java#all diff --git a/akka-docs/java/fault-tolerance.rst b/akka-docs/java/fault-tolerance.rst index fc172b8fa6..a444f79ec0 100644 --- a/akka-docs/java/fault-tolerance.rst +++ b/akka-docs/java/fault-tolerance.rst @@ -24,7 +24,7 @@ sample as it is easy to follow the log output to understand what is happening in fault-tolerance-sample -.. includecode:: code/akka/docs/actor/japi/FaultHandlingDocSample.java#all +.. includecode:: code/docs/actor/japi/FaultHandlingDocSample.java#all :exclude: imports,messages,dummydb Creating a Supervisor Strategy @@ -35,7 +35,7 @@ in more depth. For the sake of demonstration let us consider the following strategy: -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: strategy I have chosen a few well-known exception types in order to demonstrate the @@ -70,49 +70,49 @@ Test Application The following section shows the effects of the different directives in practice, wherefor a test setup is needed. First off, we need a suitable supervisor: -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: supervisor This supervisor will be used to create a child, with which we can experiment: -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: child The test is easier by using the utilities described in :ref:`akka-testkit`, where ``TestProbe`` provides an actor ref useful for receiving and inspecting replies. -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: testkit Let us create actors: -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: create The first test shall demonstrate the ``Resume`` directive, so we try it out by setting some non-initial state in the actor and have it fail: -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: resume As you can see the value 42 survives the fault handling directive. Now, if we change the failure to a more serious ``NullPointerException``, that will no longer be the case: -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: restart And finally in case of the fatal ``IllegalArgumentException`` the child will be terminated by the supervisor: -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: stop Up to now the supervisor was completely unaffected by the child’s failure, because the directives set did handle it. In case of an ``Exception``, this is not true anymore and the supervisor escalates the failure. -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: escalate-kill The supervisor itself is supervised by the top-level actor provided by the @@ -125,12 +125,12 @@ child not to survive this failure. In case this is not desired (which depends on the use case), we need to use a different supervisor which overrides this behavior. -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: supervisor2 With this parent, the child survives the escalated restart, as demonstrated in the last test: -.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java +.. includecode:: code/docs/actor/FaultHandlingTestBase.java :include: escalate-restart diff --git a/akka-docs/java/fsm.rst b/akka-docs/java/fsm.rst index bd8712d460..582b3528f2 100644 --- a/akka-docs/java/fsm.rst +++ b/akka-docs/java/fsm.rst @@ -35,9 +35,9 @@ using a small well-defined set of methods. One way to achieve this is to assemble all mutable state in a superclass which keeps it private and offers protected methods for mutating it. -.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#imports-data +.. includecode:: code/docs/actor/FSMDocTestBase.java#imports-data -.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#base +.. includecode:: code/docs/actor/FSMDocTestBase.java#base The benefit of this approach is that state changes can be acted upon in one central place, which makes it impossible to forget inserting code for reacting @@ -50,15 +50,15 @@ The base class shown above is designed to support a similar example as for the Scala FSM documentation: an actor which receives and queues messages, to be delivered in batches to a configurable target actor. The messages involved are: -.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#data +.. includecode:: code/docs/actor/FSMDocTestBase.java#data This actor has only the two states ``IDLE`` and ``ACTIVE``, making their handling quite straight-forward in the concrete actor derived from the base class: -.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#imports-actor +.. includecode:: code/docs/actor/FSMDocTestBase.java#imports-actor -.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#actor +.. includecode:: code/docs/actor/FSMDocTestBase.java#actor The trick here is to factor out common functionality like :meth:`whenUnhandled` and :meth:`transition` in order to obtain a few well-defined points for diff --git a/akka-docs/java/futures.rst b/akka-docs/java/futures.rst index cc119c46e6..e0639420cb 100644 --- a/akka-docs/java/futures.rst +++ b/akka-docs/java/futures.rst @@ -18,7 +18,7 @@ which is very similar to a ``java.util.concurrent.Executor``. if you have an ``A it will use its default dispatcher as the ``ExecutionContext``, or you can use the factory methods provided by the ``ExecutionContexts`` class to wrap ``Executors`` and ``ExecutorServices``, or even create your own. -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: imports1,imports7,diy-execution-context Use with Actors @@ -30,7 +30,7 @@ which only works if the original sender was an ``UntypedActor``) and the second Using the ``ActorRef``\'s ``ask`` method to send a message will return a Future. To wait for and retrieve the actual result the simplest method is: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: imports1,ask-blocking This will cause the current thread to block and wait for the ``UntypedActor`` to 'complete' the ``Future`` with it's reply. @@ -47,7 +47,7 @@ A common use case within Akka is to have some computation performed concurrently the extra utility of an ``UntypedActor``. If you find yourself creating a pool of ``UntypedActor``\s for the sole reason of performing a calculation in parallel, there is an easier (and faster) way: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: imports2,future-eval In the above code the block passed to ``future`` will be executed by the default ``Dispatcher``, @@ -57,12 +57,12 @@ and we also avoid the overhead of managing an ``UntypedActor``. You can also create already completed Futures using the ``Futures`` class, which can be either successes: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: successful Or failures: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: failed Functional Futures @@ -78,7 +78,7 @@ The first method for working with ``Future`` functionally is ``map``. This metho some operation on the result of the ``Future``, and returning a new result. The return value of the ``map`` method is another ``Future`` that will contain the new result: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: imports2,map In this example we are joining two strings together within a Future. Instead of waiting for f1 to complete, @@ -92,7 +92,7 @@ Something to note when using these methods: if the ``Future`` is still being pro it will be the completing thread that actually does the work. If the ``Future`` is already complete though, it will be run in our current thread. For example: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: map2 The original ``Future`` will take at least 0.1 second to execute now, which means it is still being processed at @@ -101,7 +101,7 @@ by the dispatcher when the result is ready. If we do the opposite: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: map3 Our little string has been processed long before our 0.1 second sleep has finished. Because of this, @@ -112,7 +112,7 @@ Normally this works quite well as it means there is very little overhead to runn If there is a possibility of the function taking a non-trivial amount of time to process it might be better to have this done concurrently, and for that we use ``flatMap``: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: flat-map Now our second Future is executed concurrently as well. This technique can also be used to combine the results @@ -120,7 +120,7 @@ of several Futures into a single calculation, which will be better explained in If you need to do conditional propagation, you can use ``filter``: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: filter Composing Futures @@ -129,7 +129,7 @@ Composing Futures It is very often desirable to be able to combine different Futures with each other, below are some examples on how that can be done in a non-blocking fashion. -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: imports3,sequence To better explain what happened in the example, ``Future.sequence`` is taking the ``Iterable>`` @@ -139,7 +139,7 @@ and we aggregate the sum of the ``Iterable``. The ``traverse`` method is similar to ``sequence``, but it takes a sequence of ``A``s and applies a function from ``A`` to ``Future`` and returns a ``Future>``, enabling parallel ``map`` over the sequence, if you use ``Futures.future`` to create the ``Future``. -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: imports4,traverse It's as simple as that! @@ -150,7 +150,7 @@ and the type of the futures and returns something with the same type as the star and then applies the function to all elements in the sequence of futures, non-blockingly, the execution will be started when the last of the Futures is completed. -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: imports5,fold That's all it takes! @@ -160,7 +160,7 @@ If the sequence passed to ``fold`` is empty, it will return the start-value, in In some cases you don't have a start-value and you're able to use the value of the first completing Future in the sequence as the start-value, you can use ``reduce``, it works like this: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: imports6,reduce Same as with ``fold``, the execution will be started when the last of the Futures is completed, you can also parallelize @@ -174,13 +174,13 @@ Callbacks Sometimes you just want to listen to a ``Future`` being completed, and react to that not by creating a new Future, but by side-effecting. For this Akka supports ``onComplete``, ``onSuccess`` and ``onFailure``, of which the latter two are specializations of the first. -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: onSuccess -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: onFailure -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: onComplete Ordering @@ -192,7 +192,7 @@ But there's a solution! And it's name is ``andThen``, and it creates a new Futur the specified callback, a Future that will have the same result as the Future it's called on, which allows for ordering like in the following sample: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: and-then Auxiliary methods @@ -201,13 +201,13 @@ Auxiliary methods ``Future`` ``fallbackTo`` combines 2 Futures into a new ``Future``, and will hold the successful value of the second ``Future`` if the first ``Future`` fails. -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: fallback-to You can also combine two Futures into a new ``Future`` that will hold a tuple of the two Futures successful results, using the ``zip`` operation. -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: zip Exceptions @@ -221,7 +221,7 @@ calling ``Await.result`` will cause it to be thrown again so it can be handled p It is also possible to handle an ``Exception`` by returning a different result. This is done with the ``recover`` method. For example: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: recover In this example, if the actor replied with a ``akka.actor.Status.Failure`` containing the ``ArithmeticException``, @@ -232,6 +232,6 @@ it will behave as if we hadn't used the ``recover`` method. You can also use the ``recoverWith`` method, which has the same relationship to ``recover`` as ``flatMap`` has to ``map``, and is use like this: -.. includecode:: code/akka/docs/future/FutureDocTestBase.java +.. includecode:: code/docs/future/FutureDocTestBase.java :include: try-recover diff --git a/akka-docs/java/howto.rst b/akka-docs/java/howto.rst new file mode 100644 index 0000000000..333fb9e498 --- /dev/null +++ b/akka-docs/java/howto.rst @@ -0,0 +1,33 @@ + +.. _howto-java: + +###################### +HowTo: Common Patterns +###################### + +This section lists common actor patterns which have been found to be useful, +elegant or instructive. Anything is welcome, example topics being message +routing strategies, supervision patterns, restart handling, etc. As a special +bonus, additions to this section are marked with the contributor’s name, and it +would be nice if every Akka user who finds a recurring pattern in his or her +code could share it for the profit of all. Where applicable it might also make +sense to add to the ``akka.pattern`` package for creating an `OTP-like library +`_. + +Template Pattern +================ + +*Contributed by: N. N.* + +This is an especially nice pattern, since it does even come with some empty example code: + +.. includecode:: code/docs/pattern/JavaTemplate.java + :include: all-of-it + :exclude: uninteresting-stuff + +.. note:: + + Spread the word: this is the easiest way to get famous! + +Please keep this pattern at the end of this file. + diff --git a/akka-docs/java/index.rst b/akka-docs/java/index.rst index 981e07f869..669bd7c362 100644 --- a/akka-docs/java/index.rst +++ b/akka-docs/java/index.rst @@ -24,3 +24,5 @@ Java API extending-akka zeromq microkernel + testing + howto diff --git a/akka-docs/java/logging.rst b/akka-docs/java/logging.rst index 177cdb7295..647525ba76 100644 --- a/akka-docs/java/logging.rst +++ b/akka-docs/java/logging.rst @@ -11,7 +11,7 @@ How to Log Create a ``LoggingAdapter`` and use the ``error``, ``warning``, ``info``, or ``debug`` methods, as illustrated in this example: -.. includecode:: code/akka/docs/event/LoggingDocTestBase.java +.. includecode:: code/docs/event/LoggingDocTestBase.java :include: imports,my-actor The first parameter to ``Logging.getLogger`` could also be any @@ -33,7 +33,7 @@ placeholders results in a warning being appended to the log statement (i.e. on the same line with the same severity). You may pass a Java array as the only substitution argument to have its elements be treated individually: -.. includecode:: code/akka/docs/event/LoggingDocTestBase.java#array +.. includecode:: code/docs/event/LoggingDocTestBase.java#array The Java :class:`Class` of the log source is also included in the generated :class:`LogEvent`. In case of a simple string this is replaced with a “marker” @@ -170,7 +170,7 @@ event handler available in the 'akka-slf4j' module. Example of creating a listener: -.. includecode:: code/akka/docs/event/LoggingDocTestBase.java +.. includecode:: code/docs/event/LoggingDocTestBase.java :include: imports,imports-listener,my-event-listener @@ -187,7 +187,7 @@ It has one single dependency; the slf4j-api jar. In runtime you also need a SLF4 ch.qos.logback logback-classic - 1.0.0 + 1.0.4 runtime @@ -211,7 +211,7 @@ the first case and ``LoggerFactory.getLogger(String s)`` in the second). .. note:: - Beware that the the actor system’s name is appended to a :class:`String` log + Beware that the actor system’s name is appended to a :class:`String` log source if the LoggingAdapter was created giving an :class:`ActorSystem` to the factory. If this is not intended, give a :class:`LoggingBus` instead as shown below: diff --git a/akka-docs/java/microkernel.rst b/akka-docs/java/microkernel.rst index 551c118e94..970b174549 100644 --- a/akka-docs/java/microkernel.rst +++ b/akka-docs/java/microkernel.rst @@ -1,9 +1,13 @@ -.. _microkernel: +.. _microkernel-java: Microkernel (Java) ================== +The purpose of the Akka Microkernel is to offer a bundling mechanism so that you can distribute +an Akka application as a single payload, without the need to run in a Java Application Server or manually +having to create a launcher script. + The Akka Microkernel is included in the Akka download found at `downloads`_. .. _downloads: http://akka.io/downloads diff --git a/akka-docs/java/remoting.rst b/akka-docs/java/remoting.rst index 376eab2584..82a736973f 100644 --- a/akka-docs/java/remoting.rst +++ b/akka-docs/java/remoting.rst @@ -82,16 +82,24 @@ The "app" in this case refers to the name of the ``ActorSystem`` (only showing d Logical path lookup is supported on the node you are on, i.e. to use the actor created above you would do the following: -.. includecode:: code/akka/docs/remoting/RemoteActorExample.java#localNodeActor +.. includecode:: code/docs/remoting/RemoteActorExample.java#localNodeActor This will obtain an ``ActorRef`` on a remote node: -.. includecode:: code/akka/docs/remoting/RemoteActorExample.java#remoteNodeActor +.. includecode:: code/docs/remoting/RemoteActorExample.java#remoteNodeActor As you can see from the example above the following pattern is used to find an ``ActorRef`` on a remote node:: akka://@:/ +.. note:: + + In order to ensure serializability of ``Props`` when passing constructor + arguments to the actor being created, do not make the factory a non-static + inner class: this will inherently capture a reference to its enclosing + object, which in most cases is not serializable. It is best to make a static + inner class which implements :class:`UntypedActorFactory`. + Programmatic Remote Deployment ------------------------------ @@ -103,15 +111,15 @@ precedence. With these imports: -.. includecode:: code/akka/docs/remoting/RemoteDeploymentDocTestBase.java#import +.. includecode:: code/docs/remoting/RemoteDeploymentDocTestBase.java#import and a remote address like this: -.. includecode:: code/akka/docs/remoting/RemoteDeploymentDocTestBase.java#make-address +.. includecode:: code/docs/remoting/RemoteDeploymentDocTestBase.java#make-address you can advise the system to create a child on that remote node like so: -.. includecode:: code/akka/docs/remoting/RemoteDeploymentDocTestBase.java#deploy +.. includecode:: code/docs/remoting/RemoteDeploymentDocTestBase.java#deploy Serialization ^^^^^^^^^^^^^ @@ -272,10 +280,6 @@ which holds the transport used (RemoteTransport) and the outbound address that i To intercept when an outbound client is shut down you listen to ``RemoteClientShutdown`` which holds the transport used (RemoteTransport) and the outbound address that it was connected to (Address). -To intercept when an outbound message cannot be sent, you listen to ``RemoteClientWriteFailed`` which holds -the payload that was not written (AnyRef), the cause of the failed send (Throwable), -the transport used (RemoteTransport) and the outbound address that was the destination (Address). - For general outbound-related errors, that do not classify as any of the others, you can listen to ``RemoteClientError``, which holds the cause (Throwable), the transport used (RemoteTransport) and the outbound address (Address). @@ -294,3 +298,63 @@ which holds the transport used (RemoteTransport) and optionally the address that To intercept when an inbound remote client has been closed you listen to ``RemoteServerClientClosed`` which holds the transport used (RemoteTransport) and optionally the address of the remote client that was closed (Option
). +Remote Security +^^^^^^^^^^^^^^^ + +Akka provides a couple of ways to enhance security between remote nodes (client/server): + +* Untrusted Mode +* Security Cookie Handshake + +Untrusted Mode +-------------- + +You can enable untrusted mode for preventing system messages to be send by clients, e.g. messages like. +This will prevent the client to send these messages to the server: + +* ``Create`` +* ``Recreate`` +* ``Suspend`` +* ``Resume`` +* ``Terminate`` +* ``Supervise`` +* ``ChildTerminated`` +* ``Link`` +* ``Unlink`` + +Here is how to turn it on in the config:: + + akka { + actor { + remote { + untrusted-mode = on + } + } + } + +Secure Cookie Handshake +----------------------- + +Akka remoting also allows you to specify a secure cookie that will be exchanged and ensured to be identical +in the connection handshake between the client and the server. If they are not identical then the client +will be refused to connect to the server. + +The secure cookie can be any kind of string. But the recommended approach is to generate a cryptographically +secure cookie using this script ``$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh`` or from code +using the ``akka.util.Crypt.generateSecureCookie()`` utility method. + +You have to ensure that both the connecting client and the server have the same secure cookie as well +as the ``require-cookie`` option turned on. + +Here is an example config:: + + akka { + actor { + remote { + netty { + secure-cookie = "090A030E0F0A05010900000A0C0E0C0B03050D05" + require-cookie = on + } + } + } + } diff --git a/akka-docs/java/routing.rst b/akka-docs/java/routing.rst index 4d01642a72..16aa4cee6f 100644 --- a/akka-docs/java/routing.rst +++ b/akka-docs/java/routing.rst @@ -21,17 +21,22 @@ Routers In Action This is an example of how to create a router that is defined in configuration: -.. includecode:: ../scala/code/akka/docs/routing/RouterViaConfigExample.scala#config +.. includecode:: ../scala/code/docs/routing/RouterViaConfigExample.scala#config -.. includecode:: code/akka/docs/jrouting/RouterViaConfigExample.java#configurableRouting +.. includecode:: code/docs/jrouting/RouterViaConfigExample.java#configurableRouting This is an example of how to programmatically create a router and set the number of routees it should create: -.. includecode:: code/akka/docs/jrouting/RouterViaProgramExample.java#programmaticRoutingNrOfInstances +.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#programmaticRoutingNrOfInstances You can also give the router already created routees as in: -.. includecode:: code/akka/docs/jrouting/RouterViaProgramExample.java#programmaticRoutingRoutees +.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#programmaticRoutingRoutees + +It should be noted that no actor factory or class needs to be provided in this +case, as the ``Router`` will not create any children on its own (which is not +true anymore when using a resizer). The routees can also be specified by giving +their path strings. When you create a router programmatically you define the number of routees *or* you pass already created routees to it. If you send both parameters to the router *only* the latter will be used, i.e. ``nrOfInstances`` is disregarded. @@ -48,7 +53,7 @@ Once you have the router actor it is just to send messages to it as you would to router.tell(new MyMsg()); -The router will apply its behavior to the message it receives and forward it to the routees. +The router will forward the message to its routees according to its routing policy. Remotely Deploying Routees ************************** @@ -60,7 +65,7 @@ configuration in a :class:`RemoteRouterConfig`, attaching the remote addresses o the nodes to deploy to. Naturally, this requires your to include the ``akka-remote`` module on your classpath: -.. includecode:: code/akka/docs/jrouting/RouterViaProgramExample.java#remoteRoutees +.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#remoteRoutees How Routing is Designed within Akka ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -90,9 +95,9 @@ deterministic fashion. Since each actor knows its own external representation as well as that of its parent, the routees decide where replies should be sent when reacting to a message: -.. includecode:: code/akka/docs/jrouting/RouterViaProgramExample.java#reply-with-parent +.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#reply-with-parent -.. includecode:: code/akka/docs/jrouting/RouterViaProgramExample.java#reply-with-self +.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#reply-with-self It is apparent now why routing needs to be enabled in code rather than being possible to “bolt on” later: whether or not an actor is routed means a change @@ -116,7 +121,7 @@ not have an effect on the number of actors in the pool. Setting the strategy is easily done: -.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java +.. includecode:: code/docs/jrouting/CustomRouterDocTestBase.java :include: supervision Another potentially useful approach is to give the router the same strategy as @@ -135,18 +140,18 @@ Router usage In this section we will describe how to use the different router types. First we need to create some actors that will be used in the examples: -.. includecode:: code/akka/docs/jrouting/PrintlnActor.java#printlnActor +.. includecode:: code/docs/jrouting/PrintlnActor.java#printlnActor and -.. includecode:: code/akka/docs/jrouting/FibonacciActor.java#fibonacciActor +.. includecode:: code/docs/jrouting/FibonacciActor.java#fibonacciActor RoundRobinRouter **************** Routes in a `round-robin `_ fashion to its routees. Code example: -.. includecode:: code/akka/docs/jrouting/ParentActor.java#roundRobinRouter +.. includecode:: code/docs/jrouting/ParentActor.java#roundRobinRouter When run you should see a similar output to this: @@ -175,7 +180,7 @@ the message it receives to this routee. This procedure will happen each time it receives a message. Code example: -.. includecode:: code/akka/docs/jrouting/ParentActor.java#randomRouter +.. includecode:: code/docs/jrouting/ParentActor.java#randomRouter When run you should see a similar output to this: @@ -208,14 +213,14 @@ The selection is done in this order: Code example: -.. includecode:: code/akka/docs/jrouting/ParentActor.java#smallestMailboxRouter +.. includecode:: code/docs/jrouting/ParentActor.java#smallestMailboxRouter BroadcastRouter *************** A broadcast router forwards the message it receives to *all* its routees. Code example: -.. includecode:: code/akka/docs/jrouting/ParentActor.java#broadcastRouter +.. includecode:: code/docs/jrouting/ParentActor.java#broadcastRouter When run you should see a similar output to this: @@ -235,7 +240,7 @@ The ScatterGatherFirstCompletedRouter will send the message on to all its routee It then waits for first result it gets back. This result will be sent back to original sender. Code example: -.. includecode:: code/akka/docs/jrouting/ParentActor.java#scatterGatherFirstCompletedRouter +.. includecode:: code/docs/jrouting/ParentActor.java#scatterGatherFirstCompletedRouter When run you should see this: @@ -267,16 +272,16 @@ of routees dynamically. This is an example of how to create a resizable router that is defined in configuration: -.. includecode:: ../scala/code/akka/docs/routing/RouterViaConfigExample.scala#config-resize +.. includecode:: ../scala/code/docs/routing/RouterViaConfigExample.scala#config-resize -.. includecode:: code/akka/docs/jrouting/RouterViaConfigExample.java#configurableRoutingWithResizer +.. includecode:: code/docs/jrouting/RouterViaConfigExample.java#configurableRoutingWithResizer Several more configuration options are available and described in ``akka.actor.deployment.default.resizer`` section of the reference :ref:`configuration`. This is an example of how to programmatically create a resizable router: -.. includecode:: code/akka/docs/jrouting/RouterViaProgramExample.java#programmaticRoutingWithResizer +.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#programmaticRoutingWithResizer *It is also worth pointing out that if you define the ``router`` in the configuration file then this value will be used instead of any programmatically sent parameters.* @@ -303,12 +308,12 @@ democrat related messages to the Democrat actor and all republican related messa We begin with defining the class: -.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java#crRouter +.. includecode:: code/docs/jrouting/CustomRouterDocTestBase.java#crRouter :exclude: crRoute The next step is to implement the ``createCustomRoute`` method in the class just defined: -.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java#crRoute +.. includecode:: code/docs/jrouting/CustomRouterDocTestBase.java#crRoute As you can see above we start off by creating the routees and put them in a collection. @@ -317,12 +322,12 @@ It registers the routees internally and failing to call this method will cause a ``ActorInitializationException`` to be thrown when the router is used. Therefore always make sure to do the following in your custom router: -.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java#crRegisterRoutees +.. includecode:: code/docs/jrouting/CustomRouterDocTestBase.java#crRegisterRoutees The routing logic is where your magic sauce is applied. In our example it inspects the message types and forwards to the correct routee based on this: -.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java#crRoutingLogic +.. includecode:: code/docs/jrouting/CustomRouterDocTestBase.java#crRoutingLogic As you can see above what's returned in the ``CustomRoute`` function, which defines the mapping from incoming sender/message to a ``List`` of ``Destination(sender, routee)``. @@ -333,11 +338,11 @@ For more information about how to alter the original sender we refer to the sour All in all the custom router looks like this: -.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java#CustomRouter +.. includecode:: code/docs/jrouting/CustomRouterDocTestBase.java#CustomRouter If you are interested in how to use the VoteCountRouter it looks like this: -.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java#crTest +.. includecode:: code/docs/jrouting/CustomRouterDocTestBase.java#crTest .. caution:: @@ -377,13 +382,47 @@ routing is not so important (i.e. no consistent hashing or round-robin is required); this enables newly created routees to pick up work immediately by stealing it from their siblings. -The “head” router, of course, cannot run on the same balancing dispatcher, -because it does not process the same messages, hence this special actor does +.. note:: + + If you provide a collection of actors to route to, then they will still use the same dispatcher + that was configured for them in their ``Props``, it is not possible to change an actors dispatcher + after it has been created. + +The “head” router cannot always run on the same dispatcher, because it +does not process the same type of messages, hence this special actor does not use the dispatcher configured in :class:`Props`, but takes the ``routerDispatcher`` from the :class:`RouterConfig` instead, which defaults to the actor system’s default dispatcher. All standard routers allow setting this property in their constructor or factory method, custom routers have to implement the method in a suitable way. -.. includecode:: code/akka/docs/jrouting/CustomRouterDocTestBase.java#dispatchers +.. includecode:: code/docs/jrouting/CustomRouterDocTestBase.java#dispatchers +.. note:: + + It is not allowed to configure the ``routerDispatcher`` to be a + :class:`BalancingDispatcher` since the messages meant for the special + router actor cannot be processed by any other actor. + +At first glance there seems to be an overlap between the +:class:`BalancingDispatcher` and Routers, but they complement each other. +The balancing dispatcher is in charge of running the actors while the routers +are in charge of deciding which message goes where. A router can also have +children that span multiple actor systems, even remote ones, but a dispatcher +lives inside a single actor system. + +When using a :class:`RoundRobinRouter` with a :class:`BalancingDispatcher` +there are some configuration settings to take into account. + +- There can only be ``nr-of-instances`` messages being processed at the same + time no matter how many threads are configured for the + :class:`BalancingDispatcher`. + +- Having ``throughput`` set to a low number makes no sense since you will only + be handing off to another actor that processes the same :class:`MailBox` + as yourself, which can be costly. Either the message just got into the + mailbox and you can receive it as well as anybody else, or everybody else + is busy and you are the only one available to receive the message. + +- Resizing the number of routees only introduce inertia, since resizing + is performed at specified intervals, but work stealing is instantaneous. diff --git a/akka-docs/java/scheduler.rst b/akka-docs/java/scheduler.rst index faff8d9fe0..28da7feeca 100644 --- a/akka-docs/java/scheduler.rst +++ b/akka-docs/java/scheduler.rst @@ -25,13 +25,13 @@ scheduled operation. Some examples ------------- -.. includecode:: code/akka/docs/actor/SchedulerDocTestBase.java +.. includecode:: code/docs/actor/SchedulerDocTestBase.java :include: imports1,schedule-one-off-message -.. includecode:: code/akka/docs/actor/SchedulerDocTestBase.java +.. includecode:: code/docs/actor/SchedulerDocTestBase.java :include: schedule-one-off-thunk -.. includecode:: code/akka/docs/actor/SchedulerDocTestBase.java +.. includecode:: code/docs/actor/SchedulerDocTestBase.java :include: imports1,imports2,schedule-recurring From ``akka.actor.ActorSystem`` diff --git a/akka-docs/java/serialization.rst b/akka-docs/java/serialization.rst index 4c7b023959..b973a1d18c 100644 --- a/akka-docs/java/serialization.rst +++ b/akka-docs/java/serialization.rst @@ -21,12 +21,12 @@ For Akka to know which ``Serializer`` to use for what, you need edit your :ref:` in the "akka.actor.serializers"-section you bind names to implementations of the ``akka.serialization.Serializer`` you wish to use, like this: -.. includecode:: ../scala/code/akka/docs/serialization/SerializationDocSpec.scala#serialize-serializers-config +.. includecode:: ../scala/code/docs/serialization/SerializationDocSpec.scala#serialize-serializers-config After you've bound names to different implementations of ``Serializer`` you need to wire which classes should be serialized using which ``Serializer``, this is done in the "akka.actor.serialization-bindings"-section: -.. includecode:: ../scala/code/akka/docs/serialization/SerializationDocSpec.scala#serialization-bindings-config +.. includecode:: ../scala/code/docs/serialization/SerializationDocSpec.scala#serialization-bindings-config You only need to specify the name of an interface or abstract base class of the messages. In case of ambiguity, i.e. the message implements several of the @@ -53,7 +53,7 @@ Verification If you want to verify that your messages are serializable you can enable the following config option: -.. includecode:: ../scala/code/akka/docs/serialization/SerializationDocSpec.scala#serialize-messages-config +.. includecode:: ../scala/code/docs/serialization/SerializationDocSpec.scala#serialize-messages-config .. warning:: @@ -62,7 +62,7 @@ If you want to verify that your messages are serializable you can enable the fol If you want to verify that your ``Props`` are serializable you can enable the following config option: -.. includecode:: ../scala/code/akka/docs/serialization/SerializationDocSpec.scala#serialize-creators-config +.. includecode:: ../scala/code/docs/serialization/SerializationDocSpec.scala#serialize-creators-config .. warning:: @@ -75,7 +75,7 @@ Programmatic If you want to programmatically serialize/deserialize using Akka Serialization, here's some examples: -.. includecode:: code/akka/docs/serialization/SerializationDocTestBase.java +.. includecode:: code/docs/serialization/SerializationDocTestBase.java :include: imports,programmatic For more information, have a look at the ``ScalaDoc`` for ``akka.serialization._`` @@ -85,7 +85,7 @@ Customization ============= So, lets say that you want to create your own ``Serializer``, -you saw the ``akka.docs.serialization.MyOwnSerializer`` in the config example above? +you saw the ``docs.serialization.MyOwnSerializer`` in the config example above? Creating new Serializers ------------------------ @@ -93,7 +93,7 @@ Creating new Serializers First you need to create a class definition of your ``Serializer``, which is done by extending ``akka.serialization.JSerializer``, like this: -.. includecode:: code/akka/docs/serialization/SerializationDocTestBase.java +.. includecode:: code/docs/serialization/SerializationDocTestBase.java :include: imports,my-own-serializer :exclude: ... @@ -106,9 +106,50 @@ Serializing ActorRefs All ActorRefs are serializable using JavaSerializer, but in case you are writing your own serializer, you might want to know how to serialize and deserialize them properly, here's the magic incantation: -.. includecode:: code/akka/docs/serialization/SerializationDocTestBase.java +.. includecode:: code/docs/serialization/SerializationDocTestBase.java :include: imports,actorref-serializer +.. note:: + + ``ActorPath.toStringWithAddress`` only differs from ``toString`` if the + address does not already have ``host`` and ``port`` components, i.e. it only + inserts address information for local addresses. + +This assumes that serialization happens in the context of sending a message +through the remote transport. There are other uses of serialization, though, +e.g. storing actor references outside of an actor application (database, +durable mailbox, etc.). In this case, it is important to keep in mind that the +address part of an actor’s path determines how that actor is communicated with. +Storing a local actor path might be the right choice if the retrieval happens +in the same logical context, but it is not enough when deserializing it on a +different network host: for that it would need to include the system’s remote +transport address. An actor system is not limited to having just one remote +transport per se, which makes this question a bit more interesting. + +In the general case, the local address to be used depends on the type of remote +address which shall be the recipient of the serialized information. Use +:meth:`ActorRefProvider.getExternalAddressFor(remoteAddr)` to query the system +for the appropriate address to use when sending to ``remoteAddr``: + +.. includecode:: code/docs/serialization/SerializationDocTestBase.java + :include: external-address + +This requires that you know at least which type of address will be supported by +the system which will deserialize the resulting actor reference; if you have no +concrete address handy you can create a dummy one for the right protocol using +``new Address(protocol, "", "", 0)`` (assuming that the actual transport used is as +lenient as Akka’s RemoteActorRefProvider). + +There is a possible simplification available if you are just using the default +:class:`NettyRemoteTransport` with the :meth:`RemoteActorRefProvider`, which is +enabled by the fact that this combination has just a single remote address: + +.. includecode:: code/docs/serialization/SerializationDocTestBase.java + :include: external-address-default + +This solution has to be adapted once other providers are used (like the planned +extensions for clustering). + Deep serialization of Actors ---------------------------- @@ -137,3 +178,13 @@ representation into a real reference. :class:`DynamicVariable` is a thread-local variable, so be sure to have it set while deserializing anything which might contain actor references. +External Akka Serializers +========================= + +`Akka-protostuff by Roman Levenstein `_ + + +`Akka-quickser by Roman Levenstein `_ + + +`Akka-kryo by Roman Levenstein `_ diff --git a/akka-docs/java/testing.rst b/akka-docs/java/testing.rst new file mode 100644 index 0000000000..d49ba2512f --- /dev/null +++ b/akka-docs/java/testing.rst @@ -0,0 +1,14 @@ +.. _akka-testkit-java: + +############################## +Testing Actor Systems (Java) +############################## + +Due to the conciseness of test DSLs available for Scala, it may be a good idea +to write the test suite in that language even if the main project is written in +Java. If that is not desirable, you can also use :class:`TestKit` and friends +from Java, albeit with more verbose syntax Munish Gupta has `published a nice +post `_ +showing several patterns you may find useful, and for reference documentation +please refer to :ref:`akka-testkit` until that section has been ported over to +cover Java in full. diff --git a/akka-docs/java/transactors.rst b/akka-docs/java/transactors.rst index c9ac0cd90f..7b7c080ffe 100644 --- a/akka-docs/java/transactors.rst +++ b/akka-docs/java/transactors.rst @@ -63,22 +63,22 @@ Here is an example of coordinating two simple counter UntypedActors so that they both increment together in coordinated transactions. If one of them was to fail to increment, the other would also fail. -.. includecode:: code/akka/docs/transactor/Increment.java#class +.. includecode:: code/docs/transactor/Increment.java#class :language: java -.. includecode:: code/akka/docs/transactor/CoordinatedCounter.java#class +.. includecode:: code/docs/transactor/CoordinatedCounter.java#class :language: java -.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#imports +.. includecode:: code/docs/transactor/TransactorDocTest.java#imports :language: java -.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#coordinated-example +.. includecode:: code/docs/transactor/TransactorDocTest.java#coordinated-example :language: java To start a new coordinated transaction that you will also participate in, create a ``Coordinated`` object, passing in a ``Timeout``: -.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#create-coordinated +.. includecode:: code/docs/transactor/TransactorDocTest.java#create-coordinated :language: java To start a coordinated transaction that you won't participate in yourself you @@ -86,7 +86,7 @@ can create a ``Coordinated`` object with a message and send it directly to an actor. The recipient of the message will be the first member of the coordination set: -.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#send-coordinated +.. includecode:: code/docs/transactor/TransactorDocTest.java#send-coordinated :language: java To include another actor in the same coordinated transaction that you've created @@ -94,13 +94,13 @@ or received, use the ``coordinate`` method on that object. This will increment the number of parties involved by one and create a new ``Coordinated`` object to be sent. -.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#include-coordinated +.. includecode:: code/docs/transactor/TransactorDocTest.java#include-coordinated :language: java To enter the coordinated transaction use the atomic method of the coordinated object, passing in a ``java.lang.Runnable``. -.. includecode:: code/akka/docs/transactor/Coordinator.java#coordinated-atomic +.. includecode:: code/docs/transactor/Coordinator.java#coordinated-atomic :language: java The coordinated transaction will wait for the other transactions before @@ -124,7 +124,7 @@ coordinating transactions, using the explicit coordination described above. Here's an example of a simple untyped transactor that will join a coordinated transaction: -.. includecode:: code/akka/docs/transactor/Counter.java#class +.. includecode:: code/docs/transactor/Counter.java#class :language: java You could send this Counter transactor a ``Coordinated(Increment)`` message. If @@ -140,7 +140,7 @@ to easily coordinate with other transactors. Here's an example of coordinating an increment, using an untyped transactor, similar to the explicitly coordinated example above. -.. includecode:: code/akka/docs/transactor/FriendlyCounter.java#class +.. includecode:: code/docs/transactor/FriendlyCounter.java#class :language: java To execute directly before or after the coordinated transaction, override the diff --git a/akka-docs/java/typed-actors.rst b/akka-docs/java/typed-actors.rst index b2d7a9bfae..4d36872f1a 100644 --- a/akka-docs/java/typed-actors.rst +++ b/akka-docs/java/typed-actors.rst @@ -38,7 +38,7 @@ The tools of the trade Before we create our first Typed Actor we should first go through the tools that we have at our disposal, it's located in ``akka.actor.TypedActor``. -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: typed-actor-extension-tools .. warning:: @@ -55,37 +55,37 @@ To create a Typed Actor you need to have one or more interfaces, and one impleme Our example interface: -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: imports,typed-actor-iface :exclude: typed-actor-iface-methods Our example implementation of that interface: -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: imports,typed-actor-impl :exclude: typed-actor-impl-methods The most trivial way of creating a Typed Actor instance of our ``Squarer``: -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: typed-actor-create1 First type is the type of the proxy, the second type is the type of the implementation. If you need to call a specific constructor you do it like this: -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: typed-actor-create2 Since you supply a ``Props``, you can specify which dispatcher to use, what the default timeout should be used and more. Now, our ``Squarer`` doesn't have any methods, so we'd better add those. -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: imports,typed-actor-iface Alright, now we've got some methods we can call, but we need to implement those in ``SquarerImpl``. -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: imports,typed-actor-impl Excellent, now we have an interface and an implementation of that interface, @@ -99,7 +99,7 @@ Methods returning: * ``void`` will be dispatched with ``fire-and-forget`` semantics, exactly like ``ActorRef.tell`` * ``akka.dispatch.Future`` will use ``send-request-reply`` semantics, exactly like ``ActorRef.ask`` * ``scala.Option`` or ``akka.japi.Option`` will use ``send-request-reply`` semantics, but *will* block to wait for an answer, - and return None if no answer was produced within the timout, or scala.Some/akka.japi.Some containing the result otherwise. + and return None if no answer was produced within the timeout, or scala.Some/akka.japi.Some containing the result otherwise. Any exception that was thrown during this call will be rethrown. * Any other type of value will use ``send-request-reply`` semantics, but *will* block to wait for an answer, throwing ``java.util.concurrent.TimeoutException`` if there was a timeout or rethrow any exception that was thrown during this call. @@ -113,7 +113,7 @@ we *strongly* recommend that parameters passed are immutable. One-way message send ^^^^^^^^^^^^^^^^^^^^ -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: typed-actor-call-oneway As simple as that! The method will be executed on another thread; asynchronously. @@ -121,22 +121,22 @@ As simple as that! The method will be executed on another thread; asynchronously Request-reply message send ^^^^^^^^^^^^^^^^^^^^^^^^^^ -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: typed-actor-call-option This will block for as long as the timeout that was set in the ``Props`` of the Typed Actor, if needed. It will return ``None`` if a timeout occurs. -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: typed-actor-call-strict -This will block for as long as the timeout that was set in the ``Props` of the Typed Actor, +This will block for as long as the timeout that was set in the ``Props`` of the Typed Actor, if needed. It will throw a ``java.util.concurrent.TimeoutException`` if a timeout occurs. Request-reply-with-future message send ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: typed-actor-call-future This call is asynchronous, and the Future returned can be used for asynchronous composition. @@ -146,12 +146,12 @@ Stopping Typed Actors Since Akka's Typed Actors are backed by Akka Actors they must be stopped when they aren't needed anymore. -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: typed-actor-stop This asynchronously stops the Typed Actor associated with the specified proxy ASAP. -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java +.. includecode:: code/docs/actor/TypedActorDocTestBase.java :include: typed-actor-poisonpill This asynchronously stops the Typed Actor associated with the specified proxy @@ -163,7 +163,11 @@ Typed Actor Hierarchies Since you can obtain a contextual Typed Actor Extension by passing in an ``ActorContext`` you can create child Typed Actors by invoking ``typedActorOf(..)`` on that. -This also works for creating child Typed Actors in regular Akka Actors. +.. includecode:: code/docs/actor/TypedActorDocTestBase.java + :include: typed-actor-hierarchy + +You can also create a child Typed Actor in regular Akka Actors by giving the ``UntypedActorContext`` +as an input parameter to TypedActor.get(…). Supervisor Strategy ------------------- @@ -204,4 +208,4 @@ Lookup & Remoting Since ``TypedActors`` are backed by ``Akka Actors``, you can use ``actorFor`` together with ``typedActorOf`` to proxy ``ActorRefs`` potentially residing on remote nodes. -.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java#typed-actor-remote \ No newline at end of file +.. includecode:: code/docs/actor/TypedActorDocTestBase.java#typed-actor-remote diff --git a/akka-docs/java/untyped-actors.rst b/akka-docs/java/untyped-actors.rst index 38d2d4c430..57dbaa5604 100644 --- a/akka-docs/java/untyped-actors.rst +++ b/akka-docs/java/untyped-actors.rst @@ -36,7 +36,7 @@ Actor in Java are implemented by extending the ``UntypedActor`` class and implem Here is an example: -.. includecode:: code/akka/docs/actor/MyUntypedActor.java#my-untyped-actor +.. includecode:: code/docs/actor/MyUntypedActor.java#my-untyped-actor Props ----- @@ -44,7 +44,7 @@ Props ``Props`` is a configuration class to specify options for the creation of actors. Here are some examples on how to create a ``Props`` instance. -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#creating-props-config +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#creating-props-config Creating Actors with Props @@ -52,13 +52,13 @@ Creating Actors with Props Actors are created by passing in a ``Props`` instance into the ``actorOf`` factory method. -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#creating-props +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#creating-props Creating Actors with default constructor ---------------------------------------- -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java :include: imports,system-actorOf The call to :meth:`actorOf` returns an instance of ``ActorRef``. This is a handle to @@ -75,20 +75,13 @@ how the supervisor hierarchy is arranged. When using the context the current act will be supervisor of the created child actor. When using the system it will be a top level actor, that is supervised by the system (internal guardian actor). -.. includecode:: code/akka/docs/actor/FirstUntypedActor.java#context-actorOf +.. includecode:: code/docs/actor/FirstUntypedActor.java#context-actorOf The name parameter is optional, but you should preferably name your actors, since that is used in log messages and for identifying actors. The name must not be empty or start with ``$``. If the given name is already in use by another child to the same parent actor an `InvalidActorNameException` is thrown. -.. warning:: - - Creating top-level actors with ``system.actorOf`` is a blocking operation, - hence it may dead-lock due to starvation if the default dispatcher is - overloaded. To avoid problems, do not call this method from within actors or - futures which run on the default dispatcher. - Actors are automatically started asynchronously when created. When you create the ``UntypedActor`` then it will automatically call the ``preStart`` callback method on the ``UntypedActor`` class. This is an excellent place to @@ -110,11 +103,17 @@ in which you can create the Actor in any way you like. Here is an example: -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#creating-constructor +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#creating-constructor This way of creating the Actor is also great for integrating with Dependency Injection (DI) frameworks like Guice or Spring. +.. warning:: + + You might be tempted at times to offer an ``UntypedActor`` factory which + always returns the same instance, e.g. by using a static field. This is not + supported, as it goes against the meaning of an actor restart, which is + described here: :ref:`supervision-restart`. UntypedActor API ================ @@ -144,7 +143,7 @@ In addition, it offers: The remaining visible methods are user-overridable life-cycle hooks which are described in the following: -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#lifecycle-callbacks +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#lifecycle-callbacks The implementations shown above are the defaults provided by the :class:`UntypedActor` class. @@ -163,7 +162,7 @@ termination (see `Stopping Actors`_). This service is provided by the Registering a monitor is easy (see fourth line, the rest is for demonstrating the whole functionality): -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#watch +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#watch It should be noted that the :class:`Terminated` message is generated independent of the order in which registration and termination occur. @@ -279,7 +278,7 @@ convention. Here is an example of an immutable message: -.. includecode:: code/akka/docs/actor/ImmutableMessage.java#immutable-message +.. includecode:: code/docs/actor/ImmutableMessage.java#immutable-message Send messages @@ -332,9 +331,9 @@ Ask: Send-And-Receive-Future The ``ask`` pattern involves actors as well as futures, hence it is offered as a use pattern rather than a method on :class:`ActorRef`: -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#import-askPipe +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#import-askPipe -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#ask-pipe +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#ask-pipe This example demonstrates ``ask`` together with the ``pipe`` pattern on futures, because this is likely to be a common combination. Please note that @@ -355,7 +354,7 @@ To complete the future with an exception you need send a Failure message to the This is *not done automatically* when an actor throws an exception while processing a message. -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#reply-exception +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java#reply-exception If the actor does not complete the future, it will expire after the timeout period, specified as parameter to the ``ask`` method; this will complete the @@ -364,7 +363,7 @@ specified as parameter to the ``ask`` method; this will complete the See :ref:`futures-java` for more information on how to await or query a future. -The ``onComplete``, ``onResult``, or ``onTimeout`` methods of the ``Future`` can be +The ``onComplete``, ``onSuccess``, or ``onFailure`` methods of the ``Future`` can be used to register a callback to get a notification when the Future completes. Gives you a way to avoid blocking. @@ -399,7 +398,7 @@ an abstract method on the ``UntypedActor`` base class that needs to be defined. Here is an example: -.. includecode:: code/akka/docs/actor/MyUntypedActor.java#my-untyped-actor +.. includecode:: code/docs/actor/MyUntypedActor.java#my-untyped-actor An alternative to using if-instanceof checks is to use `Apache Commons MethodUtils `_ @@ -432,7 +431,7 @@ received within a certain time. To receive this timeout you have to set the ``receiveTimeout`` property and declare handing for the ReceiveTimeout message. -.. includecode:: code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java#receive-timeout +.. includecode:: code/docs/actor/MyReceivedTimeoutUntypedActor.java#receive-timeout .. _stopping-actors-java: @@ -494,7 +493,7 @@ in the mailbox. Use it like this: -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java :include: import-actors,poison-pill Graceful Stop @@ -503,7 +502,7 @@ Graceful Stop :meth:`gracefulStop` is useful if you need to wait for termination or compose ordered termination of several actors: -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java :include: import-gracefulStop,gracefulStop When ``gracefulStop()`` returns successfully, the actor’s ``postStop()`` hook @@ -537,7 +536,7 @@ The hotswapped code is kept in a Stack which can be pushed and popped. To hotswap the Actor using ``getContext().become``: -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java :include: import-procedure,hot-swap-actor The ``become`` method is useful for many different things, such as to implement @@ -545,7 +544,7 @@ a Finite State Machine (FSM). Here is another little cute example of ``become`` and ``unbecome`` in action: -.. includecode:: code/akka/docs/actor/UntypedActorSwapper.java#swapper +.. includecode:: code/docs/actor/UntypedActorSwapper.java#swapper Downgrade --------- @@ -567,7 +566,7 @@ through regular supervisor semantics. Use it like this: -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java +.. includecode:: code/docs/actor/UntypedActorDocTestBase.java :include: import-actors,kill Actors and exceptions @@ -580,7 +579,7 @@ What happens to the Message --------------------------- If an exception is thrown while a message is being processed (so taken of his -mailbox and handed over the the receive), then this message will be lost. It is +mailbox and handed over to the receive), then this message will be lost. It is important to understand that it is not put back on the mailbox. So if you want to retry processing of a message, you need to deal with it yourself by catching the exception and retry your flow. Make sure that you put a bound on the number diff --git a/akka-docs/java/zeromq.rst b/akka-docs/java/zeromq.rst index ca136a588e..08d1a9541f 100644 --- a/akka-docs/java/zeromq.rst +++ b/akka-docs/java/zeromq.rst @@ -19,15 +19,15 @@ Connection ZeroMQ supports multiple connectivity patterns, each aimed to meet a different set of requirements. Currently, this module supports publisher-subscriber connections and connections based on dealers and routers. For connecting or accepting connections, a socket must be created. Sockets are always created using the ``akka.zeromq.ZeroMQExtension``, for example: -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#pub-socket +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#pub-socket Above examples will create a ZeroMQ Publisher socket that is Bound to the port 1233 on localhost. Similarly you can create a subscription socket, with a listener, that subscribes to all messages from the publisher using: -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#sub-socket +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#sub-socket -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#listener-actor +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#listener-actor The following sub-sections describe the supported connection patterns and how they can be used in an Akka environment. However, for a comprehensive discussion of connection patterns, please refer to `ZeroMQ -- The Guide `_. @@ -43,18 +43,18 @@ When you're using zeromq pub/sub you should be aware that it needs multicast - c An actor is subscribed to a topic as follows: -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#sub-topic-socket +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#sub-topic-socket It is a prefix match so it is subscribed to all topics starting with ``foo.bar``. Note that if the given string is empty or ``Subscribe.all()`` is used, the actor is subscribed to all topics. To unsubscribe from a topic you do the following: -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#unsub-topic-socket +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#unsub-topic-socket To publish messages to a topic you must use two Frames with the topic in the first frame. -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#pub-topic +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#pub-topic Pub-Sub in Action ^^^^^^^^^^^^^^^^^ @@ -64,22 +64,22 @@ The following example illustrates one publisher with two subscribers. The publisher monitors current heap usage and system load and periodically publishes ``Heap`` events on the ``"health.heap"`` topic and ``Load`` events on the ``"health.load"`` topic. -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#health +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#health -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#health2 +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#health2 Let's add one subscriber that logs the information. It subscribes to all topics starting with ``"health"``, i.e. both ``Heap`` and ``Load`` events. -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#logger +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#logger -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#logger2 +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#logger2 Another subscriber keep track of used heap and warns if too much heap is used. It only subscribes to ``Heap`` events. -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#alerter +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#alerter -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#alerter2 +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#alerter2 Router-Dealer Connection ------------------------ @@ -90,7 +90,7 @@ With those socket types you can build your own reliable pub sub broker that uses To create a Router socket that has a high watermark configured, you would do: -.. includecode:: code/akka/docs/zeromq/ZeromqDocTestBase.java#high-watermark +.. includecode:: code/docs/zeromq/ZeromqDocTestBase.java#high-watermark The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket. diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala deleted file mode 100644 index 2f67c607ed..0000000000 --- a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala +++ /dev/null @@ -1,40 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ -package akka.docs.actor.mailbox - -//#imports -import akka.actor.Props - -//#imports - -import org.scalatest.{ BeforeAndAfterAll, WordSpec } -import org.scalatest.matchers.MustMatchers -import akka.testkit.AkkaSpec -import akka.actor.Actor - -class MyActor extends Actor { - def receive = { - case x ⇒ - } -} - -object DurableMailboxDocSpec { - val config = """ - //#dispatcher-config - my-dispatcher { - mailbox-type = akka.actor.mailbox.FileBasedMailboxType - } - //#dispatcher-config - """ -} - -class DurableMailboxDocSpec extends AkkaSpec(DurableMailboxDocSpec.config) { - - "configuration of dispatcher with durable mailbox" in { - //#dispatcher-config-use - val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), name = "myactor") - //#dispatcher-config-use - } - -} diff --git a/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocSpec.scala b/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocSpec.scala new file mode 100644 index 0000000000..fc62cd940d --- /dev/null +++ b/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocSpec.scala @@ -0,0 +1,137 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package docs.actor.mailbox + +//#imports +import akka.actor.Props + +//#imports + +import org.scalatest.{ BeforeAndAfterAll, WordSpec } +import org.scalatest.matchers.MustMatchers +import akka.testkit.AkkaSpec +import akka.actor.{ Actor, ExtendedActorSystem } + +class MyActor extends Actor { + def receive = { + case x ⇒ + } +} + +object DurableMailboxDocSpec { + val config = """ + //#dispatcher-config + my-dispatcher { + mailbox-type = akka.actor.mailbox.FileBasedMailboxType + } + //#dispatcher-config + """ +} + +class DurableMailboxDocSpec extends AkkaSpec(DurableMailboxDocSpec.config) { + + "configuration of dispatcher with durable mailbox" in { + //#dispatcher-config-use + val myActor = system.actorOf(Props[MyActor]. + withDispatcher("my-dispatcher"), name = "myactor") + //#dispatcher-config-use + } + +} + +//#custom-mailbox +import com.typesafe.config.Config +import akka.actor.ActorContext +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.dispatch.Envelope +import akka.dispatch.MailboxType +import akka.dispatch.MessageQueue +import akka.actor.mailbox.DurableMessageQueue +import akka.actor.mailbox.DurableMessageSerialization +import akka.pattern.CircuitBreaker +import akka.util.duration._ + +class MyMailboxType(systemSettings: ActorSystem.Settings, config: Config) + extends MailboxType { + + override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = owner zip system headOption match { + case Some((o, s: ExtendedActorSystem)) ⇒ new MyMessageQueue(o, s) + case None ⇒ throw new IllegalArgumentException( + "requires an owner (i.e. does not work with BalancingDispatcher)") + } +} + +class MyMessageQueue(_owner: ActorRef, _system: ExtendedActorSystem) + extends DurableMessageQueue(_owner, _system) with DurableMessageSerialization { + + val storage = new QueueStorage + // A real-world implmentation would use configuration to set the last + // three parameters below + val breaker = CircuitBreaker(system.scheduler, 5, 30.seconds, 1.minute) + + def enqueue(receiver: ActorRef, envelope: Envelope): Unit = breaker.withSyncCircuitBreaker { + val data: Array[Byte] = serialize(envelope) + storage.push(data) + } + + def dequeue(): Envelope = breaker.withSyncCircuitBreaker { + val data: Option[Array[Byte]] = storage.pull() + data.map(deserialize).orNull + } + + def hasMessages: Boolean = breaker.withSyncCircuitBreaker { !storage.isEmpty } + + def numberOfMessages: Int = breaker.withSyncCircuitBreaker { storage.size } + + /** + * Called when the mailbox is disposed. + * An ordinary mailbox would send remaining messages to deadLetters, + * but the purpose of a durable mailbox is to continue + * with the same message queue when the actor is started again. + */ + def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = () + +} +//#custom-mailbox + +// dummy +class QueueStorage { + import java.util.concurrent.ConcurrentLinkedQueue + val queue = new ConcurrentLinkedQueue[Array[Byte]] + def push(data: Array[Byte]): Unit = queue.offer(data) + def pull(): Option[Array[Byte]] = Option(queue.poll()) + def isEmpty: Boolean = queue.isEmpty + def size: Int = queue.size +} + +//#custom-mailbox-test +import akka.actor.mailbox.DurableMailboxSpec + +object MyMailboxSpec { + val config = """ + MyStorage-dispatcher { + mailbox-type = docs.actor.mailbox.MyMailboxType + } + """ +} + +class MyMailboxSpec extends DurableMailboxSpec("MyStorage", MyMailboxSpec.config) { + override def atStartup() { + } + + override def atTermination() { + } + + "MyMailbox" must { + "deliver a message" in { + val actor = createMailboxTestActor() + implicit val sender = testActor + actor ! "hello" + expectMsg("hello") + } + + // add more tests + } +} \ No newline at end of file diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala b/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocTest.scala similarity index 85% rename from akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala rename to akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocTest.scala index eba732e6a7..6b156ef791 100644 --- a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala +++ b/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocTest.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor.mailbox +package docs.actor.mailbox import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java b/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocTestBase.java similarity index 75% rename from akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java rename to akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocTestBase.java index 834dc6f0fb..06e867c786 100644 --- a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java +++ b/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocTestBase.java @@ -1,12 +1,11 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor.mailbox; +package docs.actor.mailbox; //#imports -import akka.actor.UntypedActorFactory; -import akka.actor.UntypedActor; import akka.actor.Props; +import akka.actor.ActorRef; //#imports @@ -16,8 +15,8 @@ import org.junit.Test; import akka.testkit.AkkaSpec; import com.typesafe.config.ConfigFactory; -import akka.actor.ActorRef; import akka.actor.ActorSystem; +import akka.actor.UntypedActor; import static org.junit.Assert.*; @@ -39,12 +38,8 @@ public class DurableMailboxDocTestBase { @Test public void configDefinedDispatcher() { //#dispatcher-config-use - ActorRef myActor = system.actorOf( - new Props().withDispatcher("my-dispatcher").withCreator(new UntypedActorFactory() { - public UntypedActor create() { - return new MyUntypedActor(); - } - }), "myactor"); + ActorRef myActor = system.actorOf(new Props(MyUntypedActor.class). + withDispatcher("my-dispatcher"), "myactor"); //#dispatcher-config-use myActor.tell("test"); } diff --git a/akka-docs/modules/durable-mailbox.rst b/akka-docs/modules/durable-mailbox.rst index 74618d978e..5be40320d0 100644 --- a/akka-docs/modules/durable-mailbox.rst +++ b/akka-docs/modules/durable-mailbox.rst @@ -9,63 +9,38 @@ Overview ======== -Akka supports a set of durable mailboxes. A durable mailbox is a replacement for -the standard actor mailbox that is durable. What this means in practice is that -if there are pending messages in the actor's mailbox when the node of the actor -resides on crashes, then when you restart the node, the actor will be able to -continue processing as if nothing had happened; with all pending messages still -in its mailbox. +A durable mailbox is a mailbox which stores the messages on durable storage. +What this means in practice is that if there are pending messages in the actor's +mailbox when the node of the actor resides on crashes, then when you restart the +node, the actor will be able to continue processing as if nothing had happened; +with all pending messages still in its mailbox. -None of these mailboxes implements transactions for current message. It's possible -if the actor crashes after receiving a message, but before completing processing of -it, that the message could be lost. - -.. warning:: **IMPORTANT** - - None of these mailboxes work with blocking message send, i.e. the message - send operations that are relying on futures; ``?`` or ``ask``. If the node - has crashed and then restarted, the thread that was blocked waiting for the - reply is gone and there is no way we can deliver the message. - -The durable mailboxes supported out-of-the-box are: - - - ``FileBasedMailbox`` -- backed by a journaling transaction log on the local file system - -You can easily implement your own mailbox. Look at the existing implementation for inspiration. - -.. _DurableMailbox.General: - -General Usage -------------- - -The durable mailboxes and their configuration options reside in the -``akka.actor.mailbox`` package. - -You configure durable mailboxes through the dispatcher. The -actor is oblivious to which type of mailbox it is using. - -In the configuration of the dispatcher you specify the fully qualified class name -of the mailbox: - -.. includecode:: code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala - :include: dispatcher-config - -Here is an example of how to create an actor with a durable dispatcher, in Scala: - -.. includecode:: code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala - :include: imports,dispatcher-config-use - -Corresponding example in Java: - -.. includecode:: code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java - :include: imports,dispatcher-config-use - -The actor is oblivious to which type of mailbox it is using. +You configure durable mailboxes through the dispatcher. The actor is oblivious +to which type of mailbox it is using. This gives you an excellent way of creating bulkheads in your application, where groups of actors sharing the same dispatcher also share the same backing storage. Read more about that in the :ref:`dispatchers-scala` documentation. +One basic file based durable mailbox is provided by Akka out-of-the-box. +Other implementations can easily be added. Some are available as separate community +Open Source projects, such as: + +* `AMQP Durable Mailbox `_ + + +A durable mailbox is like any other mailbox not likely to be transactional. It's possible +if the actor crashes after receiving a message, but before completing processing of +it, that the message could be lost. + +.. warning:: + + A durable mailbox typically doesn't work with blocking message send, i.e. the message + send operations that are relying on futures; ``?`` or ``ask``. If the node + has crashed and then restarted, the thread that was blocked waiting for the + reply is gone and there is no way we can deliver the message. + + File-based durable mailbox ========================== @@ -74,14 +49,21 @@ system. It is the simplest to use since it does not require an extra infrastructure piece to administer, but it is usually sufficient and just what you need. -You configure durable mailboxes through the dispatcher, as described in -:ref:`DurableMailbox.General` with the following mailbox type. +In the configuration of the dispatcher you specify the fully qualified class name +of the mailbox: -Config:: +.. includecode:: code/docs/actor/mailbox/DurableMailboxDocSpec.scala + :include: dispatcher-config - my-dispatcher { - mailbox-type = akka.actor.mailbox.FileBasedMailboxType - } +Here is an example of how to create an actor with a durable dispatcher, in Scala: + +.. includecode:: code/docs/actor/mailbox/DurableMailboxDocSpec.scala + :include: imports,dispatcher-config-use + +Corresponding example in Java: + +.. includecode:: code/docs/actor/mailbox/DurableMailboxDocTestBase.java + :include: imports,dispatcher-config-use You can also configure and tune the file-based durable mailbox. This is done in the ``akka.actor.mailbox.file-based`` section in the :ref:`configuration`. @@ -89,3 +71,34 @@ the ``akka.actor.mailbox.file-based`` section in the :ref:`configuration`. .. literalinclude:: ../../akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf :language: none +How to implement a durable mailbox +================================== + +Here is an example of how to implement a custom durable mailbox. Essentially it consists of +a configurator (MailboxType) and a queue implementation (DurableMessageQueue). + +The envelope contains the message sent to the actor, and information about sender. It is the +envelope that needs to be stored. As a help utility you can mixin DurableMessageSerialization +to serialize and deserialize the envelope using the ordinary :ref:`serialization-scala` +mechanism. This optional and you may store the envelope data in any way you like. Durable +mailboxes are an excellent fit for usage of circuit breakers. These are described in the +:ref:`circuit-breaker` documentation. + +.. includecode:: code/docs/actor/mailbox/DurableMailboxDocSpec.scala + :include: custom-mailbox + +To facilitate testing of a durable mailbox you may use ``DurableMailboxSpec`` as base class. +It implements a few basic tests and helps you setup the a fixture. More tests can be +added in concrete subclass like this: + +.. includecode:: code/docs/actor/mailbox/DurableMailboxDocSpec.scala + :include: custom-mailbox-test + +You find DurableMailboxDocSpec in ``akka-mailboxes-common-test-2.1-SNAPSHOT.jar``. +Add this dependency:: + + "com.typesafe.akka" % "akka-mailboxes-common-test" % "2.1-SNAPSHOT" + +For more inspiration you can look at the old implementations based on Redis, MongoDB, Beanstalk, +and ZooKeeper, which can be found in Akka git repository tag +`v2.0.1 `_. \ No newline at end of file diff --git a/akka-docs/modules/http.rst b/akka-docs/modules/http.rst index c2c01d3ebc..b0f54948d3 100644 --- a/akka-docs/modules/http.rst +++ b/akka-docs/modules/http.rst @@ -1,18 +1,30 @@ .. _http-module: HTTP -==== +#### -Play2-mini ----------- +Play2 Mini +========== -The Akka team recommends the `Play2-mini `_ framework when building RESTful +The Akka team recommends the `Play2 Mini `_ framework when building RESTful service applications that integrates with Akka. It provides a REST API on top of `Play2 `_. Getting started --------------- First you must make your application aware of play-mini. -In SBT you just have to add the following to your _libraryDependencies_:: +In SBT you just have to add the following to your ``libraryDependencies``:: libraryDependencies += "com.typesafe" %% "play-mini" % "" + +Akka Mist +========= + +If you are using Akka Mist (Akka's old HTTP/REST module) with Akka 1.x and wish to upgrade to 2.x +there is now a port of Akka Mist to Akka 2.x. You can find it `here `_. + +Other Alternatives +================== + +There are a bunch of other alternatives for using Akka with HTTP/REST. You can find some of them +among the `Community Projects `_. diff --git a/akka-docs/project/licenses.rst b/akka-docs/project/licenses.rst index d7d9865b31..7dbcf5ef9f 100644 --- a/akka-docs/project/licenses.rst +++ b/akka-docs/project/licenses.rst @@ -12,7 +12,7 @@ Akka License This software is licensed under the Apache 2 license, quoted below. - Copyright 2009-2011 Typesafe Inc. + Copyright 2009-2012 Typesafe Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of @@ -196,4 +196,4 @@ Licenses for Dependency Libraries --------------------------------- Each dependency and its license can be seen in the project build file (the comment on the side of each dependency): -``_ +``_ diff --git a/akka-docs/project/sponsors.rst b/akka-docs/project/sponsors.rst index 085d35cc0d..630493f8c5 100644 --- a/akka-docs/project/sponsors.rst +++ b/akka-docs/project/sponsors.rst @@ -1,11 +1,23 @@ .. _sponsors: Sponsors -============ +======== + +Typesafe +-------- + +Typesafe is the company behind the Akka Project, Scala Programming Language, +Play Web Framework, Scala IDE, Simple Build Tool and many other open source +projects. It also provides the Typesafe Stack, a full-featured development +stack consisting of AKka, Play and Scala. Learn more at +`typesafe.com `_. YourKit ------- YourKit is kindly supporting open source projects with its full-featured Java Profiler. -YourKit, LLC is the creator of innovative and intelligent tools for profiling Java and .NET applications. -Take a look at YourKit’s leading software products: `YourKit Java Profiler `_ and `YourKit .NET Profiler `_ + +YourKit, LLC is the creator of innovative and intelligent tools for profiling Java +and .NET applications. Take a look at YourKit’s leading software products: +`YourKit Java Profiler `_ +and `YourKit .NET Profiler `_ diff --git a/akka-docs/scala/actors.rst b/akka-docs/scala/actors.rst index 5374c8a37c..47a2318e53 100644 --- a/akka-docs/scala/actors.rst +++ b/akka-docs/scala/actors.rst @@ -39,7 +39,7 @@ along with the implementation of how the messages should be processed. Here is an example: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala +.. includecode:: code/docs/actor/ActorDocSpec.scala :include: imports1,my-actor Please note that the Akka Actor ``receive`` message loop is exhaustive, which is @@ -52,7 +52,7 @@ published to the ``ActorSystem``'s ``EventStream``. Creating Actors with default constructor ---------------------------------------- -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala +.. includecode:: code/docs/actor/ActorDocSpec.scala :include: imports2,system-actorOf The call to :meth:`actorOf` returns an instance of ``ActorRef``. This is a handle to @@ -69,20 +69,13 @@ how the supervisor hierarchy is arranged. When using the context the current act will be supervisor of the created child actor. When using the system it will be a top level actor, that is supervised by the system (internal guardian actor). -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#context-actorOf +.. includecode:: code/docs/actor/ActorDocSpec.scala#context-actorOf The name parameter is optional, but you should preferably name your actors, since that is used in log messages and for identifying actors. The name must not be empty or start with ``$``. If the given name is already in use by another child to the same parent actor an `InvalidActorNameException` is thrown. -.. warning:: - - Creating top-level actors with ``system.actorOf`` is a blocking operation, - hence it may dead-lock due to starvation if the default dispatcher is - overloaded. To avoid problems, do not call this method from within actors or - futures which run on the default dispatcher. - Actors are automatically started asynchronously when created. When you create the ``Actor`` then it will automatically call the ``preStart`` callback method on the ``Actor`` trait. This is an excellent place to @@ -103,8 +96,15 @@ a call-by-name block in which you can create the Actor in any way you like. Here is an example: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#creating-constructor +.. includecode:: code/docs/actor/ActorDocSpec.scala#creating-constructor +.. warning:: + + You might be tempted at times to offer an ``Actor`` factory which always + returns the same instance, e.g. by using a ``lazy val`` or an + ``object ... extends Actor``. This is not supported, as it goes against the + meaning of an actor restart, which is described here: + :ref:`supervision-restart`. Props ----- @@ -112,7 +112,7 @@ Props ``Props`` is a configuration class to specify options for the creation of actors. Here are some examples on how to create a ``Props`` instance. -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#creating-props-config +.. includecode:: code/docs/actor/ActorDocSpec.scala#creating-props-config Creating Actors with Props @@ -120,7 +120,7 @@ Creating Actors with Props Actors are created by passing in a ``Props`` instance into the ``actorOf`` factory method. -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#creating-props +.. includecode:: code/docs/actor/ActorDocSpec.scala#creating-props Creating Actors using anonymous classes @@ -128,7 +128,7 @@ Creating Actors using anonymous classes When spawning actors for specific sub-tasks from within an actor, it may be convenient to include the code to be executed directly in place, using an anonymous class. -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#anonymous-actor +.. includecode:: code/docs/actor/ActorDocSpec.scala#anonymous-actor .. warning:: @@ -170,7 +170,7 @@ In addition, it offers: You can import the members in the :obj:`context` to avoid prefixing access with ``context.`` -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#import-context +.. includecode:: code/docs/actor/ActorDocSpec.scala#import-context The remaining visible methods are user-overridable life-cycle hooks which are described in the following:: @@ -199,7 +199,7 @@ termination (see `Stopping Actors`_). This service is provided by the Registering a monitor is easy: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#watch +.. includecode:: code/docs/actor/ActorDocSpec.scala#watch It should be noted that the :class:`Terminated` message is generated independent of the order in which registration and termination occur. @@ -371,7 +371,7 @@ Ask: Send-And-Receive-Future The ``ask`` pattern involves actors as well as futures, hence it is offered as a use pattern rather than a method on :class:`ActorRef`: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#ask-pipeTo +.. includecode:: code/docs/actor/ActorDocSpec.scala#ask-pipeTo This example demonstrates ``ask`` together with the ``pipeTo`` pattern on futures, because this is likely to be a common combination. Please note that @@ -391,7 +391,7 @@ To complete the future with an exception you need send a Failure message to the This is *not done automatically* when an actor throws an exception while processing a message. -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#reply-exception +.. includecode:: code/docs/actor/ActorDocSpec.scala#reply-exception If the actor does not complete the future, it will expire after the timeout period, completing it with an :class:`AskTimeoutException`. The timeout is @@ -399,16 +399,16 @@ taken from one of the following locations in order of precedence: 1. explicitly given timeout as in: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#using-explicit-timeout +.. includecode:: code/docs/actor/ActorDocSpec.scala#using-explicit-timeout 2. implicit argument of type :class:`akka.util.Timeout`, e.g. -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#using-implicit-timeout +.. includecode:: code/docs/actor/ActorDocSpec.scala#using-implicit-timeout See :ref:`futures-scala` for more information on how to await or query a future. -The ``onComplete``, ``onResult``, or ``onTimeout`` methods of the ``Future`` can be +The ``onComplete``, ``onSuccess``, or ``onFailure`` methods of the ``Future`` can be used to register a callback to get a notification when the Future completes. Gives you a way to avoid blocking. @@ -443,7 +443,7 @@ An Actor has to implement the ``receive`` method to receive messages: .. code-block:: scala - protected def receive: PartialFunction[Any, Unit] + def receive: PartialFunction[Any, Unit] Note: Akka has an alias to the ``PartialFunction[Any, Unit]`` type called ``Receive`` (``akka.actor.Actor.Receive``), so you can use this type instead for @@ -453,7 +453,7 @@ This method should return a ``PartialFunction``, e.g. a ‘match/case’ clause which the message can be matched against the different case clauses using Scala pattern matching. Here is an example: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala +.. includecode:: code/docs/actor/ActorDocSpec.scala :include: imports1,my-actor @@ -483,7 +483,7 @@ received within a certain time. To receive this timeout you have to set the ``receiveTimeout`` property and declare a case handing the ReceiveTimeout object. -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#receive-timeout +.. includecode:: code/docs/actor/ActorDocSpec.scala#receive-timeout .. _stopping-actors-scala: @@ -548,7 +548,7 @@ Graceful Stop :meth:`gracefulStop` is useful if you need to wait for termination or compose ordered termination of several actors: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#gracefulStop +.. includecode:: code/docs/actor/ActorDocSpec.scala#gracefulStop When ``gracefulStop()`` returns successfully, the actor’s ``postStop()`` hook will have been executed: there exists a happens-before edge between the end of @@ -584,7 +584,7 @@ pushed and popped. To hotswap the Actor behavior using ``become``: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#hot-swap-actor +.. includecode:: code/docs/actor/ActorDocSpec.scala#hot-swap-actor The ``become`` method is useful for many different things, but a particular nice example of it is in example where it is used to implement a Finite State Machine @@ -594,12 +594,12 @@ example of it is in example where it is used to implement a Finite State Machine Here is another little cute example of ``become`` and ``unbecome`` in action: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#swapper +.. includecode:: code/docs/actor/ActorDocSpec.scala#swapper Encoding Scala Actors nested receives without accidentally leaking memory ------------------------------------------------------------------------- -See this `Unnested receive example `_. +See this `Unnested receive example `_. Downgrade @@ -644,7 +644,7 @@ What happens to the Message --------------------------- If an exception is thrown while a message is being processed (so taken of his -mailbox and handed over the the receive), then this message will be lost. It is +mailbox and handed over to the receive), then this message will be lost. It is important to understand that it is not put back on the mailbox. So if you want to retry processing of a message, you need to deal with it yourself by catching the exception and retry your flow. Make sure that you put a bound on the number @@ -675,8 +675,8 @@ A bit advanced but very useful way of defining a base message handler and then extend that, either through inheritance or delegation, is to use ``PartialFunction.orElse`` chaining. -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#receive-orElse +.. includecode:: code/docs/actor/ActorDocSpec.scala#receive-orElse Or: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#receive-orElse2 +.. includecode:: code/docs/actor/ActorDocSpec.scala#receive-orElse2 diff --git a/akka-docs/scala/agents.rst b/akka-docs/scala/agents.rst index aad2dc6bd2..5657986a88 100644 --- a/akka-docs/scala/agents.rst +++ b/akka-docs/scala/agents.rst @@ -36,22 +36,22 @@ Creating and stopping Agents Agents are created by invoking ``Agent(value)`` passing in the Agent's initial value: -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#create +.. includecode:: code/docs/agent/AgentDocSpec.scala#create Note that creating an Agent requires an implicit ``ActorSystem`` (for creating the underlying actors). See :ref:`actor-systems` for more information about actor systems. An ActorSystem can be in implicit scope when creating an Agent: -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#create-implicit-system +.. includecode:: code/docs/agent/AgentDocSpec.scala#create-implicit-system Or the ActorSystem can be passed explicitly when creating an Agent: -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#create-explicit-system +.. includecode:: code/docs/agent/AgentDocSpec.scala#create-explicit-system An Agent will be running until you invoke ``close`` on it. Then it will be eligible for garbage collection (unless you hold on to it in some way). -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#close +.. includecode:: code/docs/agent/AgentDocSpec.scala#close Updating Agents @@ -65,7 +65,7 @@ the update will be applied but dispatches to an Agent from a single thread will occur in order. You apply a value or a function by invoking the ``send`` function. -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#send +.. includecode:: code/docs/agent/AgentDocSpec.scala#send You can also dispatch a function to update the internal state but on its own thread. This does not use the reactive thread pool and can be used for @@ -73,7 +73,7 @@ long-running or blocking operations. You do this with the ``sendOff`` method. Dispatches using either ``sendOff`` or ``send`` will still be executed in order. -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#send-off +.. includecode:: code/docs/agent/AgentDocSpec.scala#send-off Reading an Agent's value @@ -82,11 +82,11 @@ Reading an Agent's value Agents can be dereferenced (you can get an Agent's value) by invoking the Agent with parentheses like this: -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#read-apply +.. includecode:: code/docs/agent/AgentDocSpec.scala#read-apply Or by using the get method: -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#read-get +.. includecode:: code/docs/agent/AgentDocSpec.scala#read-get Reading an Agent's current value does not involve any message passing and happens immediately. So while updates to an Agent are asynchronous, reading the @@ -99,12 +99,12 @@ Awaiting an Agent's value It is also possible to read the value after all currently queued sends have completed. You can do this with ``await``: -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#read-await +.. includecode:: code/docs/agent/AgentDocSpec.scala#read-await You can also get a ``Future`` to this value, that will be completed after the currently queued updates have completed: -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#read-future +.. includecode:: code/docs/agent/AgentDocSpec.scala#read-future Transactional Agents @@ -115,7 +115,7 @@ that transaction. If you send to an Agent within a transaction then the dispatch to the Agent will be held until that transaction commits, and discarded if the transaction is aborted. Here's an example: -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#transfer-example +.. includecode:: code/docs/agent/AgentDocSpec.scala#transfer-example Monadic usage @@ -128,4 +128,4 @@ as-is. They are so-called 'persistent'. Example of monadic usage: -.. includecode:: code/akka/docs/agent/AgentDocSpec.scala#monadic-example +.. includecode:: code/docs/agent/AgentDocSpec.scala#monadic-example diff --git a/akka-docs/scala/camel.rst b/akka-docs/scala/camel.rst index 9c24d123f0..886e00c320 100644 --- a/akka-docs/scala/camel.rst +++ b/akka-docs/scala/camel.rst @@ -52,7 +52,7 @@ Consumer Usage of Camel's integration components in Akka is essentially a one-liner. Here's an example. -.. includecode:: code/akka/docs/camel/Introduction.scala#Consumer-mina +.. includecode:: code/docs/camel/Introduction.scala#Consumer-mina The above example exposes an actor over a tcp endpoint on port 6200 via Apache Camel's `Mina component`_. The actor implements the endpointUri method to define @@ -64,14 +64,14 @@ component`_), only the actor's endpointUri method must be changed. .. _Mina component: http://camel.apache.org/mina.html .. _Jetty component: http://camel.apache.org/jetty.html -.. includecode:: code/akka/docs/camel/Introduction.scala#Consumer +.. includecode:: code/docs/camel/Introduction.scala#Consumer Producer -------- Actors can also trigger message exchanges with external systems i.e. produce to Camel endpoints. -.. includecode:: code/akka/docs/camel/Introduction.scala +.. includecode:: code/docs/camel/Introduction.scala :include: imports,Producer In the above example, any message sent to this actor will be sent to @@ -127,7 +127,7 @@ messages from the ``file:data/input/actor`` Camel endpoint. .. _Consumer: http://github.com/akka/akka/blob/master/akka-camel/src/main/scala/akka/camel/Consumer.scala -.. includecode:: code/akka/docs/camel/Consumers.scala#Consumer1 +.. includecode:: code/docs/camel/Consumers.scala#Consumer1 Whenever a file is put into the data/input/actor directory, its content is picked up by the Camel `file component`_ and sent as message to the @@ -146,7 +146,7 @@ from localhost on port 8877. .. _Jetty component: http://camel.apache.org/jetty.html .. _Jetty: http://www.eclipse.org/jetty/ -.. includecode:: code/akka/docs/camel/Consumers.scala#Consumer2 +.. includecode:: code/docs/camel/Consumers.scala#Consumer2 After starting the actor, clients can send messages to that actor by POSTing to ``http://localhost:8877/camel/default``. The actor sends a response by using the diff --git a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala b/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala deleted file mode 100644 index e614cc9903..0000000000 --- a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala +++ /dev/null @@ -1,180 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ -package akka.docs.serialization - -import org.scalatest.matchers.MustMatchers -import akka.testkit._ -import akka.actor.{ ActorRef, ActorSystem } - -//#imports -import akka.serialization._ -import com.typesafe.config.ConfigFactory - -//#imports - -//#my-own-serializer -class MyOwnSerializer extends Serializer { - - // This is whether "fromBinary" requires a "clazz" or not - def includeManifest: Boolean = false - - // Pick a unique identifier for your Serializer, - // you've got a couple of billions to choose from, - // 0 - 16 is reserved by Akka itself - def identifier = 1234567 - - // "toBinary" serializes the given object to an Array of Bytes - def toBinary(obj: AnyRef): Array[Byte] = { - // Put the code that serializes the object here - //#... - Array[Byte]() - //#... - } - - // "fromBinary" deserializes the given array, - // using the type hint (if any, see "includeManifest" above) - // into the optionally provided classLoader. - def fromBinary(bytes: Array[Byte], - clazz: Option[Class[_]]): AnyRef = { - // Put your code that deserializes here - //#... - null - //#... - } -} -//#my-own-serializer - -trait MyOwnSerializable -case class Customer(name: String) extends MyOwnSerializable - -class SerializationDocSpec extends AkkaSpec { - "demonstrate configuration of serialize messages" in { - //#serialize-messages-config - val config = ConfigFactory.parseString(""" - akka { - actor { - serialize-messages = on - } - } - """) - //#serialize-messages-config - val a = ActorSystem("system", config) - a.settings.SerializeAllMessages must be(true) - a.shutdown() - } - - "demonstrate configuration of serialize creators" in { - //#serialize-creators-config - val config = ConfigFactory.parseString(""" - akka { - actor { - serialize-creators = on - } - } - """) - //#serialize-creators-config - val a = ActorSystem("system", config) - a.settings.SerializeAllCreators must be(true) - a.shutdown() - } - - "demonstrate configuration of serializers" in { - //#serialize-serializers-config - val config = ConfigFactory.parseString(""" - akka { - actor { - serializers { - java = "akka.serialization.JavaSerializer" - proto = "akka.serialization.ProtobufSerializer" - myown = "akka.docs.serialization.MyOwnSerializer" - } - } - } - """) - //#serialize-serializers-config - val a = ActorSystem("system", config) - a.shutdown() - } - - "demonstrate configuration of serialization-bindings" in { - //#serialization-bindings-config - val config = ConfigFactory.parseString(""" - akka { - actor { - serializers { - java = "akka.serialization.JavaSerializer" - proto = "akka.serialization.ProtobufSerializer" - myown = "akka.docs.serialization.MyOwnSerializer" - } - - serialization-bindings { - "java.lang.String" = java - "akka.docs.serialization.Customer" = java - "com.google.protobuf.Message" = proto - "akka.docs.serialization.MyOwnSerializable" = myown - "java.lang.Boolean" = myown - } - } - } - """) - //#serialization-bindings-config - val a = ActorSystem("system", config) - SerializationExtension(a).serializerFor(classOf[String]).getClass must equal(classOf[JavaSerializer]) - SerializationExtension(a).serializerFor(classOf[Customer]).getClass must equal(classOf[JavaSerializer]) - SerializationExtension(a).serializerFor(classOf[java.lang.Boolean]).getClass must equal(classOf[MyOwnSerializer]) - a.shutdown() - } - - "demonstrate the programmatic API" in { - //#programmatic - val system = ActorSystem("example") - - // Get the Serialization Extension - val serialization = SerializationExtension(system) - - // Have something to serialize - val original = "woohoo" - - // Find the Serializer for it - val serializer = serialization.findSerializerFor(original) - - // Turn it into bytes - val bytes = serializer.toBinary(original) - - // Turn it back into an object - val back = serializer.fromBinary(bytes, manifest = None) - - // Voilá! - back must equal(original) - - //#programmatic - system.shutdown() - } - - "demonstrate serialization of ActorRefs" in { - val theActorRef: ActorRef = system.deadLetters - val theActorSystem: ActorSystem = system - - //#actorref-serializer - // Serialize - // (beneath toBinary) - - // If there is no transportAddress, - // it means that either this Serializer isn't called - // within a piece of code that sets it, - // so either you need to supply your own, - // or simply use the local path. - val identifier: String = Serialization.currentTransportAddress.value match { - case null ⇒ theActorRef.path.toString - case address ⇒ theActorRef.path.toStringWithAddress(address) - } - // Then just serialize the identifier however you like - - // Deserialize - // (beneath fromBinary) - val deserializedActorRef = theActorSystem actorFor identifier - // Then just use the ActorRef - //#actorref-serializer - } -} diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/docs/actor/ActorDocSpec.scala similarity index 98% rename from akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala rename to akka-docs/scala/code/docs/actor/ActorDocSpec.scala index 0bc540f970..ee05e95d42 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/docs/actor/ActorDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor //#imports1 import akka.actor.Actor @@ -326,14 +326,13 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { //#gracefulStop import akka.pattern.gracefulStop import akka.dispatch.Await - import akka.actor.ActorTimeoutException try { val stopped: Future[Boolean] = gracefulStop(actorRef, 5 seconds)(system) Await.result(stopped, 6 seconds) // the actor has been stopped } catch { - case e: ActorTimeoutException ⇒ // the actor wasn't stopped within 5 seconds + case e: akka.pattern.AskTimeoutException ⇒ // the actor wasn't stopped within 5 seconds } //#gracefulStop } diff --git a/akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala b/akka-docs/scala/code/docs/actor/FSMDocSpec.scala similarity index 99% rename from akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala rename to akka-docs/scala/code/docs/actor/FSMDocSpec.scala index 158f8979a0..75b0309a42 100644 --- a/akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala +++ b/akka-docs/scala/code/docs/actor/FSMDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor //#test-code import akka.testkit.AkkaSpec diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala b/akka-docs/scala/code/docs/actor/FaultHandlingDocSample.scala similarity index 99% rename from akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala rename to akka-docs/scala/code/docs/actor/FaultHandlingDocSample.scala index d08bcb53b2..79f5a5d084 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala +++ b/akka-docs/scala/code/docs/actor/FaultHandlingDocSample.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor //#all //#imports diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala b/akka-docs/scala/code/docs/actor/FaultHandlingDocSpec.scala similarity index 96% rename from akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala rename to akka-docs/scala/code/docs/actor/FaultHandlingDocSpec.scala index ca1eccb73a..65e03bd2ea 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala +++ b/akka-docs/scala/code/docs/actor/FaultHandlingDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor //#testkit import akka.testkit.{ AkkaSpec, ImplicitSender, EventFilter } @@ -111,7 +111,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender { //#stop watch(child) // have testActor watch “child” child ! new IllegalArgumentException // break it - expectMsg(Terminated(child)) + expectMsg(Terminated(child)(existenceConfirmed = true)) child.isTerminated must be(true) //#stop } @@ -125,7 +125,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender { expectMsg(0) child2 ! new Exception("CRASH") // escalate failure - expectMsg(Terminated(child2)) + expectMsg(Terminated(child2)(existenceConfirmed = true)) //#escalate-kill //#escalate-restart val supervisor2 = system.actorOf(Props[Supervisor2], "supervisor2") diff --git a/akka-docs/scala/code/akka/docs/actor/SchedulerDocSpec.scala b/akka-docs/scala/code/docs/actor/SchedulerDocSpec.scala similarity index 98% rename from akka-docs/scala/code/akka/docs/actor/SchedulerDocSpec.scala rename to akka-docs/scala/code/docs/actor/SchedulerDocSpec.scala index b6bffecb46..f711d85129 100644 --- a/akka-docs/scala/code/akka/docs/actor/SchedulerDocSpec.scala +++ b/akka-docs/scala/code/docs/actor/SchedulerDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor //#imports1 import akka.actor.Actor diff --git a/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala b/akka-docs/scala/code/docs/actor/TypedActorDocSpec.scala similarity index 93% rename from akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala rename to akka-docs/scala/code/docs/actor/TypedActorDocSpec.scala index f7c5fa9bf7..0c2f3bd5b8 100644 --- a/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala +++ b/akka-docs/scala/code/docs/actor/TypedActorDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor //#imports import akka.dispatch.{ Promise, Future, Await } @@ -151,6 +151,18 @@ class TypedActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { //#typed-actor-remote } + "create hierarchies" in { + try { + //#typed-actor-hierarchy + //Inside your Typed Actor + val childSquarer: Squarer = TypedActor(TypedActor.context).typedActorOf(TypedProps[SquarerImpl]()) + //Use "childSquarer" as a Squarer + //#typed-actor-hierarchy + } catch { + case e: Exception ⇒ //ignore + } + } + "supercharge" in { //#typed-actor-supercharge-usage val awesomeFooBar: Foo with Bar = TypedActor(system).typedActorOf(TypedProps[FooBar]()) diff --git a/akka-docs/scala/code/akka/docs/actor/UnnestedReceives.scala b/akka-docs/scala/code/docs/actor/UnnestedReceives.scala similarity index 98% rename from akka-docs/scala/code/akka/docs/actor/UnnestedReceives.scala rename to akka-docs/scala/code/docs/actor/UnnestedReceives.scala index 194a958cce..bb77fe4d1d 100644 --- a/akka-docs/scala/code/akka/docs/actor/UnnestedReceives.scala +++ b/akka-docs/scala/code/docs/actor/UnnestedReceives.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.actor +package docs.actor import akka.actor._ import scala.collection.mutable.ListBuffer diff --git a/akka-docs/scala/code/akka/docs/agent/AgentDocSpec.scala b/akka-docs/scala/code/docs/agent/AgentDocSpec.scala similarity index 99% rename from akka-docs/scala/code/akka/docs/agent/AgentDocSpec.scala rename to akka-docs/scala/code/docs/agent/AgentDocSpec.scala index 1e9ec1fd69..418159f638 100644 --- a/akka-docs/scala/code/akka/docs/agent/AgentDocSpec.scala +++ b/akka-docs/scala/code/docs/agent/AgentDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.agent +package docs.agent import akka.agent.Agent import akka.util.duration._ diff --git a/akka-docs/scala/code/akka/docs/camel/Consumers.scala b/akka-docs/scala/code/docs/camel/Consumers.scala similarity index 96% rename from akka-docs/scala/code/akka/docs/camel/Consumers.scala rename to akka-docs/scala/code/docs/camel/Consumers.scala index 90f181df3f..df7161b9e6 100644 --- a/akka-docs/scala/code/akka/docs/camel/Consumers.scala +++ b/akka-docs/scala/code/docs/camel/Consumers.scala @@ -1,4 +1,4 @@ -package akka.docs.camel +package docs.camel object Consumers { { diff --git a/akka-docs/scala/code/akka/docs/camel/Introduction.scala b/akka-docs/scala/code/docs/camel/Introduction.scala similarity index 98% rename from akka-docs/scala/code/akka/docs/camel/Introduction.scala rename to akka-docs/scala/code/docs/camel/Introduction.scala index 4899843a27..eaf4c400f6 100644 --- a/akka-docs/scala/code/akka/docs/camel/Introduction.scala +++ b/akka-docs/scala/code/docs/camel/Introduction.scala @@ -1,4 +1,4 @@ -package akka.docs.camel +package docs.camel object Introduction { { diff --git a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala b/akka-docs/scala/code/docs/dispatcher/DispatcherDocSpec.scala similarity index 93% rename from akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala rename to akka-docs/scala/code/docs/dispatcher/DispatcherDocSpec.scala index 1452d72088..7fdd0cd9bf 100644 --- a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala +++ b/akka-docs/scala/code/docs/dispatcher/DispatcherDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.dispatcher +package docs.dispatcher import org.scalatest.{ BeforeAndAfterAll, WordSpec } import org.scalatest.matchers.MustMatchers @@ -91,13 +91,13 @@ object DispatcherDocSpec { //#prio-dispatcher-config prio-dispatcher { - mailbox-type = "akka.docs.dispatcher.DispatcherDocSpec$MyPrioMailbox" + mailbox-type = "docs.dispatcher.DispatcherDocSpec$MyPrioMailbox" } //#prio-dispatcher-config //#prio-dispatcher-config-java prio-dispatcher-java { - mailbox-type = "akka.docs.dispatcher.DispatcherDocTestBase$MyPrioMailbox" + mailbox-type = "docs.dispatcher.DispatcherDocTestBase$MyPrioMailbox" //Other dispatcher configuration goes here } //#prio-dispatcher-config-java @@ -134,8 +134,8 @@ object DispatcherDocSpec { } //#mailbox-implementation-example - case class MyUnboundedMailbox() extends akka.dispatch.MailboxType { - import akka.actor.ActorContext + class MyUnboundedMailbox extends akka.dispatch.MailboxType { + import akka.actor.{ ActorRef, ActorSystem } import com.typesafe.config.Config import java.util.concurrent.ConcurrentLinkedQueue import akka.dispatch.{ @@ -149,12 +149,12 @@ object DispatcherDocSpec { def this(settings: ActorSystem.Settings, config: Config) = this() // The create method is called to create the MessageQueue - final override def create(owner: Option[ActorContext]): MessageQueue = + final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new QueueBasedMessageQueue with UnboundedMessageQueueSemantics { final val queue = new ConcurrentLinkedQueue[Envelope]() } - //#mailbox-implementation-example } + //#mailbox-implementation-example } class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) { diff --git a/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala b/akka-docs/scala/code/docs/event/LoggingDocSpec.scala similarity index 99% rename from akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala rename to akka-docs/scala/code/docs/event/LoggingDocSpec.scala index 0aa29549c9..7e2fccb876 100644 --- a/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala +++ b/akka-docs/scala/code/docs/event/LoggingDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.event +package docs.event import akka.testkit.AkkaSpec import akka.actor.Actor diff --git a/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala b/akka-docs/scala/code/docs/extension/ExtensionDocSpec.scala similarity index 96% rename from akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala rename to akka-docs/scala/code/docs/extension/ExtensionDocSpec.scala index 05baa28ecb..c2558fb4f1 100644 --- a/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala +++ b/akka-docs/scala/code/docs/extension/ExtensionDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.extension +package docs.extension import java.util.concurrent.atomic.AtomicLong import akka.actor.Actor @@ -45,7 +45,7 @@ object ExtensionDocSpec { val config = """ //#config akka { - extensions = ["akka.docs.extension.CountExtension$"] + extensions = ["docs.extension.CountExtension$"] } //#config """ diff --git a/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala b/akka-docs/scala/code/docs/extension/SettingsExtensionDocSpec.scala similarity index 98% rename from akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala rename to akka-docs/scala/code/docs/extension/SettingsExtensionDocSpec.scala index 05765d27a5..a1e033e386 100644 --- a/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala +++ b/akka-docs/scala/code/docs/extension/SettingsExtensionDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.extension +package docs.extension //#imports import akka.actor.Extension diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/docs/future/FutureDocSpec.scala similarity index 99% rename from akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala rename to akka-docs/scala/code/docs/future/FutureDocSpec.scala index cee2eaeef8..66e80578fd 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/docs/future/FutureDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.future +package docs.future import org.scalatest.{ BeforeAndAfterAll, WordSpec } import org.scalatest.matchers.MustMatchers diff --git a/akka-docs/scala/code/akka/docs/io/HTTPServer.scala b/akka-docs/scala/code/docs/io/HTTPServer.scala similarity index 98% rename from akka-docs/scala/code/akka/docs/io/HTTPServer.scala rename to akka-docs/scala/code/docs/io/HTTPServer.scala index 837dbf4264..b6b80aa27f 100644 --- a/akka-docs/scala/code/akka/docs/io/HTTPServer.scala +++ b/akka-docs/scala/code/docs/io/HTTPServer.scala @@ -1,7 +1,7 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.io +package docs.io //#imports import akka.actor._ diff --git a/akka-docs/scala/code/docs/pattern/ScalaTemplate.scala b/akka-docs/scala/code/docs/pattern/ScalaTemplate.scala new file mode 100644 index 0000000000..beceae17b7 --- /dev/null +++ b/akka-docs/scala/code/docs/pattern/ScalaTemplate.scala @@ -0,0 +1,16 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package docs.pattern + +// this part will not appear in the docs + +//#all-of-it +class ScalaTemplate { + println("Hello, Template!") + //#uninteresting-stuff + // don’t show this plumbimg + //#uninteresting-stuff +} +//#all-of-it diff --git a/akka-docs/scala/code/akka/docs/remoting/RemoteDeploymentDocSpec.scala b/akka-docs/scala/code/docs/remoting/RemoteDeploymentDocSpec.scala similarity index 98% rename from akka-docs/scala/code/akka/docs/remoting/RemoteDeploymentDocSpec.scala rename to akka-docs/scala/code/docs/remoting/RemoteDeploymentDocSpec.scala index 0c65b3dc76..b391494a3b 100644 --- a/akka-docs/scala/code/akka/docs/remoting/RemoteDeploymentDocSpec.scala +++ b/akka-docs/scala/code/docs/remoting/RemoteDeploymentDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.remoting +package docs.remoting import akka.actor.{ ExtendedActorSystem, ActorSystem, Actor, ActorRef } import akka.testkit.{ AkkaSpec, ImplicitSender } diff --git a/akka-docs/scala/code/akka/docs/routing/RouterDocSpec.scala b/akka-docs/scala/code/docs/routing/RouterDocSpec.scala similarity index 96% rename from akka-docs/scala/code/akka/docs/routing/RouterDocSpec.scala rename to akka-docs/scala/code/docs/routing/RouterDocSpec.scala index 229c66f13e..c71228d06c 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterDocSpec.scala +++ b/akka-docs/scala/code/docs/routing/RouterDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.routing +package docs.routing import RouterDocSpec.MyActor import akka.actor.{ Props, Actor } diff --git a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala b/akka-docs/scala/code/docs/routing/RouterTypeExample.scala similarity index 99% rename from akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala rename to akka-docs/scala/code/docs/routing/RouterTypeExample.scala index 6ec475a874..421c7af3bb 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala +++ b/akka-docs/scala/code/docs/routing/RouterTypeExample.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.routing +package docs.routing import akka.routing.{ ScatterGatherFirstCompletedRouter, BroadcastRouter, RandomRouter, RoundRobinRouter } import annotation.tailrec diff --git a/akka-docs/scala/code/akka/docs/routing/RouterViaConfigExample.scala b/akka-docs/scala/code/docs/routing/RouterViaConfigExample.scala similarity index 98% rename from akka-docs/scala/code/akka/docs/routing/RouterViaConfigExample.scala rename to akka-docs/scala/code/docs/routing/RouterViaConfigExample.scala index cc840eedc5..5d34e429bb 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterViaConfigExample.scala +++ b/akka-docs/scala/code/docs/routing/RouterViaConfigExample.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.routing +package docs.routing import akka.actor.{ Actor, Props, ActorSystem } import com.typesafe.config.ConfigFactory diff --git a/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala b/akka-docs/scala/code/docs/routing/RouterViaProgramExample.scala similarity index 95% rename from akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala rename to akka-docs/scala/code/docs/routing/RouterViaProgramExample.scala index 50b141e7b7..79219b742b 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala +++ b/akka-docs/scala/code/docs/routing/RouterViaProgramExample.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.routing +package docs.routing import akka.routing.RoundRobinRouter import akka.actor.{ ActorRef, Props, Actor, ActorSystem } @@ -29,7 +29,7 @@ object RoutingProgrammaticallyExample extends App { val actor2 = system.actorOf(Props[ExampleActor1]) val actor3 = system.actorOf(Props[ExampleActor1]) val routees = Vector[ActorRef](actor1, actor2, actor3) - val router2 = system.actorOf(Props[ExampleActor1].withRouter( + val router2 = system.actorOf(Props().withRouter( RoundRobinRouter(routees = routees))) //#programmaticRoutingRoutees 1 to 6 foreach { i ⇒ router2 ! Message1(i) } diff --git a/akka-docs/scala/code/docs/serialization/SerializationDocSpec.scala b/akka-docs/scala/code/docs/serialization/SerializationDocSpec.scala new file mode 100644 index 0000000000..9b222436da --- /dev/null +++ b/akka-docs/scala/code/docs/serialization/SerializationDocSpec.scala @@ -0,0 +1,227 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +//#extract-transport +package object akka { + // needs to be inside the akka package because accessing unsupported API ! + def transportOf(system: actor.ExtendedActorSystem): remote.RemoteTransport = + system.provider match { + case r: remote.RemoteActorRefProvider ⇒ r.transport + case _ ⇒ + throw new UnsupportedOperationException( + "this method requires the RemoteActorRefProvider to be configured") + } +} +//#extract-transport + +package docs.serialization { + + import org.scalatest.matchers.MustMatchers + import akka.testkit._ + //#imports + import akka.actor.{ ActorRef, ActorSystem } + import akka.serialization._ + import com.typesafe.config.ConfigFactory + + //#imports + import akka.actor.ExtensionKey + import akka.actor.ExtendedActorSystem + import akka.actor.Extension + import akka.actor.Address + import akka.remote.RemoteActorRefProvider + + //#my-own-serializer + class MyOwnSerializer extends Serializer { + + // This is whether "fromBinary" requires a "clazz" or not + def includeManifest: Boolean = false + + // Pick a unique identifier for your Serializer, + // you've got a couple of billions to choose from, + // 0 - 16 is reserved by Akka itself + def identifier = 1234567 + + // "toBinary" serializes the given object to an Array of Bytes + def toBinary(obj: AnyRef): Array[Byte] = { + // Put the code that serializes the object here + //#... + Array[Byte]() + //#... + } + + // "fromBinary" deserializes the given array, + // using the type hint (if any, see "includeManifest" above) + // into the optionally provided classLoader. + def fromBinary(bytes: Array[Byte], + clazz: Option[Class[_]]): AnyRef = { + // Put your code that deserializes here + //#... + null + //#... + } + } + //#my-own-serializer + + trait MyOwnSerializable + case class Customer(name: String) extends MyOwnSerializable + + class SerializationDocSpec extends AkkaSpec { + "demonstrate configuration of serialize messages" in { + //#serialize-messages-config + val config = ConfigFactory.parseString(""" + akka { + actor { + serialize-messages = on + } + } + """) + //#serialize-messages-config + val a = ActorSystem("system", config) + a.settings.SerializeAllMessages must be(true) + a.shutdown() + } + + "demonstrate configuration of serialize creators" in { + //#serialize-creators-config + val config = ConfigFactory.parseString(""" + akka { + actor { + serialize-creators = on + } + } + """) + //#serialize-creators-config + val a = ActorSystem("system", config) + a.settings.SerializeAllCreators must be(true) + a.shutdown() + } + + "demonstrate configuration of serializers" in { + //#serialize-serializers-config + val config = ConfigFactory.parseString(""" + akka { + actor { + serializers { + java = "akka.serialization.JavaSerializer" + proto = "akka.serialization.ProtobufSerializer" + myown = "docs.serialization.MyOwnSerializer" + } + } + } + """) + //#serialize-serializers-config + val a = ActorSystem("system", config) + a.shutdown() + } + + "demonstrate configuration of serialization-bindings" in { + //#serialization-bindings-config + val config = ConfigFactory.parseString(""" + akka { + actor { + serializers { + java = "akka.serialization.JavaSerializer" + proto = "akka.serialization.ProtobufSerializer" + myown = "docs.serialization.MyOwnSerializer" + } + + serialization-bindings { + "java.lang.String" = java + "docs.serialization.Customer" = java + "com.google.protobuf.Message" = proto + "docs.serialization.MyOwnSerializable" = myown + "java.lang.Boolean" = myown + } + } + } + """) + //#serialization-bindings-config + val a = ActorSystem("system", config) + SerializationExtension(a).serializerFor(classOf[String]).getClass must equal(classOf[JavaSerializer]) + SerializationExtension(a).serializerFor(classOf[Customer]).getClass must equal(classOf[JavaSerializer]) + SerializationExtension(a).serializerFor(classOf[java.lang.Boolean]).getClass must equal(classOf[MyOwnSerializer]) + a.shutdown() + } + + "demonstrate the programmatic API" in { + //#programmatic + val system = ActorSystem("example") + + // Get the Serialization Extension + val serialization = SerializationExtension(system) + + // Have something to serialize + val original = "woohoo" + + // Find the Serializer for it + val serializer = serialization.findSerializerFor(original) + + // Turn it into bytes + val bytes = serializer.toBinary(original) + + // Turn it back into an object + val back = serializer.fromBinary(bytes, manifest = None) + + // Voilá! + back must equal(original) + + //#programmatic + system.shutdown() + } + + "demonstrate serialization of ActorRefs" in { + val theActorRef: ActorRef = system.deadLetters + val theActorSystem: ActorSystem = system + + //#actorref-serializer + // Serialize + // (beneath toBinary) + + // If there is no transportAddress, + // it means that either this Serializer isn't called + // within a piece of code that sets it, + // so either you need to supply your own, + // or simply use the local path. + val identifier: String = Serialization.currentTransportAddress.value match { + case null ⇒ theActorRef.path.toString + case address ⇒ theActorRef.path.toStringWithAddress(address) + } + // Then just serialize the identifier however you like + + // Deserialize + // (beneath fromBinary) + val deserializedActorRef = theActorSystem actorFor identifier + // Then just use the ActorRef + //#actorref-serializer + + //#external-address + object ExternalAddress extends ExtensionKey[ExternalAddressExt] + + class ExternalAddressExt(system: ExtendedActorSystem) extends Extension { + def addressFor(remoteAddr: Address): Address = + system.provider.getExternalAddressFor(remoteAddr) getOrElse + (throw new UnsupportedOperationException("cannot send to " + remoteAddr)) + } + + def serializeTo(ref: ActorRef, remote: Address): String = + ref.path.toStringWithAddress(ExternalAddress(theActorSystem).addressFor(remote)) + //#external-address + } + + "demonstrate how to do default Akka serialization of ActorRef" in { + val theActorSystem: ActorSystem = system + + //#external-address-default + object ExternalAddress extends ExtensionKey[ExternalAddressExt] + + class ExternalAddressExt(system: ExtendedActorSystem) extends Extension { + def addressForAkka: Address = akka.transportOf(system).address + } + + def serializeAkkaDefault(ref: ActorRef): String = + ref.path.toStringWithAddress(ExternalAddress(theActorSystem).addressForAkka) + //#external-address-default + } + } +} diff --git a/akka-docs/scala/code/akka/docs/testkit/PlainWordSpec.scala b/akka-docs/scala/code/docs/testkit/PlainWordSpec.scala similarity index 97% rename from akka-docs/scala/code/akka/docs/testkit/PlainWordSpec.scala rename to akka-docs/scala/code/docs/testkit/PlainWordSpec.scala index 8df13da2ca..2da67c9156 100644 --- a/akka-docs/scala/code/akka/docs/testkit/PlainWordSpec.scala +++ b/akka-docs/scala/code/docs/testkit/PlainWordSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.testkit +package docs.testkit //#plain-spec import akka.actor.ActorSystem diff --git a/akka-docs/scala/code/docs/testkit/Specs2DemoAcceptance.scala b/akka-docs/scala/code/docs/testkit/Specs2DemoAcceptance.scala new file mode 100644 index 0000000000..ab8bac9bf3 --- /dev/null +++ b/akka-docs/scala/code/docs/testkit/Specs2DemoAcceptance.scala @@ -0,0 +1,36 @@ +package docs.testkit + +import org.specs2.Specification +import org.specs2.specification.{ Step, Scope } + +import akka.actor.{ Props, ActorSystem, Actor } +import akka.testkit.{ TestKit, ImplicitSender } + +class Specs2DemoAcceptance extends Specification { + def is = + + "This is a specification of basic TestKit interop" ^ + p ^ + "A TestKit should" ^ + "work properly with Specs2 acceptance tests" ! e1 ^ + "correctly convert durations" ! e2 ^ + Step(system.shutdown()) ^ end // do not forget to shutdown! + + val system = ActorSystem() + + // an alternative to mixing in NoTimeConversions + implicit def d2d(d: org.specs2.time.Duration): akka.util.FiniteDuration = + akka.util.Duration(d.inMilliseconds, "millis") + + def e1 = new TestKit(system) with Scope with ImplicitSender { + within(1 second) { + system.actorOf(Props(new Actor { + def receive = { case x ⇒ sender ! x } + })) ! "hallo" + + expectMsgType[String] must be equalTo "hallo" + } + } + + def e2 = ((1 second): akka.util.Duration).toMillis must be equalTo 1000 +} diff --git a/akka-docs/scala/code/docs/testkit/Specs2DemoSpec.scala b/akka-docs/scala/code/docs/testkit/Specs2DemoSpec.scala new file mode 100644 index 0000000000..a620c5139b --- /dev/null +++ b/akka-docs/scala/code/docs/testkit/Specs2DemoSpec.scala @@ -0,0 +1,35 @@ +package docs.testkit + +import org.specs2.mutable.Specification +import org.specs2.specification.Scope +import org.specs2.time.NoTimeConversions + +import akka.actor.{ Props, ActorSystem, Actor } +import akka.testkit.{ TestKit, ImplicitSender } +import akka.util.duration._ + +class Specs2DemoUnitSpec extends Specification with NoTimeConversions { + + val system = ActorSystem() + + /* + * this is needed if different test cases would clash when run concurrently, + * e.g. when creating specifically named top-level actors; leave out otherwise + */ + sequential + + "A TestKit" should { + "work properly with Specs2 unit tests" in + new TestKit(system) with Scope with ImplicitSender { + within(1 second) { + system.actorOf(Props(new Actor { + def receive = { case x ⇒ sender ! x } + })) ! "hallo" + + expectMsgType[String] must be equalTo "hallo" + } + } + } + + step(system.shutdown) // do not forget to shutdown! +} diff --git a/akka-docs/scala/code/docs/testkit/TestKitUsageSpec.scala b/akka-docs/scala/code/docs/testkit/TestKitUsageSpec.scala new file mode 100644 index 0000000000..2ca1dbcef8 --- /dev/null +++ b/akka-docs/scala/code/docs/testkit/TestKitUsageSpec.scala @@ -0,0 +1,156 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package docs.testkit + +//#testkit-usage +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.WordSpec +import org.scalatest.matchers.ShouldMatchers + +import com.typesafe.config.ConfigFactory + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.actor.Props +import akka.testkit.DefaultTimeout +import akka.testkit.ImplicitSender +import akka.testkit.TestKit +import akka.util.duration._ + +/** + * a Test to show some TestKit examples + */ +class TestKitUsageSpec + extends TestKit(ActorSystem("TestKitUsageSpec", + ConfigFactory.parseString(TestKitUsageSpec.config))) + with DefaultTimeout with ImplicitSender + with WordSpec with ShouldMatchers with BeforeAndAfterAll { + import TestKitUsageSpec._ + + val echoRef = system.actorOf(Props(new EchoActor)) + val forwardRef = system.actorOf(Props(new ForwardingActor(testActor))) + val filterRef = system.actorOf(Props(new FilteringActor(testActor))) + val randomHead = Random.nextInt(6) + val randomTail = Random.nextInt(10) + val headList = Seq().padTo(randomHead, "0") + val tailList = Seq().padTo(randomTail, "1") + val seqRef = system.actorOf(Props(new SequencingActor(testActor, headList, tailList))) + + override def afterAll { + system.shutdown() + } + + "An EchoActor" should { + "Respond with the same message it receives" in { + within(500 millis) { + echoRef ! "test" + expectMsg("test") + } + } + } + "A ForwardingActor" should { + "Forward a message it receives" in { + within(500 millis) { + forwardRef ! "test" + expectMsg("test") + } + } + } + "A FilteringActor" should { + "Filter all messages, except expected messagetypes it receives" in { + var messages = Seq[String]() + within(500 millis) { + filterRef ! "test" + expectMsg("test") + filterRef ! 1 + expectNoMsg + filterRef ! "some" + filterRef ! "more" + filterRef ! 1 + filterRef ! "text" + filterRef ! 1 + + receiveWhile(500 millis) { + case msg: String ⇒ messages = msg +: messages + } + } + messages.length should be(3) + messages.reverse should be(Seq("some", "more", "text")) + } + } + "A SequencingActor" should { + "receive an interesting message at some point " in { + within(500 millis) { + ignoreMsg { + case msg: String ⇒ msg != "something" + } + seqRef ! "something" + expectMsg("something") + ignoreMsg { + case msg: String ⇒ msg == "1" + } + expectNoMsg + ignoreNoMsg + } + } + } +} + +object TestKitUsageSpec { + // Define your test specific configuration here + val config = """ + akka { + loglevel = "WARNING" + } + """ + + /** + * An Actor that echoes everything you send to it + */ + class EchoActor extends Actor { + def receive = { + case msg ⇒ sender ! msg + } + } + + /** + * An Actor that forwards every message to a next Actor + */ + class ForwardingActor(next: ActorRef) extends Actor { + def receive = { + case msg ⇒ next ! msg + } + } + + /** + * An Actor that only forwards certain messages to a next Actor + */ + class FilteringActor(next: ActorRef) extends Actor { + def receive = { + case msg: String ⇒ next ! msg + case _ ⇒ None + } + } + + /** + * An actor that sends a sequence of messages with a random head list, an + * interesting value and a random tail list. The idea is that you would + * like to test that the interesting value is received and that you cant + * be bothered with the rest + */ + class SequencingActor(next: ActorRef, head: Seq[String], tail: Seq[String]) + extends Actor { + def receive = { + case msg ⇒ { + head foreach { next ! _ } + next ! msg + tail foreach { next ! _ } + } + } + } +} +//#testkit-usage \ No newline at end of file diff --git a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala b/akka-docs/scala/code/docs/testkit/TestkitDocSpec.scala similarity index 92% rename from akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala rename to akka-docs/scala/code/docs/testkit/TestkitDocSpec.scala index 2b2cb003a9..564b7929ce 100644 --- a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala +++ b/akka-docs/scala/code/docs/testkit/TestkitDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.testkit +package docs.testkit //#imports-test-probe import akka.testkit.TestProbe @@ -14,6 +14,8 @@ import akka.dispatch.Futures import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout import akka.testkit.ImplicitSender +import akka.util.NonFatal + object TestkitDocSpec { case object Say42 case object Unknown @@ -208,7 +210,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { val probe = TestProbe() val future = probe.ref ? "hello" probe.expectMsg(0 millis, "hello") // TestActor runs on CallingThreadDispatcher - probe.sender ! "world" + probe.reply("world") assert(future.isCompleted && future.value == Some(Right("world"))) //#test-probe-reply } @@ -252,4 +254,22 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { //#event-filter } + "demonstrate TestKitBase" in { + //#test-kit-base + import akka.testkit.TestKitBase + + class MyTest extends TestKitBase { + implicit lazy val system = ActorSystem() + + //#put-your-test-code-here + val probe = TestProbe() + probe.send(testActor, "hello") + try expectMsg("hello") catch { case NonFatal(e) ⇒ system.shutdown(); throw e } + //#put-your-test-code-here + + system.shutdown() + } + //#test-kit-base + } + } diff --git a/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala b/akka-docs/scala/code/docs/transactor/TransactorDocSpec.scala similarity index 99% rename from akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala rename to akka-docs/scala/code/docs/transactor/TransactorDocSpec.scala index fa76f54744..c1556b837d 100644 --- a/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala +++ b/akka-docs/scala/code/docs/transactor/TransactorDocSpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.transactor +package docs.transactor import akka.actor._ import akka.transactor._ diff --git a/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala b/akka-docs/scala/code/docs/zeromq/ZeromqDocSpec.scala similarity index 99% rename from akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala rename to akka-docs/scala/code/docs/zeromq/ZeromqDocSpec.scala index dba4989d87..812e0edaaa 100644 --- a/akka-docs/scala/code/akka/docs/zeromq/ZeromqDocSpec.scala +++ b/akka-docs/scala/code/docs/zeromq/ZeromqDocSpec.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.docs.zeromq +package docs.zeromq import akka.actor.Actor import akka.actor.Props diff --git a/akka-docs/scala/dispatchers.rst b/akka-docs/scala/dispatchers.rst index 7d6a1f6334..4253d3a1e4 100644 --- a/akka-docs/scala/dispatchers.rst +++ b/akka-docs/scala/dispatchers.rst @@ -18,7 +18,7 @@ Setting the dispatcher for an Actor So in case you want to give your ``Actor`` a different dispatcher than the default, you need to do two things, of which the first is: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#defining-dispatcher +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#defining-dispatcher .. note:: The "dispatcherId" you specify in withDispatcher is in fact a path into your configuration. @@ -27,11 +27,11 @@ So in case you want to give your ``Actor`` a different dispatcher than the defau And then you just need to configure that dispatcher in your configuration: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config And here's another example that uses the "thread-pool-executor": -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#my-thread-pool-dispatcher-config For more options, see the default-dispatcher section of the :ref:`configuration`. @@ -71,7 +71,9 @@ There are 4 different types of message dispatchers: * BalancingDispatcher - - This is an executor based event driven dispatcher that will try to redistribute work from busy actors to idle actors. + - This is an executor based event driven dispatcher that will try to redistribute work from busy actors to idle actors. + + - All the actors share a single Mailbox that they get their messages from. - It is assumed that all actors using the same instance of this dispatcher can process all messages that have been sent to one of the actors; i.e. the actors belong to a pool of actors, and to the client there is no guarantee about which actor instance actually processes a given message. @@ -86,9 +88,11 @@ There are 4 different types of message dispatchers: "thread-pool-executor" or the FQCN of an ``akka.dispatcher.ExecutorServiceConfigurator`` + - Note that you can **not** use a ``BalancingDispatcher`` as a **Router Dispatcher**. (You can however use it for the **Routees**) + * CallingThreadDispatcher - - This dispatcher runs invocations on the current thread only. This dispatcher does not create any new threads, + - This dispatcher runs invocations on the current thread only. This dispatcher does not create any new threads, but it can be used from different threads concurrently for the same actor. See :ref:`TestCallingThreadDispatcherRef` for details and restrictions. @@ -106,14 +110,14 @@ More dispatcher configuration examples Configuring a ``PinnedDispatcher``: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-pinned-dispatcher-config +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#my-pinned-dispatcher-config And then using it: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#defining-pinned-dispatcher +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#defining-pinned-dispatcher -Note that ``thread-pool-executor`` configuration as per the above ``my-thread-pool-dispatcher`` exmaple is -NOT applicable. This is because every actor will have its own thread pool when using ``PinnedDispatcher``, +Note that ``thread-pool-executor`` configuration as per the above ``my-thread-pool-dispatcher`` example is +NOT applicable. This is because every actor will have its own thread pool when using ``PinnedDispatcher``, and that pool will have only one thread. Mailboxes @@ -166,22 +170,22 @@ Mailbox configuration examples How to create a PriorityMailbox: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#prio-mailbox +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#prio-mailbox And then add it to the configuration: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher-config +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher-config And then an example on how you would use it: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher Creating your own Mailbox type ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ An example is worth a thousand quacks: -.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#mailbox-implementation-example +.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#mailbox-implementation-example And then you just specify the FQCN of your MailboxType as the value of the "mailbox-type" in the dispatcher configuration. @@ -194,3 +198,25 @@ And then you just specify the FQCN of your MailboxType as the value of the "mail the configuration which describes the dispatcher using this mailbox type; the mailbox type will be instantiated once for each dispatcher using it. +Special Semantics of ``system.actorOf`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +In order to make ``system.actorOf`` both synchronous and non-blocking while +keeping the return type :class:`ActorRef` (and the semantics that the returned +ref is fully functional), special handling takes place for this case. Behind +the scenes, a hollow kind of actor reference is constructed, which is sent to +the system’s guardian actor who actually creates the actor and its context and +puts those inside the reference. Until that has happened, messages sent to the +:class:`ActorRef` will be queued locally, and only upon swapping the real +filling in will they be transferred into the real mailbox. Thus, + +.. code-block:: scala + + val props: Props = ... + // this actor uses MyCustomMailbox, which is assumed to be a singleton + system.actorOf(props.withDispatcher("myCustomMailbox")) ! "bang" + assert(MyCustomMailbox.instance.getLastEnqueuedMessage == "bang") + +will probably fail; you will have to allow for some time to pass and retry the +check à la :meth:`TestKit.awaitCond`. + diff --git a/akka-docs/scala/event-bus.rst b/akka-docs/scala/event-bus.rst index aec59083c6..2487e806fe 100644 --- a/akka-docs/scala/event-bus.rst +++ b/akka-docs/scala/event-bus.rst @@ -158,7 +158,7 @@ Classification`_ which enables registering to related sets of channels (as is used for :class:`RemoteLifeCycleMessage`). The following example demonstrates how a simple subscription works: -.. includecode:: code/akka/docs/event/LoggingDocSpec.scala#deadletters +.. includecode:: code/docs/event/LoggingDocSpec.scala#deadletters Default Handlers ---------------- diff --git a/akka-docs/scala/extending-akka.rst b/akka-docs/scala/extending-akka.rst index 0f0934a799..9c890de252 100644 --- a/akka-docs/scala/extending-akka.rst +++ b/akka-docs/scala/extending-akka.rst @@ -24,27 +24,27 @@ So let's create a sample extension that just lets us count the number of times s First, we define what our ``Extension`` should do: -.. includecode:: code/akka/docs/extension/ExtensionDocSpec.scala +.. includecode:: code/docs/extension/ExtensionDocSpec.scala :include: extension Then we need to create an ``ExtensionId`` for our extension so we can grab ahold of it. -.. includecode:: code/akka/docs/extension/ExtensionDocSpec.scala +.. includecode:: code/docs/extension/ExtensionDocSpec.scala :include: extensionid Wicked! Now all we need to do is to actually use it: -.. includecode:: code/akka/docs/extension/ExtensionDocSpec.scala +.. includecode:: code/docs/extension/ExtensionDocSpec.scala :include: extension-usage Or from inside of an Akka Actor: -.. includecode:: code/akka/docs/extension/ExtensionDocSpec.scala +.. includecode:: code/docs/extension/ExtensionDocSpec.scala :include: extension-usage-actor You can also hide extension behind traits: -.. includecode:: code/akka/docs/extension/ExtensionDocSpec.scala +.. includecode:: code/docs/extension/ExtensionDocSpec.scala :include: extension-usage-actor-trait That's all there is to it! @@ -55,7 +55,7 @@ Loading from Configuration To be able to load extensions from your Akka configuration you must add FQCNs of implementations of either ``ExtensionId`` or ``ExtensionIdProvider`` in the ``akka.extensions`` section of the config you provide to your ``ActorSystem``. -.. includecode:: code/akka/docs/extension/ExtensionDocSpec.scala +.. includecode:: code/docs/extension/ExtensionDocSpec.scala :include: config Note that in this case ``CountExtension`` is an object and therefore the class name ends with ``$``. @@ -75,17 +75,17 @@ The :ref:`configuration` can be used for application specific settings. A good p Sample configuration: -.. includecode:: code/akka/docs/extension/SettingsExtensionDocSpec.scala +.. includecode:: code/docs/extension/SettingsExtensionDocSpec.scala :include: config The ``Extension``: -.. includecode:: code/akka/docs/extension/SettingsExtensionDocSpec.scala +.. includecode:: code/docs/extension/SettingsExtensionDocSpec.scala :include: imports,extension,extensionid Use it: -.. includecode:: code/akka/docs/extension/SettingsExtensionDocSpec.scala +.. includecode:: code/docs/extension/SettingsExtensionDocSpec.scala :include: extension-usage-actor diff --git a/akka-docs/scala/fault-tolerance-sample.rst b/akka-docs/scala/fault-tolerance-sample.rst index ccda303e45..12621e968b 100644 --- a/akka-docs/scala/fault-tolerance-sample.rst +++ b/akka-docs/scala/fault-tolerance-sample.rst @@ -45,11 +45,11 @@ Step Description 9, 10, 11 and tells the ``Counter`` that there is no ``Storage``. 12 The ``CounterService`` schedules a ``Reconnect`` message to itself. 13, 14 When it receives the ``Reconnect`` message it creates a new ``Storage`` ... -15, 16 and tells the the ``Counter`` to use the new ``Storage`` +15, 16 and tells the ``Counter`` to use the new ``Storage`` =========== ================================================================================== Full Source Code of the Fault Tolerance Sample (Scala) ------------------------------------------------------ -.. includecode:: code/akka/docs/actor/FaultHandlingDocSample.scala#all +.. includecode:: code/docs/actor/FaultHandlingDocSample.scala#all diff --git a/akka-docs/scala/fault-tolerance.rst b/akka-docs/scala/fault-tolerance.rst index 8448bd2cce..c1d6158954 100644 --- a/akka-docs/scala/fault-tolerance.rst +++ b/akka-docs/scala/fault-tolerance.rst @@ -24,7 +24,7 @@ sample as it is easy to follow the log output to understand what is happening in fault-tolerance-sample -.. includecode:: code/akka/docs/actor/FaultHandlingDocSample.scala#all +.. includecode:: code/docs/actor/FaultHandlingDocSample.scala#all :exclude: imports,messages,dummydb Creating a Supervisor Strategy @@ -35,7 +35,7 @@ in more depth. For the sake of demonstration let us consider the following strategy: -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: strategy I have chosen a few well-known exception types in order to demonstrate the @@ -75,50 +75,50 @@ Test Application The following section shows the effects of the different directives in practice, wherefor a test setup is needed. First off, we need a suitable supervisor: -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: supervisor This supervisor will be used to create a child, with which we can experiment: -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: child The test is easier by using the utilities described in :ref:`akka-testkit`, where ``AkkaSpec`` is a convenient mixture of ``TestKit with WordSpec with MustMatchers`` -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: testkit Let us create actors: -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: create The first test shall demonstrate the ``Resume`` directive, so we try it out by setting some non-initial state in the actor and have it fail: -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: resume As you can see the value 42 survives the fault handling directive. Now, if we change the failure to a more serious ``NullPointerException``, that will no longer be the case: -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: restart And finally in case of the fatal ``IllegalArgumentException`` the child will be terminated by the supervisor: -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: stop Up to now the supervisor was completely unaffected by the child’s failure, because the directives set did handle it. In case of an ``Exception``, this is not true anymore and the supervisor escalates the failure. -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: escalate-kill The supervisor itself is supervised by the top-level actor provided by the @@ -131,12 +131,12 @@ child not to survive this failure. In case this is not desired (which depends on the use case), we need to use a different supervisor which overrides this behavior. -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: supervisor2 With this parent, the child survives the escalated restart, as demonstrated in the last test: -.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala +.. includecode:: code/docs/actor/FaultHandlingDocSpec.scala :include: escalate-restart diff --git a/akka-docs/scala/fsm.rst b/akka-docs/scala/fsm.rst index 807cd7567c..e47fdaa055 100644 --- a/akka-docs/scala/fsm.rst +++ b/akka-docs/scala/fsm.rst @@ -30,17 +30,17 @@ send them on after the burst ended or a flush request is received. First, consider all of the below to use these import statements: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#simple-imports +.. includecode:: code/docs/actor/FSMDocSpec.scala#simple-imports The contract of our “Buncher” actor is that is accepts or produces the following messages: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#simple-events +.. includecode:: code/docs/actor/FSMDocSpec.scala#simple-events ``SetTarget`` is needed for starting it up, setting the destination for the ``Batches`` to be passed on; ``Queue`` will add to the internal queue while ``Flush`` will mark the end of a burst. -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#simple-state +.. includecode:: code/docs/actor/FSMDocSpec.scala#simple-state The actor can be in two states: no message queued (aka ``Idle``) or some message queued (aka ``Active``). It will stay in the active state as long as @@ -50,7 +50,7 @@ the actual queue of messages. Now let’s take a look at the skeleton for our FSM actor: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: simple-fsm :exclude: transition-elided,unhandled-elided @@ -79,7 +79,7 @@ shall work identically in both states, we make use of the fact that any event which is not handled by the ``when()`` block is passed to the ``whenUnhandled()`` block: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#unhandled-elided +.. includecode:: code/docs/actor/FSMDocSpec.scala#unhandled-elided The first case handled here is adding ``Queue()`` requests to the internal queue and going to the ``Active`` state (this does the obvious thing of staying @@ -93,7 +93,7 @@ target, for which we use the ``onTransition`` mechanism: you can declare multiple such blocks and all of them will be tried for matching behavior in case a state transition occurs (i.e. only when the state actually changes). -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#transition-elided +.. includecode:: code/docs/actor/FSMDocSpec.scala#transition-elided The transition callback is a partial function which takes as input a pair of states—the current and the next state. The FSM trait includes a convenience @@ -106,7 +106,7 @@ To verify that this buncher actually works, it is quite easy to write a test using the :ref:`akka-testkit`, which is conveniently bundled with ScalaTest traits into ``AkkaSpec``: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: test-code :exclude: fsm-code-elided @@ -120,7 +120,7 @@ The :class:`FSM` trait may only be mixed into an :class:`Actor`. Instead of extending :class:`Actor`, the self type approach was chosen in order to make it obvious that an actor is actually created: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: simple-fsm :exclude: fsm-body @@ -165,7 +165,7 @@ The :meth:`stateFunction` argument is a :class:`PartialFunction[Event, State]`, which is conveniently given using the partial function literal syntax as demonstrated below: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: when-syntax The :class:`Event(msg: Any, data: D)` case class is parameterized with the data @@ -189,7 +189,7 @@ If a state doesn't handle a received event a warning is logged. If you want to do something else in this case you can specify that with :func:`whenUnhandled(stateFunction)`: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: unhandled-syntax **IMPORTANT**: This handler is not stacked, meaning that each invocation of @@ -230,7 +230,7 @@ of the modifiers described in the following: All modifier can be chained to achieve a nice and concise description: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: modifier-syntax The parentheses are not actually needed in all cases, but they visually @@ -267,7 +267,7 @@ The handler is a partial function which takes a pair of states as input; no resulting state is needed as it is not possible to modify the transition in progress. -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: transition-syntax The convenience extractor :obj:`->` enables decomposition of the pair of states @@ -280,7 +280,7 @@ It is also possible to pass a function object accepting two states to :func:`onTransition`, in case your transition handling logic is implemented as a method: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: alt-transition-syntax The handlers registered with this method are stacked, so you can intersperse @@ -319,14 +319,14 @@ transformed using Scala’s full supplement of functional programming tools. In order to retain type inference, there is a helper function which may be used in case some common handling logic shall be applied to different clauses: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: transform-syntax It goes without saying that the arguments to this method may also be stored, to be used several times, e.g. when applying the same transformation to several ``when()`` blocks: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: alt-transform-syntax Timers @@ -371,14 +371,14 @@ state data which is available during termination handling. the same way as a state transition (but note that the ``return`` statement may not be used within a :meth:`when` block). -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: stop-syntax You can use :func:`onTermination(handler)` to specify custom code that is executed when the FSM is stopped. The handler is a partial function which takes a :class:`StopEvent(reason, stateName, stateData)` as argument: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: termination-syntax As for the :func:`whenUnhandled` case, this handler is not stacked, so each @@ -412,7 +412,7 @@ Event Tracing The setting ``akka.actor.debug.fsm`` in :ref:`configuration` enables logging of an event trace by :class:`LoggingFSM` instances: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: logging-fsm :exclude: body-elided @@ -433,7 +433,7 @@ The :class:`LoggingFSM` trait adds one more feature to the FSM: a rolling event log which may be used during debugging (for tracing how the FSM entered a certain failure state) or for other creative uses: -.. includecode:: code/akka/docs/actor/FSMDocSpec.scala +.. includecode:: code/docs/actor/FSMDocSpec.scala :include: logging-fsm The :meth:`logDepth` defaults to zero, which turns off the event log. diff --git a/akka-docs/scala/futures.rst b/akka-docs/scala/futures.rst index d84b742c6f..26936b0493 100644 --- a/akka-docs/scala/futures.rst +++ b/akka-docs/scala/futures.rst @@ -19,7 +19,7 @@ which is very similar to a ``java.util.concurrent.Executor``. if you have an ``A it will use its default dispatcher as the ``ExecutionContext``, or you can use the factory methods provided by the ``ExecutionContext`` companion object to wrap ``Executors`` and ``ExecutorServices``, or even create your own. -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: diy-execution-context Use With Actors @@ -30,7 +30,7 @@ which only works if the original sender was an ``Actor``) and the second is thro Using an ``Actor``\'s ``?`` method to send a message will return a Future. To wait for and retrieve the actual result the simplest method is: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: ask-blocking This will cause the current thread to block and wait for the ``Actor`` to 'complete' the ``Future`` with it's reply. @@ -40,7 +40,7 @@ Alternatives to blocking are discussed further within this documentation. Also n an ``Actor`` is a ``Future[Any]`` since an ``Actor`` is dynamic. That is why the ``asInstanceOf`` is used in the above sample. When using non-blocking it is better to use the ``mapTo`` method to safely try to cast a ``Future`` to an expected type: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: map-to The ``mapTo`` method will return a new ``Future`` that contains the result if the cast was successful, @@ -53,7 +53,7 @@ A common use case within Akka is to have some computation performed concurrently If you find yourself creating a pool of ``Actor``\s for the sole reason of performing a calculation in parallel, there is an easier (and faster) way: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: future-eval In the above code the block passed to ``Future`` will be executed by the default ``Dispatcher``, @@ -63,12 +63,12 @@ and we also avoid the overhead of managing an ``Actor``. You can also create already completed Futures using the ``Promise`` companion, which can be either successes: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: successful Or failures: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: failed Functional Futures @@ -84,7 +84,7 @@ The first method for working with ``Future`` functionally is ``map``. This metho which performs some operation on the result of the ``Future``, and returning a new result. The return value of the ``map`` method is another ``Future`` that will contain the new result: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: map In this example we are joining two strings together within a ``Future``. Instead of waiting for this to complete, @@ -97,12 +97,12 @@ string "HelloWorld" and is unaffected by the ``map``. The ``map`` method is fine if we are modifying a single ``Future``, but if 2 or more ``Future``\s are involved ``map`` will not allow you to combine them together: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: wrong-nested-map ``f3`` is a ``Future[Future[Int]]`` instead of the desired ``Future[Int]``. Instead, the ``flatMap`` method should be used: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: flat-map Composing futures using nested combinators it can sometimes become quite complicated and hard read, in these cases using Scala's @@ -110,7 +110,7 @@ Composing futures using nested combinators it can sometimes become quite complic If you need to do conditional propagation, you can use ``filter``: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: filter For Comprehensions @@ -118,7 +118,7 @@ For Comprehensions Since ``Future`` has a ``map``, ``filter`` and ``flatMap`` method it can be easily used in a 'for comprehension': -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: for-comprehension Something to keep in mind when doing this is even though it looks like parts of the above example can run in parallel, @@ -134,14 +134,14 @@ A common use case for this is combining the replies of several ``Actor``\s into without resorting to calling ``Await.result`` or ``Await.ready`` to block for each result. First an example of using ``Await.result``: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: composing-wrong Here we wait for the results from the first 2 ``Actor``\s before sending that result to the third ``Actor``. We called ``Await.result`` 3 times, which caused our little program to block 3 times before getting our final result. Now compare that to this example: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: composing Here we have 2 actors processing a single message each. Once the 2 results are available @@ -153,7 +153,7 @@ The ``sequence`` and ``traverse`` helper methods can make it easier to handle mo Both of these methods are ways of turning, for a subclass ``T`` of ``Traversable``, ``T[Future[A]]`` into a ``Future[T[A]]``. For example: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: sequence-ask To better explain what happened in the example, ``Future.sequence`` is taking the ``List[Future[Int]]`` @@ -163,12 +163,12 @@ and we find the sum of the ``List``. The ``traverse`` method is similar to ``sequence``, but it takes a ``T[A]`` and a function ``A => Future[B]`` to return a ``Future[T[B]]``, where ``T`` is again a subclass of Traversable. For example, to use ``traverse`` to sum the first 100 odd numbers: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: traverse This is the same result as this example: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: sequence But it may be faster to use ``traverse`` as it doesn't have to create an intermediate ``List[Future[Int]]``. @@ -178,7 +178,7 @@ from the type of the start-value and the type of the futures and returns somethi and then applies the function to all elements in the sequence of futures, asynchronously, the execution will start when the last of the Futures is completed. -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: fold That's all it takes! @@ -188,7 +188,7 @@ If the sequence passed to ``fold`` is empty, it will return the start-value, in In some cases you don't have a start-value and you're able to use the value of the first completing Future in the sequence as the start-value, you can use ``reduce``, it works like this: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: reduce Same as with ``fold``, the execution will be done asynchronously when the last of the Future is completed, @@ -200,13 +200,13 @@ Callbacks Sometimes you just want to listen to a ``Future`` being completed, and react to that not by creating a new Future, but by side-effecting. For this Akka supports ``onComplete``, ``onSuccess`` and ``onFailure``, of which the latter two are specializations of the first. -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: onSuccess -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: onFailure -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: onComplete Define Ordering @@ -218,7 +218,7 @@ But there's a solution and it's name is ``andThen``. It creates a new ``Future`` the specified callback, a ``Future`` that will have the same result as the ``Future`` it's called on, which allows for ordering like in the following sample: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: and-then Auxiliary Methods @@ -227,13 +227,13 @@ Auxiliary Methods ``Future`` ``fallbackTo`` combines 2 Futures into a new ``Future``, and will hold the successful value of the second ``Future`` if the first ``Future`` fails. -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: fallback-to You can also combine two Futures into a new ``Future`` that will hold a tuple of the two Futures successful results, using the ``zip`` operation. -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: zip Exceptions @@ -247,7 +247,7 @@ If a ``Future`` does contain an ``Exception``, calling ``Await.result`` will cau It is also possible to handle an ``Exception`` by returning a different result. This is done with the ``recover`` method. For example: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: recover In this example, if the actor replied with a ``akka.actor.Status.Failure`` containing the ``ArithmeticException``, @@ -258,6 +258,6 @@ it will behave as if we hadn't used the ``recover`` method. You can also use the ``recoverWith`` method, which has the same relationship to ``recover`` as ``flatMap`` has to ``map``, and is use like this: -.. includecode:: code/akka/docs/future/FutureDocSpec.scala +.. includecode:: code/docs/future/FutureDocSpec.scala :include: try-recover diff --git a/akka-docs/scala/howto.rst b/akka-docs/scala/howto.rst new file mode 100644 index 0000000000..9436480327 --- /dev/null +++ b/akka-docs/scala/howto.rst @@ -0,0 +1,33 @@ + +.. _howto-scala: + +###################### +HowTo: Common Patterns +###################### + +This section lists common actor patterns which have been found to be useful, +elegant or instructive. Anything is welcome, example topics being message +routing strategies, supervision patterns, restart handling, etc. As a special +bonus, additions to this section are marked with the contributor’s name, and it +would be nice if every Akka user who finds a recurring pattern in his or her +code could share it for the profit of all. Where applicable it might also make +sense to add to the ``akka.pattern`` package for creating an `OTP-like library +`_. + +Template Pattern +================ + +*Contributed by: N. N.* + +This is an especially nice pattern, since it does even come with some empty example code: + +.. includecode:: code/docs/pattern/ScalaTemplate.scala + :include: all-of-it + :exclude: uninteresting-stuff + +.. note:: + + Spread the word: this is the easiest way to get famous! + +Please keep this pattern at the end of this file. + diff --git a/akka-docs/scala/index.rst b/akka-docs/scala/index.rst index fc1b619e26..ddceb9fcf8 100644 --- a/akka-docs/scala/index.rst +++ b/akka-docs/scala/index.rst @@ -28,3 +28,4 @@ Scala API zeromq microkernel camel + howto diff --git a/akka-docs/scala/io.rst b/akka-docs/scala/io.rst index 85d2b6bef5..9063e010f5 100644 --- a/akka-docs/scala/io.rst +++ b/akka-docs/scala/io.rst @@ -155,29 +155,29 @@ Http Server This example will create a simple high performance HTTP server. We begin with our imports: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: imports Some commonly used constants: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: constants And case classes to hold the resulting request: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: request-class Now for our first ``Iteratee``. There are 3 main sections of a HTTP request: the request line, the headers, and an optional body. The main request ``Iteratee`` handles each section separately: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: read-request In the above code ``readRequest`` takes the results of 3 different ``Iteratees`` (``readRequestLine``, ``readHeaders``, ``readBody``) and combines them into a single ``Request`` object. ``readRequestLine`` actually returns a tuple, so we extract it's individual components. ``readBody`` depends on values contained within the header section, so we must pass those to the method. The request line has 3 parts to it: the HTTP method, the requested URI, and the HTTP version. The parts are separated by a single space, and the entire request line ends with a ``CRLF``. -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: read-request-line Reading the request method is simple as it is a single string ending in a space. The simple ``Iteratee`` that performs this is ``IO.takeUntil(delimiter: ByteString): Iteratee[ByteString]``. It keeps consuming input until the specified delimiter is found. Reading the HTTP version is also a simple string that ends with a ``CRLF``. @@ -186,14 +186,14 @@ The ``ascii`` method is a helper that takes a ``ByteString`` and parses it as a Reading the request URI is a bit more complicated because we want to parse the individual components of the URI instead of just returning a simple string: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: read-request-uri For this example we are only interested in handling absolute paths. To detect if we the URI is an absolute path we use ``IO.peek(length: Int): Iteratee[ByteString]``, which returns a ``ByteString`` of the request length but doesn't actually consume the input. We peek at the next bit of input and see if it matches our ``PATH`` constant (defined above as ``ByteString("/")``). If it doesn't match we throw an error, but for a more robust solution we would want to handle other valid URIs. Next we handle the path itself: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: read-path The ``step`` method is a recursive method that takes a ``List`` of the accumulated path segments. It first checks if the remaining input starts with the ``PATH`` constant, and if it does, it drops that input, and returns the ``readUriPart`` ``Iteratee`` which has it's result added to the path segment accumulator and the ``step`` method is run again. @@ -202,39 +202,39 @@ If after reading in a path segment the next input does not start with a path, we Following the path we read in the query (if it exists): -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: read-query It is much simpler then reading the path since we aren't doing any parsing of the query since there is no standard format of the query string. Both the path and query used the ``readUriPart`` ``Iteratee``, which is next: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: read-uri-part Here we have several ``Set``\s that contain valid characters pulled from the URI spec. The ``readUriPart`` method takes a ``Set`` of valid characters (already mapped to ``Byte``\s) and will continue to match characters until it reaches on that is not part of the ``Set``. If it is a percent encoded character then that is handled as a valid character and processing continues, or else we are done collecting this part of the URI. Headers are next: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: read-headers And if applicable, we read in the message body: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: read-body Finally we get to the actual ``Actor``: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: actor And it's companion object: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: actor-companion A ``main`` method to start everything up: -.. includecode:: code/akka/docs/io/HTTPServer.scala +.. includecode:: code/docs/io/HTTPServer.scala :include: main diff --git a/akka-docs/scala/logging.rst b/akka-docs/scala/logging.rst index 9b1fe42d3e..8f765b4f7e 100644 --- a/akka-docs/scala/logging.rst +++ b/akka-docs/scala/logging.rst @@ -11,7 +11,7 @@ How to Log Create a ``LoggingAdapter`` and use the ``error``, ``warning``, ``info``, or ``debug`` methods, as illustrated in this example: -.. includecode:: code/akka/docs/event/LoggingDocSpec.scala +.. includecode:: code/docs/event/LoggingDocSpec.scala :include: my-actor For convenience you can mixin the ``log`` member into actors, instead of defining it as above. @@ -37,7 +37,7 @@ placeholders results in a warning being appended to the log statement (i.e. on the same line with the same severity). You may pass a Java array as the only substitution argument to have its elements be treated individually: -.. includecode:: code/akka/docs/event/LoggingDocSpec.scala#array +.. includecode:: code/docs/event/LoggingDocSpec.scala#array The Java :class:`Class` of the log source is also included in the generated :class:`LogEvent`. In case of a simple string this is replaced with a “marker” @@ -176,7 +176,7 @@ using implicit parameters and thus fully customizable: simply create your own instance of :class:`LogSource[T]` and have it in scope when creating the logger. -.. includecode:: code/akka/docs/event/LoggingDocSpec.scala#my-source +.. includecode:: code/docs/event/LoggingDocSpec.scala#my-source This example creates a log source which mimics traditional usage of Java loggers, which are based upon the originating object’s class name as log @@ -217,7 +217,7 @@ event handler available in the 'akka-slf4j' module. Example of creating a listener: -.. includecode:: code/akka/docs/event/LoggingDocSpec.scala +.. includecode:: code/docs/event/LoggingDocSpec.scala :include: my-event-listener .. _slf4j-scala: @@ -230,7 +230,7 @@ It has one single dependency; the slf4j-api jar. In runtime you also need a SLF4 .. code-block:: scala - lazy val logback = "ch.qos.logback" % "logback-classic" % "1.0.0" % "runtime" + lazy val logback = "ch.qos.logback" % "logback-classic" % "1.0.4" % "runtime" You need to enable the Slf4jEventHandler in the 'event-handlers' element in @@ -253,7 +253,7 @@ the first case and ``LoggerFactory.getLogger(s: String)`` in the second). .. note:: - Beware that the the actor system’s name is appended to a :class:`String` log + Beware that the actor system’s name is appended to a :class:`String` log source if the LoggingAdapter was created giving an :class:`ActorSystem` to the factory. If this is not intended, give a :class:`LoggingBus` instead as shown below: diff --git a/akka-docs/scala/microkernel.rst b/akka-docs/scala/microkernel.rst index 8fb1aec2c2..c2bc95cb8b 100644 --- a/akka-docs/scala/microkernel.rst +++ b/akka-docs/scala/microkernel.rst @@ -1,9 +1,13 @@ -.. _microkernel: +.. _microkernel-scala: Microkernel (Scala) =================== +The purpose of the Akka Microkernel is to offer a bundling mechanism so that you can distribute +an Akka application as a single payload, without the need to run in a Java Application Server or manually +having to create a launcher script. + The Akka Microkernel is included in the Akka download found at `downloads`_. .. _downloads: http://akka.io/downloads diff --git a/akka-docs/scala/remoting.rst b/akka-docs/scala/remoting.rst index 88096d90d1..ab49765fad 100644 --- a/akka-docs/scala/remoting.rst +++ b/akka-docs/scala/remoting.rst @@ -105,6 +105,14 @@ Once you have configured the properties above you would do the following in code ``SampleActor`` has to be available to the runtimes using it, i.e. the classloader of the actor systems has to have a JAR containing the class. +.. note:: + + In order to ensure serializability of ``Props`` when passing constructor + arguments to the actor being created, do not make the factory an inner class: + this will inherently capture a reference to its enclosing object, which in + most cases is not serializable. It is best to create a factory method in the + companion object of the actor’s class. + Programmatic Remote Deployment ------------------------------ @@ -116,15 +124,15 @@ precedence. With these imports: -.. includecode:: code/akka/docs/remoting/RemoteDeploymentDocSpec.scala#import +.. includecode:: code/docs/remoting/RemoteDeploymentDocSpec.scala#import and a remote address like this: -.. includecode:: code/akka/docs/remoting/RemoteDeploymentDocSpec.scala#make-address +.. includecode:: code/docs/remoting/RemoteDeploymentDocSpec.scala#make-address you can advise the system to create a child on that remote node like so: -.. includecode:: code/akka/docs/remoting/RemoteDeploymentDocSpec.scala#deploy +.. includecode:: code/docs/remoting/RemoteDeploymentDocSpec.scala#deploy Serialization ^^^^^^^^^^^^^ @@ -280,10 +288,6 @@ which holds the transport used (RemoteTransport) and the outbound address that i To intercept when an outbound client is shut down you listen to ``RemoteClientShutdown`` which holds the transport used (RemoteTransport) and the outbound address that it was connected to (Address). -To intercept when an outbound message cannot be sent, you listen to ``RemoteClientWriteFailed`` which holds -the payload that was not written (AnyRef), the cause of the failed send (Throwable), -the transport used (RemoteTransport) and the outbound address that was the destination (Address). - For general outbound-related errors, that do not classify as any of the others, you can listen to ``RemoteClientError``, which holds the cause (Throwable), the transport used (RemoteTransport) and the outbound address (Address). @@ -301,3 +305,64 @@ which holds the transport used (RemoteTransport) and optionally the address that To intercept when an inbound remote client has been closed you listen to ``RemoteServerClientClosed`` which holds the transport used (RemoteTransport) and optionally the address of the remote client that was closed (Option[Address]). + +Remote Security +^^^^^^^^^^^^^^^ + +Akka provides a couple of ways to enhance security between remote nodes (client/server): + +* Untrusted Mode +* Security Cookie Handshake + +Untrusted Mode +-------------- + +You can enable untrusted mode for preventing system messages to be send by clients, e.g. messages like. +This will prevent the client to send these messages to the server: + +* ``Create`` +* ``Recreate`` +* ``Suspend`` +* ``Resume`` +* ``Terminate`` +* ``Supervise`` +* ``ChildTerminated`` +* ``Link`` +* ``Unlink`` + +Here is how to turn it on in the config:: + + akka { + actor { + remote { + untrusted-mode = on + } + } + } + +Secure Cookie Handshake +----------------------- + +Akka remoting also allows you to specify a secure cookie that will be exchanged and ensured to be identical +in the connection handshake between the client and the server. If they are not identical then the client +will be refused to connect to the server. + +The secure cookie can be any kind of string. But the recommended approach is to generate a cryptographically +secure cookie using this script ``$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh`` or from code +using the ``akka.util.Crypt.generateSecureCookie()`` utility method. + +You have to ensure that both the connecting client and the server have the same secure cookie as well +as the ``require-cookie`` option turned on. + +Here is an example config:: + + akka { + actor { + remote { + netty { + secure-cookie = "090A030E0F0A05010900000A0C0E0C0B03050D05" + require-cookie = on + } + } + } + } diff --git a/akka-docs/scala/routing.rst b/akka-docs/scala/routing.rst index 737c9e31e7..5a37b3471a 100644 --- a/akka-docs/scala/routing.rst +++ b/akka-docs/scala/routing.rst @@ -21,17 +21,22 @@ Routers In Action This is an example of how to create a router that is defined in configuration: -.. includecode:: code/akka/docs/routing/RouterViaConfigExample.scala#config +.. includecode:: code/docs/routing/RouterViaConfigExample.scala#config -.. includecode:: code/akka/docs/routing/RouterViaConfigExample.scala#configurableRouting +.. includecode:: code/docs/routing/RouterViaConfigExample.scala#configurableRouting This is an example of how to programmatically create a router and set the number of routees it should create: -.. includecode:: code/akka/docs/routing/RouterViaProgramExample.scala#programmaticRoutingNrOfInstances +.. includecode:: code/docs/routing/RouterViaProgramExample.scala#programmaticRoutingNrOfInstances You can also give the router already created routees as in: -.. includecode:: code/akka/docs/routing/RouterViaProgramExample.scala#programmaticRoutingRoutees +.. includecode:: code/docs/routing/RouterViaProgramExample.scala#programmaticRoutingRoutees + +It should be noted that no actor factory or class needs to be provided in this +case, as the ``Router`` will not create any children on its own (which is not +true anymore when using a resizer). The routees can also be specified by giving +their path strings. When you create a router programmatically you define the number of routees *or* you pass already created routees to it. If you send both parameters to the router *only* the latter will be used, i.e. ``nrOfInstances`` is disregarded. @@ -48,7 +53,7 @@ Once you have the router actor it is just to send messages to it as you would to router ! MyMsg -The router will apply its behavior to the message it receives and forward it to the routees. +The router will forward the message to its routees according to its routing policy. Remotely Deploying Routees ************************** @@ -60,7 +65,7 @@ configuration in a :class:`RemoteRouterConfig`, attaching the remote addresses o the nodes to deploy to. Naturally, this requires your to include the ``akka-remote`` module on your classpath: -.. includecode:: code/akka/docs/routing/RouterViaProgramExample.scala#remoteRoutees +.. includecode:: code/docs/routing/RouterViaProgramExample.scala#remoteRoutees How Routing is Designed within Akka ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -90,9 +95,9 @@ deterministic fashion. Since each actor knows its own external representation as well as that of its parent, the routees decide where replies should be sent when reacting to a message: -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#reply-with-sender +.. includecode:: code/docs/actor/ActorDocSpec.scala#reply-with-sender -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#reply-without-sender +.. includecode:: code/docs/actor/ActorDocSpec.scala#reply-without-sender It is apparent now why routing needs to be enabled in code rather than being possible to “bolt on” later: whether or not an actor is routed means a change @@ -136,11 +141,11 @@ Router usage In this section we will describe how to use the different router types. First we need to create some actors that will be used in the examples: -.. includecode:: code/akka/docs/routing/RouterTypeExample.scala#printlnActor +.. includecode:: code/docs/routing/RouterTypeExample.scala#printlnActor and -.. includecode:: code/akka/docs/routing/RouterTypeExample.scala#fibonacciActor +.. includecode:: code/docs/routing/RouterTypeExample.scala#fibonacciActor RoundRobinRouter @@ -148,7 +153,7 @@ RoundRobinRouter Routes in a `round-robin `_ fashion to its routees. Code example: -.. includecode:: code/akka/docs/routing/RouterTypeExample.scala#roundRobinRouter +.. includecode:: code/docs/routing/RouterTypeExample.scala#roundRobinRouter When run you should see a similar output to this: @@ -177,7 +182,7 @@ the message it receives to this routee. This procedure will happen each time it receives a message. Code example: -.. includecode:: code/akka/docs/routing/RouterTypeExample.scala#randomRouter +.. includecode:: code/docs/routing/RouterTypeExample.scala#randomRouter When run you should see a similar output to this: @@ -210,14 +215,14 @@ The selection is done in this order: Code example: -.. includecode:: code/akka/docs/routing/RouterTypeExample.scala#smallestMailboxRouter +.. includecode:: code/docs/routing/RouterTypeExample.scala#smallestMailboxRouter BroadcastRouter *************** A broadcast router forwards the message it receives to *all* its routees. Code example: -.. includecode:: code/akka/docs/routing/RouterTypeExample.scala#broadcastRouter +.. includecode:: code/docs/routing/RouterTypeExample.scala#broadcastRouter When run you should see a similar output to this: @@ -237,7 +242,7 @@ The ScatterGatherFirstCompletedRouter will send the message on to all its routee It then waits for first result it gets back. This result will be sent back to original sender. Code example: -.. includecode:: code/akka/docs/routing/RouterTypeExample.scala#scatterGatherFirstCompletedRouter +.. includecode:: code/docs/routing/RouterTypeExample.scala#scatterGatherFirstCompletedRouter When run you should see this: @@ -269,16 +274,16 @@ of routees dynamically. This is an example of how to create a resizable router that is defined in configuration: -.. includecode:: code/akka/docs/routing/RouterViaConfigExample.scala#config-resize +.. includecode:: code/docs/routing/RouterViaConfigExample.scala#config-resize -.. includecode:: code/akka/docs/routing/RouterViaConfigExample.scala#configurableRoutingWithResizer +.. includecode:: code/docs/routing/RouterViaConfigExample.scala#configurableRoutingWithResizer Several more configuration options are available and described in ``akka.actor.deployment.default.resizer`` section of the reference :ref:`configuration`. This is an example of how to programmatically create a resizable router: -.. includecode:: code/akka/docs/routing/RouterViaProgramExample.scala#programmaticRoutingWithResizer +.. includecode:: code/docs/routing/RouterViaProgramExample.scala#programmaticRoutingWithResizer *It is also worth pointing out that if you define the ``router`` in the configuration file then this value will be used instead of any programmatically sent parameters.* @@ -383,13 +388,41 @@ stealing it from their siblings. that was configured for them in their ``Props``, it is not possible to change an actors dispatcher after it has been created. -The “head” router, of course, cannot run on the same balancing dispatcher, -because it does not process the same messages, hence this special actor does +The “head” router cannot always run on the same dispatcher, because it +does not process the same type of messages, hence this special actor does not use the dispatcher configured in :class:`Props`, but takes the ``routerDispatcher`` from the :class:`RouterConfig` instead, which defaults to the actor system’s default dispatcher. All standard routers allow setting this property in their constructor or factory method, custom routers have to implement the method in a suitable way. -.. includecode:: code/akka/docs/routing/RouterDocSpec.scala#dispatchers +.. includecode:: code/docs/routing/RouterDocSpec.scala#dispatchers +.. note:: + + It is not allowed to configure the ``routerDispatcher`` to be a + :class:`BalancingDispatcher` since the messages meant for the special + router actor cannot be processed by any other actor. + +At first glance there seems to be an overlap between the +:class:`BalancingDispatcher` and Routers, but they complement each other. +The balancing dispatcher is in charge of running the actors while the routers +are in charge of deciding which message goes where. A router can also have +children that span multiple actor systems, even remote ones, but a dispatcher +lives inside a single actor system. + +When using a :class:`RoundRobinRouter` with a :class:`BalancingDispatcher` +there are some configuration settings to take into account. + +- There can only be ``nr-of-instances`` messages being processed at the same + time no matter how many threads are configured for the + :class:`BalancingDispatcher`. + +- Having ``throughput`` set to a low number makes no sense since you will only + be handing off to another actor that processes the same :class:`MailBox` + as yourself, which can be costly. Either the message just got into the + mailbox and you can receive it as well as anybody else, or everybody else + is busy and you are the only one available to receive the message. + +- Resizing the number of routees only introduce inertia, since resizing + is performed at specified intervals, but work stealing is instantaneous. diff --git a/akka-docs/scala/scheduler.rst b/akka-docs/scala/scheduler.rst index a98f0f563c..0a5b8aed51 100644 --- a/akka-docs/scala/scheduler.rst +++ b/akka-docs/scala/scheduler.rst @@ -25,13 +25,13 @@ scheduled operation. Some examples ------------- -.. includecode:: code/akka/docs/actor/SchedulerDocSpec.scala +.. includecode:: code/docs/actor/SchedulerDocSpec.scala :include: imports1,schedule-one-off-message -.. includecode:: code/akka/docs/actor/SchedulerDocSpec.scala +.. includecode:: code/docs/actor/SchedulerDocSpec.scala :include: schedule-one-off-thunk -.. includecode:: code/akka/docs/actor/SchedulerDocSpec.scala +.. includecode:: code/docs/actor/SchedulerDocSpec.scala :include: schedule-recurring From ``akka.actor.ActorSystem`` diff --git a/akka-docs/scala/serialization.rst b/akka-docs/scala/serialization.rst index 2ab0a7b633..10283b441f 100644 --- a/akka-docs/scala/serialization.rst +++ b/akka-docs/scala/serialization.rst @@ -21,12 +21,12 @@ For Akka to know which ``Serializer`` to use for what, you need edit your :ref:` in the "akka.actor.serializers"-section you bind names to implementations of the ``akka.serialization.Serializer`` you wish to use, like this: -.. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala#serialize-serializers-config +.. includecode:: code/docs/serialization/SerializationDocSpec.scala#serialize-serializers-config After you've bound names to different implementations of ``Serializer`` you need to wire which classes should be serialized using which ``Serializer``, this is done in the "akka.actor.serialization-bindings"-section: -.. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala#serialization-bindings-config +.. includecode:: code/docs/serialization/SerializationDocSpec.scala#serialization-bindings-config You only need to specify the name of an interface or abstract base class of the messages. In case of ambiguity, i.e. the message implements several of the @@ -53,7 +53,7 @@ Verification If you want to verify that your messages are serializable you can enable the following config option: -.. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala#serialize-messages-config +.. includecode:: code/docs/serialization/SerializationDocSpec.scala#serialize-messages-config .. warning:: @@ -62,7 +62,7 @@ If you want to verify that your messages are serializable you can enable the fol If you want to verify that your ``Props`` are serializable you can enable the following config option: -.. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala#serialize-creators-config +.. includecode:: code/docs/serialization/SerializationDocSpec.scala#serialize-creators-config .. warning:: @@ -75,7 +75,7 @@ Programmatic If you want to programmatically serialize/deserialize using Akka Serialization, here's some examples: -.. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala +.. includecode:: code/docs/serialization/SerializationDocSpec.scala :include: imports,programmatic For more information, have a look at the ``ScalaDoc`` for ``akka.serialization._`` @@ -84,14 +84,14 @@ Customization ============= So, lets say that you want to create your own ``Serializer``, -you saw the ``akka.docs.serialization.MyOwnSerializer`` in the config example above? +you saw the ``docs.serialization.MyOwnSerializer`` in the config example above? Creating new Serializers ------------------------ First you need to create a class definition of your ``Serializer`` like so: -.. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala +.. includecode:: code/docs/serialization/SerializationDocSpec.scala :include: imports,my-own-serializer :exclude: ... @@ -104,9 +104,58 @@ Serializing ActorRefs All ActorRefs are serializable using JavaSerializer, but in case you are writing your own serializer, you might want to know how to serialize and deserialize them properly, here's the magic incantation: -.. includecode:: code/akka/docs/serialization/SerializationDocSpec.scala +.. includecode:: code/docs/serialization/SerializationDocSpec.scala :include: imports,actorref-serializer +.. note:: + + ``ActorPath.toStringWithAddress`` only differs from ``toString`` if the + address does not already have ``host`` and ``port`` components, i.e. it only + inserts address information for local addresses. + +This assumes that serialization happens in the context of sending a message +through the remote transport. There are other uses of serialization, though, +e.g. storing actor references outside of an actor application (database, +durable mailbox, etc.). In this case, it is important to keep in mind that the +address part of an actor’s path determines how that actor is communicated with. +Storing a local actor path might be the right choice if the retrieval happens +in the same logical context, but it is not enough when deserializing it on a +different network host: for that it would need to include the system’s remote +transport address. An actor system is not limited to having just one remote +transport per se, which makes this question a bit more interesting. + +In the general case, the local address to be used depends on the type of remote +address which shall be the recipient of the serialized information. Use +:meth:`ActorRefProvider.getExternalAddressFor(remoteAddr)` to query the system +for the appropriate address to use when sending to ``remoteAddr``: + +.. includecode:: code/docs/serialization/SerializationDocSpec.scala + :include: external-address + +This requires that you know at least which type of address will be supported by +the system which will deserialize the resulting actor reference; if you have no +concrete address handy you can create a dummy one for the right protocol using +``Address(protocol, "", "", 0)`` (assuming that the actual transport used is as +lenient as Akka’s RemoteActorRefProvider). + +There is a possible simplification available if you are just using the default +:class:`NettyRemoteTransport` with the :meth:`RemoteActorRefProvider`, which is +enabled by the fact that this combination has just a single remote address. +This approach relies on internal API, which means that it is not guaranteed to +be supported in future versions. To make this caveat more obvious, some bridge +code in the ``akka`` package is required to make it work: + +.. includecode:: code/docs/serialization/SerializationDocSpec.scala + :include: extract-transport + +And with this, the address extraction goes like this: + +.. includecode:: code/docs/serialization/SerializationDocSpec.scala + :include: external-address-default + +This solution has to be adapted once other providers are used (like the planned +extensions for clustering). + Deep serialization of Actors ---------------------------- @@ -135,3 +184,14 @@ representation into a real reference. :class:`DynamicVariable` is a thread-local variable, so be sure to have it set while deserializing anything which might contain actor references. + +External Akka Serializers +========================= + +`Akka-protostuff by Roman Levenstein `_ + + +`Akka-quickser by Roman Levenstein `_ + + +`Akka-kryo by Roman Levenstein `_ diff --git a/akka-docs/scala/testing.rst b/akka-docs/scala/testing.rst index 15f73f4ef0..0835db18e7 100644 --- a/akka-docs/scala/testing.rst +++ b/akka-docs/scala/testing.rst @@ -61,7 +61,7 @@ Having access to the actual :class:`Actor` object allows application of all traditional unit testing techniques on the contained methods. Obtaining a reference is done like this: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#test-actor-ref +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-actor-ref Since :class:`TestActorRef` is generic in the actor type it returns the underlying actor with its proper static type. From this point on you may bring @@ -74,7 +74,7 @@ Testing that an expected exception is thrown while processing a message sent to the actor under test can be done by using a :class:`TestActorRef` :meth:`receive` based invocation: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#test-expecting-exceptions +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-expecting-exceptions .. _TestFSMRef: @@ -85,7 +85,7 @@ If your actor under test is a :class:`FSM`, you may use the special :class:`TestFSMRef` which offers all features of a normal :class:`TestActorRef` and in addition allows access to the internal state: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#test-fsm-ref +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-fsm-ref Due to a limitation in Scala’s type inference, there is only the factory method shown above, so you will probably write code like ``TestFSMRef(new MyFSM)`` @@ -114,7 +114,7 @@ usual. This trick is made possible by the :class:`CallingThreadDispatcher` described below; this dispatcher is set implicitly for any actor instantiated into a :class:`TestActorRef`. -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#test-behavior +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-behavior As the :class:`TestActorRef` is a subclass of :class:`LocalActorRef` with a few special extras, also aspects like supervision and restarting work properly, but @@ -143,7 +143,7 @@ any thrown exceptions, then there is another mode available for you: just use the :meth:`receive` method :class:`TestActorRef`, which will be forwarded to the underlying actor: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#test-unhandled +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-unhandled The above sample assumes the default behavior for unhandled messages, i.e. that the actor doesn't swallow all messages and doesn't override :meth:`unhandled`. @@ -182,7 +182,7 @@ single procedure drives the test. The :class:`TestKit` class contains a collection of tools which makes this common task easy. -.. includecode:: code/akka/docs/testkit/PlainWordSpec.scala#plain-spec +.. includecode:: code/docs/testkit/PlainWordSpec.scala#plain-spec The :class:`TestKit` contains an actor named :obj:`testActor` which is the entry point for messages to be examined with the various ``expectMsg...`` @@ -194,8 +194,10 @@ is a whole set of examination methods, e.g. receiving all consecutive messages matching certain criteria, receiving a whole sequence of fixed messages or classes, receiving nothing for some time, etc. -Remember to shut down the actor system after the test is finished (also in case -of failure) so that all actors—including the test actor—are stopped. +The ActorSystem passed in to the constructor of TestKit is accessible via the +:obj:`system` member. Remember to shut down the actor system after the test is +finished (also in case of failure) so that all actors—including the test +actor—are stopped. Built-In Assertions ------------------- @@ -337,7 +339,7 @@ handler with the :class:`TestEventListener` and using an :class:`EventFilter` allows assertions on log messages, including those which are generated by exceptions: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#event-filter +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#event-filter .. _TestKit.within: @@ -370,7 +372,7 @@ It should be noted that if the last message-receiving assertion of the block is latencies. This means that while individual contained assertions still use the maximum time bound, the overall block may take arbitrarily longer in this case. -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#test-within +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-within .. note:: @@ -393,7 +395,7 @@ internally scaled by a factor taken from the :ref:`configuration`, You can scale other durations with the same factor by using the implicit conversion in ``akka.testkit`` package object to add dilated function to :class:`Duration`. -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#duration-dilation +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#duration-dilation Resolving Conflicts with Implicit ActorRef ------------------------------------------ @@ -401,7 +403,7 @@ Resolving Conflicts with Implicit ActorRef If you want the sender of messages inside your TestKit-based tests to be the ``testActor`` simply mix in ``ÌmplicitSender`` into your test. -.. includecode:: code/akka/docs/testkit/PlainWordSpec.scala#implicit-sender +.. includecode:: code/docs/testkit/PlainWordSpec.scala#implicit-sender Using Multiple Probe Actors --------------------------- @@ -414,7 +416,7 @@ message flows. To make this more powerful and convenient, there is a concrete implementation called :class:`TestProbe`. The functionality is best explained using a small example: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala +.. includecode:: code/docs/testkit/TestkitDocSpec.scala :include: imports-test-probe,my-double-echo,test-probe Here a the system under test is simulated by :class:`MyDoubleEcho`, which is @@ -428,7 +430,7 @@ the test setup. Probes may also be equipped with custom assertions to make your test code even more concise and clear: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala +.. includecode:: code/docs/testkit/TestkitDocSpec.scala :include: test-special-probe You have complete flexibility here in mixing and matching the :class:`TestKit` @@ -442,7 +444,7 @@ Replying to Messages Received by Probes The probes keep track of the communications channel for replies, if possible, so they can also reply: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#test-probe-reply +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-probe-reply Forwarding Messages Received by Probes ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -453,7 +455,7 @@ sent to a :class:`TestProbe` ``probe`` instead, you can make assertions concerning volume and timing of the message flow while still keeping the network functioning: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala +.. includecode:: code/docs/testkit/TestkitDocSpec.scala :include: test-probe-forward-actors,test-probe-forward The ``dest`` actor will receive the same message invocation as if no test probe @@ -514,7 +516,7 @@ How to use it Just set the dispatcher as you normally would: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#calling-thread-dispatcher +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#calling-thread-dispatcher How it works ------------ @@ -618,7 +620,7 @@ options: ``akka.actor.debug.receive`` — which enables the :meth:`loggable` statement to be applied to an actor’s :meth:`receive` function: -.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#logging-receive +.. includecode:: code/docs/testkit/TestkitDocSpec.scala#logging-receive . If the abovementioned setting is not given in the :ref:`configuration`, this method will @@ -657,3 +659,65 @@ full logging of actor activities using this configuration fragment:: } } +Different Testing Frameworks +============================ + +Akka’s own test suite is written using `ScalaTest `_, +which also shines through in documentation examples. However, the TestKit and +its facilities do not depend on that framework, you can essentially use +whichever suits your development style best. + +This section contains a collection of known gotchas with some other frameworks, +which is by no means exhaustive and does not imply endorsement or special +support. + +When you need it to be a trait +------------------------------ + +If for some reason it is a problem to inherit from :class:`TestKit` due to it +being a concrete class instead of a trait, there’s :class:`TestKitBase`: + +.. includecode:: code/docs/testkit/TestkitDocSpec.scala + :include: test-kit-base + :exclude: put-your-test-code-here + +The ``implicit lazy val system`` must be declared exactly like that (you can of +course pass arguments to the actor system factory as needed) because trait +:class:`TestKitBase` needs the system during its construction. + +.. warning:: + + Use of the trait is discouraged because of potential issues with binary + backwards compatibility in the future, use at own risk. + +Specs2 +------ + +Some `Specs2 `_ users have contributed examples of how to work around some clashes which may arise: + +* Mixing TestKit into :class:`org.specs2.mutable.Specification` results in a + name clash involving the ``end`` method (which is a private variable in + TestKit and an abstract method in Specification); if mixing in TestKit first, + the code may compile but might then fail at runtime. The work-around—which is + actually beneficial also for the third point—is to apply the TestKit together + with :class:`org.specs2.specification.Scope`. +* The Specification traits provide a :class:`Duration` DSL which uses partly + the same method names as :class:`akka.util.Duration`, resulting in ambiguous + implicits if ``akka.util.duration._`` is imported. There are two work-arounds: + + * either use the Specification variant of Duration and supply an implicit + conversion to the Akka Duration. This conversion is not supplied with the + Akka distribution because that would mean that our JAR files would dependon + Specs2, which is not justified by this little feature. + + * or mix :class:`org.specs2.time.NoTimeConversions` into the Specification. + +* Specifications are by default executed concurrently, which requires some care + when writing the tests or alternatively the ``sequential`` keyword. + +You can use the following two examples as guidelines: + +.. includecode:: code/docs/testkit/Specs2DemoSpec.scala + +.. includecode:: code/docs/testkit/Specs2DemoAcceptance.scala + diff --git a/akka-docs/scala/testkit-example.rst b/akka-docs/scala/testkit-example.rst index 54a848d267..dd7aba0812 100644 --- a/akka-docs/scala/testkit-example.rst +++ b/akka-docs/scala/testkit-example.rst @@ -6,142 +6,5 @@ TestKit Example (Scala) Ray Roestenburg's example code from `his blog `_ adapted to work with Akka 2.x. -.. code-block:: scala +.. includecode:: code/docs/testkit/TestkitUsageSpec.scala#testkit-usage - package unit.akka - - import org.scalatest.matchers.ShouldMatchers - import org.scalatest.{WordSpec, BeforeAndAfterAll} - import akka.actor.Actor._ - import akka.util.duration._ - import akka.testkit.TestKit - import java.util.concurrent.TimeUnit - import akka.actor.{ActorRef, Actor} - import util.Random - - /** - * a Test to show some TestKit examples - */ - - class TestKitUsageSpec extends WordSpec with BeforeAndAfterAll with ShouldMatchers with TestKit { - val system = ActorSystem() - import system._ - val echoRef = actorOf(Props(new EchoActor)) - val forwardRef = actorOf(Props(new ForwardingActor(testActor))) - val filterRef = actorOf(Props(new FilteringActor(testActor))) - val randomHead = Random.nextInt(6) - val randomTail = Random.nextInt(10) - val headList = List().padTo(randomHead, "0") - val tailList = List().padTo(randomTail, "1") - val seqRef = actorOf(Props(new SequencingActor(testActor, headList, tailList))) - - override protected def afterAll(): scala.Unit = { - stopTestActor - echoRef.stop() - forwardRef.stop() - filterRef.stop() - seqRef.stop() - } - - "An EchoActor" should { - "Respond with the same message it receives" in { - within(100 millis) { - echoRef ! "test" - expectMsg("test") - } - } - } - "A ForwardingActor" should { - "Forward a message it receives" in { - within(100 millis) { - forwardRef ! "test" - expectMsg("test") - } - } - } - "A FilteringActor" should { - "Filter all messages, except expected messagetypes it receives" in { - var messages = List[String]() - within(100 millis) { - filterRef ! "test" - expectMsg("test") - filterRef ! 1 - expectNoMsg - filterRef ! "some" - filterRef ! "more" - filterRef ! 1 - filterRef ! "text" - filterRef ! 1 - - receiveWhile(500 millis) { - case msg: String => messages = msg :: messages - } - } - messages.length should be(3) - messages.reverse should be(List("some", "more", "text")) - } - } - "A SequencingActor" should { - "receive an interesting message at some point " in { - within(100 millis) { - seqRef ! "something" - ignoreMsg { - case msg: String => msg != "something" - } - expectMsg("something") - ignoreMsg { - case msg: String => msg == "1" - } - expectNoMsg - } - } - } - } - - /** - * An Actor that echoes everything you send to it - */ - class EchoActor extends Actor { - def receive = { - case msg => { - self.reply(msg) - } - } - } - - /** - * An Actor that forwards every message to a next Actor - */ - class ForwardingActor(next: ActorRef) extends Actor { - def receive = { - case msg => { - next ! msg - } - } - } - - /** - * An Actor that only forwards certain messages to a next Actor - */ - class FilteringActor(next: ActorRef) extends Actor { - def receive = { - case msg: String => { - next ! msg - } - case _ => None - } - } - - /** - * An actor that sends a sequence of messages with a random head list, an interesting value and a random tail list - * The idea is that you would like to test that the interesting value is received and that you cant be bothered with the rest - */ - class SequencingActor(next: ActorRef, head: List[String], tail: List[String]) extends Actor { - def receive = { - case msg => { - head map (next ! _) - next ! msg - tail map (next ! _) - } - } - } diff --git a/akka-docs/scala/transactors.rst b/akka-docs/scala/transactors.rst index 1dc1d76c28..d915b15aa4 100644 --- a/akka-docs/scala/transactors.rst +++ b/akka-docs/scala/transactors.rst @@ -63,9 +63,9 @@ Here is an example of coordinating two simple counter Actors so that they both increment together in coordinated transactions. If one of them was to fail to increment, the other would also fail. -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#coordinated-example +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#coordinated-example -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#run-coordinated-example +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#run-coordinated-example Note that creating a ``Coordinated`` object requires a ``Timeout`` to be specified for the coordinated transaction. This can be done implicitly, by @@ -73,36 +73,36 @@ having an implicit ``Timeout`` in scope, or explicitly, by passing the timeout when creating a a ``Coordinated`` object. Here's an example of specifying an implicit timeout: -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#implicit-timeout +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#implicit-timeout To start a new coordinated transaction that you will also participate in, just create a ``Coordinated`` object (this assumes an implicit timeout): -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#create-coordinated +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#create-coordinated To start a coordinated transaction that you won't participate in yourself you can create a ``Coordinated`` object with a message and send it directly to an actor. The recipient of the message will be the first member of the coordination set: -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#send-coordinated +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#send-coordinated To receive a coordinated message in an actor simply match it in a case statement: -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#receive-coordinated +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#receive-coordinated :exclude: coordinated-atomic To include another actor in the same coordinated transaction that you've created or received, use the apply method on that object. This will increment the number of parties involved by one and create a new ``Coordinated`` object to be sent. -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#include-coordinated +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#include-coordinated To enter the coordinated transaction use the atomic method of the coordinated object: -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#coordinated-atomic +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#coordinated-atomic The coordinated transaction will wait for the other transactions before committing. If any of the coordinated transactions fail then they all fail. @@ -125,7 +125,7 @@ transactions, using the explicit coordination described above. Here's an example of a simple transactor that will join a coordinated transaction: -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#counter-example +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#counter-example You could send this Counter transactor a ``Coordinated(Increment)`` message. If you were to send it just an ``Increment`` message it will create its own @@ -141,16 +141,16 @@ allows you to specify both the actor to send to, and the message to send. Example of coordinating an increment: -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#friendly-counter-example +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#friendly-counter-example Using ``include`` to include more than one transactor: -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#coordinate-include +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#coordinate-include Using ``sendTo`` to coordinate transactions but pass-on a different message than the one that was received: -.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#coordinate-sendto +.. includecode:: code/docs/transactor/TransactorDocSpec.scala#coordinate-sendto To execute directly before or after the coordinated transaction, override the ``before`` and ``after`` methods. These methods also expect partial functions diff --git a/akka-docs/scala/typed-actors.rst b/akka-docs/scala/typed-actors.rst index fc570e60a7..bd7d92f924 100644 --- a/akka-docs/scala/typed-actors.rst +++ b/akka-docs/scala/typed-actors.rst @@ -38,7 +38,7 @@ The tools of the trade Before we create our first Typed Actor we should first go through the tools that we have at our disposal, it's located in ``akka.actor.TypedActor``. -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: typed-actor-extension-tools .. warning:: @@ -55,37 +55,37 @@ To create a Typed Actor you need to have one or more interfaces, and one impleme Our example interface: -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: imports,typed-actor-iface :exclude: typed-actor-iface-methods Our example implementation of that interface: -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: imports,typed-actor-impl :exclude: typed-actor-impl-methods The most trivial way of creating a Typed Actor instance of our Squarer: -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: typed-actor-create1 First type is the type of the proxy, the second type is the type of the implementation. If you need to call a specific constructor you do it like this: -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: typed-actor-create2 Since you supply a Props, you can specify which dispatcher to use, what the default timeout should be used and more. Now, our Squarer doesn't have any methods, so we'd better add those. -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: imports,typed-actor-iface Alright, now we've got some methods we can call, but we need to implement those in SquarerImpl. -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: imports,typed-actor-impl Excellent, now we have an interface and an implementation of that interface, @@ -99,7 +99,7 @@ Methods returning: * ``Unit`` will be dispatched with ``fire-and-forget`` semantics, exactly like ``ActorRef.tell`` * ``akka.dispatch.Future[_]`` will use ``send-request-reply`` semantics, exactly like ``ActorRef.ask`` * ``scala.Option[_]`` or ``akka.japi.Option`` will use ``send-request-reply`` semantics, but *will* block to wait for an answer, - and return None if no answer was produced within the timout, or scala.Some/akka.japi.Some containing the result otherwise. + and return None if no answer was produced within the timeout, or scala.Some/akka.japi.Some containing the result otherwise. Any exception that was thrown during this call will be rethrown. * Any other type of value will use ``send-request-reply`` semantics, but *will* block to wait for an answer, throwing ``java.util.concurrent.TimeoutException`` if there was a timeout or rethrow any exception that was thrown during this call. @@ -113,7 +113,7 @@ we *strongly* recommend that parameters passed are immutable. One-way message send ^^^^^^^^^^^^^^^^^^^^ -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: typed-actor-call-oneway As simple as that! The method will be executed on another thread; asynchronously. @@ -121,13 +121,13 @@ As simple as that! The method will be executed on another thread; asynchronously Request-reply message send ^^^^^^^^^^^^^^^^^^^^^^^^^^ -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: typed-actor-call-option This will block for as long as the timeout that was set in the Props of the Typed Actor, if needed. It will return ``None`` if a timeout occurs. -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: typed-actor-call-strict This will block for as long as the timeout that was set in the Props of the Typed Actor, @@ -136,7 +136,7 @@ if needed. It will throw a ``java.util.concurrent.TimeoutException`` if a timeou Request-reply-with-future message send ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: typed-actor-call-future This call is asynchronous, and the Future returned can be used for asynchronous composition. @@ -146,12 +146,12 @@ Stopping Typed Actors Since Akkas Typed Actors are backed by Akka Actors they must be stopped when they aren't needed anymore. -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: typed-actor-stop This asynchronously stops the Typed Actor associated with the specified proxy ASAP. -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala +.. includecode:: code/docs/actor/TypedActorDocSpec.scala :include: typed-actor-poisonpill This asynchronously stops the Typed Actor associated with the specified proxy @@ -161,9 +161,13 @@ Typed Actor Hierarchies ----------------------- Since you can obtain a contextual Typed Actor Extension by passing in an ``ActorContext`` -you can create child Typed Actors by invoking ``typedActorOf(..)`` on that. +you can create child Typed Actors by invoking ``typedActorOf(..)`` on that: -This also works for creating child Typed Actors in regular Akka Actors. +.. includecode:: code/docs/actor/TypedActorDocSpec.scala + :include: typed-actor-hierarchy + +You can also create a child Typed Actor in regular Akka Actors by giving the ``ActorContext`` +as an input parameter to TypedActor.get(…). Supervisor Strategy ------------------- @@ -208,13 +212,13 @@ Lookup & Remoting Since ``TypedActors`` are backed by ``Akka Actors``, you can use ``actorFor`` together with ``typedActorOf`` to proxy ``ActorRefs`` potentially residing on remote nodes. -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala#typed-actor-remote +.. includecode:: code/docs/actor/TypedActorDocSpec.scala#typed-actor-remote Supercharging ------------- Here's an example on how you can use traits to mix in behavior in your Typed Actors. -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala#typed-actor-supercharge +.. includecode:: code/docs/actor/TypedActorDocSpec.scala#typed-actor-supercharge -.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala#typed-actor-supercharge-usage +.. includecode:: code/docs/actor/TypedActorDocSpec.scala#typed-actor-supercharge-usage diff --git a/akka-docs/scala/zeromq.rst b/akka-docs/scala/zeromq.rst index 5c06aeda23..d94ee81270 100644 --- a/akka-docs/scala/zeromq.rst +++ b/akka-docs/scala/zeromq.rst @@ -19,18 +19,18 @@ Connection ZeroMQ supports multiple connectivity patterns, each aimed to meet a different set of requirements. Currently, this module supports publisher-subscriber connections and connections based on dealers and routers. For connecting or accepting connections, a socket must be created. Sockets are always created using the ``akka.zeromq.ZeroMQExtension``, for example: -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-socket +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#pub-socket or by importing the ``akka.zeromq._`` package to make newSocket method available on system, via an implicit conversion. -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-socket2 +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#pub-socket2 Above examples will create a ZeroMQ Publisher socket that is Bound to the port 1234 on localhost. Similarly you can create a subscription socket, with a listener, that subscribes to all messages from the publisher using: -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#sub-socket +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#sub-socket The following sub-sections describe the supported connection patterns and how they can be used in an Akka environment. However, for a comprehensive discussion of connection patterns, please refer to `ZeroMQ -- The Guide `_. @@ -46,18 +46,18 @@ When you're using zeromq pub/sub you should be aware that it needs multicast - c An actor is subscribed to a topic as follows: -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#sub-topic-socket +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#sub-topic-socket It is a prefix match so it is subscribed to all topics starting with ``foo.bar``. Note that if the given string is empty or ``SubscribeAll`` is used, the actor is subscribed to all topics. To unsubscribe from a topic you do the following: -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#unsub-topic-socket +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#unsub-topic-socket To publish messages to a topic you must use two Frames with the topic in the first frame. -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#pub-topic +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#pub-topic Pub-Sub in Action ^^^^^^^^^^^^^^^^^ @@ -67,16 +67,16 @@ The following example illustrates one publisher with two subscribers. The publisher monitors current heap usage and system load and periodically publishes ``Heap`` events on the ``"health.heap"`` topic and ``Load`` events on the ``"health.load"`` topic. -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#health +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#health Let's add one subscriber that logs the information. It subscribes to all topics starting with ``"health"``, i.e. both ``Heap`` and ``Load`` events. -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#logger +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#logger Another subscriber keep track of used heap and warns if too much heap is used. It only subscribes to ``Heap`` events. -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#alerter +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#alerter Router-Dealer Connection ------------------------ @@ -87,7 +87,7 @@ With those socket types you can build your own reliable pub sub broker that uses To create a Router socket that has a high watermark configured, you would do: -.. includecode:: code/akka/docs/zeromq/ZeromqDocSpec.scala#high-watermark +.. includecode:: code/docs/zeromq/ZeromqDocSpec.scala#high-watermark The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket. diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf b/akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf index 1a1b7b721b..1fb5cceeb1 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/resources/reference.conf @@ -13,39 +13,51 @@ akka { file-based { # directory below which this queue resides directory-path = "./_mb" - + # attempting to add an item after the queue reaches this size (in items) will fail. max-items = 2147483647 - + # attempting to add an item after the queue reaches this size (in bytes) will fail. max-size = 2147483647 bytes - + # attempting to add an item larger than this size (in bytes) will fail. max-item-size = 2147483647 bytes - + # maximum expiration time for this queue (seconds). max-age = 0s - + # maximum journal size before the journal should be rotated. max-journal-size = 16 MiB - + # maximum size of a queue before it drops into read-behind mode. max-memory-size = 128 MiB - + # maximum overflow (multiplier) of a journal file before we re-create it. max-journal-overflow = 10 - + # absolute maximum size of a journal file until we rebuild it, no matter what. max-journal-size-absolute = 9223372036854775807 bytes - + # whether to drop older items (instead of newer) when the queue is full - discard-old-when-full = on - + discard-old-when-full = on + # whether to keep a journal file at all - keep-journal = on - + keep-journal = on + # whether to sync the journal after each transaction - sync-journal = off + sync-journal = off + + # circuit breaker configuration + circuit-breaker { + # maximum number of failures before opening breaker + max-failures = 3 + + # duration of time beyond which a call is assumed to be timed out and considered a failure + call-timeout = 3 seconds + + # duration of time to wait until attempting to reset the breaker during which all calls fail-fast + reset-timeout = 30 seconds + } } } } diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailbox.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailbox.scala index ef8a28b2cf..c703bf0b49 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailbox.scala @@ -5,64 +5,68 @@ package akka.actor.mailbox import akka.actor.ActorContext -import akka.dispatch.{ Envelope, MessageQueue } import akka.event.Logging import akka.actor.ActorRef -import akka.dispatch.MailboxType import com.typesafe.config.Config -import akka.util.NonFatal -import akka.config.ConfigurationException +import akka.ConfigurationException import akka.actor.ActorSystem +import akka.dispatch._ +import akka.util.{ Duration, NonFatal } +import akka.pattern.{ CircuitBreakerOpenException, CircuitBreaker } +import akka.actor.ExtendedActorSystem class FileBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType { private val settings = new FileBasedMailboxSettings(systemSettings, config) - override def create(owner: Option[ActorContext]): MessageQueue = owner match { - case Some(o) ⇒ new FileBasedMessageQueue(o, settings) - case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") + override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = owner zip system headOption match { + case Some((o, s: ExtendedActorSystem)) ⇒ new FileBasedMessageQueue(o, s, settings) + case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)") } } -class FileBasedMessageQueue(_owner: ActorContext, val settings: FileBasedMailboxSettings) extends DurableMessageQueue(_owner) with DurableMessageSerialization { +class FileBasedMessageQueue(_owner: ActorRef, _system: ExtendedActorSystem, val settings: FileBasedMailboxSettings) + extends DurableMessageQueue(_owner, _system) with DurableMessageSerialization { + // TODO Is it reasonable for all FileBasedMailboxes to have their own logger? + private val log = Logging(system, "FileBasedMessageQueue") - val log = Logging(system, "FileBasedMessageQueue") - - val queuePath = settings.QueuePath + val breaker = CircuitBreaker(system.scheduler, settings.CircuitBreakerMaxFailures, settings.CircuitBreakerCallTimeout, settings.CircuitBreakerResetTimeout) private val queue = try { - (new java.io.File(queuePath)) match { + (new java.io.File(settings.QueuePath)) match { case dir if dir.exists && !dir.isDirectory ⇒ throw new IllegalStateException("Path already occupied by non-directory " + dir) case dir if !dir.exists ⇒ if (!dir.mkdirs() && !dir.isDirectory) throw new IllegalStateException("Creation of directory failed " + dir) - case _ ⇒ //All good + case _ ⇒ // All good } - val queue = new filequeue.PersistentQueue(queuePath, name, settings, log) + val queue = new filequeue.PersistentQueue(settings.QueuePath, name, settings, log) queue.setup // replays journal queue.discardExpired queue } catch { - case e: Exception ⇒ + case NonFatal(e) ⇒ log.error(e, "Could not create a file-based mailbox") throw e } def enqueue(receiver: ActorRef, envelope: Envelope) { - queue.add(serialize(envelope)) + breaker.withSyncCircuitBreaker(queue.add(serialize(envelope))) } - def dequeue(): Envelope = try { - val item = queue.remove - if (item.isDefined) { - queue.confirmRemove(item.get.xid) - deserialize(item.get.data) - } else null - } catch { - case e: java.util.NoSuchElementException ⇒ null - case e: Exception ⇒ - log.error(e, "Couldn't dequeue from file-based mailbox") - throw e + def dequeue(): Envelope = { + breaker.withSyncCircuitBreaker( + try { + queue.remove.map(item ⇒ { queue.confirmRemove(item.xid); deserialize(item.data) }).orNull + } catch { + case _: java.util.NoSuchElementException ⇒ null + case e: CircuitBreakerOpenException ⇒ + log.debug(e.getMessage()) + throw e + case NonFatal(e) ⇒ + log.error(e, "Couldn't dequeue from file-based mailbox, due to [{}]", e.getMessage()) + throw e + }) } def numberOfMessages: Int = { - queue.length.toInt + breaker.withSyncCircuitBreaker(queue.length.toInt) } def hasMessages: Boolean = numberOfMessages > 0 @@ -77,6 +81,5 @@ class FileBasedMessageQueue(_owner: ActorContext, val settings: FileBasedMailbox case NonFatal(_) ⇒ false } - def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = () - + def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = () } diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxSettings.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxSettings.scala index 6511bf9e00..27088dfc92 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxSettings.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxSettings.scala @@ -11,24 +11,25 @@ import akka.actor.ActorSystem class FileBasedMailboxSettings(val systemSettings: ActorSystem.Settings, val userConfig: Config) extends DurableMailboxSettings { - def name = "file-based" + def name: String = "file-based" val config = initialize - import config._ - val QueuePath = getString("directory-path") + final val QueuePath: String = getString("directory-path") + final val MaxItems: Int = getInt("max-items") + final val MaxSize: Long = getBytes("max-size") + final val MaxItemSize: Long = getBytes("max-item-size") + final val MaxAge: Duration = Duration(getMilliseconds("max-age"), MILLISECONDS) + final val MaxJournalSize: Long = getBytes("max-journal-size") + final val MaxMemorySize: Long = getBytes("max-memory-size") + final val MaxJournalOverflow: Int = getInt("max-journal-overflow") + final val MaxJournalSizeAbsolute: Long = getBytes("max-journal-size-absolute") + final val DiscardOldWhenFull: Boolean = getBoolean("discard-old-when-full") + final val KeepJournal: Boolean = getBoolean("keep-journal") + final val SyncJournal: Boolean = getBoolean("sync-journal") - val MaxItems = getInt("max-items") - val MaxSize = getBytes("max-size") - val MaxItemSize = getBytes("max-item-size") - val MaxAge = Duration(getMilliseconds("max-age"), MILLISECONDS) - val MaxJournalSize = getBytes("max-journal-size") - val MaxMemorySize = getBytes("max-memory-size") - val MaxJournalOverflow = getInt("max-journal-overflow") - val MaxJournalSizeAbsolute = getBytes("max-journal-size-absolute") - val DiscardOldWhenFull = getBoolean("discard-old-when-full") - val KeepJournal = getBoolean("keep-journal") - val SyncJournal = getBoolean("sync-journal") - -} \ No newline at end of file + final val CircuitBreakerMaxFailures = getInt("circuit-breaker.max-failures") + final val CircuitBreakerCallTimeout = Duration.fromNanos(getNanoseconds("circuit-breaker.call-timeout")) + final val CircuitBreakerResetTimeout = Duration.fromNanos(getNanoseconds("circuit-breaker.reset-timeout")) +} diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/PersistentQueue.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/PersistentQueue.scala index 1a5ddf4a8c..152b29406c 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/PersistentQueue.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/PersistentQueue.scala @@ -68,44 +68,44 @@ class PersistentQueue(persistencePath: String, val name: String, val settings: F def overlay[T](base: ⇒ T) = new OverlaySetting(base) // attempting to add an item after the queue reaches this size (in items) will fail. - val maxItems = overlay(PersistentQueue.maxItems) + final val maxItems = overlay(PersistentQueue.maxItems) // attempting to add an item after the queue reaches this size (in bytes) will fail. - val maxSize = overlay(PersistentQueue.maxSize) + final val maxSize = overlay(PersistentQueue.maxSize) // attempting to add an item larger than this size (in bytes) will fail. - val maxItemSize = overlay(PersistentQueue.maxItemSize) + final val maxItemSize = overlay(PersistentQueue.maxItemSize) // maximum expiration time for this queue (seconds). - val maxAge = overlay(PersistentQueue.maxAge) + final val maxAge = overlay(PersistentQueue.maxAge) // maximum journal size before the journal should be rotated. - val maxJournalSize = overlay(PersistentQueue.maxJournalSize) + final val maxJournalSize = overlay(PersistentQueue.maxJournalSize) // maximum size of a queue before it drops into read-behind mode. - val maxMemorySize = overlay(PersistentQueue.maxMemorySize) + final val maxMemorySize = overlay(PersistentQueue.maxMemorySize) // maximum overflow (multiplier) of a journal file before we re-create it. - val maxJournalOverflow = overlay(PersistentQueue.maxJournalOverflow) + final val maxJournalOverflow = overlay(PersistentQueue.maxJournalOverflow) // absolute maximum size of a journal file until we rebuild it, no matter what. - val maxJournalSizeAbsolute = overlay(PersistentQueue.maxJournalSizeAbsolute) + final val maxJournalSizeAbsolute = overlay(PersistentQueue.maxJournalSizeAbsolute) // whether to drop older items (instead of newer) when the queue is full - val discardOldWhenFull = overlay(PersistentQueue.discardOldWhenFull) + final val discardOldWhenFull = overlay(PersistentQueue.discardOldWhenFull) // whether to keep a journal file at all - val keepJournal = overlay(PersistentQueue.keepJournal) + final val keepJournal = overlay(PersistentQueue.keepJournal) // whether to sync the journal after each transaction - val syncJournal = overlay(PersistentQueue.syncJournal) + final val syncJournal = overlay(PersistentQueue.syncJournal) // (optional) move expired items over to this queue - val expiredQueue = overlay(PersistentQueue.expiredQueue) + final val expiredQueue = overlay(PersistentQueue.expiredQueue) private var journal = new Journal(new File(persistencePath, name).getCanonicalPath, syncJournal(), log) - // track tentative removals + // track tentative remofinal vals private var xidCounter: Int = 0 private val openTransactions = new mutable.HashMap[Int, QItem] def openTransactionCount = openTransactions.size diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/test/scala/akka/actor/mailbox/FileBasedMailboxSpec.scala b/akka-durable-mailboxes/akka-file-mailbox/src/test/scala/akka/actor/mailbox/FileBasedMailboxSpec.scala index be82e0fcb3..e3ad811b52 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/test/scala/akka/actor/mailbox/FileBasedMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/test/scala/akka/actor/mailbox/FileBasedMailboxSpec.scala @@ -1,7 +1,6 @@ package akka.actor.mailbox import org.apache.commons.io.FileUtils -import com.typesafe.config.ConfigFactory import akka.dispatch.Mailbox object FileBasedMailboxSpec { @@ -10,34 +9,41 @@ object FileBasedMailboxSpec { mailbox-type = akka.actor.mailbox.FileBasedMailboxType throughput = 1 file-based.directory-path = "file-based" + file-based.circuit-breaker.max-failures = 5 + file-based.circuit-breaker.call-timeout = 5 seconds } - """ + """ } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class FileBasedMailboxSpec extends DurableMailboxSpec("File", FileBasedMailboxSpec.config) { - val queuePath = new FileBasedMailboxSettings(system.settings, system.settings.config.getConfig("File-dispatcher")).QueuePath + val settings = new FileBasedMailboxSettings(system.settings, system.settings.config.getConfig("File-dispatcher")) "FileBasedMailboxSettings" must { "read the file-based section" in { - queuePath must be("file-based") + settings.QueuePath must be("file-based") + settings.CircuitBreakerMaxFailures must be(5) + + import akka.util.duration._ + + settings.CircuitBreakerCallTimeout must be(5 seconds) } } def isDurableMailbox(m: Mailbox): Boolean = m.messageQueue.isInstanceOf[FileBasedMessageQueue] - def clean { - FileUtils.deleteDirectory(new java.io.File(queuePath)) + def clean() { + FileUtils.deleteDirectory(new java.io.File(settings.QueuePath)) } override def atStartup() { - clean + clean() super.atStartup() } override def atTermination() { - clean + clean() super.atTermination() } } diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/protocol/MailboxProtocol.proto b/akka-durable-mailboxes/akka-mailboxes-common/src/main/protocol/MailboxProtocol.proto deleted file mode 100644 index 96fab2bf95..0000000000 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/protocol/MailboxProtocol.proto +++ /dev/null @@ -1,30 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -option java_package = "akka.actor.mailbox"; -option optimize_for = SPEED; - -/****************************************** - Compile with: - cd ./akka-durable-mailboxes/akka-mailboxes-common/src/main/protocol - protoc MailboxProtocol.proto --java_out ../java -*******************************************/ - -/** - * Defines the durable mailbox message. - */ -message DurableMailboxMessageProtocol { - required string ownerAddress = 1; - optional string senderAddress = 2; - optional UuidProtocol futureUuid = 3; - required bytes message = 4; -} - -/** - * Defines a UUID. - */ -message UuidProtocol { - required uint64 high = 1; - required uint64 low = 2; -} 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 41ec6d7307..79ece7625d 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 @@ -3,42 +3,54 @@ */ package akka.actor.mailbox -import akka.actor.{ ActorContext, ActorRef, ExtendedActorSystem } import akka.dispatch.{ Envelope, MessageQueue } import akka.remote.MessageSerializer import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol } import com.typesafe.config.Config -import akka.actor.ActorSystem +import akka.actor._ private[akka] object DurableExecutableMailboxConfig { val Name = "[\\.\\/\\$\\s]".r } -abstract class DurableMessageQueue(val owner: ActorContext) extends MessageQueue { +abstract class DurableMessageQueue(val owner: ActorRef, val system: ExtendedActorSystem) extends MessageQueue { import DurableExecutableMailboxConfig._ - def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem] - def ownerPath = owner.self.path - val ownerPathString = ownerPath.elements.mkString("/") - val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_") + def ownerPath: ActorPath = owner.path + val ownerPathString: String = ownerPath.elements.mkString("/") + val name: String = "mailbox_" + Name.replaceAllIn(ownerPathString, "_") } +/** + * DurableMessageSerialization can be mixed into a DurableMessageQueue and adds functionality + * to serialize and deserialize Envelopes (messages) + */ trait DurableMessageSerialization { this: DurableMessageQueue ⇒ + /** + * Serializes the given Envelope into an Array of Bytes using an efficient serialization/deserialization strategy + */ def serialize(durableMessage: Envelope): Array[Byte] = { + // It's alright to use ref.path.toString here + // When the sender is a LocalActorRef it should be local when deserialized also. + // When the sender is a RemoteActorRef the path.toString already contains remote address information. def serializeActorRef(ref: ActorRef): ActorRefProtocol = ActorRefProtocol.newBuilder.setPath(ref.path.toString).build val message = MessageSerializer.serialize(system, durableMessage.message.asInstanceOf[AnyRef]) val builder = RemoteMessageProtocol.newBuilder .setMessage(message) - .setRecipient(serializeActorRef(owner.self)) + .setRecipient(serializeActorRef(owner)) .setSender(serializeActorRef(durableMessage.sender)) builder.build.toByteArray } + /** + * Deserializes an array of Bytes that were serialized using the DurableMessageSerialization.serialize method, + * into an Envelope. + */ def deserialize(bytes: Array[Byte]): Envelope = { def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = system.actorFor(refProtocol.getPath) @@ -47,7 +59,7 @@ trait DurableMessageSerialization { this: DurableMessageQueue ⇒ val message = MessageSerializer.deserialize(system, durableMessage.getMessage) val sender = deserializeActorRef(durableMessage.getSender) - new Envelope(message, sender)(system) + Envelope(message, sender, system) } } @@ -56,11 +68,15 @@ trait DurableMessageSerialization { this: DurableMessageQueue ⇒ * Conventional organization of durable mailbox settings: * * {{{ - * my-durable-dispatcher { - * mailbox-type = "my.durable.mailbox" - * my-durable-mailbox { - * setting1 = 1 - * setting2 = 2 + * akka { + * actor { + * my-durable-dispatcher { + * mailbox-type = "my.durable.mailbox" + * my-durable-mailbox { + * setting1 = 1 + * setting2 = 2 + * } + * } * } * } * }}} 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 ff436c227e..8264bd0348 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 @@ -3,14 +3,21 @@ */ package akka.actor.mailbox -import akka.testkit.AkkaSpec -import akka.testkit.TestLatch -import akka.util.duration._ import java.io.InputStream +import java.util.concurrent.TimeoutException + import scala.annotation.tailrec -import com.typesafe.config.Config -import akka.actor._ -import akka.dispatch.{ Mailbox, Await } + +import org.scalatest.{ WordSpec, BeforeAndAfterAll } +import org.scalatest.matchers.MustMatchers + +import com.typesafe.config.{ ConfigFactory, Config } + +import DurableMailboxSpecActorFactory.{ MailboxTestActor, AccumulatorActor } +import akka.actor.{ RepointableRef, Props, ActorSystem, ActorRefWithCell, ActorRef, ActorCell, Actor } +import akka.dispatch.Mailbox +import akka.testkit.TestKit +import akka.util.duration.intToDurationInt object DurableMailboxSpecActorFactory { @@ -28,13 +35,62 @@ object DurableMailboxSpecActorFactory { } +object DurableMailboxSpec { + def fallbackConfig: Config = ConfigFactory.parseString(""" + akka { + event-handlers = ["akka.testkit.TestEventListener"] + loglevel = "WARNING" + stdout-loglevel = "WARNING" + } + """) +} + /** + * Reusable test fixture for durable mailboxes. Implements a few basic tests. More + * tests can be added in concrete subclass. + * * Subclass must define dispatcher in the supplied config for the specific backend. * The id of the dispatcher must be the same as the `-dispatcher`. */ -abstract class DurableMailboxSpec(val backendName: String, config: String) extends AkkaSpec(config) { +abstract class DurableMailboxSpec(system: ActorSystem, val backendName: String) + extends TestKit(system) with WordSpec with MustMatchers with BeforeAndAfterAll { + import DurableMailboxSpecActorFactory._ + /** + * Subclass must define dispatcher in the supplied config for the specific backend. + * The id of the dispatcher must be the same as the `-dispatcher`. + */ + def this(backendName: String, config: String) = { + this(ActorSystem(backendName + "BasedDurableMailboxSpec", + ConfigFactory.parseString(config).withFallback(DurableMailboxSpec.fallbackConfig)), + backendName) + } + + final override def beforeAll { + atStartup() + } + + /** + * May be implemented in concrete subclass to do additional things once before test + * cases are run. + */ + protected def atStartup() {} + + final override def afterAll { + system.shutdown() + try system.awaitTermination(5 seconds) catch { + case _: TimeoutException ⇒ system.log.warning("Failed to stop [{}] within 5 seconds", system.name) + } + atTermination() + } + + /** + * May be implemented in concrete subclass to do additional things once after all + * test cases have been run. + */ + def atTermination() {} + protected def streamMustContain(in: InputStream, words: String): Unit = { val output = new Array[Byte](8192) @@ -55,20 +111,29 @@ abstract class DurableMailboxSpec(val backendName: String, config: String) exten if (!result.contains(words)) throw new Exception("stream did not contain '" + words + "':\n" + result) } - def createMailboxTestActor(props: Props = Props[MailboxTestActor], id: String = ""): ActorRef = id match { - case null | "" ⇒ system.actorOf(props.withDispatcher(backendName + "-dispatcher")) - case some ⇒ system.actorOf(props.withDispatcher(backendName + "-dispatcher"), some) + def createMailboxTestActor(props: Props = Props[MailboxTestActor], id: String = ""): ActorRef = { + val ref = id match { + case null | "" ⇒ system.actorOf(props.withDispatcher(backendName + "-dispatcher")) + case some ⇒ system.actorOf(props.withDispatcher(backendName + "-dispatcher"), some) + } + awaitCond(ref match { + case r: RepointableRef ⇒ r.isStarted + }, 1 second, 10 millis) + ref } - def isDurableMailbox(m: Mailbox): Boolean + private def isDurableMailbox(m: Mailbox): Boolean = + m.messageQueue.isInstanceOf[DurableMessageQueue] "A " + backendName + " based mailbox backed actor" must { "get a new, unique, durable mailbox" in { val a1, a2 = createMailboxTestActor() - isDurableMailbox(a1.asInstanceOf[LocalActorRef].underlying.mailbox) must be(true) - isDurableMailbox(a2.asInstanceOf[LocalActorRef].underlying.mailbox) must be(true) - (a1.asInstanceOf[LocalActorRef].underlying.mailbox ne a2.asInstanceOf[LocalActorRef].underlying.mailbox) must be(true) + val mb1 = a1.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].mailbox + val mb2 = a2.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].mailbox + isDurableMailbox(mb1) must be(true) + isDurableMailbox(mb2) must be(true) + (mb1 ne mb2) must be(true) } "deliver messages at most once" in { @@ -87,7 +152,7 @@ abstract class DurableMailboxSpec(val backendName: String, config: String) exten "support having multiple actors at the same time" in { val actors = Vector.fill(3)(createMailboxTestActor(Props[AccumulatorActor])) - actors foreach { a ⇒ isDurableMailbox(a.asInstanceOf[LocalActorRef].underlying.mailbox) must be(true) } + actors foreach { a ⇒ isDurableMailbox(a.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].mailbox) must be(true) } val msgs = 1 to 3 diff --git a/akka-kernel/src/main/dist/bin/akka-cluster b/akka-kernel/src/main/dist/bin/akka-cluster index ecca52fa9b..fe3af38449 100755 --- a/akka-kernel/src/main/dist/bin/akka-cluster +++ b/akka-kernel/src/main/dist/bin/akka-cluster @@ -3,7 +3,6 @@ # ============== Akka Cluster Administration Tool ============== # # This script is meant to be used from within the Akka distribution. -# Requires setting $AKKA_HOME to the root of the distribution. # # Add these options to the sbt or startup script: # java \ @@ -15,9 +14,12 @@ # FIXME support authentication? if so add: -Dcom.sun.management.jmxremote.password.file= AND tweak this script to support it (arg need 'user:passwd' instead of '-') -# NOTE: The 'cmdline-jmxclient' JAR is available as part of the Akka distribution. -# Provided by Typesafe Maven Repository: http://repo.typesafe.com/typesafe/releases/cmdline-jmxclient. -JMX_CLIENT="java -jar $AKKA_HOME/lib/akka/cmdline-jmxclient-0.10.3.jar -" +declare AKKA_HOME="$(cd "$(cd "$(dirname "$0")"; pwd -P)"/..; pwd)" + +[ -n "$JMX_CLIENT_CLASSPATH" ] || JMX_CLIENT_CLASSPATH="$AKKA_HOME/lib/akka/akka-kernel-*" + +# NOTE: The 'cmdline-jmxclient' is available as part of the Akka distribution. +JMX_CLIENT="java -cp $JMX_CLIENT_CLASSPATH akka.jmx.Client -" SELF=`basename $0` # script name HOST=$1 # cluster node:port to talk to through JMX @@ -61,20 +63,6 @@ case "$2" in $JMX_CLIENT $HOST akka:type=Cluster leave=$ACTOR_SYSTEM_URL ;; - remove) - if [ $# -ne 3 ]; then - echo "Usage: $SELF remove " - exit 1 - fi - - ensureNodeIsRunningAndAvailable - shift - - ACTOR_SYSTEM_URL=$2 - echo "Scheduling $ACTOR_SYSTEM_URL to REMOVE" - $JMX_CLIENT $HOST akka:type=Cluster remove=$ACTOR_SYSTEM_URL - ;; - down) if [ $# -ne 3 ]; then echo "Usage: $SELF down " @@ -167,25 +155,38 @@ case "$2" in $JMX_CLIENT $HOST akka:type=Cluster Available ;; + is-running) + if [ $# -ne 2 ]; then + echo "Usage: $SELF is-running" + exit 1 + fi + + ensureNodeIsRunningAndAvailable + shift + + echo "Checking if member node on $HOST is AVAILABLE" + $JMX_CLIENT $HOST akka:type=Cluster Running + ;; + *) - printf "Usage: $SELF ...\n" + printf "Usage: bin/$SELF ...\n" printf "\n" printf "Supported commands are:\n" printf "%26s - %s\n" "join " "Sends request a JOIN node with the specified URL" printf "%26s - %s\n" "leave " "Sends a request for node with URL to LEAVE the cluster" - printf "%26s - %s\n" "remove " "Sends a request for node with URL to be instantly REMOVED from the cluster" printf "%26s - %s\n" "down " "Sends a request for marking node with URL as DOWN" printf "%26s - %s\n" member-status "Asks the member node for its current status" printf "%26s - %s\n" cluster-status "Asks the cluster for its current status (member ring, unavailable nodes, meta data etc.)" printf "%26s - %s\n" leader "Asks the cluster who the current leader is" printf "%26s - %s\n" is-singleton "Checks if the cluster is a singleton cluster (single node cluster)" printf "%26s - %s\n" is-available "Checks if the member node is available" + printf "%26s - %s\n" is-running "Checks if the member node is running" printf "%26s - %s\n" has-convergence "Checks if there is a cluster convergence" printf "Where the should be on the format of 'akka://actor-system-name@hostname:port'\n" printf "\n" - printf "Examples: $SELF localhost:9999 is-available\n" - printf " $SELF localhost:9999 join akka://MySystem@darkstar:2552\n" - printf " $SELF localhost:9999 cluster-status\n" + printf "Examples: bin/$SELF localhost:9999 is-available\n" + printf " bin/$SELF localhost:9999 join akka://MySystem@darkstar:2552\n" + printf " bin/$SELF localhost:9999 cluster-status\n" exit 1 ;; esac diff --git a/akka-kernel/src/main/java/akka/jmx/Client.java b/akka-kernel/src/main/java/akka/jmx/Client.java new file mode 100644 index 0000000000..83a8f9246f --- /dev/null +++ b/akka-kernel/src/main/java/akka/jmx/Client.java @@ -0,0 +1,777 @@ +/* + * Client + * + * $Id$ + * + * Created on Nov 12, 2004 + * + * Copyright (C) 2004 Internet Archive. + * + * This file is part of the Heritrix web crawler (crawler.archive.org). + * + * Heritrix is free software; you can redistribute it and/or modify + * it under the terms of the GNU Lesser Public License as published by + * the Free Software Foundation; either version 2.1 of the License, or + * any later version. + * + * Heritrix is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Lesser Public License for more details. + * + * You should have received a copy of the GNU Lesser Public License + * along with Heritrix; if not, write to the Free Software + * Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + */ +package akka.jmx; + +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.text.FieldPosition; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.logging.ConsoleHandler; +import java.util.logging.Handler; +import java.util.logging.LogRecord; +import java.util.logging.Logger; +import java.util.logging.SimpleFormatter; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import javax.management.Attribute; +import javax.management.AttributeList; +import javax.management.InstanceNotFoundException; +import javax.management.IntrospectionException; +import javax.management.MBeanAttributeInfo; +import javax.management.MBeanFeatureInfo; +import javax.management.MBeanInfo; +import javax.management.MBeanOperationInfo; +import javax.management.MBeanParameterInfo; +import javax.management.MBeanServerConnection; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectInstance; +import javax.management.ObjectName; +import javax.management.ReflectionException; +import javax.management.openmbean.CompositeData; +import javax.management.openmbean.TabularData; +import javax.management.remote.JMXConnector; +import javax.management.remote.JMXConnectorFactory; +import javax.management.remote.JMXServiceURL; + + +/** + * A Simple Command-Line JMX Client. + * Tested against the JDK 1.5.0 JMX Agent. + * See Monitoring + * and Management Using JMX. + *

Can supply credentials and do primitive string representation of tabular + * and composite openmbeans. + * @author stack + */ +public class Client { + private static final Logger logger = + Logger.getLogger(Client.class.getName()); + + /** + * Usage string. + */ + private static final String USAGE = "Usage: java -jar" + + " cmdline-jmxclient.jar USER:PASS HOST:PORT [BEAN] [COMMAND]\n" + + "Options:\n" + + " USER:PASS Username and password. Required. If none, pass '-'.\n" + + " E.g. 'controlRole:secret'\n" + + " HOST:PORT Hostname and port to connect to. Required." + + " E.g. localhost:8081.\n" + + " Lists registered beans if only USER:PASS and this" + + " argument.\n" + + " BEAN Optional target bean name. If present we list" + + " available operations\n" + + " and attributes.\n" + + " COMMAND Optional operation to run or attribute to fetch. If" + + " none supplied,\n" + + " all operations and attributes are listed. Attributes" + + " begin with a\n" + + " capital letter: e.g. 'Status' or 'Started'." + + " Operations do not.\n" + + " Operations can take arguments by adding an '=' " + + "followed by\n" + + " comma-delimited params. Pass multiple " + + "attributes/operations to run\n" + + " more than one per invocation. Use commands 'create' and " + + "'destroy'\n" + + " to instantiate and unregister beans ('create' takes name " + + "of class).\n" + + " Pass 'Attributes' to get listing of all attributes and " + + "and their\n" + + " values.\n" + + "Requirements:\n" + + " JDK1.5.0. If connecting to a SUN 1.5.0 JDK JMX Agent, remote side" + + " must be\n" + + " started with system properties such as the following:\n" + + " -Dcom.sun.management.jmxremote.port=PORT\n" + + " -Dcom.sun.management.jmxremote.authenticate=false\n" + + " -Dcom.sun.management.jmxremote.ssl=false\n" + + " The above will start the remote server with no password. See\n" + + " http://java.sun.com/j2se/1.5.0/docs/guide/management/agent.html" + + " for more on\n" + + " 'Monitoring and Management via JMX'.\n" + + "Client Use Examples:\n" + + " To list MBeans on a non-password protected remote agent:\n" + + " % java -jar cmdline-jmxclient-X.X.jar - localhost:8081 \\\n" + + " org.archive.crawler:name=Heritrix,type=Service\n" + + " To list attributes and attributes of the Heritrix MBean:\n" + + " % java -jar cmdline-jmxclient-X.X.jar - localhost:8081 \\\n" + + " org.archive.crawler:name=Heritrix,type=Service \\\n" + + " schedule=http://www.archive.org\n" + + " To set set logging level to FINE on a password protected JVM:\n" + + " % java -jar cmdline-jmxclient-X.X.jar controlRole:secret" + + " localhost:8081 \\\n" + + " java.util.logging:type=Logging \\\n" + + " setLoggerLevel=org.archive.crawler.Heritrix,FINE"; + + /** + * Pattern that matches a command name followed by + * an optional equals and optional comma-delimited list + * of arguments. + */ + protected static final Pattern CMD_LINE_ARGS_PATTERN = + Pattern.compile("^([^=]+)(?:(?:\\=)(.+))?$"); + + private static final String CREATE_CMD_PREFIX = "create="; + + public static void main(String[] args) throws Exception { + Client client = new Client(); + // Set the logger to use our all-on-one-line formatter. + Logger l = Logger.getLogger(""); + Handler [] hs = l.getHandlers(); + for (int i = 0; i < hs.length; i++) { + Handler h = hs[0]; + if (h instanceof ConsoleHandler) { + h.setFormatter(client.new OneLineSimpleLogger()); + } + } + client.execute(args); + } + + protected static void usage() { + usage(0, null); + } + + protected static void usage(int exitCode, String message) { + if (message != null && message.length() > 0) { + System.out.println(message); + } + System.out.println(USAGE); + System.exit(exitCode); + } + + /** + * Constructor. + */ + public Client() { + super(); + } + + /** + * Parse a 'login:password' string. Assumption is that no + * colon in the login name. + * @param userpass + * @return Array of strings with login in first position. + */ + protected String [] parseUserpass(final String userpass) { + if (userpass == null || userpass.equals("-")) { + return null; + } + int index = userpass.indexOf(':'); + if (index <= 0) { + throw new RuntimeException("Unable to parse: " +userpass); + } + return new String [] {userpass.substring(0, index), + userpass.substring(index + 1)}; + } + + /** + * @param login + * @param password + * @return Credentials as map for RMI. + */ + protected Map formatCredentials(final String login, + final String password) { + Map env = new HashMap(1); + env.put(JMXConnector.CREDENTIALS, new String[] {login, password}); + return env; + } + + protected JMXConnector getJMXConnector(final String hostport, + final String login, final String password) + throws IOException { + // Make up the jmx rmi URL and get a connector. + JMXServiceURL rmiurl = new JMXServiceURL("service:jmx:rmi://"+hostport+"/jndi/rmi://"+hostport+"/jmxrmi"); + return JMXConnectorFactory.connect(rmiurl,formatCredentials(login, password)); + } + + protected ObjectName getObjectName(final String beanname) + throws MalformedObjectNameException, NullPointerException { + return notEmpty(beanname)? new ObjectName(beanname): null; + } + + /** + * Version of execute called from the cmdline. + * Prints out result of execution on stdout. + * Parses cmdline args. Then calls {@link #execute(String, String, + * String, String, String[], boolean)}. + * @param args Cmdline args. + * @throws Exception + */ + protected void execute(final String [] args) + throws Exception { + // Process command-line. + if (args.length == 0 || args.length == 1) { + usage(); + } + String userpass = args[0]; + String hostport = args[1]; + String beanname = null; + String [] command = null; + if (args.length > 2) { + beanname = args[2]; + } + if (args.length > 3) { + command = new String [args.length - 3]; + for (int i = 3; i < args.length; i++) { + command[i - 3] = args[i]; + } + } + String [] loginPassword = parseUserpass(userpass); + Object [] result = execute(hostport, + ((loginPassword == null)? null: loginPassword[0]), + ((loginPassword == null)? null: loginPassword[1]), beanname, + command); + // Print out results on stdout. Only log if a result. + if (result != null) { + for (int i = 0; i < result.length; i++) { + if (result[i] != null && result[i].toString().length() > 0) { + if (command != null) { + logger.info(command[i] + ": " + result[i]); + } else { + logger.info("\n" + result[i].toString()); + } + } + } + } + } + + protected Object [] execute(final String hostport, final String login, + final String password, final String beanname, + final String [] command) + throws Exception { + return execute(hostport, login, password, beanname, command, false); + } + + public Object [] executeOneCmd(final String hostport, final String login, + final String password, final String beanname, + final String command) + throws Exception { + return execute(hostport, login, password, beanname, + new String[] {command}, true); + } + + /** + * Execute command against remote JMX agent. + * @param hostport 'host:port' combination. + * @param login RMI login to use. + * @param password RMI password to use. + * @param beanname Name of remote bean to run command against. + * @param command Array of commands to run. + * @param oneBeanOnly Set true if passed beanname is + * an exact name and the query for a bean is only supposed to return + * one bean instance. If not, we raise an exception (Otherwise, if false, + * then we deal with possibility of multiple bean instances coming back + * from query). Set to true when want to get an attribute or run an + * operation. + * @return Array of results -- one per command. + * @throws Exception + */ + protected Object [] execute(final String hostport, final String login, + final String password, final String beanname, + final String [] command, final boolean oneBeanOnly) + throws Exception { + JMXConnector jmxc = getJMXConnector(hostport, login, password); + Object [] result = null; + try { + result = doBeans(jmxc.getMBeanServerConnection(), + getObjectName(beanname), command, oneBeanOnly); + } finally { + jmxc.close(); + } + return result; + } + + protected boolean notEmpty(String s) { + return s != null && s.length() > 0; + } + + protected Object [] doBeans(final MBeanServerConnection mbsc, + final ObjectName objName, final String[] command, + final boolean oneBeanOnly) + throws Exception { + Object [] result = null; + Set beans = mbsc.queryMBeans(objName, null); + if (beans.size() == 0) { + // No bean found. Check if we are to create a bean? + if (command.length == 1 && notEmpty(command[0]) + && command[0].startsWith(CREATE_CMD_PREFIX)) { + String className = + command[0].substring(CREATE_CMD_PREFIX.length()); + mbsc.createMBean(className, objName); + } else { + // TODO: Is there a better JMX exception that RE for this + // scenario? + throw new RuntimeException(objName.getCanonicalName() + + " not registered."); + } + } else if (beans.size() == 1) { + result = doBean(mbsc, (ObjectInstance) beans.iterator().next(), + command); + } else { + if (oneBeanOnly) { + throw new RuntimeException("Only supposed to be one bean " + + "query result"); + } + // This is case of multiple beans in query results. + // Print name of each into a StringBuffer. Return as one + // result. + StringBuffer buffer = new StringBuffer(); + for (Iterator i = beans.iterator(); i.hasNext();) { + Object obj = i.next(); + if (obj instanceof ObjectName) { + buffer.append((((ObjectName) obj).getCanonicalName())); + } else if (obj instanceof ObjectInstance) { + buffer.append((((ObjectInstance) obj).getObjectName() + .getCanonicalName())); + } else { + throw new RuntimeException("Unexpected object type: " + obj); + } + buffer.append("\n"); + } + result = new String [] {buffer.toString()}; + } + return result; + } + + /** + * Get attribute or run operation against passed bean instance. + * + * @param mbsc Server connection. + * @param instance Bean instance we're to get attributes from or run + * operation against. + * @param command Command to run (May be null). + * @return Result. If multiple commands, multiple results. + * @throws Exception + */ + protected Object [] doBean(MBeanServerConnection mbsc, + ObjectInstance instance, String [] command) + throws Exception { + // If no command, then print out list of attributes and operations. + if (command == null || command.length <= 0) { + return new String [] {listOptions(mbsc, instance)}; + } + + // Maybe multiple attributes/operations listed on one command line. + Object [] result = new Object[command.length]; + for (int i = 0; i < command.length; i++) { + result[i] = doSubCommand(mbsc, instance, command[i]); + } + return result; + } + + public Object doSubCommand(MBeanServerConnection mbsc, + ObjectInstance instance, String subCommand) + throws Exception { + // First, handle special case of our being asked to destroy a bean. + if (subCommand.equals("destroy")) { + mbsc.unregisterMBean(instance.getObjectName()); + return null; + } else if (subCommand.startsWith(CREATE_CMD_PREFIX)) { + throw new IllegalArgumentException("You cannot call create " + + "on an already existing bean."); + } + + // Get attribute and operation info. + MBeanAttributeInfo [] attributeInfo = + mbsc.getMBeanInfo(instance.getObjectName()).getAttributes(); + MBeanOperationInfo [] operationInfo = + mbsc.getMBeanInfo(instance.getObjectName()).getOperations(); + // Now, bdbje JMX bean doesn't follow the convention of attributes + // having uppercase first letter and operations having lowercase + // first letter. But most beans do. Be prepared to handle the bdbje + // case. + Object result = null; + if (Character.isUpperCase(subCommand.charAt(0))) { + // Probably an attribute. + if (!isFeatureInfo(attributeInfo, subCommand) && + isFeatureInfo(operationInfo, subCommand)) { + // Its not an attribute name. Looks like its name of an + // operation. Try it. + result = + doBeanOperation(mbsc, instance, subCommand, operationInfo); + } else { + // Then it is an attribute OR its not an attribute name nor + // operation name and the below invocation will throw a + // AttributeNotFoundException. + result = doAttributeOperation(mbsc, instance, subCommand, + attributeInfo); + } + } else { + // Must be an operation. + if (!isFeatureInfo(operationInfo, subCommand) && + isFeatureInfo(attributeInfo, subCommand)) { + // Its not an operation name but looks like it could be an + // attribute name. Try it. + result = doAttributeOperation(mbsc, instance, subCommand, + attributeInfo); + } else { + // Its an operation name OR its neither operation nor attribute + // name and the below will throw a NoSuchMethodException. + result = + doBeanOperation(mbsc, instance, subCommand, operationInfo); + } + } + + // Look at the result. Is it of composite or tabular type? + // If so, convert to a String representation. + if (result instanceof CompositeData) { + result = recurseCompositeData(new StringBuffer("\n"), "", "", + (CompositeData)result); + } else if (result instanceof TabularData) { + result = recurseTabularData(new StringBuffer("\n"), "", "", + (TabularData)result); + } else if (result instanceof String []) { + String [] strs = (String [])result; + StringBuffer buffer = new StringBuffer("\n"); + for (int i = 0; i < strs.length; i++) { + buffer.append(strs[i]); + buffer.append("\n"); + } + result = buffer; + } else if (result instanceof AttributeList) { + AttributeList list = (AttributeList)result; + if (list.size() <= 0) { + result = null; + } else { + StringBuffer buffer = new StringBuffer("\n"); + for (Iterator ii = list.iterator(); ii.hasNext();) { + Attribute a = (Attribute)ii.next(); + buffer.append(a.getName()); + buffer.append(": "); + buffer.append(a.getValue()); + buffer.append("\n"); + } + result = buffer; + } + } + return result; + } + + protected boolean isFeatureInfo(MBeanFeatureInfo [] infos, String cmd) { + return getFeatureInfo(infos, cmd) != null; + } + + protected MBeanFeatureInfo getFeatureInfo(MBeanFeatureInfo [] infos, + String cmd) { + // Cmd may be carrying arguments. Don't count them in the compare. + int index = cmd.indexOf('='); + String name = (index > 0)? cmd.substring(0, index): cmd; + for (int i = 0; i < infos.length; i++) { + if (infos[i].getName().equals(name)) { + return infos[i]; + } + } + return null; + } + + protected StringBuffer recurseTabularData(StringBuffer buffer, + String indent, String name, TabularData data) { + addNameToBuffer(buffer, indent, name); + java.util.Collection c = data.values(); + for (Iterator i = c.iterator(); i.hasNext();) { + Object obj = i.next(); + if (obj instanceof CompositeData) { + recurseCompositeData(buffer, indent + " ", "", + (CompositeData)obj); + } else if (obj instanceof TabularData) { + recurseTabularData(buffer, indent, "", + (TabularData)obj); + } else { + buffer.append(obj); + } + } + return buffer; + } + + protected StringBuffer recurseCompositeData(StringBuffer buffer, + String indent, String name, CompositeData data) { + indent = addNameToBuffer(buffer, indent, name); + for (Iterator i = data.getCompositeType().keySet().iterator(); + i.hasNext();) { + String key = (String)i.next(); + Object o = data.get(key); + if (o instanceof CompositeData) { + recurseCompositeData(buffer, indent + " ", key, + (CompositeData)o); + } else if (o instanceof TabularData) { + recurseTabularData(buffer, indent, key, (TabularData)o); + } else { + buffer.append(indent); + buffer.append(key); + buffer.append(": "); + buffer.append(o); + buffer.append("\n"); + } + } + return buffer; + } + + protected String addNameToBuffer(StringBuffer buffer, String indent, + String name) { + if (name == null || name.length() == 0) { + return indent; + } + buffer.append(indent); + buffer.append(name); + buffer.append(":\n"); + // Move all that comes under this 'name' over by one space. + return indent + " "; + } + + /** + * Class that parses commandline arguments. + * Expected format is 'operationName=arg0,arg1,arg2...'. We are assuming no + * spaces nor comma's in argument values. + */ + protected class CommandParse { + private String cmd; + private String [] args; + + protected CommandParse(String command) throws ParseException { + parse(command); + } + + private void parse(String command) throws ParseException { + Matcher m = CMD_LINE_ARGS_PATTERN.matcher(command); + if (m == null || !m.matches()) { + throw new ParseException("Failed parse of " + command, 0); + } + + this.cmd = m.group(1); + if (m.group(2) != null && m.group(2).length() > 0) { + this.args = m.group(2).split(","); + } else { + this.args = null; + } + } + + protected String getCmd() { + return this.cmd; + } + + protected String [] getArgs() { + return this.args; + } + } + + protected Object doAttributeOperation(MBeanServerConnection mbsc, + ObjectInstance instance, String command, MBeanAttributeInfo [] infos) + throws Exception { + // Usually we get attributes. If an argument, then we're being asked + // to set attribute. + CommandParse parse = new CommandParse(command); + if (parse.getArgs() == null || parse.getArgs().length == 0) { + // Special-casing. If the subCommand is 'Attributes', then return + // list of all attributes. + if (command.equals("Attributes")) { + String [] names = new String[infos.length]; + for (int i = 0; i < infos.length; i++) { + names[i] = infos[i].getName(); + } + return mbsc.getAttributes(instance.getObjectName(), names); + } + return mbsc.getAttribute(instance.getObjectName(), parse.getCmd()); + } + if (parse.getArgs().length != 1) { + throw new IllegalArgumentException("One only argument setting " + + "attribute values: " + parse.getArgs()); + } + // Get first attribute of name 'cmd'. Assumption is no method + // overrides. Then, look at the attribute and use its type. + MBeanAttributeInfo info = + (MBeanAttributeInfo)getFeatureInfo(infos, parse.getCmd()); + java.lang.reflect.Constructor c = Class.forName( + info.getType()).getConstructor(new Class[] {String.class}); + Attribute a = new Attribute(parse.getCmd(), + c.newInstance(new Object[] {parse.getArgs()[0]})); + mbsc.setAttribute(instance.getObjectName(), a); + return null; + } + + protected Object doBeanOperation(MBeanServerConnection mbsc, + ObjectInstance instance, String command, MBeanOperationInfo [] infos) + throws Exception { + // Parse command line. + CommandParse parse = new CommandParse(command); + + // Get first method of name 'cmd'. Assumption is no method + // overrides. Then, look at the method and use its signature + // to make sure client sends over parameters of the correct type. + MBeanOperationInfo op = + (MBeanOperationInfo)getFeatureInfo(infos, parse.getCmd()); + Object result = null; + if (op == null) { + result = "Operation " + parse.getCmd() + " not found."; + } else { + MBeanParameterInfo [] paraminfos = op.getSignature(); + int paraminfosLength = (paraminfos == null)? 0: paraminfos.length; + int objsLength = (parse.getArgs() == null)? + 0: parse.getArgs().length; + if (paraminfosLength != objsLength) { + result = "Passed param count does not match signature count"; + } else { + String [] signature = new String[paraminfosLength]; + Object [] params = (paraminfosLength == 0)? null + : new Object[paraminfosLength]; + for (int i = 0; i < paraminfosLength; i++) { + MBeanParameterInfo paraminfo = paraminfos[i]; + java.lang.reflect.Constructor c = Class.forName( + paraminfo.getType()).getConstructor( + new Class[] {String.class}); + params[i] = + c.newInstance(new Object[] {parse.getArgs()[i]}); + signature[i] = paraminfo.getType(); + } + result = mbsc.invoke(instance.getObjectName(), parse.getCmd(), + params, signature); + } + } + return result; + } + + protected String listOptions(MBeanServerConnection mbsc, + ObjectInstance instance) + throws InstanceNotFoundException, IntrospectionException, + ReflectionException, IOException { + StringBuffer result = new StringBuffer(); + MBeanInfo info = mbsc.getMBeanInfo(instance.getObjectName()); + MBeanAttributeInfo [] attributes = info.getAttributes(); + if (attributes.length > 0) { + result.append("Attributes:"); + result.append("\n"); + for (int i = 0; i < attributes.length; i++) { + result.append(' ' + attributes[i].getName() + + ": " + attributes[i].getDescription() + + " (type=" + attributes[i].getType() + + ")"); + result.append("\n"); + } + } + MBeanOperationInfo [] operations = info.getOperations(); + if (operations.length > 0) { + result.append("Operations:"); + result.append("\n"); + for (int i = 0; i < operations.length; i++) { + MBeanParameterInfo [] params = operations[i].getSignature(); + StringBuffer paramsStrBuffer = new StringBuffer(); + if (params != null) { + for (int j = 0; j < params.length; j++) { + paramsStrBuffer.append("\n name="); + paramsStrBuffer.append(params[j].getName()); + paramsStrBuffer.append(" type="); + paramsStrBuffer.append(params[j].getType()); + paramsStrBuffer.append(" "); + paramsStrBuffer.append(params[j].getDescription()); + } + } + result.append(' ' + operations[i].getName() + + ": " + operations[i].getDescription() + + "\n Parameters " + params.length + + ", return type=" + operations[i].getReturnType() + + paramsStrBuffer.toString()); + result.append("\n"); + } + } + return result.toString(); + } + + /** + * Logger that writes entry on one line with less verbose date. + * Modelled on the OneLineSimpleLogger from Heritrix. + * + * @author stack + * @version $Revision$, $Date$ + */ + private class OneLineSimpleLogger extends SimpleFormatter { + /** + * Date instance. + * + * Keep around instance of date. + */ + private Date date = new Date(); + + /** + * Field position instance. + * + * Keep around this instance. + */ + private FieldPosition position = new FieldPosition(0); + + /** + * MessageFormatter for date. + */ + private SimpleDateFormat formatter = + new SimpleDateFormat("MM/dd/yyyy HH:mm:ss Z"); + + /** + * Persistent buffer in which we conjure the log. + */ + private StringBuffer buffer = new StringBuffer(); + + + public OneLineSimpleLogger() { + super(); + } + + public synchronized String format(LogRecord record) { + this.buffer.setLength(0); + this.date.setTime(record.getMillis()); + this.position.setBeginIndex(0); + this.formatter.format(this.date, this.buffer, this.position); + this.buffer.append(' '); + if (record.getSourceClassName() != null) { + this.buffer.append(record.getSourceClassName()); + } else { + this.buffer.append(record.getLoggerName()); + } + this.buffer.append(' '); + this.buffer.append(formatMessage(record)); + this.buffer.append(System.getProperty("line.separator")); + if (record.getThrown() != null) { + try { + StringWriter writer = new StringWriter(); + PrintWriter printer = new PrintWriter(writer); + record.getThrown().printStackTrace(printer); + writer.close(); + this.buffer.append(writer.toString()); + } catch (Exception e) { + this.buffer.append("Failed to get stack trace: " + + e.getMessage()); + } + } + return this.buffer.toString(); + } + } +} \ No newline at end of file diff --git a/akka-kernel/src/main/scala/akka/kernel/Main.scala b/akka-kernel/src/main/scala/akka/kernel/Main.scala index ead2c28121..97ff625ab8 100644 --- a/akka-kernel/src/main/scala/akka/kernel/Main.scala +++ b/akka-kernel/src/main/scala/akka/kernel/Main.scala @@ -59,9 +59,9 @@ trait Bootable { * Main class for running the microkernel. */ object Main { - val quiet = getBoolean("akka.kernel.quiet") + private val quiet = getBoolean("akka.kernel.quiet") - def log(s: String) = if (!quiet) println(s) + private def log(s: String) = if (!quiet) println(s) def main(args: Array[String]) = { if (args.isEmpty) { @@ -90,7 +90,7 @@ object Main { log("Successfully started Akka") } - def createClassLoader(): ClassLoader = { + private def createClassLoader(): ClassLoader = { if (ActorSystem.GlobalHome.isDefined) { val home = ActorSystem.GlobalHome.get val deploy = new File(home, "deploy") @@ -106,7 +106,7 @@ object Main { } } - def loadDeployJars(deploy: File): ClassLoader = { + private def loadDeployJars(deploy: File): ClassLoader = { val jars = deploy.listFiles.filter(_.getName.endsWith(".jar")) val nestedJars = jars flatMap { jar ⇒ @@ -122,7 +122,7 @@ object Main { new URLClassLoader(urls, Thread.currentThread.getContextClassLoader) } - def addShutdownHook(bootables: Seq[Bootable]): Unit = { + private def addShutdownHook(bootables: Seq[Bootable]): Unit = { Runtime.getRuntime.addShutdownHook(new Thread(new Runnable { def run = { log("") @@ -138,7 +138,7 @@ object Main { })) } - def banner = """ + private def banner = """ ============================================================================== ZZ: diff --git a/akka-osgi-aries/src/main/resources/OSGI-INF/blueprint/akka-namespacehandler.xml b/akka-osgi-aries/src/main/resources/OSGI-INF/blueprint/akka-namespacehandler.xml new file mode 100644 index 0000000000..99492bedf2 --- /dev/null +++ b/akka-osgi-aries/src/main/resources/OSGI-INF/blueprint/akka-namespacehandler.xml @@ -0,0 +1,18 @@ + + + + + + + + + http://akka.io/xmlns/blueprint/v1.0.0 + + + + + + + diff --git a/akka-osgi-aries/src/main/resources/akka/osgi/aries/blueprint/akka.xsd b/akka-osgi-aries/src/main/resources/akka/osgi/aries/blueprint/akka.xsd new file mode 100644 index 0000000000..d7d0f77a2c --- /dev/null +++ b/akka-osgi-aries/src/main/resources/akka/osgi/aries/blueprint/akka.xsd @@ -0,0 +1,42 @@ + + + + + + + + Defines the configuration elements for setting up Akka with Blueprint + + + + + + + + Defines an Akka ActorSystem + + + + + + + + + + + + + + + + Defines an Akka ActorSystem configuration + + + + + diff --git a/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/BlueprintActorSystemFactory.scala b/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/BlueprintActorSystemFactory.scala new file mode 100644 index 0000000000..40c9d7367b --- /dev/null +++ b/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/BlueprintActorSystemFactory.scala @@ -0,0 +1,44 @@ +package akka.osgi.aries.blueprint + +import org.osgi.framework.BundleContext +import akka.osgi.OsgiActorSystemFactory +import com.typesafe.config.ConfigFactory + +/** + * A set of helper/factory classes to build a Akka system using Blueprint. This class is only meant to be used by + * the [[akka.osgi.aries.blueprint.NamespaceHandler]] class, you should not use this class directly. + * + * If you're looking for a way to set up Akka using Blueprint without the namespace handler, you should use + * [[akka.osgi.OsgiActorSystemFactory]] instead. + */ +class BlueprintActorSystemFactory(context: BundleContext, name: String) extends OsgiActorSystemFactory(context) { + + var config: Option[String] = None + + lazy val system = super.createActorSystem(stringToOption(name)) + + def setConfig(config: String) = { this.config = Some(config) } + + def create = system + + def destroy = system.shutdown() + + def stringToOption(original: String) = if (original == null || original.isEmpty) { + None + } else { + Some(original) + } + + /** + * Strategy method to create the Config for the ActorSystem, ensuring that the default/reference configuration is + * loaded from the akka-actor bundle. + */ + override def actorSystemConfig(context: BundleContext) = { + config match { + case Some(value) ⇒ ConfigFactory.parseString(value).withFallback(super.actorSystemConfig(context)) + case None ⇒ super.actorSystemConfig(context) + } + + } +} + diff --git a/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/NamespaceHandler.scala b/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/NamespaceHandler.scala new file mode 100644 index 0000000000..0570a027b6 --- /dev/null +++ b/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/NamespaceHandler.scala @@ -0,0 +1,148 @@ +package akka.osgi.aries.blueprint + +import org.apache.aries.blueprint.ParserContext +import org.osgi.service.blueprint.container.ComponentDefinitionException +import org.apache.aries.blueprint.mutable.MutableBeanMetadata + +import collection.JavaConversions.setAsJavaSet +import org.osgi.framework.BundleContext +import org.apache.aries.blueprint.reflect.{ ValueMetadataImpl, RefMetadataImpl, BeanArgumentImpl } +import org.w3c.dom.{ Element, Node } +import org.osgi.service.blueprint.reflect.{ BeanMetadata, ComponentMetadata } +import akka.actor.ActorSystem +import java.util.concurrent.atomic.AtomicInteger + +import ParserHelper.childElements + +/** + * Aries Blueprint namespace handler implementation. This namespace handler will allow users of Apache Aries' Blueprint + * implementation to define their Akka [[akka.actor.ActorSystem]] using a syntax like this: + * + * {{{ + * + * + * + * + * + * some.config { + * key=value + * } + * + * + * + * + * }}} + * + * Users of other IoC frameworks in an OSGi environment should use [[akka.osgi.OsgiActorSystemFactory]] instead. + */ +class NamespaceHandler extends org.apache.aries.blueprint.NamespaceHandler { + + import NamespaceHandler._ + + val idCounter = new AtomicInteger(0) + + def getSchemaLocation(namespace: String) = getClass().getResource("akka.xsd") + + def getManagedClasses = setAsJavaSet(Set(classOf[BlueprintActorSystemFactory])) + + def parse(element: Element, context: ParserContext) = element.getLocalName match { + case ACTORSYSTEM_ELEMENT_NAME ⇒ parseActorSystem(element, context) + case _ ⇒ throw new ComponentDefinitionException("Unexpected element for Akka namespace: %s".format(element)) + } + + def decorate(node: Node, component: ComponentMetadata, context: ParserContext) = + throw new ComponentDefinitionException("Bad xml syntax: node decoration is not supported") + + /* + * Parse + */ + def parseActorSystem(element: Element, context: ParserContext) = { + val factory = createFactoryBean(context, element.getAttribute(NAME_ATTRIBUTE)) + + for (child ← childElements(element)) { + child.getLocalName match { + case CONFIG_ELEMENT_NAME ⇒ parseConfig(child, context, factory) + case _ ⇒ throw new ComponentDefinitionException("Unexpected child element %s found in %s".format(child, element)) + } + } + + createActorSystemBean(context, element, factory) + } + + /* + * Parse + */ + def parseConfig(node: Element, context: ParserContext, factory: MutableBeanMetadata) = { + factory.addProperty("config", new ValueMetadataImpl(node.getTextContent)) + } + + /* + * Create the bean definition for the ActorSystem + */ + def createActorSystemBean(context: ParserContext, element: Element, factory: MutableBeanMetadata): MutableBeanMetadata = { + val system = context.createMetadata(classOf[MutableBeanMetadata]) + system.setId(getId(context, element)) + system.setFactoryComponent(factory) + + system.setFactoryMethod(FACTORY_METHOD_NAME) + system.setRuntimeClass(classOf[ActorSystem]) + system + } + + /* + * Create the bean definition for the BlueprintActorSystemFactory + */ + def createFactoryBean(context: ParserContext, name: String): MutableBeanMetadata = { + val factory = context.createMetadata(classOf[MutableBeanMetadata]) + factory.setId(findAvailableId(context)) + factory.setScope(BeanMetadata.SCOPE_SINGLETON) + factory.setProcessor(true) + factory.setRuntimeClass(classOf[BlueprintActorSystemFactory]) + + factory.setDestroyMethod(DESTROY_METHOD_NAME) + + factory.addArgument(new BeanArgumentImpl(new RefMetadataImpl(BUNDLE_CONTEXT_REFID), classOf[BundleContext].getName, -1)) + factory.addArgument(new BeanArgumentImpl(new ValueMetadataImpl(name), classOf[String].getName, -1)) + factory.setProcessor(true) + context.getComponentDefinitionRegistry.registerComponentDefinition(factory) + factory + } + + /* + * Get the assigned id or generate a suitable id + */ + def getId(context: ParserContext, element: Element) = { + if (element.hasAttribute(ID_ATTRIBUTE)) { + element.getAttribute(ID_ATTRIBUTE) + } else { + findAvailableId(context) + } + } + + /* + * Find the next available component id + */ + def findAvailableId(context: ParserContext): String = { + val id = ".akka-" + idCounter.incrementAndGet() + if (context.getComponentDefinitionRegistry.containsComponentDefinition(id)) { + // id already exists, let's try the next one + findAvailableId(context) + } else id + } +} + +object NamespaceHandler { + + private val ID_ATTRIBUTE = "id" + private val NAME_ATTRIBUTE = "name" + + private val BUNDLE_CONTEXT_REFID = "blueprintBundleContext" + + private val ACTORSYSTEM_ELEMENT_NAME = "actor-system" + private val CONFIG_ELEMENT_NAME = "config" + + private val DESTROY_METHOD_NAME = "destroy" + private val FACTORY_METHOD_NAME = "create" + +} diff --git a/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/ParserHelper.scala b/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/ParserHelper.scala new file mode 100644 index 0000000000..585037db09 --- /dev/null +++ b/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/ParserHelper.scala @@ -0,0 +1,17 @@ +package akka.osgi.aries.blueprint + +import org.w3c.dom.{ Node, Element } + +/** + * Helper class to deal with the W3C DOM types + */ +object ParserHelper { + + def childElements(element: Element): Seq[Element] = + children(element).filter(_.getNodeType == Node.ELEMENT_NODE).asInstanceOf[Seq[Element]] + + private[this] def children(element: Element): Seq[Node] = { + val nodelist = element.getChildNodes + for (index ← 0 until nodelist.getLength) yield nodelist.item(index) + } +} diff --git a/akka-osgi-aries/src/test/resources/akka/osgi/aries/blueprint/config.xml b/akka-osgi-aries/src/test/resources/akka/osgi/aries/blueprint/config.xml new file mode 100644 index 0000000000..ce9f48c551 --- /dev/null +++ b/akka-osgi-aries/src/test/resources/akka/osgi/aries/blueprint/config.xml @@ -0,0 +1,15 @@ + + + + + + + + some.config { + key=value + } + + + + diff --git a/akka-osgi-aries/src/test/resources/akka/osgi/aries/blueprint/injection.xml b/akka-osgi-aries/src/test/resources/akka/osgi/aries/blueprint/injection.xml new file mode 100644 index 0000000000..6fd21db5ef --- /dev/null +++ b/akka-osgi-aries/src/test/resources/akka/osgi/aries/blueprint/injection.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + diff --git a/akka-osgi-aries/src/test/resources/akka/osgi/aries/blueprint/simple.xml b/akka-osgi-aries/src/test/resources/akka/osgi/aries/blueprint/simple.xml new file mode 100644 index 0000000000..2ac6552f80 --- /dev/null +++ b/akka-osgi-aries/src/test/resources/akka/osgi/aries/blueprint/simple.xml @@ -0,0 +1,9 @@ + + + + + + + + diff --git a/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/ActorSystemAwareBean.scala b/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/ActorSystemAwareBean.scala new file mode 100644 index 0000000000..6e4bac39dd --- /dev/null +++ b/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/ActorSystemAwareBean.scala @@ -0,0 +1,11 @@ +package akka.osgi.aries.blueprint + +import akka.actor.ActorSystem + +/** + * Just a simple POJO that can contain an actor system. + * Used for testing dependency injection with Blueprint + */ +class ActorSystemAwareBean(val system: ActorSystem) { + +} diff --git a/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/NamespaceHandlerTest.scala b/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/NamespaceHandlerTest.scala new file mode 100644 index 0000000000..3bc32c6141 --- /dev/null +++ b/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/NamespaceHandlerTest.scala @@ -0,0 +1,100 @@ +package akka.osgi.aries.blueprint + +import org.scalatest.WordSpec +import akka.actor.ActorSystem +import de.kalpatec.pojosr.framework.launch.BundleDescriptor +import akka.osgi.PojoSRTestSupport +import akka.osgi.PojoSRTestSupport.bundle +import org.scalatest.matchers.MustMatchers + +/** + * Test cases for {@link ActorSystemActivator} + */ +object NamespaceHandlerTest { + + /* + * Bundle-SymbolicName to easily find our test bundle + */ + val TEST_BUNDLE_NAME = "akka.osgi.test.aries.namespace" + + /* + * Bundle descriptor representing the akka-osgi bundle itself + */ + val AKKA_OSGI_BLUEPRINT = + bundle("akka-osgi").withBlueprintFile(getClass.getResource("/OSGI-INF/blueprint/akka-namespacehandler.xml")) + +} + +class SimpleNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRTestSupport { + + import NamespaceHandlerTest._ + + val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq( + AKKA_OSGI_BLUEPRINT, + bundle(TEST_BUNDLE_NAME).withBlueprintFile(getClass.getResource("simple.xml")))) + + "simple.xml" must { + "set up ActorSystem when bundle starts" in { + serviceForType[ActorSystem] must not be (null) + } + + "stop the ActorSystem when bundle stops" in { + val system = serviceForType[ActorSystem] + system.isTerminated must be(false) + + bundleForName(TEST_BUNDLE_NAME).stop() + + system.awaitTermination() + system.isTerminated must be(true) + } + } + +} + +class ConfigNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRTestSupport { + + import NamespaceHandlerTest._ + + val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq( + AKKA_OSGI_BLUEPRINT, + bundle(TEST_BUNDLE_NAME).withBlueprintFile(getClass.getResource("config.xml")))) + + "config.xml" must { + "set up ActorSystem when bundle starts" in { + val system = serviceForType[ActorSystem] + system must not be (null) + + system.settings.config.getString("some.config.key") must be("value") + } + + "stop the ActorSystem when bundle stops" in { + val system = serviceForType[ActorSystem] + system.isTerminated must be(false) + + bundleForName(TEST_BUNDLE_NAME).stop() + + system.awaitTermination() + system.isTerminated must be(true) + } + } + +} + +class DependencyInjectionNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRTestSupport { + + import NamespaceHandlerTest._ + + val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq( + AKKA_OSGI_BLUEPRINT, + bundle(TEST_BUNDLE_NAME).withBlueprintFile(getClass.getResource("injection.xml")))) + + "injection.xml" must { + + "set up bean containing ActorSystem" in { + val bean = serviceForType[ActorSystemAwareBean] + bean must not be (null) + bean.system must not be (null) + } + } + +} diff --git a/akka-osgi/src/main/scala/akka/osgi/ActorSystemActivator.scala b/akka-osgi/src/main/scala/akka/osgi/ActorSystemActivator.scala new file mode 100644 index 0000000000..e279247dbc --- /dev/null +++ b/akka-osgi/src/main/scala/akka/osgi/ActorSystemActivator.scala @@ -0,0 +1,74 @@ +package akka.osgi + +import akka.actor.ActorSystem +import java.util.{ Dictionary, Properties } +import org.osgi.framework.{ ServiceRegistration, BundleContext, BundleActivator } + +/** + * Abstract bundle activator implementation to bootstrap and configure an actor system in an + * OSGi environment. It also provides a convenience method to register the actor system in + * the OSGi Service Registry for sharing it with other OSGi bundles. + * + * This convenience activator is mainly useful for setting up a single [[akka.actor.ActorSystem]] instance and sharing that + * with other bundles in the OSGi Framework. If you want to set up multiple systems in the same bundle context, look at + * the [[akka.osgi.OsgiActorSystemFactory]] instead. + */ +abstract class ActorSystemActivator extends BundleActivator { + + private var system: Option[ActorSystem] = None + private var registration: Option[ServiceRegistration] = None + + /** + * Implement this method to add your own actors to the ActorSystem. If you want to share the actor + * system with other bundles, call the `registerService(BundleContext, ActorSystem)` method from within + * this method. + * + * @param context the bundle context + * @param system the ActorSystem that was created by the activator + */ + def configure(context: BundleContext, system: ActorSystem): Unit + + /** + * Sets up a new ActorSystem + * + * @param context the BundleContext + */ + def start(context: BundleContext): Unit = { + system = Some(OsgiActorSystemFactory(context).createActorSystem(Option(getActorSystemName(context)))) + system foreach (configure(context, _)) + } + + /** + * Shuts down the ActorSystem when the bundle is stopped and, if necessary, unregisters a service registration. + * + * @param context the BundleContext + */ + def stop(context: BundleContext): Unit = { + registration foreach (_.unregister()) + system foreach (_.shutdown()) + } + + /** + * Register the actor system in the OSGi service registry. The activator itself will ensure that this service + * is unregistered again when the bundle is being stopped. + * + * @param context the bundle context + * @param system the actor system + */ + def registerService(context: BundleContext, system: ActorSystem): Unit = { + val properties = new Properties() + properties.put("name", system.name) + registration = Some(context.registerService(classOf[ActorSystem].getName, system, + properties.asInstanceOf[Dictionary[String, Any]])) + } + + /** + * By default, the [[akka.actor.ActorSystem]] name will be set to `bundle--ActorSystem`. Override this + * method to define another name for your [[akka.actor.ActorSystem]] instance. + * + * @param context the bundle context + * @return the actor system name + */ + def getActorSystemName(context: BundleContext): String = null + +} diff --git a/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala new file mode 100644 index 0000000000..ae36406a60 --- /dev/null +++ b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala @@ -0,0 +1,57 @@ +package akka.osgi + +import impl.BundleDelegatingClassLoader +import akka.actor.ActorSystem +import com.typesafe.config.{ ConfigFactory, Config } +import org.osgi.framework.BundleContext + +/** + * Factory class to create ActorSystem implementations in an OSGi environment. This mainly involves dealing with + * bundle classloaders appropriately to ensure that configuration files and classes get loaded properly + */ +class OsgiActorSystemFactory(val context: BundleContext) { + + /* + * Classloader that delegates to the bundle for which the factory is creating an ActorSystem + */ + private val classloader = BundleDelegatingClassLoader.createFor(context) + + /** + * Creates the [[akka.actor.ActorSystem]], using the name specified + */ + def createActorSystem(name: String): ActorSystem = createActorSystem(Option(name)) + + /** + * Creates the [[akka.actor.ActorSystem]], using the name specified. + * + * A default name (`bundle--ActorSystem`) is assigned when you pass along [[scala.None]] instead. + */ + def createActorSystem(name: Option[String]): ActorSystem = + ActorSystem(actorSystemName(name), actorSystemConfig(context), classloader) + + /** + * Strategy method to create the Config for the ActorSystem, ensuring that the default/reference configuration is + * loaded from the akka-actor bundle. + */ + def actorSystemConfig(context: BundleContext): Config = { + val reference = ConfigFactory.defaultReference(classOf[ActorSystem].getClassLoader) + ConfigFactory.load(classloader).withFallback(reference) + } + + /** + * Determine the name for the [[akka.actor.ActorSystem]] + * Returns a default value of `bundle--ActorSystem` is no name is being specified + */ + def actorSystemName(name: Option[String]): String = + name.getOrElse("bundle-%s-ActorSystem".format(context.getBundle().getBundleId)) + +} + +object OsgiActorSystemFactory { + + /* + * Create an [[OsgiActorSystemFactory]] instance to set up Akka in an OSGi environment + */ + def apply(context: BundleContext): OsgiActorSystemFactory = new OsgiActorSystemFactory(context) + +} diff --git a/akka-osgi/src/main/scala/akka/osgi/impl/BundleDelegatingClassLoader.scala b/akka-osgi/src/main/scala/akka/osgi/impl/BundleDelegatingClassLoader.scala new file mode 100644 index 0000000000..08dee0344e --- /dev/null +++ b/akka-osgi/src/main/scala/akka/osgi/impl/BundleDelegatingClassLoader.scala @@ -0,0 +1,72 @@ +package akka.osgi.impl + +import java.net.URL +import java.util.Enumeration + +import org.osgi.framework.{ BundleContext, Bundle } + +/* + * Companion object to create bundle delegating classloader instances + */ +object BundleDelegatingClassLoader { + + /* + * Create a bundle delegating classloader for the bundle context's bundle + */ + def createFor(context: BundleContext) = new BundleDelegatingClassLoader(context.getBundle) + +} + +/* + * A bundle delegating classloader implemenation - this will try to load classes and resources from the bundle + * specified first and if there's a classloader specified, that will be used as a fallback + */ +class BundleDelegatingClassLoader(bundle: Bundle, classLoader: Option[ClassLoader]) extends ClassLoader { + + def this(bundle: Bundle) = this(bundle, None) + + protected override def findClass(name: String): Class[_] = bundle.loadClass(name) + + protected override def findResource(name: String): URL = { + val resource = bundle.getResource(name) + classLoader match { + case Some(loader) if resource == null ⇒ loader.getResource(name) + case _ ⇒ resource + } + } + + @SuppressWarnings(Array("unchecked", "rawtypes")) + protected override def findResources(name: String): Enumeration[URL] = + bundle.getResources(name).asInstanceOf[Enumeration[URL]] + + protected override def loadClass(name: String, resolve: Boolean): Class[_] = { + val clazz = try { + findClass(name) + } catch { + case cnfe: ClassNotFoundException ⇒ { + classLoader match { + case Some(loader) ⇒ loadClass(name, loader) + case None ⇒ rethrowClassNotFoundException(name, cnfe) + } + } + } + if (resolve) { + resolveClass(clazz) + } + clazz + } + + private def loadClass(name: String, classLoader: ClassLoader) = + try { + classLoader.loadClass(name) + } catch { + case cnfe: ClassNotFoundException ⇒ rethrowClassNotFoundException(name, cnfe) + } + + private def rethrowClassNotFoundException(name: String, cnfe: ClassNotFoundException): Nothing = + throw new ClassNotFoundException(name + " from bundle " + bundle.getBundleId + " (" + bundle.getSymbolicName + ")", cnfe) + + override def toString: String = String.format("BundleDelegatingClassLoader(%s)", bundle) + +} + diff --git a/akka-osgi/src/test/resources/logback-test.xml b/akka-osgi/src/test/resources/logback-test.xml new file mode 100644 index 0000000000..9c441a6fb6 --- /dev/null +++ b/akka-osgi/src/test/resources/logback-test.xml @@ -0,0 +1,23 @@ + + + + + + + %date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n + + + + + target/akka-osgi.log + true + + %date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n + + + + + + + + diff --git a/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala b/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala new file mode 100644 index 0000000000..6fa89886dd --- /dev/null +++ b/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala @@ -0,0 +1,69 @@ +package akka.osgi + +import org.scalatest.WordSpec +import akka.actor.ActorSystem +import akka.pattern.ask +import akka.dispatch.Await +import akka.util.duration._ +import akka.util.Timeout +import de.kalpatec.pojosr.framework.launch.BundleDescriptor +import test.{ RuntimeNameActorSystemActivator, TestActivators, PingPongActorSystemActivator } +import test.PingPong._ +import PojoSRTestSupport.bundle +import org.scalatest.matchers.MustMatchers + +/** + * Test cases for [[akka.osgi.ActorSystemActivator]] in 2 different scenarios: + * - no name configured for [[akka.actor.ActorSystem]] + * - runtime name configuration + */ +object ActorSystemActivatorTest { + + val TEST_BUNDLE_NAME = "akka.osgi.test.activator" + +} + +class PingPongActorSystemActivatorTest extends WordSpec with MustMatchers with PojoSRTestSupport { + + import ActorSystemActivatorTest._ + + val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq( + bundle(TEST_BUNDLE_NAME).withActivator(classOf[PingPongActorSystemActivator]))) + + "PingPongActorSystemActivator" must { + + "start and register the ActorSystem when bundle starts" in { + val system = serviceForType[ActorSystem] + val actor = system.actorFor("/user/pong") + + implicit val timeout = Timeout(5 seconds) + Await.result(actor ? Ping, timeout.duration) must be(Pong) + } + + "stop the ActorSystem when bundle stops" in { + val system = serviceForType[ActorSystem] + system.isTerminated must be(false) + + bundleForName(TEST_BUNDLE_NAME).stop() + + system.awaitTermination() + system.isTerminated must be(true) + } + } + +} + +class RuntimeNameActorSystemActivatorTest extends WordSpec with MustMatchers with PojoSRTestSupport { + + import ActorSystemActivatorTest._ + + val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq(bundle(TEST_BUNDLE_NAME).withActivator(classOf[RuntimeNameActorSystemActivator]))) + + "RuntimeNameActorSystemActivator" must { + + "register an ActorSystem and add the bundle id to the system name" in { + serviceForType[ActorSystem].name must equal(TestActivators.ACTOR_SYSTEM_NAME_PATTERN.format(bundleForName(TEST_BUNDLE_NAME).getBundleId)) + } + } + +} \ No newline at end of file diff --git a/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala b/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala new file mode 100644 index 0000000000..b19a90bf21 --- /dev/null +++ b/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala @@ -0,0 +1,141 @@ +package akka.osgi + +import de.kalpatec.pojosr.framework.launch.{ BundleDescriptor, PojoServiceRegistryFactory, ClasspathScanner } + +import scala.collection.JavaConversions.seqAsJavaList +import scala.collection.JavaConversions.collectionAsScalaIterable +import org.apache.commons.io.IOUtils.copy + +import org.osgi.framework._ +import java.net.URL + +import java.util.jar.JarInputStream +import java.io.{ FileInputStream, FileOutputStream, File } +import java.util.{ Date, ServiceLoader, HashMap } +import org.scalatest.{ BeforeAndAfterAll, Suite } + +/** + * Trait that provides support for building akka-osgi tests using PojoSR + */ +trait PojoSRTestSupport extends Suite with BeforeAndAfterAll { + + val MAX_WAIT_TIME = 12800 + val START_WAIT_TIME = 50 + + /** + * All bundles being found on the test classpath are automatically installed and started in the PojoSR runtime. + * Implement this to define the extra bundles that should be available for testing. + */ + val testBundles: Seq[BundleDescriptor] + + lazy val context: BundleContext = { + val config = new HashMap[String, AnyRef]() + System.setProperty("org.osgi.framework.storage", "target/akka-osgi/" + System.currentTimeMillis) + + val bundles = new ClasspathScanner().scanForBundles() + bundles.addAll(testBundles) + config.put(PojoServiceRegistryFactory.BUNDLE_DESCRIPTORS, bundles) + + ServiceLoader.load(classOf[PojoServiceRegistryFactory]).iterator.next.newPojoServiceRegistry(config).getBundleContext + } + + // Ensure bundles get stopped at the end of the test to release resources and stop threads + override protected def afterAll() = context.getBundles.foreach(_.stop) + + /** + * Convenience method to find a bundle by symbolic name + */ + def bundleForName(name: String) = + context.getBundles.find(_.getSymbolicName == name).getOrElse(fail("Unable to find bundle with symbolic name %s".format(name))) + + /** + * Convenience method to find a service by interface. If the service is not already available in the OSGi Service + * Registry, this method will wait for a few seconds for the service to appear. + */ + def serviceForType[T](implicit manifest: Manifest[T]): T = + context.getService(awaitReference(manifest.erasure)).asInstanceOf[T] + + def awaitReference(serviceType: Class[_]): ServiceReference = awaitReference(serviceType, START_WAIT_TIME) + + def awaitReference(serviceType: Class[_], wait: Long): ServiceReference = { + val option = Option(context.getServiceReference(serviceType.getName)) + Thread.sleep(wait) //FIXME No sleep please + option match { + case Some(reference) ⇒ reference + case None if (wait > MAX_WAIT_TIME) ⇒ fail("Gave up waiting for service of type %s".format(serviceType)) + case None ⇒ awaitReference(serviceType, wait * 2) + } + } + + protected def buildTestBundles(builders: Seq[BundleDescriptorBuilder]): Seq[BundleDescriptor] = builders map (_.build) +} + +object PojoSRTestSupport { + /** + * Convenience method to define additional test bundles + */ + def bundle(name: String) = new BundleDescriptorBuilder(name) +} + +/** + * Helper class to make it easier to define test bundles + */ +class BundleDescriptorBuilder(name: String) { + + import org.ops4j.pax.tinybundles.core.TinyBundles + + val tinybundle = TinyBundles.bundle.set(Constants.BUNDLE_SYMBOLICNAME, name) + + /** + * Add a Blueprint XML file to our test bundle + */ + def withBlueprintFile(name: String, contents: URL): BundleDescriptorBuilder = { + tinybundle.add("OSGI-INF/blueprint/%s".format(name), contents) + this + } + + /** + * Add a Blueprint XML file to our test bundle + */ + def withBlueprintFile(contents: URL): BundleDescriptorBuilder = { + val filename = contents.getFile.split("/").last + withBlueprintFile(filename, contents) + } + + /** + * Add a Bundle activator to our test bundle + */ + def withActivator(activator: Class[_ <: BundleActivator]): BundleDescriptorBuilder = { + tinybundle.set(Constants.BUNDLE_ACTIVATOR, activator.getName) + this + } + + /** + * Build the actual PojoSR BundleDescriptor instance + */ + def build: BundleDescriptor = { + val file: File = tinybundleToJarFile(name) + new BundleDescriptor(getClass().getClassLoader(), new URL("jar:" + file.toURI().toString() + "!/"), extractHeaders(file)) + } + + def extractHeaders(file: File): HashMap[String, String] = { + val headers = new HashMap[String, String]() + + val jis = new JarInputStream(new FileInputStream(file)) + try { + for (entry ← jis.getManifest().getMainAttributes().entrySet()) + headers.put(entry.getKey().toString(), entry.getValue().toString()) + } finally jis.close() + + headers + } + + def tinybundleToJarFile(name: String): File = { + val file = new File("target/%s-%tQ.jar".format(name, new Date())) + val fos = new FileOutputStream(file) + try copy(tinybundle.build(), fos) finally fos.close() + + file + } +} + diff --git a/akka-osgi/src/test/scala/akka/osgi/test/PingPong.scala b/akka-osgi/src/test/scala/akka/osgi/test/PingPong.scala new file mode 100644 index 0000000000..6a7409c667 --- /dev/null +++ b/akka-osgi/src/test/scala/akka/osgi/test/PingPong.scala @@ -0,0 +1,22 @@ +package akka.osgi.test + +import akka.actor.Actor + +/** + * Simple ping-pong actor, used for testing + */ +object PingPong { + + abstract class TestMessage + + case object Ping extends TestMessage + case object Pong extends TestMessage + + class PongActor extends Actor { + def receive = { + case Ping ⇒ + sender ! Pong + } + } + +} diff --git a/akka-osgi/src/test/scala/akka/osgi/test/TestActivators.scala b/akka-osgi/src/test/scala/akka/osgi/test/TestActivators.scala new file mode 100644 index 0000000000..54369d88ca --- /dev/null +++ b/akka-osgi/src/test/scala/akka/osgi/test/TestActivators.scala @@ -0,0 +1,39 @@ +package akka.osgi.test + +import akka.osgi.ActorSystemActivator +import akka.actor.{ Props, ActorSystem } +import PingPong._ +import org.osgi.framework.BundleContext + +/** + * A set of [[akka.osgi.ActorSystemActivator]]s for testing purposes + */ +object TestActivators { + + val ACTOR_SYSTEM_NAME_PATTERN = "actor-system-for-bundle-%s" + +} + +/** + * Simple ActorSystemActivator that starts the sample ping-pong application + */ +class PingPongActorSystemActivator extends ActorSystemActivator { + + def configure(context: BundleContext, system: ActorSystem) { + system.actorOf(Props(new PongActor), name = "pong") + registerService(context, system) + } + +} + +/** + * [[akka.osgi.ActorSystemActivator]] implementation that determines [[akka.actor.ActorSystem]] name at runtime + */ +class RuntimeNameActorSystemActivator extends ActorSystemActivator { + + def configure(context: BundleContext, system: ActorSystem) = registerService(context, system); + + override def getActorSystemName(context: BundleContext) = + TestActivators.ACTOR_SYSTEM_NAME_PATTERN.format(context.getBundle.getBundleId) + +} \ No newline at end of file diff --git a/akka-remote-tests/src/main/java/akka/remote/testconductor/TestConductorProtocol.java b/akka-remote-tests/src/main/java/akka/remote/testconductor/TestConductorProtocol.java new file mode 100644 index 0000000000..bd8de8a052 --- /dev/null +++ b/akka-remote-tests/src/main/java/akka/remote/testconductor/TestConductorProtocol.java @@ -0,0 +1,4287 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: TestConductorProtocol.proto + +package akka.remote.testconductor; + +public final class TestConductorProtocol { + private TestConductorProtocol() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public enum BarrierOp + implements com.google.protobuf.ProtocolMessageEnum { + Enter(0, 1), + Fail(1, 2), + Succeeded(2, 3), + Failed(3, 4), + ; + + public static final int Enter_VALUE = 1; + public static final int Fail_VALUE = 2; + public static final int Succeeded_VALUE = 3; + public static final int Failed_VALUE = 4; + + + public final int getNumber() { return value; } + + public static BarrierOp valueOf(int value) { + switch (value) { + case 1: return Enter; + case 2: return Fail; + case 3: return Succeeded; + case 4: return Failed; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public BarrierOp findValueByNumber(int number) { + return BarrierOp.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.getDescriptor().getEnumTypes().get(0); + } + + private static final BarrierOp[] VALUES = { + Enter, Fail, Succeeded, Failed, + }; + + public static BarrierOp valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private BarrierOp(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:BarrierOp) + } + + public enum FailType + implements com.google.protobuf.ProtocolMessageEnum { + Throttle(0, 1), + Disconnect(1, 2), + Abort(2, 3), + Shutdown(3, 4), + ; + + public static final int Throttle_VALUE = 1; + public static final int Disconnect_VALUE = 2; + public static final int Abort_VALUE = 3; + public static final int Shutdown_VALUE = 4; + + + public final int getNumber() { return value; } + + public static FailType valueOf(int value) { + switch (value) { + case 1: return Throttle; + case 2: return Disconnect; + case 3: return Abort; + case 4: return Shutdown; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public FailType findValueByNumber(int number) { + return FailType.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.getDescriptor().getEnumTypes().get(1); + } + + private static final FailType[] VALUES = { + Throttle, Disconnect, Abort, Shutdown, + }; + + public static FailType valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private FailType(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:FailType) + } + + public enum Direction + implements com.google.protobuf.ProtocolMessageEnum { + Send(0, 1), + Receive(1, 2), + Both(2, 3), + ; + + public static final int Send_VALUE = 1; + public static final int Receive_VALUE = 2; + public static final int Both_VALUE = 3; + + + public final int getNumber() { return value; } + + public static Direction valueOf(int value) { + switch (value) { + case 1: return Send; + case 2: return Receive; + case 3: return Both; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public Direction findValueByNumber(int number) { + return Direction.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.getDescriptor().getEnumTypes().get(2); + } + + private static final Direction[] VALUES = { + Send, Receive, Both, + }; + + public static Direction valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private Direction(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:Direction) + } + + public interface WrapperOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional .Hello hello = 1; + boolean hasHello(); + akka.remote.testconductor.TestConductorProtocol.Hello getHello(); + akka.remote.testconductor.TestConductorProtocol.HelloOrBuilder getHelloOrBuilder(); + + // optional .EnterBarrier barrier = 2; + boolean hasBarrier(); + akka.remote.testconductor.TestConductorProtocol.EnterBarrier getBarrier(); + akka.remote.testconductor.TestConductorProtocol.EnterBarrierOrBuilder getBarrierOrBuilder(); + + // optional .InjectFailure failure = 3; + boolean hasFailure(); + akka.remote.testconductor.TestConductorProtocol.InjectFailure getFailure(); + akka.remote.testconductor.TestConductorProtocol.InjectFailureOrBuilder getFailureOrBuilder(); + + // optional string done = 4; + boolean hasDone(); + String getDone(); + + // optional .AddressRequest addr = 5; + boolean hasAddr(); + akka.remote.testconductor.TestConductorProtocol.AddressRequest getAddr(); + akka.remote.testconductor.TestConductorProtocol.AddressRequestOrBuilder getAddrOrBuilder(); + } + public static final class Wrapper extends + com.google.protobuf.GeneratedMessage + implements WrapperOrBuilder { + // Use Wrapper.newBuilder() to construct. + private Wrapper(Builder builder) { + super(builder); + } + private Wrapper(boolean noInit) {} + + private static final Wrapper defaultInstance; + public static Wrapper getDefaultInstance() { + return defaultInstance; + } + + public Wrapper getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Wrapper_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Wrapper_fieldAccessorTable; + } + + private int bitField0_; + // optional .Hello hello = 1; + public static final int HELLO_FIELD_NUMBER = 1; + private akka.remote.testconductor.TestConductorProtocol.Hello hello_; + public boolean hasHello() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.testconductor.TestConductorProtocol.Hello getHello() { + return hello_; + } + public akka.remote.testconductor.TestConductorProtocol.HelloOrBuilder getHelloOrBuilder() { + return hello_; + } + + // optional .EnterBarrier barrier = 2; + public static final int BARRIER_FIELD_NUMBER = 2; + private akka.remote.testconductor.TestConductorProtocol.EnterBarrier barrier_; + public boolean hasBarrier() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.EnterBarrier getBarrier() { + return barrier_; + } + public akka.remote.testconductor.TestConductorProtocol.EnterBarrierOrBuilder getBarrierOrBuilder() { + return barrier_; + } + + // optional .InjectFailure failure = 3; + public static final int FAILURE_FIELD_NUMBER = 3; + private akka.remote.testconductor.TestConductorProtocol.InjectFailure failure_; + public boolean hasFailure() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public akka.remote.testconductor.TestConductorProtocol.InjectFailure getFailure() { + return failure_; + } + public akka.remote.testconductor.TestConductorProtocol.InjectFailureOrBuilder getFailureOrBuilder() { + return failure_; + } + + // optional string done = 4; + public static final int DONE_FIELD_NUMBER = 4; + private java.lang.Object done_; + public boolean hasDone() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getDone() { + java.lang.Object ref = done_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + done_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getDoneBytes() { + java.lang.Object ref = done_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + done_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional .AddressRequest addr = 5; + public static final int ADDR_FIELD_NUMBER = 5; + private akka.remote.testconductor.TestConductorProtocol.AddressRequest addr_; + public boolean hasAddr() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public akka.remote.testconductor.TestConductorProtocol.AddressRequest getAddr() { + return addr_; + } + public akka.remote.testconductor.TestConductorProtocol.AddressRequestOrBuilder getAddrOrBuilder() { + return addr_; + } + + private void initFields() { + hello_ = akka.remote.testconductor.TestConductorProtocol.Hello.getDefaultInstance(); + barrier_ = akka.remote.testconductor.TestConductorProtocol.EnterBarrier.getDefaultInstance(); + failure_ = akka.remote.testconductor.TestConductorProtocol.InjectFailure.getDefaultInstance(); + done_ = ""; + addr_ = akka.remote.testconductor.TestConductorProtocol.AddressRequest.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (hasHello()) { + if (!getHello().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasBarrier()) { + if (!getBarrier().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasFailure()) { + if (!getFailure().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasAddr()) { + if (!getAddr().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, hello_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, barrier_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, failure_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, getDoneBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeMessage(5, addr_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, hello_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, barrier_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, failure_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(4, getDoneBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(5, addr_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Wrapper parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.testconductor.TestConductorProtocol.Wrapper prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.testconductor.TestConductorProtocol.WrapperOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Wrapper_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Wrapper_fieldAccessorTable; + } + + // Construct using akka.remote.testconductor.TestConductorProtocol.Wrapper.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getHelloFieldBuilder(); + getBarrierFieldBuilder(); + getFailureFieldBuilder(); + getAddrFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (helloBuilder_ == null) { + hello_ = akka.remote.testconductor.TestConductorProtocol.Hello.getDefaultInstance(); + } else { + helloBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (barrierBuilder_ == null) { + barrier_ = akka.remote.testconductor.TestConductorProtocol.EnterBarrier.getDefaultInstance(); + } else { + barrierBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + if (failureBuilder_ == null) { + failure_ = akka.remote.testconductor.TestConductorProtocol.InjectFailure.getDefaultInstance(); + } else { + failureBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + done_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); + if (addrBuilder_ == null) { + addr_ = akka.remote.testconductor.TestConductorProtocol.AddressRequest.getDefaultInstance(); + } else { + addrBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.testconductor.TestConductorProtocol.Wrapper.getDescriptor(); + } + + public akka.remote.testconductor.TestConductorProtocol.Wrapper getDefaultInstanceForType() { + return akka.remote.testconductor.TestConductorProtocol.Wrapper.getDefaultInstance(); + } + + public akka.remote.testconductor.TestConductorProtocol.Wrapper build() { + akka.remote.testconductor.TestConductorProtocol.Wrapper result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private akka.remote.testconductor.TestConductorProtocol.Wrapper buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + akka.remote.testconductor.TestConductorProtocol.Wrapper result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public akka.remote.testconductor.TestConductorProtocol.Wrapper buildPartial() { + akka.remote.testconductor.TestConductorProtocol.Wrapper result = new akka.remote.testconductor.TestConductorProtocol.Wrapper(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (helloBuilder_ == null) { + result.hello_ = hello_; + } else { + result.hello_ = helloBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (barrierBuilder_ == null) { + result.barrier_ = barrier_; + } else { + result.barrier_ = barrierBuilder_.build(); + } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (failureBuilder_ == null) { + result.failure_ = failure_; + } else { + result.failure_ = failureBuilder_.build(); + } + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.done_ = done_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + if (addrBuilder_ == null) { + result.addr_ = addr_; + } else { + result.addr_ = addrBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof akka.remote.testconductor.TestConductorProtocol.Wrapper) { + return mergeFrom((akka.remote.testconductor.TestConductorProtocol.Wrapper)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.testconductor.TestConductorProtocol.Wrapper other) { + if (other == akka.remote.testconductor.TestConductorProtocol.Wrapper.getDefaultInstance()) return this; + if (other.hasHello()) { + mergeHello(other.getHello()); + } + if (other.hasBarrier()) { + mergeBarrier(other.getBarrier()); + } + if (other.hasFailure()) { + mergeFailure(other.getFailure()); + } + if (other.hasDone()) { + setDone(other.getDone()); + } + if (other.hasAddr()) { + mergeAddr(other.getAddr()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (hasHello()) { + if (!getHello().isInitialized()) { + + return false; + } + } + if (hasBarrier()) { + if (!getBarrier().isInitialized()) { + + return false; + } + } + if (hasFailure()) { + if (!getFailure().isInitialized()) { + + return false; + } + } + if (hasAddr()) { + if (!getAddr().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + akka.remote.testconductor.TestConductorProtocol.Hello.Builder subBuilder = akka.remote.testconductor.TestConductorProtocol.Hello.newBuilder(); + if (hasHello()) { + subBuilder.mergeFrom(getHello()); + } + input.readMessage(subBuilder, extensionRegistry); + setHello(subBuilder.buildPartial()); + break; + } + case 18: { + akka.remote.testconductor.TestConductorProtocol.EnterBarrier.Builder subBuilder = akka.remote.testconductor.TestConductorProtocol.EnterBarrier.newBuilder(); + if (hasBarrier()) { + subBuilder.mergeFrom(getBarrier()); + } + input.readMessage(subBuilder, extensionRegistry); + setBarrier(subBuilder.buildPartial()); + break; + } + case 26: { + akka.remote.testconductor.TestConductorProtocol.InjectFailure.Builder subBuilder = akka.remote.testconductor.TestConductorProtocol.InjectFailure.newBuilder(); + if (hasFailure()) { + subBuilder.mergeFrom(getFailure()); + } + input.readMessage(subBuilder, extensionRegistry); + setFailure(subBuilder.buildPartial()); + break; + } + case 34: { + bitField0_ |= 0x00000008; + done_ = input.readBytes(); + break; + } + case 42: { + akka.remote.testconductor.TestConductorProtocol.AddressRequest.Builder subBuilder = akka.remote.testconductor.TestConductorProtocol.AddressRequest.newBuilder(); + if (hasAddr()) { + subBuilder.mergeFrom(getAddr()); + } + input.readMessage(subBuilder, extensionRegistry); + setAddr(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // optional .Hello hello = 1; + private akka.remote.testconductor.TestConductorProtocol.Hello hello_ = akka.remote.testconductor.TestConductorProtocol.Hello.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Hello, akka.remote.testconductor.TestConductorProtocol.Hello.Builder, akka.remote.testconductor.TestConductorProtocol.HelloOrBuilder> helloBuilder_; + public boolean hasHello() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.testconductor.TestConductorProtocol.Hello getHello() { + if (helloBuilder_ == null) { + return hello_; + } else { + return helloBuilder_.getMessage(); + } + } + public Builder setHello(akka.remote.testconductor.TestConductorProtocol.Hello value) { + if (helloBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + hello_ = value; + onChanged(); + } else { + helloBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder setHello( + akka.remote.testconductor.TestConductorProtocol.Hello.Builder builderForValue) { + if (helloBuilder_ == null) { + hello_ = builderForValue.build(); + onChanged(); + } else { + helloBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder mergeHello(akka.remote.testconductor.TestConductorProtocol.Hello value) { + if (helloBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + hello_ != akka.remote.testconductor.TestConductorProtocol.Hello.getDefaultInstance()) { + hello_ = + akka.remote.testconductor.TestConductorProtocol.Hello.newBuilder(hello_).mergeFrom(value).buildPartial(); + } else { + hello_ = value; + } + onChanged(); + } else { + helloBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder clearHello() { + if (helloBuilder_ == null) { + hello_ = akka.remote.testconductor.TestConductorProtocol.Hello.getDefaultInstance(); + onChanged(); + } else { + helloBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + public akka.remote.testconductor.TestConductorProtocol.Hello.Builder getHelloBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getHelloFieldBuilder().getBuilder(); + } + public akka.remote.testconductor.TestConductorProtocol.HelloOrBuilder getHelloOrBuilder() { + if (helloBuilder_ != null) { + return helloBuilder_.getMessageOrBuilder(); + } else { + return hello_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Hello, akka.remote.testconductor.TestConductorProtocol.Hello.Builder, akka.remote.testconductor.TestConductorProtocol.HelloOrBuilder> + getHelloFieldBuilder() { + if (helloBuilder_ == null) { + helloBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Hello, akka.remote.testconductor.TestConductorProtocol.Hello.Builder, akka.remote.testconductor.TestConductorProtocol.HelloOrBuilder>( + hello_, + getParentForChildren(), + isClean()); + hello_ = null; + } + return helloBuilder_; + } + + // optional .EnterBarrier barrier = 2; + private akka.remote.testconductor.TestConductorProtocol.EnterBarrier barrier_ = akka.remote.testconductor.TestConductorProtocol.EnterBarrier.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.EnterBarrier, akka.remote.testconductor.TestConductorProtocol.EnterBarrier.Builder, akka.remote.testconductor.TestConductorProtocol.EnterBarrierOrBuilder> barrierBuilder_; + public boolean hasBarrier() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.EnterBarrier getBarrier() { + if (barrierBuilder_ == null) { + return barrier_; + } else { + return barrierBuilder_.getMessage(); + } + } + public Builder setBarrier(akka.remote.testconductor.TestConductorProtocol.EnterBarrier value) { + if (barrierBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + barrier_ = value; + onChanged(); + } else { + barrierBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder setBarrier( + akka.remote.testconductor.TestConductorProtocol.EnterBarrier.Builder builderForValue) { + if (barrierBuilder_ == null) { + barrier_ = builderForValue.build(); + onChanged(); + } else { + barrierBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeBarrier(akka.remote.testconductor.TestConductorProtocol.EnterBarrier value) { + if (barrierBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + barrier_ != akka.remote.testconductor.TestConductorProtocol.EnterBarrier.getDefaultInstance()) { + barrier_ = + akka.remote.testconductor.TestConductorProtocol.EnterBarrier.newBuilder(barrier_).mergeFrom(value).buildPartial(); + } else { + barrier_ = value; + } + onChanged(); + } else { + barrierBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder clearBarrier() { + if (barrierBuilder_ == null) { + barrier_ = akka.remote.testconductor.TestConductorProtocol.EnterBarrier.getDefaultInstance(); + onChanged(); + } else { + barrierBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + public akka.remote.testconductor.TestConductorProtocol.EnterBarrier.Builder getBarrierBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getBarrierFieldBuilder().getBuilder(); + } + public akka.remote.testconductor.TestConductorProtocol.EnterBarrierOrBuilder getBarrierOrBuilder() { + if (barrierBuilder_ != null) { + return barrierBuilder_.getMessageOrBuilder(); + } else { + return barrier_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.EnterBarrier, akka.remote.testconductor.TestConductorProtocol.EnterBarrier.Builder, akka.remote.testconductor.TestConductorProtocol.EnterBarrierOrBuilder> + getBarrierFieldBuilder() { + if (barrierBuilder_ == null) { + barrierBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.EnterBarrier, akka.remote.testconductor.TestConductorProtocol.EnterBarrier.Builder, akka.remote.testconductor.TestConductorProtocol.EnterBarrierOrBuilder>( + barrier_, + getParentForChildren(), + isClean()); + barrier_ = null; + } + return barrierBuilder_; + } + + // optional .InjectFailure failure = 3; + private akka.remote.testconductor.TestConductorProtocol.InjectFailure failure_ = akka.remote.testconductor.TestConductorProtocol.InjectFailure.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.InjectFailure, akka.remote.testconductor.TestConductorProtocol.InjectFailure.Builder, akka.remote.testconductor.TestConductorProtocol.InjectFailureOrBuilder> failureBuilder_; + public boolean hasFailure() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public akka.remote.testconductor.TestConductorProtocol.InjectFailure getFailure() { + if (failureBuilder_ == null) { + return failure_; + } else { + return failureBuilder_.getMessage(); + } + } + public Builder setFailure(akka.remote.testconductor.TestConductorProtocol.InjectFailure value) { + if (failureBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + failure_ = value; + onChanged(); + } else { + failureBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + public Builder setFailure( + akka.remote.testconductor.TestConductorProtocol.InjectFailure.Builder builderForValue) { + if (failureBuilder_ == null) { + failure_ = builderForValue.build(); + onChanged(); + } else { + failureBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + public Builder mergeFailure(akka.remote.testconductor.TestConductorProtocol.InjectFailure value) { + if (failureBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + failure_ != akka.remote.testconductor.TestConductorProtocol.InjectFailure.getDefaultInstance()) { + failure_ = + akka.remote.testconductor.TestConductorProtocol.InjectFailure.newBuilder(failure_).mergeFrom(value).buildPartial(); + } else { + failure_ = value; + } + onChanged(); + } else { + failureBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + public Builder clearFailure() { + if (failureBuilder_ == null) { + failure_ = akka.remote.testconductor.TestConductorProtocol.InjectFailure.getDefaultInstance(); + onChanged(); + } else { + failureBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + public akka.remote.testconductor.TestConductorProtocol.InjectFailure.Builder getFailureBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getFailureFieldBuilder().getBuilder(); + } + public akka.remote.testconductor.TestConductorProtocol.InjectFailureOrBuilder getFailureOrBuilder() { + if (failureBuilder_ != null) { + return failureBuilder_.getMessageOrBuilder(); + } else { + return failure_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.InjectFailure, akka.remote.testconductor.TestConductorProtocol.InjectFailure.Builder, akka.remote.testconductor.TestConductorProtocol.InjectFailureOrBuilder> + getFailureFieldBuilder() { + if (failureBuilder_ == null) { + failureBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.InjectFailure, akka.remote.testconductor.TestConductorProtocol.InjectFailure.Builder, akka.remote.testconductor.TestConductorProtocol.InjectFailureOrBuilder>( + failure_, + getParentForChildren(), + isClean()); + failure_ = null; + } + return failureBuilder_; + } + + // optional string done = 4; + private java.lang.Object done_ = ""; + public boolean hasDone() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getDone() { + java.lang.Object ref = done_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + done_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setDone(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + done_ = value; + onChanged(); + return this; + } + public Builder clearDone() { + bitField0_ = (bitField0_ & ~0x00000008); + done_ = getDefaultInstance().getDone(); + onChanged(); + return this; + } + void setDone(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000008; + done_ = value; + onChanged(); + } + + // optional .AddressRequest addr = 5; + private akka.remote.testconductor.TestConductorProtocol.AddressRequest addr_ = akka.remote.testconductor.TestConductorProtocol.AddressRequest.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.AddressRequest, akka.remote.testconductor.TestConductorProtocol.AddressRequest.Builder, akka.remote.testconductor.TestConductorProtocol.AddressRequestOrBuilder> addrBuilder_; + public boolean hasAddr() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public akka.remote.testconductor.TestConductorProtocol.AddressRequest getAddr() { + if (addrBuilder_ == null) { + return addr_; + } else { + return addrBuilder_.getMessage(); + } + } + public Builder setAddr(akka.remote.testconductor.TestConductorProtocol.AddressRequest value) { + if (addrBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + addr_ = value; + onChanged(); + } else { + addrBuilder_.setMessage(value); + } + bitField0_ |= 0x00000010; + return this; + } + public Builder setAddr( + akka.remote.testconductor.TestConductorProtocol.AddressRequest.Builder builderForValue) { + if (addrBuilder_ == null) { + addr_ = builderForValue.build(); + onChanged(); + } else { + addrBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000010; + return this; + } + public Builder mergeAddr(akka.remote.testconductor.TestConductorProtocol.AddressRequest value) { + if (addrBuilder_ == null) { + if (((bitField0_ & 0x00000010) == 0x00000010) && + addr_ != akka.remote.testconductor.TestConductorProtocol.AddressRequest.getDefaultInstance()) { + addr_ = + akka.remote.testconductor.TestConductorProtocol.AddressRequest.newBuilder(addr_).mergeFrom(value).buildPartial(); + } else { + addr_ = value; + } + onChanged(); + } else { + addrBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000010; + return this; + } + public Builder clearAddr() { + if (addrBuilder_ == null) { + addr_ = akka.remote.testconductor.TestConductorProtocol.AddressRequest.getDefaultInstance(); + onChanged(); + } else { + addrBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + public akka.remote.testconductor.TestConductorProtocol.AddressRequest.Builder getAddrBuilder() { + bitField0_ |= 0x00000010; + onChanged(); + return getAddrFieldBuilder().getBuilder(); + } + public akka.remote.testconductor.TestConductorProtocol.AddressRequestOrBuilder getAddrOrBuilder() { + if (addrBuilder_ != null) { + return addrBuilder_.getMessageOrBuilder(); + } else { + return addr_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.AddressRequest, akka.remote.testconductor.TestConductorProtocol.AddressRequest.Builder, akka.remote.testconductor.TestConductorProtocol.AddressRequestOrBuilder> + getAddrFieldBuilder() { + if (addrBuilder_ == null) { + addrBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.AddressRequest, akka.remote.testconductor.TestConductorProtocol.AddressRequest.Builder, akka.remote.testconductor.TestConductorProtocol.AddressRequestOrBuilder>( + addr_, + getParentForChildren(), + isClean()); + addr_ = null; + } + return addrBuilder_; + } + + // @@protoc_insertion_point(builder_scope:Wrapper) + } + + static { + defaultInstance = new Wrapper(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:Wrapper) + } + + public interface HelloOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string name = 1; + boolean hasName(); + String getName(); + + // required .Address address = 2; + boolean hasAddress(); + akka.remote.testconductor.TestConductorProtocol.Address getAddress(); + akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder getAddressOrBuilder(); + } + public static final class Hello extends + com.google.protobuf.GeneratedMessage + implements HelloOrBuilder { + // Use Hello.newBuilder() to construct. + private Hello(Builder builder) { + super(builder); + } + private Hello(boolean noInit) {} + + private static final Hello defaultInstance; + public static Hello getDefaultInstance() { + return defaultInstance; + } + + public Hello getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Hello_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Hello_fieldAccessorTable; + } + + private int bitField0_; + // required string name = 1; + public static final int NAME_FIELD_NUMBER = 1; + private java.lang.Object name_; + public boolean hasName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getName() { + java.lang.Object ref = name_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + name_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required .Address address = 2; + public static final int ADDRESS_FIELD_NUMBER = 2; + private akka.remote.testconductor.TestConductorProtocol.Address address_; + public boolean hasAddress() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.Address getAddress() { + return address_; + } + public akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder getAddressOrBuilder() { + return address_; + } + + private void initFields() { + name_ = ""; + address_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasAddress()) { + memoizedIsInitialized = 0; + return false; + } + if (!getAddress().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, address_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, address_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.testconductor.TestConductorProtocol.Hello parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Hello parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Hello parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Hello parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Hello parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Hello parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Hello parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.Hello parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.Hello parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Hello parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.testconductor.TestConductorProtocol.Hello prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.testconductor.TestConductorProtocol.HelloOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Hello_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Hello_fieldAccessorTable; + } + + // Construct using akka.remote.testconductor.TestConductorProtocol.Hello.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getAddressFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + name_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + if (addressBuilder_ == null) { + address_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + } else { + addressBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.testconductor.TestConductorProtocol.Hello.getDescriptor(); + } + + public akka.remote.testconductor.TestConductorProtocol.Hello getDefaultInstanceForType() { + return akka.remote.testconductor.TestConductorProtocol.Hello.getDefaultInstance(); + } + + public akka.remote.testconductor.TestConductorProtocol.Hello build() { + akka.remote.testconductor.TestConductorProtocol.Hello result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private akka.remote.testconductor.TestConductorProtocol.Hello buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + akka.remote.testconductor.TestConductorProtocol.Hello result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public akka.remote.testconductor.TestConductorProtocol.Hello buildPartial() { + akka.remote.testconductor.TestConductorProtocol.Hello result = new akka.remote.testconductor.TestConductorProtocol.Hello(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.name_ = name_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (addressBuilder_ == null) { + result.address_ = address_; + } else { + result.address_ = addressBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof akka.remote.testconductor.TestConductorProtocol.Hello) { + return mergeFrom((akka.remote.testconductor.TestConductorProtocol.Hello)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.testconductor.TestConductorProtocol.Hello other) { + if (other == akka.remote.testconductor.TestConductorProtocol.Hello.getDefaultInstance()) return this; + if (other.hasName()) { + setName(other.getName()); + } + if (other.hasAddress()) { + mergeAddress(other.getAddress()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasName()) { + + return false; + } + if (!hasAddress()) { + + return false; + } + if (!getAddress().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + name_ = input.readBytes(); + break; + } + case 18: { + akka.remote.testconductor.TestConductorProtocol.Address.Builder subBuilder = akka.remote.testconductor.TestConductorProtocol.Address.newBuilder(); + if (hasAddress()) { + subBuilder.mergeFrom(getAddress()); + } + input.readMessage(subBuilder, extensionRegistry); + setAddress(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required string name = 1; + private java.lang.Object name_ = ""; + public boolean hasName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + name_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + name_ = value; + onChanged(); + return this; + } + public Builder clearName() { + bitField0_ = (bitField0_ & ~0x00000001); + name_ = getDefaultInstance().getName(); + onChanged(); + return this; + } + void setName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + name_ = value; + onChanged(); + } + + // required .Address address = 2; + private akka.remote.testconductor.TestConductorProtocol.Address address_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Address, akka.remote.testconductor.TestConductorProtocol.Address.Builder, akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder> addressBuilder_; + public boolean hasAddress() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.Address getAddress() { + if (addressBuilder_ == null) { + return address_; + } else { + return addressBuilder_.getMessage(); + } + } + public Builder setAddress(akka.remote.testconductor.TestConductorProtocol.Address value) { + if (addressBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + address_ = value; + onChanged(); + } else { + addressBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder setAddress( + akka.remote.testconductor.TestConductorProtocol.Address.Builder builderForValue) { + if (addressBuilder_ == null) { + address_ = builderForValue.build(); + onChanged(); + } else { + addressBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeAddress(akka.remote.testconductor.TestConductorProtocol.Address value) { + if (addressBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + address_ != akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance()) { + address_ = + akka.remote.testconductor.TestConductorProtocol.Address.newBuilder(address_).mergeFrom(value).buildPartial(); + } else { + address_ = value; + } + onChanged(); + } else { + addressBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder clearAddress() { + if (addressBuilder_ == null) { + address_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + onChanged(); + } else { + addressBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + public akka.remote.testconductor.TestConductorProtocol.Address.Builder getAddressBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getAddressFieldBuilder().getBuilder(); + } + public akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder getAddressOrBuilder() { + if (addressBuilder_ != null) { + return addressBuilder_.getMessageOrBuilder(); + } else { + return address_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Address, akka.remote.testconductor.TestConductorProtocol.Address.Builder, akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder> + getAddressFieldBuilder() { + if (addressBuilder_ == null) { + addressBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Address, akka.remote.testconductor.TestConductorProtocol.Address.Builder, akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder>( + address_, + getParentForChildren(), + isClean()); + address_ = null; + } + return addressBuilder_; + } + + // @@protoc_insertion_point(builder_scope:Hello) + } + + static { + defaultInstance = new Hello(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:Hello) + } + + public interface EnterBarrierOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string name = 1; + boolean hasName(); + String getName(); + + // required .BarrierOp op = 2; + boolean hasOp(); + akka.remote.testconductor.TestConductorProtocol.BarrierOp getOp(); + + // optional int64 timeout = 3; + boolean hasTimeout(); + long getTimeout(); + } + public static final class EnterBarrier extends + com.google.protobuf.GeneratedMessage + implements EnterBarrierOrBuilder { + // Use EnterBarrier.newBuilder() to construct. + private EnterBarrier(Builder builder) { + super(builder); + } + private EnterBarrier(boolean noInit) {} + + private static final EnterBarrier defaultInstance; + public static EnterBarrier getDefaultInstance() { + return defaultInstance; + } + + public EnterBarrier getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_EnterBarrier_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_EnterBarrier_fieldAccessorTable; + } + + private int bitField0_; + // required string name = 1; + public static final int NAME_FIELD_NUMBER = 1; + private java.lang.Object name_; + public boolean hasName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getName() { + java.lang.Object ref = name_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + name_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getNameBytes() { + java.lang.Object ref = name_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + name_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required .BarrierOp op = 2; + public static final int OP_FIELD_NUMBER = 2; + private akka.remote.testconductor.TestConductorProtocol.BarrierOp op_; + public boolean hasOp() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.BarrierOp getOp() { + return op_; + } + + // optional int64 timeout = 3; + public static final int TIMEOUT_FIELD_NUMBER = 3; + private long timeout_; + public boolean hasTimeout() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTimeout() { + return timeout_; + } + + private void initFields() { + name_ = ""; + op_ = akka.remote.testconductor.TestConductorProtocol.BarrierOp.Enter; + timeout_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasOp()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, op_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt64(3, timeout_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(2, op_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(3, timeout_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.EnterBarrier parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.testconductor.TestConductorProtocol.EnterBarrier prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.testconductor.TestConductorProtocol.EnterBarrierOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_EnterBarrier_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_EnterBarrier_fieldAccessorTable; + } + + // Construct using akka.remote.testconductor.TestConductorProtocol.EnterBarrier.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + name_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + op_ = akka.remote.testconductor.TestConductorProtocol.BarrierOp.Enter; + bitField0_ = (bitField0_ & ~0x00000002); + timeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.testconductor.TestConductorProtocol.EnterBarrier.getDescriptor(); + } + + public akka.remote.testconductor.TestConductorProtocol.EnterBarrier getDefaultInstanceForType() { + return akka.remote.testconductor.TestConductorProtocol.EnterBarrier.getDefaultInstance(); + } + + public akka.remote.testconductor.TestConductorProtocol.EnterBarrier build() { + akka.remote.testconductor.TestConductorProtocol.EnterBarrier result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private akka.remote.testconductor.TestConductorProtocol.EnterBarrier buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + akka.remote.testconductor.TestConductorProtocol.EnterBarrier result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public akka.remote.testconductor.TestConductorProtocol.EnterBarrier buildPartial() { + akka.remote.testconductor.TestConductorProtocol.EnterBarrier result = new akka.remote.testconductor.TestConductorProtocol.EnterBarrier(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.name_ = name_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.op_ = op_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.timeout_ = timeout_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof akka.remote.testconductor.TestConductorProtocol.EnterBarrier) { + return mergeFrom((akka.remote.testconductor.TestConductorProtocol.EnterBarrier)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.testconductor.TestConductorProtocol.EnterBarrier other) { + if (other == akka.remote.testconductor.TestConductorProtocol.EnterBarrier.getDefaultInstance()) return this; + if (other.hasName()) { + setName(other.getName()); + } + if (other.hasOp()) { + setOp(other.getOp()); + } + if (other.hasTimeout()) { + setTimeout(other.getTimeout()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasName()) { + + return false; + } + if (!hasOp()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + name_ = input.readBytes(); + break; + } + case 16: { + int rawValue = input.readEnum(); + akka.remote.testconductor.TestConductorProtocol.BarrierOp value = akka.remote.testconductor.TestConductorProtocol.BarrierOp.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(2, rawValue); + } else { + bitField0_ |= 0x00000002; + op_ = value; + } + break; + } + case 24: { + bitField0_ |= 0x00000004; + timeout_ = input.readInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required string name = 1; + private java.lang.Object name_ = ""; + public boolean hasName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + name_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + name_ = value; + onChanged(); + return this; + } + public Builder clearName() { + bitField0_ = (bitField0_ & ~0x00000001); + name_ = getDefaultInstance().getName(); + onChanged(); + return this; + } + void setName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + name_ = value; + onChanged(); + } + + // required .BarrierOp op = 2; + private akka.remote.testconductor.TestConductorProtocol.BarrierOp op_ = akka.remote.testconductor.TestConductorProtocol.BarrierOp.Enter; + public boolean hasOp() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.BarrierOp getOp() { + return op_; + } + public Builder setOp(akka.remote.testconductor.TestConductorProtocol.BarrierOp value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + op_ = value; + onChanged(); + return this; + } + public Builder clearOp() { + bitField0_ = (bitField0_ & ~0x00000002); + op_ = akka.remote.testconductor.TestConductorProtocol.BarrierOp.Enter; + onChanged(); + return this; + } + + // optional int64 timeout = 3; + private long timeout_ ; + public boolean hasTimeout() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTimeout() { + return timeout_; + } + public Builder setTimeout(long value) { + bitField0_ |= 0x00000004; + timeout_ = value; + onChanged(); + return this; + } + public Builder clearTimeout() { + bitField0_ = (bitField0_ & ~0x00000004); + timeout_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:EnterBarrier) + } + + static { + defaultInstance = new EnterBarrier(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:EnterBarrier) + } + + public interface AddressRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string node = 1; + boolean hasNode(); + String getNode(); + + // optional .Address addr = 2; + boolean hasAddr(); + akka.remote.testconductor.TestConductorProtocol.Address getAddr(); + akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder getAddrOrBuilder(); + } + public static final class AddressRequest extends + com.google.protobuf.GeneratedMessage + implements AddressRequestOrBuilder { + // Use AddressRequest.newBuilder() to construct. + private AddressRequest(Builder builder) { + super(builder); + } + private AddressRequest(boolean noInit) {} + + private static final AddressRequest defaultInstance; + public static AddressRequest getDefaultInstance() { + return defaultInstance; + } + + public AddressRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_AddressRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_AddressRequest_fieldAccessorTable; + } + + private int bitField0_; + // required string node = 1; + public static final int NODE_FIELD_NUMBER = 1; + private java.lang.Object node_; + public boolean hasNode() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getNode() { + java.lang.Object ref = node_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + node_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getNodeBytes() { + java.lang.Object ref = node_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + node_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional .Address addr = 2; + public static final int ADDR_FIELD_NUMBER = 2; + private akka.remote.testconductor.TestConductorProtocol.Address addr_; + public boolean hasAddr() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.Address getAddr() { + return addr_; + } + public akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder getAddrOrBuilder() { + return addr_; + } + + private void initFields() { + node_ = ""; + addr_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasNode()) { + memoizedIsInitialized = 0; + return false; + } + if (hasAddr()) { + if (!getAddr().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getNodeBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, addr_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getNodeBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, addr_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.AddressRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.testconductor.TestConductorProtocol.AddressRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.testconductor.TestConductorProtocol.AddressRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_AddressRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_AddressRequest_fieldAccessorTable; + } + + // Construct using akka.remote.testconductor.TestConductorProtocol.AddressRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getAddrFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + node_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + if (addrBuilder_ == null) { + addr_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + } else { + addrBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.testconductor.TestConductorProtocol.AddressRequest.getDescriptor(); + } + + public akka.remote.testconductor.TestConductorProtocol.AddressRequest getDefaultInstanceForType() { + return akka.remote.testconductor.TestConductorProtocol.AddressRequest.getDefaultInstance(); + } + + public akka.remote.testconductor.TestConductorProtocol.AddressRequest build() { + akka.remote.testconductor.TestConductorProtocol.AddressRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private akka.remote.testconductor.TestConductorProtocol.AddressRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + akka.remote.testconductor.TestConductorProtocol.AddressRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public akka.remote.testconductor.TestConductorProtocol.AddressRequest buildPartial() { + akka.remote.testconductor.TestConductorProtocol.AddressRequest result = new akka.remote.testconductor.TestConductorProtocol.AddressRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.node_ = node_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (addrBuilder_ == null) { + result.addr_ = addr_; + } else { + result.addr_ = addrBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof akka.remote.testconductor.TestConductorProtocol.AddressRequest) { + return mergeFrom((akka.remote.testconductor.TestConductorProtocol.AddressRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.testconductor.TestConductorProtocol.AddressRequest other) { + if (other == akka.remote.testconductor.TestConductorProtocol.AddressRequest.getDefaultInstance()) return this; + if (other.hasNode()) { + setNode(other.getNode()); + } + if (other.hasAddr()) { + mergeAddr(other.getAddr()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasNode()) { + + return false; + } + if (hasAddr()) { + if (!getAddr().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + node_ = input.readBytes(); + break; + } + case 18: { + akka.remote.testconductor.TestConductorProtocol.Address.Builder subBuilder = akka.remote.testconductor.TestConductorProtocol.Address.newBuilder(); + if (hasAddr()) { + subBuilder.mergeFrom(getAddr()); + } + input.readMessage(subBuilder, extensionRegistry); + setAddr(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required string node = 1; + private java.lang.Object node_ = ""; + public boolean hasNode() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getNode() { + java.lang.Object ref = node_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + node_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setNode(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + node_ = value; + onChanged(); + return this; + } + public Builder clearNode() { + bitField0_ = (bitField0_ & ~0x00000001); + node_ = getDefaultInstance().getNode(); + onChanged(); + return this; + } + void setNode(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + node_ = value; + onChanged(); + } + + // optional .Address addr = 2; + private akka.remote.testconductor.TestConductorProtocol.Address addr_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Address, akka.remote.testconductor.TestConductorProtocol.Address.Builder, akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder> addrBuilder_; + public boolean hasAddr() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.Address getAddr() { + if (addrBuilder_ == null) { + return addr_; + } else { + return addrBuilder_.getMessage(); + } + } + public Builder setAddr(akka.remote.testconductor.TestConductorProtocol.Address value) { + if (addrBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + addr_ = value; + onChanged(); + } else { + addrBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder setAddr( + akka.remote.testconductor.TestConductorProtocol.Address.Builder builderForValue) { + if (addrBuilder_ == null) { + addr_ = builderForValue.build(); + onChanged(); + } else { + addrBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeAddr(akka.remote.testconductor.TestConductorProtocol.Address value) { + if (addrBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + addr_ != akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance()) { + addr_ = + akka.remote.testconductor.TestConductorProtocol.Address.newBuilder(addr_).mergeFrom(value).buildPartial(); + } else { + addr_ = value; + } + onChanged(); + } else { + addrBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder clearAddr() { + if (addrBuilder_ == null) { + addr_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + onChanged(); + } else { + addrBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + public akka.remote.testconductor.TestConductorProtocol.Address.Builder getAddrBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getAddrFieldBuilder().getBuilder(); + } + public akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder getAddrOrBuilder() { + if (addrBuilder_ != null) { + return addrBuilder_.getMessageOrBuilder(); + } else { + return addr_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Address, akka.remote.testconductor.TestConductorProtocol.Address.Builder, akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder> + getAddrFieldBuilder() { + if (addrBuilder_ == null) { + addrBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Address, akka.remote.testconductor.TestConductorProtocol.Address.Builder, akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder>( + addr_, + getParentForChildren(), + isClean()); + addr_ = null; + } + return addrBuilder_; + } + + // @@protoc_insertion_point(builder_scope:AddressRequest) + } + + static { + defaultInstance = new AddressRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:AddressRequest) + } + + public interface AddressOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string protocol = 1; + boolean hasProtocol(); + String getProtocol(); + + // required string system = 2; + boolean hasSystem(); + String getSystem(); + + // required string host = 3; + boolean hasHost(); + String getHost(); + + // required int32 port = 4; + boolean hasPort(); + int getPort(); + } + public static final class Address extends + com.google.protobuf.GeneratedMessage + implements AddressOrBuilder { + // Use Address.newBuilder() to construct. + private Address(Builder builder) { + super(builder); + } + private Address(boolean noInit) {} + + private static final Address defaultInstance; + public static Address getDefaultInstance() { + return defaultInstance; + } + + public Address getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Address_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Address_fieldAccessorTable; + } + + private int bitField0_; + // required string protocol = 1; + public static final int PROTOCOL_FIELD_NUMBER = 1; + private java.lang.Object protocol_; + public boolean hasProtocol() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getProtocol() { + java.lang.Object ref = protocol_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + protocol_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getProtocolBytes() { + java.lang.Object ref = protocol_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + protocol_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required string system = 2; + public static final int SYSTEM_FIELD_NUMBER = 2; + private java.lang.Object system_; + public boolean hasSystem() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getSystem() { + java.lang.Object ref = system_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + system_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getSystemBytes() { + java.lang.Object ref = system_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + system_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required string host = 3; + public static final int HOST_FIELD_NUMBER = 3; + private java.lang.Object host_; + public boolean hasHost() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getHost() { + java.lang.Object ref = host_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + host_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getHostBytes() { + java.lang.Object ref = host_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + host_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required int32 port = 4; + public static final int PORT_FIELD_NUMBER = 4; + private int port_; + public boolean hasPort() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public int getPort() { + return port_; + } + + private void initFields() { + protocol_ = ""; + system_ = ""; + host_ = ""; + port_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasProtocol()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSystem()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasHost()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasPort()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getProtocolBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getSystemBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getHostBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeInt32(4, port_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getProtocolBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, getSystemBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, getHostBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(4, port_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.testconductor.TestConductorProtocol.Address parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Address parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Address parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Address parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Address parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Address parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Address parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.Address parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.Address parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.Address parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.testconductor.TestConductorProtocol.Address prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Address_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_Address_fieldAccessorTable; + } + + // Construct using akka.remote.testconductor.TestConductorProtocol.Address.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + protocol_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + system_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + host_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + port_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.testconductor.TestConductorProtocol.Address.getDescriptor(); + } + + public akka.remote.testconductor.TestConductorProtocol.Address getDefaultInstanceForType() { + return akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + } + + public akka.remote.testconductor.TestConductorProtocol.Address build() { + akka.remote.testconductor.TestConductorProtocol.Address result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private akka.remote.testconductor.TestConductorProtocol.Address buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + akka.remote.testconductor.TestConductorProtocol.Address result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public akka.remote.testconductor.TestConductorProtocol.Address buildPartial() { + akka.remote.testconductor.TestConductorProtocol.Address result = new akka.remote.testconductor.TestConductorProtocol.Address(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.protocol_ = protocol_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.system_ = system_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.host_ = host_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.port_ = port_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof akka.remote.testconductor.TestConductorProtocol.Address) { + return mergeFrom((akka.remote.testconductor.TestConductorProtocol.Address)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.testconductor.TestConductorProtocol.Address other) { + if (other == akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance()) return this; + if (other.hasProtocol()) { + setProtocol(other.getProtocol()); + } + if (other.hasSystem()) { + setSystem(other.getSystem()); + } + if (other.hasHost()) { + setHost(other.getHost()); + } + if (other.hasPort()) { + setPort(other.getPort()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasProtocol()) { + + return false; + } + if (!hasSystem()) { + + return false; + } + if (!hasHost()) { + + return false; + } + if (!hasPort()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + protocol_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + system_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + host_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + port_ = input.readInt32(); + break; + } + } + } + } + + private int bitField0_; + + // required string protocol = 1; + private java.lang.Object protocol_ = ""; + public boolean hasProtocol() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getProtocol() { + java.lang.Object ref = protocol_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + protocol_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setProtocol(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + protocol_ = value; + onChanged(); + return this; + } + public Builder clearProtocol() { + bitField0_ = (bitField0_ & ~0x00000001); + protocol_ = getDefaultInstance().getProtocol(); + onChanged(); + return this; + } + void setProtocol(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + protocol_ = value; + onChanged(); + } + + // required string system = 2; + private java.lang.Object system_ = ""; + public boolean hasSystem() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getSystem() { + java.lang.Object ref = system_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + system_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setSystem(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + system_ = value; + onChanged(); + return this; + } + public Builder clearSystem() { + bitField0_ = (bitField0_ & ~0x00000002); + system_ = getDefaultInstance().getSystem(); + onChanged(); + return this; + } + void setSystem(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + system_ = value; + onChanged(); + } + + // required string host = 3; + private java.lang.Object host_ = ""; + public boolean hasHost() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getHost() { + java.lang.Object ref = host_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + host_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setHost(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + host_ = value; + onChanged(); + return this; + } + public Builder clearHost() { + bitField0_ = (bitField0_ & ~0x00000004); + host_ = getDefaultInstance().getHost(); + onChanged(); + return this; + } + void setHost(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + host_ = value; + onChanged(); + } + + // required int32 port = 4; + private int port_ ; + public boolean hasPort() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public int getPort() { + return port_; + } + public Builder setPort(int value) { + bitField0_ |= 0x00000008; + port_ = value; + onChanged(); + return this; + } + public Builder clearPort() { + bitField0_ = (bitField0_ & ~0x00000008); + port_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:Address) + } + + static { + defaultInstance = new Address(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:Address) + } + + public interface InjectFailureOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .FailType failure = 1; + boolean hasFailure(); + akka.remote.testconductor.TestConductorProtocol.FailType getFailure(); + + // optional .Direction direction = 2; + boolean hasDirection(); + akka.remote.testconductor.TestConductorProtocol.Direction getDirection(); + + // optional .Address address = 3; + boolean hasAddress(); + akka.remote.testconductor.TestConductorProtocol.Address getAddress(); + akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder getAddressOrBuilder(); + + // optional float rateMBit = 6; + boolean hasRateMBit(); + float getRateMBit(); + + // optional int32 exitValue = 7; + boolean hasExitValue(); + int getExitValue(); + } + public static final class InjectFailure extends + com.google.protobuf.GeneratedMessage + implements InjectFailureOrBuilder { + // Use InjectFailure.newBuilder() to construct. + private InjectFailure(Builder builder) { + super(builder); + } + private InjectFailure(boolean noInit) {} + + private static final InjectFailure defaultInstance; + public static InjectFailure getDefaultInstance() { + return defaultInstance; + } + + public InjectFailure getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_InjectFailure_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_InjectFailure_fieldAccessorTable; + } + + private int bitField0_; + // required .FailType failure = 1; + public static final int FAILURE_FIELD_NUMBER = 1; + private akka.remote.testconductor.TestConductorProtocol.FailType failure_; + public boolean hasFailure() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.testconductor.TestConductorProtocol.FailType getFailure() { + return failure_; + } + + // optional .Direction direction = 2; + public static final int DIRECTION_FIELD_NUMBER = 2; + private akka.remote.testconductor.TestConductorProtocol.Direction direction_; + public boolean hasDirection() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.Direction getDirection() { + return direction_; + } + + // optional .Address address = 3; + public static final int ADDRESS_FIELD_NUMBER = 3; + private akka.remote.testconductor.TestConductorProtocol.Address address_; + public boolean hasAddress() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public akka.remote.testconductor.TestConductorProtocol.Address getAddress() { + return address_; + } + public akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder getAddressOrBuilder() { + return address_; + } + + // optional float rateMBit = 6; + public static final int RATEMBIT_FIELD_NUMBER = 6; + private float rateMBit_; + public boolean hasRateMBit() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public float getRateMBit() { + return rateMBit_; + } + + // optional int32 exitValue = 7; + public static final int EXITVALUE_FIELD_NUMBER = 7; + private int exitValue_; + public boolean hasExitValue() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public int getExitValue() { + return exitValue_; + } + + private void initFields() { + failure_ = akka.remote.testconductor.TestConductorProtocol.FailType.Throttle; + direction_ = akka.remote.testconductor.TestConductorProtocol.Direction.Send; + address_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + rateMBit_ = 0F; + exitValue_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasFailure()) { + memoizedIsInitialized = 0; + return false; + } + if (hasAddress()) { + if (!getAddress().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, failure_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, direction_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, address_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeFloat(6, rateMBit_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeInt32(7, exitValue_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(1, failure_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(2, direction_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(3, address_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeFloatSize(6, rateMBit_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(7, exitValue_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.testconductor.TestConductorProtocol.InjectFailure parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.testconductor.TestConductorProtocol.InjectFailure prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.testconductor.TestConductorProtocol.InjectFailureOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_InjectFailure_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.testconductor.TestConductorProtocol.internal_static_InjectFailure_fieldAccessorTable; + } + + // Construct using akka.remote.testconductor.TestConductorProtocol.InjectFailure.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getAddressFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + failure_ = akka.remote.testconductor.TestConductorProtocol.FailType.Throttle; + bitField0_ = (bitField0_ & ~0x00000001); + direction_ = akka.remote.testconductor.TestConductorProtocol.Direction.Send; + bitField0_ = (bitField0_ & ~0x00000002); + if (addressBuilder_ == null) { + address_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + } else { + addressBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + rateMBit_ = 0F; + bitField0_ = (bitField0_ & ~0x00000008); + exitValue_ = 0; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.testconductor.TestConductorProtocol.InjectFailure.getDescriptor(); + } + + public akka.remote.testconductor.TestConductorProtocol.InjectFailure getDefaultInstanceForType() { + return akka.remote.testconductor.TestConductorProtocol.InjectFailure.getDefaultInstance(); + } + + public akka.remote.testconductor.TestConductorProtocol.InjectFailure build() { + akka.remote.testconductor.TestConductorProtocol.InjectFailure result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private akka.remote.testconductor.TestConductorProtocol.InjectFailure buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + akka.remote.testconductor.TestConductorProtocol.InjectFailure result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public akka.remote.testconductor.TestConductorProtocol.InjectFailure buildPartial() { + akka.remote.testconductor.TestConductorProtocol.InjectFailure result = new akka.remote.testconductor.TestConductorProtocol.InjectFailure(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.failure_ = failure_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.direction_ = direction_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (addressBuilder_ == null) { + result.address_ = address_; + } else { + result.address_ = addressBuilder_.build(); + } + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.rateMBit_ = rateMBit_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.exitValue_ = exitValue_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof akka.remote.testconductor.TestConductorProtocol.InjectFailure) { + return mergeFrom((akka.remote.testconductor.TestConductorProtocol.InjectFailure)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.testconductor.TestConductorProtocol.InjectFailure other) { + if (other == akka.remote.testconductor.TestConductorProtocol.InjectFailure.getDefaultInstance()) return this; + if (other.hasFailure()) { + setFailure(other.getFailure()); + } + if (other.hasDirection()) { + setDirection(other.getDirection()); + } + if (other.hasAddress()) { + mergeAddress(other.getAddress()); + } + if (other.hasRateMBit()) { + setRateMBit(other.getRateMBit()); + } + if (other.hasExitValue()) { + setExitValue(other.getExitValue()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasFailure()) { + + return false; + } + if (hasAddress()) { + if (!getAddress().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 8: { + int rawValue = input.readEnum(); + akka.remote.testconductor.TestConductorProtocol.FailType value = akka.remote.testconductor.TestConductorProtocol.FailType.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(1, rawValue); + } else { + bitField0_ |= 0x00000001; + failure_ = value; + } + break; + } + case 16: { + int rawValue = input.readEnum(); + akka.remote.testconductor.TestConductorProtocol.Direction value = akka.remote.testconductor.TestConductorProtocol.Direction.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(2, rawValue); + } else { + bitField0_ |= 0x00000002; + direction_ = value; + } + break; + } + case 26: { + akka.remote.testconductor.TestConductorProtocol.Address.Builder subBuilder = akka.remote.testconductor.TestConductorProtocol.Address.newBuilder(); + if (hasAddress()) { + subBuilder.mergeFrom(getAddress()); + } + input.readMessage(subBuilder, extensionRegistry); + setAddress(subBuilder.buildPartial()); + break; + } + case 53: { + bitField0_ |= 0x00000008; + rateMBit_ = input.readFloat(); + break; + } + case 56: { + bitField0_ |= 0x00000010; + exitValue_ = input.readInt32(); + break; + } + } + } + } + + private int bitField0_; + + // required .FailType failure = 1; + private akka.remote.testconductor.TestConductorProtocol.FailType failure_ = akka.remote.testconductor.TestConductorProtocol.FailType.Throttle; + public boolean hasFailure() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.testconductor.TestConductorProtocol.FailType getFailure() { + return failure_; + } + public Builder setFailure(akka.remote.testconductor.TestConductorProtocol.FailType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + failure_ = value; + onChanged(); + return this; + } + public Builder clearFailure() { + bitField0_ = (bitField0_ & ~0x00000001); + failure_ = akka.remote.testconductor.TestConductorProtocol.FailType.Throttle; + onChanged(); + return this; + } + + // optional .Direction direction = 2; + private akka.remote.testconductor.TestConductorProtocol.Direction direction_ = akka.remote.testconductor.TestConductorProtocol.Direction.Send; + public boolean hasDirection() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.testconductor.TestConductorProtocol.Direction getDirection() { + return direction_; + } + public Builder setDirection(akka.remote.testconductor.TestConductorProtocol.Direction value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + direction_ = value; + onChanged(); + return this; + } + public Builder clearDirection() { + bitField0_ = (bitField0_ & ~0x00000002); + direction_ = akka.remote.testconductor.TestConductorProtocol.Direction.Send; + onChanged(); + return this; + } + + // optional .Address address = 3; + private akka.remote.testconductor.TestConductorProtocol.Address address_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Address, akka.remote.testconductor.TestConductorProtocol.Address.Builder, akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder> addressBuilder_; + public boolean hasAddress() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public akka.remote.testconductor.TestConductorProtocol.Address getAddress() { + if (addressBuilder_ == null) { + return address_; + } else { + return addressBuilder_.getMessage(); + } + } + public Builder setAddress(akka.remote.testconductor.TestConductorProtocol.Address value) { + if (addressBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + address_ = value; + onChanged(); + } else { + addressBuilder_.setMessage(value); + } + bitField0_ |= 0x00000004; + return this; + } + public Builder setAddress( + akka.remote.testconductor.TestConductorProtocol.Address.Builder builderForValue) { + if (addressBuilder_ == null) { + address_ = builderForValue.build(); + onChanged(); + } else { + addressBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; + return this; + } + public Builder mergeAddress(akka.remote.testconductor.TestConductorProtocol.Address value) { + if (addressBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + address_ != akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance()) { + address_ = + akka.remote.testconductor.TestConductorProtocol.Address.newBuilder(address_).mergeFrom(value).buildPartial(); + } else { + address_ = value; + } + onChanged(); + } else { + addressBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000004; + return this; + } + public Builder clearAddress() { + if (addressBuilder_ == null) { + address_ = akka.remote.testconductor.TestConductorProtocol.Address.getDefaultInstance(); + onChanged(); + } else { + addressBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + public akka.remote.testconductor.TestConductorProtocol.Address.Builder getAddressBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getAddressFieldBuilder().getBuilder(); + } + public akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder getAddressOrBuilder() { + if (addressBuilder_ != null) { + return addressBuilder_.getMessageOrBuilder(); + } else { + return address_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Address, akka.remote.testconductor.TestConductorProtocol.Address.Builder, akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder> + getAddressFieldBuilder() { + if (addressBuilder_ == null) { + addressBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.testconductor.TestConductorProtocol.Address, akka.remote.testconductor.TestConductorProtocol.Address.Builder, akka.remote.testconductor.TestConductorProtocol.AddressOrBuilder>( + address_, + getParentForChildren(), + isClean()); + address_ = null; + } + return addressBuilder_; + } + + // optional float rateMBit = 6; + private float rateMBit_ ; + public boolean hasRateMBit() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public float getRateMBit() { + return rateMBit_; + } + public Builder setRateMBit(float value) { + bitField0_ |= 0x00000008; + rateMBit_ = value; + onChanged(); + return this; + } + public Builder clearRateMBit() { + bitField0_ = (bitField0_ & ~0x00000008); + rateMBit_ = 0F; + onChanged(); + return this; + } + + // optional int32 exitValue = 7; + private int exitValue_ ; + public boolean hasExitValue() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public int getExitValue() { + return exitValue_; + } + public Builder setExitValue(int value) { + bitField0_ |= 0x00000010; + exitValue_ = value; + onChanged(); + return this; + } + public Builder clearExitValue() { + bitField0_ = (bitField0_ & ~0x00000010); + exitValue_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:InjectFailure) + } + + static { + defaultInstance = new InjectFailure(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:InjectFailure) + } + + private static com.google.protobuf.Descriptors.Descriptor + internal_static_Wrapper_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_Wrapper_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_Hello_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_Hello_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_EnterBarrier_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_EnterBarrier_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_AddressRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_AddressRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_Address_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_Address_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_InjectFailure_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_InjectFailure_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\033TestConductorProtocol.proto\"\216\001\n\007Wrappe" + + "r\022\025\n\005hello\030\001 \001(\0132\006.Hello\022\036\n\007barrier\030\002 \001(" + + "\0132\r.EnterBarrier\022\037\n\007failure\030\003 \001(\0132\016.Inje" + + "ctFailure\022\014\n\004done\030\004 \001(\t\022\035\n\004addr\030\005 \001(\0132\017." + + "AddressRequest\"0\n\005Hello\022\014\n\004name\030\001 \002(\t\022\031\n" + + "\007address\030\002 \002(\0132\010.Address\"E\n\014EnterBarrier" + + "\022\014\n\004name\030\001 \002(\t\022\026\n\002op\030\002 \002(\0162\n.BarrierOp\022\017" + + "\n\007timeout\030\003 \001(\003\"6\n\016AddressRequest\022\014\n\004nod" + + "e\030\001 \002(\t\022\026\n\004addr\030\002 \001(\0132\010.Address\"G\n\007Addre" + + "ss\022\020\n\010protocol\030\001 \002(\t\022\016\n\006system\030\002 \002(\t\022\014\n\004", + "host\030\003 \002(\t\022\014\n\004port\030\004 \002(\005\"\212\001\n\rInjectFailu" + + "re\022\032\n\007failure\030\001 \002(\0162\t.FailType\022\035\n\tdirect" + + "ion\030\002 \001(\0162\n.Direction\022\031\n\007address\030\003 \001(\0132\010" + + ".Address\022\020\n\010rateMBit\030\006 \001(\002\022\021\n\texitValue\030" + + "\007 \001(\005*;\n\tBarrierOp\022\t\n\005Enter\020\001\022\010\n\004Fail\020\002\022" + + "\r\n\tSucceeded\020\003\022\n\n\006Failed\020\004*A\n\010FailType\022\014" + + "\n\010Throttle\020\001\022\016\n\nDisconnect\020\002\022\t\n\005Abort\020\003\022" + + "\014\n\010Shutdown\020\004*,\n\tDirection\022\010\n\004Send\020\001\022\013\n\007" + + "Receive\020\002\022\010\n\004Both\020\003B\035\n\031akka.remote.testc" + + "onductorH\001" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + internal_static_Wrapper_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_Wrapper_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Wrapper_descriptor, + new java.lang.String[] { "Hello", "Barrier", "Failure", "Done", "Addr", }, + akka.remote.testconductor.TestConductorProtocol.Wrapper.class, + akka.remote.testconductor.TestConductorProtocol.Wrapper.Builder.class); + internal_static_Hello_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_Hello_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Hello_descriptor, + new java.lang.String[] { "Name", "Address", }, + akka.remote.testconductor.TestConductorProtocol.Hello.class, + akka.remote.testconductor.TestConductorProtocol.Hello.Builder.class); + internal_static_EnterBarrier_descriptor = + getDescriptor().getMessageTypes().get(2); + internal_static_EnterBarrier_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_EnterBarrier_descriptor, + new java.lang.String[] { "Name", "Op", "Timeout", }, + akka.remote.testconductor.TestConductorProtocol.EnterBarrier.class, + akka.remote.testconductor.TestConductorProtocol.EnterBarrier.Builder.class); + internal_static_AddressRequest_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_AddressRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_AddressRequest_descriptor, + new java.lang.String[] { "Node", "Addr", }, + akka.remote.testconductor.TestConductorProtocol.AddressRequest.class, + akka.remote.testconductor.TestConductorProtocol.AddressRequest.Builder.class); + internal_static_Address_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_Address_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_Address_descriptor, + new java.lang.String[] { "Protocol", "System", "Host", "Port", }, + akka.remote.testconductor.TestConductorProtocol.Address.class, + akka.remote.testconductor.TestConductorProtocol.Address.Builder.class); + internal_static_InjectFailure_descriptor = + getDescriptor().getMessageTypes().get(5); + internal_static_InjectFailure_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_InjectFailure_descriptor, + new java.lang.String[] { "Failure", "Direction", "Address", "RateMBit", "ExitValue", }, + akka.remote.testconductor.TestConductorProtocol.InjectFailure.class, + akka.remote.testconductor.TestConductorProtocol.InjectFailure.Builder.class); + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/akka-remote-tests/src/main/protocol/TestConductorProtocol.proto b/akka-remote-tests/src/main/protocol/TestConductorProtocol.proto new file mode 100644 index 0000000000..1ff8a83c24 --- /dev/null +++ b/akka-remote-tests/src/main/protocol/TestConductorProtocol.proto @@ -0,0 +1,74 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ + +option java_package = "akka.remote.testconductor"; +option optimize_for = SPEED; + +/****************************************** + Compile with: + cd ./akka-remote-tests/src/main/protocol + protoc TestConductorProtocol.proto --java_out ../java + cd ../../../.. + ./scripts/fix-protobuf.sh +*******************************************/ + +message Wrapper { + optional Hello hello = 1; + optional EnterBarrier barrier = 2; + optional InjectFailure failure = 3; + optional string done = 4; + optional AddressRequest addr = 5; +} + +message Hello { + required string name = 1; + required Address address = 2; +} + +enum BarrierOp { + Enter = 1; + Fail = 2; + Succeeded = 3; + Failed = 4; +} + +message EnterBarrier { + required string name = 1; + required BarrierOp op = 2; + optional int64 timeout = 3; +} + +message AddressRequest { + required string node = 1; + optional Address addr = 2; +} + +message Address { + required string protocol = 1; + required string system = 2; + required string host = 3; + required int32 port = 4; +} + +enum FailType { + Throttle = 1; + Disconnect = 2; + Abort = 3; + Shutdown = 4; +} + +enum Direction { + Send = 1; + Receive = 2; + Both = 3; +} + +message InjectFailure { + required FailType failure = 1; + optional Direction direction = 2; + optional Address address = 3; + optional float rateMBit = 6; + optional int32 exitValue = 7; +} + diff --git a/akka-remote-tests/src/main/resources/reference.conf b/akka-remote-tests/src/main/resources/reference.conf new file mode 100644 index 0000000000..40c16c4ccd --- /dev/null +++ b/akka-remote-tests/src/main/resources/reference.conf @@ -0,0 +1,33 @@ +############################################# +# Akka Remote Testing Reference Config File # +############################################# + +# This is the reference config file that contains all the default settings. +# Make your edits/overrides in your application.conf. + +akka { + testconductor { + + # Timeout for joining a barrier: this is the maximum time any participants + # waits for everybody else to join a named barrier. + barrier-timeout = 30s + + # Timeout for interrogation of TestConductor’s Controller actor + query-timeout = 5s + + # Threshold for packet size in time unit above which the failure injector will + # split the packet and deliver in smaller portions; do not give value smaller + # than HashedWheelTimer resolution (would not make sense) + packet-split-threshold = 100ms + + # amount of time for the ClientFSM to wait for the connection to the conductor + # to be successful + connect-timeout = 20s + + # Number of connect attempts to be made to the conductor controller + client-reconnects = 10 + + # minimum time interval which is to be inserted between reconnect attempts + reconnect-backoff = 1s + } +} \ No newline at end of file diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala new file mode 100644 index 0000000000..24377d54a1 --- /dev/null +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala @@ -0,0 +1,609 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.testconductor + +import akka.actor.{ Actor, ActorRef, ActorSystem, LoggingFSM, Props } +import RemoteConnection.getAddrString +import TestConductorProtocol._ +import org.jboss.netty.channel.{ Channel, SimpleChannelUpstreamHandler, ChannelHandlerContext, ChannelStateEvent, MessageEvent } +import com.typesafe.config.ConfigFactory +import akka.util.duration._ +import akka.pattern.ask +import java.util.concurrent.TimeUnit.MILLISECONDS +import akka.dispatch.Await +import akka.event.LoggingAdapter +import akka.actor.PoisonPill +import akka.event.Logging +import scala.util.control.NoStackTrace +import akka.event.LoggingReceive +import akka.actor.Address +import java.net.InetSocketAddress +import akka.dispatch.Future +import akka.actor.OneForOneStrategy +import akka.actor.SupervisorStrategy +import java.util.concurrent.ConcurrentHashMap +import akka.actor.Status +import akka.util.{ Deadline, Timeout, Duration } + +sealed trait Direction { + def includes(other: Direction): Boolean +} + +object Direction { + case object Send extends Direction { + override def includes(other: Direction): Boolean = other match { + case Send ⇒ true + case _ ⇒ false + } + } + case object Receive extends Direction { + override def includes(other: Direction): Boolean = other match { + case Receive ⇒ true + case _ ⇒ false + } + } + case object Both extends Direction { + override def includes(other: Direction): Boolean = true + } +} + +/** + * The conductor is the one orchestrating the test: it governs the + * [[akka.remote.testconductor.Controller]]’s port to which all + * [[akka.remote.testconductor.Player]]s connect, it issues commands to their + * [[akka.remote.testconductor.NetworkFailureInjector]] and provides support + * for barriers using the [[akka.remote.testconductor.BarrierCoordinator]]. + * All of this is bundled inside the [[akka.remote.testconductor.TestConductorExt]] + * extension. + */ +trait Conductor { this: TestConductorExt ⇒ + + import Controller._ + + private var _controller: ActorRef = _ + private def controller: ActorRef = _controller match { + case null ⇒ throw new IllegalStateException("TestConductorServer was not started") + case x ⇒ x + } + + /** + * Start the [[akka.remote.testconductor.Controller]], which in turn will + * bind to a TCP port as specified in the `akka.testconductor.port` config + * property, where 0 denotes automatic allocation. Since the latter is + * actually preferred, a `Future[Int]` is returned which will be completed + * with the port number actually chosen, so that this can then be communicated + * to the players for their proper start-up. + * + * This method also invokes [[akka.remote.testconductor.Player]].startClient, + * since it is expected that the conductor participates in barriers for + * overall coordination. The returned Future will only be completed once the + * client’s start-up finishes, which in fact waits for all other players to + * connect. + * + * @param participants gives the number of participants which shall connect + * before any of their startClient() operations complete. + */ + def startController(participants: Int, name: RoleName, controllerPort: InetSocketAddress): Future[InetSocketAddress] = { + if (_controller ne null) throw new RuntimeException("TestConductorServer was already started") + _controller = system.actorOf(Props(new Controller(participants, controllerPort)), "controller") + import Settings.BarrierTimeout + controller ? GetSockAddr flatMap { case sockAddr: InetSocketAddress ⇒ startClient(name, sockAddr) map (_ ⇒ sockAddr) } + } + + /** + * Obtain the port to which the controller’s socket is actually bound. This + * will deviate from the configuration in `akka.testconductor.port` in case + * that was given as zero. + */ + def sockAddr: Future[InetSocketAddress] = { + import Settings.QueryTimeout + controller ? GetSockAddr mapTo + } + + /** + * Make the remoting pipeline on the node throttle data sent to or received + * from the given remote peer. Throttling works by delaying packet submission + * within the netty pipeline until the packet would have been completely sent + * according to the given rate, the previous packet completion and the current + * packet length. In case of large packets they are split up if the calculated + * send pause would exceed `akka.testconductor.packet-split-threshold` + * (roughly). All of this uses the system’s HashedWheelTimer, which is not + * terribly precise and will execute tasks later than they are schedule (even + * on average), but that is countered by using the actual execution time for + * determining how much to send, leading to the correct output rate, but with + * increased latency. + * + * @param node is the symbolic name of the node which is to be affected + * @param target is the symbolic name of the other node to which connectivity shall be throttled + * @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both` + * @param rateMBit is the maximum data rate in MBit + */ + def throttle(node: RoleName, target: RoleName, direction: Direction, rateMBit: Double): Future[Done] = { + import Settings.QueryTimeout + controller ? Throttle(node, target, direction, rateMBit.toFloat) mapTo + } + + /** + * Switch the Netty pipeline of the remote support into blackhole mode for + * sending and/or receiving: it will just drop all messages right before + * submitting them to the Socket or right after receiving them from the + * Socket. + * + * @param node is the symbolic name of the node which is to be affected + * @param target is the symbolic name of the other node to which connectivity shall be impeded + * @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both` + */ + def blackhole(node: RoleName, target: RoleName, direction: Direction): Future[Done] = { + import Settings.QueryTimeout + controller ? Throttle(node, target, direction, 0f) mapTo + } + + /** + * Switch the Netty pipeline of the remote support into pass through mode for + * sending and/or receiving. + * + * @param node is the symbolic name of the node which is to be affected + * @param target is the symbolic name of the other node to which connectivity shall be impeded + * @param direction can be either `Direction.Send`, `Direction.Receive` or `Direction.Both` + */ + def passThrough(node: RoleName, target: RoleName, direction: Direction): Future[Done] = { + import Settings.QueryTimeout + controller ? Throttle(node, target, direction, -1f) mapTo + } + + /** + * Tell the remote support to shutdown the connection to the given remote + * peer. It works regardless of whether the recipient was initiator or + * responder. + * + * @param node is the symbolic name of the node which is to be affected + * @param target is the symbolic name of the other node to which connectivity shall be impeded + */ + def disconnect(node: RoleName, target: RoleName): Future[Done] = { + import Settings.QueryTimeout + controller ? Disconnect(node, target, false) mapTo + } + + /** + * Tell the remote support to TCP_RESET the connection to the given remote + * peer. It works regardless of whether the recipient was initiator or + * responder. + * + * @param node is the symbolic name of the node which is to be affected + * @param target is the symbolic name of the other node to which connectivity shall be impeded + */ + def abort(node: RoleName, target: RoleName): Future[Done] = { + import Settings.QueryTimeout + controller ? Disconnect(node, target, true) mapTo + } + + /** + * Tell the remote node to shut itself down using System.exit with the given + * exitValue. The node will also be removed, so that the remaining nodes may still + * pass subsequent barriers. + * + * @param node is the symbolic name of the node which is to be affected + * @param exitValue is the return code which shall be given to System.exit + */ + def shutdown(node: RoleName, exitValue: Int): Future[Done] = { + import Settings.QueryTimeout + controller ? Terminate(node, exitValue) mapTo + } + + /** + * Tell the SBT plugin to forcibly terminate the given remote node using Process.destroy. + * + * @param node is the symbolic name of the node which is to be affected + */ + // TODO: uncomment (and implement in Controller) if really needed + // def kill(node: RoleName): Future[Done] = { + // import Settings.QueryTimeout + // controller ? Terminate(node, -1) mapTo + // } + + /** + * Obtain the list of remote host names currently registered. + */ + def getNodes: Future[Iterable[RoleName]] = { + import Settings.QueryTimeout + controller ? GetNodes mapTo + } + + /** + * Remove a remote host from the list, so that the remaining nodes may still + * pass subsequent barriers. This must be done before the client connection + * breaks down in order to affect an “orderly” removal (i.e. without failing + * present and future barriers). + * + * @param node is the symbolic name of the node which is to be removed + */ + def removeNode(node: RoleName): Future[Done] = { + import Settings.QueryTimeout + controller ? Remove(node) mapTo + } + +} + +/** + * This handler is installed at the end of the controller’s netty pipeline. Its only + * purpose is to dispatch incoming messages to the right ServerFSM actor. There is + * one shared instance of this class for all connections accepted by one Controller. + * + * INTERNAL API. + */ +private[akka] class ConductorHandler(_createTimeout: Timeout, controller: ActorRef, log: LoggingAdapter) extends SimpleChannelUpstreamHandler { + + implicit val createTimeout = _createTimeout + val clients = new ConcurrentHashMap[Channel, ActorRef]() + + override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { + val channel = event.getChannel + log.debug("connection from {}", getAddrString(channel)) + val fsm: ActorRef = Await.result(controller ? Controller.CreateServerFSM(channel) mapTo, Duration.Inf) + clients.put(channel, fsm) + } + + override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { + val channel = event.getChannel + log.debug("disconnect from {}", getAddrString(channel)) + val fsm = clients.get(channel) + fsm ! Controller.ClientDisconnected + clients.remove(channel) + } + + override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = { + val channel = event.getChannel + log.debug("message from {}: {}", getAddrString(channel), event.getMessage) + event.getMessage match { + case msg: NetworkOp ⇒ + clients.get(channel) ! msg + case msg ⇒ + log.info("client {} sent garbage '{}', disconnecting", getAddrString(channel), msg) + channel.close() + } + } + +} + +/** + * INTERNAL API. + */ +private[akka] object ServerFSM { + sealed trait State + case object Initial extends State + case object Ready extends State +} + +/** + * The server part of each client connection is represented by a ServerFSM. + * The Initial state handles reception of the new client’s + * [[akka.remote.testconductor.Hello]] message (which is needed for all subsequent + * node name translations). + * + * In the Ready state, messages from the client are forwarded to the controller + * and [[akka.remote.testconductor.Send]] requests are sent, but the latter is + * treated specially: all client operations are to be confirmed by a + * [[akka.remote.testconductor.Done]] message, and there can be only one such + * request outstanding at a given time (i.e. a Send fails if the previous has + * not yet been acknowledged). + * + * INTERNAL API. + */ +private[akka] class ServerFSM(val controller: ActorRef, val channel: Channel) extends Actor with LoggingFSM[ServerFSM.State, Option[ActorRef]] { + import ServerFSM._ + import akka.actor.FSM._ + import Controller._ + + var roleName: RoleName = null + + startWith(Initial, None) + + whenUnhandled { + case Event(ClientDisconnected, Some(s)) ⇒ + s ! Status.Failure(new RuntimeException("client disconnected in state " + stateName + ": " + channel)) + stop() + case Event(ClientDisconnected, None) ⇒ stop() + } + + onTermination { + case _ ⇒ + controller ! ClientDisconnected(roleName) + channel.close() + } + + when(Initial, stateTimeout = 10 seconds) { + case Event(Hello(name, addr), _) ⇒ + roleName = RoleName(name) + controller ! NodeInfo(roleName, addr, self) + goto(Ready) + case Event(x: NetworkOp, _) ⇒ + log.warning("client {} sent no Hello in first message (instead {}), disconnecting", getAddrString(channel), x) + channel.close() + stop() + case Event(ToClient(msg), _) ⇒ + log.warning("cannot send {} in state Initial", msg) + stay + case Event(StateTimeout, _) ⇒ + log.info("closing channel to {} because of Hello timeout", getAddrString(channel)) + channel.close() + stop() + } + + when(Ready) { + case Event(d: Done, Some(s)) ⇒ + s ! d + stay using None + case Event(op: ServerOp, _) ⇒ + controller ! op + stay + case Event(msg: NetworkOp, _) ⇒ + log.warning("client {} sent unsupported message {}", getAddrString(channel), msg) + stop() + case Event(ToClient(msg: UnconfirmedClientOp), _) ⇒ + channel.write(msg) + stay + case Event(ToClient(msg), None) ⇒ + channel.write(msg) + stay using Some(sender) + case Event(ToClient(msg), _) ⇒ + log.warning("cannot send {} while waiting for previous ACK", msg) + stay + } + + initialize +} + +/** + * INTERNAL API. + */ +private[akka] object Controller { + case class ClientDisconnected(name: RoleName) + case object GetNodes + case object GetSockAddr + case class CreateServerFSM(channel: Channel) + + case class NodeInfo(name: RoleName, addr: Address, fsm: ActorRef) +} + +/** + * This controls test execution by managing barriers (delegated to + * [[akka.remote.testconductor.BarrierCoordinator]], its child) and allowing + * network and other failures to be injected at the test nodes. + * + * INTERNAL API. + */ +private[akka] class Controller(private var initialParticipants: Int, controllerPort: InetSocketAddress) extends Actor { + import Controller._ + import BarrierCoordinator._ + + val settings = TestConductor().Settings + val connection = RemoteConnection(Server, controllerPort, + new ConductorHandler(settings.QueryTimeout, self, Logging(context.system, "ConductorHandler"))) + + /* + * Supervision of the BarrierCoordinator means to catch all his bad emotions + * and sometimes console him (BarrierEmpty, BarrierTimeout), sometimes tell + * him to hate the world (WrongBarrier, DuplicateNode, ClientLost). The latter shall help + * terminate broken tests as quickly as possible (i.e. without awaiting + * BarrierTimeouts in the players). + */ + override def supervisorStrategy = OneForOneStrategy() { + case BarrierTimeout(data) ⇒ failBarrier(data) + case FailedBarrier(data) ⇒ failBarrier(data) + case BarrierEmpty(data, msg) ⇒ SupervisorStrategy.Resume + case WrongBarrier(name, client, data) ⇒ client ! ToClient(BarrierResult(name, false)); failBarrier(data) + case ClientLost(data, node) ⇒ failBarrier(data) + case DuplicateNode(data, node) ⇒ failBarrier(data) + } + + def failBarrier(data: Data): SupervisorStrategy.Directive = { + for (c ← data.arrived) c ! ToClient(BarrierResult(data.barrier, false)) + SupervisorStrategy.Restart + } + + val barrier = context.actorOf(Props[BarrierCoordinator], "barriers") + var nodes = Map[RoleName, NodeInfo]() + + // map keeping unanswered queries for node addresses (enqueued upon GetAddress, serviced upon NodeInfo) + var addrInterest = Map[RoleName, Set[ActorRef]]() + val generation = Iterator from 1 + + override def receive = LoggingReceive { + case CreateServerFSM(channel) ⇒ + val (ip, port) = channel.getRemoteAddress match { case s: InetSocketAddress ⇒ (s.getAddress.getHostAddress, s.getPort) } + val name = ip + ":" + port + "-server" + generation.next + sender ! context.actorOf(Props(new ServerFSM(self, channel)), name) + case c @ NodeInfo(name, addr, fsm) ⇒ + barrier forward c + if (nodes contains name) { + if (initialParticipants > 0) { + for (NodeInfo(_, _, client) ← nodes.values) client ! ToClient(BarrierResult("initial startup", false)) + initialParticipants = 0 + } + fsm ! ToClient(BarrierResult("initial startup", false)) + } else { + nodes += name -> c + if (initialParticipants <= 0) fsm ! ToClient(Done) + else if (nodes.size == initialParticipants) { + for (NodeInfo(_, _, client) ← nodes.values) client ! ToClient(Done) + initialParticipants = 0 + } + if (addrInterest contains name) { + addrInterest(name) foreach (_ ! ToClient(AddressReply(name, addr))) + addrInterest -= name + } + } + case c @ ClientDisconnected(name) ⇒ + nodes -= name + barrier forward c + case op: ServerOp ⇒ + op match { + case _: EnterBarrier ⇒ barrier forward op + case _: FailBarrier ⇒ barrier forward op + case GetAddress(node) ⇒ + if (nodes contains node) sender ! ToClient(AddressReply(node, nodes(node).addr)) + else addrInterest += node -> ((addrInterest get node getOrElse Set()) + sender) + } + case op: CommandOp ⇒ + op match { + case Throttle(node, target, direction, rateMBit) ⇒ + val t = nodes(target) + nodes(node).fsm forward ToClient(ThrottleMsg(t.addr, direction, rateMBit)) + case Disconnect(node, target, abort) ⇒ + val t = nodes(target) + nodes(node).fsm forward ToClient(DisconnectMsg(t.addr, abort)) + case Terminate(node, exitValueOrKill) ⇒ + if (exitValueOrKill < 0) { + // TODO: kill via SBT + } else { + barrier ! BarrierCoordinator.RemoveClient(node) + nodes(node).fsm forward ToClient(TerminateMsg(exitValueOrKill)) + } + case Remove(node) ⇒ + barrier ! BarrierCoordinator.RemoveClient(node) + } + case GetNodes ⇒ sender ! nodes.keys + case GetSockAddr ⇒ sender ! connection.getLocalAddress + } +} + +/** + * INTERNAL API. + */ +private[akka] object BarrierCoordinator { + sealed trait State + case object Idle extends State + case object Waiting extends State + + case class RemoveClient(name: RoleName) + + case class Data(clients: Set[Controller.NodeInfo], barrier: String, arrived: List[ActorRef], deadline: Deadline) + + trait Printer { this: Product with Throwable with NoStackTrace ⇒ + override def toString = productPrefix + productIterator.mkString("(", ", ", ")") + } + + case class BarrierTimeout(data: Data) + extends RuntimeException("timeout while waiting for barrier '" + data.barrier + "'") with NoStackTrace with Printer + case class FailedBarrier(data: Data) + extends RuntimeException("failing barrier '" + data.barrier + "'") with NoStackTrace with Printer + case class DuplicateNode(data: Data, node: Controller.NodeInfo) + extends RuntimeException(node.toString) with NoStackTrace with Printer + case class WrongBarrier(barrier: String, client: ActorRef, data: Data) + extends RuntimeException(data.clients.find(_.fsm == client).map(_.name.toString).getOrElse(client.toString) + + " tried to enter '" + barrier + "' while we were waiting for '" + data.barrier + "'") with NoStackTrace with Printer + case class BarrierEmpty(data: Data, msg: String) extends RuntimeException(msg) with NoStackTrace with Printer + case class ClientLost(data: Data, client: RoleName) + extends RuntimeException("unannounced disconnect of " + client) with NoStackTrace with Printer +} + +/** + * This barrier coordinator gets informed of players connecting (NodeInfo), + * players being deliberately removed (RemoveClient) or failing (ClientDisconnected) + * by the controller. It also receives EnterBarrier requests, where upon the first + * one received the name of the current barrier is set and all other known clients + * are expected to join the barrier, whereupon all of the will be sent the successful + * EnterBarrier return message. In case of planned removals, this may just happen + * earlier, in case of failures the current barrier (and all subsequent ones) will + * be failed by sending BarrierFailed responses. + * + * INTERNAL API. + */ +private[akka] class BarrierCoordinator extends Actor with LoggingFSM[BarrierCoordinator.State, BarrierCoordinator.Data] { + import BarrierCoordinator._ + import akka.actor.FSM._ + import Controller._ + import akka.util.{ Timeout ⇒ auTimeout } + + // this shall be set to true if all subsequent barriers shall fail + var failed = false + + override def preRestart(reason: Throwable, message: Option[Any]) {} + override def postRestart(reason: Throwable) { failed = true } + + // TODO what happens with the other waiting players in case of a test failure? + + startWith(Idle, Data(Set(), "", Nil, null)) + + whenUnhandled { + case Event(n: NodeInfo, d @ Data(clients, _, _, _)) ⇒ + if (clients.find(_.name == n.name).isDefined) throw new DuplicateNode(d, n) + stay using d.copy(clients = clients + n) + case Event(ClientDisconnected(name), d @ Data(clients, _, arrived, _)) ⇒ + if (arrived.isEmpty) + stay using d.copy(clients = clients.filterNot(_.name == name)) + else { + (clients find (_.name == name)) match { + case None ⇒ stay + case Some(c) ⇒ throw ClientLost(d.copy(clients = clients - c, arrived = arrived filterNot (_ == c.fsm)), name) + } + } + } + + when(Idle) { + case Event(EnterBarrier(name, timeout), d @ Data(clients, _, _, _)) ⇒ + if (failed) + stay replying ToClient(BarrierResult(name, false)) + else if (clients.map(_.fsm) == Set(sender)) + stay replying ToClient(BarrierResult(name, true)) + else if (clients.find(_.fsm == sender).isEmpty) + stay replying ToClient(BarrierResult(name, false)) + else { + goto(Waiting) using d.copy(barrier = name, arrived = sender :: Nil, + deadline = getDeadline(timeout)) + } + case Event(RemoveClient(name), d @ Data(clients, _, _, _)) ⇒ + if (clients.isEmpty) throw BarrierEmpty(d, "cannot remove " + name + ": no client to remove") + stay using d.copy(clients = clients filterNot (_.name == name)) + } + + onTransition { + case Idle -> Waiting ⇒ setTimer("Timeout", StateTimeout, nextStateData.deadline.timeLeft, false) + case Waiting -> Idle ⇒ cancelTimer("Timeout") + } + + when(Waiting) { + case Event(EnterBarrier(name, timeout), d @ Data(clients, barrier, arrived, deadline)) ⇒ + if (name != barrier) throw WrongBarrier(name, sender, d) + val together = if (clients.exists(_.fsm == sender)) sender :: arrived else arrived + val enterDeadline = getDeadline(timeout) + // we only allow the deadlines to get shorter + if (enterDeadline < deadline) { + setTimer("Timeout", StateTimeout, enterDeadline.timeLeft, false) + handleBarrier(d.copy(arrived = together, deadline = enterDeadline)) + } else + handleBarrier(d.copy(arrived = together)) + case Event(RemoveClient(name), d @ Data(clients, barrier, arrived, _)) ⇒ + clients find (_.name == name) match { + case None ⇒ stay + case Some(client) ⇒ + handleBarrier(d.copy(clients = clients - client, arrived = arrived filterNot (_ == client.fsm))) + } + case Event(FailBarrier(name), d @ Data(_, barrier, _, _)) ⇒ + if (name != barrier) throw WrongBarrier(name, sender, d) + throw FailedBarrier(d) + case Event(StateTimeout, d) ⇒ + throw BarrierTimeout(d) + } + + initialize + + def handleBarrier(data: Data): State = { + log.debug("handleBarrier({})", data) + if (data.arrived.isEmpty) { + goto(Idle) using data.copy(barrier = "") + } else if ((data.clients.map(_.fsm) -- data.arrived).isEmpty) { + data.arrived foreach (_ ! ToClient(BarrierResult(data.barrier, true))) + goto(Idle) using data.copy(barrier = "", arrived = Nil) + } else { + stay using data + } + } + + def getDeadline(timeout: Option[Duration]): Deadline = { + Deadline.now + timeout.getOrElse(TestConductor().Settings.BarrierTimeout.duration) + } + +} + diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/DataTypes.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/DataTypes.scala new file mode 100644 index 0000000000..830b32e485 --- /dev/null +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/DataTypes.scala @@ -0,0 +1,153 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.remote.testconductor + +import org.jboss.netty.handler.codec.oneone.OneToOneEncoder +import org.jboss.netty.channel.ChannelHandlerContext +import org.jboss.netty.channel.Channel +import akka.remote.testconductor.{ TestConductorProtocol ⇒ TCP } +import com.google.protobuf.Message +import akka.actor.Address +import org.jboss.netty.handler.codec.oneone.OneToOneDecoder +import akka.util.Duration +import akka.remote.testconductor.TestConductorProtocol.BarrierOp + +case class RoleName(name: String) + +private[akka] case class ToClient(msg: ClientOp with NetworkOp) +private[akka] case class ToServer(msg: ServerOp with NetworkOp) + +private[akka] sealed trait ClientOp // messages sent to from Conductor to Player +private[akka] sealed trait ServerOp // messages sent to from Player to Conductor +private[akka] sealed trait CommandOp // messages sent from TestConductorExt to Conductor +private[akka] sealed trait NetworkOp // messages sent over the wire +private[akka] sealed trait UnconfirmedClientOp extends ClientOp // unconfirmed messages going to the Player +private[akka] sealed trait ConfirmedClientOp extends ClientOp + +/** + * First message of connection sets names straight. + */ +private[akka] case class Hello(name: String, addr: Address) extends NetworkOp + +private[akka] case class EnterBarrier(name: String, timeout: Option[Duration]) extends ServerOp with NetworkOp +private[akka] case class FailBarrier(name: String) extends ServerOp with NetworkOp +private[akka] case class BarrierResult(name: String, success: Boolean) extends UnconfirmedClientOp with NetworkOp + +private[akka] case class Throttle(node: RoleName, target: RoleName, direction: Direction, rateMBit: Float) extends CommandOp +private[akka] case class ThrottleMsg(target: Address, direction: Direction, rateMBit: Float) extends ConfirmedClientOp with NetworkOp + +private[akka] case class Disconnect(node: RoleName, target: RoleName, abort: Boolean) extends CommandOp +private[akka] case class DisconnectMsg(target: Address, abort: Boolean) extends ConfirmedClientOp with NetworkOp + +private[akka] case class Terminate(node: RoleName, exitValueOrKill: Int) extends CommandOp +private[akka] case class TerminateMsg(exitValue: Int) extends ConfirmedClientOp with NetworkOp + +private[akka] case class GetAddress(node: RoleName) extends ServerOp with NetworkOp +private[akka] case class AddressReply(node: RoleName, addr: Address) extends UnconfirmedClientOp with NetworkOp + +private[akka] abstract class Done extends ServerOp with UnconfirmedClientOp with NetworkOp +private[akka] case object Done extends Done { + def getInstance: Done = this +} + +private[akka] case class Remove(node: RoleName) extends CommandOp + +private[akka] class MsgEncoder extends OneToOneEncoder { + + implicit def address2proto(addr: Address): TCP.Address = + TCP.Address.newBuilder + .setProtocol(addr.protocol) + .setSystem(addr.system) + .setHost(addr.host.get) + .setPort(addr.port.get) + .build + + implicit def direction2proto(dir: Direction): TCP.Direction = dir match { + case Direction.Send ⇒ TCP.Direction.Send + case Direction.Receive ⇒ TCP.Direction.Receive + case Direction.Both ⇒ TCP.Direction.Both + } + + def encode(ctx: ChannelHandlerContext, ch: Channel, msg: AnyRef): AnyRef = msg match { + case x: NetworkOp ⇒ + val w = TCP.Wrapper.newBuilder + x match { + case Hello(name, addr) ⇒ + w.setHello(TCP.Hello.newBuilder.setName(name).setAddress(addr)) + case EnterBarrier(name, timeout) ⇒ + val barrier = TCP.EnterBarrier.newBuilder.setName(name) + timeout foreach (t ⇒ barrier.setTimeout(t.toNanos)) + barrier.setOp(BarrierOp.Enter) + w.setBarrier(barrier) + case BarrierResult(name, success) ⇒ + val res = if (success) BarrierOp.Succeeded else BarrierOp.Failed + w.setBarrier(TCP.EnterBarrier.newBuilder.setName(name).setOp(res)) + case FailBarrier(name) ⇒ + w.setBarrier(TCP.EnterBarrier.newBuilder.setName(name).setOp(BarrierOp.Fail)) + case ThrottleMsg(target, dir, rate) ⇒ + w.setFailure(TCP.InjectFailure.newBuilder.setAddress(target) + .setFailure(TCP.FailType.Throttle).setDirection(dir).setRateMBit(rate)) + case DisconnectMsg(target, abort) ⇒ + w.setFailure(TCP.InjectFailure.newBuilder.setAddress(target) + .setFailure(if (abort) TCP.FailType.Abort else TCP.FailType.Disconnect)) + case TerminateMsg(exitValue) ⇒ + w.setFailure(TCP.InjectFailure.newBuilder.setFailure(TCP.FailType.Shutdown).setExitValue(exitValue)) + case GetAddress(node) ⇒ + w.setAddr(TCP.AddressRequest.newBuilder.setNode(node.name)) + case AddressReply(node, addr) ⇒ + w.setAddr(TCP.AddressRequest.newBuilder.setNode(node.name).setAddr(addr)) + case _: Done ⇒ + w.setDone("") + } + w.build + case _ ⇒ throw new IllegalArgumentException("wrong message " + msg) + } +} + +private[akka] class MsgDecoder extends OneToOneDecoder { + + implicit def address2scala(addr: TCP.Address): Address = + Address(addr.getProtocol, addr.getSystem, addr.getHost, addr.getPort) + + implicit def direction2scala(dir: TCP.Direction): Direction = dir match { + case TCP.Direction.Send ⇒ Direction.Send + case TCP.Direction.Receive ⇒ Direction.Receive + case TCP.Direction.Both ⇒ Direction.Both + } + + def decode(ctx: ChannelHandlerContext, ch: Channel, msg: AnyRef): AnyRef = msg match { + case w: TCP.Wrapper if w.getAllFields.size == 1 ⇒ + if (w.hasHello) { + val h = w.getHello + Hello(h.getName, h.getAddress) + } else if (w.hasBarrier) { + val barrier = w.getBarrier + barrier.getOp match { + case BarrierOp.Succeeded ⇒ BarrierResult(barrier.getName, true) + case BarrierOp.Failed ⇒ BarrierResult(barrier.getName, false) + case BarrierOp.Fail ⇒ FailBarrier(barrier.getName) + case BarrierOp.Enter ⇒ EnterBarrier(barrier.getName, + if (barrier.hasTimeout) Option(Duration.fromNanos(barrier.getTimeout)) else None) + } + } else if (w.hasFailure) { + val f = w.getFailure + import TCP.{ FailType ⇒ FT } + f.getFailure match { + case FT.Throttle ⇒ ThrottleMsg(f.getAddress, f.getDirection, f.getRateMBit) + case FT.Abort ⇒ DisconnectMsg(f.getAddress, true) + case FT.Disconnect ⇒ DisconnectMsg(f.getAddress, false) + case FT.Shutdown ⇒ TerminateMsg(f.getExitValue) + } + } else if (w.hasAddr) { + val a = w.getAddr + if (a.hasAddr) AddressReply(RoleName(a.getNode), a.getAddr) + else GetAddress(RoleName(a.getNode)) + } else if (w.hasDone) { + Done + } else { + throw new IllegalArgumentException("unknown message " + msg) + } + case _ ⇒ throw new IllegalArgumentException("wrong message " + msg) + } +} diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Extension.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Extension.scala new file mode 100644 index 0000000000..48f3983a78 --- /dev/null +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Extension.scala @@ -0,0 +1,73 @@ +package akka.remote.testconductor + +import akka.actor.ExtensionKey +import akka.actor.Extension +import akka.actor.ExtendedActorSystem +import akka.remote.RemoteActorRefProvider +import akka.actor.ActorContext +import akka.util.{ Duration, Timeout } +import java.util.concurrent.TimeUnit.MILLISECONDS +import akka.actor.ActorRef +import java.util.concurrent.ConcurrentHashMap +import akka.actor.Address +import akka.actor.ActorSystemImpl +import akka.actor.Props + +/** + * Access to the [[akka.remote.testconductor.TestConductorExt]] extension: + * + * {{{ + * val tc = TestConductor(system) + * tc.startController(numPlayers) + * // OR + * tc.startClient(conductorPort) + * }}} + */ +object TestConductor extends ExtensionKey[TestConductorExt] { + + def apply()(implicit ctx: ActorContext): TestConductorExt = apply(ctx.system) + +} + +/** + * This binds together the [[akka.remote.testconductor.Conductor]] and + * [[akka.remote.testconductor.Player]] roles inside an Akka + * [[akka.actor.Extension]]. Please follow the aforementioned links for + * more information. + * + * This extension requires the `akka.actor.provider` + * to be a [[akka.remote.RemoteActorRefProvider]]. + */ +class TestConductorExt(val system: ExtendedActorSystem) extends Extension with Conductor with Player { + + object Settings { + val config = system.settings.config + + val ConnectTimeout = Duration(config.getMilliseconds("akka.testconductor.connect-timeout"), MILLISECONDS) + val ClientReconnects = config.getInt("akka.testconductor.client-reconnects") + val ReconnectBackoff = Duration(config.getMilliseconds("akka.testconductor.reconnect-backoff"), MILLISECONDS) + + implicit val BarrierTimeout = Timeout(Duration(config.getMilliseconds("akka.testconductor.barrier-timeout"), MILLISECONDS)) + implicit val QueryTimeout = Timeout(Duration(config.getMilliseconds("akka.testconductor.query-timeout"), MILLISECONDS)) + val PacketSplitThreshold = Duration(config.getMilliseconds("akka.testconductor.packet-split-threshold"), MILLISECONDS) + } + + /** + * Remote transport used by the actor ref provider. + */ + val transport = system.provider.asInstanceOf[RemoteActorRefProvider].transport + + /** + * Transport address of this Netty-like remote transport. + */ + val address = transport.address + + /** + * INTERNAL API. + * + * [[akka.remote.testconductor.NetworkFailureInjector]]s register themselves here so that + * failures can be injected. + */ + private[akka] val failureInjector = system.asInstanceOf[ActorSystemImpl].systemActorOf(Props[FailureInjector], "FailureInjector") + +} diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala new file mode 100644 index 0000000000..2d5b73216e --- /dev/null +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala @@ -0,0 +1,388 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.testconductor + +import java.net.InetSocketAddress + +import scala.annotation.tailrec +import scala.collection.immutable.Queue + +import org.jboss.netty.buffer.ChannelBuffer +import org.jboss.netty.channel.{ SimpleChannelHandler, MessageEvent, Channels, ChannelStateEvent, ChannelHandlerContext, ChannelFutureListener, ChannelFuture } + +import akka.actor.{ Props, LoggingFSM, Address, ActorSystem, ActorRef, ActorLogging, Actor, FSM } +import akka.event.Logging +import akka.remote.netty.ChannelAddress +import akka.util.Duration +import akka.util.duration._ + +/** + * INTERNAL API. + */ +private[akka] class FailureInjector extends Actor with ActorLogging { + import ThrottleActor._ + import NetworkFailureInjector._ + + case class ChannelSettings( + ctx: Option[ChannelHandlerContext] = None, + throttleSend: Option[SetRate] = None, + throttleReceive: Option[SetRate] = None) + case class Injectors(sender: ActorRef, receiver: ActorRef, known: Boolean) + + var channels = Map[ChannelHandlerContext, Injectors]() + var settings = Map[Address, ChannelSettings]() + var generation = Iterator from 1 + + /** + * Only for a NEW ctx, start ThrottleActors, prime them and update all maps. + */ + def ingestContextAddress(ctx: ChannelHandlerContext, addr: Address, known: Boolean, + snd: Option[ActorRef] = None, rcv: Option[ActorRef] = None): Injectors = { + lazy val gen = generation.next + val name = addr.host.get + ":" + addr.port.get + val thrSend = snd getOrElse context.actorOf(Props(new ThrottleActor(ctx)), name + "-snd" + gen) + val thrRecv = rcv getOrElse context.actorOf(Props(new ThrottleActor(ctx)), name + "-rcv" + gen) + val injectors = Injectors(thrSend, thrRecv, known) + channels += ctx -> injectors + settings += addr -> (settings get addr map { + case c @ ChannelSettings(prevCtx, ts, tr) ⇒ + ts foreach (thrSend ! _) + tr foreach (thrRecv ! _) + prevCtx match { + case Some(p) ⇒ log.warning("installing context {} instead of {} for address {}", ctx, p, addr) + case None ⇒ // okay + } + c.copy(ctx = Some(ctx)) + } getOrElse ChannelSettings(Some(ctx))) + injectors + } + + /** + * Retrieve target settings, also if they were sketchy before (i.e. no system name). + * In the latter case, copy settings from the sketchy address and remove the old + * mapping. + */ + def retrieveTargetSettings(target: Address): Option[ChannelSettings] = { + settings get target orElse { + val host = target.host + val port = target.port + settings find { + case (Address("akka", "", `host`, `port`), s) ⇒ true + case _ ⇒ false + } map { + case (a, s) ⇒ settings -= a; settings += target -> s; s + } + } + } + + def receive = { + /* + * If a channel handler tells us that he’s been disconnected, stop the + * throttle actors and forget about them (but not possibly applied settings) + */ + case RemoveContext(ctx) ⇒ + channels get ctx foreach { inj ⇒ + context stop inj.sender + context stop inj.receiver + } + channels -= ctx + settings ++= settings collect { case (addr, c @ ChannelSettings(Some(`ctx`), _, _)) ⇒ (addr, c.copy(ctx = None)) } + /* + * Throttle/Blackhole/Unblock connections, based on the sign of rateMBit; + * will inform throttle actors for that destination if currently connected + * and update the settings for the target address; reply is needed to + * confirm this operation and tell the master that he can proceed. + */ + case ThrottleMsg(target, dir, rateMBit) ⇒ + val setting = retrieveTargetSettings(target) + settings += target -> ((setting getOrElse ChannelSettings() match { + case cs @ ChannelSettings(ctx, _, _) if dir includes Direction.Send ⇒ + ctx foreach (c ⇒ channels get c foreach (_.sender ! SetRate(rateMBit))) + cs.copy(throttleSend = Some(SetRate(rateMBit))) + case x ⇒ x + }) match { + case cs @ ChannelSettings(ctx, _, _) if dir includes Direction.Receive ⇒ + ctx foreach (c ⇒ channels get c foreach (_.receiver ! SetRate(rateMBit))) + cs.copy(throttleReceive = Some(SetRate(rateMBit))) + case x ⇒ x + }) + sender ! "ok" + /* + * Disconnect the currently active connection to the given target; reply is + * needed to confirm this operation and tell the master the he can proceed. + */ + case DisconnectMsg(target, abort) ⇒ + retrieveTargetSettings(target) foreach { + case ChannelSettings(Some(ctx), _, _) ⇒ + val ch = ctx.getChannel + if (abort) { + ch.getConfig.setOption("soLinger", 0) + log.info("aborting connection {}", ch) + } else log.info("closing connection {}", ch) + ch.close + case _ ⇒ log.debug("no connection to {} to close or abort", target) + } + sender ! "ok" + /* + * All data transfers up or down the pipeline are redirected through this + * case statement, which dispatches to the throttle actors for the given + * channel handler context. If none exist yet, they will be created, and + * this is a bit complicated in the case where the first message has not + * yet been exchanged, i.e. the other side’s Address is not yet known + * (keep in mind that an actor system’s remote address is not necessarily + * connected in any way to the IP from which we receive the connection). + */ + case s @ Send(ctx, direction, future, msg) ⇒ + channels get ctx match { + case Some(Injectors(snd, rcv, known)) ⇒ + // if the system registered with an empty name then check if we know it now + if (!known) ChannelAddress.get(ctx.getChannel).foreach(addr ⇒ + ingestContextAddress(ctx, addr, true, Some(snd), Some(rcv))) + if (direction includes Direction.Send) snd ! s + if (direction includes Direction.Receive) rcv ! s + case None ⇒ + // don’t do reverse lookup at first + ctx.getChannel.getRemoteAddress match { + case sockAddr: InetSocketAddress ⇒ + val (ipaddr, ip, port) = (sockAddr.getAddress, sockAddr.getAddress.getHostAddress, sockAddr.getPort) + val (addr, known) = ChannelAddress.get(ctx.getChannel) orElse { + settings collect { case (a @ Address("akka", _, Some(`ip`), Some(`port`)), _) ⇒ a } headOption + } orElse { + // only if raw IP failed, try with hostname + val name = ipaddr.getHostName + if (name == ip) None + else settings collect { case (a @ Address("akka", _, Some(`name`), Some(`port`)), _) ⇒ a } headOption + } match { + case Some(a) ⇒ (a, true) + case None ⇒ (Address("akka", "", ip, port), false) + } + /* + * ^- the above last resort will not match later requests directly, but be + * picked up by retrieveTargetSettings, so that throttle ops are + * applied to the right throttle actors, assuming that there can + * be only one actor system per host:port. + */ + val inj = ingestContextAddress(ctx, addr, known) + if (direction includes Direction.Send) inj.sender ! s + if (direction includes Direction.Receive) inj.receiver ! s + case null ⇒ + log.debug("sending {} in direction {} when socket {} already closed, dropping", msg, direction, ctx.getChannel) + } + } + } +} + +private[akka] object NetworkFailureInjector { + case class RemoveContext(ctx: ChannelHandlerContext) +} + +/** + * Brief overview: all network traffic passes through the `sender`/`receiver` FSMs managed + * by the FailureInjector of the TestConductor extension. These can + * pass through requests immediately, drop them or throttle to a desired rate. The FSMs are + * registered in the TestConductorExt.failureInjector so that settings can be applied from + * the ClientFSMs. + * + * I found that simply forwarding events using ctx.sendUpstream/sendDownstream does not work, + * it deadlocks and gives strange errors; in the end I just trusted the Netty docs which + * recommend to prefer `Channels.write()` and `Channels.fireMessageReceived()`. + * + * INTERNAL API. + */ +private[akka] class NetworkFailureInjector(system: ActorSystem) extends SimpleChannelHandler { + import NetworkFailureInjector._ + + private val log = Logging(system, "FailureInjector") + + private val conductor = TestConductor(system) + private var announced = false + + override def channelConnected(ctx: ChannelHandlerContext, state: ChannelStateEvent) { + state.getValue match { + case a: InetSocketAddress ⇒ + val addr = Address("akka", "", a.getHostName, a.getPort) + log.debug("connected to {}", addr) + case x ⇒ throw new IllegalArgumentException("unknown address type: " + x) + } + } + + override def channelDisconnected(ctx: ChannelHandlerContext, state: ChannelStateEvent) { + log.debug("disconnected from {}", state.getChannel) + conductor.failureInjector ! RemoveContext(ctx) + } + + override def messageReceived(ctx: ChannelHandlerContext, msg: MessageEvent) { + log.debug("upstream(queued): {}", msg) + conductor.failureInjector ! ThrottleActor.Send(ctx, Direction.Receive, Option(msg.getFuture), msg.getMessage) + } + + override def writeRequested(ctx: ChannelHandlerContext, msg: MessageEvent) { + log.debug("downstream(queued): {}", msg) + conductor.failureInjector ! ThrottleActor.Send(ctx, Direction.Send, Option(msg.getFuture), msg.getMessage) + } + +} + +/** + * INTERNAL API. + */ +private[akka] object ThrottleActor { + sealed trait State + case object PassThrough extends State + case object Throttle extends State + case object Blackhole extends State + + case class Data(lastSent: Long, rateMBit: Float, queue: Queue[Send]) + + case class Send(ctx: ChannelHandlerContext, direction: Direction, future: Option[ChannelFuture], msg: AnyRef) + case class SetRate(rateMBit: Float) + case object Tick +} + +/** + * INTERNAL API. + */ +private[akka] class ThrottleActor(channelContext: ChannelHandlerContext) + extends Actor with LoggingFSM[ThrottleActor.State, ThrottleActor.Data] { + + import ThrottleActor._ + import FSM._ + + private val packetSplitThreshold = TestConductor(context.system).Settings.PacketSplitThreshold + + startWith(PassThrough, Data(0, -1, Queue())) + + when(PassThrough) { + case Event(s @ Send(_, _, _, msg), _) ⇒ + log.debug("sending msg (PassThrough): {}", msg) + send(s) + stay + } + + when(Throttle) { + case Event(s: Send, data @ Data(_, _, Queue())) ⇒ + stay using sendThrottled(data.copy(lastSent = System.nanoTime, queue = Queue(s))) + case Event(s: Send, data) ⇒ + stay using sendThrottled(data.copy(queue = data.queue.enqueue(s))) + case Event(Tick, data) ⇒ + stay using sendThrottled(data) + } + + onTransition { + case Throttle -> PassThrough ⇒ + for (s ← stateData.queue) { + log.debug("sending msg (Transition): {}", s.msg) + send(s) + } + cancelTimer("send") + case Throttle -> Blackhole ⇒ + cancelTimer("send") + } + + when(Blackhole) { + case Event(Send(_, _, _, msg), _) ⇒ + log.debug("dropping msg {}", msg) + stay + } + + whenUnhandled { + case Event(SetRate(rate), d) ⇒ + if (rate > 0) { + goto(Throttle) using d.copy(lastSent = System.nanoTime, rateMBit = rate, queue = Queue()) + } else if (rate == 0) { + goto(Blackhole) + } else { + goto(PassThrough) + } + } + + initialize + + private def sendThrottled(d: Data): Data = { + val (data, toSend, toTick) = schedule(d) + for (s ← toSend) { + log.debug("sending msg (Tick): {}", s.msg) + send(s) + } + if (!timerActive_?("send")) + for (time ← toTick) { + log.debug("scheduling next Tick in {}", time) + setTimer("send", Tick, time, false) + } + data + } + + private def send(s: Send): Unit = s.direction match { + case Direction.Send ⇒ Channels.write(s.ctx, s.future getOrElse Channels.future(s.ctx.getChannel), s.msg) + case Direction.Receive ⇒ Channels.fireMessageReceived(s.ctx, s.msg) + case _ ⇒ + } + + /** + * Core of the throttling engine: delay Send operations until their bit count + * would actually have had time to travel down the line at the configured + * data rate, and split up send operations which are so big that gaps larger + * than packetSplitThreshold would be planned (they will happen nevertheless + * due to HashedWheelTimer’s semantics, but we compensate by sending more the + * next time, in proportion to how long the Tick was overdue). So, this should + * lead to the correct rate on average, with increased latency of the order of + * HWT granularity. + */ + private def schedule(d: Data): (Data, Seq[Send], Option[Duration]) = { + val now = System.nanoTime + @tailrec def rec(d: Data, toSend: Seq[Send]): (Data, Seq[Send], Option[Duration]) = { + if (d.queue.isEmpty) (d, toSend, None) + else { + val timeForPacket = d.lastSent + (1000 * size(d.queue.head.msg) / d.rateMBit).toLong + if (timeForPacket <= now) rec(Data(timeForPacket, d.rateMBit, d.queue.tail), toSend :+ d.queue.head) + else { + val splitThreshold = d.lastSent + packetSplitThreshold.toNanos + if (now < splitThreshold) (d, toSend, Some((timeForPacket - now).nanos min (splitThreshold - now).nanos)) + else { + val microsToSend = (now - d.lastSent) / 1000 + val (s1, s2) = split(d.queue.head, (microsToSend * d.rateMBit / 8).toInt) + (d.copy(queue = s2 +: d.queue.tail), toSend :+ s1, Some((timeForPacket - now).nanos min packetSplitThreshold)) + } + } + } + } + rec(d, Seq()) + } + + /** + * Split one Send operation in two, cutting off the given number of bytes at + * the front. If it was Direction.Send, i.e. a channel.write(), then also + * split the Future so that a failure in either part will complete the original + * with that failure. Data are not copied, as long as ChannelBuffer.slice does + * not copy them. + */ + private def split(s: Send, bytes: Int): (Send, Send) = { + s.msg match { + case buf: ChannelBuffer ⇒ + val f = s.future map { f ⇒ + val newF = Channels.future(s.ctx.getChannel) + newF.addListener(new ChannelFutureListener { + def operationComplete(future: ChannelFuture) { + if (future.isCancelled) f.cancel() + else future.getCause match { + case null ⇒ + case thr ⇒ f.setFailure(thr) + } + } + }) + newF + } + val b = buf.slice() + b.writerIndex(b.readerIndex + bytes) + buf.readerIndex(buf.readerIndex + bytes) + (Send(s.ctx, s.direction, f, b), Send(s.ctx, s.direction, s.future, buf)) + } + } + + private def size(msg: AnyRef) = msg match { + case b: ChannelBuffer ⇒ b.readableBytes() * 8 + case _ ⇒ throw new UnsupportedOperationException("NetworkFailureInjector only supports ChannelBuffer messages") + } +} + diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala new file mode 100644 index 0000000000..c7f69091cf --- /dev/null +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala @@ -0,0 +1,320 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.remote.testconductor + +import akka.actor.{ Actor, ActorRef, ActorSystem, LoggingFSM, Props } +import RemoteConnection.getAddrString +import akka.util.duration._ +import org.jboss.netty.channel.{ Channel, SimpleChannelUpstreamHandler, ChannelHandlerContext, ChannelStateEvent, MessageEvent } +import com.typesafe.config.ConfigFactory +import akka.util.Timeout +import akka.util.Duration +import java.util.concurrent.TimeUnit.MILLISECONDS +import akka.pattern.{ ask, pipe, AskTimeoutException } +import akka.dispatch.Await +import scala.util.control.NoStackTrace +import akka.actor.Status +import akka.event.LoggingAdapter +import akka.actor.PoisonPill +import akka.event.Logging +import akka.dispatch.Future +import java.net.InetSocketAddress +import akka.actor.Address +import org.jboss.netty.channel.ExceptionEvent +import org.jboss.netty.channel.WriteCompletionEvent +import java.net.ConnectException +import akka.util.Deadline +import akka.actor.Scheduler +import java.util.concurrent.TimeoutException + +/** + * The Player is the client component of the + * [[akka.remote.testconductor.TestConductorExt]] extension. It registers with + * the [[akka.remote.testconductor.Conductor]]’s [[akka.remote.testconductor.Controller]] + * in order to participate in barriers and enable network failure injection. + */ +trait Player { this: TestConductorExt ⇒ + + private var _client: ActorRef = _ + private def client = _client match { + case null ⇒ throw new IllegalStateException("TestConductor client not yet started") + case x ⇒ x + } + + /** + * Connect to the conductor on the given port (the host is taken from setting + * `akka.testconductor.host`). The connection is made asynchronously, but you + * should await completion of the returned Future because that implies that + * all expected participants of this test have successfully connected (i.e. + * this is a first barrier in itself). The number of expected participants is + * set in [[akka.remote.testconductor.Conductor]]`.startController()`. + */ + def startClient(name: RoleName, controllerAddr: InetSocketAddress): Future[Done] = { + import ClientFSM._ + import akka.actor.FSM._ + import Settings.BarrierTimeout + + if (_client ne null) throw new IllegalStateException("TestConductorClient already started") + _client = system.actorOf(Props(new ClientFSM(name, controllerAddr)), "TestConductorClient") + val a = system.actorOf(Props(new Actor { + var waiting: ActorRef = _ + def receive = { + case fsm: ActorRef ⇒ waiting = sender; fsm ! SubscribeTransitionCallBack(self) + case Transition(_, Connecting, AwaitDone) ⇒ // step 1, not there yet + case Transition(_, AwaitDone, Connected) ⇒ waiting ! Done; context stop self + case t: Transition[_] ⇒ waiting ! Status.Failure(new RuntimeException("unexpected transition: " + t)); context stop self + case CurrentState(_, Connected) ⇒ waiting ! Done; context stop self + case _: CurrentState[_] ⇒ + } + })) + + a ? client mapTo + } + + /** + * Enter the named barriers, one after the other, in the order given. Will + * throw an exception in case of timeouts or other errors. + */ + def enter(name: String*) { + enter(Settings.BarrierTimeout, name) + } + + /** + * Enter the named barriers, one after the other, in the order given. Will + * throw an exception in case of timeouts or other errors. + */ + def enter(timeout: Timeout, name: Seq[String]) { + system.log.debug("entering barriers " + name.mkString("(", ", ", ")")) + val stop = Deadline.now + timeout.duration + name foreach { b ⇒ + val barrierTimeout = stop.timeLeft + if (barrierTimeout < Duration.Zero) { + client ! ToServer(FailBarrier(b)) + throw new TimeoutException("Server timed out while waiting for barrier " + b); + } + try { + implicit val timeout = Timeout(barrierTimeout + Settings.QueryTimeout.duration) + Await.result(client ? ToServer(EnterBarrier(b, Option(barrierTimeout))), Duration.Inf) + } catch { + case e: AskTimeoutException ⇒ + client ! ToServer(FailBarrier(b)) + // Why don't TimeoutException have a constructor that takes a cause? + throw new TimeoutException("Client timed out while waiting for barrier " + b); + } + system.log.debug("passed barrier {}", b) + } + } + + /** + * Query remote transport address of named node. + */ + def getAddressFor(name: RoleName): Future[Address] = { + import Settings.QueryTimeout + client ? ToServer(GetAddress(name)) mapTo + } +} + +/** + * INTERNAL API. + */ +private[akka] object ClientFSM { + sealed trait State + case object Connecting extends State + case object AwaitDone extends State + case object Connected extends State + case object Failed extends State + + case class Data(channel: Option[Channel], runningOp: Option[(String, ActorRef)]) + + case class Connected(channel: Channel) + case class ConnectionFailure(msg: String) extends RuntimeException(msg) with NoStackTrace + case object Disconnected +} + +/** + * This is the controlling entity on the [[akka.remote.testconductor.Player]] + * side: in a first step it registers itself with a symbolic name and its remote + * address at the [[akka.remote.testconductor.Controller]], then waits for the + * `Done` message which signals that all other expected test participants have + * done the same. After that, it will pass barrier requests to and from the + * coordinator and react to the [[akka.remote.testconductor.Conductor]]’s + * requests for failure injection. + * + * INTERNAL API. + */ +private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress) extends Actor with LoggingFSM[ClientFSM.State, ClientFSM.Data] { + import ClientFSM._ + + val settings = TestConductor().Settings + + val handler = new PlayerHandler(controllerAddr, settings.ClientReconnects, settings.ReconnectBackoff, + self, Logging(context.system, "PlayerHandler"), context.system.scheduler) + + startWith(Connecting, Data(None, None)) + + when(Connecting, stateTimeout = settings.ConnectTimeout) { + case Event(msg: ClientOp, _) ⇒ + stay replying Status.Failure(new IllegalStateException("not connected yet")) + case Event(Connected(channel), _) ⇒ + channel.write(Hello(name.name, TestConductor().address)) + goto(AwaitDone) using Data(Some(channel), None) + case Event(_: ConnectionFailure, _) ⇒ + goto(Failed) + case Event(StateTimeout, _) ⇒ + log.error("connect timeout to TestConductor") + goto(Failed) + } + + when(AwaitDone, stateTimeout = settings.BarrierTimeout.duration) { + case Event(Done, _) ⇒ + log.debug("received Done: starting test") + goto(Connected) + case Event(msg: NetworkOp, _) ⇒ + log.error("received {} instead of Done", msg) + goto(Failed) + case Event(msg: ServerOp, _) ⇒ + stay replying Status.Failure(new IllegalStateException("not connected yet")) + case Event(StateTimeout, _) ⇒ + log.error("connect timeout to TestConductor") + goto(Failed) + } + + when(Connected) { + case Event(Disconnected, _) ⇒ + log.info("disconnected from TestConductor") + throw new ConnectionFailure("disconnect") + case Event(ToServer(Done), Data(Some(channel), _)) ⇒ + channel.write(Done) + stay + case Event(ToServer(msg), d @ Data(Some(channel), None)) ⇒ + channel.write(msg) + val token = msg match { + case EnterBarrier(barrier, timeout) ⇒ barrier + case GetAddress(node) ⇒ node.name + } + stay using d.copy(runningOp = Some(token, sender)) + case Event(ToServer(op), Data(channel, Some((token, _)))) ⇒ + log.error("cannot write {} while waiting for {}", op, token) + stay + case Event(op: ClientOp, d @ Data(Some(channel), runningOp)) ⇒ + op match { + case BarrierResult(b, success) ⇒ + runningOp match { + case Some((barrier, requester)) ⇒ + if (b != barrier) { + requester ! Status.Failure(new RuntimeException("wrong barrier " + b + " received while waiting for " + barrier)) + } else if (!success) { + requester ! Status.Failure(new RuntimeException("barrier failed: " + b)) + } else { + requester ! b + } + case None ⇒ + log.warning("did not expect {}", op) + } + stay using d.copy(runningOp = None) + case AddressReply(node, addr) ⇒ + runningOp match { + case Some((_, requester)) ⇒ + requester ! addr + case None ⇒ + log.warning("did not expect {}", op) + } + stay using d.copy(runningOp = None) + case t: ThrottleMsg ⇒ + import settings.QueryTimeout + TestConductor().failureInjector ? t map (_ ⇒ ToServer(Done)) pipeTo self + stay + case d: DisconnectMsg ⇒ + import settings.QueryTimeout + TestConductor().failureInjector ? d map (_ ⇒ ToServer(Done)) pipeTo self + stay + case TerminateMsg(exit) ⇒ + System.exit(exit) + stay // needed because Java doesn’t have Nothing + } + } + + when(Failed) { + case Event(msg: ClientOp, _) ⇒ + stay replying Status.Failure(new RuntimeException("cannot do " + msg + " while Failed")) + case Event(msg: NetworkOp, _) ⇒ + log.warning("ignoring network message {} while Failed", msg) + stay + } + + onTermination { + case StopEvent(_, _, Data(Some(channel), _)) ⇒ + channel.close() + } + + initialize + +} + +/** + * This handler only forwards messages received from the conductor to the [[akka.remote.testconductor.ClientFSM]]. + * + * INTERNAL API. + */ +private[akka] class PlayerHandler( + server: InetSocketAddress, + private var reconnects: Int, + backoff: Duration, + fsm: ActorRef, + log: LoggingAdapter, + scheduler: Scheduler) + extends SimpleChannelUpstreamHandler { + + import ClientFSM._ + + reconnect() + + var nextAttempt: Deadline = _ + + override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = log.debug("channel {} open", event.getChannel) + override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = log.debug("channel {} closed", event.getChannel) + override def channelBound(ctx: ChannelHandlerContext, event: ChannelStateEvent) = log.debug("channel {} bound", event.getChannel) + override def channelUnbound(ctx: ChannelHandlerContext, event: ChannelStateEvent) = log.debug("channel {} unbound", event.getChannel) + override def writeComplete(ctx: ChannelHandlerContext, event: WriteCompletionEvent) = log.debug("channel {} written {}", event.getChannel, event.getWrittenAmount) + + override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { + log.debug("channel {} exception {}", event.getChannel, event.getCause) + event.getCause match { + case c: ConnectException if reconnects > 0 ⇒ + reconnects -= 1 + scheduler.scheduleOnce(nextAttempt.timeLeft)(reconnect()) + case e ⇒ fsm ! ConnectionFailure(e.getMessage) + } + } + + private def reconnect(): Unit = { + nextAttempt = Deadline.now + backoff + RemoteConnection(Client, server, this) + } + + override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { + val ch = event.getChannel + log.debug("connected to {}", getAddrString(ch)) + fsm ! Connected(ch) + } + + override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { + val channel = event.getChannel + log.debug("disconnected from {}", getAddrString(channel)) + fsm ! PoisonPill + } + + override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = { + val channel = event.getChannel + log.debug("message from {}: {}", getAddrString(channel), event.getMessage) + event.getMessage match { + case msg: NetworkOp ⇒ + fsm ! msg + case msg ⇒ + log.info("server {} sent garbage '{}', disconnecting", getAddrString(channel), msg) + channel.close() + } + } +} + diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/RemoteConnection.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/RemoteConnection.scala new file mode 100644 index 0000000000..1979857bf0 --- /dev/null +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/RemoteConnection.scala @@ -0,0 +1,70 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.remote.testconductor + +import org.jboss.netty.channel.{ Channel, ChannelPipeline, ChannelPipelineFactory, ChannelUpstreamHandler, SimpleChannelUpstreamHandler, DefaultChannelPipeline } +import org.jboss.netty.channel.socket.nio.{ NioClientSocketChannelFactory, NioServerSocketChannelFactory } +import org.jboss.netty.bootstrap.{ ClientBootstrap, ServerBootstrap } +import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender } +import org.jboss.netty.handler.codec.compression.{ ZlibDecoder, ZlibEncoder } +import org.jboss.netty.handler.codec.protobuf.{ ProtobufDecoder, ProtobufEncoder } +import org.jboss.netty.handler.timeout.{ ReadTimeoutHandler, ReadTimeoutException } +import java.net.InetSocketAddress +import java.util.concurrent.Executors +import akka.event.Logging + +/** + * INTERNAL API. + */ +private[akka] class TestConductorPipelineFactory(handler: ChannelUpstreamHandler) extends ChannelPipelineFactory { + def getPipeline: ChannelPipeline = { + val encap = List(new LengthFieldPrepender(4), new LengthFieldBasedFrameDecoder(10000, 0, 4, 0, 4)) + val proto = List(new ProtobufEncoder, new ProtobufDecoder(TestConductorProtocol.Wrapper.getDefaultInstance)) + val msg = List(new MsgEncoder, new MsgDecoder) + (encap ::: proto ::: msg ::: handler :: Nil).foldLeft(new DefaultChannelPipeline) { + (pipe, handler) ⇒ pipe.addLast(Logging.simpleName(handler.getClass), handler); pipe + } + } +} + +/** + * INTERNAL API. + */ +private[akka] sealed trait Role +/** + * INTERNAL API. + */ +private[akka] case object Client extends Role +/** + * INTERNAL API. + */ +private[akka] case object Server extends Role + +/** + * INTERNAL API. + */ +private[akka] object RemoteConnection { + def apply(role: Role, sockaddr: InetSocketAddress, handler: ChannelUpstreamHandler): Channel = { + role match { + case Client ⇒ + val socketfactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool) + val bootstrap = new ClientBootstrap(socketfactory) + bootstrap.setPipelineFactory(new TestConductorPipelineFactory(handler)) + bootstrap.setOption("tcpNoDelay", true) + bootstrap.connect(sockaddr).getChannel + case Server ⇒ + val socketfactory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool) + val bootstrap = new ServerBootstrap(socketfactory) + bootstrap.setPipelineFactory(new TestConductorPipelineFactory(handler)) + bootstrap.setOption("reuseAddress", true) + bootstrap.setOption("child.tcpNoDelay", true) + bootstrap.bind(sockaddr) + } + } + + def getAddrString(channel: Channel) = channel.getRemoteAddress match { + case i: InetSocketAddress ⇒ i.toString + case _ ⇒ "[unknown]" + } +} diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/TestConductorTransport.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/TestConductorTransport.scala new file mode 100644 index 0000000000..f7b7943275 --- /dev/null +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/TestConductorTransport.scala @@ -0,0 +1,24 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.testconductor + +import akka.remote.netty.NettyRemoteTransport +import akka.remote.RemoteSettings +import akka.actor.ExtendedActorSystem +import akka.remote.RemoteActorRefProvider +import org.jboss.netty.channel.ChannelHandler +import org.jboss.netty.channel.ChannelPipelineFactory + +/** + * INTERNAL API. + */ +private[akka] class TestConductorTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) + extends NettyRemoteTransport(_system, _provider) { + + override def createPipeline(endpoint: ⇒ ChannelHandler, withTimeout: Boolean, isClient: Boolean): ChannelPipelineFactory = + new ChannelPipelineFactory { + def getPipeline = PipelineFactory(new NetworkFailureInjector(system) +: PipelineFactory.defaultStack(withTimeout, isClient) :+ endpoint) + } + +} \ No newline at end of file diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala new file mode 100644 index 0000000000..f49dc53e2b --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/LookupRemoteActorSpec.scala @@ -0,0 +1,55 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Props +import akka.pattern.ask +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ + +object LookupRemoteActorMultiJvmSpec extends MultiNodeConfig { + + class SomeActor extends Actor with Serializable { + def receive = { + case "identify" ⇒ sender ! self + } + } + + commonConfig(debugConfig(on = false)) + + val master = role("master") + val slave = role("slave") + +} + +class LookupRemoteActorMultiJvmNode1 extends LookupRemoteActorSpec +class LookupRemoteActorMultiJvmNode2 extends LookupRemoteActorSpec + +class LookupRemoteActorSpec extends MultiNodeSpec(LookupRemoteActorMultiJvmSpec) + with ImplicitSender with DefaultTimeout { + import LookupRemoteActorMultiJvmSpec._ + + def initialParticipants = 2 + + runOn(master) { + system.actorOf(Props[SomeActor], "service-hello") + } + + "Remoting" must { + "lookup remote actor" taggedAs LongRunningTest in { + runOn(slave) { + val hello = system.actorFor(node(master) / "user" / "service-hello") + hello.isInstanceOf[RemoteActorRef] must be(true) + val masterAddress = testConductor.getAddressFor(master).await + (hello ? "identify").await.asInstanceOf[ActorRef].path.address must equal(masterAddress) + } + enterBarrier("done") + } + } + +} + diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala new file mode 100644 index 0000000000..eca91495d6 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/NewRemoteActorSpec.scala @@ -0,0 +1,80 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote + +import com.typesafe.config.ConfigFactory + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Props +import akka.pattern.ask +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit._ + +object NewRemoteActorMultiJvmSpec extends MultiNodeConfig { + + class SomeActor extends Actor with Serializable { + def receive = { + case "identify" ⇒ sender ! self + } + } + + commonConfig(debugConfig(on = false)) + + val master = role("master") + val slave = role("slave") + + deployOn(master, """/service-hello.remote = "@slave@" """) + + deployOnAll("""/service-hello2.remote = "@slave@" """) +} + +class NewRemoteActorMultiJvmNode1 extends NewRemoteActorSpec +class NewRemoteActorMultiJvmNode2 extends NewRemoteActorSpec + +class NewRemoteActorSpec extends MultiNodeSpec(NewRemoteActorMultiJvmSpec) + with ImplicitSender with DefaultTimeout { + import NewRemoteActorMultiJvmSpec._ + + def initialParticipants = 2 + + "A new remote actor" must { + "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in { + + runOn(master) { + val actor = system.actorOf(Props[SomeActor], "service-hello") + actor.isInstanceOf[RemoteActorRef] must be(true) + + val slaveAddress = testConductor.getAddressFor(slave).await + actor ! "identify" + expectMsgType[ActorRef].path.address must equal(slaveAddress) + + // shut down the actor before we let the other node(s) shut down so we don't try to send + // "Terminate" to a shut down node + system.stop(actor) + } + + enterBarrier("done") + } + + "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef (with deployOnAll)" taggedAs LongRunningTest in { + + runOn(master) { + val actor = system.actorOf(Props[SomeActor], "service-hello2") + actor.isInstanceOf[RemoteActorRef] must be(true) + + val slaveAddress = testConductor.getAddressFor(slave).await + actor ! "identify" + expectMsgType[ActorRef].path.address must equal(slaveAddress) + + // shut down the actor before we let the other node(s) shut down so we don't try to send + // "Terminate" to a shut down node + system.stop(actor) + } + + enterBarrier("done") + } + } +} diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/QuietReporter.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/QuietReporter.scala similarity index 67% rename from akka-remote/src/multi-jvm/scala/akka/remote/QuietReporter.scala rename to akka-remote-tests/src/multi-jvm/scala/akka/remote/QuietReporter.scala index f323b75e23..eea5f079d3 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/QuietReporter.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/QuietReporter.scala @@ -8,8 +8,8 @@ import org.scalatest.tools.StandardOutReporter import org.scalatest.events._ import java.lang.Boolean.getBoolean -class QuietReporter(inColor: Boolean) extends StandardOutReporter(false, inColor, false, true) { - def this() = this(!getBoolean("akka.test.nocolor")) +class QuietReporter(inColor: Boolean, withDurations: Boolean = false) extends StandardOutReporter(withDurations, inColor, false, true) { + def this() = this(!getBoolean("akka.test.nocolor"), !getBoolean("akka.test.nodurations")) override def apply(event: Event): Unit = event match { case _: RunStarting ⇒ () diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/router/RandomRoutedRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/router/RandomRoutedRemoteActorSpec.scala new file mode 100644 index 0000000000..44c7ae5047 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/router/RandomRoutedRemoteActorSpec.scala @@ -0,0 +1,93 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.router + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Props +import akka.actor.PoisonPill +import akka.actor.Address +import akka.dispatch.Await +import akka.pattern.ask +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.routing.Broadcast +import akka.routing.RandomRouter +import akka.routing.RoutedActorRef +import akka.testkit._ +import akka.util.duration._ + +object RandomRoutedRemoteActorMultiJvmSpec extends MultiNodeConfig { + + class SomeActor extends Actor with Serializable { + def receive = { + case "hit" ⇒ sender ! self + } + } + + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + + commonConfig(debugConfig(on = false)) + + deployOnAll(""" + /service-hello.router = "random" + /service-hello.nr-of-instances = 3 + /service-hello.target.nodes = ["@first@", "@second@", "@third@"] + """) +} + +class RandomRoutedRemoteActorMultiJvmNode1 extends RandomRoutedRemoteActorSpec +class RandomRoutedRemoteActorMultiJvmNode2 extends RandomRoutedRemoteActorSpec +class RandomRoutedRemoteActorMultiJvmNode3 extends RandomRoutedRemoteActorSpec +class RandomRoutedRemoteActorMultiJvmNode4 extends RandomRoutedRemoteActorSpec + +class RandomRoutedRemoteActorSpec extends MultiNodeSpec(RandomRoutedRemoteActorMultiJvmSpec) + with ImplicitSender with DefaultTimeout { + import RandomRoutedRemoteActorMultiJvmSpec._ + + def initialParticipants = 4 + + "A new remote actor configured with a Random router" must { + "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in { + + runOn(first, second, third) { + enterBarrier("start", "broadcast-end", "end", "done") + } + + runOn(fourth) { + enterBarrier("start") + val actor = system.actorOf(Props[SomeActor].withRouter(RandomRouter()), "service-hello") + actor.isInstanceOf[RoutedActorRef] must be(true) + + val connectionCount = 3 + val iterationCount = 10 + + for (i ← 0 until iterationCount; k ← 0 until connectionCount) { + actor ! "hit" + } + + val replies: Map[Address, Int] = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) { + case ref: ActorRef ⇒ ref.path.address + }).foldLeft(Map(node(first).address -> 0, node(second).address -> 0, node(third).address -> 0)) { + case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1)) + } + + enterBarrier("broadcast-end") + actor ! Broadcast(PoisonPill) + + enterBarrier("end") + replies.values foreach { _ must be > (0) } + replies.get(node(fourth).address) must be(None) + + // shut down the actor before we let the other node(s) shut down so we don't try to send + // "Terminate" to a shut down node + system.stop(actor) + enterBarrier("done") + } + } + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/router/RoundRobinRoutedRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/router/RoundRobinRoutedRemoteActorSpec.scala new file mode 100644 index 0000000000..76a7e41ad1 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/router/RoundRobinRoutedRemoteActorSpec.scala @@ -0,0 +1,93 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.router + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Props +import akka.actor.PoisonPill +import akka.actor.Address +import akka.dispatch.Await +import akka.pattern.ask +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.routing.Broadcast +import akka.routing.RoundRobinRouter +import akka.routing.RoutedActorRef +import akka.testkit._ +import akka.util.duration._ + +object RoundRobinRoutedRemoteActorMultiJvmSpec extends MultiNodeConfig { + + class SomeActor extends Actor with Serializable { + def receive = { + case "hit" ⇒ sender ! self + } + } + + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + + commonConfig(debugConfig(on = false)) + + deployOnAll(""" + /service-hello.router = "round-robin" + /service-hello.nr-of-instances = 3 + /service-hello.target.nodes = ["@first@", "@second@", "@third@"] + """) +} + +class RoundRobinRoutedRemoteActorMultiJvmNode1 extends RoundRobinRoutedRemoteActorSpec +class RoundRobinRoutedRemoteActorMultiJvmNode2 extends RoundRobinRoutedRemoteActorSpec +class RoundRobinRoutedRemoteActorMultiJvmNode3 extends RoundRobinRoutedRemoteActorSpec +class RoundRobinRoutedRemoteActorMultiJvmNode4 extends RoundRobinRoutedRemoteActorSpec + +class RoundRobinRoutedRemoteActorSpec extends MultiNodeSpec(RoundRobinRoutedRemoteActorMultiJvmSpec) + with ImplicitSender with DefaultTimeout { + import RoundRobinRoutedRemoteActorMultiJvmSpec._ + + def initialParticipants = 4 + + "A new remote actor configured with a RoundRobin router" must { + "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in { + + runOn(first, second, third) { + enterBarrier("start", "broadcast-end", "end", "done") + } + + runOn(fourth) { + enterBarrier("start") + val actor = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "service-hello") + actor.isInstanceOf[RoutedActorRef] must be(true) + + val connectionCount = 3 + val iterationCount = 10 + + for (i ← 0 until iterationCount; k ← 0 until connectionCount) { + actor ! "hit" + } + + val replies: Map[Address, Int] = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) { + case ref: ActorRef ⇒ ref.path.address + }).foldLeft(Map(node(first).address -> 0, node(second).address -> 0, node(third).address -> 0)) { + case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1)) + } + + enterBarrier("broadcast-end") + actor ! Broadcast(PoisonPill) + + enterBarrier("end") + replies.values foreach { _ must be(iterationCount) } + replies.get(node(fourth).address) must be(None) + + // shut down the actor before we let the other node(s) shut down so we don't try to send + // "Terminate" to a shut down node + system.stop(actor) + enterBarrier("done") + } + } + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/router/ScatterGatherRoutedRemoteActorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/router/ScatterGatherRoutedRemoteActorSpec.scala new file mode 100644 index 0000000000..b77b0c196e --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/router/ScatterGatherRoutedRemoteActorSpec.scala @@ -0,0 +1,93 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.router + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Props +import akka.dispatch.Await +import akka.pattern.ask +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.routing.Broadcast +import akka.routing.ScatterGatherFirstCompletedRouter +import akka.routing.RoutedActorRef +import akka.testkit._ +import akka.util.duration._ +import akka.actor.PoisonPill +import akka.actor.Address + +object ScatterGatherRoutedRemoteActorMultiJvmSpec extends MultiNodeConfig { + + class SomeActor extends Actor with Serializable { + def receive = { + case "hit" ⇒ sender ! self + } + } + + val first = role("first") + val second = role("second") + val third = role("third") + val fourth = role("fourth") + + commonConfig(debugConfig(on = false)) + + deployOnAll(""" + /service-hello.router = "scatter-gather" + /service-hello.nr-of-instances = 3 + /service-hello.target.nodes = ["@first@", "@second@", "@third@"] + """) +} + +class ScatterGatherRoutedRemoteActorMultiJvmNode1 extends ScatterGatherRoutedRemoteActorSpec +class ScatterGatherRoutedRemoteActorMultiJvmNode2 extends ScatterGatherRoutedRemoteActorSpec +class ScatterGatherRoutedRemoteActorMultiJvmNode3 extends ScatterGatherRoutedRemoteActorSpec +class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends ScatterGatherRoutedRemoteActorSpec + +class ScatterGatherRoutedRemoteActorSpec extends MultiNodeSpec(ScatterGatherRoutedRemoteActorMultiJvmSpec) + with ImplicitSender with DefaultTimeout { + import ScatterGatherRoutedRemoteActorMultiJvmSpec._ + + def initialParticipants = 4 + + "A new remote actor configured with a ScatterGatherFirstCompleted router" must { + "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in { + + runOn(first, second, third) { + enterBarrier("start", "broadcast-end", "end", "done") + } + + runOn(fourth) { + enterBarrier("start") + val actor = system.actorOf(Props[SomeActor].withRouter(ScatterGatherFirstCompletedRouter(within = 10 seconds)), "service-hello") + actor.isInstanceOf[RoutedActorRef] must be(true) + + val connectionCount = 3 + val iterationCount = 10 + + for (i ← 0 until iterationCount; k ← 0 until connectionCount) { + actor ! "hit" + } + + val replies: Map[Address, Int] = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) { + case ref: ActorRef ⇒ ref.path.address + }).foldLeft(Map(node(first).address -> 0, node(second).address -> 0, node(third).address -> 0)) { + case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1)) + } + + enterBarrier("broadcast-end") + actor ! Broadcast(PoisonPill) + + enterBarrier("end") + replies.values.sum must be === connectionCount * iterationCount + replies.get(node(fourth).address) must be(None) + + // shut down the actor before we let the other node(s) shut down so we don't try to send + // "Terminate" to a shut down node + system.stop(actor) + enterBarrier("done") + } + } + } +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala new file mode 100644 index 0000000000..86fabc489d --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala @@ -0,0 +1,110 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.testconductor + +import com.typesafe.config.ConfigFactory +import akka.actor.Props +import akka.actor.Actor +import akka.dispatch.Await +import akka.dispatch.Await.Awaitable +import akka.util.Duration +import akka.util.duration._ +import akka.testkit.ImplicitSender +import akka.testkit.LongRunningTest +import java.net.InetSocketAddress +import java.net.InetAddress +import akka.remote.testkit.MultiNodeSpec +import akka.remote.testkit.MultiNodeConfig + +object TestConductorMultiJvmSpec extends MultiNodeConfig { + commonConfig(debugConfig(on = false)) + + val master = role("master") + val slave = role("slave") +} + +class TestConductorMultiJvmNode1 extends TestConductorSpec +class TestConductorMultiJvmNode2 extends TestConductorSpec + +class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with ImplicitSender { + + import TestConductorMultiJvmSpec._ + + def initialParticipants = 2 + + lazy val echo = system.actorFor(node(master) / "user" / "echo") + + "A TestConductor" must { + + "enter a barrier" taggedAs LongRunningTest in { + runOn(master) { + system.actorOf(Props(new Actor { + def receive = { + case x ⇒ testActor ! x; sender ! x + } + }), "echo") + } + + enterBarrier("name") + } + + "support throttling of network connections" taggedAs LongRunningTest in { + + runOn(slave) { + // start remote network connection so that it can be throttled + echo ! "start" + } + + expectMsg("start") + + runOn(master) { + testConductor.throttle(slave, master, Direction.Send, rateMBit = 0.01).await + } + + enterBarrier("throttled_send") + + runOn(slave) { + for (i ← 0 to 9) echo ! i + } + + within(0.6 seconds, 2 seconds) { + expectMsg(500 millis, 0) + receiveN(9) must be(1 to 9) + } + + enterBarrier("throttled_send2") + + runOn(master) { + testConductor.throttle(slave, master, Direction.Send, -1).await + testConductor.throttle(slave, master, Direction.Receive, rateMBit = 0.01).await + } + + enterBarrier("throttled_recv") + + runOn(slave) { + for (i ← 10 to 19) echo ! i + } + + val (min, max) = + ifNode(master) { + (0 seconds, 500 millis) + } { + (0.6 seconds, 2 seconds) + } + + within(min, max) { + expectMsg(500 millis, 10) + receiveN(9) must be(11 to 19) + } + + enterBarrier("throttled_recv2") + + runOn(master) { + testConductor.throttle(slave, master, Direction.Receive, -1).await + } + } + + } + +} diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testkit/MultiNodeSpecSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testkit/MultiNodeSpecSpec.scala new file mode 100644 index 0000000000..2a709a99a7 --- /dev/null +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testkit/MultiNodeSpecSpec.scala @@ -0,0 +1,36 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.testkit + +import akka.testkit.LongRunningTest + +object MultiNodeSpecMultiJvmSpec extends MultiNodeConfig { + commonConfig(debugConfig(on = false)) + + val node1 = role("node1") + val node2 = role("node2") + val node3 = role("node3") + val node4 = role("node4") +} + +class MultiNodeSpecSpecMultiJvmNode1 extends MultiNodeSpecSpec +class MultiNodeSpecSpecMultiJvmNode2 extends MultiNodeSpecSpec +class MultiNodeSpecSpecMultiJvmNode3 extends MultiNodeSpecSpec +class MultiNodeSpecSpecMultiJvmNode4 extends MultiNodeSpecSpec + +class MultiNodeSpecSpec extends MultiNodeSpec(MultiNodeSpecMultiJvmSpec) { + + import MultiNodeSpecMultiJvmSpec._ + + def initialParticipants = 4 + + "A MultiNodeSpec" must { + + "wait for all nodes to remove themselves before we shut the conductor down" taggedAs LongRunningTest in { + enterBarrier("startup") + // this test is empty here since it only exercises the shutdown code in the MultiNodeSpec + } + + } +} diff --git a/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala b/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala new file mode 100644 index 0000000000..8ff95d0831 --- /dev/null +++ b/akka-remote-tests/src/test/scala/akka/remote/testconductor/BarrierSpec.scala @@ -0,0 +1,555 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.testconductor + +import akka.testkit.AkkaSpec +import akka.actor.Props +import akka.actor.AddressFromURIString +import akka.actor.ActorRef +import akka.testkit.ImplicitSender +import akka.actor.Actor +import akka.actor.OneForOneStrategy +import akka.actor.SupervisorStrategy +import akka.testkit.EventFilter +import akka.testkit.TestProbe +import akka.util.duration._ +import akka.event.Logging +import org.scalatest.BeforeAndAfterEach +import java.net.InetSocketAddress +import java.net.InetAddress +import akka.testkit.TimingTest +import akka.util.{ Timeout, Duration } + +object BarrierSpec { + case class Failed(ref: ActorRef, thr: Throwable) + val config = """ + akka.testconductor.barrier-timeout = 5s + akka.actor.provider = akka.remote.RemoteActorRefProvider + akka.remote.netty.port = 0 + akka.actor.debug.fsm = on + akka.actor.debug.lifecycle = on + """ +} + +class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender { + + import BarrierSpec._ + import Controller._ + import BarrierCoordinator._ + + val A = RoleName("a") + val B = RoleName("b") + val C = RoleName("c") + + "A BarrierCoordinator" must { + + "register clients and remove them" taggedAs TimingTest in { + val b = getBarrier() + b ! NodeInfo(A, AddressFromURIString("akka://sys"), system.deadLetters) + b ! RemoveClient(B) + b ! RemoveClient(A) + EventFilter[BarrierEmpty](occurrences = 1) intercept { + b ! RemoveClient(A) + } + expectMsg(Failed(b, BarrierEmpty(Data(Set(), "", Nil, null), "cannot remove RoleName(a): no client to remove"))) + } + + "register clients and disconnect them" taggedAs TimingTest in { + val b = getBarrier() + b ! NodeInfo(A, AddressFromURIString("akka://sys"), system.deadLetters) + b ! ClientDisconnected(B) + expectNoMsg(1 second) + b ! ClientDisconnected(A) + expectNoMsg(1 second) + } + + "fail entering barrier when nobody registered" taggedAs TimingTest in { + val b = getBarrier() + b ! EnterBarrier("bar1", None) + expectMsg(ToClient(BarrierResult("bar1", false))) + } + + "enter barrier" taggedAs TimingTest in { + val barrier = getBarrier() + val a, b = TestProbe() + barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + a.send(barrier, EnterBarrier("bar2", None)) + noMsg(a, b) + within(2 seconds) { + b.send(barrier, EnterBarrier("bar2", None)) + a.expectMsg(ToClient(BarrierResult("bar2", true))) + b.expectMsg(ToClient(BarrierResult("bar2", true))) + } + } + + "enter barrier with joining node" taggedAs TimingTest in { + val barrier = getBarrier() + val a, b, c = TestProbe() + barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + a.send(barrier, EnterBarrier("bar3", None)) + barrier ! NodeInfo(C, AddressFromURIString("akka://sys"), c.ref) + b.send(barrier, EnterBarrier("bar3", None)) + noMsg(a, b, c) + within(2 seconds) { + c.send(barrier, EnterBarrier("bar3", None)) + a.expectMsg(ToClient(BarrierResult("bar3", true))) + b.expectMsg(ToClient(BarrierResult("bar3", true))) + c.expectMsg(ToClient(BarrierResult("bar3", true))) + } + } + + "enter barrier with leaving node" taggedAs TimingTest in { + val barrier = getBarrier() + val a, b, c = TestProbe() + barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + barrier ! NodeInfo(C, AddressFromURIString("akka://sys"), c.ref) + a.send(barrier, EnterBarrier("bar4", None)) + b.send(barrier, EnterBarrier("bar4", None)) + barrier ! RemoveClient(A) + barrier ! ClientDisconnected(A) + noMsg(a, b, c) + b.within(2 seconds) { + barrier ! RemoveClient(C) + b.expectMsg(ToClient(BarrierResult("bar4", true))) + } + barrier ! ClientDisconnected(C) + expectNoMsg(1 second) + } + + "leave barrier when last “arrived” is removed" taggedAs TimingTest in { + val barrier = getBarrier() + val a, b = TestProbe() + barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + a.send(barrier, EnterBarrier("bar5", None)) + barrier ! RemoveClient(A) + b.send(barrier, EnterBarrier("foo", None)) + b.expectMsg(ToClient(BarrierResult("foo", true))) + } + + "fail barrier with disconnecing node" taggedAs TimingTest in { + val barrier = getBarrier() + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! nodeA + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + a.send(barrier, EnterBarrier("bar6", None)) + EventFilter[ClientLost](occurrences = 1) intercept { + barrier ! ClientDisconnected(B) + } + val msg = expectMsgType[Failed] + msg match { + case Failed(barrier, thr: ClientLost) if (thr == ClientLost(Data(Set(nodeA), "bar6", a.ref :: Nil, thr.data.deadline), B)) ⇒ + case x ⇒ fail("Expected " + Failed(barrier, ClientLost(Data(Set(nodeA), "bar6", a.ref :: Nil, null), B)) + " but got " + x) + } + } + + "fail barrier with disconnecing node who already arrived" taggedAs TimingTest in { + val barrier = getBarrier() + val a, b, c = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeC = NodeInfo(C, AddressFromURIString("akka://sys"), c.ref) + barrier ! nodeA + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + barrier ! nodeC + a.send(barrier, EnterBarrier("bar7", None)) + b.send(barrier, EnterBarrier("bar7", None)) + EventFilter[ClientLost](occurrences = 1) intercept { + barrier ! ClientDisconnected(B) + } + val msg = expectMsgType[Failed] + msg match { + case Failed(barrier, thr: ClientLost) if (thr == ClientLost(Data(Set(nodeA, nodeC), "bar7", a.ref :: Nil, thr.data.deadline), B)) ⇒ + case x ⇒ fail("Expected " + Failed(barrier, ClientLost(Data(Set(nodeA, nodeC), "bar7", a.ref :: Nil, null), B)) + " but got " + x) + } + } + + "fail when entering wrong barrier" taggedAs TimingTest in { + val barrier = getBarrier() + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! nodeA + val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + barrier ! nodeB + a.send(barrier, EnterBarrier("bar8", None)) + EventFilter[WrongBarrier](occurrences = 1) intercept { + b.send(barrier, EnterBarrier("foo", None)) + } + val msg = expectMsgType[Failed] + msg match { + case Failed(barrier, thr: WrongBarrier) if (thr == WrongBarrier("foo", b.ref, Data(Set(nodeA, nodeB), "bar8", a.ref :: Nil, thr.data.deadline))) ⇒ + case x ⇒ fail("Expected " + Failed(barrier, WrongBarrier("foo", b.ref, Data(Set(nodeA, nodeB), "bar8", a.ref :: Nil, null))) + " but got " + x) + } + } + + "fail barrier after first failure" taggedAs TimingTest in { + val barrier = getBarrier() + val a = TestProbe() + EventFilter[BarrierEmpty](occurrences = 1) intercept { + barrier ! RemoveClient(A) + } + val msg = expectMsgType[Failed] + msg match { + case Failed(barrier, thr: BarrierEmpty) if (thr == BarrierEmpty(Data(Set(), "", Nil, thr.data.deadline), "cannot remove RoleName(a): no client to remove")) ⇒ + case x ⇒ fail("Expected " + Failed(barrier, BarrierEmpty(Data(Set(), "", Nil, null), "cannot remove RoleName(a): no client to remove")) + " but got " + x) + } + barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + a.send(barrier, EnterBarrier("bar9", None)) + a.expectMsg(ToClient(BarrierResult("bar9", false))) + } + + "fail after barrier timeout" taggedAs TimingTest in { + val barrier = getBarrier() + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + barrier ! nodeA + barrier ! nodeB + a.send(barrier, EnterBarrier("bar10", None)) + EventFilter[BarrierTimeout](occurrences = 1) intercept { + val msg = expectMsgType[Failed](7 seconds) + msg match { + case Failed(barrier, thr: BarrierTimeout) if (thr == BarrierTimeout(Data(Set(nodeA, nodeB), "bar10", a.ref :: Nil, thr.data.deadline))) ⇒ + case x ⇒ fail("Expected " + Failed(barrier, BarrierTimeout(Data(Set(nodeA, nodeB), "bar10", a.ref :: Nil, null))) + " but got " + x) + } + } + } + + "fail if a node registers twice" taggedAs TimingTest in { + val barrier = getBarrier() + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeB = NodeInfo(A, AddressFromURIString("akka://sys"), b.ref) + barrier ! nodeA + EventFilter[DuplicateNode](occurrences = 1) intercept { + barrier ! nodeB + } + val msg = expectMsgType[Failed] + msg match { + case Failed(barrier, thr: DuplicateNode) if (thr == DuplicateNode(Data(Set(nodeA), "", Nil, thr.data.deadline), nodeB)) ⇒ + case x ⇒ fail("Expected " + Failed(barrier, DuplicateNode(Data(Set(nodeA), "", Nil, null), nodeB)) + " but got " + x) + } + } + + "finally have no failure messages left" taggedAs TimingTest in { + expectNoMsg(1 second) + } + + } + + "A Controller with BarrierCoordinator" must { + + "register clients and remove them" taggedAs TimingTest in { + val b = getController(1) + b ! NodeInfo(A, AddressFromURIString("akka://sys"), testActor) + expectMsg(ToClient(Done)) + b ! Remove(B) + b ! Remove(A) + EventFilter[BarrierEmpty](occurrences = 1) intercept { + b ! Remove(A) + } + } + + "register clients and disconnect them" taggedAs TimingTest in { + val b = getController(1) + b ! NodeInfo(A, AddressFromURIString("akka://sys"), testActor) + expectMsg(ToClient(Done)) + b ! ClientDisconnected(B) + expectNoMsg(1 second) + b ! ClientDisconnected(A) + expectNoMsg(1 second) + } + + "fail entering barrier when nobody registered" taggedAs TimingTest in { + val b = getController(0) + b ! EnterBarrier("b", None) + expectMsg(ToClient(BarrierResult("b", false))) + } + + "enter barrier" taggedAs TimingTest in { + val barrier = getController(2) + val a, b = TestProbe() + barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar11", None)) + noMsg(a, b) + within(2 seconds) { + b.send(barrier, EnterBarrier("bar11", None)) + a.expectMsg(ToClient(BarrierResult("bar11", true))) + b.expectMsg(ToClient(BarrierResult("bar11", true))) + } + } + + "enter barrier with joining node" taggedAs TimingTest in { + val barrier = getController(2) + val a, b, c = TestProbe() + barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar12", None)) + barrier ! NodeInfo(C, AddressFromURIString("akka://sys"), c.ref) + c.expectMsg(ToClient(Done)) + b.send(barrier, EnterBarrier("bar12", None)) + noMsg(a, b, c) + within(2 seconds) { + c.send(barrier, EnterBarrier("bar12", None)) + a.expectMsg(ToClient(BarrierResult("bar12", true))) + b.expectMsg(ToClient(BarrierResult("bar12", true))) + c.expectMsg(ToClient(BarrierResult("bar12", true))) + } + } + + "enter barrier with leaving node" taggedAs TimingTest in { + val barrier = getController(3) + val a, b, c = TestProbe() + barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + barrier ! NodeInfo(C, AddressFromURIString("akka://sys"), c.ref) + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + c.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar13", None)) + b.send(barrier, EnterBarrier("bar13", None)) + barrier ! Remove(A) + barrier ! ClientDisconnected(A) + noMsg(a, b, c) + b.within(2 seconds) { + barrier ! Remove(C) + b.expectMsg(ToClient(BarrierResult("bar13", true))) + } + barrier ! ClientDisconnected(C) + expectNoMsg(1 second) + } + + "leave barrier when last “arrived” is removed" taggedAs TimingTest in { + val barrier = getController(2) + val a, b = TestProbe() + barrier ! NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar14", None)) + barrier ! Remove(A) + b.send(barrier, EnterBarrier("foo", None)) + b.expectMsg(ToClient(BarrierResult("foo", true))) + } + + "fail barrier with disconnecing node" taggedAs TimingTest in { + val barrier = getController(2) + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! nodeA + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar15", None)) + barrier ! ClientDisconnected(RoleName("unknown")) + noMsg(a) + EventFilter[ClientLost](occurrences = 1) intercept { + barrier ! ClientDisconnected(B) + } + a.expectMsg(ToClient(BarrierResult("bar15", false))) + } + + "fail barrier with disconnecing node who already arrived" taggedAs TimingTest in { + val barrier = getController(3) + val a, b, c = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeC = NodeInfo(C, AddressFromURIString("akka://sys"), c.ref) + barrier ! nodeA + barrier ! NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + barrier ! nodeC + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + c.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar16", None)) + b.send(barrier, EnterBarrier("bar16", None)) + EventFilter[ClientLost](occurrences = 1) intercept { + barrier ! ClientDisconnected(B) + } + a.expectMsg(ToClient(BarrierResult("bar16", false))) + } + + "fail when entering wrong barrier" taggedAs TimingTest in { + val barrier = getController(2) + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + barrier ! nodeA + val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + barrier ! nodeB + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar17", None)) + EventFilter[WrongBarrier](occurrences = 1) intercept { + b.send(barrier, EnterBarrier("foo", None)) + } + a.expectMsg(ToClient(BarrierResult("bar17", false))) + b.expectMsg(ToClient(BarrierResult("foo", false))) + } + + "fail after barrier timeout" taggedAs TimingTest in { + val barrier = getController(2) + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + barrier ! nodeA + barrier ! nodeB + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar18", Option(2 seconds))) + EventFilter[BarrierTimeout](occurrences = 1) intercept { + Thread.sleep(4000) + } + b.send(barrier, EnterBarrier("bar18", None)) + a.expectMsg(ToClient(BarrierResult("bar18", false))) + b.expectMsg(ToClient(BarrierResult("bar18", false))) + } + + "fail if a node registers twice" taggedAs TimingTest in { + val controller = getController(2) + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeB = NodeInfo(A, AddressFromURIString("akka://sys"), b.ref) + controller ! nodeA + EventFilter[DuplicateNode](occurrences = 1) intercept { + controller ! nodeB + } + a.expectMsg(ToClient(BarrierResult("initial startup", false))) + b.expectMsg(ToClient(BarrierResult("initial startup", false))) + } + + "fail subsequent barriers if a node registers twice" taggedAs TimingTest in { + val controller = getController(1) + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeB = NodeInfo(A, AddressFromURIString("akka://sys"), b.ref) + controller ! nodeA + a.expectMsg(ToClient(Done)) + EventFilter[DuplicateNode](occurrences = 1) intercept { + controller ! nodeB + b.expectMsg(ToClient(BarrierResult("initial startup", false))) + } + a.send(controller, EnterBarrier("bar19", None)) + a.expectMsg(ToClient(BarrierResult("bar19", false))) + } + + "fail subsequent barriers after foreced failure" taggedAs TimingTest in { + val barrier = getController(2) + val a, b = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + barrier ! nodeA + barrier ! nodeB + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar20", Option(2 seconds))) + EventFilter[FailedBarrier](occurrences = 1) intercept { + b.send(barrier, FailBarrier("bar20")) + a.expectMsg(ToClient(BarrierResult("bar20", false))) + b.expectNoMsg(1 second) + } + a.send(barrier, EnterBarrier("bar21", None)) + b.send(barrier, EnterBarrier("bar21", None)) + a.expectMsg(ToClient(BarrierResult("bar21", false))) + b.expectMsg(ToClient(BarrierResult("bar21", false))) + } + + "timeout within the shortest timeout if the new timeout is shorter" taggedAs TimingTest in { + val barrier = getController(3) + val a, b, c = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + val nodeC = NodeInfo(C, AddressFromURIString("akka://sys"), c.ref) + barrier ! nodeA + barrier ! nodeB + barrier ! nodeC + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + c.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar22", Option(10 seconds))) + b.send(barrier, EnterBarrier("bar22", Option(2 seconds))) + EventFilter[BarrierTimeout](occurrences = 1) intercept { + Thread.sleep(4000) + } + c.send(barrier, EnterBarrier("bar22", None)) + a.expectMsg(ToClient(BarrierResult("bar22", false))) + b.expectMsg(ToClient(BarrierResult("bar22", false))) + c.expectMsg(ToClient(BarrierResult("bar22", false))) + } + + "timeout within the shortest timeout if the new timeout is longer" taggedAs TimingTest in { + val barrier = getController(3) + val a, b, c = TestProbe() + val nodeA = NodeInfo(A, AddressFromURIString("akka://sys"), a.ref) + val nodeB = NodeInfo(B, AddressFromURIString("akka://sys"), b.ref) + val nodeC = NodeInfo(C, AddressFromURIString("akka://sys"), c.ref) + barrier ! nodeA + barrier ! nodeB + barrier ! nodeC + a.expectMsg(ToClient(Done)) + b.expectMsg(ToClient(Done)) + c.expectMsg(ToClient(Done)) + a.send(barrier, EnterBarrier("bar23", Option(2 seconds))) + b.send(barrier, EnterBarrier("bar23", Option(10 seconds))) + EventFilter[BarrierTimeout](occurrences = 1) intercept { + Thread.sleep(4000) + } + c.send(barrier, EnterBarrier("bar23", None)) + a.expectMsg(ToClient(BarrierResult("bar23", false))) + b.expectMsg(ToClient(BarrierResult("bar23", false))) + c.expectMsg(ToClient(BarrierResult("bar23", false))) + } + + "finally have no failure messages left" taggedAs TimingTest in { + expectNoMsg(1 second) + } + + } + + private def getController(participants: Int): ActorRef = { + system.actorOf(Props(new Actor { + val controller = context.actorOf(Props(new Controller(participants, new InetSocketAddress(InetAddress.getLocalHost, 0)))) + controller ! GetSockAddr + override def supervisorStrategy = OneForOneStrategy() { + case x ⇒ testActor ! Failed(controller, x); SupervisorStrategy.Restart + } + def receive = { + case x: InetSocketAddress ⇒ testActor ! controller + } + })) + expectMsgType[ActorRef] + } + + /** + * Produce a BarrierCoordinator which is supervised with a strategy which + * forwards all failures to the testActor. + */ + private def getBarrier(): ActorRef = { + system.actorOf(Props(new Actor { + val barrier = context.actorOf(Props[BarrierCoordinator]) + override def supervisorStrategy = OneForOneStrategy() { + case x ⇒ testActor ! Failed(barrier, x); SupervisorStrategy.Restart + } + def receive = { + case _ ⇒ sender ! barrier + } + })) ! "" + expectMsgType[ActorRef] + } + + private def noMsg(probes: TestProbe*) { + expectNoMsg(1 second) + probes foreach (_.msgAvailable must be(false)) + } + + private def data(clients: Set[Controller.NodeInfo], barrier: String, arrived: List[ActorRef], previous: Data): Data = { + Data(clients, barrier, arrived, previous.deadline) + } +} \ No newline at end of file diff --git a/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala b/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala new file mode 100644 index 0000000000..13140adfb5 --- /dev/null +++ b/akka-remote-tests/src/test/scala/akka/remote/testconductor/ControllerSpec.scala @@ -0,0 +1,43 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.testconductor + +import akka.testkit.AkkaSpec +import akka.actor.Props +import akka.testkit.ImplicitSender +import akka.remote.testconductor.Controller.NodeInfo +import akka.actor.AddressFromURIString +import java.net.InetSocketAddress +import java.net.InetAddress + +object ControllerSpec { + val config = """ + akka.testconductor.barrier-timeout = 5s + akka.actor.provider = akka.remote.RemoteActorRefProvider + akka.remote.netty.port = 0 + akka.actor.debug.fsm = on + akka.actor.debug.lifecycle = on + """ +} + +class ControllerSpec extends AkkaSpec(ControllerSpec.config) with ImplicitSender { + + val A = RoleName("a") + val B = RoleName("b") + + "A Controller" must { + + "publish its nodes" in { + val c = system.actorOf(Props(new Controller(1, new InetSocketAddress(InetAddress.getLocalHost, 0)))) + c ! NodeInfo(A, AddressFromURIString("akka://sys"), testActor) + expectMsg(ToClient(Done)) + c ! NodeInfo(B, AddressFromURIString("akka://sys"), testActor) + expectMsg(ToClient(Done)) + c ! Controller.GetNodes + expectMsgType[Iterable[RoleName]].toSet must be(Set(A, B)) + } + + } + +} \ No newline at end of file diff --git a/akka-remote-tests/src/test/scala/akka/remote/testkit/LogRoleReplace.scala b/akka-remote-tests/src/test/scala/akka/remote/testkit/LogRoleReplace.scala new file mode 100644 index 0000000000..5dd41365bf --- /dev/null +++ b/akka-remote-tests/src/test/scala/akka/remote/testkit/LogRoleReplace.scala @@ -0,0 +1,141 @@ +package akka.remote.testkit + +import java.awt.Toolkit +import java.awt.datatransfer.Clipboard +import java.awt.datatransfer.ClipboardOwner +import java.awt.datatransfer.DataFlavor +import java.awt.datatransfer.StringSelection +import java.awt.datatransfer.Transferable +import java.io.BufferedReader +import java.io.FileReader +import java.io.FileWriter +import java.io.InputStreamReader +import java.io.OutputStreamWriter +import java.io.PrintWriter +import java.io.StringReader +import java.io.StringWriter +import scala.annotation.tailrec + +/** + * Utility to make log files from multi-node tests easier to analyze. + * Replaces jvm names and host:port with corresponding logical role name. + */ +object LogRoleReplace extends ClipboardOwner { + + /** + * Main program. Use with 0, 1 or 2 arguments. + * + * When using 0 arguments it reads from standard input + * (System.in) and writes to standard output (System.out). + * + * With 1 argument it reads from the file specified in the first argument + * and writes to standard output. + * + * With 2 arguments it reads the file specified in the first argument + * and writes to the file specified in the second argument. + * + * You can also replace the contents of the clipboard instead of using files + * by supplying `clipboard` as argument + */ + def main(args: Array[String]): Unit = { + val replacer = new LogRoleReplace + + if (args.length == 0) { + replacer.process( + new BufferedReader(new InputStreamReader(System.in)), + new PrintWriter(new OutputStreamWriter(System.out))) + + } else if (args(0) == "clipboard") { + val clipboard = Toolkit.getDefaultToolkit.getSystemClipboard + val contents = clipboard.getContents(null) + if (contents != null && contents.isDataFlavorSupported(DataFlavor.stringFlavor)) { + val text = contents.getTransferData(DataFlavor.stringFlavor).asInstanceOf[String] + val result = new StringWriter + replacer.process( + new BufferedReader(new StringReader(text)), + new PrintWriter(result)) + clipboard.setContents(new StringSelection(result.toString), this) + println("Replaced clipboard contents") + } + + } else if (args.length == 1) { + val inputFile = new BufferedReader(new FileReader(args(0))) + try { + replacer.process( + inputFile, + new PrintWriter(new OutputStreamWriter(System.out))) + } finally { + inputFile.close() + } + + } else if (args.length == 2) { + val outputFile = new PrintWriter(new FileWriter(args(1))) + val inputFile = new BufferedReader(new FileReader(args(0))) + try { + replacer.process(inputFile, outputFile) + } finally { + outputFile.close() + inputFile.close() + } + } + } + + /** + * Empty implementation of the ClipboardOwner interface + */ + def lostOwnership(clipboard: Clipboard, contents: Transferable): Unit = () +} + +class LogRoleReplace { + + private val RoleStarted = """\[([\w\-]+)\].*Role \[([\w]+)\] started with address \[akka://.*@([\w\-\.]+):([0-9]+)\]""".r + private val ColorCode = """\[[0-9]+m""" + + private var replacements: Map[String, String] = Map.empty + + def process(in: BufferedReader, out: PrintWriter): Unit = { + + @tailrec + def processLines(line: String): Unit = if (line ne null) { + out.println(processLine(line)) + processLines(in.readLine) + } + + processLines(in.readLine()) + } + + def processLine(line: String): String = { + val cleanLine = removeColorCodes(line) + if (updateReplacements(cleanLine)) + replaceLine(cleanLine) + else + cleanLine + } + + private def removeColorCodes(line: String): String = + line.replaceAll(ColorCode, "") + + private def updateReplacements(line: String): Boolean = { + if (line.startsWith("[info] * ")) { + // reset when new test begins + replacements = Map.empty + } + + line match { + case RoleStarted(jvm, role, host, port) ⇒ + replacements += (jvm -> role) + replacements += ((host + ":" + port) -> role) + false + case _ ⇒ true + } + } + + private def replaceLine(line: String): String = { + var result = line + for ((from, to) ← replacements) { + result = result.replaceAll(from, to) + } + result + } + +} \ No newline at end of file diff --git a/akka-remote-tests/src/test/scala/akka/remote/testkit/MultiNodeSpec.scala b/akka-remote-tests/src/test/scala/akka/remote/testkit/MultiNodeSpec.scala new file mode 100644 index 0000000000..838681b071 --- /dev/null +++ b/akka-remote-tests/src/test/scala/akka/remote/testkit/MultiNodeSpec.scala @@ -0,0 +1,278 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote.testkit + +import java.net.InetSocketAddress +import com.typesafe.config.{ ConfigObject, ConfigFactory, Config } +import akka.actor.{ RootActorPath, ActorPath, ActorSystem, ExtendedActorSystem } +import akka.dispatch.Await +import akka.dispatch.Await.Awaitable +import akka.remote.testconductor.{ TestConductorExt, TestConductor, RoleName } +import akka.testkit.AkkaSpec +import akka.util.{ Timeout, NonFatal } +import akka.util.duration._ +import akka.remote.RemoteActorRefProvider + +/** + * Configure the role names and participants of the test, including configuration settings. + */ +abstract class MultiNodeConfig { + + private var _commonConf: Option[Config] = None + private var _nodeConf = Map[RoleName, Config]() + private var _roles = Vector[RoleName]() + private var _deployments = Map[RoleName, Seq[String]]() + private var _allDeploy = Vector[String]() + + /** + * Register a common base config for all test participants, if so desired. + */ + def commonConfig(config: Config): Unit = _commonConf = Some(config) + + /** + * Register a config override for a specific participant. + */ + def nodeConfig(role: RoleName, config: Config): Unit = _nodeConf += role -> config + + /** + * Include for verbose debug logging + * @param on when `true` debug Config is returned, otherwise config with info logging + */ + def debugConfig(on: Boolean): Config = + if (on) + ConfigFactory.parseString(""" + akka.loglevel = DEBUG + akka.remote { + log-received-messages = on + log-sent-messages = on + } + akka.actor.debug { + receive = on + fsm = on + } + """) + else + ConfigFactory.parseString("akka.loglevel = INFO") + + /** + * Construct a RoleName and return it, to be used as an identifier in the + * test. Registration of a role name creates a role which then needs to be + * filled. + */ + def role(name: String): RoleName = { + if (_roles exists (_.name == name)) throw new IllegalArgumentException("non-unique role name " + name) + val r = RoleName(name) + _roles :+= r + r + } + + def deployOn(role: RoleName, deployment: String): Unit = + _deployments += role -> ((_deployments get role getOrElse Vector()) :+ deployment) + + def deployOnAll(deployment: String): Unit = _allDeploy :+= deployment + + private[testkit] lazy val myself: RoleName = { + require(_roles.size > MultiNodeSpec.selfIndex, "not enough roles declared for this test") + _roles(MultiNodeSpec.selfIndex) + } + + private[testkit] def config: Config = { + val configs = (_nodeConf get myself).toList ::: _commonConf.toList ::: MultiNodeSpec.nodeConfig :: AkkaSpec.testConf :: Nil + configs reduce (_ withFallback _) + } + + private[testkit] def deployments(node: RoleName): Seq[String] = (_deployments get node getOrElse Nil) ++ _allDeploy + + private[testkit] def roles: Seq[RoleName] = _roles + +} + +object MultiNodeSpec { + + /** + * Names (or IP addresses; must be resolvable using InetAddress.getByName) + * of all nodes taking part in this test, including symbolic name and host + * definition: + * + * {{{ + * -D"multinode.hosts=host1@workerA.example.com,host2@workerB.example.com" + * }}} + */ + val nodeNames: Seq[String] = Vector.empty ++ ( + Option(System.getProperty("multinode.hosts")) getOrElse + (throw new IllegalStateException("need system property multinode.hosts to be set")) split ",") + + require(nodeNames != List(""), "multinode.hosts must not be empty") + + /** + * Index of this node in the nodeNames / nodeAddresses lists. The TestConductor + * is started in “controller” mode on selfIndex 0, i.e. there you can inject + * failures and shutdown other nodes etc. + */ + val selfIndex = Option(Integer.getInteger("multinode.index")) getOrElse + (throw new IllegalStateException("need system property multinode.index to be set")) + + require(selfIndex >= 0 && selfIndex < nodeNames.size, "selfIndex out of bounds: " + selfIndex) + + val nodeConfig = AkkaSpec.mapToConfig(Map( + "akka.actor.provider" -> "akka.remote.RemoteActorRefProvider", + "akka.remote.transport" -> "akka.remote.testconductor.TestConductorTransport", + "akka.remote.netty.hostname" -> nodeNames(selfIndex), + "akka.remote.netty.port" -> 0)) + +} + +/** + * Note: To be able to run tests with everything ignored or excluded by tags + * you must not use `testconductor`, or helper methods that use `testconductor`, + * from the constructor of your test class. Otherwise the controller node might + * be shutdown before other nodes have completed and you will see errors like: + * `AskTimeoutException: sending to terminated ref breaks promises`. Using lazy + * val is fine. + */ +abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: Seq[RoleName], deployments: RoleName ⇒ Seq[String]) + extends AkkaSpec(_system) { + + import MultiNodeSpec._ + + def this(config: MultiNodeConfig) = + this(config.myself, ActorSystem(AkkaSpec.getCallerName(classOf[MultiNodeSpec]), ConfigFactory.load(config.config)), + config.roles, config.deployments) + + /* + * Test Class Interface + */ + + /** + * All registered roles + */ + def roles: Seq[RoleName] = _roles + + /** + * TO BE DEFINED BY USER: Defines the number of participants required for starting the test. This + * might not be equals to the number of nodes available to the test. + * + * Must be a `def`: + * {{{ + * def initialParticipants = 5 + * }}} + */ + def initialParticipants: Int + require(initialParticipants > 0, "initialParticipants must be a 'def' or early initializer, and it must be greater zero") + require(initialParticipants <= nodeNames.size, "not enough nodes to run this test") + + /** + * Access to the barriers, failure injection, etc. The extension will have + * been started either in Conductor or Player mode when the constructor of + * MultiNodeSpec finishes, i.e. do not call the start*() methods yourself! + */ + val testConductor: TestConductorExt = TestConductor(system) + + /** + * Execute the given block of code only on the given nodes (names according + * to the `roleMap`). + */ + def runOn(nodes: RoleName*)(thunk: ⇒ Unit): Unit = { + if (nodes exists (_ == myself)) { + thunk + } + } + + def ifNode[T](nodes: RoleName*)(yes: ⇒ T)(no: ⇒ T): T = { + if (nodes exists (_ == myself)) yes else no + } + + /** + * Enter the named barriers in the order given. Use the remaining duration from + * the innermost enclosing `within` block or the default `BarrierTimeout` + */ + def enterBarrier(name: String*) { + testConductor.enter(Timeout.durationToTimeout(remainingOr(testConductor.Settings.BarrierTimeout.duration)), name) + } + + /** + * Query the controller for the transport address of the given node (by role name) and + * return that as an ActorPath for easy composition: + * + * {{{ + * val serviceA = system.actorFor(node("master") / "user" / "serviceA") + * }}} + */ + def node(role: RoleName): ActorPath = RootActorPath(testConductor.getAddressFor(role).await) + + /** + * Enrich `.await()` onto all Awaitables, using remaining duration from the innermost + * enclosing `within` block or QueryTimeout. + */ + implicit def awaitHelper[T](w: Awaitable[T]) = new AwaitHelper(w) + class AwaitHelper[T](w: Awaitable[T]) { + def await: T = Await.result(w, remainingOr(testConductor.Settings.QueryTimeout.duration)) + } + + /* + * Implementation (i.e. wait for start etc.) + */ + + private val controllerAddr = new InetSocketAddress(nodeNames(0), 4711) + if (selfIndex == 0) { + Await.result(testConductor.startController(initialParticipants, myself, controllerAddr), + testConductor.Settings.BarrierTimeout.duration) + } else { + Await.result(testConductor.startClient(myself, controllerAddr), + testConductor.Settings.BarrierTimeout.duration) + } + + // now add deployments, if so desired + + private case class Replacement(tag: String, role: RoleName) { + lazy val addr = node(role).address.toString + } + private val replacements = roles map (r ⇒ Replacement("@" + r.name + "@", r)) + private val deployer = system.asInstanceOf[ExtendedActorSystem].provider.deployer + deployments(myself) foreach { str ⇒ + val deployString = (str /: replacements) { + case (base, r @ Replacement(tag, _)) ⇒ + base.indexOf(tag) match { + case -1 ⇒ base + case start ⇒ + val replaceWith = try + r.addr + catch { + case NonFatal(e) ⇒ + // might happen if all test cases are ignored (excluded) and + // controller node is finished/exited before r.addr is run + // on the other nodes + val unresolved = "akka://unresolved-replacement-" + r.role.name + log.warning(unresolved + " due to: " + e.getMessage) + unresolved + } + base.replace(tag, replaceWith) + } + } + import scala.collection.JavaConverters._ + ConfigFactory.parseString(deployString).root.asScala foreach { + case (key, value: ConfigObject) ⇒ + deployer.parseConfig(key, value.toConfig) foreach deployer.deploy + case (key, x) ⇒ + throw new IllegalArgumentException("key " + key + " must map to deployment section, not simple value " + x) + } + } + + // useful to see which jvm is running which role, used by LogRoleReplace utility + log.info("Role [{}] started with address [{}]", myself.name, + system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.address) + + // wait for all nodes to remove themselves before we shut the conductor down + final override def beforeShutdown() = { + if (selfIndex == 0) { + testConductor.removeNode(myself) + within(testConductor.Settings.BarrierTimeout.duration) { + awaitCond { + testConductor.getNodes.await.filterNot(_ == myself).isEmpty + } + } + } + } + +} diff --git a/akka-remote/src/main/java/akka/remote/RemoteProtocol.java b/akka-remote/src/main/java/akka/remote/RemoteProtocol.java index 21074a44c0..204a68fca5 100644 --- a/akka-remote/src/main/java/akka/remote/RemoteProtocol.java +++ b/akka-remote/src/main/java/akka/remote/RemoteProtocol.java @@ -4172,115 +4172,166 @@ public final class RemoteProtocol { // @@protoc_insertion_point(class_scope:AddressProtocol) } - public interface DurableMailboxMessageProtocolOrBuilder + public interface DaemonMsgCreateProtocolOrBuilder extends com.google.protobuf.MessageOrBuilder { - // required .ActorRefProtocol recipient = 1; - boolean hasRecipient(); - akka.remote.RemoteProtocol.ActorRefProtocol getRecipient(); - akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getRecipientOrBuilder(); + // required .PropsProtocol props = 1; + boolean hasProps(); + akka.remote.RemoteProtocol.PropsProtocol getProps(); + akka.remote.RemoteProtocol.PropsProtocolOrBuilder getPropsOrBuilder(); - // optional .ActorRefProtocol sender = 2; - boolean hasSender(); - akka.remote.RemoteProtocol.ActorRefProtocol getSender(); - akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getSenderOrBuilder(); + // required .DeployProtocol deploy = 2; + boolean hasDeploy(); + akka.remote.RemoteProtocol.DeployProtocol getDeploy(); + akka.remote.RemoteProtocol.DeployProtocolOrBuilder getDeployOrBuilder(); - // required bytes message = 3; - boolean hasMessage(); - com.google.protobuf.ByteString getMessage(); + // required string path = 3; + boolean hasPath(); + String getPath(); + + // required .ActorRefProtocol supervisor = 4; + boolean hasSupervisor(); + akka.remote.RemoteProtocol.ActorRefProtocol getSupervisor(); + akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getSupervisorOrBuilder(); } - public static final class DurableMailboxMessageProtocol extends + public static final class DaemonMsgCreateProtocol extends com.google.protobuf.GeneratedMessage - implements DurableMailboxMessageProtocolOrBuilder { - // Use DurableMailboxMessageProtocol.newBuilder() to construct. - private DurableMailboxMessageProtocol(Builder builder) { + implements DaemonMsgCreateProtocolOrBuilder { + // Use DaemonMsgCreateProtocol.newBuilder() to construct. + private DaemonMsgCreateProtocol(Builder builder) { super(builder); } - private DurableMailboxMessageProtocol(boolean noInit) {} + private DaemonMsgCreateProtocol(boolean noInit) {} - private static final DurableMailboxMessageProtocol defaultInstance; - public static DurableMailboxMessageProtocol getDefaultInstance() { + private static final DaemonMsgCreateProtocol defaultInstance; + public static DaemonMsgCreateProtocol getDefaultInstance() { return defaultInstance; } - public DurableMailboxMessageProtocol getDefaultInstanceForType() { + public DaemonMsgCreateProtocol getDefaultInstanceForType() { return defaultInstance; } public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return akka.remote.RemoteProtocol.internal_static_DurableMailboxMessageProtocol_descriptor; + return akka.remote.RemoteProtocol.internal_static_DaemonMsgCreateProtocol_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.remote.RemoteProtocol.internal_static_DurableMailboxMessageProtocol_fieldAccessorTable; + return akka.remote.RemoteProtocol.internal_static_DaemonMsgCreateProtocol_fieldAccessorTable; } private int bitField0_; - // required .ActorRefProtocol recipient = 1; - public static final int RECIPIENT_FIELD_NUMBER = 1; - private akka.remote.RemoteProtocol.ActorRefProtocol recipient_; - public boolean hasRecipient() { + // required .PropsProtocol props = 1; + public static final int PROPS_FIELD_NUMBER = 1; + private akka.remote.RemoteProtocol.PropsProtocol props_; + public boolean hasProps() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public akka.remote.RemoteProtocol.ActorRefProtocol getRecipient() { - return recipient_; + public akka.remote.RemoteProtocol.PropsProtocol getProps() { + return props_; } - public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getRecipientOrBuilder() { - return recipient_; + public akka.remote.RemoteProtocol.PropsProtocolOrBuilder getPropsOrBuilder() { + return props_; } - // optional .ActorRefProtocol sender = 2; - public static final int SENDER_FIELD_NUMBER = 2; - private akka.remote.RemoteProtocol.ActorRefProtocol sender_; - public boolean hasSender() { + // required .DeployProtocol deploy = 2; + public static final int DEPLOY_FIELD_NUMBER = 2; + private akka.remote.RemoteProtocol.DeployProtocol deploy_; + public boolean hasDeploy() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public akka.remote.RemoteProtocol.ActorRefProtocol getSender() { - return sender_; + public akka.remote.RemoteProtocol.DeployProtocol getDeploy() { + return deploy_; } - public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getSenderOrBuilder() { - return sender_; + public akka.remote.RemoteProtocol.DeployProtocolOrBuilder getDeployOrBuilder() { + return deploy_; } - // required bytes message = 3; - public static final int MESSAGE_FIELD_NUMBER = 3; - private com.google.protobuf.ByteString message_; - public boolean hasMessage() { + // required string path = 3; + public static final int PATH_FIELD_NUMBER = 3; + private java.lang.Object path_; + public boolean hasPath() { return ((bitField0_ & 0x00000004) == 0x00000004); } - public com.google.protobuf.ByteString getMessage() { - return message_; + public String getPath() { + java.lang.Object ref = path_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + path_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getPathBytes() { + java.lang.Object ref = path_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + path_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required .ActorRefProtocol supervisor = 4; + public static final int SUPERVISOR_FIELD_NUMBER = 4; + private akka.remote.RemoteProtocol.ActorRefProtocol supervisor_; + public boolean hasSupervisor() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public akka.remote.RemoteProtocol.ActorRefProtocol getSupervisor() { + return supervisor_; + } + public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getSupervisorOrBuilder() { + return supervisor_; } private void initFields() { - recipient_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); - sender_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); - message_ = com.google.protobuf.ByteString.EMPTY; + props_ = akka.remote.RemoteProtocol.PropsProtocol.getDefaultInstance(); + deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance(); + path_ = ""; + supervisor_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { byte isInitialized = memoizedIsInitialized; if (isInitialized != -1) return isInitialized == 1; - if (!hasRecipient()) { + if (!hasProps()) { memoizedIsInitialized = 0; return false; } - if (!hasMessage()) { + if (!hasDeploy()) { memoizedIsInitialized = 0; return false; } - if (!getRecipient().isInitialized()) { + if (!hasPath()) { memoizedIsInitialized = 0; return false; } - if (hasSender()) { - if (!getSender().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } + if (!hasSupervisor()) { + memoizedIsInitialized = 0; + return false; + } + if (!getProps().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getDeploy().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getSupervisor().isInitialized()) { + memoizedIsInitialized = 0; + return false; } memoizedIsInitialized = 1; return true; @@ -4290,13 +4341,16 @@ public final class RemoteProtocol { throws java.io.IOException { getSerializedSize(); if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeMessage(1, recipient_); + output.writeMessage(1, props_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, sender_); + output.writeMessage(2, deploy_); } if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, message_); + output.writeBytes(3, getPathBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(4, supervisor_); } getUnknownFields().writeTo(output); } @@ -4309,15 +4363,19 @@ public final class RemoteProtocol { size = 0; if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, recipient_); + .computeMessageSize(1, props_); } if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, sender_); + .computeMessageSize(2, deploy_); } if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(3, message_); + .computeBytesSize(3, getPathBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, supervisor_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; @@ -4331,41 +4389,41 @@ public final class RemoteProtocol { return super.writeReplace(); } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseFrom( + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseFrom( + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseFrom( com.google.protobuf.ByteString data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseFrom(byte[] data) + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseFrom(byte[] data) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data).buildParsed(); } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseFrom( + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseFrom( byte[] data, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { return newBuilder().mergeFrom(data, extensionRegistry) .buildParsed(); } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseFrom(java.io.InputStream input) + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseFrom(java.io.InputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseFrom( + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { return newBuilder().mergeFrom(input, extensionRegistry) .buildParsed(); } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseDelimitedFrom(java.io.InputStream input) + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseDelimitedFrom(java.io.InputStream input) throws java.io.IOException { Builder builder = newBuilder(); if (builder.mergeDelimitedFrom(input)) { @@ -4374,7 +4432,7 @@ public final class RemoteProtocol { return null; } } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseDelimitedFrom( + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseDelimitedFrom( java.io.InputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -4385,12 +4443,12 @@ public final class RemoteProtocol { return null; } } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseFrom( + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseFrom( com.google.protobuf.CodedInputStream input) throws java.io.IOException { return newBuilder().mergeFrom(input).buildParsed(); } - public static akka.remote.RemoteProtocol.DurableMailboxMessageProtocol parseFrom( + public static akka.remote.RemoteProtocol.DaemonMsgCreateProtocol parseFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws java.io.IOException { @@ -4400,7 +4458,7 @@ public final class RemoteProtocol { public static Builder newBuilder() { return Builder.create(); } public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(akka.remote.RemoteProtocol.DurableMailboxMessageProtocol prototype) { + public static Builder newBuilder(akka.remote.RemoteProtocol.DaemonMsgCreateProtocol prototype) { return newBuilder().mergeFrom(prototype); } public Builder toBuilder() { return newBuilder(this); } @@ -4413,18 +4471,18 @@ public final class RemoteProtocol { } public static final class Builder extends com.google.protobuf.GeneratedMessage.Builder - implements akka.remote.RemoteProtocol.DurableMailboxMessageProtocolOrBuilder { + implements akka.remote.RemoteProtocol.DaemonMsgCreateProtocolOrBuilder { public static final com.google.protobuf.Descriptors.Descriptor getDescriptor() { - return akka.remote.RemoteProtocol.internal_static_DurableMailboxMessageProtocol_descriptor; + return akka.remote.RemoteProtocol.internal_static_DaemonMsgCreateProtocol_descriptor; } protected com.google.protobuf.GeneratedMessage.FieldAccessorTable internalGetFieldAccessorTable() { - return akka.remote.RemoteProtocol.internal_static_DurableMailboxMessageProtocol_fieldAccessorTable; + return akka.remote.RemoteProtocol.internal_static_DaemonMsgCreateProtocol_fieldAccessorTable; } - // Construct using akka.remote.RemoteProtocol.DurableMailboxMessageProtocol.newBuilder() + // Construct using akka.remote.RemoteProtocol.DaemonMsgCreateProtocol.newBuilder() private Builder() { maybeForceBuilderInitialization(); } @@ -4435,8 +4493,9 @@ public final class RemoteProtocol { } private void maybeForceBuilderInitialization() { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getRecipientFieldBuilder(); - getSenderFieldBuilder(); + getPropsFieldBuilder(); + getDeployFieldBuilder(); + getSupervisorFieldBuilder(); } } private static Builder create() { @@ -4445,20 +4504,26 @@ public final class RemoteProtocol { public Builder clear() { super.clear(); - if (recipientBuilder_ == null) { - recipient_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); + if (propsBuilder_ == null) { + props_ = akka.remote.RemoteProtocol.PropsProtocol.getDefaultInstance(); } else { - recipientBuilder_.clear(); + propsBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); - if (senderBuilder_ == null) { - sender_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); + if (deployBuilder_ == null) { + deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance(); } else { - senderBuilder_.clear(); + deployBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); - message_ = com.google.protobuf.ByteString.EMPTY; + path_ = ""; bitField0_ = (bitField0_ & ~0x00000004); + if (supervisorBuilder_ == null) { + supervisor_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); + } else { + supervisorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -4468,24 +4533,24 @@ public final class RemoteProtocol { public com.google.protobuf.Descriptors.Descriptor getDescriptorForType() { - return akka.remote.RemoteProtocol.DurableMailboxMessageProtocol.getDescriptor(); + return akka.remote.RemoteProtocol.DaemonMsgCreateProtocol.getDescriptor(); } - public akka.remote.RemoteProtocol.DurableMailboxMessageProtocol getDefaultInstanceForType() { - return akka.remote.RemoteProtocol.DurableMailboxMessageProtocol.getDefaultInstance(); + public akka.remote.RemoteProtocol.DaemonMsgCreateProtocol getDefaultInstanceForType() { + return akka.remote.RemoteProtocol.DaemonMsgCreateProtocol.getDefaultInstance(); } - public akka.remote.RemoteProtocol.DurableMailboxMessageProtocol build() { - akka.remote.RemoteProtocol.DurableMailboxMessageProtocol result = buildPartial(); + public akka.remote.RemoteProtocol.DaemonMsgCreateProtocol build() { + akka.remote.RemoteProtocol.DaemonMsgCreateProtocol result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException(result); } return result; } - private akka.remote.RemoteProtocol.DurableMailboxMessageProtocol buildParsed() + private akka.remote.RemoteProtocol.DaemonMsgCreateProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - akka.remote.RemoteProtocol.DurableMailboxMessageProtocol result = buildPartial(); + akka.remote.RemoteProtocol.DaemonMsgCreateProtocol result = buildPartial(); if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); @@ -4493,77 +4558,98 @@ public final class RemoteProtocol { return result; } - public akka.remote.RemoteProtocol.DurableMailboxMessageProtocol buildPartial() { - akka.remote.RemoteProtocol.DurableMailboxMessageProtocol result = new akka.remote.RemoteProtocol.DurableMailboxMessageProtocol(this); + public akka.remote.RemoteProtocol.DaemonMsgCreateProtocol buildPartial() { + akka.remote.RemoteProtocol.DaemonMsgCreateProtocol result = new akka.remote.RemoteProtocol.DaemonMsgCreateProtocol(this); int from_bitField0_ = bitField0_; int to_bitField0_ = 0; if (((from_bitField0_ & 0x00000001) == 0x00000001)) { to_bitField0_ |= 0x00000001; } - if (recipientBuilder_ == null) { - result.recipient_ = recipient_; + if (propsBuilder_ == null) { + result.props_ = props_; } else { - result.recipient_ = recipientBuilder_.build(); + result.props_ = propsBuilder_.build(); } if (((from_bitField0_ & 0x00000002) == 0x00000002)) { to_bitField0_ |= 0x00000002; } - if (senderBuilder_ == null) { - result.sender_ = sender_; + if (deployBuilder_ == null) { + result.deploy_ = deploy_; } else { - result.sender_ = senderBuilder_.build(); + result.deploy_ = deployBuilder_.build(); } if (((from_bitField0_ & 0x00000004) == 0x00000004)) { to_bitField0_ |= 0x00000004; } - result.message_ = message_; + result.path_ = path_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + if (supervisorBuilder_ == null) { + result.supervisor_ = supervisor_; + } else { + result.supervisor_ = supervisorBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; } public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof akka.remote.RemoteProtocol.DurableMailboxMessageProtocol) { - return mergeFrom((akka.remote.RemoteProtocol.DurableMailboxMessageProtocol)other); + if (other instanceof akka.remote.RemoteProtocol.DaemonMsgCreateProtocol) { + return mergeFrom((akka.remote.RemoteProtocol.DaemonMsgCreateProtocol)other); } else { super.mergeFrom(other); return this; } } - public Builder mergeFrom(akka.remote.RemoteProtocol.DurableMailboxMessageProtocol other) { - if (other == akka.remote.RemoteProtocol.DurableMailboxMessageProtocol.getDefaultInstance()) return this; - if (other.hasRecipient()) { - mergeRecipient(other.getRecipient()); + public Builder mergeFrom(akka.remote.RemoteProtocol.DaemonMsgCreateProtocol other) { + if (other == akka.remote.RemoteProtocol.DaemonMsgCreateProtocol.getDefaultInstance()) return this; + if (other.hasProps()) { + mergeProps(other.getProps()); } - if (other.hasSender()) { - mergeSender(other.getSender()); + if (other.hasDeploy()) { + mergeDeploy(other.getDeploy()); } - if (other.hasMessage()) { - setMessage(other.getMessage()); + if (other.hasPath()) { + setPath(other.getPath()); + } + if (other.hasSupervisor()) { + mergeSupervisor(other.getSupervisor()); } this.mergeUnknownFields(other.getUnknownFields()); return this; } public final boolean isInitialized() { - if (!hasRecipient()) { + if (!hasProps()) { return false; } - if (!hasMessage()) { + if (!hasDeploy()) { return false; } - if (!getRecipient().isInitialized()) { + if (!hasPath()) { return false; } - if (hasSender()) { - if (!getSender().isInitialized()) { - - return false; - } + if (!hasSupervisor()) { + + return false; + } + if (!getProps().isInitialized()) { + + return false; + } + if (!getDeploy().isInitialized()) { + + return false; + } + if (!getSupervisor().isInitialized()) { + + return false; } return true; } @@ -4592,26 +4678,35 @@ public final class RemoteProtocol { break; } case 10: { - akka.remote.RemoteProtocol.ActorRefProtocol.Builder subBuilder = akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(); - if (hasRecipient()) { - subBuilder.mergeFrom(getRecipient()); + akka.remote.RemoteProtocol.PropsProtocol.Builder subBuilder = akka.remote.RemoteProtocol.PropsProtocol.newBuilder(); + if (hasProps()) { + subBuilder.mergeFrom(getProps()); } input.readMessage(subBuilder, extensionRegistry); - setRecipient(subBuilder.buildPartial()); + setProps(subBuilder.buildPartial()); break; } case 18: { - akka.remote.RemoteProtocol.ActorRefProtocol.Builder subBuilder = akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(); - if (hasSender()) { - subBuilder.mergeFrom(getSender()); + akka.remote.RemoteProtocol.DeployProtocol.Builder subBuilder = akka.remote.RemoteProtocol.DeployProtocol.newBuilder(); + if (hasDeploy()) { + subBuilder.mergeFrom(getDeploy()); } input.readMessage(subBuilder, extensionRegistry); - setSender(subBuilder.buildPartial()); + setDeploy(subBuilder.buildPartial()); break; } case 26: { bitField0_ |= 0x00000004; - message_ = input.readBytes(); + path_ = input.readBytes(); + break; + } + case 34: { + akka.remote.RemoteProtocol.ActorRefProtocol.Builder subBuilder = akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(); + if (hasSupervisor()) { + subBuilder.mergeFrom(getSupervisor()); + } + input.readMessage(subBuilder, extensionRegistry); + setSupervisor(subBuilder.buildPartial()); break; } } @@ -4620,219 +4715,1640 @@ public final class RemoteProtocol { private int bitField0_; - // required .ActorRefProtocol recipient = 1; - private akka.remote.RemoteProtocol.ActorRefProtocol recipient_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); + // required .PropsProtocol props = 1; + private akka.remote.RemoteProtocol.PropsProtocol props_ = akka.remote.RemoteProtocol.PropsProtocol.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< - akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> recipientBuilder_; - public boolean hasRecipient() { + akka.remote.RemoteProtocol.PropsProtocol, akka.remote.RemoteProtocol.PropsProtocol.Builder, akka.remote.RemoteProtocol.PropsProtocolOrBuilder> propsBuilder_; + public boolean hasProps() { return ((bitField0_ & 0x00000001) == 0x00000001); } - public akka.remote.RemoteProtocol.ActorRefProtocol getRecipient() { - if (recipientBuilder_ == null) { - return recipient_; + public akka.remote.RemoteProtocol.PropsProtocol getProps() { + if (propsBuilder_ == null) { + return props_; } else { - return recipientBuilder_.getMessage(); + return propsBuilder_.getMessage(); } } - public Builder setRecipient(akka.remote.RemoteProtocol.ActorRefProtocol value) { - if (recipientBuilder_ == null) { + public Builder setProps(akka.remote.RemoteProtocol.PropsProtocol value) { + if (propsBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - recipient_ = value; + props_ = value; onChanged(); } else { - recipientBuilder_.setMessage(value); + propsBuilder_.setMessage(value); } bitField0_ |= 0x00000001; return this; } - public Builder setRecipient( - akka.remote.RemoteProtocol.ActorRefProtocol.Builder builderForValue) { - if (recipientBuilder_ == null) { - recipient_ = builderForValue.build(); + public Builder setProps( + akka.remote.RemoteProtocol.PropsProtocol.Builder builderForValue) { + if (propsBuilder_ == null) { + props_ = builderForValue.build(); onChanged(); } else { - recipientBuilder_.setMessage(builderForValue.build()); + propsBuilder_.setMessage(builderForValue.build()); } bitField0_ |= 0x00000001; return this; } - public Builder mergeRecipient(akka.remote.RemoteProtocol.ActorRefProtocol value) { - if (recipientBuilder_ == null) { + public Builder mergeProps(akka.remote.RemoteProtocol.PropsProtocol value) { + if (propsBuilder_ == null) { if (((bitField0_ & 0x00000001) == 0x00000001) && - recipient_ != akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance()) { - recipient_ = - akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(recipient_).mergeFrom(value).buildPartial(); + props_ != akka.remote.RemoteProtocol.PropsProtocol.getDefaultInstance()) { + props_ = + akka.remote.RemoteProtocol.PropsProtocol.newBuilder(props_).mergeFrom(value).buildPartial(); } else { - recipient_ = value; + props_ = value; } onChanged(); } else { - recipientBuilder_.mergeFrom(value); + propsBuilder_.mergeFrom(value); } bitField0_ |= 0x00000001; return this; } - public Builder clearRecipient() { - if (recipientBuilder_ == null) { - recipient_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); + public Builder clearProps() { + if (propsBuilder_ == null) { + props_ = akka.remote.RemoteProtocol.PropsProtocol.getDefaultInstance(); onChanged(); } else { - recipientBuilder_.clear(); + propsBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000001); return this; } - public akka.remote.RemoteProtocol.ActorRefProtocol.Builder getRecipientBuilder() { + public akka.remote.RemoteProtocol.PropsProtocol.Builder getPropsBuilder() { bitField0_ |= 0x00000001; onChanged(); - return getRecipientFieldBuilder().getBuilder(); + return getPropsFieldBuilder().getBuilder(); } - public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getRecipientOrBuilder() { - if (recipientBuilder_ != null) { - return recipientBuilder_.getMessageOrBuilder(); + public akka.remote.RemoteProtocol.PropsProtocolOrBuilder getPropsOrBuilder() { + if (propsBuilder_ != null) { + return propsBuilder_.getMessageOrBuilder(); } else { - return recipient_; + return props_; } } private com.google.protobuf.SingleFieldBuilder< - akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> - getRecipientFieldBuilder() { - if (recipientBuilder_ == null) { - recipientBuilder_ = new com.google.protobuf.SingleFieldBuilder< - akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>( - recipient_, + akka.remote.RemoteProtocol.PropsProtocol, akka.remote.RemoteProtocol.PropsProtocol.Builder, akka.remote.RemoteProtocol.PropsProtocolOrBuilder> + getPropsFieldBuilder() { + if (propsBuilder_ == null) { + propsBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.RemoteProtocol.PropsProtocol, akka.remote.RemoteProtocol.PropsProtocol.Builder, akka.remote.RemoteProtocol.PropsProtocolOrBuilder>( + props_, getParentForChildren(), isClean()); - recipient_ = null; + props_ = null; } - return recipientBuilder_; + return propsBuilder_; } - // optional .ActorRefProtocol sender = 2; - private akka.remote.RemoteProtocol.ActorRefProtocol sender_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); + // required .DeployProtocol deploy = 2; + private akka.remote.RemoteProtocol.DeployProtocol deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance(); private com.google.protobuf.SingleFieldBuilder< - akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> senderBuilder_; - public boolean hasSender() { + akka.remote.RemoteProtocol.DeployProtocol, akka.remote.RemoteProtocol.DeployProtocol.Builder, akka.remote.RemoteProtocol.DeployProtocolOrBuilder> deployBuilder_; + public boolean hasDeploy() { return ((bitField0_ & 0x00000002) == 0x00000002); } - public akka.remote.RemoteProtocol.ActorRefProtocol getSender() { - if (senderBuilder_ == null) { - return sender_; + public akka.remote.RemoteProtocol.DeployProtocol getDeploy() { + if (deployBuilder_ == null) { + return deploy_; } else { - return senderBuilder_.getMessage(); + return deployBuilder_.getMessage(); } } - public Builder setSender(akka.remote.RemoteProtocol.ActorRefProtocol value) { - if (senderBuilder_ == null) { + public Builder setDeploy(akka.remote.RemoteProtocol.DeployProtocol value) { + if (deployBuilder_ == null) { if (value == null) { throw new NullPointerException(); } - sender_ = value; + deploy_ = value; onChanged(); } else { - senderBuilder_.setMessage(value); + deployBuilder_.setMessage(value); } bitField0_ |= 0x00000002; return this; } - public Builder setSender( - akka.remote.RemoteProtocol.ActorRefProtocol.Builder builderForValue) { - if (senderBuilder_ == null) { - sender_ = builderForValue.build(); + public Builder setDeploy( + akka.remote.RemoteProtocol.DeployProtocol.Builder builderForValue) { + if (deployBuilder_ == null) { + deploy_ = builderForValue.build(); onChanged(); } else { - senderBuilder_.setMessage(builderForValue.build()); + deployBuilder_.setMessage(builderForValue.build()); } bitField0_ |= 0x00000002; return this; } - public Builder mergeSender(akka.remote.RemoteProtocol.ActorRefProtocol value) { - if (senderBuilder_ == null) { + public Builder mergeDeploy(akka.remote.RemoteProtocol.DeployProtocol value) { + if (deployBuilder_ == null) { if (((bitField0_ & 0x00000002) == 0x00000002) && - sender_ != akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance()) { - sender_ = - akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(sender_).mergeFrom(value).buildPartial(); + deploy_ != akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance()) { + deploy_ = + akka.remote.RemoteProtocol.DeployProtocol.newBuilder(deploy_).mergeFrom(value).buildPartial(); } else { - sender_ = value; + deploy_ = value; } onChanged(); } else { - senderBuilder_.mergeFrom(value); + deployBuilder_.mergeFrom(value); } bitField0_ |= 0x00000002; return this; } - public Builder clearSender() { - if (senderBuilder_ == null) { - sender_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); + public Builder clearDeploy() { + if (deployBuilder_ == null) { + deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance(); onChanged(); } else { - senderBuilder_.clear(); + deployBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000002); return this; } - public akka.remote.RemoteProtocol.ActorRefProtocol.Builder getSenderBuilder() { + public akka.remote.RemoteProtocol.DeployProtocol.Builder getDeployBuilder() { bitField0_ |= 0x00000002; onChanged(); - return getSenderFieldBuilder().getBuilder(); + return getDeployFieldBuilder().getBuilder(); } - public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getSenderOrBuilder() { - if (senderBuilder_ != null) { - return senderBuilder_.getMessageOrBuilder(); + public akka.remote.RemoteProtocol.DeployProtocolOrBuilder getDeployOrBuilder() { + if (deployBuilder_ != null) { + return deployBuilder_.getMessageOrBuilder(); } else { - return sender_; + return deploy_; } } private com.google.protobuf.SingleFieldBuilder< - akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> - getSenderFieldBuilder() { - if (senderBuilder_ == null) { - senderBuilder_ = new com.google.protobuf.SingleFieldBuilder< - akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>( - sender_, + akka.remote.RemoteProtocol.DeployProtocol, akka.remote.RemoteProtocol.DeployProtocol.Builder, akka.remote.RemoteProtocol.DeployProtocolOrBuilder> + getDeployFieldBuilder() { + if (deployBuilder_ == null) { + deployBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.RemoteProtocol.DeployProtocol, akka.remote.RemoteProtocol.DeployProtocol.Builder, akka.remote.RemoteProtocol.DeployProtocolOrBuilder>( + deploy_, getParentForChildren(), isClean()); - sender_ = null; + deploy_ = null; } - return senderBuilder_; + return deployBuilder_; } - // required bytes message = 3; - private com.google.protobuf.ByteString message_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasMessage() { + // required string path = 3; + private java.lang.Object path_ = ""; + public boolean hasPath() { return ((bitField0_ & 0x00000004) == 0x00000004); } - public com.google.protobuf.ByteString getMessage() { - return message_; + public String getPath() { + java.lang.Object ref = path_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + path_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setMessage(com.google.protobuf.ByteString value) { + public Builder setPath(String value) { if (value == null) { throw new NullPointerException(); } bitField0_ |= 0x00000004; - message_ = value; + path_ = value; onChanged(); return this; } - public Builder clearMessage() { + public Builder clearPath() { bitField0_ = (bitField0_ & ~0x00000004); - message_ = getDefaultInstance().getMessage(); + path_ = getDefaultInstance().getPath(); + onChanged(); + return this; + } + void setPath(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + path_ = value; + onChanged(); + } + + // required .ActorRefProtocol supervisor = 4; + private akka.remote.RemoteProtocol.ActorRefProtocol supervisor_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> supervisorBuilder_; + public boolean hasSupervisor() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public akka.remote.RemoteProtocol.ActorRefProtocol getSupervisor() { + if (supervisorBuilder_ == null) { + return supervisor_; + } else { + return supervisorBuilder_.getMessage(); + } + } + public Builder setSupervisor(akka.remote.RemoteProtocol.ActorRefProtocol value) { + if (supervisorBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + supervisor_ = value; + onChanged(); + } else { + supervisorBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder setSupervisor( + akka.remote.RemoteProtocol.ActorRefProtocol.Builder builderForValue) { + if (supervisorBuilder_ == null) { + supervisor_ = builderForValue.build(); + onChanged(); + } else { + supervisorBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder mergeSupervisor(akka.remote.RemoteProtocol.ActorRefProtocol value) { + if (supervisorBuilder_ == null) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + supervisor_ != akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance()) { + supervisor_ = + akka.remote.RemoteProtocol.ActorRefProtocol.newBuilder(supervisor_).mergeFrom(value).buildPartial(); + } else { + supervisor_ = value; + } + onChanged(); + } else { + supervisorBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000008; + return this; + } + public Builder clearSupervisor() { + if (supervisorBuilder_ == null) { + supervisor_ = akka.remote.RemoteProtocol.ActorRefProtocol.getDefaultInstance(); + onChanged(); + } else { + supervisorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + public akka.remote.RemoteProtocol.ActorRefProtocol.Builder getSupervisorBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getSupervisorFieldBuilder().getBuilder(); + } + public akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder getSupervisorOrBuilder() { + if (supervisorBuilder_ != null) { + return supervisorBuilder_.getMessageOrBuilder(); + } else { + return supervisor_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder> + getSupervisorFieldBuilder() { + if (supervisorBuilder_ == null) { + supervisorBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.RemoteProtocol.ActorRefProtocol, akka.remote.RemoteProtocol.ActorRefProtocol.Builder, akka.remote.RemoteProtocol.ActorRefProtocolOrBuilder>( + supervisor_, + getParentForChildren(), + isClean()); + supervisor_ = null; + } + return supervisorBuilder_; + } + + // @@protoc_insertion_point(builder_scope:DaemonMsgCreateProtocol) + } + + static { + defaultInstance = new DaemonMsgCreateProtocol(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:DaemonMsgCreateProtocol) + } + + public interface PropsProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string dispatcher = 1; + boolean hasDispatcher(); + String getDispatcher(); + + // required .DeployProtocol deploy = 2; + boolean hasDeploy(); + akka.remote.RemoteProtocol.DeployProtocol getDeploy(); + akka.remote.RemoteProtocol.DeployProtocolOrBuilder getDeployOrBuilder(); + + // optional string fromClassCreator = 3; + boolean hasFromClassCreator(); + String getFromClassCreator(); + + // optional bytes creator = 4; + boolean hasCreator(); + com.google.protobuf.ByteString getCreator(); + + // optional bytes routerConfig = 5; + boolean hasRouterConfig(); + com.google.protobuf.ByteString getRouterConfig(); + } + public static final class PropsProtocol extends + com.google.protobuf.GeneratedMessage + implements PropsProtocolOrBuilder { + // Use PropsProtocol.newBuilder() to construct. + private PropsProtocol(Builder builder) { + super(builder); + } + private PropsProtocol(boolean noInit) {} + + private static final PropsProtocol defaultInstance; + public static PropsProtocol getDefaultInstance() { + return defaultInstance; + } + + public PropsProtocol getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.RemoteProtocol.internal_static_PropsProtocol_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.RemoteProtocol.internal_static_PropsProtocol_fieldAccessorTable; + } + + private int bitField0_; + // required string dispatcher = 1; + public static final int DISPATCHER_FIELD_NUMBER = 1; + private java.lang.Object dispatcher_; + public boolean hasDispatcher() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getDispatcher() { + java.lang.Object ref = dispatcher_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + dispatcher_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getDispatcherBytes() { + java.lang.Object ref = dispatcher_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + dispatcher_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required .DeployProtocol deploy = 2; + public static final int DEPLOY_FIELD_NUMBER = 2; + private akka.remote.RemoteProtocol.DeployProtocol deploy_; + public boolean hasDeploy() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.RemoteProtocol.DeployProtocol getDeploy() { + return deploy_; + } + public akka.remote.RemoteProtocol.DeployProtocolOrBuilder getDeployOrBuilder() { + return deploy_; + } + + // optional string fromClassCreator = 3; + public static final int FROMCLASSCREATOR_FIELD_NUMBER = 3; + private java.lang.Object fromClassCreator_; + public boolean hasFromClassCreator() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getFromClassCreator() { + java.lang.Object ref = fromClassCreator_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + fromClassCreator_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getFromClassCreatorBytes() { + java.lang.Object ref = fromClassCreator_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + fromClassCreator_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional bytes creator = 4; + public static final int CREATOR_FIELD_NUMBER = 4; + private com.google.protobuf.ByteString creator_; + public boolean hasCreator() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getCreator() { + return creator_; + } + + // optional bytes routerConfig = 5; + public static final int ROUTERCONFIG_FIELD_NUMBER = 5; + private com.google.protobuf.ByteString routerConfig_; + public boolean hasRouterConfig() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public com.google.protobuf.ByteString getRouterConfig() { + return routerConfig_; + } + + private void initFields() { + dispatcher_ = ""; + deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance(); + fromClassCreator_ = ""; + creator_ = com.google.protobuf.ByteString.EMPTY; + routerConfig_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasDispatcher()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasDeploy()) { + memoizedIsInitialized = 0; + return false; + } + if (!getDeploy().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getDispatcherBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, deploy_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getFromClassCreatorBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, creator_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, routerConfig_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getDispatcherBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, deploy_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, getFromClassCreatorBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(4, creator_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(5, routerConfig_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.RemoteProtocol.PropsProtocol parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.RemoteProtocol.PropsProtocol parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.RemoteProtocol.PropsProtocol parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.RemoteProtocol.PropsProtocol parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.RemoteProtocol.PropsProtocol parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.RemoteProtocol.PropsProtocol parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static akka.remote.RemoteProtocol.PropsProtocol parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.RemoteProtocol.PropsProtocol parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.RemoteProtocol.PropsProtocol parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.RemoteProtocol.PropsProtocol parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.RemoteProtocol.PropsProtocol prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.RemoteProtocol.PropsProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.RemoteProtocol.internal_static_PropsProtocol_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.RemoteProtocol.internal_static_PropsProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.RemoteProtocol.PropsProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getDeployFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + dispatcher_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + if (deployBuilder_ == null) { + deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance(); + } else { + deployBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + fromClassCreator_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + creator_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + routerConfig_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.RemoteProtocol.PropsProtocol.getDescriptor(); + } + + public akka.remote.RemoteProtocol.PropsProtocol getDefaultInstanceForType() { + return akka.remote.RemoteProtocol.PropsProtocol.getDefaultInstance(); + } + + public akka.remote.RemoteProtocol.PropsProtocol build() { + akka.remote.RemoteProtocol.PropsProtocol result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private akka.remote.RemoteProtocol.PropsProtocol buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + akka.remote.RemoteProtocol.PropsProtocol result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public akka.remote.RemoteProtocol.PropsProtocol buildPartial() { + akka.remote.RemoteProtocol.PropsProtocol result = new akka.remote.RemoteProtocol.PropsProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.dispatcher_ = dispatcher_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (deployBuilder_ == null) { + result.deploy_ = deploy_; + } else { + result.deploy_ = deployBuilder_.build(); + } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.fromClassCreator_ = fromClassCreator_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.creator_ = creator_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.routerConfig_ = routerConfig_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof akka.remote.RemoteProtocol.PropsProtocol) { + return mergeFrom((akka.remote.RemoteProtocol.PropsProtocol)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.RemoteProtocol.PropsProtocol other) { + if (other == akka.remote.RemoteProtocol.PropsProtocol.getDefaultInstance()) return this; + if (other.hasDispatcher()) { + setDispatcher(other.getDispatcher()); + } + if (other.hasDeploy()) { + mergeDeploy(other.getDeploy()); + } + if (other.hasFromClassCreator()) { + setFromClassCreator(other.getFromClassCreator()); + } + if (other.hasCreator()) { + setCreator(other.getCreator()); + } + if (other.hasRouterConfig()) { + setRouterConfig(other.getRouterConfig()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasDispatcher()) { + + return false; + } + if (!hasDeploy()) { + + return false; + } + if (!getDeploy().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + dispatcher_ = input.readBytes(); + break; + } + case 18: { + akka.remote.RemoteProtocol.DeployProtocol.Builder subBuilder = akka.remote.RemoteProtocol.DeployProtocol.newBuilder(); + if (hasDeploy()) { + subBuilder.mergeFrom(getDeploy()); + } + input.readMessage(subBuilder, extensionRegistry); + setDeploy(subBuilder.buildPartial()); + break; + } + case 26: { + bitField0_ |= 0x00000004; + fromClassCreator_ = input.readBytes(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + creator_ = input.readBytes(); + break; + } + case 42: { + bitField0_ |= 0x00000010; + routerConfig_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string dispatcher = 1; + private java.lang.Object dispatcher_ = ""; + public boolean hasDispatcher() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getDispatcher() { + java.lang.Object ref = dispatcher_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + dispatcher_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setDispatcher(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + dispatcher_ = value; + onChanged(); + return this; + } + public Builder clearDispatcher() { + bitField0_ = (bitField0_ & ~0x00000001); + dispatcher_ = getDefaultInstance().getDispatcher(); + onChanged(); + return this; + } + void setDispatcher(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + dispatcher_ = value; + onChanged(); + } + + // required .DeployProtocol deploy = 2; + private akka.remote.RemoteProtocol.DeployProtocol deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.RemoteProtocol.DeployProtocol, akka.remote.RemoteProtocol.DeployProtocol.Builder, akka.remote.RemoteProtocol.DeployProtocolOrBuilder> deployBuilder_; + public boolean hasDeploy() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.RemoteProtocol.DeployProtocol getDeploy() { + if (deployBuilder_ == null) { + return deploy_; + } else { + return deployBuilder_.getMessage(); + } + } + public Builder setDeploy(akka.remote.RemoteProtocol.DeployProtocol value) { + if (deployBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + deploy_ = value; + onChanged(); + } else { + deployBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder setDeploy( + akka.remote.RemoteProtocol.DeployProtocol.Builder builderForValue) { + if (deployBuilder_ == null) { + deploy_ = builderForValue.build(); + onChanged(); + } else { + deployBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeDeploy(akka.remote.RemoteProtocol.DeployProtocol value) { + if (deployBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + deploy_ != akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance()) { + deploy_ = + akka.remote.RemoteProtocol.DeployProtocol.newBuilder(deploy_).mergeFrom(value).buildPartial(); + } else { + deploy_ = value; + } + onChanged(); + } else { + deployBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder clearDeploy() { + if (deployBuilder_ == null) { + deploy_ = akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance(); + onChanged(); + } else { + deployBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + public akka.remote.RemoteProtocol.DeployProtocol.Builder getDeployBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getDeployFieldBuilder().getBuilder(); + } + public akka.remote.RemoteProtocol.DeployProtocolOrBuilder getDeployOrBuilder() { + if (deployBuilder_ != null) { + return deployBuilder_.getMessageOrBuilder(); + } else { + return deploy_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.RemoteProtocol.DeployProtocol, akka.remote.RemoteProtocol.DeployProtocol.Builder, akka.remote.RemoteProtocol.DeployProtocolOrBuilder> + getDeployFieldBuilder() { + if (deployBuilder_ == null) { + deployBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.RemoteProtocol.DeployProtocol, akka.remote.RemoteProtocol.DeployProtocol.Builder, akka.remote.RemoteProtocol.DeployProtocolOrBuilder>( + deploy_, + getParentForChildren(), + isClean()); + deploy_ = null; + } + return deployBuilder_; + } + + // optional string fromClassCreator = 3; + private java.lang.Object fromClassCreator_ = ""; + public boolean hasFromClassCreator() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getFromClassCreator() { + java.lang.Object ref = fromClassCreator_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + fromClassCreator_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setFromClassCreator(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + fromClassCreator_ = value; + onChanged(); + return this; + } + public Builder clearFromClassCreator() { + bitField0_ = (bitField0_ & ~0x00000004); + fromClassCreator_ = getDefaultInstance().getFromClassCreator(); + onChanged(); + return this; + } + void setFromClassCreator(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + fromClassCreator_ = value; + onChanged(); + } + + // optional bytes creator = 4; + private com.google.protobuf.ByteString creator_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasCreator() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getCreator() { + return creator_; + } + public Builder setCreator(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + creator_ = value; + onChanged(); + return this; + } + public Builder clearCreator() { + bitField0_ = (bitField0_ & ~0x00000008); + creator_ = getDefaultInstance().getCreator(); onChanged(); return this; } - // @@protoc_insertion_point(builder_scope:DurableMailboxMessageProtocol) + // optional bytes routerConfig = 5; + private com.google.protobuf.ByteString routerConfig_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasRouterConfig() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public com.google.protobuf.ByteString getRouterConfig() { + return routerConfig_; + } + public Builder setRouterConfig(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + routerConfig_ = value; + onChanged(); + return this; + } + public Builder clearRouterConfig() { + bitField0_ = (bitField0_ & ~0x00000010); + routerConfig_ = getDefaultInstance().getRouterConfig(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:PropsProtocol) } static { - defaultInstance = new DurableMailboxMessageProtocol(true); + defaultInstance = new PropsProtocol(true); defaultInstance.initFields(); } - // @@protoc_insertion_point(class_scope:DurableMailboxMessageProtocol) + // @@protoc_insertion_point(class_scope:PropsProtocol) + } + + public interface DeployProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string path = 1; + boolean hasPath(); + String getPath(); + + // optional bytes config = 2; + boolean hasConfig(); + com.google.protobuf.ByteString getConfig(); + + // optional bytes routerConfig = 3; + boolean hasRouterConfig(); + com.google.protobuf.ByteString getRouterConfig(); + + // optional bytes scope = 4; + boolean hasScope(); + com.google.protobuf.ByteString getScope(); + } + public static final class DeployProtocol extends + com.google.protobuf.GeneratedMessage + implements DeployProtocolOrBuilder { + // Use DeployProtocol.newBuilder() to construct. + private DeployProtocol(Builder builder) { + super(builder); + } + private DeployProtocol(boolean noInit) {} + + private static final DeployProtocol defaultInstance; + public static DeployProtocol getDefaultInstance() { + return defaultInstance; + } + + public DeployProtocol getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.RemoteProtocol.internal_static_DeployProtocol_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.RemoteProtocol.internal_static_DeployProtocol_fieldAccessorTable; + } + + private int bitField0_; + // required string path = 1; + public static final int PATH_FIELD_NUMBER = 1; + private java.lang.Object path_; + public boolean hasPath() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getPath() { + java.lang.Object ref = path_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + path_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getPathBytes() { + java.lang.Object ref = path_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + path_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional bytes config = 2; + public static final int CONFIG_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString config_; + public boolean hasConfig() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public com.google.protobuf.ByteString getConfig() { + return config_; + } + + // optional bytes routerConfig = 3; + public static final int ROUTERCONFIG_FIELD_NUMBER = 3; + private com.google.protobuf.ByteString routerConfig_; + public boolean hasRouterConfig() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public com.google.protobuf.ByteString getRouterConfig() { + return routerConfig_; + } + + // optional bytes scope = 4; + public static final int SCOPE_FIELD_NUMBER = 4; + private com.google.protobuf.ByteString scope_; + public boolean hasScope() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getScope() { + return scope_; + } + + private void initFields() { + path_ = ""; + config_ = com.google.protobuf.ByteString.EMPTY; + routerConfig_ = com.google.protobuf.ByteString.EMPTY; + scope_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasPath()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getPathBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, config_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, routerConfig_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, scope_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getPathBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, config_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, routerConfig_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(4, scope_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static akka.remote.RemoteProtocol.DeployProtocol parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.RemoteProtocol.DeployProtocol parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.RemoteProtocol.DeployProtocol parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static akka.remote.RemoteProtocol.DeployProtocol parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static akka.remote.RemoteProtocol.DeployProtocol parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.RemoteProtocol.DeployProtocol parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static akka.remote.RemoteProtocol.DeployProtocol parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.RemoteProtocol.DeployProtocol parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static akka.remote.RemoteProtocol.DeployProtocol parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static akka.remote.RemoteProtocol.DeployProtocol parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(akka.remote.RemoteProtocol.DeployProtocol prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.RemoteProtocol.DeployProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.RemoteProtocol.internal_static_DeployProtocol_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.RemoteProtocol.internal_static_DeployProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.RemoteProtocol.DeployProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + path_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + config_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + routerConfig_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); + scope_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return akka.remote.RemoteProtocol.DeployProtocol.getDescriptor(); + } + + public akka.remote.RemoteProtocol.DeployProtocol getDefaultInstanceForType() { + return akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance(); + } + + public akka.remote.RemoteProtocol.DeployProtocol build() { + akka.remote.RemoteProtocol.DeployProtocol result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private akka.remote.RemoteProtocol.DeployProtocol buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + akka.remote.RemoteProtocol.DeployProtocol result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public akka.remote.RemoteProtocol.DeployProtocol buildPartial() { + akka.remote.RemoteProtocol.DeployProtocol result = new akka.remote.RemoteProtocol.DeployProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.path_ = path_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.config_ = config_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.routerConfig_ = routerConfig_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.scope_ = scope_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof akka.remote.RemoteProtocol.DeployProtocol) { + return mergeFrom((akka.remote.RemoteProtocol.DeployProtocol)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(akka.remote.RemoteProtocol.DeployProtocol other) { + if (other == akka.remote.RemoteProtocol.DeployProtocol.getDefaultInstance()) return this; + if (other.hasPath()) { + setPath(other.getPath()); + } + if (other.hasConfig()) { + setConfig(other.getConfig()); + } + if (other.hasRouterConfig()) { + setRouterConfig(other.getRouterConfig()); + } + if (other.hasScope()) { + setScope(other.getScope()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasPath()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + path_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + config_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + routerConfig_ = input.readBytes(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + scope_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string path = 1; + private java.lang.Object path_ = ""; + public boolean hasPath() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getPath() { + java.lang.Object ref = path_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + path_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setPath(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + path_ = value; + onChanged(); + return this; + } + public Builder clearPath() { + bitField0_ = (bitField0_ & ~0x00000001); + path_ = getDefaultInstance().getPath(); + onChanged(); + return this; + } + void setPath(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + path_ = value; + onChanged(); + } + + // optional bytes config = 2; + private com.google.protobuf.ByteString config_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasConfig() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public com.google.protobuf.ByteString getConfig() { + return config_; + } + public Builder setConfig(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + config_ = value; + onChanged(); + return this; + } + public Builder clearConfig() { + bitField0_ = (bitField0_ & ~0x00000002); + config_ = getDefaultInstance().getConfig(); + onChanged(); + return this; + } + + // optional bytes routerConfig = 3; + private com.google.protobuf.ByteString routerConfig_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasRouterConfig() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public com.google.protobuf.ByteString getRouterConfig() { + return routerConfig_; + } + public Builder setRouterConfig(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + routerConfig_ = value; + onChanged(); + return this; + } + public Builder clearRouterConfig() { + bitField0_ = (bitField0_ & ~0x00000004); + routerConfig_ = getDefaultInstance().getRouterConfig(); + onChanged(); + return this; + } + + // optional bytes scope = 4; + private com.google.protobuf.ByteString scope_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasScope() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getScope() { + return scope_; + } + public Builder setScope(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + scope_ = value; + onChanged(); + return this; + } + public Builder clearScope() { + bitField0_ = (bitField0_ & ~0x00000008); + scope_ = getDefaultInstance().getScope(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:DeployProtocol) + } + + static { + defaultInstance = new DeployProtocol(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:DeployProtocol) } private static com.google.protobuf.Descriptors.Descriptor @@ -4871,10 +6387,20 @@ public final class RemoteProtocol { com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_AddressProtocol_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor - internal_static_DurableMailboxMessageProtocol_descriptor; + internal_static_DaemonMsgCreateProtocol_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_DurableMailboxMessageProtocol_fieldAccessorTable; + internal_static_DaemonMsgCreateProtocol_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_PropsProtocol_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_PropsProtocol_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_DeployProtocol_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_DeployProtocol_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -4900,12 +6426,18 @@ public final class RemoteProtocol { "anifest\030\003 \001(\014\"3\n\025MetadataEntryProtocol\022\013" + "\n\003key\030\001 \002(\t\022\r\n\005value\030\002 \002(\014\"A\n\017AddressPro" + "tocol\022\016\n\006system\030\001 \002(\t\022\020\n\010hostname\030\002 \002(\t\022" + - "\014\n\004port\030\003 \002(\r\"y\n\035DurableMailboxMessagePr" + - "otocol\022$\n\trecipient\030\001 \002(\0132\021.ActorRefProt" + - "ocol\022!\n\006sender\030\002 \001(\0132\021.ActorRefProtocol\022" + - "\017\n\007message\030\003 \002(\014*7\n\013CommandType\022\013\n\007CONNE", - "CT\020\001\022\014\n\010SHUTDOWN\020\002\022\r\n\tHEARTBEAT\020\003B\017\n\013akk" + - "a.remoteH\001" + "\014\n\004port\030\003 \002(\r\"\216\001\n\027DaemonMsgCreateProtoco" + + "l\022\035\n\005props\030\001 \002(\0132\016.PropsProtocol\022\037\n\006depl" + + "oy\030\002 \002(\0132\017.DeployProtocol\022\014\n\004path\030\003 \002(\t\022" + + "%\n\nsupervisor\030\004 \002(\0132\021.ActorRefProtocol\"\205", + "\001\n\rPropsProtocol\022\022\n\ndispatcher\030\001 \002(\t\022\037\n\006" + + "deploy\030\002 \002(\0132\017.DeployProtocol\022\030\n\020fromCla" + + "ssCreator\030\003 \001(\t\022\017\n\007creator\030\004 \001(\014\022\024\n\014rout" + + "erConfig\030\005 \001(\014\"S\n\016DeployProtocol\022\014\n\004path" + + "\030\001 \002(\t\022\016\n\006config\030\002 \001(\014\022\024\n\014routerConfig\030\003" + + " \001(\014\022\r\n\005scope\030\004 \001(\014*7\n\013CommandType\022\013\n\007CO" + + "NNECT\020\001\022\014\n\010SHUTDOWN\020\002\022\r\n\tHEARTBEAT\020\003B\017\n\013" + + "akka.remoteH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -4968,14 +6500,30 @@ public final class RemoteProtocol { new java.lang.String[] { "System", "Hostname", "Port", }, akka.remote.RemoteProtocol.AddressProtocol.class, akka.remote.RemoteProtocol.AddressProtocol.Builder.class); - internal_static_DurableMailboxMessageProtocol_descriptor = + internal_static_DaemonMsgCreateProtocol_descriptor = getDescriptor().getMessageTypes().get(7); - internal_static_DurableMailboxMessageProtocol_fieldAccessorTable = new + internal_static_DaemonMsgCreateProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_DurableMailboxMessageProtocol_descriptor, - new java.lang.String[] { "Recipient", "Sender", "Message", }, - akka.remote.RemoteProtocol.DurableMailboxMessageProtocol.class, - akka.remote.RemoteProtocol.DurableMailboxMessageProtocol.Builder.class); + internal_static_DaemonMsgCreateProtocol_descriptor, + new java.lang.String[] { "Props", "Deploy", "Path", "Supervisor", }, + akka.remote.RemoteProtocol.DaemonMsgCreateProtocol.class, + akka.remote.RemoteProtocol.DaemonMsgCreateProtocol.Builder.class); + internal_static_PropsProtocol_descriptor = + getDescriptor().getMessageTypes().get(8); + internal_static_PropsProtocol_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_PropsProtocol_descriptor, + new java.lang.String[] { "Dispatcher", "Deploy", "FromClassCreator", "Creator", "RouterConfig", }, + akka.remote.RemoteProtocol.PropsProtocol.class, + akka.remote.RemoteProtocol.PropsProtocol.Builder.class); + internal_static_DeployProtocol_descriptor = + getDescriptor().getMessageTypes().get(9); + internal_static_DeployProtocol_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_DeployProtocol_descriptor, + new java.lang.String[] { "Path", "Config", "RouterConfig", "Scope", }, + akka.remote.RemoteProtocol.DeployProtocol.class, + akka.remote.RemoteProtocol.DeployProtocol.Builder.class); return null; } }; diff --git a/akka-remote/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto index 730c3a5883..ddcfe26d1d 100644 --- a/akka-remote/src/main/protocol/RemoteProtocol.proto +++ b/akka-remote/src/main/protocol/RemoteProtocol.proto @@ -9,6 +9,8 @@ option optimize_for = SPEED; Compile with: cd ./akka-remote/src/main/protocol protoc RemoteProtocol.proto --java_out ../java + cd ../../../.. + ./scripts/fix-protobuf.sh *******************************************/ message AkkaRemoteProtocol { @@ -79,10 +81,32 @@ message AddressProtocol { } /** - * Defines the durable mailbox message. + * Defines akka.remote.DaemonMsgCreate */ -message DurableMailboxMessageProtocol { - required ActorRefProtocol recipient= 1; - optional ActorRefProtocol sender = 2; - required bytes message = 3; +message DaemonMsgCreateProtocol { + required PropsProtocol props = 1; + required DeployProtocol deploy = 2; + required string path = 3; + required ActorRefProtocol supervisor = 4; } + +/** + * Serialization of akka.actor.Props + */ +message PropsProtocol { + required string dispatcher = 1; + required DeployProtocol deploy = 2; + optional string fromClassCreator = 3; + optional bytes creator = 4; + optional bytes routerConfig = 5; +} + +/** + * Serialization of akka.actor.Deploy + */ +message DeployProtocol { + required string path = 1; + optional bytes config = 2; + optional bytes routerConfig = 3; + optional bytes scope = 4; +} \ No newline at end of file diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 4512ea3a98..f365d5ce19 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -14,6 +14,7 @@ akka { serializers { proto = "akka.serialization.ProtobufSerializer" + daemon-create = "akka.serialization.DaemonMsgCreateSerializer" } @@ -21,6 +22,7 @@ akka { # Since com.google.protobuf.Message does not extend Serializable but GeneratedMessage # does, need to use the more specific one here in order to avoid ambiguity "com.google.protobuf.GeneratedMessage" = proto + "akka.remote.DaemonMsgCreate" = daemon-create } deployment { @@ -56,8 +58,8 @@ akka { # default is a TCP-based remote transport based on Netty transport = "akka.remote.netty.NettyRemoteTransport" - # Enable untrusted mode for full security of server managed actors, allows - # untrusted clients to connect. + # Enable untrusted mode for full security of server managed actors, prevents system messages to be send + # by clients, e.g. messages like 'Create', 'Suspend', 'Resume', 'Terminate', 'Supervise', 'Link' etc. untrusted-mode = off # Timeout for ACK of cluster operations, like checking actor out etc. @@ -69,6 +71,9 @@ akka { # If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged log-sent-messages = off + # If this is "on", Akka will log all RemoteLifeCycleEvents at the level defined for each, if off then they are not logged + log-remote-lifecycle-events = off + # Each property is annotated with (I) or (O) or (I&O), where I stands for “inbound” and O for “outbound” connections. # The NettyRemoteTransport always starts the server role to allow inbound connections, and it starts # active client connections whenever sending to a destination which is not yet connected; if configured @@ -120,7 +125,8 @@ akka { # (I) Length in akka.time-unit how long core threads will be kept alive if idling execution-pool-keepalive = 60s - # (I) Size of the core pool of the remote execution unit + # (I) Size in number of threads of the core pool of the remote execution unit. + # A value of 0 will turn this off, which is can lead to deadlocks under some configurations! execution-pool-size = 4 # (I) Maximum channel size, 0 for off @@ -129,6 +135,18 @@ akka { # (I) Maximum total size of all channels, 0 for off max-total-memory-size = 0b + # (I&O) Sets the high water mark for the in and outbound sockets, set to 0b for platform default + write-buffer-high-water-mark = 0b + + # (I&O) Sets the low water mark for the in and outbound sockets, set to 0b for platform default + write-buffer-low-water-mark = 0b + + # (I&O) Sets the send buffer size of the Sockets, set to 0b for platform default + send-buffer-size = 0b + + # (I&O) Sets the receive buffer size of the Sockets, set to 0b for platform default + receive-buffer-size = 0b + # (O) Time between reconnect attempts for active clients reconnect-delay = 5s @@ -151,6 +169,52 @@ akka { # (O) Maximum time window that a client should try to reconnect for reconnection-time-window = 600s + + ssl { + # (I&O) Enable SSL/TLS encryption. + # This must be enabled on both the client and server to work. + enable = off + + # (I) This is the Java Key Store used by the server connection + key-store = "keystore" + + # This password is used for decrypting the key store + key-store-password = "changeme" + + # (O) This is the Java Key Store used by the client connection + trust-store = "truststore" + + # This password is used for decrypting the trust store + trust-store-password = "changeme" + + # (I&O) Protocol to use for SSL encryption, choose from: + # Java 6 & 7: + # 'SSLv3', 'TLSv1' + # Java 7: + # 'TLSv1.1', 'TLSv1.2' + protocol = "TLSv1" + + # Examples: [ "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA" ] + # You need to install the JCE Unlimited Strength Jurisdiction Policy Files to use AES 256 + # More info here: http://docs.oracle.com/javase/7/docs/technotes/guides/security/SunProviders.html#SunJCEProvider + enabled-algorithms = ["TLS_RSA_WITH_AES_128_CBC_SHA"] + + # Using /dev/./urandom is only necessary when using SHA1PRNG on Linux to prevent blocking + # It is NOT as secure because it reuses the seed + # '' => defaults to /dev/random or whatever is set in java.security for example: securerandom.source=file:/dev/random + # '/dev/./urandom' => NOT '/dev/urandom' as that doesn't work according to: http://bugs.sun.com/view_bug.do?bug_id=6202721 + sha1prng-random-source = "" + + # There are three options, in increasing order of security: + # "" or SecureRandom => (default) + # "SHA1PRNG" => Can be slow because of blocking issues on Linux + # "AES128CounterSecureRNG" => fastest startup and based on AES encryption algorithm + # The following use one of 3 possible seed sources, depending on availability: /dev/random, random.org and SecureRandom (provided by Java) + # "AES128CounterInetRNG" + # "AES256CounterInetRNG" (Install JCE Unlimited Strength Jurisdiction Policy Files first) + # Setting a value here may require you to supply the appropriate cipher suite (see enabled-algorithms section above) + random-number-generator = "" + } } } } diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 65777d49ca..6bd61dd812 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -9,8 +9,14 @@ import com.google.protobuf.ByteString import akka.actor.ExtendedActorSystem import akka.serialization.SerializationExtension -object MessageSerializer { +/** + * MessageSerializer is a helper for serialize and deserialize messages + */ +private[akka] object MessageSerializer { + /** + * Uses Akka Serialization for the specified ActorSystem to transform the given MessageProtocol to a message + */ def deserialize(system: ExtendedActorSystem, messageProtocol: MessageProtocol): AnyRef = { val clazz = if (messageProtocol.hasMessageManifest) { @@ -24,6 +30,9 @@ object MessageSerializer { } } + /** + * Uses Akka Serialization for the specified ActorSystem to transform the given message to a MessageProtocol + */ def serialize(system: ExtendedActorSystem, message: AnyRef): MessageProtocol = { val s = SerializationExtension(system) val serializer = s.findSerializerFor(message) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index ab53d9e99d..cdf9ad9d70 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -4,34 +4,26 @@ package akka.remote -import akka.AkkaException import akka.actor._ import akka.dispatch._ -import akka.event.{ DeathWatch, Logging, LoggingAdapter } +import akka.event.{ Logging, LoggingAdapter } import akka.event.EventStream -import akka.config.ConfigurationException -import java.util.concurrent.{ TimeoutException } -import com.typesafe.config.Config import akka.serialization.Serialization import akka.serialization.SerializationExtension -class RemoteException(msg: String) extends AkkaException(msg) -class RemoteCommunicationException(msg: String) extends RemoteException(msg) -class RemoteConnectionException(msg: String) extends RemoteException(msg) - /** * Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it. */ -class RemoteActorRefProvider( +private[akka] class RemoteActorRefProvider( val systemName: String, val settings: ActorSystem.Settings, val eventStream: EventStream, val scheduler: Scheduler, val dynamicAccess: DynamicAccess) extends ActorRefProvider { - val remoteSettings = new RemoteSettings(settings.config, systemName) + val remoteSettings: RemoteSettings = new RemoteSettings(settings.config, systemName) - val deployer = new RemoteDeployer(settings, dynamicAccess) + val deployer: RemoteDeployer = new RemoteDeployer(settings, dynamicAccess) private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer) @@ -39,21 +31,19 @@ class RemoteActorRefProvider( private var _log = local.log def log: LoggingAdapter = _log - def rootPath = local.rootPath - def deadLetters = local.deadLetters - - val deathWatch = new RemoteDeathWatch(local.deathWatch, this) + override def rootPath: ActorPath = local.rootPath + override def deadLetters: InternalActorRef = local.deadLetters // these are only available after init() - def rootGuardian = local.rootGuardian - def guardian = local.guardian - def systemGuardian = local.systemGuardian - def terminationFuture = local.terminationFuture - def dispatcher = local.dispatcher - def registerTempActor(actorRef: InternalActorRef, path: ActorPath) = local.registerTempActor(actorRef, path) - def unregisterTempActor(path: ActorPath) = local.unregisterTempActor(path) - def tempPath() = local.tempPath() - def tempContainer = local.tempContainer + override def rootGuardian: InternalActorRef = local.rootGuardian + override def guardian: LocalActorRef = local.guardian + override def systemGuardian: LocalActorRef = local.systemGuardian + override def terminationFuture: Promise[Unit] = local.terminationFuture + override def dispatcher: MessageDispatcher = local.dispatcher + override def registerTempActor(actorRef: InternalActorRef, path: ActorPath): Unit = local.registerTempActor(actorRef, path) + override def unregisterTempActor(path: ActorPath): Unit = local.unregisterTempActor(path) + override def tempPath(): ActorPath = local.tempPath() + override def tempContainer: VirtualPathContainer = local.tempContainer @volatile private var _transport: RemoteTransport = _ @@ -61,13 +51,13 @@ class RemoteActorRefProvider( @volatile private var _serialization: Serialization = _ - def serialization = _serialization + def serialization: Serialization = _serialization @volatile private var _remoteDaemon: InternalActorRef = _ - def remoteDaemon = _remoteDaemon + def remoteDaemon: InternalActorRef = _remoteDaemon - def init(system: ActorSystemImpl) { + def init(system: ActorSystemImpl): Unit = { local.init(system) _remoteDaemon = new RemoteSystemDaemon(system, rootPath / "remote", rootGuardian, log) @@ -106,8 +96,8 @@ class RemoteActorRefProvider( } def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, - systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean): InternalActorRef = { - if (systemService) local.actorOf(system, props, supervisor, path, systemService, deploy, lookupDeploy) + systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = { + if (systemService) local.actorOf(system, props, supervisor, path, systemService, deploy, lookupDeploy, async) else { /* @@ -165,14 +155,14 @@ class RemoteActorRefProvider( Iterator(props.deploy) ++ deployment.iterator reduce ((a, b) ⇒ b withFallback a) match { case d @ Deploy(_, _, _, RemoteScope(addr)) ⇒ if (addr == rootPath.address || addr == transport.address) { - local.actorOf(system, props, supervisor, path, false, deployment.headOption, false) + local.actorOf(system, props, supervisor, path, false, deployment.headOption, false, async) } else { val rpath = RootActorPath(addr) / "remote" / transport.address.hostPort / path.elements useActorOnNode(rpath, props, d, supervisor) new RemoteActorRef(this, transport, rpath, supervisor) } - case _ ⇒ local.actorOf(system, props, supervisor, path, systemService, deployment.headOption, false) + case _ ⇒ local.actorOf(system, props, supervisor, path, systemService, deployment.headOption, false, async) } } } @@ -193,7 +183,7 @@ class RemoteActorRefProvider( /** * Using (checking out) actor on a specific node. */ - def useActorOnNode(path: ActorPath, props: Props, deploy: Deploy, supervisor: ActorRef) { + def useActorOnNode(path: ActorPath, props: Props, deploy: Deploy, supervisor: ActorRef): Unit = { log.debug("[{}] Instantiating Remote Actor [{}]", rootPath, path) // we don’t wait for the ACK, because the remote end will process this command before any other message to the new actor @@ -211,7 +201,7 @@ class RemoteActorRefProvider( } } -trait RemoteRef extends ActorRefScope { +private[akka] trait RemoteRef extends ActorRefScope { final def isLocal = false } @@ -254,26 +244,4 @@ private[akka] class RemoteActorRef private[akka] ( @throws(classOf[java.io.ObjectStreamException]) private def writeReplace(): AnyRef = SerializedActorRef(path) -} - -class RemoteDeathWatch(val local: LocalDeathWatch, val provider: RemoteActorRefProvider) extends DeathWatch { - - def subscribe(watcher: ActorRef, watched: ActorRef): Boolean = watched match { - case r: RemoteRef ⇒ - val ret = local.subscribe(watcher, watched) - provider.actorFor(r.path.root / "remote") ! DaemonMsgWatch(watcher, watched) - ret - case l: LocalRef ⇒ - local.subscribe(watcher, watched) - case _ ⇒ - provider.log.error("unknown ActorRef type {} as DeathWatch target", watched.getClass) - false - } - - def unsubscribe(watcher: ActorRef, watched: ActorRef): Boolean = local.unsubscribe(watcher, watched) - - def unsubscribe(watcher: ActorRef): Unit = local.unsubscribe(watcher) - - def publish(event: Terminated): Unit = local.publish(event) - -} +} \ No newline at end of file diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index 295227081a..53023687c0 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -1,17 +1,18 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote import scala.annotation.tailrec - import akka.actor.{ VirtualPathContainer, Terminated, Deploy, Props, Nobody, LocalActorRef, InternalActorRef, Address, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor } import akka.event.LoggingAdapter +import akka.dispatch.Watch +import akka.actor.ActorRefWithCell +import akka.actor.ActorRefScope private[akka] sealed trait DaemonMsg private[akka] case class DaemonMsgCreate(props: Props, deploy: Deploy, path: String, supervisor: ActorRef) extends DaemonMsg -private[akka] case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMsg /** * Internal system "daemon" actor for remote internal communication. @@ -60,22 +61,20 @@ private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath val subpath = elems.drop(1) val path = this.path / subpath val actor = system.provider.actorOf(system, props, supervisor.asInstanceOf[InternalActorRef], - path, false, Some(deploy), true) + path, systemService = false, Some(deploy), lookupDeploy = true, async = false) addChild(subpath.mkString("/"), actor) - system.deathWatch.subscribe(this, actor) + this.sendSystemMessage(Watch(actor, this)) case _ ⇒ log.error("remote path does not match path from message [{}]", message) } - case DaemonMsgWatch(watcher, watched) ⇒ - val other = system.actorFor(watcher.path.root / "remote") - system.deathWatch.subscribe(other, watched) } - case Terminated(child: LocalActorRef) ⇒ removeChild(child.path.elements.drop(1).mkString("/")) + case Terminated(child: ActorRefWithCell) if child.asInstanceOf[ActorRefScope].isLocal ⇒ + removeChild(child.path.elements.drop(1).mkString("/")) - case t: Terminated ⇒ system.deathWatch.publish(t) + case t: Terminated ⇒ - case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this) + case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this) } } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala index 0858c66405..58a3b8452d 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala @@ -6,15 +6,14 @@ package akka.remote import akka.actor._ import akka.routing._ import com.typesafe.config._ -import akka.config.ConfigurationException +import akka.ConfigurationException case class RemoteScope(node: Address) extends Scope { def withFallback(other: Scope): Scope = this } -class RemoteDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extends Deployer(_settings, _pm) { - - override protected def parseConfig(path: String, config: Config): Option[Deploy] = { +private[akka] class RemoteDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extends Deployer(_settings, _pm) { + override def parseConfig(path: String, config: Config): Option[Deploy] = { import scala.collection.JavaConverters._ super.parseConfig(path, config) match { @@ -30,5 +29,4 @@ class RemoteDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extend case None ⇒ None } } - } \ No newline at end of file diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index ef30206a42..88a7003309 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -6,17 +6,13 @@ package akka.remote import com.typesafe.config.Config import akka.util.Duration import java.util.concurrent.TimeUnit.MILLISECONDS -import java.net.InetAddress -import akka.config.ConfigurationException -import scala.collection.JavaConverters._ -import akka.actor.Address -import akka.actor.AddressFromURIString class RemoteSettings(val config: Config, val systemName: String) { import config._ - val RemoteTransport = getString("akka.remote.transport") - val LogReceive = getBoolean("akka.remote.log-received-messages") - val LogSend = getBoolean("akka.remote.log-sent-messages") - val RemoteSystemDaemonAckTimeout = Duration(getMilliseconds("akka.remote.remote-daemon-ack-timeout"), MILLISECONDS) - val UntrustedMode = getBoolean("akka.remote.untrusted-mode") + val RemoteTransport: String = getString("akka.remote.transport") + val LogReceive: Boolean = getBoolean("akka.remote.log-received-messages") + val LogSend: Boolean = getBoolean("akka.remote.log-sent-messages") + val RemoteSystemDaemonAckTimeout: Duration = Duration(getMilliseconds("akka.remote.remote-daemon-ack-timeout"), MILLISECONDS) + val UntrustedMode: Boolean = getBoolean("akka.remote.untrusted-mode") + val LogRemoteLifeCycleEvents: Boolean = getBoolean("akka.remote.log-remote-lifecycle-events") } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index 3bade97460..f91c5b03d0 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -10,7 +10,6 @@ import akka.event.{ LoggingAdapter, Logging } import akka.AkkaException import akka.serialization.Serialization import akka.remote.RemoteProtocol._ -import akka.dispatch.ChildTerminated import akka.actor._ /** @@ -27,57 +26,55 @@ trait RemoteClientLifeCycleEvent extends RemoteLifeCycleEvent { def remoteAddress: Address } +/** + * A RemoteClientError is a general error that is thrown within or from a RemoteClient + */ case class RemoteClientError( @BeanProperty cause: Throwable, @transient @BeanProperty remote: RemoteTransport, @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.ErrorLevel - override def toString = - "RemoteClientError@" + remoteAddress + ": Error[" + cause + "]" + override def logLevel: Logging.LogLevel = Logging.ErrorLevel + override def toString: String = "RemoteClientError@" + remoteAddress + ": Error[" + cause + "]" } +/** + * RemoteClientDisconnected is published when a RemoteClient's connection is disconnected + */ case class RemoteClientDisconnected( @transient @BeanProperty remote: RemoteTransport, @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteClientDisconnected@" + remoteAddress + override def logLevel: Logging.LogLevel = Logging.DebugLevel + override def toString: String = "RemoteClientDisconnected@" + remoteAddress } +/** + * RemoteClientConnected is published when a RemoteClient's connection is established + */ case class RemoteClientConnected( @transient @BeanProperty remote: RemoteTransport, @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteClientConnected@" + remoteAddress + override def logLevel: Logging.LogLevel = Logging.DebugLevel + override def toString: String = "RemoteClientConnected@" + remoteAddress } +/** + * RemoteClientStarted is published when a RemoteClient has started up + */ case class RemoteClientStarted( @transient @BeanProperty remote: RemoteTransport, @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.InfoLevel - override def toString = - "RemoteClientStarted@" + remoteAddress + override def logLevel: Logging.LogLevel = Logging.InfoLevel + override def toString: String = "RemoteClientStarted@" + remoteAddress } +/** + * RemoteClientShutdown is published when a RemoteClient has shut down + */ case class RemoteClientShutdown( @transient @BeanProperty remote: RemoteTransport, @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.InfoLevel - override def toString = - "RemoteClientShutdown@" + remoteAddress -} - -case class RemoteClientWriteFailed( - @BeanProperty request: AnyRef, - @BeanProperty cause: Throwable, - @transient @BeanProperty remote: RemoteTransport, - @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.WarningLevel - override def toString = - "RemoteClientWriteFailed@" + remoteAddress + - ": MessageClass[" + (if (request ne null) request.getClass.getName else "no message") + - "] Error[" + cause + "]" + override def logLevel: Logging.LogLevel = Logging.InfoLevel + override def toString: String = "RemoteClientShutdown@" + remoteAddress } /** @@ -85,53 +82,66 @@ case class RemoteClientWriteFailed( */ trait RemoteServerLifeCycleEvent extends RemoteLifeCycleEvent +/** + * RemoteServerStarted is published when a local RemoteServer has started up + */ case class RemoteServerStarted( @transient @BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.InfoLevel - override def toString = - "RemoteServerStarted@" + remote + override def logLevel: Logging.LogLevel = Logging.InfoLevel + override def toString: String = "RemoteServerStarted@" + remote } +/** + * RemoteServerShutdown is published when a local RemoteServer has shut down + */ case class RemoteServerShutdown( @transient @BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.InfoLevel - override def toString = - "RemoteServerShutdown@" + remote + override def logLevel: Logging.LogLevel = Logging.InfoLevel + override def toString: String = "RemoteServerShutdown@" + remote } +/** + * A RemoteServerError is a general error that is thrown within or from a RemoteServer + */ case class RemoteServerError( @BeanProperty val cause: Throwable, @transient @BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.ErrorLevel - override def toString = - "RemoteServerError@" + remote + "] Error[" + cause + "]" + + override def logLevel: Logging.LogLevel = Logging.ErrorLevel + override def toString: String = "RemoteServerError@" + remote + "] Error[" + cause + "]" } +/** + * RemoteServerClientConnected is published when an inbound connection has been established + */ case class RemoteServerClientConnected( @transient @BeanProperty remote: RemoteTransport, @BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteServerClientConnected@" + remote + - ": Client[" + clientAddress.getOrElse("no address") + "]" + override def logLevel: Logging.LogLevel = Logging.DebugLevel + override def toString: String = + "RemoteServerClientConnected@" + remote + ": Client[" + clientAddress.getOrElse("no address") + "]" } +/** + * RemoteServerClientConnected is published when an inbound connection has been disconnected + */ case class RemoteServerClientDisconnected( @transient @BeanProperty remote: RemoteTransport, @BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteServerClientDisconnected@" + remote + - ": Client[" + clientAddress.getOrElse("no address") + "]" + override def logLevel: Logging.LogLevel = Logging.DebugLevel + override def toString: String = + "RemoteServerClientDisconnected@" + remote + ": Client[" + clientAddress.getOrElse("no address") + "]" } +/** + * RemoteServerClientClosed is published when an inbound RemoteClient is closed + */ case class RemoteServerClientClosed( @transient @BeanProperty remote: RemoteTransport, @BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteServerClientClosed@" + remote + - ": Client[" + clientAddress.getOrElse("no address") + "]" + override def logLevel: Logging.LogLevel = Logging.DebugLevel + override def toString: String = + "RemoteServerClientClosed@" + remote + ": Client[" + clientAddress.getOrElse("no address") + "]" } /** @@ -142,6 +152,10 @@ class RemoteClientException private[akka] ( @transient @BeanProperty val client: RemoteTransport, val remoteAddress: Address, cause: Throwable = null) extends AkkaException(message, cause) +/** + * RemoteTransportException represents a general failure within a RemoteTransport, + * such as inability to start, wrong configuration etc. + */ class RemoteTransportException(message: String, cause: Throwable) extends AkkaException(message, cause) /** @@ -178,71 +192,61 @@ abstract class RemoteTransport(val system: ExtendedActorSystem, val provider: Re */ def restartClientConnection(address: Address): Boolean - /** Methods that needs to be implemented by a transport **/ - - def send(message: Any, - senderOption: Option[ActorRef], - recipient: RemoteActorRef): Unit + /** + * Sends the given message to the recipient supplying the sender if any + */ + def send(message: Any, senderOption: Option[ActorRef], recipient: RemoteActorRef): Unit + /** + * Default implementation both publishes the message to the eventStream as well as logs it using the system logger + */ def notifyListeners(message: RemoteLifeCycleEvent): Unit = { system.eventStream.publish(message) - system.log.log(message.logLevel, "{}", message) + if (logRemoteLifeCycleEvents) log.log(message.logLevel, "{}", message) } - override def toString = address.toString -} - -class RemoteMessage(input: RemoteMessageProtocol, system: ExtendedActorSystem) { - - def originalReceiver = input.getRecipient.getPath - - lazy val sender: ActorRef = - if (input.hasSender) system.provider.actorFor(system.provider.rootGuardian, input.getSender.getPath) - else system.deadLetters - - lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver) - - lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage) - - override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender -} - -trait RemoteMarshallingOps { + /** + * Returns this RemoteTransports Address' textual representation + */ + override def toString: String = address.toString + /** + * A Logger that can be used to log issues that may occur + */ def log: LoggingAdapter - def system: ExtendedActorSystem - - def provider: RemoteActorRefProvider - - def address: Address - + /** + * When this method returns true, some functionality will be turned off for security purposes. + */ protected def useUntrustedMode: Boolean - def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = { - val arp = AkkaRemoteProtocol.newBuilder - arp.setMessage(rmp) - arp.build - } + /** + * When this method returns true, RemoteLifeCycleEvents will be logged as well as be put onto the eventStream. + */ + protected def logRemoteLifeCycleEvents: Boolean - def createControlEnvelope(rcp: RemoteControlProtocol): AkkaRemoteProtocol = { - val arp = AkkaRemoteProtocol.newBuilder - arp.setInstruction(rcp) - arp.build - } + /** + * Returns a newly created AkkaRemoteProtocol with the given message payload. + */ + def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = + AkkaRemoteProtocol.newBuilder.setMessage(rmp).build + + /** + * Returns a newly created AkkaRemoteProtocol with the given control payload. + */ + def createControlEnvelope(rcp: RemoteControlProtocol): AkkaRemoteProtocol = + AkkaRemoteProtocol.newBuilder.setInstruction(rcp).build /** * Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message. */ - def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = { + def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = ActorRefProtocol.newBuilder.setPath(actor.path.toStringWithAddress(address)).build - } - - def createRemoteMessageProtocolBuilder( - recipient: ActorRef, - message: Any, - senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = { + /** + * Returns a new RemoteMessageProtocol containing the serialized representation of the given parameters. + */ + def createRemoteMessageProtocolBuilder(recipient: ActorRef, message: Any, senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = { val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient)) if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get)) @@ -253,7 +257,12 @@ trait RemoteMarshallingOps { messageBuilder } - def receiveMessage(remoteMessage: RemoteMessage) { + /** + * Call this method with an inbound RemoteMessage and this will take care of security (see: "useUntrustedMode") + * as well as making sure that the message ends up at its destination (best effort). + * There is also a fair amount of logging produced by this method, which is good for debugging. + */ + def receiveMessage(remoteMessage: RemoteMessage): Unit = { val remoteDaemon = provider.remoteDaemon remoteMessage.recipient match { @@ -266,18 +275,14 @@ trait RemoteMarshallingOps { } case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender) } - case l: LocalRef ⇒ + case l @ (_: LocalRef | _: RepointableRef) if l.isLocal ⇒ if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage) remoteMessage.payload match { - case msg: SystemMessage ⇒ - if (useUntrustedMode) - throw new SecurityException("RemoteModule server is operating is untrusted mode, can not send system message") - else l.sendSystemMessage(msg) - case _: AutoReceivedMessage if (useUntrustedMode) ⇒ - throw new SecurityException("RemoteModule server is operating is untrusted mode, can not pass on a AutoReceivedMessage to the remote actor") - case m ⇒ l.!(m)(remoteMessage.sender) + case msg: PossiblyHarmful if useUntrustedMode ⇒ log.warning("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type {}", msg.getClass) + case msg: SystemMessage ⇒ l.sendSystemMessage(msg) + case msg ⇒ l.!(msg)(remoteMessage.sender) } - case r: RemoteRef ⇒ + case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal ⇒ if (provider.remoteSettings.LogReceive) log.debug("received remote-destined message {}", remoteMessage) remoteMessage.originalReceiver match { case AddressFromURIString(address) if address == provider.transport.address ⇒ @@ -285,7 +290,47 @@ trait RemoteMarshallingOps { r.!(remoteMessage.payload)(remoteMessage.sender) case r ⇒ log.error("dropping message {} for non-local recipient {} arriving at {} inbound address is {}", remoteMessage.payload, r, address, provider.transport.address) } - case r ⇒ log.error("dropping message {} for non-local recipient {} arriving at {} inbound address is {}", remoteMessage.payload, r, address, provider.transport.address) + case r ⇒ log.error("dropping message {} for unknown recipient {} arriving at {} inbound address is {}", remoteMessage.payload, r, address, provider.transport.address) } } } + +/** + * RemoteMessage is a wrapper around a message that has come in over the wire, + * it allows to easily obtain references to the deserialized message, its intended recipient + * and the sender. + */ +class RemoteMessage(input: RemoteMessageProtocol, system: ExtendedActorSystem) { + /** + * Returns a String-representation of the ActorPath that this RemoteMessage is destined for + */ + def originalReceiver: String = input.getRecipient.getPath + + /** + * Returns an Option with the String representation of the ActorPath of the Actor who is the sender of this message + */ + def originalSender: Option[String] = if (input.hasSender) Some(input.getSender.getPath) else None + + /** + * Returns a reference to the Actor that sent this message, or DeadLetterActorRef if not present or found. + */ + lazy val sender: ActorRef = + if (input.hasSender) system.provider.actorFor(system.provider.rootGuardian, input.getSender.getPath) + else system.deadLetters + + /** + * Returns a reference to the Actor that this message is destined for. + * In case this returns a DeadLetterActorRef, you have access to the path using the "originalReceiver" method. + */ + lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver) + + /** + * Returns the message + */ + lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage) + + /** + * Returns a String representation of this RemoteMessage, intended for debugging purposes. + */ + override def toString: String = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender +} diff --git a/akka-remote/src/main/scala/akka/remote/netty/Client.scala b/akka-remote/src/main/scala/akka/remote/netty/Client.scala index 7baf3011ee..35c0674d23 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Client.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Client.scala @@ -1,46 +1,35 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote.netty -import java.net.InetSocketAddress -import org.jboss.netty.util.HashedWheelTimer +import java.util.concurrent.TimeUnit +import java.net.{ InetAddress, InetSocketAddress } +import org.jboss.netty.util.{ Timeout, TimerTask, HashedWheelTimer } import org.jboss.netty.bootstrap.ClientBootstrap import org.jboss.netty.channel.group.DefaultChannelGroup -import org.jboss.netty.channel.{ ChannelHandler, StaticChannelPipeline, SimpleChannelUpstreamHandler, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelFuture, Channel } +import org.jboss.netty.channel.{ ChannelFutureListener, ChannelHandler, DefaultChannelPipeline, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelFuture, Channel } import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder } import org.jboss.netty.handler.execution.ExecutionHandler +import org.jboss.netty.handler.timeout.{ IdleState, IdleStateEvent, IdleStateAwareChannelHandler, IdleStateHandler } import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol } import akka.remote.{ RemoteProtocol, RemoteMessage, RemoteLifeCycleEvent, RemoteClientStarted, RemoteClientShutdown, RemoteClientException, RemoteClientError, RemoteClientDisconnected, RemoteClientConnected } -import akka.actor.{ simpleName, Address } import akka.AkkaException import akka.event.Logging -import akka.util.Switch -import akka.actor.ActorRef -import org.jboss.netty.channel.ChannelFutureListener -import akka.remote.RemoteClientWriteFailed -import java.net.InetAddress -import org.jboss.netty.util.TimerTask -import org.jboss.netty.util.Timeout -import java.util.concurrent.TimeUnit -import org.jboss.netty.handler.timeout.{ IdleState, IdleStateEvent, IdleStateAwareChannelHandler, IdleStateHandler } - -class RemoteClientMessageBufferException(message: String, cause: Throwable) extends AkkaException(message, cause) { - def this(msg: String) = this(msg, null) -} +import akka.actor.{ DeadLetter, Address, ActorRef } +import akka.util.{ NonFatal, Switch } +import org.jboss.netty.handler.ssl.SslHandler /** * This is the abstract baseclass for netty remote clients, currently there's only an * ActiveRemoteClient, but others could be feasible, like a PassiveRemoteClient that * reuses an already established connection. */ -abstract class RemoteClient private[akka] ( - val netty: NettyRemoteTransport, - val remoteAddress: Address) { +private[akka] abstract class RemoteClient private[akka] (val netty: NettyRemoteTransport, val remoteAddress: Address) { val log = Logging(netty.system, "RemoteClient") - val name = simpleName(this) + "@" + remoteAddress + val name = Logging.simpleName(this) + "@" + remoteAddress private[remote] val runSwitch = new Switch() @@ -75,11 +64,13 @@ abstract class RemoteClient private[akka] ( val f = channel.write(request) f.addListener( new ChannelFutureListener { - def operationComplete(future: ChannelFuture) { - if (future.isCancelled || !future.isSuccess) { - netty.notifyListeners(RemoteClientWriteFailed(request, future.getCause, netty, remoteAddress)) + import netty.system.deadLetters + def operationComplete(future: ChannelFuture): Unit = + if (future.isCancelled || !future.isSuccess) request match { + case (msg, sender, recipient) ⇒ deadLetters ! DeadLetter(msg, sender.getOrElse(deadLetters), recipient) + // We don't call notifyListeners here since we don't think failed message deliveries are errors + /// If the connection goes down we'll get the error reporting done by the pipeline. } - } }) // Check if we should back off if (!channel.isWritable) { @@ -87,17 +78,17 @@ abstract class RemoteClient private[akka] ( if (backoff.length > 0 && !f.await(backoff.length, backoff.unit)) f.cancel() //Waited as long as we could, now back off } } catch { - case e: Exception ⇒ netty.notifyListeners(RemoteClientError(e, netty, remoteAddress)) + case NonFatal(e) ⇒ netty.notifyListeners(RemoteClientError(e, netty, remoteAddress)) } } - override def toString = name + override def toString: String = name } /** * RemoteClient represents a connection to an Akka node. Is used to send messages to remote actors on the node. */ -class ActiveRemoteClient private[akka] ( +private[akka] class ActiveRemoteClient private[akka] ( netty: NettyRemoteTransport, remoteAddress: Address, localAddress: Address) @@ -112,8 +103,6 @@ class ActiveRemoteClient private[akka] ( private var connection: ChannelFuture = _ @volatile private[remote] var openChannels: DefaultChannelGroup = _ - @volatile - private var executionHandler: ExecutionHandler = _ @volatile private var reconnectionTimeWindowStart = 0L @@ -127,15 +116,27 @@ class ActiveRemoteClient private[akka] ( */ def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = { - def sendSecureCookie(connection: ChannelFuture) { - val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT) - if (settings.SecureCookie.nonEmpty) handshake.setCookie(settings.SecureCookie.get) - handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder - .setSystem(localAddress.system) - .setHostname(localAddress.host.get) - .setPort(localAddress.port.get) - .build) - connection.getChannel.write(netty.createControlEnvelope(handshake.build)) + // Returns whether the handshake was written to the channel or not + def sendSecureCookie(connection: ChannelFuture): Boolean = { + val future = + if (!connection.isSuccess || !settings.EnableSSL) connection + else connection.getChannel.getPipeline.get[SslHandler](classOf[SslHandler]).handshake().awaitUninterruptibly() + + if (!future.isSuccess) { + notifyListeners(RemoteClientError(future.getCause, netty, remoteAddress)) + false + } else { + ChannelAddress.set(connection.getChannel, Some(remoteAddress)) + val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT) + if (settings.SecureCookie.nonEmpty) handshake.setCookie(settings.SecureCookie.get) + handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder + .setSystem(localAddress.system) + .setHostname(localAddress.host.get) + .setPort(localAddress.port.get) + .build) + connection.getChannel.write(netty.createControlEnvelope(handshake.build)) + true + } } def attemptReconnect(): Boolean = { @@ -143,25 +144,21 @@ class ActiveRemoteClient private[akka] ( log.debug("Remote client reconnecting to [{}|{}]", remoteAddress, remoteIP) connection = bootstrap.connect(new InetSocketAddress(remoteIP, remoteAddress.port.get)) openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails. - - if (!connection.isSuccess) { - notifyListeners(RemoteClientError(connection.getCause, netty, remoteAddress)) - false - } else { - sendSecureCookie(connection) - true - } + sendSecureCookie(connection) } runSwitch switchOn { openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName) - executionHandler = new ExecutionHandler(netty.executor) val b = new ClientBootstrap(netty.clientChannelFactory) - b.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, b, executionHandler, remoteAddress, localAddress, this)) + b.setPipelineFactory(netty.createPipeline(new ActiveRemoteClientHandler(name, b, remoteAddress, localAddress, netty.timer, this), withTimeout = true, isClient = true)) b.setOption("tcpNoDelay", true) b.setOption("keepAlive", true) b.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis) + settings.ReceiveBufferSize.foreach(sz ⇒ b.setOption("receiveBufferSize", sz)) + settings.SendBufferSize.foreach(sz ⇒ b.setOption("sendBufferSize", sz)) + settings.WriteBufferHighWaterMark.foreach(sz ⇒ b.setOption("writeBufferHighWaterMark", sz)) + settings.WriteBufferLowWaterMark.foreach(sz ⇒ b.setOption("writeBufferLowWaterMark", sz)) settings.OutboundLocalAddress.foreach(s ⇒ b.setOption("localAddress", new InetSocketAddress(s, 0))) bootstrap = b @@ -172,23 +169,19 @@ class ActiveRemoteClient private[akka] ( openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails. - if (!connection.isSuccess) { - notifyListeners(RemoteClientError(connection.getCause, netty, remoteAddress)) - false - } else { - sendSecureCookie(connection) + if (sendSecureCookie(connection)) { notifyListeners(RemoteClientStarted(netty, remoteAddress)) true + } else { + connection.getChannel.close() + openChannels.remove(connection.getChannel) + false } } match { case true ⇒ true case false if reconnectIfAlreadyConnected ⇒ - connection.getChannel.close() - openChannels.remove(connection.getChannel) - log.debug("Remote client reconnecting to [{}]", remoteAddress) attemptReconnect() - case false ⇒ false } } @@ -199,14 +192,15 @@ class ActiveRemoteClient private[akka] ( notifyListeners(RemoteClientShutdown(netty, remoteAddress)) try { - if ((connection ne null) && (connection.getChannel ne null)) + if ((connection ne null) && (connection.getChannel ne null)) { + ChannelAddress.remove(connection.getChannel) connection.getChannel.close() + } } finally { try { if (openChannels ne null) openChannels.close.awaitUninterruptibly() } finally { connection = null - executionHandler = null } } @@ -230,7 +224,7 @@ class ActiveRemoteClient private[akka] ( } @ChannelHandler.Sharable -class ActiveRemoteClientHandler( +private[akka] class ActiveRemoteClientHandler( val name: String, val bootstrap: ClientBootstrap, val remoteAddress: Address, @@ -319,35 +313,9 @@ class ActiveRemoteClientHandler( } } -class ActiveRemoteClientPipelineFactory( - name: String, - bootstrap: ClientBootstrap, - executionHandler: ExecutionHandler, - remoteAddress: Address, - localAddress: Address, - client: ActiveRemoteClient) extends ChannelPipelineFactory { - - import client.netty.settings - - def getPipeline: ChannelPipeline = { - val timeout = new IdleStateHandler(client.netty.timer, - settings.ReadTimeout.toSeconds.toInt, - settings.WriteTimeout.toSeconds.toInt, - settings.AllTimeout.toSeconds.toInt) - val lenDec = new LengthFieldBasedFrameDecoder(settings.MessageFrameSize, 0, 4, 0, 4) - val lenPrep = new LengthFieldPrepender(4) - val messageDec = new RemoteMessageDecoder - val messageEnc = new RemoteMessageEncoder(client.netty) - val remoteClient = new ActiveRemoteClientHandler(name, bootstrap, remoteAddress, localAddress, client.netty.timer, client) - - new StaticChannelPipeline(timeout, lenDec, messageDec, lenPrep, messageEnc, executionHandler, remoteClient) - } -} - -class PassiveRemoteClient(val currentChannel: Channel, - netty: NettyRemoteTransport, - remoteAddress: Address) - extends RemoteClient(netty, remoteAddress) { +private[akka] class PassiveRemoteClient(val currentChannel: Channel, + netty: NettyRemoteTransport, + remoteAddress: Address) extends RemoteClient(netty, remoteAddress) { def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = runSwitch switchOn { netty.notifyListeners(RemoteClientStarted(netty, remoteAddress)) diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index c45fa986f2..5c506abfc8 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote.netty @@ -12,48 +12,142 @@ import java.util.concurrent.Executors import scala.collection.mutable.HashMap import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroupFuture } import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory -import org.jboss.netty.channel.{ ChannelHandlerContext, Channel } +import org.jboss.netty.channel.{ ChannelHandlerContext, Channel, DefaultChannelPipeline, ChannelHandler, ChannelPipelineFactory, ChannelLocal } +import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder } import org.jboss.netty.handler.codec.protobuf.{ ProtobufEncoder, ProtobufDecoder } -import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor +import org.jboss.netty.handler.execution.{ ExecutionHandler, OrderedMemoryAwareThreadPoolExecutor } +import org.jboss.netty.handler.timeout.IdleStateHandler import org.jboss.netty.util.HashedWheelTimer -import akka.dispatch.MonitorableThreadFactory import akka.event.Logging import akka.remote.RemoteProtocol.AkkaRemoteProtocol -import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteSettings, RemoteMarshallingOps, RemoteActorRefProvider, RemoteActorRef, RemoteServerStarted } +import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteActorRefProvider, RemoteActorRef, RemoteServerStarted } import akka.util.NonFatal import akka.actor.{ ExtendedActorSystem, Address, ActorRef } +private[akka] object ChannelAddress extends ChannelLocal[Option[Address]] { + override def initialValue(ch: Channel): Option[Address] = None +} + /** * Provides the implementation of the Netty remote support */ -class NettyRemoteTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) with RemoteMarshallingOps { +private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) { import provider.remoteSettings val settings = new NettySettings(remoteSettings.config.getConfig("akka.remote.netty"), remoteSettings.systemName) + // TODO replace by system.scheduler val timer: HashedWheelTimer = new HashedWheelTimer(system.threadFactory) - val executor = new OrderedMemoryAwareThreadPoolExecutor( - settings.ExecutionPoolSize, - settings.MaxChannelMemorySize, - settings.MaxTotalMemorySize, - settings.ExecutionPoolKeepalive.length, - settings.ExecutionPoolKeepalive.unit, - system.threadFactory) + val clientChannelFactory = settings.UseDispatcherForIO match { + case Some(id) ⇒ + val d = system.dispatchers.lookup(id) + new NioClientSocketChannelFactory(d, d) + case None ⇒ + new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool()) + } - val clientChannelFactory = new NioClientSocketChannelFactory( - Executors.newCachedThreadPool(system.threadFactory), - Executors.newCachedThreadPool(system.threadFactory)) + /** + * Backing scaffolding for the default implementation of NettyRemoteSupport.createPipeline. + */ + object PipelineFactory { + /** + * Construct a DefaultChannelPipeline from a sequence of handlers; to be used + * in implementations of ChannelPipelineFactory. + */ + def apply(handlers: Seq[ChannelHandler]): DefaultChannelPipeline = + (new DefaultChannelPipeline /: handlers) { (p, h) ⇒ p.addLast(Logging.simpleName(h.getClass), h); p } + + /** + * Constructs the NettyRemoteTransport default pipeline with the give “head” handler, which + * is taken by-name to allow it not to be shared across pipelines. + * + * @param withTimeout determines whether an IdleStateHandler shall be included + */ + def apply(endpoint: ⇒ Seq[ChannelHandler], withTimeout: Boolean, isClient: Boolean): ChannelPipelineFactory = + new ChannelPipelineFactory { override def getPipeline = apply(defaultStack(withTimeout, isClient) ++ endpoint) } + + /** + * Construct a default protocol stack, excluding the “head” handler (i.e. the one which + * actually dispatches the received messages to the local target actors). + */ + def defaultStack(withTimeout: Boolean, isClient: Boolean): Seq[ChannelHandler] = + (if (settings.EnableSSL) List(NettySSLSupport(settings, NettyRemoteTransport.this.log, isClient)) else Nil) ::: + (if (withTimeout) List(timeout) else Nil) ::: + msgFormat ::: + authenticator ::: + executionHandler + + /** + * Construct an IdleStateHandler which uses [[akka.remote.netty.NettyRemoteTransport]].timer. + */ + def timeout = new IdleStateHandler(timer, + settings.ReadTimeout.toSeconds.toInt, + settings.WriteTimeout.toSeconds.toInt, + settings.AllTimeout.toSeconds.toInt) + + /** + * Construct frame&protobuf encoder/decoder. + */ + def msgFormat = new LengthFieldBasedFrameDecoder(settings.MessageFrameSize, 0, 4, 0, 4) :: + new LengthFieldPrepender(4) :: + new RemoteMessageDecoder :: + new RemoteMessageEncoder(NettyRemoteTransport.this) :: + Nil + + /** + * Construct an ExecutionHandler which is used to ensure that message dispatch does not + * happen on a netty thread (that could be bad if re-sending over the network for + * remote-deployed actors). + */ + val executionHandler = if (settings.ExecutionPoolSize != 0) + List(new ExecutionHandler(new OrderedMemoryAwareThreadPoolExecutor( + settings.ExecutionPoolSize, + settings.MaxChannelMemorySize, + settings.MaxTotalMemorySize, + settings.ExecutionPoolKeepalive.length, + settings.ExecutionPoolKeepalive.unit, + system.threadFactory))) + else Nil + + /** + * Construct and authentication handler which uses the SecureCookie to somewhat + * protect the TCP port from unauthorized use (don’t rely on it too much, though, + * as this is NOT a cryptographic feature). + */ + def authenticator = if (settings.RequireCookie) List(new RemoteServerAuthenticationHandler(settings.SecureCookie)) else Nil + } + + /** + * This method is factored out to provide an extension point in case the + * pipeline shall be changed. It is recommended to use + */ + def createPipeline(endpoint: ⇒ ChannelHandler, withTimeout: Boolean, isClient: Boolean): ChannelPipelineFactory = + PipelineFactory(Seq(endpoint), withTimeout, isClient) private val remoteClients = new HashMap[Address, RemoteClient] private val clientsLock = new ReentrantReadWriteLock override protected def useUntrustedMode = remoteSettings.UntrustedMode - val server = try new NettyRemoteServer(this) catch { - case ex ⇒ shutdown(); throw ex - } + override protected def logRemoteLifeCycleEvents = remoteSettings.LogRemoteLifeCycleEvents + + val server: NettyRemoteServer = try createServer() catch { case NonFatal(ex) ⇒ shutdown(); throw ex } + + /** + * Override this method to inject a subclass of NettyRemoteServer instead of + * the normal one, e.g. for inserting security hooks. If this method throws + * an exception, the transport will shut itself down and re-throw. + */ + protected def createServer(): NettyRemoteServer = new NettyRemoteServer(this) + + /** + * Override this method to inject a subclass of RemoteClient instead of + * the normal one, e.g. for inserting security hooks. Get this transport’s + * address from `this.address`. + */ + protected def createClient(recipient: Address): RemoteClient = new ActiveRemoteClient(this, recipient, address) // the address is set in start() or from the RemoteServerHandler, whichever comes first private val _address = new AtomicReference[Address] @@ -92,11 +186,7 @@ class NettyRemoteTransport(_system: ExtendedActorSystem, _provider: RemoteActorR try { timer.stop() } finally { - try { - clientChannelFactory.releaseExternalResources() - } finally { - executor.shutdown() - } + clientChannelFactory.releaseExternalResources() } } } @@ -122,7 +212,7 @@ class NettyRemoteTransport(_system: ExtendedActorSystem, _provider: RemoteActorR //Recheck for addition, race between upgrades case Some(client) ⇒ client //If already populated by other writer case None ⇒ //Populate map - val client = new ActiveRemoteClient(this, recipientAddress, address) + val client = createClient(recipientAddress) remoteClients += recipientAddress -> client client } @@ -192,7 +282,7 @@ class NettyRemoteTransport(_system: ExtendedActorSystem, _provider: RemoteActorR } -class RemoteMessageEncoder(remoteSupport: NettyRemoteTransport) extends ProtobufEncoder { +private[akka] class RemoteMessageEncoder(remoteSupport: NettyRemoteTransport) extends ProtobufEncoder { override def encode(ctx: ChannelHandlerContext, channel: Channel, msg: AnyRef): AnyRef = { msg match { case (message: Any, sender: Option[_], recipient: ActorRef) ⇒ @@ -207,9 +297,9 @@ class RemoteMessageEncoder(remoteSupport: NettyRemoteTransport) extends Protobuf } } -class RemoteMessageDecoder extends ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance) +private[akka] class RemoteMessageDecoder extends ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance) -class DefaultDisposableChannelGroup(name: String) extends DefaultChannelGroup(name) { +private[akka] class DefaultDisposableChannelGroup(name: String) extends DefaultChannelGroup(name) { protected val guard = new ReentrantReadWriteLock protected val open = new AtomicBoolean(true) diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettySSLSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettySSLSupport.scala new file mode 100644 index 0000000000..690b4522ec --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/netty/NettySSLSupport.scala @@ -0,0 +1,148 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.remote.netty + +import org.jboss.netty.handler.ssl.SslHandler +import javax.net.ssl.{ KeyManagerFactory, TrustManager, TrustManagerFactory, SSLContext } +import akka.remote.RemoteTransportException +import akka.event.LoggingAdapter +import java.io.{ IOException, FileNotFoundException, FileInputStream } +import akka.security.provider.AkkaProvider +import java.security._ + +/** + * Used for adding SSL support to Netty pipeline + * Internal use only + */ +private[akka] object NettySSLSupport { + + Security addProvider AkkaProvider + + /** + * Construct a SSLHandler which can be inserted into a Netty server/client pipeline + */ + def apply(settings: NettySettings, log: LoggingAdapter, isClient: Boolean): SslHandler = + if (isClient) initializeClientSSL(settings, log) else initializeServerSSL(settings, log) + + def initializeCustomSecureRandom(rngName: Option[String], sourceOfRandomness: Option[String], log: LoggingAdapter): SecureRandom = { + /** + * According to this bug report: http://bugs.sun.com/view_bug.do?bug_id=6202721 + * Using /dev/./urandom is only necessary when using SHA1PRNG on Linux + * Use 'new SecureRandom()' instead of 'SecureRandom.getInstance("SHA1PRNG")' to avoid having problems + */ + sourceOfRandomness foreach { path ⇒ + System.setProperty("java.security.egd", path) + System.setProperty("securerandom.source", path) + } + + val rng = rngName match { + case Some(r @ ("AES128CounterSecureRNG" | "AES128CounterInetRNG" | "AES256CounterInetRNG")) ⇒ + log.debug("SSL random number generator set to: {}", r) + SecureRandom.getInstance(r, AkkaProvider) + case Some(s @ ("SHA1PRNG" | "NativePRNG")) ⇒ + log.debug("SSL random number generator set to: " + s) + // SHA1PRNG needs /dev/urandom to be the source on Linux to prevent problems with /dev/random blocking + // However, this also makes the seed source insecure as the seed is reused to avoid blocking (not a problem on FreeBSD). + SecureRandom.getInstance(s) + case Some(unknown) ⇒ + log.debug("Unknown SSLRandomNumberGenerator [{}] falling back to SecureRandom", unknown) + new SecureRandom + case None ⇒ + log.debug("SSLRandomNumberGenerator not specified, falling back to SecureRandom") + new SecureRandom + } + rng.nextInt() // prevent stall on first access + rng + } + + def initializeClientSSL(settings: NettySettings, log: LoggingAdapter): SslHandler = { + log.debug("Client SSL is enabled, initialising ...") + + def constructClientContext(settings: NettySettings, log: LoggingAdapter, trustStorePath: String, trustStorePassword: String, protocol: String): Option[SSLContext] = + try { + val rng = initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, settings.SSLRandomSource, log) + val trustManagers: Array[TrustManager] = { + val trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm) + trustManagerFactory.init({ + val trustStore = KeyStore.getInstance(KeyStore.getDefaultType) + val fin = new FileInputStream(trustStorePath) + try trustStore.load(fin, trustStorePassword.toCharArray) finally fin.close() + trustStore + }) + trustManagerFactory.getTrustManagers + } + Option(SSLContext.getInstance(protocol)) map { ctx ⇒ ctx.init(null, trustManagers, rng); ctx } + } catch { + case e: FileNotFoundException ⇒ throw new RemoteTransportException("Client SSL connection could not be established because trust store could not be loaded", e) + case e: IOException ⇒ throw new RemoteTransportException("Client SSL connection could not be established because: " + e.getMessage, e) + case e: GeneralSecurityException ⇒ throw new RemoteTransportException("Client SSL connection could not be established because SSL context could not be constructed", e) + } + + ((settings.SSLTrustStore, settings.SSLTrustStorePassword, settings.SSLProtocol) match { + case (Some(trustStore), Some(password), Some(protocol)) ⇒ constructClientContext(settings, log, trustStore, password, protocol) + case (trustStore, password, protocol) ⇒ throw new GeneralSecurityException( + "One or several SSL trust store settings are missing: [trust-store: %s] [trust-store-password: %s] [protocol: %s]".format( + trustStore, + password, + protocol)) + }) match { + case Some(context) ⇒ + log.debug("Using client SSL context to create SSLEngine ...") + new SslHandler({ + val sslEngine = context.createSSLEngine + sslEngine.setUseClientMode(true) + sslEngine.setEnabledCipherSuites(settings.SSLEnabledAlgorithms.toArray) + sslEngine + }) + case None ⇒ + throw new GeneralSecurityException( + """Failed to initialize client SSL because SSL context could not be found." + + "Make sure your settings are correct: [trust-store: %s] [trust-store-password: %s] [protocol: %s]""".format( + settings.SSLTrustStore, + settings.SSLTrustStorePassword, + settings.SSLProtocol)) + } + } + + def initializeServerSSL(settings: NettySettings, log: LoggingAdapter): SslHandler = { + log.debug("Server SSL is enabled, initialising ...") + + def constructServerContext(settings: NettySettings, log: LoggingAdapter, keyStorePath: String, keyStorePassword: String, protocol: String): Option[SSLContext] = + try { + val rng = initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, settings.SSLRandomSource, log) + val factory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm) + factory.init({ + val keyStore = KeyStore.getInstance(KeyStore.getDefaultType) + val fin = new FileInputStream(keyStorePath) + try keyStore.load(fin, keyStorePassword.toCharArray) finally fin.close() + keyStore + }, keyStorePassword.toCharArray) + Option(SSLContext.getInstance(protocol)) map { ctx ⇒ ctx.init(factory.getKeyManagers, null, rng); ctx } + } catch { + case e: FileNotFoundException ⇒ throw new RemoteTransportException("Server SSL connection could not be established because key store could not be loaded", e) + case e: IOException ⇒ throw new RemoteTransportException("Server SSL connection could not be established because: " + e.getMessage, e) + case e: GeneralSecurityException ⇒ throw new RemoteTransportException("Server SSL connection could not be established because SSL context could not be constructed", e) + } + + ((settings.SSLKeyStore, settings.SSLKeyStorePassword, settings.SSLProtocol) match { + case (Some(keyStore), Some(password), Some(protocol)) ⇒ constructServerContext(settings, log, keyStore, password, protocol) + case (keyStore, password, protocol) ⇒ throw new GeneralSecurityException( + "SSL key store settings went missing. [key-store: %s] [key-store-password: %s] [protocol: %s]".format(keyStore, password, protocol)) + }) match { + case Some(context) ⇒ + log.debug("Using server SSL context to create SSLEngine ...") + val sslEngine = context.createSSLEngine + sslEngine.setUseClientMode(false) + sslEngine.setEnabledCipherSuites(settings.SSLEnabledAlgorithms.toArray) + new SslHandler(sslEngine) + case None ⇒ throw new GeneralSecurityException( + """Failed to initialize server SSL because SSL context could not be found. + Make sure your settings are correct: [key-store: %s] [key-store-password: %s] [protocol: %s]""".format( + settings.SSLKeyStore, + settings.SSLKeyStorePassword, + settings.SSLProtocol)) + } + } +} diff --git a/akka-remote/src/main/scala/akka/remote/netty/Server.scala b/akka-remote/src/main/scala/akka/remote/netty/Server.scala index 7e4d1eaaa9..895fea9212 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote.netty @@ -12,7 +12,6 @@ import org.jboss.netty.channel.group.ChannelGroup import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder } import org.jboss.netty.handler.execution.ExecutionHandler -import akka.event.Logging import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol } import akka.remote.{ RemoteServerShutdown, RemoteServerError, RemoteServerClientDisconnected, RemoteServerClientConnected, RemoteServerClientClosed, RemoteProtocol, RemoteMessage } import akka.actor.Address @@ -20,7 +19,7 @@ import java.net.InetAddress import akka.actor.ActorSystemImpl import org.jboss.netty.channel._ -class NettyRemoteServer(val netty: NettyRemoteTransport) { +private[akka] class NettyRemoteServer(val netty: NettyRemoteTransport) { import netty.settings @@ -35,18 +34,20 @@ class NettyRemoteServer(val netty: NettyRemoteTransport) { new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool()) } - private val executionHandler = new ExecutionHandler(netty.executor) - // group of open channels, used for clean-up private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server") private val bootstrap = { val b = new ServerBootstrap(factory) - b.setPipelineFactory(new RemoteServerPipelineFactory(openChannels, executionHandler, netty)) + b.setPipelineFactory(netty.createPipeline(new RemoteServerHandler(openChannels, netty), withTimeout = false, isClient = false)) b.setOption("backlog", settings.Backlog) b.setOption("tcpNoDelay", true) b.setOption("child.keepAlive", true) b.setOption("reuseAddress", true) + settings.ReceiveBufferSize.foreach(sz ⇒ b.setOption("receiveBufferSize", sz)) + settings.SendBufferSize.foreach(sz ⇒ b.setOption("sendBufferSize", sz)) + settings.WriteBufferHighWaterMark.foreach(sz ⇒ b.setOption("writeBufferHighWaterMark", sz)) + settings.WriteBufferLowWaterMark.foreach(sz ⇒ b.setOption("writeBufferLowWaterMark", sz)) b } @@ -82,28 +83,8 @@ class NettyRemoteServer(val netty: NettyRemoteTransport) { } } -class RemoteServerPipelineFactory( - val openChannels: ChannelGroup, - val executionHandler: ExecutionHandler, - val netty: NettyRemoteTransport) extends ChannelPipelineFactory { - - import netty.settings - - def getPipeline: ChannelPipeline = { - val lenDec = new LengthFieldBasedFrameDecoder(settings.MessageFrameSize, 0, 4, 0, 4) - val lenPrep = new LengthFieldPrepender(4) - val messageDec = new RemoteMessageDecoder - val messageEnc = new RemoteMessageEncoder(netty) - - val authenticator = if (settings.RequireCookie) new RemoteServerAuthenticationHandler(settings.SecureCookie) :: Nil else Nil - val remoteServer = new RemoteServerHandler(openChannels, netty) - val stages: List[ChannelHandler] = lenDec :: messageDec :: lenPrep :: messageEnc :: executionHandler :: authenticator ::: remoteServer :: Nil - new StaticChannelPipeline(stages: _*) - } -} - @ChannelHandler.Sharable -class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends SimpleChannelUpstreamHandler { +private[akka] class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends SimpleChannelUpstreamHandler { val authenticated = new AnyRef override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = secureCookie match { @@ -130,14 +111,10 @@ class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends Si } @ChannelHandler.Sharable -class RemoteServerHandler( +private[akka] class RemoteServerHandler( val openChannels: ChannelGroup, val netty: NettyRemoteTransport) extends SimpleChannelUpstreamHandler { - val channelAddress = new ChannelLocal[Option[Address]](false) { - override def initialValue(channel: Channel) = None - } - import netty.settings private var addressToSet = true @@ -161,16 +138,16 @@ class RemoteServerHandler( override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = () override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { - netty.notifyListeners(RemoteServerClientDisconnected(netty, channelAddress.get(ctx.getChannel))) + netty.notifyListeners(RemoteServerClientDisconnected(netty, ChannelAddress.get(ctx.getChannel))) } override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { - val address = channelAddress.get(ctx.getChannel) + val address = ChannelAddress.get(ctx.getChannel) if (address.isDefined && settings.UsePassiveConnections) netty.unbindClient(address.get) netty.notifyListeners(RemoteServerClientClosed(netty, address)) - channelAddress.remove(ctx.getChannel) + ChannelAddress.remove(ctx.getChannel) } override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try { @@ -184,7 +161,7 @@ class RemoteServerHandler( case CommandType.CONNECT ⇒ val origin = instruction.getOrigin val inbound = Address("akka", origin.getSystem, origin.getHostname, origin.getPort) - channelAddress.set(event.getChannel, Option(inbound)) + ChannelAddress.set(event.getChannel, Option(inbound)) //If we want to reuse the inbound connections as outbound we need to get busy if (settings.UsePassiveConnections) diff --git a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala index e2f69d77b5..9babf6005c 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote.netty @@ -7,39 +7,53 @@ import com.typesafe.config.Config import akka.util.Duration import java.util.concurrent.TimeUnit._ import java.net.InetAddress -import akka.config.ConfigurationException +import akka.ConfigurationException +import scala.collection.JavaConverters.iterableAsScalaIterableConverter -class NettySettings(config: Config, val systemName: String) { +private[akka] class NettySettings(config: Config, val systemName: String) { import config._ - val BackoffTimeout = Duration(getMilliseconds("backoff-timeout"), MILLISECONDS) + val BackoffTimeout: Duration = Duration(getMilliseconds("backoff-timeout"), MILLISECONDS) val SecureCookie: Option[String] = getString("secure-cookie") match { case "" ⇒ None case cookie ⇒ Some(cookie) } - val RequireCookie = { + val RequireCookie: Boolean = { val requireCookie = getBoolean("require-cookie") if (requireCookie && SecureCookie.isEmpty) throw new ConfigurationException( "Configuration option 'akka.remote.netty.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.netty.secure-cookie'.") requireCookie } - val UsePassiveConnections = getBoolean("use-passive-connections") - val UseDispatcherForIO = getString("use-dispatcher-for-io") match { + val UsePassiveConnections: Boolean = getBoolean("use-passive-connections") + val UseDispatcherForIO: Option[String] = getString("use-dispatcher-for-io") match { case "" | null ⇒ None case dispatcher ⇒ Some(dispatcher) } - val ReconnectionTimeWindow = Duration(getMilliseconds("reconnection-time-window"), MILLISECONDS) - val ReadTimeout = Duration(getMilliseconds("read-timeout"), MILLISECONDS) - val WriteTimeout = Duration(getMilliseconds("write-timeout"), MILLISECONDS) - val AllTimeout = Duration(getMilliseconds("all-timeout"), MILLISECONDS) - val ReconnectDelay = Duration(getMilliseconds("reconnect-delay"), MILLISECONDS) - val MessageFrameSize = getBytes("message-frame-size").toInt + val ReconnectionTimeWindow: Duration = Duration(getMilliseconds("reconnection-time-window"), MILLISECONDS) + val ReadTimeout: Duration = Duration(getMilliseconds("read-timeout"), MILLISECONDS) + val WriteTimeout: Duration = Duration(getMilliseconds("write-timeout"), MILLISECONDS) + val AllTimeout: Duration = Duration(getMilliseconds("all-timeout"), MILLISECONDS) + val ReconnectDelay: Duration = Duration(getMilliseconds("reconnect-delay"), MILLISECONDS) - val Hostname = getString("hostname") match { + val MessageFrameSize: Int = getBytes("message-frame-size").toInt + + private[this] def optionSize(s: String): Option[Int] = getBytes(s).toInt match { + case 0 ⇒ None + case x if x < 0 ⇒ + throw new ConfigurationException("Setting '%s' must be 0 or positive (and fit in an Int)" format s) + case other ⇒ Some(other) + } + + val WriteBufferHighWaterMark: Option[Int] = optionSize("write-buffer-high-water-mark") + val WriteBufferLowWaterMark: Option[Int] = optionSize("write-buffer-low-water-mark") + val SendBufferSize: Option[Int] = optionSize("send-buffer-size") + val ReceiveBufferSize: Option[Int] = optionSize("receive-buffer-size") + + val Hostname: String = getString("hostname") match { case "" ⇒ InetAddress.getLocalHost.getHostAddress case value ⇒ value } @@ -50,27 +64,78 @@ class NettySettings(config: Config, val systemName: String) { } @deprecated("WARNING: This should only be used by professionals.", "2.0") - val PortSelector = getInt("port") + val PortSelector: Int = getInt("port") - val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS) + val ConnectionTimeout: Duration = Duration(getMilliseconds("connection-timeout"), MILLISECONDS) - val Backlog = getInt("backlog") + val Backlog: Int = getInt("backlog") - val ExecutionPoolKeepalive = Duration(getMilliseconds("execution-pool-keepalive"), MILLISECONDS) + val ExecutionPoolKeepalive: Duration = Duration(getMilliseconds("execution-pool-keepalive"), MILLISECONDS) - val ExecutionPoolSize = getInt("execution-pool-size") match { - case sz if sz < 1 ⇒ throw new IllegalArgumentException("akka.remote.netty.execution-pool-size is less than 1") + val ExecutionPoolSize: Int = getInt("execution-pool-size") match { + case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.netty.execution-pool-size is less than 0") case sz ⇒ sz } - val MaxChannelMemorySize = getBytes("max-channel-memory-size") match { + val MaxChannelMemorySize: Long = getBytes("max-channel-memory-size") match { case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.netty.max-channel-memory-size is less than 0 bytes") case sz ⇒ sz } - val MaxTotalMemorySize = getBytes("max-total-memory-size") match { + val MaxTotalMemorySize: Long = getBytes("max-total-memory-size") match { case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.netty.max-total-memory-size is less than 0 bytes") case sz ⇒ sz } -} \ No newline at end of file + val SSLKeyStore = getString("ssl.key-store") match { + case "" ⇒ None + case keyStore ⇒ Some(keyStore) + } + + val SSLTrustStore = getString("ssl.trust-store") match { + case "" ⇒ None + case trustStore ⇒ Some(trustStore) + } + + val SSLKeyStorePassword = getString("ssl.key-store-password") match { + case "" ⇒ None + case password ⇒ Some(password) + } + + val SSLTrustStorePassword = getString("ssl.trust-store-password") match { + case "" ⇒ None + case password ⇒ Some(password) + } + + val SSLEnabledAlgorithms = iterableAsScalaIterableConverter(getStringList("ssl.enabled-algorithms")).asScala.toSet[String] + + val SSLProtocol = getString("ssl.protocol") match { + case "" ⇒ None + case protocol ⇒ Some(protocol) + } + + val SSLRandomSource = getString("ssl.sha1prng-random-source") match { + case "" ⇒ None + case path ⇒ Some(path) + } + + val SSLRandomNumberGenerator = getString("ssl.random-number-generator") match { + case "" ⇒ None + case rng ⇒ Some(rng) + } + + val EnableSSL = { + val enableSSL = getBoolean("ssl.enable") + if (enableSSL) { + if (SSLProtocol.isEmpty) throw new ConfigurationException( + "Configuration option 'akka.remote.netty.ssl.enable is turned on but no protocol is defined in 'akka.remote.netty.ssl.protocol'.") + if (SSLKeyStore.isEmpty && SSLTrustStore.isEmpty) throw new ConfigurationException( + "Configuration option 'akka.remote.netty.ssl.enable is turned on but no key/trust store is defined in 'akka.remote.netty.ssl.key-store' / 'akka.remote.netty.ssl.trust-store'.") + if (SSLKeyStore.isDefined && SSLKeyStorePassword.isEmpty) throw new ConfigurationException( + "Configuration option 'akka.remote.netty.ssl.key-store' is defined but no key-store password is defined in 'akka.remote.netty.ssl.key-store-password'.") + if (SSLTrustStore.isDefined && SSLTrustStorePassword.isEmpty) throw new ConfigurationException( + "Configuration option 'akka.remote.netty.ssl.trust-store' is defined but no trust-store password is defined in 'akka.remote.netty.ssl.trust-store-password'.") + } + enableSSL + } +} diff --git a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala index c9bb6dba0f..a0b7ae4a49 100644 --- a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala +++ b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala @@ -10,7 +10,7 @@ import akka.actor.ActorSystemImpl import akka.actor.Deploy import akka.actor.InternalActorRef import akka.actor.Props -import akka.config.ConfigurationException +import akka.ConfigurationException import akka.remote.RemoteScope import akka.actor.AddressFromURIString import akka.actor.SupervisorStrategy @@ -59,7 +59,7 @@ class RemoteRouteeProvider(nodes: Iterable[Address], _context: ActorContext, _re extends RouteeProvider(_context, _resizer) { // need this iterator as instance variable since Resizer may call createRoutees several times - private val nodeAddressIter = Stream.continually(nodes).flatten.iterator + private val nodeAddressIter: Iterator[Address] = Stream.continually(nodes).flatten.iterator override def createRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): IndexedSeq[ActorRef] = (nrOfInstances, routees, nodes) match { @@ -71,7 +71,8 @@ class RemoteRouteeProvider(nodes: Iterable[Address], _context: ActorContext, _re IndexedSeq.empty[ActorRef] ++ (for (i ← 1 to nrOfInstances) yield { val name = "c" + i val deploy = Deploy("", ConfigFactory.empty(), props.routerConfig, RemoteScope(nodeAddressIter.next)) - impl.provider.actorOf(impl, props, context.self.asInstanceOf[InternalActorRef], context.self.path / name, false, Some(deploy), false) + impl.provider.actorOf(impl, props, context.self.asInstanceOf[InternalActorRef], context.self.path / name, + systemService = false, Some(deploy), lookupDeploy = false, async = false) }) case (_, xs, _) ⇒ throw new ConfigurationException("Remote target.nodes can not be combined with routees for [%s]" diff --git a/akka-remote/src/main/scala/akka/security/provider/AES128CounterInetRNG.scala b/akka-remote/src/main/scala/akka/security/provider/AES128CounterInetRNG.scala new file mode 100644 index 0000000000..41d12b275f --- /dev/null +++ b/akka-remote/src/main/scala/akka/security/provider/AES128CounterInetRNG.scala @@ -0,0 +1,40 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.security.provider + +import org.uncommons.maths.random.{ AESCounterRNG, DefaultSeedGenerator } + +/** + * Internal API + * This class is a wrapper around the 128-bit AESCounterRNG algorithm provided by http://maths.uncommons.org/ + * It uses the default seed generator which uses one of the following 3 random seed sources: + * Depending on availability: /dev/random, random.org and SecureRandom (provided by Java) + * The only method used by netty ssl is engineNextBytes(bytes) + */ +class AES128CounterInetRNG extends java.security.SecureRandomSpi { + private val rng = new AESCounterRNG() + + /** + * This is managed internally by AESCounterRNG + */ + override protected def engineSetSeed(seed: Array[Byte]): Unit = () + + /** + * Generates a user-specified number of random bytes. + * + * @param bytes the array to be filled in with random bytes. + */ + override protected def engineNextBytes(bytes: Array[Byte]): Unit = rng.nextBytes(bytes) + + /** + * Unused method + * Returns the given number of seed bytes. This call may be used to + * seed other random number generators. + * + * @param numBytes the number of seed bytes to generate. + * @return the seed bytes. + */ + override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = DefaultSeedGenerator.getInstance.generateSeed(numBytes) +} + diff --git a/akka-remote/src/main/scala/akka/security/provider/AES128CounterSecureRNG.scala b/akka-remote/src/main/scala/akka/security/provider/AES128CounterSecureRNG.scala new file mode 100644 index 0000000000..cda59ee03b --- /dev/null +++ b/akka-remote/src/main/scala/akka/security/provider/AES128CounterSecureRNG.scala @@ -0,0 +1,39 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.security.provider + +import org.uncommons.maths.random.{ AESCounterRNG, SecureRandomSeedGenerator } + +/** + * Internal API + * This class is a wrapper around the AESCounterRNG algorithm provided by http://maths.uncommons.org/ * + * The only method used by netty ssl is engineNextBytes(bytes) + * This RNG is good to use to prevent startup delay when you don't have Internet access to random.org + */ +class AES128CounterSecureRNG extends java.security.SecureRandomSpi { + private val rng = new AESCounterRNG(new SecureRandomSeedGenerator()) + + /** + * This is managed internally by AESCounterRNG + */ + override protected def engineSetSeed(seed: Array[Byte]): Unit = () + + /** + * Generates a user-specified number of random bytes. + * + * @param bytes the array to be filled in with random bytes. + */ + override protected def engineNextBytes(bytes: Array[Byte]): Unit = rng.nextBytes(bytes) + + /** + * Unused method + * Returns the given number of seed bytes. This call may be used to + * seed other random number generators. + * + * @param numBytes the number of seed bytes to generate. + * @return the seed bytes. + */ + override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = (new SecureRandomSeedGenerator()).generateSeed(numBytes) +} + diff --git a/akka-remote/src/main/scala/akka/security/provider/AES256CounterInetRNG.scala b/akka-remote/src/main/scala/akka/security/provider/AES256CounterInetRNG.scala new file mode 100644 index 0000000000..076d4fcd7f --- /dev/null +++ b/akka-remote/src/main/scala/akka/security/provider/AES256CounterInetRNG.scala @@ -0,0 +1,46 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.security.provider + +import org.uncommons.maths.random.{ AESCounterRNG, DefaultSeedGenerator } + +/** + * Internal API + * This class is a wrapper around the 256-bit AESCounterRNG algorithm provided by http://maths.uncommons.org/ + * It uses the default seed generator which uses one of the following 3 random seed sources: + * Depending on availability: /dev/random, random.org and SecureRandom (provided by Java) + * The only method used by netty ssl is engineNextBytes(bytes) + */ +class AES256CounterInetRNG extends java.security.SecureRandomSpi { + /** + * From AESCounterRNG API docs: + * Valid values are 16 (128 bits), 24 (192 bits) and 32 (256 bits). + * Any other values will result in an exception from the AES implementation. + */ + private val AES_256_BIT = 32 // Magic number is magic + private val rng = new AESCounterRNG(AES_256_BIT) + + /** + * This is managed internally by AESCounterRNG + */ + override protected def engineSetSeed(seed: Array[Byte]): Unit = () + + /** + * Generates a user-specified number of random bytes. + * + * @param bytes the array to be filled in with random bytes. + */ + override protected def engineNextBytes(bytes: Array[Byte]): Unit = rng.nextBytes(bytes) + + /** + * Unused method + * Returns the given number of seed bytes. This call may be used to + * seed other random number generators. + * + * @param numBytes the number of seed bytes to generate. + * @return the seed bytes. + */ + override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = DefaultSeedGenerator.getInstance.generateSeed(numBytes) +} + diff --git a/akka-remote/src/main/scala/akka/security/provider/AkkaProvider.scala b/akka-remote/src/main/scala/akka/security/provider/AkkaProvider.scala new file mode 100644 index 0000000000..707ad0c399 --- /dev/null +++ b/akka-remote/src/main/scala/akka/security/provider/AkkaProvider.scala @@ -0,0 +1,27 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.security.provider + +import java.security.{ PrivilegedAction, AccessController, Provider, Security } + +/** + * A provider that for AES128CounterRNGFast, a cryptographically secure random number generator through SecureRandom + */ +object AkkaProvider extends Provider("Akka", 1.0, "Akka provider 1.0 that implements a secure AES random number generator") { + AccessController.doPrivileged(new PrivilegedAction[this.type] { + def run = { + //SecureRandom + put("SecureRandom.AES128CounterSecureRNG", classOf[AES128CounterSecureRNG].getName) + put("SecureRandom.AES128CounterInetRNG", classOf[AES128CounterInetRNG].getName) + put("SecureRandom.AES256CounterInetRNG", classOf[AES256CounterInetRNG].getName) + + //Implementation type: software or hardware + put("SecureRandom.AES128CounterSecureRNG ImplementedIn", "Software") + put("SecureRandom.AES128CounterInetRNG ImplementedIn", "Software") + put("SecureRandom.AES256CounterInetRNG ImplementedIn", "Software") + null //Magic null is magic + } + }) +} + diff --git a/akka-remote/src/main/scala/akka/serialization/DaemonMsgCreateSerializer.scala b/akka-remote/src/main/scala/akka/serialization/DaemonMsgCreateSerializer.scala new file mode 100644 index 0000000000..2905c3ef3b --- /dev/null +++ b/akka-remote/src/main/scala/akka/serialization/DaemonMsgCreateSerializer.scala @@ -0,0 +1,152 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.serialization + +import java.io.Serializable +import com.google.protobuf.ByteString +import com.typesafe.config.Config +import com.typesafe.config.ConfigFactory +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Deploy +import akka.actor.ExtendedActorSystem +import akka.actor.NoScopeGiven +import akka.actor.Props +import akka.actor.Scope +import akka.remote.DaemonMsgCreate +import akka.remote.RemoteProtocol.ActorRefProtocol +import akka.remote.RemoteProtocol.DaemonMsgCreateProtocol +import akka.remote.RemoteProtocol.DeployProtocol +import akka.remote.RemoteProtocol.PropsProtocol +import akka.routing.NoRouter +import akka.routing.RouterConfig +import akka.actor.FromClassCreator + +/** + * Serializes akka's internal DaemonMsgCreate using protobuf + * for the core structure of DaemonMsgCreate, Props and Deploy. + * Serialization of contained RouterConfig, Config, and Scope + * is done with configured serializer for those classes, by + * default java.io.Serializable. + * + * INTERNAL API + */ +private[akka] class DaemonMsgCreateSerializer(val system: ExtendedActorSystem) extends Serializer { + import ProtobufSerializer.serializeActorRef + import ProtobufSerializer.deserializeActorRef + + def includeManifest: Boolean = false + def identifier = 3 + lazy val serialization = SerializationExtension(system) + + def toBinary(obj: AnyRef): Array[Byte] = obj match { + case DaemonMsgCreate(props, deploy, path, supervisor) ⇒ + + def deployProto(d: Deploy): DeployProtocol = { + val builder = DeployProtocol.newBuilder.setPath(d.path) + if (d.config != ConfigFactory.empty) + builder.setConfig(serialize(d.config)) + if (d.routerConfig != NoRouter) + builder.setRouterConfig(serialize(d.routerConfig)) + if (d.scope != NoScopeGiven) + builder.setScope(serialize(d.scope)) + builder.build + } + + def propsProto = { + val builder = PropsProtocol.newBuilder. + setDispatcher(props.dispatcher). + setDeploy(deployProto(props.deploy)) + props.creator match { + case FromClassCreator(clazz) ⇒ builder.setFromClassCreator(clazz.getName) + case creator ⇒ builder.setCreator(serialize(creator)) + } + if (props.routerConfig != NoRouter) + builder.setRouterConfig(serialize(props.routerConfig)) + builder.build + } + + DaemonMsgCreateProtocol.newBuilder. + setProps(propsProto). + setDeploy(deployProto(deploy)). + setPath(path). + setSupervisor(serializeActorRef(supervisor)). + build.toByteArray + + case _ ⇒ + throw new IllegalArgumentException( + "Can't serialize a non-DaemonMsgCreate message using DaemonMsgCreateSerializer [%s]".format(obj)) + } + + def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = { + val proto = DaemonMsgCreateProtocol.parseFrom(bytes) + + def deploy(protoDeploy: DeployProtocol): Deploy = { + val config = + if (protoDeploy.hasConfig) deserialize(protoDeploy.getConfig, classOf[Config]) + else ConfigFactory.empty + val routerConfig = + if (protoDeploy.hasRouterConfig) deserialize(protoDeploy.getRouterConfig, classOf[RouterConfig]) + else NoRouter + val scope = + if (protoDeploy.hasScope) deserialize(protoDeploy.getScope, classOf[Scope]) + else NoScopeGiven + Deploy(protoDeploy.getPath, config, routerConfig, scope) + } + + def props = { + val creator = + if (proto.getProps.hasFromClassCreator) { + system.dynamicAccess.getClassFor(proto.getProps.getFromClassCreator) match { + case Right(clazz) ⇒ FromClassCreator(clazz) + case Left(e) ⇒ throw e + } + } else { + deserialize(proto.getProps.getCreator, classOf[() ⇒ Actor]) + } + + val routerConfig = + if (proto.getProps.hasRouterConfig) deserialize(proto.getProps.getRouterConfig, classOf[RouterConfig]) + else NoRouter + + Props( + creator = creator, + dispatcher = proto.getProps.getDispatcher, + routerConfig = routerConfig, + deploy = deploy(proto.getProps.getDeploy)) + } + + DaemonMsgCreate( + props = props, + deploy = deploy(proto.getDeploy), + path = proto.getPath, + supervisor = deserializeActorRef(system, proto.getSupervisor)) + } + + protected def serialize(any: AnyRef): ByteString = + serialization.serialize(any) match { + case Right(bytes) ⇒ ByteString.copyFrom(bytes) + case Left(e) ⇒ throw e + } + + protected def deserialize[T: ClassManifest](data: ByteString, clazz: Class[T]): T = { + val bytes = data.toByteArray + serialization.deserialize(bytes, clazz) match { + case Right(x) if classManifest[T].erasure.isInstance(x) ⇒ x.asInstanceOf[T] + case Right(other) ⇒ throw new IllegalArgumentException("Can't deserialize to [%s], got [%s]". + format(clazz.getName, other)) + case Left(e) ⇒ + // Fallback to the java serializer, because some interfaces don't implement java.io.Serializable, + // but the impl instance does. This could be optimized by adding java serializers in reference.conf: + // com.typesafe.config.Config + // akka.routing.RouterConfig + // akka.actor.Scope + serialization.deserialize(bytes, classOf[java.io.Serializable]) match { + case Right(x) if classManifest[T].erasure.isInstance(x) ⇒ x.asInstanceOf[T] + case _ ⇒ throw e // the first exception + } + } + } +} \ No newline at end of file diff --git a/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala b/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala index 813a22fba4..77f6702a77 100644 --- a/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala +++ b/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala @@ -6,6 +6,32 @@ package akka.serialization import com.google.protobuf.Message import akka.actor.DynamicAccess +import akka.remote.RemoteProtocol.ActorRefProtocol +import akka.actor.ActorSystem +import akka.actor.ActorRef + +object ProtobufSerializer { + + /** + * Helper to serialize an [[akka.actor.ActorRef]] to Akka's + * protobuf representation. + */ + def serializeActorRef(ref: ActorRef): ActorRefProtocol = { + val identifier: String = Serialization.currentTransportAddress.value match { + case null ⇒ ref.path.toString + case address ⇒ ref.path.toStringWithAddress(address) + } + ActorRefProtocol.newBuilder.setPath(identifier).build + } + + /** + * Helper to materialize (lookup) an [[akka.actor.ActorRef]] + * from Akka's protobuf representation in the supplied + * [[akka.actor.ActorSystem]]. + */ + def deserializeActorRef(system: ActorSystem, refProtocol: ActorRefProtocol): ActorRef = + system.actorFor(refProtocol.getPath) +} /** * This Serializer serializes `com.google.protobuf.Message`s diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala deleted file mode 100644 index ab8bdadae6..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala +++ /dev/null @@ -1,29 +0,0 @@ -package akka.remote - -import com.typesafe.config.{Config, ConfigFactory} - -trait AbstractRemoteActorMultiJvmSpec { - def NrOfNodes: Int - def commonConfig: Config - - def PortRangeStart = 1990 - def NodeRange = 1 to NrOfNodes - def PortRange = PortRangeStart to NrOfNodes - - private[this] val remotes: IndexedSeq[String] = { - val nodesOpt = Option(AkkaRemoteSpec.testNodes).map(_.split(",").toIndexedSeq) - nodesOpt getOrElse IndexedSeq.fill(NrOfNodes)("localhost") - } - - val nodeConfigs = (NodeRange.toList zip remotes) map { - case (port, host) => - ConfigFactory.parseString(""" - akka { - remote.netty.hostname="%s" - remote.netty.port = "%d" - }""".format(host, PortRangeStart + port, port)) withFallback commonConfig - } - - def akkaSpec(port: Int) = "AkkaRemoteSpec@%s:%d".format(remotes(port), PortRangeStart + 1 + port) - def akkaURIs(count: Int): String = 0 until count map {idx => "\"akka://" + akkaSpec(idx) + "\""} mkString "," -} diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala deleted file mode 100644 index c1a2109bc0..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala +++ /dev/null @@ -1,33 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.remote - -import akka.testkit._ -import com.typesafe.config.Config -import com.typesafe.config.ConfigFactory -import com.typesafe.config.ConfigParseOptions -import com.typesafe.config.ConfigResolveOptions -import java.io.File -import akka.actor.{ActorSystem, ActorSystemImpl} - -object AkkaRemoteSpec { - private def configParseOptions = ConfigParseOptions.defaults.setAllowMissing(false) - - val testConf: Config = { - System.getProperty("akka.config") match { - case null ⇒ AkkaSpec.testConf - case location ⇒ - ConfigFactory.systemProperties - .withFallback(ConfigFactory.parseFileAnySyntax(new File(location), configParseOptions)) - .withFallback(ConfigFactory.defaultReference(ActorSystem.findClassLoader())).resolve(ConfigResolveOptions.defaults) - } - } - - val testNodes = System.getProperty("test.hosts") -} - -abstract class AkkaRemoteSpec(config: Config) - extends AkkaSpec(config.withFallback(AkkaRemoteSpec.testConf)) - with MultiJvmSync diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/Barrier.scala b/akka-remote/src/multi-jvm/scala/akka/remote/Barrier.scala deleted file mode 100644 index e99fca2a45..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/Barrier.scala +++ /dev/null @@ -1,19 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.remote - -trait Barrier { - def await() = { enter(); leave() } - - def apply(body: ⇒ Unit) { - enter() - body - leave() - } - - def enter(): Unit - - def leave(): Unit -} diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala deleted file mode 100644 index 3026ddd613..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala +++ /dev/null @@ -1,64 +0,0 @@ -package akka.remote - -import akka.actor.{ Actor, ActorRef, Props } -import akka.testkit._ -import akka.dispatch.Await -import akka.pattern.ask - -object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { - override def NrOfNodes = 2 - - class SomeActor extends Actor with Serializable { - def receive = { - case "identify" ⇒ sender ! self - } - } - - import com.typesafe.config.ConfigFactory - override def commonConfig = ConfigFactory.parseString(""" - akka { - loglevel = "WARNING" - actor { - provider = "akka.remote.RemoteActorRefProvider" - deployment { - /service-hello.remote = %s - } - } - }""" format akkaURIs(1)) -} - -import DirectRoutedRemoteActorMultiJvmSpec._ - -class DirectRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(nodeConfigs(0)) { - import DirectRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - - "___" must { - "___" in { - barrier("start") - barrier("done") - } - } -} - -class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(nodeConfigs(1)) with DefaultTimeout { - - import DirectRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - - "A new remote actor configured with a Direct router" must { - "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" in { - barrier("start") - - val actor = system.actorOf(Props[SomeActor], "service-hello") - actor.isInstanceOf[RemoteActorRef] must be(true) - - Await.result(actor ? "identify", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort must equal(akkaSpec(0)) - - // shut down the actor before we let the other node(s) shut down so we don't try to send - // "Terminate" to a shut down node - system.stop(actor) - barrier("done") - } - } -} diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/FileBasedBarrier.scala b/akka-remote/src/multi-jvm/scala/akka/remote/FileBasedBarrier.scala deleted file mode 100644 index a1773fc86e..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/FileBasedBarrier.scala +++ /dev/null @@ -1,83 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.remote - -import akka.util.duration._ -import akka.util.Duration -import System.{ currentTimeMillis ⇒ now } - -import java.io.File - -class BarrierTimeoutException(message: String) extends RuntimeException(message) - -object FileBasedBarrier { - val HomeDir = ".multi-jvm" - val DefaultTimeout = 30.seconds - val DefaultSleep = 100.millis -} - -import FileBasedBarrier._ - -class FileBasedBarrier( - name: String, - count: Int, - group: String, - node: String, - timeout: Duration = FileBasedBarrier.DefaultTimeout, - sleep: Duration = FileBasedBarrier.DefaultSleep) extends Barrier { - - val barrierDir = { - val dir = new File(new File(new File(FileBasedBarrier.HomeDir), group), name) - dir.mkdirs() - dir - } - - val nodeFile = new File(barrierDir, node) - - val readyFile = new File(barrierDir, "ready") - - def enter() = { - createNode() - if (nodesPresent >= count) createReady() - val ready = waitFor(readyFile.exists, timeout, sleep) - if (!ready) expire("entry") - } - - def leave() = { - removeNode() - val empty = waitFor(nodesPresent <= 1, timeout, sleep) - removeReady() - if (!empty) expire("exit") - } - - def nodesPresent = barrierDir.list.size - - def createNode() = nodeFile.createNewFile() - - def removeNode() = nodeFile.delete() - - def createReady() = readyFile.createNewFile() - - def removeReady() = readyFile.delete() - - def waitFor(test: ⇒ Boolean, timeout: Duration, sleep: Duration): Boolean = { - val start = now - val limit = start + timeout.toMillis - var passed = test - var expired = false - while (!passed && !expired) { - if (now > limit) expired = true - else { - Thread.sleep(sleep.toMillis) - passed = test - } - } - passed - } - - def expire(barrier: String) = { - throw new BarrierTimeoutException("Timeout (%s) waiting for %s barrier" format (timeout, barrier)) - } -} diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/MultiJvmSync.scala b/akka-remote/src/multi-jvm/scala/akka/remote/MultiJvmSync.scala deleted file mode 100644 index c1e6080e6e..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/MultiJvmSync.scala +++ /dev/null @@ -1,49 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.remote - -import akka.testkit.AkkaSpec -import akka.util.Duration - -trait MultiJvmSync extends AkkaSpec { - def nodes: Int - - override def atStartup() = { - onStart() - MultiJvmSync.start(getClass.getName, nodes) - } - - def onStart() {} - - override def atTermination() = { - MultiJvmSync.end(getClass.getName, nodes) - onEnd() - } - - def onEnd() {} - - def barrier(name: String, timeout: Duration = FileBasedBarrier.DefaultTimeout) = { - MultiJvmSync.barrier(name, nodes, getClass.getName, timeout) - } -} - -object MultiJvmSync { - val TestMarker = "MultiJvm" - val StartBarrier = "multi-jvm-start" - val EndBarrier = "multi-jvm-end" - - def start(className: String, count: Int) = barrier(StartBarrier, count, className) - - def end(className: String, count: Int) = barrier(EndBarrier, count, className) - - def barrier(name: String, count: Int, className: String, timeout: Duration = FileBasedBarrier.DefaultTimeout) = { - val Array(testName, nodeName) = className split TestMarker - val barrier = if (AkkaRemoteSpec.testNodes eq null) - new FileBasedBarrier(name, count, testName, nodeName, timeout) - else - new ZkClient.ZkBarrier(nodeName, count, "/" + testName + "_" + name) - barrier.await() - } -} diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala deleted file mode 100644 index c3dc1ae9de..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala +++ /dev/null @@ -1,65 +0,0 @@ -package akka.remote - -import akka.actor.{ Actor, ActorRef, Props } -import akka.testkit._ -import akka.dispatch.Await -import akka.pattern.ask - -object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { - override def NrOfNodes = 2 - - class SomeActor extends Actor with Serializable { - def receive = { - case "identify" ⇒ sender ! self - } - } - - import com.typesafe.config.ConfigFactory - override def commonConfig = ConfigFactory.parseString(""" - akka { - loglevel = "WARNING" - actor { - provider = "akka.remote.RemoteActorRefProvider" - deployment { - /service-hello.remote = %s - } - } - }""" format akkaURIs(1)) -} - -class NewRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(NewRemoteActorMultiJvmSpec.nodeConfigs(0)) { - - import NewRemoteActorMultiJvmSpec._ - - val nodes = NrOfNodes - - "___" must { - "___" in { - barrier("start") - - barrier("done") - } - } -} - -class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(NewRemoteActorMultiJvmSpec.nodeConfigs(1)) with DefaultTimeout { - - import NewRemoteActorMultiJvmSpec._ - - val nodes = NrOfNodes - - "A new remote actor" must { - "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" in { - barrier("start") - - val actor = system.actorOf(Props[SomeActor], "service-hello") - Await.result(actor ? "identify", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort must equal(akkaSpec(0)) - - // shut down the actor before we let the other node(s) shut down so we don't try to send - // "Terminate" to a shut down node - system.stop(actor) - barrier("done") - } - } -} - diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala deleted file mode 100644 index 2b2b233dee..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala +++ /dev/null @@ -1,110 +0,0 @@ -package akka.remote - -import akka.actor.{ Actor, ActorRef, Props } -import akka.routing._ -import akka.testkit.DefaultTimeout -import akka.dispatch.Await -import akka.pattern.ask - -object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { - override def NrOfNodes = 4 - class SomeActor extends Actor with Serializable { - def receive = { - case "hit" ⇒ sender ! self - case "end" ⇒ context.stop(self) - } - } - - import com.typesafe.config.ConfigFactory - override def commonConfig = ConfigFactory.parseString(""" - akka { - loglevel = "WARNING" - actor { - provider = "akka.remote.RemoteActorRefProvider" - deployment { - /service-hello.router = "random" - /service-hello.nr-of-instances = %d - /service-hello.target.nodes = [%s] - } - } - }""" format (3, akkaURIs(3))) -} - -class RandomRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(RandomRoutedRemoteActorMultiJvmSpec.nodeConfigs(0)) { - import RandomRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "___" must { - "___" in { - barrier("start") - barrier("broadcast-end") - barrier("end") - barrier("done") - } - } -} - -class RandomRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(RandomRoutedRemoteActorMultiJvmSpec.nodeConfigs(1)) { - import RandomRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "___" must { - "___" in { - barrier("start") - barrier("broadcast-end") - barrier("end") - barrier("done") - } - } -} - -class RandomRoutedRemoteActorMultiJvmNode3 extends AkkaRemoteSpec(RandomRoutedRemoteActorMultiJvmSpec.nodeConfigs(2)) { - import RandomRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "___" must { - "___" in { - barrier("start") - barrier("broadcast-end") - barrier("end") - barrier("done") - } - } -} - -class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RandomRoutedRemoteActorMultiJvmSpec.nodeConfigs(3)) with DefaultTimeout { - import RandomRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "A new remote actor configured with a Random router" must { - "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" in { - - barrier("start") - val actor = system.actorOf(Props[SomeActor].withRouter(RandomRouter()), "service-hello") - actor.isInstanceOf[RoutedActorRef] must be(true) - - val connectionCount = NrOfNodes - 1 - val iterationCount = 10 - - var replies = Map( - akkaSpec(0) -> 0, - akkaSpec(1) -> 0, - akkaSpec(2) -> 0) - - for (i ← 0 until iterationCount) { - for (k ← 0 until connectionCount) { - val nodeName = Await.result(actor ? "hit", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort - replies = replies + (nodeName -> (replies(nodeName) + 1)) - } - } - - barrier("broadcast-end") - actor ! Broadcast("end") - - barrier("end") - replies.values foreach { _ must be > (0) } - - // shut down the actor before we let the other node(s) shut down so we don't try to send - // "Terminate" to a shut down node - system.stop(actor) - barrier("done") - } - } -} - diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala deleted file mode 100644 index c84aa46366..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ /dev/null @@ -1,112 +0,0 @@ -package akka.remote - -import akka.actor.{ Actor, ActorRef, Props } -import akka.routing._ -import akka.testkit.DefaultTimeout -import akka.dispatch.Await -import akka.pattern.ask - -object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { - override def NrOfNodes = 4 - - class SomeActor extends Actor with Serializable { - def receive = { - case "hit" ⇒ sender ! self - case "end" ⇒ context.stop(self) - } - } - - import com.typesafe.config.ConfigFactory - override def commonConfig = ConfigFactory.parseString(""" - akka { - loglevel = "WARNING" - actor { - provider = "akka.remote.RemoteActorRefProvider" - deployment { - /service-hello.router = "round-robin" - /service-hello.nr-of-instances = %d - /service-hello.target.nodes = [%s] - } - } - }""" format (3, akkaURIs(3))) -} - -class RoundRobinRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(RoundRobinRoutedRemoteActorMultiJvmSpec.nodeConfigs(0)) { - import RoundRobinRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "___" must { - "___" in { - barrier("start") - barrier("broadcast-end") - barrier("end") - barrier("done") - } - } -} - -class RoundRobinRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(RoundRobinRoutedRemoteActorMultiJvmSpec.nodeConfigs(1)) { - import RoundRobinRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "___" must { - "___" in { - barrier("start") - barrier("broadcast-end") - barrier("end") - barrier("done") - } - } -} - -class RoundRobinRoutedRemoteActorMultiJvmNode3 extends AkkaRemoteSpec(RoundRobinRoutedRemoteActorMultiJvmSpec.nodeConfigs(2)) { - import RoundRobinRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "___" must { - "___" in { - barrier("start") - barrier("broadcast-end") - barrier("end") - barrier("done") - } - } -} - -class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RoundRobinRoutedRemoteActorMultiJvmSpec.nodeConfigs(3)) with DefaultTimeout { - import RoundRobinRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "A new remote actor configured with a RoundRobin router" must { - "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" in { - - barrier("start") - val actor = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "service-hello") - actor.isInstanceOf[RoutedActorRef] must be(true) - - val connectionCount = NrOfNodes - 1 - val iterationCount = 10 - - var replies = Map( - akkaSpec(0) -> 0, - akkaSpec(1) -> 0, - akkaSpec(2) -> 0) - - for (i ← 0 until iterationCount) { - for (k ← 0 until connectionCount) { - val nodeName = Await.result(actor ? "hit", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort - - replies = replies + (nodeName -> (replies(nodeName) + 1)) - } - } - - barrier("broadcast-end") - actor ! Broadcast("end") - - barrier("end") - replies.values foreach { _ must be(10) } - - // shut down the actor before we let the other node(s) shut down so we don't try to send - // "Terminate" to a shut down node - system.stop(actor) - barrier("done") - } - } -} - diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala deleted file mode 100644 index b618300ff2..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala +++ /dev/null @@ -1,107 +0,0 @@ -package akka.remote - -import akka.actor.{ Actor, ActorRef, Props } -import akka.routing._ -import akka.testkit._ -import akka.util.duration._ - -object ScatterGatherRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { - override def NrOfNodes = 4 - class SomeActor extends Actor with Serializable { - def receive = { - case "hit" ⇒ sender ! self - case "end" ⇒ context.stop(self) - } - } - - import com.typesafe.config.ConfigFactory - override def commonConfig = ConfigFactory.parseString(""" - akka { - loglevel = "WARNING" - actor { - provider = "akka.remote.RemoteActorRefProvider" - deployment { - /service-hello.router = "scatter-gather" - /service-hello.nr-of-instances = %d - /service-hello.target.nodes = [%s] - } - } - }""" format (3, akkaURIs(3))) -} - -class ScatterGatherRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(ScatterGatherRoutedRemoteActorMultiJvmSpec.nodeConfigs(0)) { - import ScatterGatherRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "___" must { - "___" in { - barrier("start") - barrier("broadcast-end") - barrier("end") - barrier("done") - } - } -} - -class ScatterGatherRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(ScatterGatherRoutedRemoteActorMultiJvmSpec.nodeConfigs(1)) { - import ScatterGatherRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "___" must { - "___" in { - barrier("start") - barrier("broadcast-end") - barrier("end") - barrier("done") - } - } -} - -class ScatterGatherRoutedRemoteActorMultiJvmNode3 extends AkkaRemoteSpec(ScatterGatherRoutedRemoteActorMultiJvmSpec.nodeConfigs(2)) { - import ScatterGatherRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "___" must { - "___" in { - barrier("start") - barrier("broadcast-end") - barrier("end") - barrier("done") - } - } -} - -class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(ScatterGatherRoutedRemoteActorMultiJvmSpec.nodeConfigs(3)) - with DefaultTimeout with ImplicitSender { - import ScatterGatherRoutedRemoteActorMultiJvmSpec._ - val nodes = NrOfNodes - "A new remote actor configured with a ScatterGather router" must { - "be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" in { - - barrier("start") - val actor = system.actorOf(Props[SomeActor].withRouter(ScatterGatherFirstCompletedRouter(within = 10 seconds)), "service-hello") - actor.isInstanceOf[RoutedActorRef] must be(true) - - val connectionCount = NrOfNodes - 1 - val iterationCount = 10 - - for (i ← 0 until iterationCount) { - for (k ← 0 until connectionCount) { - actor ! "hit" - } - } - - val replies = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) { - case ref: ActorRef ⇒ (ref.path.address.hostPort, 1) - }).foldLeft(Map(akkaSpec(0) -> 0, akkaSpec(1) -> 0, akkaSpec(2) -> 0)) { - case (m, (n, c)) ⇒ m + (n -> (m(n) + c)) - } - - barrier("broadcast-end") - actor ! Broadcast("end") - - barrier("end") - replies.values.sum must be === connectionCount * iterationCount - - barrier("done") - } - } -} - diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/ZKClient.scala b/akka-remote/src/multi-jvm/scala/akka/remote/ZKClient.scala deleted file mode 100644 index 611478babb..0000000000 --- a/akka-remote/src/multi-jvm/scala/akka/remote/ZKClient.scala +++ /dev/null @@ -1,71 +0,0 @@ -/** - * Copyright (C) 2011-2012 Typesafe - */ -package akka.remote - -import org.apache.zookeeper._ -import ZooDefs.Ids - -object ZkClient extends Watcher { - // Don't forget to close! - lazy val zk: ZooKeeper = { - val remoteNodes = AkkaRemoteSpec.testNodes split ',' - - // ZkServers are configured to listen on a specific port. - val connectString = remoteNodes map (_+":2181") mkString "," - new ZooKeeper(connectString, 3000, this) - } - - def process(ev: WatchedEvent) { - synchronized { notify() } - } - - class ZkBarrier(name: String, count: Int, root: String) extends Barrier { - @annotation.tailrec - private def waitForServer() { - // SI-1672 - val r = try { - zk.exists("/", false) - true - } catch { - case _: KeeperException.ConnectionLossException => - Thread.sleep(10000) - false - } - if (!r) waitForServer() - } - waitForServer() - - try zk.create(root, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT) catch { - case _: KeeperException.NodeExistsException => - } - - val timeoutMs = 300*1000 - - private def block(num: Int) { - val start = System.currentTimeMillis - while (true) { - if (System.currentTimeMillis - start > timeoutMs) throw new InterruptedException("Timed out blocking in zk") - ZkClient.this.synchronized { - val children = zk.getChildren(root, true) - if (children.size < num) { - ZkClient.this.wait(timeoutMs) - } else - return - } - } - } - - def enter() { - zk.create(root + "/" + name, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL) - block(count) - } - - final def leave() { - zk.create(root + "/" + name + ".leave", Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL) - block(2*count) - } - } - - def barrier(name: String, count: Int, root: String) = new ZkBarrier(name, count, root) -} diff --git a/akka-remote/src/test/protocol/ProtobufProtocol.proto b/akka-remote/src/test/protocol/ProtobufProtocol.proto index ccb92aa1e3..0ff2663821 100644 --- a/akka-remote/src/test/protocol/ProtobufProtocol.proto +++ b/akka-remote/src/test/protocol/ProtobufProtocol.proto @@ -4,11 +4,13 @@ package akka.actor; -/* +/****************************************** Compile with: cd ./akka-remote/src/test/protocol protoc ProtobufProtocol.proto --java_out ../java -*/ + cd ../../../.. + ./scripts/fix-protobuf.sh +*******************************************/ message MyMessage { required uint64 id = 1; diff --git a/akka-remote/src/test/resources/keystore b/akka-remote/src/test/resources/keystore new file mode 100644 index 0000000000..ee5581d930 Binary files /dev/null and b/akka-remote/src/test/resources/keystore differ diff --git a/akka-remote/src/test/resources/truststore b/akka-remote/src/test/resources/truststore new file mode 100644 index 0000000000..cc07616dad Binary files /dev/null and b/akka-remote/src/test/resources/truststore differ diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index ac4127fe17..7f92e3089b 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -118,7 +118,7 @@ akka { val r = expectMsgType[ActorRef] r ! (Props[Echo], "grandchild") val remref = expectMsgType[ActorRef] - remref.isInstanceOf[LocalActorRef] must be(true) + remref.asInstanceOf[ActorRefScope].isLocal must be(true) val myref = system.actorFor(system / "looker" / "child" / "grandchild") myref.isInstanceOf[RemoteActorRef] must be(true) myref ! 43 diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala index f1809d42a5..e35cd42cd8 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala @@ -27,6 +27,7 @@ class RemoteConfigSpec extends AkkaSpec( RemoteTransport must be("akka.remote.netty.NettyRemoteTransport") UntrustedMode must be(false) RemoteSystemDaemonAckTimeout must be(30 seconds) + LogRemoteLifeCycleEvents must be(false) } "be able to parse Netty config elements" in { @@ -56,6 +57,10 @@ class RemoteConfigSpec extends AkkaSpec( WriteTimeout must be(10 seconds) AllTimeout must be(0 millis) ReconnectionTimeWindow must be(10 minutes) + WriteBufferHighWaterMark must be(None) + WriteBufferLowWaterMark must be(None) + SendBufferSize must be(None) + ReceiveBufferSize must be(None) } } diff --git a/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala new file mode 100644 index 0000000000..64408f15b1 --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala @@ -0,0 +1,151 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote + +import akka.testkit._ +import akka.actor._ +import com.typesafe.config._ +import akka.dispatch.{ Await, Future } +import akka.pattern.ask +import java.io.File +import java.security.{ NoSuchAlgorithmException, SecureRandom, PrivilegedAction, AccessController } +import netty.{ NettySettings, NettySSLSupport } +import javax.net.ssl.SSLException +import akka.util.{ Timeout, Duration } +import akka.util.duration._ +import akka.event.{ Logging, NoLogging, LoggingAdapter } + +object Configuration { + // set this in your JAVA_OPTS to see all ssl debug info: "-Djavax.net.debug=ssl,keymanager" + // The certificate will expire in 2109 + private val trustStore = getClass.getClassLoader.getResource("truststore").getPath + private val keyStore = getClass.getClassLoader.getResource("keystore").getPath + private val conf = """ + akka { + actor.provider = "akka.remote.RemoteActorRefProvider" + test { + single-expect-default = 10s + filter-leeway = 10s + default-timeout = 10s + } + + remote.netty { + hostname = localhost + port = %d + ssl { + enable = on + trust-store = "%s" + key-store = "%s" + random-number-generator = "%s" + enabled-algorithms = [%s] + sha1prng-random-source = "/dev/./urandom" + } + } + } + """ + + case class CipherConfig(runTest: Boolean, config: Config, cipher: String, localPort: Int, remotePort: Int) + + def getCipherConfig(cipher: String, enabled: String*): CipherConfig = { + val localPort, remotePort = { val s = new java.net.ServerSocket(0); try s.getLocalPort finally s.close() } + try { + //if (true) throw new IllegalArgumentException("Ticket1978*Spec isn't enabled") + + val config = ConfigFactory.parseString(conf.format(localPort, trustStore, keyStore, cipher, enabled.mkString(", "))) + val fullConfig = config.withFallback(AkkaSpec.testConf).withFallback(ConfigFactory.load).getConfig("akka.remote.netty") + val settings = new NettySettings(fullConfig, "placeholder") + + val rng = NettySSLSupport.initializeCustomSecureRandom(settings.SSLRandomNumberGenerator, settings.SSLRandomSource, NoLogging) + + rng.nextInt() // Has to work + settings.SSLRandomNumberGenerator foreach { sRng ⇒ rng.getAlgorithm == sRng || (throw new NoSuchAlgorithmException(sRng)) } + + val engine = NettySSLSupport.initializeClientSSL(settings, NoLogging).getEngine + val gotAllSupported = enabled.toSet -- engine.getSupportedCipherSuites.toSet + val gotAllEnabled = enabled.toSet -- engine.getEnabledCipherSuites.toSet + gotAllSupported.isEmpty || (throw new IllegalArgumentException("Cipher Suite not supported: " + gotAllSupported)) + gotAllEnabled.isEmpty || (throw new IllegalArgumentException("Cipher Suite not enabled: " + gotAllEnabled)) + engine.getSupportedProtocols.contains(settings.SSLProtocol.get) || (throw new IllegalArgumentException("Protocol not supported: " + settings.SSLProtocol.get)) + + CipherConfig(true, config, cipher, localPort, remotePort) + } catch { + case (_: IllegalArgumentException) | (_: NoSuchAlgorithmException) ⇒ CipherConfig(false, AkkaSpec.testConf, cipher, localPort, remotePort) // Cannot match against the message since the message might be localized :S + } + } +} + +import Configuration.{ CipherConfig, getCipherConfig } + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class Ticket1978SHA1PRNGSpec extends Ticket1978CommunicationSpec(getCipherConfig("SHA1PRNG", "TLS_RSA_WITH_AES_128_CBC_SHA")) + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class Ticket1978AES128CounterSecureRNGSpec extends Ticket1978CommunicationSpec(getCipherConfig("AES128CounterSecureRNG", "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + +/** + * Both of the Inet variants require access to the Internet to access random.org. + */ +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class Ticket1978AES128CounterInetRNGSpec extends Ticket1978CommunicationSpec(getCipherConfig("AES128CounterInetRNG", "TLS_RSA_WITH_AES_128_CBC_SHA")) + +/** + * Both of the Inet variants require access to the Internet to access random.org. + */ +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class Ticket1978AES256CounterInetRNGSpec extends Ticket1978CommunicationSpec(getCipherConfig("AES256CounterInetRNG", "TLS_RSA_WITH_AES_256_CBC_SHA")) + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class Ticket1978DefaultRNGSecureSpec extends Ticket1978CommunicationSpec(getCipherConfig("", "TLS_RSA_WITH_AES_128_CBC_SHA")) + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class Ticket1978CrappyRSAWithMD5OnlyHereToMakeSureThingsWorkSpec extends Ticket1978CommunicationSpec(getCipherConfig("", "SSL_RSA_WITH_NULL_MD5")) + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class Ticket1978NonExistingRNGSecureSpec extends Ticket1978CommunicationSpec(CipherConfig(false, AkkaSpec.testConf, "NonExistingRNG", 12345, 12346)) + +abstract class Ticket1978CommunicationSpec(val cipherConfig: CipherConfig) extends AkkaSpec(cipherConfig.config) with ImplicitSender { + + implicit val timeout: Timeout = Timeout(10 seconds) + + import RemoteCommunicationSpec._ + + lazy val other: ActorSystem = ActorSystem( + "remote-sys", + ConfigFactory.parseString("akka.remote.netty.port=" + cipherConfig.remotePort).withFallback(system.settings.config)) + + override def atTermination() { + if (cipherConfig.runTest) { + other.shutdown() + other.awaitTermination() + } + } + + ("-") must { + if (cipherConfig.runTest) { + val ignoreMe = other.actorOf(Props(new Actor { def receive = { case ("ping", x) ⇒ sender ! ((("pong", x), sender)) } }), "echo") + val otherAddress = other.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.address + + "support tell" in { + val here = system.actorFor(otherAddress.toString + "/user/echo") + + for (i ← 1 to 1000) here ! (("ping", i)) + for (i ← 1 to 1000) expectMsgPF(timeout.duration) { case (("pong", i), `testActor`) ⇒ true } + } + + "support ask" in { + val here = system.actorFor(otherAddress.toString + "/user/echo") + + val f = for (i ← 1 to 1000) yield here ? (("ping", i)) mapTo manifest[((String, Int), ActorRef)] + Await.result(Future.sequence(f), timeout.duration).map(_._1._1).toSet must be(Set("pong")) + } + + } else { + "not be run when the cipher is not supported by the platform this test is currently being executed on" ignore { + + } + } + + } + +} diff --git a/akka-remote/src/test/scala/akka/remote/Ticket1978ConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/Ticket1978ConfigSpec.scala new file mode 100644 index 0000000000..0a39d20a9a --- /dev/null +++ b/akka-remote/src/test/scala/akka/remote/Ticket1978ConfigSpec.scala @@ -0,0 +1,43 @@ +package akka.remote + +import akka.testkit._ +import akka.actor._ +import com.typesafe.config._ +import akka.actor.ExtendedActorSystem +import akka.util.duration._ +import akka.util.Duration +import akka.remote.netty.NettyRemoteTransport +import java.util.ArrayList + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class Ticket1978ConfigSpec extends AkkaSpec(""" +akka { + actor.provider = "akka.remote.RemoteActorRefProvider" + remote.netty { + hostname = localhost + port = 0 + } +} +""") with ImplicitSender with DefaultTimeout { + + "SSL Remoting" must { + "be able to parse these extra Netty config elements" in { + val settings = + system.asInstanceOf[ExtendedActorSystem] + .provider.asInstanceOf[RemoteActorRefProvider] + .transport.asInstanceOf[NettyRemoteTransport] + .settings + import settings._ + + EnableSSL must be(false) + SSLKeyStore must be(Some("keystore")) + SSLKeyStorePassword must be(Some("changeme")) + SSLTrustStore must be(Some("truststore")) + SSLTrustStorePassword must be(Some("changeme")) + SSLProtocol must be(Some("TLSv1")) + SSLEnabledAlgorithms must be(Set("TLS_RSA_WITH_AES_128_CBC_SHA")) + SSLRandomSource must be(None) + SSLRandomNumberGenerator must be(None) + } + } +} diff --git a/akka-remote/src/test/scala/akka/serialization/DaemonMsgCreateSerializerSpec.scala b/akka-remote/src/test/scala/akka/serialization/DaemonMsgCreateSerializerSpec.scala new file mode 100644 index 0000000000..e38a3e1d1f --- /dev/null +++ b/akka-remote/src/test/scala/akka/serialization/DaemonMsgCreateSerializerSpec.scala @@ -0,0 +1,113 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.serialization + +import com.typesafe.config.ConfigFactory +import akka.testkit.AkkaSpec +import akka.actor.Actor +import akka.actor.Address +import akka.actor.Props +import akka.actor.Deploy +import akka.actor.OneForOneStrategy +import akka.actor.SupervisorStrategy +import akka.remote.DaemonMsgCreate +import akka.remote.RemoteScope +import akka.routing.RoundRobinRouter +import akka.routing.FromConfig +import akka.util.duration._ +import akka.actor.FromClassCreator + +object DaemonMsgCreateSerializerSpec { + class MyActor extends Actor { + def receive = { + case _ ⇒ + } + } +} + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class DaemonMsgCreateSerializerSpec extends AkkaSpec { + + import DaemonMsgCreateSerializerSpec._ + val ser = SerializationExtension(system) + val supervisor = system.actorOf(Props[MyActor], "supervisor") + + "Serialization" must { + + "resolve DaemonMsgCreateSerializer" in { + ser.serializerFor(classOf[DaemonMsgCreate]).getClass must be(classOf[DaemonMsgCreateSerializer]) + } + + "serialize and de-serialize DaemonMsgCreate with FromClassCreator" in { + verifySerialization { + DaemonMsgCreate( + props = Props[MyActor], + deploy = Deploy(), + path = "foo", + supervisor = supervisor) + } + } + + "serialize and de-serialize DaemonMsgCreate with function creator" in { + verifySerialization { + DaemonMsgCreate( + props = Props().withCreator(new MyActor), + deploy = Deploy(), + path = "foo", + supervisor = supervisor) + } + } + + "serialize and de-serialize DaemonMsgCreate with Deploy and RouterConfig" in { + verifySerialization { + // Duration.Inf doesn't equal Duration.Inf, so we use another for test + val supervisorStrategy = OneForOneStrategy(3, 10 seconds) { + case _ ⇒ SupervisorStrategy.Escalate + } + val deploy1 = Deploy( + path = "path1", + config = ConfigFactory.parseString("a=1"), + routerConfig = RoundRobinRouter(nrOfInstances = 5, supervisorStrategy = supervisorStrategy), + scope = RemoteScope(Address("akka", "Test", "host1", 1921))) + val deploy2 = Deploy( + path = "path2", + config = ConfigFactory.parseString("a=2"), + routerConfig = FromConfig, + scope = RemoteScope(Address("akka", "Test", "host2", 1922))) + DaemonMsgCreate( + props = Props[MyActor].withDispatcher("my-disp").withDeploy(deploy1), + deploy = deploy2, + path = "foo", + supervisor = supervisor) + } + } + + def verifySerialization(msg: DaemonMsgCreate): Unit = { + val bytes = ser.serialize(msg) match { + case Left(exception) ⇒ fail(exception) + case Right(bytes) ⇒ bytes + } + ser.deserialize(bytes.asInstanceOf[Array[Byte]], classOf[DaemonMsgCreate]) match { + case Left(exception) ⇒ fail(exception) + case Right(m: DaemonMsgCreate) ⇒ assertDaemonMsgCreate(msg, m) + } + } + + def assertDaemonMsgCreate(expected: DaemonMsgCreate, got: DaemonMsgCreate): Unit = { + // can't compare props.creator when function + if (expected.props.creator.isInstanceOf[FromClassCreator]) + assert(got.props.creator === expected.props.creator) + assert(got.props.dispatcher === expected.props.dispatcher) + assert(got.props.dispatcher === expected.props.dispatcher) + assert(got.props.routerConfig === expected.props.routerConfig) + assert(got.props.deploy === expected.props.deploy) + assert(got.deploy === expected.deploy) + assert(got.path === expected.path) + assert(got.supervisor === expected.supervisor) + } + + } +} + diff --git a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala index b82699ebe4..65d7d7c23c 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala @@ -131,17 +131,15 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor { object DiningHakkers { val system = ActorSystem() - def main(args: Array[String]): Unit = { - run - } + def main(args: Array[String]): Unit = run def run { //Create 5 chopsticks - val chopsticks = for (i ← 1 to 5) yield system.actorOf(Props[Chopstick], "Chopstick " + i) + val chopsticks = for (i ← 1 to 5) yield system.actorOf(Props[Chopstick], "Chopstick" + i) //Create 5 awesome hakkers and assign them their left and right chopstick val hakkers = for { - (name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex + (name, i) ← List("Ghosh", "Boner", "Klang", "Krasser", "Manie").zipWithIndex } yield system.actorOf(Props(new Hakker(name, chopsticks(i), chopsticks((i + 1) % 5)))) //Signal all hakkers that they should start thinking, and watch the show diff --git a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala index 52ed49797a..7928a85334 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -169,16 +169,14 @@ object DiningHakkersOnFsm { val system = ActorSystem() - def main(args: Array[String]): Unit = { - run - } + def main(args: Array[String]): Unit = run def run = { // Create 5 chopsticks - val chopsticks = for (i ← 1 to 5) yield system.actorOf(Props[Chopstick], "Chopstick " + i) + val chopsticks = for (i ← 1 to 5) yield system.actorOf(Props[Chopstick], "Chopstick" + i) // Create 5 awesome fsm hakkers and assign them their left and right chopstick val hakkers = for { - (name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex + (name, i) ← List("Ghosh", "Boner", "Klang", "Krasser", "Manie").zipWithIndex } yield system.actorOf(Props(new FSMHakker(name, chopsticks(i), chopsticks((i + 1) % 5)))) hakkers.foreach(_ ! Think) diff --git a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala index 08826fa5dd..835a596a4a 100644 --- a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala +++ b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala @@ -75,7 +75,9 @@ object AkkaKernelPlugin extends Plugin { copyFiles(libFiles(cp, conf.libFilter), distLibPath) copyFiles(conf.additionalLibs, distLibPath) - for (subTarget ← subProjectDependencies.map(_.target)) { + for (subProjectDependency ← subProjectDependencies) { + val subTarget = subProjectDependency.target + EvaluateTask(buildStruct, packageBin in Compile, st, subProjectDependency.projectRef) copyJars(subTarget, distLibPath) } log.info("Distribution created.") @@ -220,10 +222,10 @@ object AkkaKernelPlugin extends Plugin { }.toList val target = setting(Keys.crossTarget, "Missing crossTarget directory") - SubProjectInfo(project.id, target, subProjects) + SubProjectInfo(projectRef, target, subProjects) } - private case class SubProjectInfo(id: String, target: File, subProjects: Seq[SubProjectInfo]) { + private case class SubProjectInfo(projectRef: ProjectRef, target: File, subProjects: Seq[SubProjectInfo]) { def recursiveSubProjects: Set[SubProjectInfo] = { val flatSubProjects = for { diff --git a/akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jEventHandler.scala b/akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jEventHandler.scala index 966f57b938..9e2fefffd9 100644 --- a/akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jEventHandler.scala +++ b/akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jEventHandler.scala @@ -18,12 +18,29 @@ trait SLF4JLogging { lazy val log = Logger(this.getClass.getName) } +/** + * Logger is a factory for obtaining SLF4J-Loggers + */ object Logger { + /** + * @param logger - which logger + * @return a Logger that corresponds for the given logger name + */ def apply(logger: String): SLFLogger = SLFLoggerFactory getLogger logger + + /** + * @param logClass - the class to log for + * @param logSource - the textual representation of the source of this log stream + * @return a Logger for the specified parameters + */ def apply(logClass: Class[_], logSource: String): SLFLogger = logClass match { case c if c == classOf[DummyClassForStringSources] ⇒ apply(logSource) case _ ⇒ SLFLoggerFactory getLogger logClass } + + /** + * Returns the SLF4J Root Logger + */ def root: SLFLogger = apply(SLFLogger.ROOT_LOGGER_NAME) } diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 2fe664d7b6..1732d5faf3 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -128,7 +128,7 @@ class CallingThreadDispatcher( override def id: String = Id - protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(actor, mailboxType) + protected[akka] override def createMailbox(actor: akka.actor.Cell) = new CallingThreadMailbox(actor, mailboxType) protected[akka] override def shutdown() {} @@ -281,17 +281,21 @@ class NestingQueue(val q: MessageQueue) { def isActive = active } -class CallingThreadMailbox(_receiver: ActorCell, val mailboxType: MailboxType) extends Mailbox(_receiver, null) with DefaultSystemMessageQueue { +class CallingThreadMailbox(_receiver: akka.actor.Cell, val mailboxType: MailboxType) + extends Mailbox(null) with DefaultSystemMessageQueue { + + val system = _receiver.system + val self = _receiver.self private val q = new ThreadLocal[NestingQueue]() { override def initialValue = { - val queue = new NestingQueue(mailboxType.create(Some(actor))) - CallingThreadDispatcherQueues(actor.system).registerQueue(CallingThreadMailbox.this, queue) + val queue = new NestingQueue(mailboxType.create(Some(self), Some(system))) + CallingThreadDispatcherQueues(system).registerQueue(CallingThreadMailbox.this, queue) queue } } - override def enqueue(receiver: ActorRef, msg: Envelope): Unit = throw new UnsupportedOperationException("CallingThreadMailbox cannot enqueue normally") + override def enqueue(receiver: ActorRef, msg: Envelope): Unit = q.get.q.enqueue(receiver, msg) override def dequeue(): Envelope = throw new UnsupportedOperationException("CallingThreadMailbox cannot dequeue normally") override def hasMessages: Boolean = q.get.q.hasMessages override def numberOfMessages: Int = 0 @@ -311,7 +315,7 @@ class CallingThreadMailbox(_receiver: ActorCell, val mailboxType: MailboxType) e val q = queue CallingThreadDispatcherQueues(actor.system).gatherFromAllOtherQueues(this, q) super.cleanUp() - q.q.cleanUp(actor, actor.systemImpl.deadLetterQueue) + q.q.cleanUp(actor.self, actor.systemImpl.deadLetterQueue) } } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 8a2f61bf76..73658cf985 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -5,9 +5,7 @@ package akka.testkit import akka.actor._ -import akka.util.Duration import java.util.concurrent.atomic.AtomicLong -import scala.collection.immutable.Stack import akka.dispatch._ import akka.pattern.ask @@ -31,18 +29,12 @@ class TestActorRef[T <: Actor]( if (_props.dispatcher == Dispatchers.DefaultDispatcherId) CallingThreadDispatcher.Id else _props.dispatcher), _supervisor, - _supervisor.path / name, - false) { + _supervisor.path / name) { import TestActorRef.InternalGetActor - override def newActorCell( - system: ActorSystemImpl, - ref: InternalActorRef, - props: Props, - supervisor: InternalActorRef, - receiveTimeout: Option[Duration]): ActorCell = - new ActorCell(system, ref, props, supervisor, receiveTimeout) { + override def newActorCell(system: ActorSystemImpl, ref: InternalActorRef, props: Props, supervisor: InternalActorRef): ActorCell = + new ActorCell(system, ref, props, supervisor) { override def autoReceiveMessage(msg: Envelope) { msg.message match { case InternalGetActor ⇒ sender ! actor @@ -56,7 +48,17 @@ class TestActorRef[T <: Actor]( * thrown will be available to you, while still being able to use * become/unbecome. */ - def receive(o: Any) { underlyingActor.apply(o) } + def receive(o: Any): Unit = receive(o, underlying.system.deadLetters) + + /** + * Directly inject messages into actor receive behavior. Any exceptions + * thrown will be available to you, while still being able to use + * become/unbecome. + */ + def receive(o: Any, sender: ActorRef): Unit = try { + underlying.currentMessage = Envelope(o, if (sender eq null) underlying.system.deadLetters else sender, underlying.system) + underlying.receiveMessage(o) + } finally underlying.currentMessage = null /** * Retrieve reference to the underlying actor, where the static type matches the factory used inside the diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index 156a9d8612..5f75ba8dfa 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -62,50 +62,28 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor { } /** - * Test kit for testing actors. Inheriting from this trait enables reception of - * replies from actors, which are queued by an internal actor and can be - * examined using the `expectMsg...` methods. Assertions and bounds concerning - * timing are available in the form of `within` blocks. + * Implementation trait behind the [[akka.testkit.TestKit]] class: you may use + * this if inheriting from a concrete class is not possible. * - *

- * class Test extends TestKit(ActorSystem()) {
- *     try {
+ * Use of the trait is discouraged because of potential issues with binary
+ * backwards compatibility in the future, use at own risk.
  *
- *       val test = system.actorOf(Props[SomeActor]
+ * This trait requires the concrete class mixing it in to provide an
+ * [[akka.actor.ActorSystem]] which is available before this traits’s
+ * constructor is run. The recommended way is this:
  *
- *       within (1 second) {
- *         test ! SomeWork
- *         expectMsg(Result1) // bounded to 1 second
- *         expectMsg(Result2) // bounded to the remainder of the 1 second
- *       }
- *
- *     } finally {
- *       system.shutdown()
- *     }
+ * {{{
+ * class MyTest extends TestKitBase {
+ *   implicit lazy val system = ActorSystem() // may add arguments here
+ *   ...
  * }
- * 
- * - * Beware of two points: - * - * - the ActorSystem passed into the constructor needs to be shutdown, - * otherwise thread pools and memory will be leaked - * - this trait is not thread-safe (only one actor with one queue, one stack - * of `within` blocks); it is expected that the code is executed from a - * constructor as shown above, which makes this a non-issue, otherwise take - * care not to run tests within a single test class instance in parallel. - * - * It should be noted that for CI servers and the like all maximum Durations - * are scaled using their Duration.dilated method, which uses the - * TestKitExtension.Settings.TestTimeFactor settable via akka.conf entry "akka.test.timefactor". - * - * @author Roland Kuhn - * @since 1.1 + * }}} */ -class TestKit(_system: ActorSystem) { +trait TestKitBase { import TestActor.{ Message, RealMessage, NullMessage } - implicit val system = _system + implicit val system: ActorSystem val testKitSettings = TestKitExtension(system) private val queue = new LinkedBlockingDeque[Message]() @@ -119,9 +97,14 @@ class TestKit(_system: ActorSystem) { */ lazy val testActor: ActorRef = { val impl = system.asInstanceOf[ActorSystemImpl] //TODO ticket #1559 - impl.systemActorOf(Props(new TestActor(queue)) + val ref = impl.systemActorOf(Props(new TestActor(queue)) .withDispatcher(CallingThreadDispatcher.Id), "testActor" + TestKit.testActorId.incrementAndGet) + awaitCond(ref match { + case r: RepointableRef ⇒ r.isStarted + case _ ⇒ true + }, 1 second, 10 millis) + ref } private var end: Duration = Duration.Undefined @@ -180,7 +163,13 @@ class TestKit(_system: ActorSystem) { * block or missing that it returns the properly dilated default for this * case from settings (key "akka.test.single-expect-default"). */ - def remaining: Duration = if (end == Duration.Undefined) testKitSettings.SingleExpectDefaultTimeout.dilated else end - now + def remaining: Duration = remainingOr(testKitSettings.SingleExpectDefaultTimeout.dilated) + + /** + * Obtain time remaining for execution of the innermost enclosing `within` + * block or missing that it returns the given duration. + */ + def remainingOr(duration: Duration): Duration = if (end == Duration.Undefined) duration else end - now /** * Query queue status. @@ -508,19 +497,21 @@ class TestKit(_system: ActorSystem) { @tailrec def doit(acc: List[T], count: Int): List[T] = { - if (count >= messages) return acc.reverse - receiveOne((stop - now) min idle) - lastMessage match { - case NullMessage ⇒ - lastMessage = msg - acc.reverse - case RealMessage(o, _) if (f isDefinedAt o) ⇒ - msg = lastMessage - doit(f(o) :: acc, count + 1) - case RealMessage(o, _) ⇒ - queue.offerFirst(lastMessage) - lastMessage = msg - acc.reverse + if (count >= messages) acc.reverse + else { + receiveOne((stop - now) min idle) + lastMessage match { + case NullMessage ⇒ + lastMessage = msg + acc.reverse + case RealMessage(o, _) if (f isDefinedAt o) ⇒ + msg = lastMessage + doit(f(o) :: acc, count + 1) + case RealMessage(o, _) ⇒ + queue.offerFirst(lastMessage) + lastMessage = msg + acc.reverse + } } } @@ -579,18 +570,54 @@ class TestKit(_system: ActorSystem) { private def format(u: TimeUnit, d: Duration) = "%.3f %s".format(d.toUnit(u), u.toString.toLowerCase) } +/** + * Test kit for testing actors. Inheriting from this trait enables reception of + * replies from actors, which are queued by an internal actor and can be + * examined using the `expectMsg...` methods. Assertions and bounds concerning + * timing are available in the form of `within` blocks. + * + *
+ * class Test extends TestKit(ActorSystem()) {
+ *     try {
+ *
+ *       val test = system.actorOf(Props[SomeActor]
+ *
+ *       within (1 second) {
+ *         test ! SomeWork
+ *         expectMsg(Result1) // bounded to 1 second
+ *         expectMsg(Result2) // bounded to the remainder of the 1 second
+ *       }
+ *
+ *     } finally {
+ *       system.shutdown()
+ *     }
+ * }
+ * 
+ * + * Beware of two points: + * + * - the ActorSystem passed into the constructor needs to be shutdown, + * otherwise thread pools and memory will be leaked + * - this trait is not thread-safe (only one actor with one queue, one stack + * of `within` blocks); it is expected that the code is executed from a + * constructor as shown above, which makes this a non-issue, otherwise take + * care not to run tests within a single test class instance in parallel. + * + * It should be noted that for CI servers and the like all maximum Durations + * are scaled using their Duration.dilated method, which uses the + * TestKitExtension.Settings.TestTimeFactor settable via akka.conf entry "akka.test.timefactor". + * + * @author Roland Kuhn + * @since 1.1 + */ +class TestKit(_system: ActorSystem) extends { implicit val system = _system } with TestKitBase + object TestKit { private[testkit] val testActorId = new AtomicInteger(0) /** * Await until the given condition evaluates to `true` or the timeout * expires, whichever comes first. - * - * If no timeout is given, take it from the innermost enclosing `within` - * block. - * - * Note that the timeout is scaled using Duration.dilated, which uses the - * configuration entry "akka.test.timefactor" */ def awaitCond(p: ⇒ Boolean, max: Duration, interval: Duration = 100.millis, noThrow: Boolean = false): Boolean = { val stop = now + max @@ -640,22 +667,23 @@ class TestProbe(_application: ActorSystem) extends TestKit(_application) { * Replies will be available for inspection with all of TestKit's assertion * methods. */ - def send(actor: ActorRef, msg: AnyRef) = { - actor.!(msg)(testActor) - } + def send(actor: ActorRef, msg: Any): Unit = actor.!(msg)(testActor) /** * Forward this message as if in the TestActor's receive method with self.forward. */ - def forward(actor: ActorRef, msg: AnyRef = lastMessage.msg) { - actor.!(msg)(lastMessage.sender) - } + def forward(actor: ActorRef, msg: Any = lastMessage.msg): Unit = actor.!(msg)(lastMessage.sender) /** * Get sender of last received message. */ def sender = lastMessage.sender + /** + * Send message to the sender of the last dequeued message. + */ + def reply(msg: Any): Unit = sender.!(msg)(ref) + } object TestProbe { diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index fd763e6bad..f381e53013 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -5,14 +5,13 @@ package akka.testkit import org.scalatest.{ WordSpec, BeforeAndAfterAll, Tag } import org.scalatest.matchers.MustMatchers -import akka.actor.{ ActorSystem, ActorSystemImpl } +import akka.actor.ActorSystem import akka.actor.{ Actor, ActorRef, Props } import akka.event.{ Logging, LoggingAdapter } import akka.util.duration._ import com.typesafe.config.Config import com.typesafe.config.ConfigFactory import akka.actor.PoisonPill -import akka.actor.CreateChild import akka.actor.DeadLetter import java.util.concurrent.TimeoutException import akka.dispatch.{ Await, MessageDispatcher } @@ -46,9 +45,13 @@ object AkkaSpec { ConfigFactory.parseMap(map.asJava) } - def getCallerName: String = { + def getCallerName(clazz: Class[_]): String = { val s = Thread.currentThread.getStackTrace map (_.getClassName) drop 1 dropWhile (_ matches ".*AkkaSpec.?$") - s.head.replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_") + val reduced = s.lastIndexWhere(_ == clazz.getName) match { + case -1 ⇒ s + case z ⇒ s drop (z + 1) + } + reduced.head.replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_") } } @@ -56,13 +59,14 @@ object AkkaSpec { abstract class AkkaSpec(_system: ActorSystem) extends TestKit(_system) with WordSpec with MustMatchers with BeforeAndAfterAll { - def this(config: Config) = this(ActorSystem(AkkaSpec.getCallerName, config.withFallback(AkkaSpec.testConf))) + def this(config: Config) = this(ActorSystem(AkkaSpec.getCallerName(getClass), + ConfigFactory.load(config.withFallback(AkkaSpec.testConf)))) def this(s: String) = this(ConfigFactory.parseString(s)) def this(configMap: Map[String, _]) = this(AkkaSpec.mapToConfig(configMap)) - def this() = this(ActorSystem(AkkaSpec.getCallerName, AkkaSpec.testConf)) + def this() = this(ActorSystem(AkkaSpec.getCallerName(getClass), AkkaSpec.testConf)) val log: LoggingAdapter = Logging(system, this.getClass) @@ -71,8 +75,9 @@ abstract class AkkaSpec(_system: ActorSystem) } final override def afterAll { + beforeShutdown() system.shutdown() - try Await.ready(system.asInstanceOf[ActorSystemImpl].terminationFuture, 5 seconds) catch { + try system.awaitTermination(5 seconds) catch { case _: TimeoutException ⇒ system.log.warning("Failed to stop [{}] within 5 seconds", system.name) } atTermination() @@ -80,6 +85,8 @@ abstract class AkkaSpec(_system: ActorSystem) protected def atStartup() {} + protected def beforeShutdown() {} + protected def atTermination() {} def spawn(dispatcherId: String = Dispatchers.DefaultDispatcherId)(body: ⇒ Unit) { @@ -111,9 +118,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers { "akka.actor.debug.lifecycle" -> true, "akka.actor.debug.event-stream" -> true, "akka.loglevel" -> "DEBUG", "akka.stdout-loglevel" -> "DEBUG") val system = ActorSystem("AkkaSpec1", ConfigFactory.parseMap(conf.asJava).withFallback(AkkaSpec.testConf)) - val spec = new AkkaSpec(system) { - val ref = Seq(testActor, system.actorOf(Props.empty, "name")) - } + val spec = new AkkaSpec(system) { val ref = Seq(testActor, system.actorOf(Props.empty, "name")) } spec.ref foreach (_.isTerminated must not be true) system.shutdown() spec.awaitCond(spec.ref forall (_.isTerminated), 2 seconds) diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 7c977884fc..492c44408c 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -246,11 +246,18 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA a.underlying.dispatcher.getClass must be(classOf[Dispatcher]) } - "proxy receive for the underlying actor" in { + "proxy receive for the underlying actor without sender" in { val ref = TestActorRef[WorkerActor] ref.receive("work") ref.isTerminated must be(true) } + "proxy receive for the underlying actor with sender" in { + val ref = TestActorRef[WorkerActor] + ref.receive("work", testActor) + ref.isTerminated must be(true) + expectMsg("workDone") + } + } } diff --git a/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala b/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala index 2463f0e436..792824be24 100644 --- a/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala +++ b/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala @@ -12,19 +12,29 @@ import java.util.concurrent.Callable /** * Akka-specific exception for coordinated transactions. */ -class CoordinatedTransactionException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { - def this(msg: String) = this(msg, null); +class CoordinatedTransactionException(message: String, cause: Throwable) extends AkkaException(message, cause) { + def this(msg: String) = this(msg, null) } /** * Coordinated transactions across actors. */ object Coordinated { - def apply(message: Any = null)(implicit timeout: Timeout) = new Coordinated(message, createInitialMember(timeout)) + /** + * Creates a new Coordinated with the given message and Timeout + * @param message - the message which will be coordinated + * @param timeout - the timeout for the coordination + * @return a new Coordinated + */ + def apply(message: Any = null)(implicit timeout: Timeout): Coordinated = + new Coordinated(message, CommitBarrier(timeout.duration.toMillis).addMember()) + + /** + * @param c - a Coordinated to be unapplied + * @return the message associated with the given Coordinated + */ def unapply(c: Coordinated): Option[Any] = Some(c.message) - - def createInitialMember(timeout: Timeout) = CommitBarrier(timeout.duration.toMillis).addMember() } /** @@ -91,16 +101,15 @@ class Coordinated(val message: Any, member: CommitBarrier.Member) { // Java API constructors - def this(message: Any, timeout: Timeout) = this(message, Coordinated.createInitialMember(timeout)) + def this(message: Any, timeout: Timeout) = this(message, CommitBarrier(timeout.duration.toMillis).addMember()) - def this(timeout: Timeout) = this(null, Coordinated.createInitialMember(timeout)) + def this(timeout: Timeout) = this(null, timeout) /** * Create a new Coordinated object and increment the number of members by one. * Use this method to ''pass on'' the coordination. */ - def apply(msg: Any): Coordinated = - new Coordinated(msg, member.commitBarrier.addMember()) + def apply(msg: Any): Coordinated = new Coordinated(msg, member.commitBarrier.addMember()) /** * Create a new Coordinated object but *do not* increment the number of members by one. diff --git a/akka-transactor/src/main/scala/akka/transactor/Transactor.scala b/akka-transactor/src/main/scala/akka/transactor/Transactor.scala index 6e390a6623..fd802e1f21 100644 --- a/akka-transactor/src/main/scala/akka/transactor/Transactor.scala +++ b/akka-transactor/src/main/scala/akka/transactor/Transactor.scala @@ -176,8 +176,10 @@ trait Transactor extends Actor { /** * Default catch-all for the different Receive methods. */ - def doNothing: Receive = new Receive { - def apply(any: Any) = {} - def isDefinedAt(any: Any) = false - } + def doNothing: Receive = EmptyReceive +} + +private[akka] object EmptyReceive extends PartialFunction[Any, Unit] { + def apply(any: Any): Unit = () + def isDefinedAt(any: Any): Boolean = false } diff --git a/akka-transactor/src/main/scala/akka/transactor/TransactorExtension.scala b/akka-transactor/src/main/scala/akka/transactor/TransactorExtension.scala index 96aea8904c..85cb8c46fd 100644 --- a/akka-transactor/src/main/scala/akka/transactor/TransactorExtension.scala +++ b/akka-transactor/src/main/scala/akka/transactor/TransactorExtension.scala @@ -15,11 +15,11 @@ import java.util.concurrent.TimeUnit.MILLISECONDS */ object TransactorExtension extends ExtensionId[TransactorSettings] with ExtensionIdProvider { override def get(system: ActorSystem): TransactorSettings = super.get(system) - override def lookup = TransactorExtension + override def lookup: TransactorExtension.type = TransactorExtension override def createExtension(system: ExtendedActorSystem): TransactorSettings = new TransactorSettings(system.settings.config) } class TransactorSettings(val config: Config) extends Extension { import config._ - val CoordinatedTimeout = Timeout(Duration(getMilliseconds("akka.transactor.coordinated-timeout"), MILLISECONDS)) + val CoordinatedTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.transactor.coordinated-timeout"), MILLISECONDS)) } \ No newline at end of file diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java index 60a887f554..36c063feaa 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java @@ -57,7 +57,7 @@ public class UntypedCoordinatedIncrementTest { Timeout timeout = new Timeout(timeoutSeconds, TimeUnit.SECONDS); @Before - public void initialise() { + public void initialize() { counters = new ArrayList(); for (int i = 1; i <= numCounters; i++) { final String name = "counter" + i; diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java index cadc4828b1..b7dc99389a 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java @@ -58,7 +58,7 @@ public class UntypedTransactorTest { Timeout timeout = new Timeout(timeoutSeconds, TimeUnit.SECONDS); @Before - public void initialise() { + public void initialize() { counters = new ArrayList(); for (int i = 1; i <= numCounters; i++) { final String name = "counter" + i; diff --git a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala index 9c019a56a5..c76a5a701c 100644 --- a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala @@ -12,7 +12,7 @@ import akka.util.duration._ import akka.util.Timeout import akka.testkit._ import scala.concurrent.stm._ -import akka.pattern.ask +import akka.pattern.{ AskTimeoutException, ask } object CoordinatedIncrement { @@ -96,7 +96,7 @@ class CoordinatedIncrementSpec extends AkkaSpec(CoordinatedIncrement.config) wit val ignoreExceptions = Seq( EventFilter[ExpectedFailureException](), EventFilter[CoordinatedTransactionException](), - EventFilter[ActorTimeoutException]()) + EventFilter[AskTimeoutException]()) filterEvents(ignoreExceptions) { val (counters, failer) = actorOfs val coordinated = Coordinated() diff --git a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala index 4f7fc89c14..9deee7b9cc 100644 --- a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala @@ -15,7 +15,7 @@ import akka.testkit.TestEvent.Mute import scala.concurrent.stm._ import scala.util.Random.{ nextInt ⇒ random } import java.util.concurrent.CountDownLatch -import akka.pattern.ask +import akka.pattern.{ AskTimeoutException, ask } object FickleFriends { case class FriendlyIncrement(friends: Seq[ActorRef], timeout: Timeout, latch: CountDownLatch) @@ -120,7 +120,7 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll { val ignoreExceptions = Seq( EventFilter[ExpectedFailureException](), EventFilter[CoordinatedTransactionException](), - EventFilter[ActorTimeoutException]()) + EventFilter[AskTimeoutException]()) system.eventStream.publish(Mute(ignoreExceptions)) val (counters, coordinator) = actorOfs val latch = new CountDownLatch(1) diff --git a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala index 1954c9a13b..df9723ffd2 100644 --- a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala @@ -10,7 +10,7 @@ import akka.util.duration._ import akka.util.Timeout import akka.testkit._ import scala.concurrent.stm._ -import akka.pattern.ask +import akka.pattern.{ AskTimeoutException, ask } object TransactorIncrement { case class Increment(friends: Seq[ActorRef], latch: TestLatch) @@ -105,7 +105,7 @@ class TransactorSpec extends AkkaSpec { val ignoreExceptions = Seq( EventFilter[ExpectedFailureException](), EventFilter[CoordinatedTransactionException](), - EventFilter[ActorTimeoutException]()) + EventFilter[AskTimeoutException]()) filterEvents(ignoreExceptions) { val (counters, failer) = createTransactors val failLatch = TestLatch(numCounters) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index c4e6d08f59..e1b1ba4ddf 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -9,14 +9,17 @@ import akka.actor._ import akka.dispatch.{ Promise, Future } import akka.event.Logging import annotation.tailrec -import akka.util.Duration import java.util.concurrent.TimeUnit +import collection.mutable.ListBuffer +import akka.util.{ NonFatal, Duration } private[zeromq] object ConcurrentSocketActor { private sealed trait PollMsg private case object Poll extends PollMsg private case object PollCareful extends PollMsg + private case object Flush + private class NoSocketHandleException() extends Exception("Couldn't create a zeromq socket.") private val DefaultContext = Context() @@ -32,19 +35,28 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A import SocketType.{ ZMQSocketType ⇒ ST } params.collectFirst { case t: ST ⇒ t }.getOrElse(throw new IllegalArgumentException("A socket type is required")) } + private val socket: Socket = zmqContext.socket(socketType) private val poller: Poller = zmqContext.poller private val log = Logging(context.system, this) + private val pendingSends = new ListBuffer[Seq[Frame]] + def receive = { case m: PollMsg ⇒ doPoll(m) - case ZMQMessage(frames) ⇒ sendMessage(frames) + case ZMQMessage(frames) ⇒ handleRequest(Send(frames)) case r: Request ⇒ handleRequest(r) + case Flush ⇒ flush() case Terminated(_) ⇒ context stop self } private def handleRequest(msg: Request): Unit = msg match { - case Send(frames) ⇒ sendMessage(frames) + case Send(frames) ⇒ + if (frames.nonEmpty) { + val flushNow = pendingSends.isEmpty + pendingSends.append(frames) + if (flushNow) flush() + } case opt: SocketOption ⇒ handleSocketOption(opt) case q: SocketOptionQuery ⇒ handleSocketOptionQuery(q) } @@ -117,48 +129,46 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A } } - private def setupConnection() { + private def setupConnection(): Unit = { params filter (_.isInstanceOf[SocketConnectOption]) foreach { self ! _ } params filter (_.isInstanceOf[PubSubOption]) foreach { self ! _ } } - private def deserializerFromParams = { + private def deserializerFromParams: Deserializer = params collectFirst { case d: Deserializer ⇒ d } getOrElse new ZMQMessageDeserializer + + private def setupSocket() = params foreach { + case _: SocketConnectOption | _: PubSubOption | _: SocketMeta ⇒ // ignore, handled differently + case m ⇒ self ! m } - private def setupSocket() = { - params foreach { - case _: SocketConnectOption | _: PubSubOption | _: SocketMeta ⇒ // ignore, handled differently - case m ⇒ self ! m + override def preRestart(reason: Throwable, message: Option[Any]): Unit = context.children foreach context.stop //Do not call postStop + + override def postRestart(reason: Throwable): Unit = () // Do nothing + + override def postStop: Unit = try { + if (socket != null) { + poller.unregister(socket) + socket.close } - } + } finally notifyListener(Closed) - override def preRestart(reason: Throwable, message: Option[Any]) { - context.children foreach context.stop //Do not call postStop - } - - override def postRestart(reason: Throwable) {} //Do nothing - - override def postStop { - try { - if (socket != null) { - poller.unregister(socket) - socket.close + @tailrec private def flushMessage(i: Seq[Frame]): Boolean = + if (i.isEmpty) + true + else { + val head = i.head + val tail = i.tail + if (socket.send(head.payload.toArray, if (tail.nonEmpty) JZMQ.SNDMORE else 0)) flushMessage(tail) + else { + pendingSends.prepend(i) // Reenqueue the rest of the message so the next flush takes care of it + self ! Flush + false } - } finally { - notifyListener(Closed) } - } - private def sendMessage(frames: Seq[Frame]) { - def sendBytes(bytes: Seq[Byte], flags: Int) = socket.send(bytes.toArray, flags) - val iter = frames.iterator - while (iter.hasNext) { - val payload = iter.next.payload - val flags = if (iter.hasNext) JZMQ.SNDMORE else 0 - sendBytes(payload, flags) - } - } + @tailrec private def flush(): Unit = + if (pendingSends.nonEmpty && flushMessage(pendingSends.remove(0))) flush() // Flush while things are going well // this is a “PollMsg=>Unit” which either polls or schedules Poll, depending on the sign of the timeout private val doPollTimeout = { @@ -180,36 +190,26 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A } @tailrec private def doPoll(mode: PollMsg, togo: Int = 10): Unit = - receiveMessage(mode) match { - case null ⇒ // receiveMessage has already done something special here - case Seq() ⇒ doPollTimeout(mode) - case frames ⇒ - notifyListener(deserializer(frames)) - if (togo > 0) doPoll(mode, togo - 1) - else self ! mode + if (togo <= 0) self ! mode + else receiveMessage(mode) match { + case Seq() ⇒ doPollTimeout(mode) + case frames ⇒ notifyListener(deserializer(frames)); doPoll(mode, togo - 1) } - @tailrec private def receiveMessage(mode: PollMsg, currentFrames: Vector[Frame] = Vector.empty): Seq[Frame] = { - val result = mode match { - case Poll ⇒ socket.recv(JZMQ.NOBLOCK) - case PollCareful ⇒ if (poller.poll(0) > 0) socket.recv(0) else null + @tailrec private def receiveMessage(mode: PollMsg, currentFrames: Vector[Frame] = Vector.empty): Seq[Frame] = + if (mode == PollCareful && (poller.poll(0) <= 0)) { + if (currentFrames.isEmpty) currentFrames else throw new IllegalStateException("Received partial transmission!") + } else { + socket.recv(if (mode == Poll) JZMQ.NOBLOCK else 0) match { + case null ⇒ /*EAGAIN*/ + if (currentFrames.isEmpty) currentFrames else receiveMessage(mode, currentFrames) + case bytes ⇒ + val frames = currentFrames :+ Frame(if (bytes.length == 0) noBytes else bytes) + if (socket.hasReceiveMore) receiveMessage(mode, frames) else frames + } } - result match { - case null ⇒ - if (currentFrames.isEmpty) currentFrames - else throw new IllegalStateException("no more frames available while socket.hasReceivedMore==true") - case bytes ⇒ - val frames = currentFrames :+ Frame(if (bytes.length == 0) noBytes else bytes) - if (socket.hasReceiveMore) receiveMessage(mode, frames) else frames - } - } private val listenerOpt = params collectFirst { case Listener(l) ⇒ l } - private def watchListener() { - listenerOpt foreach context.watch - } - - private def notifyListener(message: Any) { - listenerOpt foreach { _ ! message } - } + private def watchListener(): Unit = listenerOpt foreach context.watch + private def notifyListener(message: Any): Unit = listenerOpt foreach { _ ! message } } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala index 1e4c83bcef..c5d5919fb7 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala @@ -255,7 +255,9 @@ case class Linger(value: Long) extends SocketOption /** * Gets the linger option @see [[akka.zeromq.Linger]] */ -object Linger extends SocketOptionQuery +object Linger extends SocketOptionQuery { + val no: Linger = Linger(0) +} /** * Sets the recovery interval for multicast transports using the specified socket. diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala index 1776f21211..2d41424e88 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala @@ -20,5 +20,5 @@ case class Frame(payload: Seq[Byte]) { * Deserializes ZeroMQ messages into an immutable sequence of frames */ class ZMQMessageDeserializer extends Deserializer { - def apply(frames: Seq[Frame]) = ZMQMessage(frames) + def apply(frames: Seq[Frame]): ZMQMessage = ZMQMessage(frames) } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 1ddd213325..4bf52a41e3 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -19,7 +19,7 @@ import org.zeromq.ZMQException * @param patch */ case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { - override def toString = "%d.%d.%d".format(major, minor, patch) + override def toString: String = "%d.%d.%d".format(major, minor, patch) } /** @@ -27,17 +27,14 @@ case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { */ object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProvider { override def get(system: ActorSystem): ZeroMQExtension = super.get(system) - def lookup() = this - def createExtension(system: ExtendedActorSystem) = new ZeroMQExtension(system) + def lookup(): this.type = this + override def createExtension(system: ExtendedActorSystem): ZeroMQExtension = new ZeroMQExtension(system) private val minVersionString = "2.1.0" private val minVersion = JZMQ.makeVersion(2, 1, 0) - private[zeromq] def check[TOption <: SocketOption: Manifest](parameters: Seq[SocketOption]) = { - parameters exists { p ⇒ - ClassManifest.singleType(p) <:< manifest[TOption] - } - } + private[zeromq] def check[TOption <: SocketOption: Manifest](parameters: Seq[SocketOption]) = + parameters exists { p ⇒ ClassManifest.singleType(p) <:< manifest[TOption] } } /** @@ -47,16 +44,14 @@ object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProv */ class ZeroMQExtension(system: ActorSystem) extends Extension { - val DefaultPollTimeout = Duration(system.settings.config.getMilliseconds("akka.zeromq.poll-timeout"), TimeUnit.MILLISECONDS) - val NewSocketTimeout = Timeout(Duration(system.settings.config.getMilliseconds("akka.zeromq.new-socket-timeout"), TimeUnit.MILLISECONDS)) + val DefaultPollTimeout: Duration = Duration(system.settings.config.getMilliseconds("akka.zeromq.poll-timeout"), TimeUnit.MILLISECONDS) + val NewSocketTimeout: Timeout = Timeout(Duration(system.settings.config.getMilliseconds("akka.zeromq.new-socket-timeout"), TimeUnit.MILLISECONDS)) /** * The version of the ZeroMQ library * @return a [[akka.zeromq.ZeroMQVersion]] */ - def version = { - ZeroMQVersion(JZMQ.getMajorVersion, JZMQ.getMinorVersion, JZMQ.getPatchVersion) - } + def version: ZeroMQVersion = ZeroMQVersion(JZMQ.getMajorVersion, JZMQ.getMinorVersion, JZMQ.getPatchVersion) /** * Factory method to create the [[akka.actor.Props]] to build the ZeroMQ socket actor. @@ -144,8 +139,7 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { */ def newSocket(socketParameters: SocketOption*): ActorRef = { implicit val timeout = NewSocketTimeout - val req = (zeromqGuardian ? newSocketProps(socketParameters: _*)).mapTo[ActorRef] - Await.result(req, timeout.duration) + Await.result((zeromqGuardian ? newSocketProps(socketParameters: _*)).mapTo[ActorRef], timeout.duration) } /** @@ -253,9 +247,7 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { case _ ⇒ false } - def receive = { - case p: Props ⇒ sender ! context.actorOf(p) - } + def receive = { case p: Props ⇒ sender ! context.actorOf(p) } }), "zeromq") } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/package.scala b/akka-zeromq/src/main/scala/akka/zeromq/package.scala index 6eeba5b92a..1241700fcb 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/package.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/package.scala @@ -20,10 +20,10 @@ package object zeromq { /** * Convenience accessor to subscribe to all events */ - val SubscribeAll = Subscribe(Seq.empty) + val SubscribeAll: Subscribe = Subscribe.all /** * Set the linger to 0, doesn't block and discards messages that haven't been sent yet. */ - val NoLinger = Linger(0) + val NoLinger: Linger = Linger.no } \ No newline at end of file diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index dcc456b544..e075ca2158 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.zeromq @@ -138,7 +138,7 @@ class ConcurrentSocketActorSpec extends AkkaSpec { } } - protected def receive = { + def receive = { case _ ⇒ val payload = "%s".format(messageNumber) messageNumber += 1 diff --git a/ls.sbt b/ls.sbt index 83e5babc79..87e68ed303 100644 --- a/ls.sbt +++ b/ls.sbt @@ -1,13 +1,13 @@ -seq(lsSettings:_*) +// seq(lsSettings:_*) -(description in LsKeys.lsync) := "Akka is the platform for the next generation of event-driven, scalable and fault-tolerant architectures on the JVM." +// (description in LsKeys.lsync) := "Akka is the platform for the next generation of event-driven, scalable and fault-tolerant architectures on the JVM." -(homepage in LsKeys.lsync) := Some(url("http://akka.io")) +// (homepage in LsKeys.lsync) := Some(url("http://akka.io")) -(LsKeys.tags in LsKeys.lsync) := Seq("actors", "stm", "concurrency", "distributed", "fault-tolerance", "scala", "java", "futures", "dataflow", "remoting") +// (LsKeys.tags in LsKeys.lsync) := Seq("actors", "stm", "concurrency", "distributed", "fault-tolerance", "scala", "java", "futures", "dataflow", "remoting") -(LsKeys.docsUrl in LsKeys.lsync) := Some(url("http://akka.io/docs")) +// (LsKeys.docsUrl in LsKeys.lsync) := Some(url("http://akka.io/docs")) -(licenses in LsKeys.lsync) := Seq(("Apache 2", url("http://www.apache.org/licenses/LICENSE-2.0.html"))) +// (licenses in LsKeys.lsync) := Seq(("Apache 2", url("http://www.apache.org/licenses/LICENSE-2.0.html"))) -(externalResolvers in LsKeys.lsync) := Seq("Typesafe Releases" at "http://repo.typesafe.com/typesafe/releases") +// (externalResolvers in LsKeys.lsync) := Seq("Typesafe Releases" at "http://repo.typesafe.com/typesafe/releases") diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 4804c0f796..7b83364ed0 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -7,11 +7,14 @@ package akka import sbt._ import sbt.Keys._ import com.typesafe.sbtmultijvm.MultiJvmPlugin -import com.typesafe.sbtmultijvm.MultiJvmPlugin.{ MultiJvm, extraOptions, jvmOptions, scalatestOptions } -import com.typesafe.schoir.SchoirPlugin.schoirSettings +import com.typesafe.sbtmultijvm.MultiJvmPlugin.{ MultiJvm, extraOptions, jvmOptions, scalatestOptions, multiNodeExecuteTests } import com.typesafe.sbtscalariform.ScalariformPlugin import com.typesafe.sbtscalariform.ScalariformPlugin.ScalariformKeys +import com.typesafe.sbtosgi.OsgiPlugin.{ OsgiKeys, osgiSettings } +import com.typesafe.tools.mima.plugin.MimaPlugin.mimaDefaultSettings +import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact import java.lang.Boolean.getBoolean +import sbt.Tests import Sphinx.{ sphinxDocs, sphinxHtml, sphinxLatex, sphinxPdf, sphinxPygments, sphinxTags } object AkkaBuild extends Build { @@ -26,7 +29,8 @@ object AkkaBuild extends Build { lazy val akka = Project( id = "akka", base = file("."), - settings = parentSettings ++ Release.settings ++ Unidoc.settings ++ Sphinx.settings ++ Publish.versionSettings ++ Dist.settings ++ Seq( + settings = parentSettings ++ Release.settings ++ Unidoc.settings ++ Sphinx.settings ++ Publish.versionSettings ++ + Dist.settings ++ mimaSettings ++ Seq( testMailbox in GlobalScope := System.getProperty("akka.testMailbox", "false").toBoolean, parallelExecution in GlobalScope := System.getProperty("akka.parallelExecution", "false").toBoolean, Publish.defaultPublishTo in ThisBuild <<= crossTarget / "repository", @@ -39,18 +43,22 @@ object AkkaBuild extends Build { sphinxLatex <<= sphinxLatex in LocalProject(docs.id), sphinxPdf <<= sphinxPdf in LocalProject(docs.id) ), - aggregate = Seq(actor, testkit, actorTests, remote, camel, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, samples, tutorials, docs) + aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, samples, tutorials, osgi, osgiAries, docs) ) lazy val actor = Project( id = "akka-actor", base = file("akka-actor"), - settings = defaultSettings ++ Seq( + settings = defaultSettings ++ OSGi.actor ++ Seq( autoCompilerPlugins := true, libraryDependencies <+= scalaVersion { v => compilerPlugin("org.scala-lang.plugins" % "continuations" % v) }, scalacOptions += "-P:continuations:enable", + packagedArtifact in (Compile, packageBin) <<= (artifact in (Compile, packageBin), OsgiKeys.bundle).identityMap, + artifact in (Compile, packageBin) ~= (_.copy(`type` = "bundle")), // to fix scaladoc generation - fullClasspath in doc in Compile <<= fullClasspath in Compile + fullClasspath in doc in Compile <<= fullClasspath in Compile, + libraryDependencies ++= Dependencies.actor, + previousArtifact := akkaPreviousArtifact("akka-actor") ) ) @@ -59,7 +67,8 @@ object AkkaBuild extends Build { base = file("akka-testkit"), dependencies = Seq(actor), settings = defaultSettings ++ Seq( - libraryDependencies ++= Dependencies.testkit + libraryDependencies ++= Dependencies.testkit, + previousArtifact := akkaPreviousArtifact("akka-testkit") ) ) @@ -79,37 +88,43 @@ object AkkaBuild extends Build { id = "akka-remote", base = file("akka-remote"), dependencies = Seq(actor, actorTests % "test->test", testkit % "test->test"), - settings = defaultSettings ++ multiJvmSettings ++ schoirSettings ++ Seq( + settings = defaultSettings ++ OSGi.remote ++ Seq( libraryDependencies ++= Dependencies.remote, + // disable parallel tests + parallelExecution in Test := false + ) + ) + + lazy val remoteTests = Project( + id = "akka-remote-tests", + base = file("akka-remote-tests"), + dependencies = Seq(remote, actorTests % "test->test", testkit % "test->test"), + settings = defaultSettings ++ multiJvmSettings ++ Seq( // disable parallel tests parallelExecution in Test := false, extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => (name: String) => (src ** (name + ".conf")).get.headOption.map("-Dakka.config=" + _.absolutePath).toSeq }, - scalatestOptions in MultiJvm := Seq("-r", "org.scalatest.akka.QuietReporter"), - jvmOptions in MultiJvm := { - if (getBoolean("sbt.log.noformat")) Seq("-Dakka.test.nocolor=true") else Nil - }, - test in Test <<= (test in Test) dependsOn (test in MultiJvm) + scalatestOptions in MultiJvm := defaultMultiJvmScalatestOptions, + jvmOptions in MultiJvm := defaultMultiJvmOptions, + previousArtifact := akkaPreviousArtifact("akka-remote") ) ) configs (MultiJvm) lazy val cluster = Project( id = "akka-cluster", base = file("akka-cluster"), - dependencies = Seq(remote, remote % "test->test", testkit % "test->test"), - settings = defaultSettings ++ multiJvmSettings ++ schoirSettings ++ Seq( + dependencies = Seq(remote, remoteTests % "compile;test->test;multi-jvm->multi-jvm", testkit % "test->test"), + settings = defaultSettings ++ multiJvmSettings ++ OSGi.cluster ++ Seq( libraryDependencies ++= Dependencies.cluster, // disable parallel tests parallelExecution in Test := false, extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => (name: String) => (src ** (name + ".conf")).get.headOption.map("-Dakka.config=" + _.absolutePath).toSeq }, - scalatestOptions in MultiJvm := Seq("-r", "org.scalatest.akka.QuietReporter"), - jvmOptions in MultiJvm := { - if (getBoolean("sbt.log.noformat")) Seq("-Dakka.test.nocolor=true") else Nil - }, - test in Test <<= (test in Test) dependsOn (test in MultiJvm) + scalatestOptions in MultiJvm := defaultMultiJvmScalatestOptions, + jvmOptions in MultiJvm := defaultMultiJvmOptions, + previousArtifact := akkaPreviousArtifact("akka-remote") ) ) configs (MultiJvm) @@ -117,7 +132,7 @@ object AkkaBuild extends Build { id = "akka-slf4j", base = file("akka-slf4j"), dependencies = Seq(actor, testkit % "test->test"), - settings = defaultSettings ++ Seq( + settings = defaultSettings ++ OSGi.slf4j ++ Seq( libraryDependencies ++= Dependencies.slf4j ) ) @@ -126,8 +141,9 @@ object AkkaBuild extends Build { id = "akka-agent", base = file("akka-agent"), dependencies = Seq(actor, testkit % "test->test"), - settings = defaultSettings ++ Seq( - libraryDependencies ++= Dependencies.agent + settings = defaultSettings ++ OSGi.agent ++ Seq( + libraryDependencies ++= Dependencies.agent, + previousArtifact := akkaPreviousArtifact("akka-agent") ) ) @@ -135,8 +151,9 @@ object AkkaBuild extends Build { id = "akka-transactor", base = file("akka-transactor"), dependencies = Seq(actor, testkit % "test->test"), - settings = defaultSettings ++ Seq( - libraryDependencies ++= Dependencies.transactor + settings = defaultSettings ++ OSGi.transactor ++ Seq( + libraryDependencies ++= Dependencies.transactor, + previousArtifact := akkaPreviousArtifact("akka-transactor") ) ) @@ -153,8 +170,11 @@ object AkkaBuild extends Build { id = "akka-mailboxes-common", base = file("akka-durable-mailboxes/akka-mailboxes-common"), dependencies = Seq(remote, testkit % "compile;test->test"), - settings = defaultSettings ++ Seq( - libraryDependencies ++= Dependencies.mailboxes + settings = defaultSettings ++ OSGi.mailboxesCommon ++ Seq( + libraryDependencies ++= Dependencies.mailboxes, + previousArtifact := akkaPreviousArtifact("akka-mailboxes-common"), + // DurableMailboxSpec published in akka-mailboxes-common-test + publishArtifact in Test := true ) ) @@ -162,8 +182,9 @@ object AkkaBuild extends Build { id = "akka-file-mailbox", base = file("akka-durable-mailboxes/akka-file-mailbox"), dependencies = Seq(mailboxesCommon % "compile;test->test", testkit % "test"), - settings = defaultSettings ++ Seq( - libraryDependencies ++= Dependencies.fileMailbox + settings = defaultSettings ++ OSGi.fileMailbox ++ Seq( + libraryDependencies ++= Dependencies.fileMailbox, + previousArtifact := akkaPreviousArtifact("akka-file-mailbox") ) ) @@ -171,8 +192,9 @@ object AkkaBuild extends Build { id = "akka-zeromq", base = file("akka-zeromq"), dependencies = Seq(actor, testkit % "test;test->test"), - settings = defaultSettings ++ Seq( - libraryDependencies ++= Dependencies.zeroMQ + settings = defaultSettings ++ OSGi.zeroMQ ++ Seq( + libraryDependencies ++= Dependencies.zeroMQ, + previousArtifact := akkaPreviousArtifact("akka-zeromq") ) ) @@ -181,7 +203,8 @@ object AkkaBuild extends Build { base = file("akka-kernel"), dependencies = Seq(actor, testkit % "test->test"), settings = defaultSettings ++ Seq( - libraryDependencies ++= Dependencies.kernel + libraryDependencies ++= Dependencies.kernel, + previousArtifact := akkaPreviousArtifact("akka-kernel") ) ) @@ -189,11 +212,29 @@ object AkkaBuild extends Build { id = "akka-camel", base = file("akka-camel"), dependencies = Seq(actor, slf4j, testkit % "test->test"), - settings = defaultSettings ++ Seq( + settings = defaultSettings ++ OSGi.camel ++ Seq( libraryDependencies ++= Dependencies.camel ) ) + lazy val osgi = Project( + id = "akka-osgi", + base = file("akka-osgi"), + dependencies = Seq(actor), + settings = defaultSettings ++ OSGi.osgi ++ Seq( + libraryDependencies ++= Dependencies.osgi + ) + ) + + lazy val osgiAries = Project( + id = "akka-osgi-aries", + base = file("akka-osgi-aries"), + dependencies = Seq(osgi % "compile;test->test"), + settings = defaultSettings ++ OSGi.osgiAries ++ Seq( + libraryDependencies ++= Dependencies.osgiAries + ) + ) + lazy val akkaSbtPlugin = Project( id = "akka-sbt-plugin", base = file("akka-sbt-plugin"), @@ -257,7 +298,8 @@ object AkkaBuild extends Build { lazy val docs = Project( id = "akka-docs", base = file("akka-docs"), - dependencies = Seq(actor, testkit % "test->test", remote, cluster, slf4j, agent, transactor, fileMailbox, zeroMQ, camel), + dependencies = Seq(actor, testkit % "test->test", mailboxesCommon % "compile;test->test", + remote, cluster, slf4j, agent, transactor, fileMailbox, zeroMQ, camel, osgi, osgiAries), settings = defaultSettings ++ Sphinx.settings ++ Seq( unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get }, libraryDependencies ++= Dependencies.docs, @@ -279,12 +321,61 @@ object AkkaBuild extends Build { ) val excludeTestNames = SettingKey[Seq[String]]("exclude-test-names") - val excludeTestTags = SettingKey[Seq[String]]("exclude-test-tags") - val includeTestTags = SettingKey[Seq[String]]("include-test-tags") + val excludeTestTags = SettingKey[Set[String]]("exclude-test-tags") + val includeTestTags = SettingKey[Set[String]]("include-test-tags") + val onlyTestTags = SettingKey[Set[String]]("only-test-tags") - val defaultExcludedTags = Seq("timing", "long-running") + val defaultExcludedTags = Set("timing", "long-running") - lazy val defaultSettings = baseSettings ++ formatSettings ++ Seq( + lazy val defaultMultiJvmOptions: Seq[String] = { + import scala.collection.JavaConverters._ + val akkaProperties = System.getProperties.propertyNames.asScala.toList.collect { + case key: String if key.startsWith("akka.") => "-D" + key + "=" + System.getProperty(key) + } + akkaProperties ::: (if (getBoolean("sbt.log.noformat")) List("-Dakka.test.nocolor=true") else Nil) + } + + // for excluding tests by name use system property: -Dakka.test.names.exclude=TimingSpec + // not supported by multi-jvm tests + lazy val useExcludeTestNames: Seq[String] = systemPropertyAsSeq("akka.test.names.exclude") + + // for excluding tests by tag use system property: -Dakka.test.tags.exclude= + // note that it will not be used if you specify -Dakka.test.tags.only + lazy val useExcludeTestTags: Set[String] = { + if (useOnlyTestTags.isEmpty) defaultExcludedTags ++ systemPropertyAsSeq("akka.test.tags.exclude").toSet + else Set.empty + } + + // for including tests by tag use system property: -Dakka.test.tags.include= + // note that it will not be used if you specify -Dakka.test.tags.only + lazy val useIncludeTestTags: Set[String] = { + if (useOnlyTestTags.isEmpty) systemPropertyAsSeq("akka.test.tags.include").toSet + else Set.empty + } + + // for running only tests by tag use system property: -Dakka.test.tags.only= + lazy val useOnlyTestTags: Set[String] = systemPropertyAsSeq("akka.test.tags.only").toSet + + def executeMultiJvmTests: Boolean = { + useOnlyTestTags.contains("long-running") || + !(useExcludeTestTags -- useIncludeTestTags).contains("long-running") + } + + def systemPropertyAsSeq(name: String): Seq[String] = { + val prop = System.getProperty(name, "") + if (prop.isEmpty) Seq.empty else prop.split(",").toSeq + } + + val multiNodeEnabled = java.lang.Boolean.getBoolean("akka.test.multi-node") + + lazy val defaultMultiJvmScalatestOptions: Seq[String] = { + val excludeTags = (useExcludeTestTags -- useIncludeTestTags).toSeq + Seq("-r", "org.scalatest.akka.QuietReporter") ++ + (if (excludeTags.isEmpty) Seq.empty else Seq("-l", if (multiNodeEnabled) excludeTags.mkString("\"", " ", "\"") else excludeTags.mkString(" "))) ++ + (if (useOnlyTestTags.isEmpty) Seq.empty else Seq("-n", if (multiNodeEnabled) useOnlyTestTags.mkString("\"", " ", "\"") else useOnlyTestTags.mkString(" "))) + } + + lazy val defaultSettings = baseSettings ++ formatSettings ++ mimaSettings ++ Seq( resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/", // compile options @@ -296,40 +387,27 @@ object AkkaBuild extends Build { parallelExecution in Test := System.getProperty("akka.parallelExecution", "false").toBoolean, - // for excluding tests by name (or use system property: -Dakka.test.names.exclude=TimingSpec) - excludeTestNames := { - val exclude = System.getProperty("akka.test.names.exclude", "") - if (exclude.isEmpty) Seq.empty else exclude.split(",").toSeq - }, - - // for excluding tests by tag (or use system property: -Dakka.test.tags.exclude=timing) - excludeTestTags := { - val exclude = System.getProperty("akka.test.tags.exclude", "") - if (exclude.isEmpty) defaultExcludedTags else exclude.split(",").toSeq - }, - - // for including tests by tag (or use system property: -Dakka.test.tags.include=timing) - includeTestTags := { - val include = System.getProperty("akka.test.tags.include", "") - if (include.isEmpty) Seq.empty else include.split(",").toSeq - }, + excludeTestNames := useExcludeTestNames, + excludeTestTags := useExcludeTestTags, + includeTestTags := useIncludeTestTags, + onlyTestTags := useOnlyTestTags, // add filters for tests excluded by name testOptions in Test <++= excludeTestNames map { _.map(exclude => Tests.Filter(test => !test.contains(exclude))) }, // add arguments for tests excluded by tag - includes override excludes (opposite to scalatest) testOptions in Test <++= (excludeTestTags, includeTestTags) map { (excludes, includes) => - val tags = (excludes.toSet -- includes.toSet).toSeq + val tags = (excludes -- includes) if (tags.isEmpty) Seq.empty else Seq(Tests.Argument("-l", tags.mkString(" "))) }, - // add arguments for tests included by tag - testOptions in Test <++= includeTestTags map { tags => + // add arguments for running only tests by tag + testOptions in Test <++= onlyTestTags map { tags => if (tags.isEmpty) Seq.empty else Seq(Tests.Argument("-n", tags.mkString(" "))) }, - // show full stack traces - testOptions in Test += Tests.Argument("-oF") + // show full stack traces and test case durations + testOptions in Test += Tests.Argument("-oDF") ) lazy val formatSettings = ScalariformPlugin.scalariformSettings ++ Seq( @@ -347,8 +425,32 @@ object AkkaBuild extends Build { lazy val multiJvmSettings = MultiJvmPlugin.settings ++ inConfig(MultiJvm)(ScalariformPlugin.scalariformSettings) ++ Seq( compileInputs in MultiJvm <<= (compileInputs in MultiJvm) dependsOn (ScalariformKeys.format in MultiJvm), - ScalariformKeys.preferences in MultiJvm := formattingPreferences + ScalariformKeys.preferences in MultiJvm := formattingPreferences) ++ + ((executeMultiJvmTests, multiNodeEnabled) match { + case (true, true) => + executeTests in Test <<= ((executeTests in Test), (multiNodeExecuteTests in MultiJvm)) map { + case ((_, testResults), (_, multiNodeResults)) => + val results = testResults ++ multiNodeResults + (Tests.overall(results.values), results) + } + case (true, false) => + executeTests in Test <<= ((executeTests in Test), (executeTests in MultiJvm)) map { + case ((_, testResults), (_, multiNodeResults)) => + val results = testResults ++ multiNodeResults + (Tests.overall(results.values), results) + } + case (false, _) => Seq.empty + }) + + lazy val mimaSettings = mimaDefaultSettings ++ Seq( + // MiMa + previousArtifact := None ) + + def akkaPreviousArtifact(id: String, organization: String = "com.typesafe.akka", version: String = "2.0"): Option[sbt.ModuleID] = { + // the artifact to compare binary compatibility with + Some(organization % id % version) + } } // Dependencies @@ -356,6 +458,8 @@ object AkkaBuild extends Build { object Dependencies { import Dependency._ + val actor = Seq(config) + val testkit = Seq(Test.scalatest, Test.junit) val actorTests = Seq( @@ -364,8 +468,7 @@ object Dependencies { ) val remote = Seq( - netty, protobuf, Test.junit, Test.scalatest, - Test.zookeeper, Test.log4j // needed for ZkBarrier in multi-jvm tests + netty, protobuf, uncommonsMath, Test.junit, Test.scalatest ) val cluster = Seq(Test.junit, Test.scalatest) @@ -380,13 +483,17 @@ object Dependencies { val fileMailbox = Seq(Test.commonsIo, Test.scalatest, Test.junit) - val kernel = Seq(jmxClient, Test.scalatest, Test.junit) + val kernel = Seq(Test.scalatest, Test.junit) val camel = Seq(camelCore, Test.scalatest, Test.junit, Test.mockito) + val osgi = Seq(osgiCore,Test.logback, Test.commonsIo, Test.pojosr, Test.tinybundles, Test.scalatest, Test.junit) + + val osgiAries = Seq(osgiCore, ariesBlueprint, Test.ariesProxy) + val tutorials = Seq(Test.scalatest, Test.junit) - val docs = Seq(Test.scalatest, Test.junit) + val docs = Seq(Test.scalatest, Test.junit, Test.specs2) val zeroMQ = Seq(protobuf, Dependency.zeroMQ, Test.scalatest, Test.junit) } @@ -397,41 +504,82 @@ object Dependency { object V { val Camel = "2.8.0" - val Logback = "0.9.28" - val Netty = "3.3.0.Final" + val Logback = "1.0.4" + val Netty = "3.5.1.Final" + val OSGi = "4.2.0" val Protobuf = "2.4.1" val ScalaStm = "0.5" val Scalatest = "1.6.1" val Slf4j = "1.6.4" + val UncommonsMath = "1.2.2a" } // Compile - + val ariesBlueprint = "org.apache.aries.blueprint" % "org.apache.aries.blueprint" % "0.3.2" // ApacheV2 + val config = "com.typesafe" % "config" % "0.4.1" // ApacheV2 val camelCore = "org.apache.camel" % "camel-core" % V.Camel // ApacheV2 - val jmxClient = "cmdline-jmxclient" % "cmdline-jmxclient" % "0.10.3" // LGPL val netty = "io.netty" % "netty" % V.Netty // ApacheV2 + val osgiCore = "org.osgi" % "org.osgi.core" % V.OSGi // ApacheV2 val protobuf = "com.google.protobuf" % "protobuf-java" % V.Protobuf // New BSD val scalaStm = "org.scala-tools" % "scala-stm_2.9.1" % V.ScalaStm // Modified BSD (Scala) val slf4jApi = "org.slf4j" % "slf4j-api" % V.Slf4j // MIT + val uncommonsMath = "org.uncommons.maths" % "uncommons-maths" % V.UncommonsMath // ApacheV2 val zeroMQ = "org.zeromq" % "zeromq-scala-binding_2.9.1" % "0.0.6" // ApacheV2 - // Runtime - - object Runtime { - val logback = "ch.qos.logback" % "logback-classic" % V.Logback % "runtime" // MIT - } - // Test object Test { + val ariesProxy = "org.apache.aries.proxy" % "org.apache.aries.proxy.impl" % "0.3" % "test" // ApacheV2 val commonsMath = "org.apache.commons" % "commons-math" % "2.1" % "test" // ApacheV2 - val commonsIo = "commons-io" % "commons-io" % "2.0.1" % "test"// ApacheV2 + val commonsIo = "commons-io" % "commons-io" % "2.0.1" % "test"// ApacheV2 val junit = "junit" % "junit" % "4.5" % "test" // Common Public License 1.0 val logback = "ch.qos.logback" % "logback-classic" % V.Logback % "test" // EPL 1.0 / LGPL 2.1 val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test" // MIT + val pojosr = "com.googlecode.pojosr" % "de.kalpatec.pojosr.framework" % "0.1.4" % "test" // ApacheV2 val scalatest = "org.scalatest" % "scalatest_2.9.1" % V.Scalatest % "test" // ApacheV2 val scalacheck = "org.scala-tools.testing" % "scalacheck_2.9.1" % "1.9" % "test" // New BSD - val zookeeper = "org.apache.hadoop.zookeeper" % "zookeeper" % "3.4.0" % "test" // ApacheV2 + val specs2 = "org.specs2" % "specs2_2.9.1" % "1.9" % "test" // Modified BSD / ApacheV2 + val tinybundles = "org.ops4j.pax.tinybundles" % "tinybundles" % "1.0.0" % "test" // ApacheV2 val log4j = "log4j" % "log4j" % "1.2.14" % "test" // ApacheV2 } } + +// OSGi settings + +object OSGi { + + val actor = exports(Seq("akka*")) + + val agent = exports(Seq("akka.agent.*")) + + val camel = exports(Seq("akka.camel.*", "akka.camelexamples")) + + val cluster = exports(Seq("akka.cluster.*")) + + val fileMailbox = exports(Seq("akka.actor.mailbox.*")) + + val mailboxesCommon = exports(Seq("akka.actor.mailbox.*")) + + val osgi = exports(Seq("akka.osgi")) ++ Seq(OsgiKeys.privatePackage := Seq("akka.osgi.impl")) + + val osgiAries = exports() ++ Seq(OsgiKeys.privatePackage := Seq("akka.osgi.aries.*")) + + val remote = exports(Seq("akka.remote.*", "akka.routing.*", "akka.serialization.*")) + + val slf4j = exports(Seq("akka.event.slf4j.*")) + + val transactor = exports(Seq("akka.transactor.*")) + + val zeroMQ = exports(Seq("akka.zeromq.*")) + + def exports(packages: Seq[String] = Seq()) = osgiSettings ++ Seq( + OsgiKeys.importPackage := defaultImports, + OsgiKeys.exportPackage := packages + ) + + def defaultImports = Seq("!sun.misc", akkaImport(), configImport(), scalaImport(), "*") + def akkaImport(packageName: String = "akka.*") = "%s;version=\"[2.1,2.2)\"".format(packageName) + def configImport(packageName: String = "com.typesafe.config.*") = "%s;version=\"[0.4.1,0.5)\"".format(packageName) + def scalaImport(packageName: String = "scala.*") = "%s;version=\"[2.9.2,2.10)\"".format(packageName) + +} diff --git a/project/Sphinx.scala b/project/Sphinx.scala index 43b7e60358..4707215875 100644 --- a/project/Sphinx.scala +++ b/project/Sphinx.scala @@ -87,16 +87,15 @@ object Sphinx { def pdfTask = (sphinxLatex, streams) map { (latex, s) => { - val empty = (latex * "*.pdf").get.isEmpty + val pdf = latex / "Akka.pdf" def failed = sys.error("Failed to build Sphinx pdf documentation.") - if (empty) { + if (!pdf.exists) { s.log.info("Building Sphinx pdf documentation...") val logger = newLogger(s) val exitCode = Process(Seq("make", "all-pdf"), latex) ! logger if (exitCode != 0) failed + s.log.info("Sphinx pdf documentation created: %s" format pdf) } - val pdf = (latex * "*.pdf").get.headOption.getOrElse(failed) - if (empty) s.log.info("Sphinx pdf documentation created: %s" format pdf) pdf } } diff --git a/project/build.properties b/project/build.properties index f4ff7a5afa..d4287112c6 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.11.2 +sbt.version=0.11.3 diff --git a/project/plugins.sbt b/project/plugins.sbt index 80ff9db95a..e077802cfa 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,14 +1,18 @@ resolvers += Classpaths.typesafeResolver -addSbtPlugin("com.typesafe.sbtmultijvm" % "sbt-multi-jvm" % "0.1.9") +addSbtPlugin("com.typesafe.sbtmultijvm" % "sbt-multi-jvm" % "0.2.0-M3") -addSbtPlugin("com.typesafe.schoir" % "schoir" % "0.1.2") +addSbtPlugin("com.typesafe.sbtscalariform" % "sbtscalariform" % "0.4.0") -addSbtPlugin("com.typesafe.sbtscalariform" % "sbtscalariform" % "0.3.1") +addSbtPlugin("com.typesafe.sbtosgi" % "sbtosgi" % "0.2.0") + +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.3") resolvers ++= Seq( + // needed for sbt-assembly, which comes with sbt-multi-jvm + Resolver.url("sbtonline", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns), "less is" at "http://repo.lessis.me", "coda" at "http://repo.codahale.com") -addSbtPlugin("me.lessis" % "ls-sbt" % "0.1.1") +// addSbtPlugin("me.lessis" % "ls-sbt" % "0.1.1") diff --git a/project/scripts/multi-node-log-replace b/project/scripts/multi-node-log-replace new file mode 100755 index 0000000000..83f1b8a136 --- /dev/null +++ b/project/scripts/multi-node-log-replace @@ -0,0 +1,11 @@ +#!/usr/bin/env bash +# +# Utility to make log files from multi-node tests easier to analyze. +# Replaces jvm names and host:port with corresponding logical role name. +# + + +# check for an sbt command +type -P sbt &> /dev/null || fail "sbt command not found" + +sbt "project akka-remote-tests" "test:run-main akka.remote.testkit.LogRoleReplace $1 $2" \ No newline at end of file diff --git a/project/scripts/release b/project/scripts/release index 058d0d1615..9e418317bd 100755 --- a/project/scripts/release +++ b/project/scripts/release @@ -93,6 +93,8 @@ fi declare -r version=$1 declare -r publish_path="${release_server}:${release_path}" +[[ `java -version 2>&1 | head -1 | cut -d ' ' -f3 | cut -d '.' -f2` -eq 6 ]] || fail "Java version is not 1.6" + # check for a git command type -P git &> /dev/null || fail "git command not found" @@ -219,6 +221,13 @@ echolog "Creating gzipped tar download..." try tar -cz -C ${unzipped_dir} -f ${release_dir}/downloads/akka-${version}.tgz akka-${version} echolog "Successfully created local release" +# check binary compatibility for dry run +if [ $dry_run ]; then + echodry "Running migration manager report..." + sbt mima-report-binary-issues + echodry "Finished migration manager report" +fi + # commit and tag this release echolog "Committing and tagging..." try git add . diff --git a/repl b/repl new file mode 100644 index 0000000000..29f505e292 --- /dev/null +++ b/repl @@ -0,0 +1,16 @@ +import akka.actor._ +import akka.dispatch.{ Future, Promise } +import com.typesafe.config.ConfigFactory +import akka.testkit._ +val remoteConfig = try { + Class.forName("akka.remote.RemoteActorRefProvider") + "\nakka.actor.provider=akka.remote.RemoteActorRefProvider" + } catch { + case _: ClassNotFoundException => "" + } +val config=ConfigFactory.parseString("akka.daemonic=on" + remoteConfig) +val sys=ActorSystem("repl", config.withFallback(ConfigFactory.load())).asInstanceOf[ExtendedActorSystem] +implicit val ec=sys.dispatcher +import akka.util.duration._ +import akka.util.Timeout +implicit val timeout=Timeout(5 seconds) diff --git a/scripts/fix-protobuf.sh b/scripts/fix-protobuf.sh new file mode 100755 index 0000000000..e53ce297ab --- /dev/null +++ b/scripts/fix-protobuf.sh @@ -0,0 +1,3 @@ +#!/bin/bash + +find . -name \*.java -print0 | xargs -0 perl -pi -e 's/\Qprivate Builder(BuilderParent parent)/private Builder(com.google.protobuf.GeneratedMessage.BuilderParent parent)/' diff --git a/scripts/multi-node-log-replace.sh b/scripts/multi-node-log-replace.sh new file mode 100755 index 0000000000..8e8af7112a --- /dev/null +++ b/scripts/multi-node-log-replace.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash +# +# Utility to make log files from multi-node tests easier to analyze. +# Replaces jvm names and host:port with corresponding logical role name. +# +# Use with 0, 1 or 2 arguments. +# +# When using 0 arguments it reads from standard input +# and writes to standard output. +# +# With 1 argument it reads from the file specified in the first argument +# and writes to standard output. +# +# With 2 arguments it reads the file specified in the first argument +# and writes to the file specified in the second argument. +# +# You can also replace the contents of the clipboard instead of using files +# by supplying `clipboard` as argument +# + + +# check for an sbt command +type -P sbt &> /dev/null || fail "sbt command not found" + +sbt "project akka-remote-tests" "test:run-main akka.remote.testkit.LogRoleReplace $1 $2" \ No newline at end of file