From 96cc0a00b42fe6d428c3791c7b596a7df334a678 Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Tue, 26 Jul 2011 08:16:26 +0300 Subject: [PATCH 1/6] ticket #958 --- .../src/main/scala/akka/actor/Actor.scala | 6 +- .../scala/akka/actor/DeploymentConfig.scala | 6 +- .../src/main/scala/akka/cluster/Cluster.scala | 3 +- .../scala/akka/cluster/ClusterActorRef.scala | 87 +++++++--- .../src/main/scala/akka/cluster/Routing.scala | 66 +++++--- .../scala/akka/cluster/ClusterTestNode.scala | 1 + .../ClusterActorRefCleanupMultiJvmNode1.conf | 6 + .../ClusterActorRefCleanupMultiJvmNode1.opts | 1 + .../ClusterActorRefCleanupMultiJvmNode2.conf | 6 + .../ClusterActorRefCleanupMultiJvmNode2.opts | 1 + .../ClusterActorRefCleanupMultiJvmNode3.conf | 6 + .../ClusterActorRefCleanupMultiJvmNode3.opts | 1 + .../ClusterActorRefCleanupMultiJvmSpec.scala | 150 ++++++++++++++++++ 13 files changed, 293 insertions(+), 47 deletions(-) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.conf create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.opts create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode2.conf create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode2.opts create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode3.conf create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode3.opts create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index b981c954ce..218fa0893c 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -82,7 +82,11 @@ case class MaximumNumberOfRestartsWithinTimeRangeReached( // Exceptions for Actors class ActorStartException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) class IllegalActorStateException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) -class ActorKilledException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) + +class ActorKilledException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause){ + def this(msg: String) = this(msg, null); +} + class ActorInitializationException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) class ActorTimeoutException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) class InvalidMessageException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) diff --git a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala index f5206b7668..ed1eefcd7a 100644 --- a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala +++ b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala @@ -133,8 +133,10 @@ object DeploymentConfig { case Host("localhost") ⇒ Config.nodename case IP("0.0.0.0") ⇒ Config.nodename case IP("127.0.0.1") ⇒ Config.nodename - case Host(hostname) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'hostname' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") - case IP(address) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'IP address' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") + case Host(hostname) ⇒ throw new UnsupportedOperationException( + "Specifying preferred node name by 'hostname' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") + case IP(address) ⇒ throw new UnsupportedOperationException( + "Specifying preferred node name by 'IP address' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") } def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home ⇒ nodeNameFor(home) == Config.nodename) diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 39c57c0778..980e5ef9e5 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -933,7 +933,8 @@ class DefaultClusterNode private[akka]( */ def release(actorAddress: String) { - // FIXME 'Cluster.release' needs to notify all existing ClusterActorRef's that are using the instance that it is no longer available. Then what to do? Should we even remove this method? + // FIXME 'Cluster.release' needs to notify all existing ClusterActorRef's that are using the instance that it is no + // longer available. Then what to do? Should we even remove this method? if (isConnected.isOn) { ignore[ZkNoNodeException](zkClient.delete(actorAddressToNodesPathFor(actorAddress, nodeAddress.nodeName))) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala index 3dc5f14b78..5afe318daa 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala @@ -3,22 +3,19 @@ */ package akka.cluster -import Cluster._ - import akka.actor._ -import Actor._ import akka.dispatch._ import akka.util._ import ReflectiveAccess._ -import ClusterModule._ -import akka.event.EventHandler import akka.dispatch.Future import java.net.InetSocketAddress import java.util.concurrent.atomic.AtomicReference -import java.util.{ Map ⇒ JMap } +import java.util.{Map ⇒ JMap} import com.eaio.uuid.UUID +import collection.immutable.Map +import annotation.tailrec /** * ActorRef representing a one or many instances of a clustered, load-balanced and sometimes replicated actor @@ -26,11 +23,11 @@ import com.eaio.uuid.UUID * * @author Jonas Bonér */ -class ClusterActorRef private[akka] ( - inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], - val address: String, - _timeout: Long) - extends ActorRef with ScalaActorRef { this: Router.Router ⇒ +class ClusterActorRef private[akka](inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], + val address: String, + _timeout: Long) + extends ActorRef with ScalaActorRef { + this: Router.Router ⇒ timeout = _timeout private[akka] val inetSocketAddressToActorRefMap = new AtomicReference[Map[InetSocketAddress, ActorRef]]( @@ -45,29 +42,71 @@ class ClusterActorRef private[akka] ( override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = { val sender = channel match { case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None + case _ ⇒ None } route(message)(sender) } - override def postMessageToMailboxAndCreateFutureResultWithTimeout( - message: Any, - timeout: Long, - channel: UntypedChannel): Future[Any] = { + override def postMessageToMailboxAndCreateFutureResultWithTimeout(message: Any, + timeout: Long, + channel: UntypedChannel): Future[Any] = { val sender = channel match { case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None + case _ ⇒ None } route[Any](message, timeout)(sender) } - private[akka] def failOver(fromInetSocketAddress: InetSocketAddress, toInetSocketAddress: InetSocketAddress) { - inetSocketAddressToActorRefMap set (inetSocketAddressToActorRefMap.get map { - case (`fromInetSocketAddress`, actorRef) ⇒ - actorRef.stop() - (toInetSocketAddress, createRemoteActorRef(actorRef.address, toInetSocketAddress)) - case other ⇒ other - }) + private[akka] def failOver(from: InetSocketAddress, to: InetSocketAddress): Unit = { + @tailrec + def doFailover(from: InetSocketAddress, to: InetSocketAddress): Unit = { + val oldValue = inetSocketAddressToActorRefMap.get + + val newValue = oldValue map { + case (`from`, actorRef) ⇒ + actorRef.stop() + (to, createRemoteActorRef(actorRef.address, to)) + case other ⇒ other + } + + if (!inetSocketAddressToActorRefMap.compareAndSet(oldValue, newValue)) + doFailover(from, to) + } + + doFailover(from, to) + } + + /** + * Removes the given address (and the corresponding actorref) from this ClusteredActorRef. + * + * Call can safely be made when the address is missing. + * + * Call is threadsafe. + */ + @tailrec + private def remove(address: InetSocketAddress): Unit = { + val oldValue = inetSocketAddressToActorRefMap.get() + + var newValue = oldValue - address + + if (!inetSocketAddressToActorRefMap.compareAndSet(oldValue, newValue)) + remove(address) + } + + def signalDeadActor(ref: ActorRef): Unit = { + //since the number remote actor refs for a clustered actor ref is quite low, we can deal with the O(N) complexity + //of the following removal. + val it = connections.keySet.iterator + + while (it.hasNext) { + val address = it.next() + val foundRef: ActorRef = connections.get(address).get + + if (foundRef == ref) { + remove(address) + return + } + } } private def createRemoteActorRef(actorAddress: String, inetSocketAddress: InetSocketAddress) = { diff --git a/akka-cluster/src/main/scala/akka/cluster/Routing.scala b/akka-cluster/src/main/scala/akka/cluster/Routing.scala index c165e699fe..c9db1e7208 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Routing.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Routing.scala @@ -3,13 +3,10 @@ */ package akka.cluster -import Cluster._ - import akka.actor._ -import Actor._ import akka.dispatch.Future import akka.event.EventHandler -import akka.routing.{ RouterType, RoutingException } +import akka.routing.{RouterType, RoutingException} import RouterType._ import com.eaio.uuid.UUID @@ -24,16 +21,16 @@ import java.util.concurrent.atomic.AtomicReference */ object Router { def newRouter( - routerType: RouterType, - inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], - actorAddress: String, - timeout: Long): ClusterActorRef = { + routerType: RouterType, + inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], + actorAddress: String, + timeout: Long): ClusterActorRef = { routerType match { - case Direct ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Direct - case Random ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Random - case RoundRobin ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with RoundRobin - case LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") - case LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") + case Direct ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Direct + case Random ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Random + case RoundRobin ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with RoundRobin + case LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") + case LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") case LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") } } @@ -42,22 +39,49 @@ object Router { * @author Jonas Bonér */ trait Router { + def connections: Map[InetSocketAddress, ActorRef] + def signalDeadActor(ref: ActorRef): Unit + def route(message: Any)(implicit sender: Option[ActorRef]): Unit def route[T](message: Any, timeout: Long)(implicit sender: Option[ActorRef]): Future[T] } + /** + * An Abstract Router implementation that already provides the basic infrastructure so that a concrete + * Router only needs to implement the next method. + * + * This also is the location where a failover is done in the future if an ActorRef fails and a different + * one needs to be selected. + */ trait BasicRouter extends Router { + def route(message: Any)(implicit sender: Option[ActorRef]): Unit = next match { - case Some(actor) ⇒ actor.!(message)(sender) - case _ ⇒ throwNoConnectionsError() + case Some(actor) ⇒ { + try { + actor.!(message)(sender) + } catch { + case e: Exception => + signalDeadActor(actor) + throw e + } + } + case _ ⇒ throwNoConnectionsError() } def route[T](message: Any, timeout: Long)(implicit sender: Option[ActorRef]): Future[T] = next match { - case Some(actor) ⇒ actor.?(message, timeout)(sender).asInstanceOf[Future[T]] - case _ ⇒ throwNoConnectionsError() + case Some(actor) ⇒ { + try { + actor.?(message, timeout)(sender).asInstanceOf[Future[T]] + } catch { + case e: Throwable => + signalDeadActor(actor) + throw e + } + } + case _ ⇒ throwNoConnectionsError() } protected def next: Option[ActorRef] @@ -73,6 +97,7 @@ object Router { * @author Jonas Bonér */ trait Direct extends BasicRouter { + lazy val next: Option[ActorRef] = { val connection = connections.values.headOption if (connection.isEmpty) EventHandler.warning(this, "Router has no replica connection") @@ -90,7 +115,9 @@ object Router { if (connections.isEmpty) { EventHandler.warning(this, "Router has no replica connections") None - } else Some(connections.valuesIterator.drop(random.nextInt(connections.size)).next()) + } else { + Some(connections.valuesIterator.drop(random.nextInt(connections.size)).next()) + } } /** @@ -109,7 +136,7 @@ object Router { val currentItems = current.get val newItems = currentItems match { case Nil ⇒ items - case xs ⇒ xs + case xs ⇒ xs } if (newItems.isEmpty) { @@ -124,4 +151,5 @@ object Router { findNext } } + } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala index aebfd8b651..6393502377 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala @@ -36,6 +36,7 @@ trait MasterClusterTestNode extends WordSpec with MustMatchers with BeforeAndAft } trait ClusterTestNode extends WordSpec with MustMatchers with BeforeAndAfterAll { + override def beforeAll() = { ClusterTestNode.waitForReady(getClass.getName) } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.conf new file mode 100644 index 0000000000..518aed1cd0 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.conf @@ -0,0 +1,6 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "ERROR" +akka.actor.deployment.service-test.router = "round-robin" +akka.actor.deployment.service-test.clustered.preferred-nodes = ["node:node2","node:node3"] +akka.actor.deployment.service-test.clustered.replication-factor = 2 +akka.cluster.client.buffering.retry-message-send-on-failure = false diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.opts new file mode 100644 index 0000000000..a88c260d8c --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode2.conf new file mode 100644 index 0000000000..518aed1cd0 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode2.conf @@ -0,0 +1,6 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "ERROR" +akka.actor.deployment.service-test.router = "round-robin" +akka.actor.deployment.service-test.clustered.preferred-nodes = ["node:node2","node:node3"] +akka.actor.deployment.service-test.clustered.replication-factor = 2 +akka.cluster.client.buffering.retry-message-send-on-failure = false diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode2.opts new file mode 100644 index 0000000000..f1e01f253d --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode2.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode3.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode3.conf new file mode 100644 index 0000000000..882d9cb7db --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode3.conf @@ -0,0 +1,6 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "ERROR" +akka.actor.deployment.service-test.router = "round-robin" +akka.actor.deployment.service-test.clustered.preferred-nodes = ["node:node2","node:node3"] +akka.actor.deployment.service-test.clustered.replication-factor = 2 +akka.cluster.client.buffering.retry-message-send-on-failure = false \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode3.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode3.opts new file mode 100644 index 0000000000..202496ad31 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode3.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node3 -Dakka.cluster.port=9993 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala new file mode 100644 index 0000000000..18c8da87da --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala @@ -0,0 +1,150 @@ +/* + * Copyright (C) 2009-2011 Typesafe Inc. + */ + +package akka.cluster.reflogic + +import akka.cluster._ +import akka.cluster.Cluster._ +import akka.actor.Actor +import akka.routing.RoutingException +import java.nio.channels.{ClosedChannelException, NotYetConnectedException} + +object ClusterActorRefCleanupMultiJvmSpec { + + val NrOfNodes = 3 + + class TestActor extends Actor with Serializable { + println("--------------------------------------") + println("TestActor created") + println("--------------------------------------") + + def receive = { + case "Die" => + println("Killing JVM: " + Cluster.node.nodeAddress) + System.exit(0) + case _ => + println("Hello") + } + } + +} + +class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { + + import ClusterActorRefCleanupMultiJvmSpec._ + + val testNodes = NrOfNodes + + "ClusterActorRef" must { + "cleanup itself" in { + node.start + barrier("awaitStarted", NrOfNodes).await() + + val ref = Actor.actorOf[ClusterActorRefCleanupMultiJvmSpec.TestActor]("service-test") + + ref.isInstanceOf[ClusterActorRef] must be(true) + + val clusteredRef = ref.asInstanceOf[ClusterActorRef] + + //verify that all remote actors are there. + clusteredRef.connections.size must be(2) + + //let one of the actors die. + clusteredRef ! "Die" + + //just some waiting to make sure that the node has died. + Thread.sleep(5000) + + //send some request, this should trigger the cleanup + try { + clusteredRef ! "hello" + clusteredRef ! "hello" + } catch { + case e: NotYetConnectedException => + } + + //since the call to the node failed, the node must have been removed from the list. + clusteredRef.connections.size must be(1) + + //send a message to this node, + clusteredRef ! "hello" + + //now kill another node + clusteredRef ! "Die" + + //just some waiting to make sure that the node has died. + Thread.sleep(5000) + + //trigger the cleanup. + try { + clusteredRef ! "hello" + } catch { + case e: ClosedChannelException => + case e: NotYetConnectedException => + case e: RoutingException => + } + + //now there must not be any remaining connections after the dead of the last actor. + clusteredRef.connections.size must be(0) + + //and lets make sure we now get the correct exception if we try to use the ref. + try { + clusteredRef ! "Hello" + assert(false) + } catch { + case e: RoutingException => + } + + node.shutdown() + } + } +} + +class ClusterActorRefCleanupMultiJvmNode2 extends ClusterTestNode { + + import ClusterActorRefCleanupMultiJvmSpec._ + + val testNodes = NrOfNodes + + //we are only using the nodes for their capacity, not for testing on this node itself. + "___" must { + "___" in { + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run() { + ClusterTestNode.exit(classOf[ClusterActorRefCleanupMultiJvmNode2].getName) + } + }) + + node.start() + barrier("awaitStarted", NrOfNodes).await() + + barrier("finished", NrOfNodes).await() + node.shutdown() + } + } +} + +class ClusterActorRefCleanupMultiJvmNode3 extends ClusterTestNode { + + import ClusterActorRefCleanupMultiJvmSpec._ + + val testNodes = NrOfNodes + + //we are only using the nodes for their capacity, not for testing on this node itself. + "___" must { + "___" in { + Runtime.getRuntime.addShutdownHook(new Thread() { + override def run() { + ClusterTestNode.exit(classOf[ClusterActorRefCleanupMultiJvmNode3].getName) + } + }) + + node.start() + barrier("awaitStarted", NrOfNodes).await() + + barrier("finished", NrOfNodes).await() + node.shutdown() + } + } +} From 6e337bfa2b1f4b2f994a6596b61fc518b47e8c58 Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Tue, 26 Jul 2011 17:27:21 +1200 Subject: [PATCH 2/6] Update to sbt 0.10.1 --- project/AkkaBuild.scala | 4 +--- project/build.properties | 2 +- project/plugins/build.sbt | 2 +- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index b05bc5b6b6..1d67923ec8 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -70,9 +70,7 @@ object AkkaBuild extends Build { extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => (name: String) => (src ** (name + ".conf")).get.headOption.map("-Dakka.config=" + _.absolutePath).toSeq }, - // TODO: use dependsOn once updated to sbt 0.10.1 -- currently doesn't fail on error - // test in Test <<= (test in Test) dependsOn (test in MultiJvm) - test in Test <<= (test in MultiJvm, (test in Test).task) flatMap { (mj, t) => t } + test in Test <<= (test in Test) dependsOn (test in MultiJvm) ) ) configs (MultiJvm) diff --git a/project/build.properties b/project/build.properties index 35e164f667..f47a3009ec 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.10.0 +sbt.version=0.10.1 diff --git a/project/plugins/build.sbt b/project/plugins/build.sbt index a5d9739061..39a11cabee 100644 --- a/project/plugins/build.sbt +++ b/project/plugins/build.sbt @@ -1,4 +1,4 @@ resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/" -libraryDependencies += "com.typesafe" %% "sbt-multi-jvm" % "0.1.1" +libraryDependencies += "com.typesafe" %% "sbt-multi-jvm" % "0.1.2" From 6f2fcc91edb21dc1c2f879d64875ad85a9101a72 Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Tue, 26 Jul 2011 17:49:08 +1200 Subject: [PATCH 3/6] Add scalariform plugin --- project/AkkaBuild.scala | 27 ++++++++++++++++++----- project/plugins/build.sbt | 2 ++ project/plugins/project/BuildPlugin.scala | 11 --------- 3 files changed, 24 insertions(+), 16 deletions(-) delete mode 100644 project/plugins/project/BuildPlugin.scala diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 1d67923ec8..3ed1c084d6 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -1,8 +1,7 @@ import sbt._ import Keys._ import MultiJvmPlugin.{ MultiJvm, extraOptions } -import com.github.oforero.sbtformatter.SbtFormatter._ -import com.github.oforero.sbtformatter.SbtFormatterSettings._ +import ScalariformPlugin.{ format, formatPreferences } object AkkaBuild extends Build { lazy val buildSettings = Seq( @@ -65,7 +64,7 @@ object AkkaBuild extends Build { id = "akka-cluster", base = file("akka-cluster"), dependencies = Seq(stm, actorTests % "test->test"), - settings = defaultSettings ++ MultiJvmPlugin.settings ++ Seq( + settings = defaultSettings ++ multiJvmSettings ++ Seq( libraryDependencies ++= Dependencies.cluster, extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => (name: String) => (src ** (name + ".conf")).get.headOption.map("-Dakka.config=" + _.absolutePath).toSeq @@ -273,7 +272,7 @@ object AkkaBuild extends Build { override lazy val settings = super.settings ++ buildSettings ++ Publish.versionSettings - lazy val baseSettings = Defaults.defaultSettings ++ Publish.settings ++ formatterPreferences ++ formatterTasks + lazy val baseSettings = Defaults.defaultSettings ++ Publish.settings lazy val parentSettings = baseSettings ++ Seq( publishArtifact in Compile := false @@ -286,7 +285,7 @@ object AkkaBuild extends Build { if (exclude.isEmpty) Seq.empty else exclude.split(",").toSeq } - lazy val defaultSettings = baseSettings ++ Seq( + lazy val defaultSettings = baseSettings ++ formatSettings ++ Seq( resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/", resolvers += "Twitter Public Repo" at "http://maven.twttr.com", // This will be going away with com.mongodb.async's next release @@ -306,6 +305,24 @@ object AkkaBuild extends Build { testOptions in Test <++= testExcludes map { _.map(exclude => Tests.Filter(test => !test.contains(exclude))) } ) + lazy val formatSettings = ScalariformPlugin.settings ++ Seq( + formatPreferences in Compile := formattingPreferences, + formatPreferences in Test := formattingPreferences + ) + + def formattingPreferences = { + import scalariform.formatter.preferences._ + FormattingPreferences() + .setPreference(RewriteArrowSymbols, true) + .setPreference(AlignParameters, true) + .setPreference(AlignSingleLineCaseStatements, true) + } + + lazy val multiJvmSettings = MultiJvmPlugin.settings ++ inConfig(MultiJvm)(ScalariformPlugin.formatSettings) ++ Seq( + compileInputs in MultiJvm <<= (compileInputs in MultiJvm) dependsOn (format in MultiJvm), + formatPreferences in MultiJvm := formattingPreferences + ) + // reStructuredText docs val rstdocDirectory = SettingKey[File]("rstdoc-directory") diff --git a/project/plugins/build.sbt b/project/plugins/build.sbt index 39a11cabee..a37add7cb7 100644 --- a/project/plugins/build.sbt +++ b/project/plugins/build.sbt @@ -2,3 +2,5 @@ resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/" libraryDependencies += "com.typesafe" %% "sbt-multi-jvm" % "0.1.2" + +libraryDependencies += "com.typesafe" %% "sbt-scalariform" % "0.1" diff --git a/project/plugins/project/BuildPlugin.scala b/project/plugins/project/BuildPlugin.scala deleted file mode 100644 index 3648f1e926..0000000000 --- a/project/plugins/project/BuildPlugin.scala +++ /dev/null @@ -1,11 +0,0 @@ -import sbt._ -import Keys._ - -object PluginDef extends Build { - - lazy val formatterPlugin = ProjectRef( uri("git://github.com/viktorklang/sbt-cool-plugins.git"), "Formatter") - - lazy override val projects = Seq(root) - - lazy val root = Project("plugins", file(".")) dependsOn (formatterPlugin) -} \ No newline at end of file From 340ed11de5d974ebd2b5a6833e5474510f2fc65a Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Tue, 26 Jul 2011 18:33:59 +1200 Subject: [PATCH 4/6] Reformat with scalariform --- .../main/scala/akka/testing/Serializers.scala | 6 +- .../test/scala/akka/AkkaExceptionSpec.scala | 2 +- .../test/scala/akka/actor/actor/Bench.scala | 2 +- .../akka/actor/supervisor/Ticket669Spec.scala | 2 +- .../scala/akka/dispatch/ActorModelSpec.scala | 39 +- .../test/scala/akka/misc/SchedulerSpec.scala | 6 +- .../trading/common/OrderReceiver.scala | 2 +- .../workbench/BenchResultRepository.scala | 2 +- .../test/scala/akka/routing/RoutingSpec.scala | 57 ++- .../CallingThreadDispatcherModelSpec.scala | 3 +- .../src/main/scala/akka/AkkaException.scala | 4 +- .../src/main/scala/akka/actor/Actor.scala | 18 +- .../src/main/scala/akka/actor/ActorRef.scala | 109 +++-- .../scala/akka/actor/DeploymentConfig.scala | 4 +- .../src/main/scala/akka/actor/FSM.scala | 2 +- .../main/scala/akka/actor/Supervisor.scala | 6 +- .../main/scala/akka/actor/TypedActor.scala | 18 +- .../scala/akka/cluster/ClusterInterface.scala | 3 +- .../src/main/scala/akka/config/Config.scala | 6 +- .../main/scala/akka/config/ConfigParser.scala | 6 +- .../scala/akka/config/Configuration.scala | 13 +- .../main/scala/akka/dispatch/Dispatcher.scala | 2 +- .../scala/akka/dispatch/Dispatchers.scala | 2 +- .../src/main/scala/akka/dispatch/Future.scala | 5 +- .../scala/akka/dispatch/MessageHandling.scala | 3 +- .../main/scala/akka/event/EventHandler.scala | 46 +- .../src/main/scala/akka/routing/Pool.scala | 26 +- .../akka/serialization/Serialization.scala | 41 +- .../scala/akka/serialization/Serializer.scala | 4 +- .../akka/camel/component/ActorComponent.scala | 6 +- .../src/main/scala/akka/cluster/Cluster.scala | 418 +++++++++--------- .../scala/akka/cluster/ClusterActorRef.scala | 12 +- .../src/main/scala/akka/cluster/Routing.scala | 26 +- .../scala/akka/cluster/TransactionLog.scala | 2 +- .../scala/akka/cluster/storage/Storage.scala | 12 +- .../akka/cluster/zookeeper/AkkaZkClient.scala | 3 +- .../remote/netty/NettyRemoteSupport.scala | 206 +++++---- .../ClusterActorRefCleanupMultiJvmSpec.scala | 16 +- .../BadAddressDirectRoutingMultiJvmSpec.scala | 1 - ...ngleReplicaDirectRoutingMultiJvmSpec.scala | 1 - .../RoundRobin2ReplicasMultiJvmSpec.scala | 3 +- .../actor/mailbox/BeanstalkBasedMailbox.scala | 27 +- .../actor/mailbox/FiledBasedMailbox.scala | 12 +- .../actor/mailbox/filequeue/Journal.scala | 68 ++- .../mailbox/filequeue/PersistentQueue.scala | 83 ++-- .../akka/actor/mailbox/filequeue/QItem.scala | 2 +- .../mailbox/filequeue/QueueCollection.scala | 53 ++- .../mailbox/filequeue/tools/QDumper.scala | 33 +- .../actor/mailbox/filequeue/tools/Util.scala | 8 +- .../actor/mailbox/DurableDispatcher.scala | 48 +- .../akka/actor/mailbox/DurableMailbox.scala | 27 +- .../actor/mailbox/DurableMailboxSpec.scala | 13 +- .../actor/mailbox/BSONSerialization.scala | 32 +- .../actor/mailbox/MongoBasedMailbox.scala | 68 +-- .../actor/mailbox/MongoDurableMessage.scala | 8 +- .../actor/mailbox/MongoBasedMailboxSpec.scala | 6 +- .../actor/mailbox/RedisBasedMailbox.scala | 19 +- .../actor/mailbox/ZooKeeperBasedMailbox.scala | 20 +- .../mailbox/ZooKeeperBasedMailboxSpec.scala | 4 +- .../src/main/scala/sample/camel/Actors.scala | 42 +- .../src/main/scala/sample/camel/Boot.scala | 4 +- .../sample/camel/StandaloneApplication.scala | 10 +- .../camel/HttpConcurrencyTestStress.scala | 17 +- .../sample/camel/RemoteConsumerTest.scala | 6 +- .../src/main/scala/Buncher.scala | 76 ++-- .../main/scala/DiningHakkersOnBecome.scala | 56 +-- .../src/main/scala/DiningHakkersOnFsm.scala | 36 +- .../src/main/scala/akka/agent/Agent.scala | 4 +- .../scala/akka/transactor/Coordinated.scala | 18 +- .../scala/akka/transactor/Transactor.scala | 2 +- .../akka/transactor/UntypedTransactor.scala | 2 +- .../transactor/CoordinatedIncrementSpec.scala | 4 +- .../scala/transactor/FickleFriendsSpec.scala | 2 +- .../src/main/scala/Pi.scala | 18 +- .../src/main/scala/Pi.scala | 24 +- 75 files changed, 985 insertions(+), 1012 deletions(-) diff --git a/akka-actor-tests/src/main/scala/akka/testing/Serializers.scala b/akka-actor-tests/src/main/scala/akka/testing/Serializers.scala index 6412619963..de7d0924ea 100644 --- a/akka-actor-tests/src/main/scala/akka/testing/Serializers.scala +++ b/akka-actor-tests/src/main/scala/akka/testing/Serializers.scala @@ -14,7 +14,7 @@ import sjson.json._ class ProtobufSerializer extends Serializer { val ARRAY_OF_BYTE_ARRAY = Array[Class[_]](classOf[Array[Byte]]) - def identifier = 2:Byte + def identifier = 2: Byte def toBinary(obj: AnyRef): Array[Byte] = { if (!obj.isInstanceOf[Message]) throw new IllegalArgumentException( @@ -33,7 +33,7 @@ object ProtobufSerializer extends ProtobufSerializer class JavaJSONSerializer extends Serializer { private val mapper = new ObjectMapper - def identifier = 3:Byte + def identifier = 3: Byte def toBinary(obj: AnyRef): Array[Byte] = { val bos = new ByteArrayOutputStream @@ -58,7 +58,7 @@ object JavaJSONSerializer extends JavaJSONSerializer class SJSONSerializer extends Serializer { - def identifier = 4:Byte + def identifier = 4: Byte def toBinary(obj: AnyRef): Array[Byte] = sjson.json.Serializer.SJSON.out(obj) diff --git a/akka-actor-tests/src/test/scala/akka/AkkaExceptionSpec.scala b/akka-actor-tests/src/test/scala/akka/AkkaExceptionSpec.scala index ac0b2b75cf..5e51e2366a 100644 --- a/akka-actor-tests/src/test/scala/akka/AkkaExceptionSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/AkkaExceptionSpec.scala @@ -22,7 +22,7 @@ class AkkaExceptionSpec extends WordSpec with MustMatchers { } } - def verify(clazz:java.lang.Class[_]):Unit = { + def verify(clazz: java.lang.Class[_]): Unit = { clazz.getConstructor(Array(classOf[String]): _*) } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/Bench.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/Bench.scala index 315798cc19..9ac58d284a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/Bench.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/Bench.scala @@ -91,7 +91,7 @@ object Chameneos { self.stop() } - case msg@Meet(a, c) ⇒ + case msg @ Meet(a, c) ⇒ if (n > 0) { waitingChameneo match { case Some(chameneo) ⇒ diff --git a/akka-actor-tests/src/test/scala/akka/actor/supervisor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/akka/actor/supervisor/Ticket669Spec.scala index 5863305ff4..2074bff0d1 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/supervisor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/supervisor/Ticket669Spec.scala @@ -14,7 +14,7 @@ import org.scalatest.matchers.MustMatchers class Ticket669Spec extends WordSpec with MustMatchers with BeforeAndAfterAll { import Ticket669Spec._ - override def beforeAll = Thread.interrupted() //remove interrupted status. + override def beforeAll = Thread.interrupted() //remove interrupted status. override def afterAll = Actor.registry.local.shutdownAll diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala index 327d6dda10..3fa4d16f2e 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala @@ -9,12 +9,12 @@ import akka.testkit.Testing import akka.dispatch._ import akka.actor.Actor._ import java.util.concurrent.atomic.AtomicLong -import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, TimeUnit} +import java.util.concurrent.{ ConcurrentHashMap, CountDownLatch, TimeUnit } import akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor import akka.util.Switch -import akka.actor.{ActorKilledException, PoisonPill, ActorRef, Actor} +import akka.actor.{ ActorKilledException, PoisonPill, ActorRef, Actor } import java.rmi.RemoteException -import org.junit.{After, Test} +import org.junit.{ After, Test } object ActorModelSpec { @@ -46,7 +46,6 @@ object ActorModelSpec { case class ThrowException(e: Throwable) extends ActorModelMessage - val Ping = "Ping" val Pong = "Pong" @@ -68,19 +67,19 @@ object ActorModelSpec { } def receive = { - case Await(latch) ⇒ ack; latch.await(); busy.switchOff() - case Meet(sign, wait) ⇒ ack; sign.countDown(); wait.await(); busy.switchOff() - case Wait(time) ⇒ ack; Thread.sleep(time); busy.switchOff() - case WaitAck(time, l) ⇒ ack; Thread.sleep(time); l.countDown(); busy.switchOff() - case Reply(msg) ⇒ ack; self.reply(msg); busy.switchOff() - case TryReply(msg) ⇒ ack; self.tryReply(msg); busy.switchOff() - case Forward(to, msg) ⇒ ack; to.forward(msg); busy.switchOff() - case CountDown(latch) ⇒ ack; latch.countDown(); busy.switchOff() - case Increment(count) ⇒ ack; count.incrementAndGet(); busy.switchOff() - case CountDownNStop(l) ⇒ ack; l.countDown(); self.stop(); busy.switchOff() - case Restart ⇒ ack; busy.switchOff(); throw new Exception("Restart requested") - case Interrupt => ack; busy.switchOff(); throw new InterruptedException("Ping!") - case ThrowException(e: Throwable) => ack; busy.switchOff(); throw e + case Await(latch) ⇒ ack; latch.await(); busy.switchOff() + case Meet(sign, wait) ⇒ ack; sign.countDown(); wait.await(); busy.switchOff() + case Wait(time) ⇒ ack; Thread.sleep(time); busy.switchOff() + case WaitAck(time, l) ⇒ ack; Thread.sleep(time); l.countDown(); busy.switchOff() + case Reply(msg) ⇒ ack; self.reply(msg); busy.switchOff() + case TryReply(msg) ⇒ ack; self.tryReply(msg); busy.switchOff() + case Forward(to, msg) ⇒ ack; to.forward(msg); busy.switchOff() + case CountDown(latch) ⇒ ack; latch.countDown(); busy.switchOff() + case Increment(count) ⇒ ack; count.incrementAndGet(); busy.switchOff() + case CountDownNStop(l) ⇒ ack; l.countDown(); self.stop(); busy.switchOff() + case Restart ⇒ ack; busy.switchOff(); throw new Exception("Restart requested") + case Interrupt ⇒ ack; busy.switchOff(); throw new InterruptedException("Ping!") + case ThrowException(e: Throwable) ⇒ ack; busy.switchOff(); throw e } } @@ -368,9 +367,8 @@ abstract class ActorModelSpec extends JUnitSuite { def flood(num: Int) { val cachedMessage = CountDownNStop(new CountDownLatch(num)) - (1 to num) foreach { - _ ⇒ - newTestActor.start() ! cachedMessage + (1 to num) foreach { _ ⇒ + newTestActor.start() ! cachedMessage } assertCountDown(cachedMessage.latch, Testing.testTime(10000), "Should process " + num + " countdowns") } @@ -454,4 +452,3 @@ class BalancingDispatcherModelTest extends ActorModelSpec { new BalancingDispatcher("foo") with MessageDispatcherInterceptor } - diff --git a/akka-actor-tests/src/test/scala/akka/misc/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/misc/SchedulerSpec.scala index 3c08d0983a..6d2b4093ed 100644 --- a/akka-actor-tests/src/test/scala/akka/misc/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/misc/SchedulerSpec.scala @@ -5,12 +5,12 @@ import Actor._ import akka.config.Supervision._ import org.multiverse.api.latches.StandardLatch import org.junit.Test -import java.util.concurrent.{ScheduledFuture, ConcurrentLinkedQueue, CountDownLatch, TimeUnit} +import java.util.concurrent.{ ScheduledFuture, ConcurrentLinkedQueue, CountDownLatch, TimeUnit } class SchedulerSpec extends JUnitSuite { private val futures = new ConcurrentLinkedQueue[ScheduledFuture[AnyRef]]() - def collectFuture(f: => ScheduledFuture[AnyRef]): ScheduledFuture[AnyRef] = { + def collectFuture(f: ⇒ ScheduledFuture[AnyRef]): ScheduledFuture[AnyRef] = { val future = f futures.add(future) future @@ -18,7 +18,7 @@ class SchedulerSpec extends JUnitSuite { def withCleanEndState(action: ⇒ Unit) { action - while(futures.peek() ne null) { Option(futures.poll()).foreach(_.cancel(true)) } + while (futures.peek() ne null) { Option(futures.poll()).foreach(_.cancel(true)) } Actor.registry.local.shutdownAll } diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala index 869c186524..ae8e879421 100755 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/OrderReceiver.scala @@ -34,7 +34,7 @@ class AkkaOrderReceiver(disp: Option[MessageDispatcher]) } def receive = { - case routing@MatchingEngineRouting(mapping) ⇒ + case routing @ MatchingEngineRouting(mapping) ⇒ refreshMatchingEnginePartitions(routing.asInstanceOf[MatchingEngineRouting[ActorRef]]) case order: Order ⇒ placeOrder(order) case unknown ⇒ EventHandler.warning(this, "Received unknown message: " + unknown) diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala index 0c8e5f0cb2..048606d322 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala @@ -12,7 +12,7 @@ import java.io.PrintWriter import java.text.SimpleDateFormat import java.util.Date -import scala.collection.mutable.{Map => MutableMap} +import scala.collection.mutable.{ Map ⇒ MutableMap } import akka.event.EventHandler 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 f4bd4f1204..6b03c75e1a 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -109,7 +109,7 @@ class RoutingSpec extends WordSpec with MustMatchers { t1.dispatcher.suspend(t1) - for (i <- 1 to 2501) t1 ! i + for (i ← 1 to 2501) t1 ! i val t2 = actorOf(new Actor { def receive = { @@ -121,7 +121,7 @@ class RoutingSpec extends WordSpec with MustMatchers { val d = loadBalancerActor(new SmallestMailboxFirstIterator(t1 :: t2 :: Nil)) //Will pick the last with the smallest mailbox, so make sure t1 is last - for (i ← 1 to 2499 ) d ! i + for (i ← 1 to 2499) d ! i latch2.await(20 seconds) @@ -529,12 +529,12 @@ class RoutingSpec extends WordSpec with MustMatchers { def pressureThreshold = 1 def factory = actorOf(new Actor { if (deathCount.get > 5) deathCount.set(0) - if (deathCount.get > 0) {deathCount.incrementAndGet;throw new IllegalStateException("keep dying")} + if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } def receive = { - case akka.Die ⇒ + case akka.Die ⇒ if (keepDying) deathCount.incrementAndGet throw new RuntimeException - case _ => pingCount.incrementAndGet + case _ ⇒ pingCount.incrementAndGet } }).start() }).start() @@ -554,12 +554,12 @@ class RoutingSpec extends WordSpec with MustMatchers { def factory = actorOf(new Actor { self.lifeCycle = Permanent if (deathCount.get > 5) deathCount.set(0) - if (deathCount.get > 0) {deathCount.incrementAndGet;throw new IllegalStateException("keep dying")} + if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } def receive = { - case akka.Die ⇒ + case akka.Die ⇒ if (keepDying) deathCount.incrementAndGet throw new RuntimeException - case _ => pingCount.incrementAndGet + case _ ⇒ pingCount.incrementAndGet } }).start() }).start() @@ -579,12 +579,12 @@ class RoutingSpec extends WordSpec with MustMatchers { def factory = actorOf(new Actor { self.lifeCycle = Temporary if (deathCount.get > 5) deathCount.set(0) - if (deathCount.get > 0) {deathCount.incrementAndGet;throw new IllegalStateException("keep dying")} + if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } def receive = { - case akka.Die ⇒ + case akka.Die ⇒ if (keepDying) deathCount.incrementAndGet throw new RuntimeException - case _ => pingCount.incrementAndGet + case _ ⇒ pingCount.incrementAndGet } }).start() }).start() @@ -598,12 +598,12 @@ class RoutingSpec extends WordSpec with MustMatchers { pool1 ! akka.Die sleepFor(2 seconds) (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) - pingCount.get must be (1) + pingCount.get must be(1) // default lifecycle // actor dies completely pingCount.set(0) - keepDying = true + keepDying = true pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool1 ! akka.Die @@ -611,35 +611,35 @@ class RoutingSpec extends WordSpec with MustMatchers { (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(1) pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) - pingCount.get must be (2) + pingCount.get must be(2) // permanent lifecycle // actor comes back right away pingCount.set(0) - keepDying = false + keepDying = false pool2 ! "ping" (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool2 ! akka.Die sleepFor(2 seconds) (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) - pingCount.get must be (1) + pingCount.get must be(1) // permanent lifecycle // actor dies completely pingCount.set(0) - keepDying = true + keepDying = true pool2 ! "ping" (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool2 ! akka.Die sleepFor(2 seconds) (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(1) pool2 ! "ping" - (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) - pingCount.get must be (2) + (pool2 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) + pingCount.get must be(2) // temporary lifecycle pingCount.set(0) - keepDying = false + keepDying = false pool3 ! "ping" (pool3 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool3 ! akka.Die @@ -649,7 +649,7 @@ class RoutingSpec extends WordSpec with MustMatchers { pool3 ! "ping" pool3 ! "ping" (pool3 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) - pingCount.get must be (4) + pingCount.get must be(4) } "support customizable supervision config of pooled actors" in { @@ -678,19 +678,18 @@ class RoutingSpec extends WordSpec with MustMatchers { def pressureThreshold = 1 def factory = actorOf(new Actor { if (deathCount.get > 5) deathCount.set(0) - if (deathCount.get > 0) {deathCount.incrementAndGet;throw new IllegalStateException("keep dying")} + if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } def receive = { - case BadState ⇒ + case BadState ⇒ if (keepDying) deathCount.incrementAndGet throw new IllegalStateException - case akka.Die => + case akka.Die ⇒ throw new RuntimeException - case _ => pingCount.incrementAndGet + case _ ⇒ pingCount.incrementAndGet } }).start() }).start() - // actor comes back right away pingCount.set(0) keepDying = false @@ -699,11 +698,11 @@ class RoutingSpec extends WordSpec with MustMatchers { pool1 ! BadState sleepFor(2 seconds) (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) - pingCount.get must be (1) + pingCount.get must be(1) // actor dies completely pingCount.set(0) - keepDying = true + keepDying = true pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) pool1 ! BadState @@ -711,7 +710,7 @@ class RoutingSpec extends WordSpec with MustMatchers { (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(1) pool1 ! "ping" (pool1 ? ActorPool.Stat).as[ActorPool.Stats].get.size must be(2) - pingCount.get must be (2) + pingCount.get must be(2) // kill it intercept[RuntimeException](pool1.?(akka.Die).get) diff --git a/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala b/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala index 9fbc5fd7ac..78d74cfdc5 100644 --- a/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala @@ -5,7 +5,7 @@ package akka.testkit import akka.actor.dispatch.ActorModelSpec import java.util.concurrent.CountDownLatch -import org.junit.{After, Test} +import org.junit.{ After, Test } class CallingThreadDispatcherModelSpec extends ActorModelSpec { import ActorModelSpec._ @@ -43,7 +43,6 @@ class CallingThreadDispatcherModelSpec extends ActorModelSpec { //Can't handle this... } - @After def after { //remove the interrupted status since we are messing with interrupted exceptions. diff --git a/akka-actor/src/main/scala/akka/AkkaException.scala b/akka-actor/src/main/scala/akka/AkkaException.scala index 24831ff589..f623c3734f 100644 --- a/akka-actor/src/main/scala/akka/AkkaException.scala +++ b/akka-actor/src/main/scala/akka/AkkaException.scala @@ -6,7 +6,7 @@ package akka import akka.actor.newUuid import java.net.{ InetAddress, UnknownHostException } - + /** * Akka base Exception. Each Exception gets: *
    @@ -26,7 +26,7 @@ class AkkaException(message: String = "", cause: Throwable = null) extends Runti lazy val toLongString = "%s: %s\n[%s]\n%s".format(getClass.getName, message, uuid, stackTraceToString) - def this(msg:String) = this(msg, null); + def this(msg: String) = this(msg, null); def stackTraceToString = { val trace = getStackTrace diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index d988bc1819..6015272a74 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -80,28 +80,28 @@ case class MaximumNumberOfRestartsWithinTimeRangeReached( @BeanProperty lastExceptionCausingRestart: Throwable) extends LifeCycleMessage // Exceptions for Actors -class ActorStartException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause){ - def this(msg:String) = this(msg, null); +class ActorStartException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) { + def this(msg: String) = this(msg, null); } 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); } -class ActorKilledException private[akka] (message: String, cause: Throwable) extends AkkaException(message, cause){ +class ActorKilledException private[akka] (message: String, cause: Throwable) extends AkkaException(message, cause) { def this(msg: String) = this(msg, null); } class ActorInitializationException 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); } -class ActorTimeoutException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) { - def this(msg:String) = this(msg, null); +class ActorTimeoutException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) { + def this(msg: String) = this(msg, null); } -class InvalidMessageException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) { - def this(msg:String) = this(msg, null); +class InvalidMessageException private[akka] (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/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 58a592a7c0..f51fca48aa 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -9,13 +9,13 @@ import akka.dispatch._ import akka.config._ import akka.config.Supervision._ import akka.util._ -import akka.serialization.{Serializer, Serialization} +import akka.serialization.{ Serializer, Serialization } import ReflectiveAccess._ import ClusterModule._ import java.net.InetSocketAddress import java.util.concurrent.atomic.AtomicReference -import java.util.concurrent.{ScheduledFuture, ConcurrentHashMap, TimeUnit} -import java.util.{Map ⇒ JMap} +import java.util.concurrent.{ ScheduledFuture, ConcurrentHashMap, TimeUnit } +import java.util.{ Map ⇒ JMap } import scala.reflect.BeanProperty import scala.collection.immutable.Stack @@ -301,7 +301,7 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com * * If you would rather have an exception, check the reply(..) version. */ - def tryReply(message: Any): Boolean = channel.safe_!(message)(this) + def tryReply(message: Any): Boolean = channel.safe_!(message)(this) /** * Sets the dispatcher for this actor. Needs to be invoked before the actor is started. @@ -409,9 +409,9 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( - message: Any, - timeout: Long, - channel: UntypedChannel): Future[Any] + message: Any, + timeout: Long, + channel: UntypedChannel): Future[Any] protected[akka] def actorInstance: AtomicReference[Actor] @@ -444,7 +444,7 @@ trait ActorRef extends ActorRefShared with ForwardableChannel with java.lang.Com * * @author Jonas Bonér */ -class LocalActorRef private[akka](private[this] val actorFactory: () ⇒ Actor, val address: String) +class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, val address: String) extends ActorRef with ScalaActorRef { protected[akka] val guard = new ReentrantGuard @@ -480,12 +480,12 @@ class LocalActorRef private[akka](private[this] val actorFactory: () ⇒ Actor, "]") private val serializer: Serializer = - try { Serialization.serializerFor(this.getClass) } catch { case e: Exception => serializerErrorDueTo(e.toString)} + try { Serialization.serializerFor(this.getClass) } catch { case e: Exception ⇒ serializerErrorDueTo(e.toString) } private lazy val replicationStorage: Option[TransactionLog] = { import DeploymentConfig._ val replicationScheme = replicationSchemeFor(Deployer.deploymentFor(address)).getOrElse(Transient) - if(isReplicated(replicationScheme)) { + if (isReplicated(replicationScheme)) { if (isReplicatedWithTransactionLog(replicationScheme)) { EventHandler.debug(this, "Creating a transaction log for Actor [%s] with replication strategy [%s]".format(address, replicationScheme)) @@ -503,16 +503,16 @@ class LocalActorRef private[akka](private[this] val actorFactory: () ⇒ Actor, // used only for deserialization private[akka] def this( - __uuid: Uuid, - __address: String, - __timeout: Long, - __receiveTimeout: Option[Long], - __lifeCycle: LifeCycle, - __supervisor: Option[ActorRef], - __hotswap: Stack[PartialFunction[Any, Unit]], - __factory: () ⇒ Actor) = { + __uuid: Uuid, + __address: String, + __timeout: Long, + __receiveTimeout: Option[Long], + __lifeCycle: LifeCycle, + __supervisor: Option[ActorRef], + __hotswap: Stack[PartialFunction[Any, Unit]], + __factory: () ⇒ Actor) = { - this (__factory, __address) + this(__factory, __address) _uuid = __uuid timeout = __timeout @@ -674,12 +674,12 @@ class LocalActorRef private[akka](private[this] val actorFactory: () ⇒ Actor, dispatcher dispatchMessage new MessageInvocation(this, message, channel) protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( - message: Any, - timeout: Long, - channel: UntypedChannel): Future[Any] = { + message: Any, + timeout: Long, + channel: UntypedChannel): Future[Any] = { val future = channel match { case f: ActorPromise ⇒ f - case _ ⇒ new ActorPromise(timeout) + case _ ⇒ new ActorPromise(timeout) } dispatcher dispatchMessage new MessageInvocation(this, message, future) future @@ -704,8 +704,7 @@ class LocalActorRef private[akka](private[this] val actorFactory: () ⇒ Actor, throw e case e ⇒ handleExceptionInDispatch(e, messageHandle.message) - } - finally { + } finally { checkReceiveTimeout // Reschedule receive timeout } } catch { @@ -810,8 +809,7 @@ class LocalActorRef private[akka](private[this] val actorFactory: () ⇒ Actor, case e ⇒ EventHandler.error(e, this, "Exception in restart of Actor [%s]".format(toString)) false // an error or exception here should trigger a retry - } - finally { + } finally { currentMessage = null } @@ -852,7 +850,7 @@ class LocalActorRef private[akka](private[this] val actorFactory: () ⇒ Actor, // ========= PRIVATE FUNCTIONS ========= private[this] def newActor: Actor = { - import Actor.{actorRefInCreation ⇒ refStack} + import Actor.{ actorRefInCreation ⇒ refStack } val stackBefore = refStack.get refStack.set(stackBefore.push(this)) try { @@ -876,7 +874,7 @@ class LocalActorRef private[akka](private[this] val actorFactory: () ⇒ Actor, refStack.set(if (stackAfter.head eq null) stackAfter.pop.pop else stackAfter.pop) //pop null marker plus self } } match { - case null ⇒ throw new ActorInitializationException("Actor instance passed to ActorRef can not be 'null'") + case null ⇒ throw new ActorInitializationException("Actor instance passed to ActorRef can not be 'null'") case valid ⇒ valid } @@ -909,21 +907,20 @@ class LocalActorRef private[akka](private[this] val actorFactory: () ⇒ Actor, private def notifySupervisorWithMessage(notification: LifeCycleMessage) { // FIXME to fix supervisor restart of remote actor for oneway calls, inject a supervisor proxy that can send notification back to client - _supervisor.foreach { - sup ⇒ - if (sup.isShutdown) { - // if supervisor is shut down, game over for all linked actors - //Scoped stop all linked actors, to avoid leaking the 'i' val - { - val i = _linkedActors.values.iterator - while (i.hasNext) { - i.next.stop() - i.remove - } + _supervisor.foreach { sup ⇒ + if (sup.isShutdown) { + // if supervisor is shut down, game over for all linked actors + //Scoped stop all linked actors, to avoid leaking the 'i' val + { + val i = _linkedActors.values.iterator + while (i.hasNext) { + i.next.stop() + i.remove } - //Stop the actor itself - stop - } else sup ! notification // else notify supervisor + } + //Stop the actor itself + stop + } else sup ! notification // else notify supervisor } } @@ -993,11 +990,11 @@ object RemoteActorSystemMessage { * * @author Jonas Bonér */ -private[akka] case class RemoteActorRef private[akka]( - val remoteAddress: InetSocketAddress, - val address: String, - _timeout: Long, - loader: Option[ClassLoader]) +private[akka] case class RemoteActorRef private[akka] ( + val remoteAddress: InetSocketAddress, + val address: String, + _timeout: Long, + loader: Option[ClassLoader]) extends ActorRef with ScalaActorRef { ClusterModule.ensureEnabled() @@ -1009,22 +1006,22 @@ private[akka] case class RemoteActorRef private[akka]( def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = { val chSender = channel match { case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None + case _ ⇒ None } Actor.remote.send[Any](message, chSender, None, remoteAddress, timeout, true, this, loader) } def postMessageToMailboxAndCreateFutureResultWithTimeout( - message: Any, - timeout: Long, - channel: UntypedChannel): Future[Any] = { + message: Any, + timeout: Long, + channel: UntypedChannel): Future[Any] = { val chSender = channel match { case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None + case _ ⇒ None } val chFuture = channel match { case f: Promise[_] ⇒ Some(f.asInstanceOf[Promise[Any]]) - case _ ⇒ None + case _ ⇒ None } val future = Actor.remote.send[Any](message, chSender, chFuture, remoteAddress, timeout, false, this, loader) if (future.isDefined) ActorPromise(future.get) @@ -1174,7 +1171,7 @@ trait ScalaActorRef extends ActorRefShared with ForwardableChannel { if (msg eq null) None else msg.channel match { case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None + case _ ⇒ None } } @@ -1188,7 +1185,7 @@ trait ScalaActorRef extends ActorRefShared with ForwardableChannel { if (msg eq null) None else msg.channel match { case f: ActorPromise ⇒ Some(f) - case _ ⇒ None + case _ ⇒ None } } diff --git a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala index ed1eefcd7a..43d895d9a5 100644 --- a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala +++ b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala @@ -133,9 +133,9 @@ object DeploymentConfig { case Host("localhost") ⇒ Config.nodename case IP("0.0.0.0") ⇒ Config.nodename case IP("127.0.0.1") ⇒ Config.nodename - case Host(hostname) ⇒ throw new UnsupportedOperationException( + case Host(hostname) ⇒ throw new UnsupportedOperationException( "Specifying preferred node name by 'hostname' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") - case IP(address) ⇒ throw new UnsupportedOperationException( + case IP(address) ⇒ throw new UnsupportedOperationException( "Specifying preferred node name by 'IP address' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") } diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index b911af4c1a..b3e9c96e9e 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -454,7 +454,7 @@ trait FSM[S, D] extends ListenerManagement { if (generation == gen) { processMsg(StateTimeout, "state timeout") } - case t@Timer(name, msg, repeat, generation) ⇒ + case t @ Timer(name, msg, repeat, generation) ⇒ if ((timers contains name) && (timers(name).generation == generation)) { processMsg(msg, t) if (!repeat) { diff --git a/akka-actor/src/main/scala/akka/actor/Supervisor.scala b/akka-actor/src/main/scala/akka/actor/Supervisor.scala index 95532e8fb7..1463c1b6cb 100644 --- a/akka-actor/src/main/scala/akka/actor/Supervisor.scala +++ b/akka-actor/src/main/scala/akka/actor/Supervisor.scala @@ -13,7 +13,7 @@ import java.util.concurrent.{ CopyOnWriteArrayList, ConcurrentHashMap } import akka.config.Supervision._ class SupervisorException 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); } /** @@ -153,7 +153,7 @@ sealed class Supervisor(handler: FaultHandlingStrategy, maxRestartsHandler: (Act supervisor.link(actorRef) if (ClusterModule.isEnabled && registerAsRemoteService) Actor.remote.register(actorRef) - case supervisorConfig@SupervisorConfig(_, _, _) ⇒ // recursive supervisor configuration + case supervisorConfig @ SupervisorConfig(_, _, _) ⇒ // recursive supervisor configuration val childSupervisor = Supervisor(supervisorConfig) supervisor.link(childSupervisor.supervisor) _childSupervisors.add(childSupervisor) @@ -179,7 +179,7 @@ final class SupervisorActor private[akka] (handler: FaultHandlingStrategy, maxRe } def receive = { - case max@MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _) ⇒ maxRestartsHandler(self, max) + case max @ MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _) ⇒ maxRestartsHandler(self, max) case unknown ⇒ throw new SupervisorException( "SupervisorActor can not respond to messages.\n\tUnknown message [" + unknown + "]") } diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 9ffff54456..6c0da81373 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -11,7 +11,7 @@ import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, import akka.util.{ Duration } import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar } import com.sun.xml.internal.ws.developer.MemberSubmissionAddressing.Validation -import akka.serialization.{Serializer, Serialization} +import akka.serialization.{ Serializer, Serialization } //TODO Document this class, not only in Scaladoc, but also in a dedicated typed-actor.rst, for both java and scala /** @@ -90,12 +90,12 @@ object TypedActor { } catch { case i: InvocationTargetException ⇒ throw i.getTargetException } private def writeReplace(): AnyRef = parameters match { - case null => SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, null, null) - case ps if ps.length == 0 => SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, Array[Serializer.Identifier](), Array[Array[Byte]]()) - case ps => + case null ⇒ SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, null, null) + case ps if ps.length == 0 ⇒ SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, Array[Serializer.Identifier](), Array[Array[Byte]]()) + case ps ⇒ val serializers: Array[Serializer] = ps map Serialization.findSerializerFor val serializedParameters: Array[Array[Byte]] = Array.ofDim[Array[Byte]](serializers.length) - for(i <- 0 until serializers.length) + for (i ← 0 until serializers.length) serializedParameters(i) = serializers(i) toBinary parameters(i) //Mutable for the sake of sanity SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, serializers.map(_.identifier), serializedParameters) @@ -110,11 +110,11 @@ object TypedActor { //TODO Possible optimization is to special encode the parameter-types to conserve space private def readResolve(): AnyRef = { MethodCall(ownerType.getDeclaredMethod(methodName, parameterTypes: _*), serializedParameters match { - case null => null - case a if a.length == 0 => Array[AnyRef]() - case a => + case null ⇒ null + case a if a.length == 0 ⇒ Array[AnyRef]() + case a ⇒ val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity - for(i <- 0 until a.length) + for (i ← 0 until a.length) deserializedParameters(i) = Serialization.serializerByIdentity(serializerIdentifiers(i)).fromBinary(serializedParameters(i)) deserializedParameters diff --git a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala index f59d2ff4ab..e36492879c 100644 --- a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala +++ b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala @@ -16,8 +16,7 @@ import akka.AkkaException import com.eaio.uuid.UUID import java.net.InetSocketAddress -import java.util.concurrent.{ ConcurrentSkipListSet} - +import java.util.concurrent.{ ConcurrentSkipListSet } class ClusterException(message: String) extends AkkaException(message) diff --git a/akka-actor/src/main/scala/akka/config/Config.scala b/akka-actor/src/main/scala/akka/config/Config.scala index 40f84a30d2..5a38f3cf2b 100644 --- a/akka-actor/src/main/scala/akka/config/Config.scala +++ b/akka-actor/src/main/scala/akka/config/Config.scala @@ -11,11 +11,11 @@ import java.net.InetAddress import com.eaio.uuid.UUID class ConfigurationException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { - def this(msg:String) = this(msg, null); + 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); +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/config/ConfigParser.scala b/akka-actor/src/main/scala/akka/config/ConfigParser.scala index 91f40df096..39b961a24d 100644 --- a/akka-actor/src/main/scala/akka/config/ConfigParser.scala +++ b/akka-actor/src/main/scala/akka/config/ConfigParser.scala @@ -66,9 +66,9 @@ class ConfigParser(var prefix: String = "", map: mutable.Map[String, Any] = muta def parse(in: String): Map[String, Any] = { parseAll(root, in) match { - case Success(result, _) ⇒ map.toMap - case x@Failure(msg, _) ⇒ throw new ConfigurationException(x.toString) - case x@Error(msg, _) ⇒ throw new ConfigurationException(x.toString) + case Success(result, _) ⇒ map.toMap + case x @ Failure(msg, _) ⇒ throw new ConfigurationException(x.toString) + case x @ Error(msg, _) ⇒ throw new ConfigurationException(x.toString) } } } diff --git a/akka-actor/src/main/scala/akka/config/Configuration.scala b/akka-actor/src/main/scala/akka/config/Configuration.scala index b32c4eeb72..32d7d79418 100644 --- a/akka-actor/src/main/scala/akka/config/Configuration.scala +++ b/akka-actor/src/main/scala/akka/config/Configuration.scala @@ -148,11 +148,10 @@ class Configuration(val map: Map[String, Any]) { getDouble(key).getOrElse(outputIfDesiredAndReturnInput(key, defaultValue)) def getBoolean(key: String): Option[Boolean] = { - getString(key) flatMap { - s ⇒ - val isTrue = trueValues.contains(s) - if (!isTrue && !falseValues.contains(s)) None - else Some(isTrue) + getString(key) flatMap { s ⇒ + val isTrue = trueValues.contains(s) + if (!isTrue && !falseValues.contains(s)) None + else Some(isTrue) } } @@ -165,7 +164,7 @@ class Configuration(val map: Map[String, Any]) { getBoolean(key, defaultValue) def apply(key: String): String = getString(key) match { - case None ⇒ throw new ConfigurationException("undefined config: " + key) + case None ⇒ throw new ConfigurationException("undefined config: " + key) case Some(v) ⇒ v } @@ -179,7 +178,7 @@ class Configuration(val map: Map[String, Any]) { def getSection(name: String): Option[Configuration] = { val l = name.length + 1 - val pattern = name+"." + val pattern = name + "." val m = map.collect { case (k, v) if k.startsWith(pattern) ⇒ (k.substring(l), v) } diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 5097f69aa0..fbd32c580b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -194,7 +194,7 @@ trait ExecutableMailbox extends Runnable { self: MessageQueue ⇒ final def run = { try { processMailbox() } catch { - case ie: InterruptedException => Thread.currentThread().interrupt() //Restore interrupt + case ie: InterruptedException ⇒ Thread.currentThread().interrupt() //Restore interrupt } finally { dispatcherLock.unlock() if (!self.isEmpty) diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index 64c0c5afb2..8384fdf949 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -200,7 +200,7 @@ object Dispatchers { case Right(clazz) ⇒ ReflectiveAccess.createInstance[MessageDispatcherConfigurator](clazz, Array[Class[_]](), Array[AnyRef]()) match { case Right(configurator) ⇒ configurator - case Left(exception)⇒ + case Left(exception) ⇒ throw new IllegalArgumentException( "Cannot instantiate MessageDispatcherConfigurator type [%s], make sure it has a default no-args constructor" format fqn, exception) } diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 5c858a0905..2d6a155afd 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -103,8 +103,7 @@ object Futures { case e: Exception ⇒ EventHandler.error(e, this, e.getMessage) result completeWithException e - } - finally { + } finally { results.clear } } @@ -265,7 +264,7 @@ object Future { */ def flow[A](body: ⇒ A @cps[Future[Any]], timeout: Long = Actor.TIMEOUT): Future[A] = { val future = Promise[A](timeout) - (reset(future.asInstanceOf[Promise[Any]].completeWithResult(body)): Future[Any]) onException { case e => future completeWithException e } + (reset(future.asInstanceOf[Promise[Any]].completeWithResult(body)): Future[Any]) onException { case e ⇒ future completeWithException e } future } } diff --git a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala index 625c7417cf..7ed4c9e89d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala @@ -34,8 +34,7 @@ final case class FutureInvocation[T](future: Promise[T], function: () ⇒ T, cle case e ⇒ EventHandler.error(e, this, e.getMessage) Left(e) - } - finally { + } finally { cleanup() }) } diff --git a/akka-actor/src/main/scala/akka/event/EventHandler.scala b/akka-actor/src/main/scala/akka/event/EventHandler.scala index 4b1fc2e1aa..c6ea69dcc0 100644 --- a/akka-actor/src/main/scala/akka/event/EventHandler.scala +++ b/akka-actor/src/main/scala/akka/event/EventHandler.scala @@ -8,7 +8,7 @@ import akka.actor._ import akka.dispatch.Dispatchers import akka.config.Config._ import akka.config.ConfigurationException -import akka.util.{ListenerManagement, ReflectiveAccess} +import akka.util.{ ListenerManagement, ReflectiveAccess } import akka.serialization._ import akka.AkkaException @@ -95,10 +95,10 @@ object EventHandler extends ListenerManagement { @volatile var level: Int = config.getString("akka.event-handler-level", "INFO") match { - case "ERROR" ⇒ ErrorLevel + case "ERROR" ⇒ ErrorLevel case "WARNING" ⇒ WarningLevel - case "INFO" ⇒ InfoLevel - case "DEBUG" ⇒ DebugLevel + case "INFO" ⇒ InfoLevel + case "DEBUG" ⇒ DebugLevel case unknown ⇒ throw new ConfigurationException( "Configuration option 'akka.event-handler-level' is invalid [" + unknown + "]") } @@ -106,22 +106,21 @@ object EventHandler extends ListenerManagement { def start() { try { val defaultListeners = config.getList("akka.event-handlers") match { - case Nil ⇒ "akka.event.EventHandler$DefaultListener" :: Nil + case Nil ⇒ "akka.event.EventHandler$DefaultListener" :: Nil case listeners ⇒ listeners } - defaultListeners foreach { - listenerName ⇒ - try { - ReflectiveAccess.getClassFor[Actor](listenerName) match { - case Right(actorClass) ⇒ addListener(Actor.localActorOf(actorClass).start()) - case Left(exception) ⇒ throw exception - } - } catch { - case e: Exception ⇒ - throw new ConfigurationException( - "Event Handler specified in config can't be loaded [" + listenerName + - "] due to [" + e.toString + "]", e) + defaultListeners foreach { listenerName ⇒ + try { + ReflectiveAccess.getClassFor[Actor](listenerName) match { + case Right(actorClass) ⇒ addListener(Actor.localActorOf(actorClass).start()) + case Left(exception) ⇒ throw exception } + } catch { + case e: Exception ⇒ + throw new ConfigurationException( + "Event Handler specified in config can't be loaded [" + listenerName + + "] due to [" + e.toString + "]", e) + } } info(this, "Starting up EventHandler") } catch { @@ -146,7 +145,7 @@ object EventHandler extends ListenerManagement { notifyListeners(event) } - def notify[T <: Event : ClassManifest](event: ⇒ T) { + def notify[T <: Event: ClassManifest](event: ⇒ T) { if (level >= levelFor(classManifest[T].erasure.asInstanceOf[Class[_ <: Event]])) notifyListeners(event) } @@ -182,7 +181,6 @@ object EventHandler extends ListenerManagement { if (level >= InfoLevel) notifyListeners(Info(instance, message)) } - def debug(instance: AnyRef, message: ⇒ String) { if (level >= DebugLevel) notifyListeners(Debug(instance, message)) } @@ -196,7 +194,7 @@ object EventHandler extends ListenerManagement { def isDebugEnabled = level >= DebugLevel def stackTraceFor(e: Throwable) = { - import java.io.{StringWriter, PrintWriter} + import java.io.{ StringWriter, PrintWriter } val sw = new StringWriter val pw = new PrintWriter(sw) e.printStackTrace(pw) @@ -223,7 +221,7 @@ object EventHandler extends ListenerManagement { def timestamp = dateFormat.format(new Date) def receive = { - case event@Error(cause, instance, message) ⇒ + case event @ Error(cause, instance, message) ⇒ println(error.format( timestamp, event.thread.getName, @@ -231,21 +229,21 @@ object EventHandler extends ListenerManagement { message, stackTraceFor(cause))) - case event@Warning(instance, message) ⇒ + case event @ Warning(instance, message) ⇒ println(warning.format( timestamp, event.thread.getName, instance.getClass.getSimpleName, message)) - case event@Info(instance, message) ⇒ + case event @ Info(instance, message) ⇒ println(info.format( timestamp, event.thread.getName, instance.getClass.getSimpleName, message)) - case event@Debug(instance, message) ⇒ + case event @ Debug(instance, message) ⇒ println(debug.format( timestamp, event.thread.getName, diff --git a/akka-actor/src/main/scala/akka/routing/Pool.scala b/akka-actor/src/main/scala/akka/routing/Pool.scala index 6bfd4c18b3..2e930e912e 100644 --- a/akka-actor/src/main/scala/akka/routing/Pool.scala +++ b/akka-actor/src/main/scala/akka/routing/Pool.scala @@ -38,7 +38,7 @@ object ActorPool { trait ActorPool { /** * Adds a new actor to the pool. The DefaultActorPool implementation will start and link (supervise) this actor. - * This method is invoked whenever the pool determines it must boost capacity. + * This method is invoked whenever the pool determines it must boost capacity. * @return A new actor for the pool */ def instance(): ActorRef @@ -48,7 +48,7 @@ trait ActorPool { * @param _delegates The current sequence of pooled actors * @return the number of delegates by which the pool should be adjusted (positive, negative or zero) */ - def capacity(delegates: Seq[ActorRef]): Int + def capacity(delegates: Seq[ActorRef]): Int /** * Provides the results of the selector, one or more actors, to which an incoming message is forwarded. * This method returns an iterator since a selector might return more than one actor to handle the message. @@ -71,7 +71,7 @@ trait ActorPoolSupervisionConfig { /** * Provides a default implementation of the supervision configuration by - * defining a One-for-One fault handling strategy, trapping exceptions, + * defining a One-for-One fault handling strategy, trapping exceptions, * limited to 5 retries within 1 second. * * This is just a basic strategy and implementors are encouraged to define @@ -89,7 +89,7 @@ trait DefaultActorPoolSupervisionConfig extends ActorPoolSupervisionConfig { * are added or existing ones are removed. Removed actors are sent the PoisonPill message. * New actors are automatically started and linked. The pool supervises the actors and will * use the fault handling strategy specified by the mixed-in ActorPoolSupervisionConfig. - * Pooled actors may be any lifecycle. If you're testing pool sizes during runtime, take a + * Pooled actors may be any lifecycle. If you're testing pool sizes during runtime, take a * look at the unit tests... Any delegate with a Permanent lifecycle will be * restarted and the pool size will be level with what it was prior to the fault. In just * about every other case, e.g. the delegates are Temporary or the delegate cannot be @@ -99,9 +99,9 @@ trait DefaultActorPoolSupervisionConfig extends ActorPoolSupervisionConfig { * * Second, invokes the pool's selector that returns a list of delegates that are to receive * the incoming message. Selectors may return more than one actor. If partialFill - * is true then it might also the case that fewer than number of desired actors will be + * is true then it might also the case that fewer than number of desired actors will be * returned. - * + * * Lastly, routes by forwarding, the incoming message to each delegate in the selected set. */ trait DefaultActorPool extends ActorPool { this: Actor with ActorPoolSupervisionConfig ⇒ @@ -126,12 +126,12 @@ trait DefaultActorPool extends ActorPool { this: Actor with ActorPoolSupervision self tryReply Stats(_delegates length) case MaximumNumberOfRestartsWithinTimeRangeReached(victim, _, _, _) ⇒ _delegates = _delegates filterNot { _.uuid == victim.uuid } - case Death(victim, _) => + case Death(victim, _) ⇒ _delegates = _delegates filterNot { _.uuid == victim.uuid } case msg ⇒ resizeIfAppropriate() - select(_delegates) foreach { _ forward msg } + select(_delegates) foreach { _ forward msg } } private def resizeIfAppropriate() { @@ -160,7 +160,7 @@ trait DefaultActorPool extends ActorPool { this: Actor with ActorPoolSupervision /** * Selectors - * + * * These traits define how, when a message needs to be routed, delegate(s) are chosen from the pool. * Note that it's acceptable to return more than one actor to handle a given message. */ @@ -211,7 +211,7 @@ trait RoundRobinSelector { /** * Capacitors - * + * * These traits define how to alter the size of the pool according to some desired behavior. * Capacitors are required (minimally) by the pool to establish bounds on the number of delegates * that may exist in the pool. @@ -269,7 +269,7 @@ trait ActiveFuturesPressureCapacitor { } /** - * + * */ trait CapacityStrategy { import ActorPool._ @@ -283,7 +283,7 @@ trait CapacityStrategy { def pressure(delegates: Seq[ActorRef]): Int /** * This method can be used to smooth the response of the capacitor by considering - * the current pressure and current capacity. + * the current pressure and current capacity. */ def filter(pressure: Int, capacity: Int): Int @@ -299,7 +299,7 @@ trait FixedCapacityStrategy extends FixedSizeCapacitor * Use this trait to setup a pool that may have a variable number of * delegates but always within an established upper and lower limit. * - * If mix this into your pool implementation, you must also provide a + * If mix this into your pool implementation, you must also provide a * PressureCapacitor and a Filter. */ trait BoundedCapacityStrategy extends CapacityStrategy with BoundedCapacitor diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index e065bbc24c..7d3375aff9 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -11,7 +11,6 @@ import akka.actor.{ ActorRef, Actor } import akka.AkkaException import akka.util.ReflectiveAccess - case class NoSerializerFoundException(m: String) extends AkkaException(m) /** @@ -22,18 +21,18 @@ object Serialization { //TODO document me def serialize(o: AnyRef): Either[Exception, Array[Byte]] = - try { Right(findSerializerFor(o).toBinary(o)) } catch { case e: Exception => Left(e) } + try { Right(findSerializerFor(o).toBinary(o)) } catch { case e: Exception ⇒ Left(e) } //TODO document me def deserialize( bytes: Array[Byte], clazz: Class[_], classLoader: Option[ClassLoader]): Either[Exception, AnyRef] = - try { Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader)) } catch { case e: Exception => Left(e) } + try { Right(serializerFor(clazz).fromBinary(bytes, Some(clazz), classLoader)) } catch { case e: Exception ⇒ Left(e) } def findSerializerFor(o: AnyRef): Serializer = o match { - case null => NullSerializer - case other => serializerFor(other.getClass) + case null ⇒ NullSerializer + case other ⇒ serializerFor(other.getClass) } //TODO document me @@ -50,12 +49,12 @@ object Serialization { if (bindings.isEmpty) Left(NoSerializerFoundException("No mapping serializer found for " + cl)) else { - bindings find { - case (clazzName, _) ⇒ - getClassFor(clazzName) match { - case Right(clazz) ⇒ clazz.isAssignableFrom(cl) - case _ ⇒ false - } + bindings find { + case (clazzName, _) ⇒ + getClassFor(clazzName) match { + case Right(clazz) ⇒ clazz.isAssignableFrom(cl) + case _ ⇒ false + } } map { case (_, ser) ⇒ serializerOf(ser) } getOrElse Left(NoSerializerFoundException("No mapping serializer found for " + cl)) @@ -69,20 +68,20 @@ object Serialization { */ val serializers: Map[String, Serializer] = config.getSection("akka.actor.serializers") - .map(_.map) - .getOrElse(Map()) - .foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) { - case (result, (k: String, v: String)) => result + (k -> serializerOf(v).fold(throw _, identity)) - case (result, _) => result - } + .map(_.map) + .getOrElse(Map()) + .foldLeft(Map[String, Serializer]("default" -> akka.serialization.JavaSerializer)) { + case (result, (k: String, v: String)) ⇒ result + (k -> serializerOf(v).fold(throw _, identity)) + case (result, _) ⇒ result + } /** * bindings is a Map whose keys = FQN of class that is serializable and values = the alias of the serializer to be used */ val bindings: Map[String, String] = config.getSection("akka.actor.serialization-bindings") map { - _.map.foldLeft(Map[String,String]()) { - case (result, (k: String, vs: List[_])) => result ++ (vs collect { case v: String => (v, k) }) //All keys which are lists, take the Strings from them and Map them - case (result, _) => result //For any other values, just skip them, TODO: print out warnings? + _.map.foldLeft(Map[String, String]()) { + case (result, (k: String, vs: List[_])) ⇒ result ++ (vs collect { case v: String ⇒ (v, k) }) //All keys which are lists, take the Strings from them and Map them + case (result, _) ⇒ result //For any other values, just skip them, TODO: print out warnings? } } getOrElse Map() @@ -95,5 +94,5 @@ object Serialization { * Maps from a Serializer.Identifier (Byte) to a Serializer instance (optimization) */ val serializerByIdentity: Map[Serializer.Identifier, Serializer] = - Map(NullSerializer.identifier -> NullSerializer) ++ serializers map { case (_, v) => (v.identifier,v) } + Map(NullSerializer.identifier -> NullSerializer) ++ serializers map { case (_, v) ⇒ (v.identifier, v) } } diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index aa57c4b47d..893e974859 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -38,7 +38,7 @@ object NullSerializer extends NullSerializer class JavaSerializer extends Serializer { - def identifier = 1:Byte + def identifier = 1: Byte def toBinary(o: AnyRef): Array[Byte] = { val bos = new ByteArrayOutputStream @@ -63,7 +63,7 @@ class NullSerializer extends Serializer { val nullAsBytes = Array[Byte]() - def identifier = 0:Byte + def identifier = 0: Byte def toBinary(o: AnyRef) = nullAsBytes def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]] = None, classLoader: Option[ClassLoader] = None): AnyRef = null } diff --git a/akka-camel/src/main/scala/akka/camel/component/ActorComponent.scala b/akka-camel/src/main/scala/akka/camel/component/ActorComponent.scala index 6fdbd37149..c591361bfb 100644 --- a/akka-camel/src/main/scala/akka/camel/component/ActorComponent.scala +++ b/akka-camel/src/main/scala/akka/camel/component/ActorComponent.scala @@ -16,7 +16,7 @@ import akka.actor._ import akka.camel.{ Ack, Failure, Message } import akka.camel.CamelMessageConversion.toExchangeAdapter import scala.reflect.BeanProperty -import akka.dispatch.{FutureTimeoutException, Promise, MessageInvocation, MessageDispatcher} +import akka.dispatch.{ FutureTimeoutException, Promise, MessageInvocation, MessageDispatcher } /** * @author Martin Krasser @@ -170,7 +170,7 @@ class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) with Asyn } } - private def sendSync(exchange: Exchange) = { + private def sendSync(exchange: Exchange) = { val actor = target(exchange) val result: Any = try { (actor ? requestFor(exchange)).as[Any] } catch { case e ⇒ Some(Failure(e)) } @@ -181,7 +181,7 @@ class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) with Asyn case None ⇒ throw new TimeoutException("timeout (%d ms) while waiting response from %s" format (actor.timeout, ep.getEndpointUri)) } - } + } private def sendAsync(exchange: Exchange, sender: Option[ActorRef] = None) = target(exchange).!(requestFor(exchange))(sender) diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 980e5ef9e5..1d3a984ff1 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -6,14 +6,14 @@ package akka.cluster import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event._ import org.apache.zookeeper.data.Stat -import org.apache.zookeeper.recipes.lock.{WriteLock, LockListener} +import org.apache.zookeeper.recipes.lock.{ WriteLock, LockListener } import org.I0Itec.zkclient._ import org.I0Itec.zkclient.serialize._ import org.I0Itec.zkclient.exception._ -import java.util.{List ⇒ JList} -import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} +import java.util.{ List ⇒ JList } +import java.util.concurrent.atomic.{ AtomicBoolean, AtomicReference } import java.net.InetSocketAddress import javax.management.StandardMBean @@ -29,15 +29,15 @@ import Status._ import DeploymentConfig._ import akka.event.EventHandler -import akka.dispatch.{Dispatchers, Future} +import akka.dispatch.{ Dispatchers, Future } import akka.remoteinterface._ import akka.routing.RouterType -import akka.config.{Config, Supervision} +import akka.config.{ Config, Supervision } import Supervision._ import Config._ -import akka.serialization.{Serialization, Serializer, ActorSerialization} +import akka.serialization.{ Serialization, Serializer, ActorSerialization } import ActorSerialization._ import akka.serialization.Compression.LZF @@ -49,7 +49,7 @@ import RemoteDaemonMessageType._ import com.eaio.uuid.UUID import com.google.protobuf.ByteString -import java.util.concurrent.{CopyOnWriteArrayList, Callable, ConcurrentHashMap} +import java.util.concurrent.{ CopyOnWriteArrayList, Callable, ConcurrentHashMap } // FIXME add watch for each node that when the entry for the node is removed then the node shuts itself down @@ -181,17 +181,17 @@ object Cluster { private def nodename: String = properties.get("akka.cluster.nodename") match { case Some(uberride) ⇒ uberride - case None ⇒ Config.nodename + case None ⇒ Config.nodename } private def hostname: String = properties.get("akka.cluster.hostname") match { case Some(uberride) ⇒ uberride - case None ⇒ Config.hostname + case None ⇒ Config.hostname } private def port: Int = properties.get("akka.cluster.port") match { case Some(uberride) ⇒ uberride.toInt - case None ⇒ Config.remoteServerPort + case None ⇒ Config.remoteServerPort } val defaultZooKeeperSerializer = new SerializableSerializer @@ -329,12 +329,12 @@ object Cluster { * * @author Jonas Bonér */ -class DefaultClusterNode private[akka]( - val nodeAddress: NodeAddress, - val hostname: String = Config.hostname, - val port: Int = Config.remoteServerPort, - val zkServerAddresses: String, - val serializer: ZkSerializer) extends ErrorHandler with ClusterNode { +class DefaultClusterNode private[akka] ( + val nodeAddress: NodeAddress, + val hostname: String = Config.hostname, + val port: Int = Config.remoteServerPort, + val zkServerAddresses: String, + val serializer: ZkSerializer) extends ErrorHandler with ClusterNode { self ⇒ if ((hostname eq null) || hostname == "") throw new NullPointerException("Host name must not be null or empty string") @@ -349,7 +349,7 @@ class DefaultClusterNode private[akka]( def receive = { case RemoteClientError(cause, client, address) ⇒ client.shutdownClientModule() case RemoteClientDisconnected(client, address) ⇒ client.shutdownClientModule() - case _ ⇒ //ignore other + case _ ⇒ //ignore other } }, "akka.cluster.RemoteClientLifeCycleListener").start() @@ -475,7 +475,6 @@ class DefaultClusterNode private[akka]( EventHandler.info(this, "Cluster node [%s] started successfully".format(nodeAddress)) } - def shutdown() { def shutdownNode() { ignore[ZkNoNodeException](zkClient.deleteRecursive(membershipNodePath)) @@ -696,12 +695,12 @@ class DefaultClusterNode private[akka]( * available durable store. */ def store( - actorAddress: String, - actorFactory: () ⇒ ActorRef, - replicationFactor: Int, - replicationScheme: ReplicationScheme, - serializeMailbox: Boolean, - serializer: Serializer): ClusterNode = if (isConnected.isOn) { + actorAddress: String, + actorFactory: () ⇒ ActorRef, + replicationFactor: Int, + replicationScheme: ReplicationScheme, + serializeMailbox: Boolean, + serializer: Serializer): ClusterNode = if (isConnected.isOn) { EventHandler.debug(this, "Storing actor with address [%s] in cluster".format(actorAddress)) @@ -730,7 +729,7 @@ class DefaultClusterNode private[akka]( } } }) match { - case Left(path) ⇒ path + case Left(path) ⇒ path case Right(exception) ⇒ actorAddressRegistryPath } } @@ -817,7 +816,7 @@ class DefaultClusterNode private[akka]( val actorFactory = Serialization.deserialize(actorFactoryBytes, classOf[() ⇒ LocalActorRef], None) match { - case Left(error) ⇒ throw error + case Left(error) ⇒ throw error case Right(instance) ⇒ instance.asInstanceOf[() ⇒ LocalActorRef] } @@ -897,12 +896,11 @@ class DefaultClusterNode private[akka]( val command = builder.build - nodes foreach { - node ⇒ - nodeConnections.get(node) foreach { - case (_, connection) ⇒ - sendCommandToNode(connection, command, async = false) - } + nodes foreach { node ⇒ + nodeConnections.get(node) foreach { + case (_, connection) ⇒ + sendCommandToNode(connection, command, async = false) + } } } } @@ -939,13 +937,12 @@ class DefaultClusterNode private[akka]( if (isConnected.isOn) { ignore[ZkNoNodeException](zkClient.delete(actorAddressToNodesPathFor(actorAddress, nodeAddress.nodeName))) - uuidsForActorAddress(actorAddress) foreach { - uuid ⇒ - EventHandler.debug(this, - "Releasing actor [%s] with UUID [%s] after usage".format(actorAddress, uuid)) + uuidsForActorAddress(actorAddress) foreach { uuid ⇒ + EventHandler.debug(this, + "Releasing actor [%s] with UUID [%s] after usage".format(actorAddress, uuid)) - ignore[ZkNoNodeException](zkClient.deleteRecursive(nodeToUuidsPathFor(nodeAddress.nodeName, uuid))) - ignore[ZkNoNodeException](zkClient.delete(actorUuidRegistryRemoteAddressPathFor(uuid))) + ignore[ZkNoNodeException](zkClient.deleteRecursive(nodeToUuidsPathFor(nodeAddress.nodeName, uuid))) + ignore[ZkNoNodeException](zkClient.delete(actorUuidRegistryRemoteAddressPathFor(uuid))) } } } @@ -963,11 +960,10 @@ class DefaultClusterNode private[akka]( .setActorAddress(actorAddress) .build - nodesForActorsInUseWithAddress(actorAddress) foreach { - node ⇒ - nodeConnections.get(node) foreach { - case (_, connection) ⇒ sendCommandToNode(connection, command, async = true) - } + nodesForActorsInUseWithAddress(actorAddress) foreach { node ⇒ + nodeConnections.get(node) foreach { + case (_, connection) ⇒ sendCommandToNode(connection, command, async = true) + } } } } @@ -1205,7 +1201,7 @@ class DefaultClusterNode private[akka]( } } }) match { - case Left(_) ⇒ /* do nothing */ + case Left(_) ⇒ /* do nothing */ case Right(exception) ⇒ throw exception } } @@ -1293,7 +1289,6 @@ class DefaultClusterNode private[akka]( private[cluster] def actorAddressToUuidsPathFor(actorAddress: String, uuid: UUID): String = "%s/%s".format(actorAddressToUuidsPathFor(actorAddress), uuid) - /** * Returns a random set with node names of size 'replicationFactor'. * Default replicationFactor is 0, which returns the empty Set. @@ -1364,16 +1359,15 @@ class DefaultClusterNode private[akka]( * @returns a Map with the remote socket addresses to of disconnected node connections */ private[cluster] def connectToAllNewlyArrivedMembershipNodesInCluster( - newlyConnectedMembershipNodes: Traversable[String], - newlyDisconnectedMembershipNodes: Traversable[String]): Map[String, InetSocketAddress] = { + newlyConnectedMembershipNodes: Traversable[String], + newlyDisconnectedMembershipNodes: Traversable[String]): Map[String, InetSocketAddress] = { // cache the disconnected connections in a map, needed for fail-over of these connections later var disconnectedConnections = Map.empty[String, InetSocketAddress] - newlyDisconnectedMembershipNodes foreach { - node ⇒ - disconnectedConnections += (node -> (nodeConnections(node) match { - case (address, _) ⇒ address - })) + newlyDisconnectedMembershipNodes foreach { node ⇒ + disconnectedConnections += (node -> (nodeConnections(node) match { + case (address, _) ⇒ address + })) } if (connectToAllNewlyArrivedMembershipNodesInClusterLock.compareAndSet(false, true)) { @@ -1382,20 +1376,18 @@ class DefaultClusterNode private[akka]( newlyDisconnectedMembershipNodes foreach (nodeConnections.remove(_)) // add connections newly arrived nodes - newlyConnectedMembershipNodes foreach { - node ⇒ - if (!nodeConnections.contains(node)) { - // only connect to each replica once + newlyConnectedMembershipNodes foreach { node ⇒ + if (!nodeConnections.contains(node)) { + // only connect to each replica once - remoteSocketAddressForNode(node) foreach { - address ⇒ - EventHandler.debug(this, - "Setting up connection to node with nodename [%s] and address [%s]".format(node, address)) + remoteSocketAddressForNode(node) foreach { address ⇒ + EventHandler.debug(this, + "Setting up connection to node with nodename [%s] and address [%s]".format(node, address)) - val clusterDaemon = Actor.remote.actorFor(RemoteClusterDaemon.Address, address.getHostName, address.getPort).start() - nodeConnections.put(node, (address, clusterDaemon)) - } + val clusterDaemon = Actor.remote.actorFor(RemoteClusterDaemon.Address, address.getHostName, address.getPort).start() + nodeConnections.put(node, (address, clusterDaemon)) } + } } } finally { connectToAllNewlyArrivedMembershipNodesInClusterLock.set(false) @@ -1442,87 +1434,85 @@ class DefaultClusterNode private[akka]( } private[cluster] def migrateActorsOnFailedNodes( - failedNodes: List[String], - currentClusterNodes: List[String], - oldClusterNodes: List[String], - disconnectedConnections: Map[String, InetSocketAddress]) { + failedNodes: List[String], + currentClusterNodes: List[String], + oldClusterNodes: List[String], + disconnectedConnections: Map[String, InetSocketAddress]) { - failedNodes.foreach { - failedNodeName ⇒ + failedNodes.foreach { failedNodeName ⇒ - val failedNodeAddress = NodeAddress(nodeAddress.clusterName, failedNodeName) + val failedNodeAddress = NodeAddress(nodeAddress.clusterName, failedNodeName) - val myIndex = oldClusterNodes.indexWhere(_.endsWith(nodeAddress.nodeName)) - val failedNodeIndex = oldClusterNodes.indexWhere(_ == failedNodeName) + val myIndex = oldClusterNodes.indexWhere(_.endsWith(nodeAddress.nodeName)) + val failedNodeIndex = oldClusterNodes.indexWhere(_ == failedNodeName) - // Migrate to the successor of the failed node (using a sorted circular list of the node names) - if ((failedNodeIndex == 0 && myIndex == oldClusterNodes.size - 1) || // No leftmost successor exists, check the tail - (failedNodeIndex == myIndex + 1)) { - // Am I the leftmost successor? + // Migrate to the successor of the failed node (using a sorted circular list of the node names) + if ((failedNodeIndex == 0 && myIndex == oldClusterNodes.size - 1) || // No leftmost successor exists, check the tail + (failedNodeIndex == myIndex + 1)) { + // Am I the leftmost successor? - // Takes the lead of migrating the actors. Not all to this node. - // All to this node except if the actor already resides here, then pick another node it is not already on. + // Takes the lead of migrating the actors. Not all to this node. + // All to this node except if the actor already resides here, then pick another node it is not already on. - // Yes I am the node to migrate the actor to (can only be one in the cluster) - val actorUuidsForFailedNode = zkClient.getChildren(nodeToUuidsPathFor(failedNodeName)).toList + // Yes I am the node to migrate the actor to (can only be one in the cluster) + val actorUuidsForFailedNode = zkClient.getChildren(nodeToUuidsPathFor(failedNodeName)).toList - actorUuidsForFailedNode.foreach { - uuidAsString ⇒ - EventHandler.debug(this, - "Cluster node [%s] has failed, migrating actor with UUID [%s] to [%s]" - .format(failedNodeName, uuidAsString, nodeAddress.nodeName)) + actorUuidsForFailedNode.foreach { uuidAsString ⇒ + EventHandler.debug(this, + "Cluster node [%s] has failed, migrating actor with UUID [%s] to [%s]" + .format(failedNodeName, uuidAsString, nodeAddress.nodeName)) - val uuid = uuidFrom(uuidAsString) - val actorAddress = actorAddressForUuid(uuid).getOrElse( - throw new IllegalStateException("No actor address found for UUID [" + uuidAsString + "]")) + val uuid = uuidFrom(uuidAsString) + val actorAddress = actorAddressForUuid(uuid).getOrElse( + throw new IllegalStateException("No actor address found for UUID [" + uuidAsString + "]")) - val migrateToNodeAddress = - if (isInUseOnNode(actorAddress)) { - // already in use on this node, pick another node to instantiate the actor on - val replicaNodesForActor = nodesForActorsInUseWithAddress(actorAddress) - val nodesAvailableForMigration = (currentClusterNodes.toSet diff failedNodes.toSet) diff replicaNodesForActor.toSet + val migrateToNodeAddress = + if (isInUseOnNode(actorAddress)) { + // already in use on this node, pick another node to instantiate the actor on + val replicaNodesForActor = nodesForActorsInUseWithAddress(actorAddress) + val nodesAvailableForMigration = (currentClusterNodes.toSet diff failedNodes.toSet) diff replicaNodesForActor.toSet - if (nodesAvailableForMigration.isEmpty) throw new ClusterException( - "Can not migrate actor to new node since there are not any available nodes left. " + - "(However, the actor already has >1 replica in cluster, so we are ok)") + if (nodesAvailableForMigration.isEmpty) throw new ClusterException( + "Can not migrate actor to new node since there are not any available nodes left. " + + "(However, the actor already has >1 replica in cluster, so we are ok)") - NodeAddress(nodeAddress.clusterName, nodesAvailableForMigration.head) - } else { - // actor is not in use on this node, migrate it here - nodeAddress - } + NodeAddress(nodeAddress.clusterName, nodesAvailableForMigration.head) + } else { + // actor is not in use on this node, migrate it here + nodeAddress + } - // if actor is replicated => pass along the UUID for the actor to replicate from (replay transaction log etc.) - val replicateFromUuid = - if (isReplicated(actorAddress)) Some(uuid) - else None + // if actor is replicated => pass along the UUID for the actor to replicate from (replay transaction log etc.) + val replicateFromUuid = + if (isReplicated(actorAddress)) Some(uuid) + else None - migrateWithoutCheckingThatActorResidesOnItsHomeNode( - failedNodeAddress, - migrateToNodeAddress, - actorAddress, - replicateFromUuid) - } - - // notify all available nodes that they should fail-over all connections from 'from' to 'to' - val from = disconnectedConnections(failedNodeName) - val to = remoteServerAddress - - Serialization.serialize((from, to)) match { - case Left(error) ⇒ throw error - case Right(bytes) ⇒ - - val command = RemoteDaemonMessageProtocol.newBuilder - .setMessageType(FAIL_OVER_CONNECTIONS) - .setPayload(ByteString.copyFrom(bytes)) - .build - - // FIXME now we are broadcasting to ALL nodes in the cluster even though a fraction might have a reference to the actors - should that be fixed? - nodeConnections.values foreach { - case (_, connection) ⇒ sendCommandToNode(connection, command, async = true) - } - } + migrateWithoutCheckingThatActorResidesOnItsHomeNode( + failedNodeAddress, + migrateToNodeAddress, + actorAddress, + replicateFromUuid) } + + // notify all available nodes that they should fail-over all connections from 'from' to 'to' + val from = disconnectedConnections(failedNodeName) + val to = remoteServerAddress + + Serialization.serialize((from, to)) match { + case Left(error) ⇒ throw error + case Right(bytes) ⇒ + + val command = RemoteDaemonMessageProtocol.newBuilder + .setMessageType(FAIL_OVER_CONNECTIONS) + .setPayload(ByteString.copyFrom(bytes)) + .build + + // FIXME now we are broadcasting to ALL nodes in the cluster even though a fraction might have a reference to the actors - should that be fixed? + nodeConnections.values foreach { + case (_, connection) ⇒ sendCommandToNode(connection, command, async = true) + } + } + } } } @@ -1530,7 +1520,7 @@ class DefaultClusterNode private[akka]( * Used when the ephemeral "home" node is already gone, so we can't check if it is available. */ private def migrateWithoutCheckingThatActorResidesOnItsHomeNode( - from: NodeAddress, to: NodeAddress, actorAddress: String, replicateFromUuid: Option[UUID]) { + from: NodeAddress, to: NodeAddress, actorAddress: String, replicateFromUuid: Option[UUID]) { EventHandler.debug(this, "Migrating actor [%s] from node [%s] to node [%s]".format(actorAddress, from, to)) if (!isInUseOnNode(actorAddress, to)) { @@ -1556,17 +1546,16 @@ class DefaultClusterNode private[akka]( EventHandler.info(this, "Created node [%s]".format(CLUSTER_PATH)) } - basePaths.foreach { - path ⇒ - try { - ignore[ZkNodeExistsException](zkClient.create(path, null, CreateMode.PERSISTENT)) - EventHandler.debug(this, "Created node [%s]".format(path)) - } catch { - case e ⇒ - val error = new ClusterException(e.toString) - EventHandler.error(error, this) - throw error - } + basePaths.foreach { path ⇒ + try { + ignore[ZkNodeExistsException](zkClient.create(path, null, CreateMode.PERSISTENT)) + EventHandler.debug(this, "Created node [%s]".format(path)) + } catch { + case e ⇒ + val error = new ClusterException(e.toString) + EventHandler.error(error, this) + throw error + } } } @@ -1794,85 +1783,81 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { try { if (message.hasActorAddress) { val actorAddress = message.getActorAddress - cluster.serializerForActor(actorAddress) foreach { - serializer ⇒ - cluster.use(actorAddress, serializer) foreach { - newActorRef ⇒ - cluster.remoteService.register(actorAddress, newActorRef) + cluster.serializerForActor(actorAddress) foreach { serializer ⇒ + cluster.use(actorAddress, serializer) foreach { newActorRef ⇒ + cluster.remoteService.register(actorAddress, newActorRef) - if (message.hasReplicateActorFromUuid) { - // replication is used - fetch the messages and replay them - import akka.remote.protocol.RemoteProtocol._ - import akka.remote.MessageSerializer + if (message.hasReplicateActorFromUuid) { + // replication is used - fetch the messages and replay them + import akka.remote.protocol.RemoteProtocol._ + import akka.remote.MessageSerializer - val replicateFromUuid = uuidProtocolToUuid(message.getReplicateActorFromUuid) - val deployment = Deployer.deploymentFor(actorAddress) - val replicationScheme = DeploymentConfig.replicationSchemeFor(deployment).getOrElse( - throw new IllegalStateException( - "Actor [" + actorAddress + "] should have been configured as a replicated actor but could not find its ReplicationScheme")) - val isWriteBehind = DeploymentConfig.isWriteBehindReplication(replicationScheme) + val replicateFromUuid = uuidProtocolToUuid(message.getReplicateActorFromUuid) + val deployment = Deployer.deploymentFor(actorAddress) + val replicationScheme = DeploymentConfig.replicationSchemeFor(deployment).getOrElse( + throw new IllegalStateException( + "Actor [" + actorAddress + "] should have been configured as a replicated actor but could not find its ReplicationScheme")) + val isWriteBehind = DeploymentConfig.isWriteBehindReplication(replicationScheme) - try { - // get the transaction log for the actor UUID - val txLog = TransactionLog.logFor(replicateFromUuid.toString, isWriteBehind, replicationScheme) + try { + // get the transaction log for the actor UUID + val txLog = TransactionLog.logFor(replicateFromUuid.toString, isWriteBehind, replicationScheme) - // get the latest snapshot (Option[Array[Byte]]) and all the subsequent messages (Array[Byte]) - val (snapshotAsBytes, entriesAsBytes) = txLog.latestSnapshotAndSubsequentEntries + // get the latest snapshot (Option[Array[Byte]]) and all the subsequent messages (Array[Byte]) + val (snapshotAsBytes, entriesAsBytes) = txLog.latestSnapshotAndSubsequentEntries - // deserialize and restore actor snapshot - val actorRefToUseForReplay = - snapshotAsBytes match { + // deserialize and restore actor snapshot + val actorRefToUseForReplay = + snapshotAsBytes match { - // we have a new actor ref - the snapshot - case Some(bytes) ⇒ - // stop the new actor ref and use the snapshot instead - cluster.remoteService.unregister(actorAddress) + // we have a new actor ref - the snapshot + case Some(bytes) ⇒ + // stop the new actor ref and use the snapshot instead + cluster.remoteService.unregister(actorAddress) - // deserialize the snapshot actor ref and register it as remote actor - val uncompressedBytes = - if (Cluster.shouldCompressData) LZF.uncompress(bytes) - else bytes + // deserialize the snapshot actor ref and register it as remote actor + val uncompressedBytes = + if (Cluster.shouldCompressData) LZF.uncompress(bytes) + else bytes - val snapshotActorRef = fromBinary(uncompressedBytes, newActorRef.uuid).start() - cluster.remoteService.register(actorAddress, snapshotActorRef) + val snapshotActorRef = fromBinary(uncompressedBytes, newActorRef.uuid).start() + cluster.remoteService.register(actorAddress, snapshotActorRef) - // FIXME we should call 'stop()' here (to GC the actor), but can't since that will currently shut down the TransactionLog for this UUID - since both this actor and the new snapshotActorRef have the same UUID (which they should) - //newActorRef.stop() + // FIXME we should call 'stop()' here (to GC the actor), but can't since that will currently shut down the TransactionLog for this UUID - since both this actor and the new snapshotActorRef have the same UUID (which they should) + //newActorRef.stop() - snapshotActorRef + snapshotActorRef - // we have no snapshot - use the new actor ref - case None ⇒ - newActorRef - } - - // deserialize the messages - val messages: Vector[AnyRef] = entriesAsBytes map { - bytes ⇒ - val messageBytes = - if (Cluster.shouldCompressData) LZF.uncompress(bytes) - else bytes - MessageSerializer.deserialize(MessageProtocol.parseFrom(messageBytes), None) - } - - EventHandler.info(this, "Replaying [%s] messages to actor [%s]".format(messages.size, actorAddress)) - - // replay all messages - messages foreach { - message ⇒ - EventHandler.debug(this, "Replaying message [%s] to actor [%s]".format(message, actorAddress)) - - // FIXME how to handle '?' messages? We can *not* replay them with the correct semantics. Should we: 1. Ignore/drop them and log warning? 2. Throw exception when about to log them? 3. Other? - actorRefToUseForReplay ! message - } - - } catch { - case e: Throwable ⇒ - EventHandler.error(e, this, e.toString) - throw e + // we have no snapshot - use the new actor ref + case None ⇒ + newActorRef } + + // deserialize the messages + val messages: Vector[AnyRef] = entriesAsBytes map { bytes ⇒ + val messageBytes = + if (Cluster.shouldCompressData) LZF.uncompress(bytes) + else bytes + MessageSerializer.deserialize(MessageProtocol.parseFrom(messageBytes), None) } + + EventHandler.info(this, "Replaying [%s] messages to actor [%s]".format(messages.size, actorAddress)) + + // replay all messages + messages foreach { message ⇒ + EventHandler.debug(this, "Replaying message [%s] to actor [%s]".format(message, actorAddress)) + + // FIXME how to handle '?' messages? We can *not* replay them with the correct semantics. Should we: 1. Ignore/drop them and log warning? 2. Throw exception when about to log them? 3. Other? + actorRefToUseForReplay ! message + } + + } catch { + case e: Throwable ⇒ + EventHandler.error(e, this, e.toString) + throw e + } } + } } } else { EventHandler.error(this, "Actor 'address' is not defined, ignoring remote cluster daemon command [%s]".format(message)) @@ -1887,9 +1872,8 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { case RELEASE ⇒ if (message.hasActorUuid) { - cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { - address ⇒ - cluster.release(address) + cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address ⇒ + cluster.release(address) } } else if (message.hasActorAddress) { cluster release message.getActorAddress @@ -1899,15 +1883,15 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { .format(message)) } - case START ⇒ cluster.start() + case START ⇒ cluster.start() - case STOP ⇒ cluster.shutdown() + case STOP ⇒ cluster.shutdown() case DISCONNECT ⇒ cluster.disconnect() - case RECONNECT ⇒ cluster.reconnect() + case RECONNECT ⇒ cluster.reconnect() - case RESIGN ⇒ cluster.resign() + case RESIGN ⇒ cluster.resign() case FAIL_OVER_CONNECTIONS ⇒ val (from, to) = payloadFor(message, classOf[(InetSocketAddress, InetSocketAddress)]) @@ -1945,7 +1929,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { def receive = { case (fun: Function[_, _], param: Any) ⇒ try { - fun.asInstanceOf[Any => Unit].apply(param) + fun.asInstanceOf[Any ⇒ Unit].apply(param) } finally { self.stop() } @@ -1958,7 +1942,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { def receive = { case (fun: Function[_, _], param: Any) ⇒ try { - self.reply(fun.asInstanceOf[Any => Any](param)) + self.reply(fun.asInstanceOf[Any ⇒ Any](param)) } finally { self.stop() } @@ -1971,7 +1955,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { private def payloadFor[T](message: RemoteDaemonMessageProtocol, clazz: Class[T]): T = { Serialization.deserialize(message.getPayload.toByteArray, clazz, None) match { - case Left(error) ⇒ throw error + case Left(error) ⇒ throw error case Right(instance) ⇒ instance.asInstanceOf[T] } } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala index 5afe318daa..9e136e50c4 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala @@ -11,7 +11,7 @@ import akka.dispatch.Future import java.net.InetSocketAddress import java.util.concurrent.atomic.AtomicReference -import java.util.{Map ⇒ JMap} +import java.util.{ Map ⇒ JMap } import com.eaio.uuid.UUID import collection.immutable.Map @@ -23,9 +23,9 @@ import annotation.tailrec * * @author Jonas Bonér */ -class ClusterActorRef private[akka](inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], - val address: String, - _timeout: Long) +class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], + val address: String, + _timeout: Long) extends ActorRef with ScalaActorRef { this: Router.Router ⇒ timeout = _timeout @@ -42,7 +42,7 @@ class ClusterActorRef private[akka](inetSocketAddresses: Array[Tuple2[UUID, Inet override def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = { val sender = channel match { case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None + case _ ⇒ None } route(message)(sender) } @@ -52,7 +52,7 @@ class ClusterActorRef private[akka](inetSocketAddresses: Array[Tuple2[UUID, Inet channel: UntypedChannel): Future[Any] = { val sender = channel match { case ref: ActorRef ⇒ Some(ref) - case _ ⇒ None + case _ ⇒ None } route[Any](message, timeout)(sender) } diff --git a/akka-cluster/src/main/scala/akka/cluster/Routing.scala b/akka-cluster/src/main/scala/akka/cluster/Routing.scala index c9db1e7208..43942b3e8d 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Routing.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Routing.scala @@ -6,7 +6,7 @@ package akka.cluster import akka.actor._ import akka.dispatch.Future import akka.event.EventHandler -import akka.routing.{RouterType, RoutingException} +import akka.routing.{ RouterType, RoutingException } import RouterType._ import com.eaio.uuid.UUID @@ -21,16 +21,16 @@ import java.util.concurrent.atomic.AtomicReference */ object Router { def newRouter( - routerType: RouterType, - inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], - actorAddress: String, - timeout: Long): ClusterActorRef = { + routerType: RouterType, + inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], + actorAddress: String, + timeout: Long): ClusterActorRef = { routerType match { - case Direct ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Direct - case Random ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Random - case RoundRobin ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with RoundRobin - case LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") - case LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") + case Direct ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Direct + case Random ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Random + case RoundRobin ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with RoundRobin + case LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") + case LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") case LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") } } @@ -63,7 +63,7 @@ object Router { try { actor.!(message)(sender) } catch { - case e: Exception => + case e: Exception ⇒ signalDeadActor(actor) throw e } @@ -76,7 +76,7 @@ object Router { try { actor.?(message, timeout)(sender).asInstanceOf[Future[T]] } catch { - case e: Throwable => + case e: Throwable ⇒ signalDeadActor(actor) throw e } @@ -136,7 +136,7 @@ object Router { val currentItems = current.get val newItems = currentItems match { case Nil ⇒ items - case xs ⇒ xs + case xs ⇒ xs } if (newItems.isEmpty) { diff --git a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala index 2261829934..d12820c130 100644 --- a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala +++ b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala @@ -14,7 +14,7 @@ import akka.config._ import Config._ import akka.util._ import akka.actor._ -import DeploymentConfig.{ ReplicationScheme} +import DeploymentConfig.{ ReplicationScheme } import akka.event.EventHandler import akka.dispatch.{ DefaultPromise, Promise, MessageInvocation } import akka.remote.MessageSerializer diff --git a/akka-cluster/src/main/scala/akka/cluster/storage/Storage.scala b/akka-cluster/src/main/scala/akka/cluster/storage/Storage.scala index 0129c39eb0..bb1fb39fc8 100755 --- a/akka-cluster/src/main/scala/akka/cluster/storage/Storage.scala +++ b/akka-cluster/src/main/scala/akka/cluster/storage/Storage.scala @@ -118,8 +118,8 @@ class VersionedData(val data: Array[Byte], val version: Long) {} /** * An AkkaException thrown by the Storage module. */ -class StorageException(msg: String = null, cause: java.lang.Throwable = null) extends AkkaException(msg, cause){ - def this(msg:String) = this(msg, null); +class StorageException(msg: String = null, cause: java.lang.Throwable = null) extends AkkaException(msg, cause) { + def this(msg: String) = this(msg, null); } /** @@ -127,21 +127,21 @@ class StorageException(msg: String = null, cause: java.lang.Throwable = null) ex * A StorageException thrown when an operation is done on a non existing node. */ class MissingDataException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause) { - def this(msg:String) = this(msg, null); + def this(msg: String) = this(msg, null); } /** * A StorageException thrown when an operation is done on an existing node, but no node was expected. */ -class DataExistsException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause){ - def this(msg:String) = this(msg, null); +class DataExistsException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause) { + def this(msg: String) = this(msg, null); } /** * A StorageException thrown when an operation causes an optimistic locking failure. */ class BadVersionException(msg: String = null, cause: java.lang.Throwable = null) extends StorageException(msg, cause) { - def this(msg:String) = this(msg, null); + def this(msg: String) = this(msg, null); } /** diff --git a/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala b/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala index c405240bfd..fd27d894bf 100644 --- a/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala +++ b/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala @@ -27,8 +27,7 @@ class AkkaZkClient(zkServers: String, _connection.connect(this) } catch { case e: InterruptedException ⇒ throw new ZkInterruptedException(e) - } - finally { + } finally { zkLock.unlock() } } diff --git a/akka-cluster/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-cluster/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index b676e9f254..015ccde474 100644 --- a/akka-cluster/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-cluster/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -4,22 +4,22 @@ package akka.remote.netty -import akka.dispatch.{ActorPromise, DefaultPromise, Promise} -import akka.remote.{MessageSerializer, RemoteClientSettings, RemoteServerSettings} +import akka.dispatch.{ ActorPromise, DefaultPromise, Promise } +import akka.remote.{ MessageSerializer, RemoteClientSettings, RemoteServerSettings } import akka.remote.protocol.RemoteProtocol._ import akka.serialization.RemoteActorSerialization import akka.serialization.RemoteActorSerialization._ import akka.remoteinterface._ import akka.actor.{ -PoisonPill, -Actor, -RemoteActorRef, -ActorRef, -IllegalActorStateException, -RemoteActorSystemMessage, -uuidFrom, -Uuid, -LifeCycleMessage + PoisonPill, + Actor, + RemoteActorRef, + ActorRef, + IllegalActorStateException, + RemoteActorSystemMessage, + uuidFrom, + Uuid, + LifeCycleMessage } import akka.actor.Actor._ import akka.config.Config @@ -28,27 +28,27 @@ import akka.util._ import akka.event.EventHandler import org.jboss.netty.channel._ -import org.jboss.netty.channel.group.{DefaultChannelGroup, ChannelGroup, ChannelGroupFuture} +import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroup, ChannelGroupFuture } import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory -import org.jboss.netty.bootstrap.{ServerBootstrap, ClientBootstrap} -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 org.jboss.netty.handler.execution.{OrderedMemoryAwareThreadPoolExecutor, ExecutionHandler} -import org.jboss.netty.util.{TimerTask, Timeout, HashedWheelTimer} +import org.jboss.netty.bootstrap.{ ServerBootstrap, ClientBootstrap } +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 org.jboss.netty.handler.execution.{ OrderedMemoryAwareThreadPoolExecutor, ExecutionHandler } +import org.jboss.netty.util.{ TimerTask, Timeout, HashedWheelTimer } import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ import java.net.InetSocketAddress -import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean} +import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean } import java.util.concurrent._ import akka.AkkaException -class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause){ - def this(msg:String) = this(msg, null); +class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { + def this(msg: String) = this(msg, null); } object RemoteEncoder { @@ -82,7 +82,7 @@ trait NettyRemoteClientModule extends RemoteClientModule { withClientFor(remoteAddress, loader)(_.send[T](message, senderOption, senderFuture, remoteAddress, timeout, isOneWay, actorRef)) private[akka] def withClientFor[T]( - address: InetSocketAddress, loader: Option[ClassLoader])(fun: RemoteClient ⇒ T): T = { + address: InetSocketAddress, loader: Option[ClassLoader])(fun: RemoteClient ⇒ T): T = { // loader.foreach(MessageSerializer.setClassLoader(_)) val key = Address(address) lock.readLock.lock @@ -119,14 +119,14 @@ trait NettyRemoteClientModule extends RemoteClientModule { def shutdownClientConnection(address: InetSocketAddress): Boolean = lock withWriteGuard { remoteClients.remove(Address(address)) match { case Some(client) ⇒ client.shutdown() - case None ⇒ false + case None ⇒ false } } def restartClientConnection(address: InetSocketAddress): Boolean = lock withReadGuard { remoteClients.get(Address(address)) match { case Some(client) ⇒ client.connect(reconnectIfAlreadyConnected = true) - case None ⇒ false + case None ⇒ false } } @@ -152,9 +152,9 @@ trait NettyRemoteClientModule extends RemoteClientModule { * ActiveRemoteClient, but others could be feasible, like a PassiveRemoteClient that * reuses an already established connection. */ -abstract class RemoteClient private[akka]( - val module: NettyRemoteClientModule, - val remoteAddress: InetSocketAddress) { +abstract class RemoteClient private[akka] ( + val module: NettyRemoteClientModule, + val remoteAddress: InetSocketAddress) { val useTransactionLog = config.getBool("akka.cluster.client.buffering.retry-message-send-on-failure", true) val transactionLogCapacity = config.getInt("akka.cluster.client.buffering.capacity", -1) @@ -198,13 +198,13 @@ abstract class RemoteClient private[akka]( * Converts the message to the wireprotocol and sends the message across the wire */ def send[T]( - message: Any, - senderOption: Option[ActorRef], - senderFuture: Option[Promise[T]], - remoteAddress: InetSocketAddress, - timeout: Long, - isOneWay: Boolean, - actorRef: ActorRef): Option[Promise[T]] = + message: Any, + senderOption: Option[ActorRef], + senderFuture: Option[Promise[T]], + remoteAddress: InetSocketAddress, + timeout: Long, + isOneWay: Boolean, + actorRef: ActorRef): Option[Promise[T]] = send(createRemoteMessageProtocolBuilder( Some(actorRef), Left(actorRef.uuid), actorRef.address, timeout, Right(message), isOneWay, senderOption).build, senderFuture) @@ -213,8 +213,8 @@ abstract class RemoteClient private[akka]( * Sends the message across the wire */ def send[T]( - request: RemoteMessageProtocol, - senderFuture: Option[Promise[T]]): Option[Promise[T]] = { + request: RemoteMessageProtocol, + senderFuture: Option[Promise[T]]): Option[Promise[T]] = { if (isRunning) { EventHandler.debug(this, "Sending to connection [%s] message [\n%s]".format(remoteAddress, request)) @@ -312,9 +312,9 @@ abstract class RemoteClient private[akka]( * * @author Jonas Bonér */ -class ActiveRemoteClient private[akka]( - module: NettyRemoteClientModule, remoteAddress: InetSocketAddress, - val loader: Option[ClassLoader] = None, notifyListenersFun: (⇒ Any) ⇒ Unit) extends RemoteClient(module, remoteAddress) { +class ActiveRemoteClient private[akka] ( + module: NettyRemoteClientModule, remoteAddress: InetSocketAddress, + val loader: Option[ClassLoader] = None, notifyListenersFun: (⇒ Any) ⇒ Unit) extends RemoteClient(module, remoteAddress) { import RemoteClientSettings._ @@ -346,7 +346,6 @@ class ActiveRemoteClient private[akka]( EventHandler.debug(this, "Starting remote client connection to [%s]".format(remoteAddress)) - // Wait until the connection attempt succeeds or fails. connection = bootstrap.connect(remoteAddress) openChannels.add(connection.awaitUninterruptibly.getChannel) @@ -390,7 +389,7 @@ class ActiveRemoteClient private[akka]( openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails. if (!connection.isSuccess) { notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress)) - EventHandler.error(connection.getCause, "Reconnection to [%s] has failed".format(remoteAddress),this) + EventHandler.error(connection.getCause, "Reconnection to [%s] has failed".format(remoteAddress), this) false } else { @@ -443,12 +442,12 @@ class ActiveRemoteClient private[akka]( * @author Jonas Bonér */ class ActiveRemoteClientPipelineFactory( - name: String, - futures: ConcurrentMap[Uuid, Promise[_]], - bootstrap: ClientBootstrap, - remoteAddress: InetSocketAddress, - timer: HashedWheelTimer, - client: ActiveRemoteClient) extends ChannelPipelineFactory { + name: String, + futures: ConcurrentMap[Uuid, Promise[_]], + bootstrap: ClientBootstrap, + remoteAddress: InetSocketAddress, + timer: HashedWheelTimer, + client: ActiveRemoteClient) extends ChannelPipelineFactory { def getPipeline: ChannelPipeline = { val timeout = new ReadTimeoutHandler(timer, RemoteClientSettings.READ_TIMEOUT.length, RemoteClientSettings.READ_TIMEOUT.unit) @@ -458,7 +457,7 @@ class ActiveRemoteClientPipelineFactory( val protobufEnc = new ProtobufEncoder val (enc, dec) = RemoteServerSettings.COMPRESSION_SCHEME match { case "zlib" ⇒ (new ZlibEncoder(RemoteServerSettings.ZLIB_COMPRESSION_LEVEL) :: Nil, new ZlibDecoder :: Nil) - case _ ⇒ (Nil, Nil) + case _ ⇒ (Nil, Nil) } val remoteClient = new ActiveRemoteClientHandler(name, futures, bootstrap, remoteAddress, timer, client) @@ -472,12 +471,12 @@ class ActiveRemoteClientPipelineFactory( */ @ChannelHandler.Sharable class ActiveRemoteClientHandler( - val name: String, - val futures: ConcurrentMap[Uuid, Promise[_]], - val bootstrap: ClientBootstrap, - val remoteAddress: InetSocketAddress, - val timer: HashedWheelTimer, - val client: ActiveRemoteClient) + val name: String, + val futures: ConcurrentMap[Uuid, Promise[_]], + val bootstrap: ClientBootstrap, + val remoteAddress: InetSocketAddress, + val timer: HashedWheelTimer, + val client: ActiveRemoteClient) extends SimpleChannelUpstreamHandler { override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) { @@ -493,20 +492,20 @@ class ActiveRemoteClientHandler( case arp: AkkaRemoteProtocol if arp.hasMessage ⇒ val reply = arp.getMessage val replyUuid = uuidFrom(reply.getActorInfo.getUuid.getHigh, reply.getActorInfo.getUuid.getLow) - EventHandler.debug(this, "Remote client received RemoteMessageProtocol[\n%s]".format(reply)) + EventHandler.debug(this, "Remote client received RemoteMessageProtocol[\n%s]".format(reply)) EventHandler.debug(this, "Trying to map back to future: %s".format(replyUuid)) futures.remove(replyUuid).asInstanceOf[Promise[Any]] match { - case null => + case null ⇒ client.notifyListeners(RemoteClientError(new IllegalActorStateException("Future mapped to UUID " + replyUuid + " does not exist"), client.module, client.remoteAddress)) - case future => - if (reply.hasMessage) { - val message = MessageSerializer.deserialize(reply.getMessage) - future.completeWithResult(message) - } else { - future.completeWithException(parseException(reply, client.loader)) - } + case future ⇒ + if (reply.hasMessage) { + val message = MessageSerializer.deserialize(reply.getMessage) + future.completeWithResult(message) + } else { + future.completeWithException(parseException(reply, client.loader)) + } } case other ⇒ throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.module, client.remoteAddress) @@ -598,11 +597,11 @@ class NettyRemoteSupport extends RemoteSupport with NettyRemoteServerModule with def optimizeLocalScoped_?() = optimizeLocal.get protected[akka] def actorFor( - actorAddress: String, - timeout: Long, - host: String, - port: Int, - loader: Option[ClassLoader]): ActorRef = { + actorAddress: String, + timeout: Long, + host: String, + port: Int, + loader: Option[ClassLoader]): ActorRef = { val homeInetSocketAddress = this.address if (optimizeLocalScoped_?) { @@ -684,7 +683,7 @@ trait NettyRemoteServerModule extends RemoteServerModule { def address = currentServer.get match { case Some(server) ⇒ server.address - case None ⇒ ReflectiveAccess.RemoteModule.configDefaultAddress + case None ⇒ ReflectiveAccess.RemoteModule.configDefaultAddress } def name = currentServer.get match { @@ -715,11 +714,10 @@ trait NettyRemoteServerModule extends RemoteServerModule { def shutdownServerModule() = guard withGuard { _isRunning switchOff { - currentServer.getAndSet(None) foreach { - instance ⇒ - EventHandler.debug(this, "Shutting down remote server on %s:%s".format(instance.host, instance.port)) + currentServer.getAndSet(None) foreach { instance ⇒ + EventHandler.debug(this, "Shutting down remote server on %s:%s".format(instance.host, instance.port)) - instance.shutdown() + instance.shutdown() } } } @@ -810,11 +808,11 @@ trait NettyRemoteServerModule extends RemoteServerModule { * @author Jonas Bonér */ class RemoteServerPipelineFactory( - val name: String, - val openChannels: ChannelGroup, - val executor: ExecutionHandler, - val loader: Option[ClassLoader], - val server: NettyRemoteServerModule) extends ChannelPipelineFactory { + val name: String, + val openChannels: ChannelGroup, + val executor: ExecutionHandler, + val loader: Option[ClassLoader], + val server: NettyRemoteServerModule) extends ChannelPipelineFactory { import RemoteServerSettings._ @@ -825,7 +823,7 @@ class RemoteServerPipelineFactory( val protobufEnc = new ProtobufEncoder val (enc, dec) = COMPRESSION_SCHEME match { case "zlib" ⇒ (new ZlibEncoder(ZLIB_COMPRESSION_LEVEL) :: Nil, new ZlibDecoder :: Nil) - case _ ⇒ (Nil, Nil) + case _ ⇒ (Nil, Nil) } val authenticator = if (REQUIRE_COOKIE) new RemoteServerAuthenticationHandler(SECURE_COOKIE) :: Nil else Nil val remoteServer = new RemoteServerHandler(name, openChannels, loader, server) @@ -865,10 +863,10 @@ class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends Si */ @ChannelHandler.Sharable class RemoteServerHandler( - val name: String, - val openChannels: ChannelGroup, - val applicationLoader: Option[ClassLoader], - val server: NettyRemoteServerModule) extends SimpleChannelUpstreamHandler { + val name: String, + val openChannels: ChannelGroup, + val applicationLoader: Option[ClassLoader], + val server: NettyRemoteServerModule) extends SimpleChannelUpstreamHandler { import RemoteServerSettings._ @@ -886,7 +884,7 @@ class RemoteServerHandler( } else if (!future.isSuccess) { val socketAddress = future.getChannel.getRemoteAddress match { case i: InetSocketAddress ⇒ Some(i) - case _ ⇒ None + case _ ⇒ None } server.notifyListeners(RemoteServerWriteFailed(payload, future.getCause, server, socketAddress)) } @@ -902,7 +900,7 @@ class RemoteServerHandler( override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { val clientAddress = getClientAddress(ctx) - EventHandler.debug(this,"Remote client [%s] connected to [%s]".format(clientAddress, server.name)) + EventHandler.debug(this, "Remote client [%s] connected to [%s]".format(clientAddress, server.name)) sessionActors.set(event.getChannel(), new ConcurrentHashMap[String, ActorRef]()) server.notifyListeners(RemoteServerClientConnected(server, clientAddress)) @@ -921,7 +919,7 @@ class RemoteServerHandler( try { actor ! PoisonPill } catch { - case e: Exception ⇒ EventHandler.error(e, "Couldn't stop %s".format(actor),this) + case e: Exception ⇒ EventHandler.error(e, "Couldn't stop %s".format(actor), this) } } @@ -930,7 +928,7 @@ class RemoteServerHandler( override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { val clientAddress = getClientAddress(ctx) - EventHandler.debug("Remote client [%s] channel closed from [%s]".format(clientAddress, server.name),this) + EventHandler.debug("Remote client [%s] channel closed from [%s]".format(clientAddress, server.name), this) server.notifyListeners(RemoteServerClientClosed(server, clientAddress)) } @@ -956,7 +954,7 @@ class RemoteServerHandler( private def getClientAddress(ctx: ChannelHandlerContext): Option[InetSocketAddress] = ctx.getChannel.getRemoteAddress match { case inet: InetSocketAddress ⇒ Some(inet) - case _ ⇒ None + case _ ⇒ None } private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = try { @@ -1005,22 +1003,22 @@ class RemoteServerHandler( request.getActorInfo.getTimeout, new ActorPromise(request.getActorInfo.getTimeout). onComplete(_.value.get match { - case Left(exception) ⇒ write(channel, createErrorReplyMessage(exception, request)) - case r: Right[_, _] ⇒ - val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder( - Some(actorRef), - Right(request.getUuid), - actorInfo.getAddress, - actorInfo.getTimeout, - r.asInstanceOf[Either[Throwable, Any]], - isOneWay = true, - Some(actorRef)) + case Left(exception) ⇒ write(channel, createErrorReplyMessage(exception, request)) + case r: Right[_, _] ⇒ + val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder( + Some(actorRef), + Right(request.getUuid), + actorInfo.getAddress, + actorInfo.getTimeout, + r.asInstanceOf[Either[Throwable, Any]], + isOneWay = true, + Some(actorRef)) - // FIXME lift in the supervisor uuid management into toh createRemoteMessageProtocolBuilder method - if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid) + // FIXME lift in the supervisor uuid management into toh createRemoteMessageProtocolBuilder method + if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid) - write(channel, RemoteEncoder.encode(messageBuilder.build)) - })) + write(channel, RemoteEncoder.encode(messageBuilder.build)) + })) } } @@ -1070,7 +1068,7 @@ class RemoteServerHandler( private def findSessionActor(id: String, channel: Channel): ActorRef = sessionActors.get(channel) match { case null ⇒ null - case map ⇒ map get id + case map ⇒ map get id } private def createErrorReplyMessage(exception: Throwable, request: RemoteMessageProtocol): AkkaRemoteProtocol = { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala index 18c8da87da..7c59a9fb93 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala @@ -8,7 +8,7 @@ import akka.cluster._ import akka.cluster.Cluster._ import akka.actor.Actor import akka.routing.RoutingException -import java.nio.channels.{ClosedChannelException, NotYetConnectedException} +import java.nio.channels.{ ClosedChannelException, NotYetConnectedException } object ClusterActorRefCleanupMultiJvmSpec { @@ -20,10 +20,10 @@ object ClusterActorRefCleanupMultiJvmSpec { println("--------------------------------------") def receive = { - case "Die" => + case "Die" ⇒ println("Killing JVM: " + Cluster.node.nodeAddress) System.exit(0) - case _ => + case _ ⇒ println("Hello") } } @@ -61,7 +61,7 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { clusteredRef ! "hello" clusteredRef ! "hello" } catch { - case e: NotYetConnectedException => + case e: NotYetConnectedException ⇒ } //since the call to the node failed, the node must have been removed from the list. @@ -80,9 +80,9 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { try { clusteredRef ! "hello" } catch { - case e: ClosedChannelException => - case e: NotYetConnectedException => - case e: RoutingException => + case e: ClosedChannelException ⇒ + case e: NotYetConnectedException ⇒ + case e: RoutingException ⇒ } //now there must not be any remaining connections after the dead of the last actor. @@ -93,7 +93,7 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { clusteredRef ! "Hello" assert(false) } catch { - case e: RoutingException => + case e: RoutingException ⇒ } node.shutdown() diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmSpec.scala index 7c6911e70d..75bc03309d 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmSpec.scala @@ -23,7 +23,6 @@ object BadAddressDirectRoutingMultiJvmSpec { } - class BadAddressDirectRoutingMultiJvmNode1 extends MasterClusterTestNode { import BadAddressDirectRoutingMultiJvmSpec._ diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmSpec.scala index 97b94d305c..35009b6d47 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmSpec.scala @@ -22,7 +22,6 @@ object SingleReplicaDirectRoutingMultiJvmSpec { } - class SingleReplicaDirectRoutingMultiJvmNode1 extends MasterClusterTestNode { import SingleReplicaDirectRoutingMultiJvmSpec._ diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala index b16addfe3d..aac06caa90 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala @@ -75,7 +75,6 @@ class RoundRobin2ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with B } } - class RoundRobin2ReplicasMultiJvmNode2 extends WordSpec with MustMatchers { import RoundRobin2ReplicasMultiJvmSpec._ @@ -109,7 +108,7 @@ class RoundRobin2ReplicasMultiJvmNode2 extends WordSpec with MustMatchers { //todo: is there a reason to check for null again since it already has been done in the previous block. hello must not equal (null) - val replies = new ConcurrentHashMap[String,AtomicInteger]() + val replies = new ConcurrentHashMap[String, AtomicInteger]() def count(reply: String) = { val counter = new AtomicInteger(0) Option(replies.putIfAbsent(reply, counter)).getOrElse(counter).incrementAndGet() diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala index 8138d04b14..c873117d41 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala @@ -22,16 +22,16 @@ class BeanstalkBasedMailboxException(message: String) extends AkkaException(mess */ class BeanstalkBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(owner) { - val hostname = config.getString("akka.actor.mailbox.beanstalk.hostname", "0.0.0.0") - val port = config.getInt("akka.actor.mailbox.beanstalk.port", 11300) - val reconnectWindow = Duration(config.getInt("akka.actor.mailbox.beanstalk.reconnect-window", 5), TIME_UNIT).toSeconds.toInt - val messageSubmitDelay = Duration(config.getInt("akka.actor.mailbox.beanstalk.message-submit-delay", 0), TIME_UNIT).toSeconds.toInt + val hostname = config.getString("akka.actor.mailbox.beanstalk.hostname", "0.0.0.0") + val port = config.getInt("akka.actor.mailbox.beanstalk.port", 11300) + val reconnectWindow = Duration(config.getInt("akka.actor.mailbox.beanstalk.reconnect-window", 5), TIME_UNIT).toSeconds.toInt + val messageSubmitDelay = Duration(config.getInt("akka.actor.mailbox.beanstalk.message-submit-delay", 0), TIME_UNIT).toSeconds.toInt val messageSubmitTimeout = Duration(config.getInt("akka.actor.mailbox.beanstalk.message-submit-timeout", 5), TIME_UNIT).toSeconds.toInt - val messageTimeToLive = Duration(config.getInt("akka.actor.mailbox.beanstalk.message-time-to-live", 120), TIME_UNIT).toSeconds.toInt + val messageTimeToLive = Duration(config.getInt("akka.actor.mailbox.beanstalk.message-time-to-live", 120), TIME_UNIT).toSeconds.toInt private val queue = new ThreadLocal[Client] { override def initialValue = connect(name) } - // ===== For MessageQueue ===== + // ===== For MessageQueue ===== def enqueue(durableMessage: MessageInvocation) = { Some(queue.get.put(65536, messageSubmitDelay, messageTimeToLive, serialize(durableMessage)).toInt) @@ -51,8 +51,8 @@ class BeanstalkBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbo } else null: MessageInvocation } } catch { - case e: Exception => - EventHandler.error(e, this, "Beanstalk connection error") + case e: Exception ⇒ + EventHandler.error(e, this, "Beanstalk connection error") reconnect(name) null: MessageInvocation } @@ -65,7 +65,7 @@ class BeanstalkBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbo queue.get.kick(100000) true } catch { - case e: Exception => false + case e: Exception ⇒ false } } @@ -77,7 +77,8 @@ class BeanstalkBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbo def isEmpty = size == 0 private def connect(name: String): Client = { - @volatile var connected = false + @volatile + var connected = false var attempts = 0 var client: Client = null while (!connected) { @@ -88,12 +89,12 @@ class BeanstalkBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbo client.watch(name) connected = true } catch { - case e: Exception => - EventHandler.error(e, this, "Unable to connect to Beanstalk. Retrying in [%s] seconds: %s".format(reconnectWindow, e)) + case e: Exception ⇒ + EventHandler.error(e, this, "Unable to connect to Beanstalk. Retrying in [%s] seconds: %s".format(reconnectWindow, e)) try { Thread.sleep(1000 * reconnectWindow) } catch { - case e: InterruptedException => {} + case e: InterruptedException ⇒ {} } } } diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala index 2964eb19b9..4795953af3 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala @@ -22,19 +22,19 @@ class FileBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(own import FileBasedMailboxUtil._ private val queue = try { - try { FileUtils.forceMkdir(new java.io.File(queuePath)) } catch { case e => {} } + try { FileUtils.forceMkdir(new java.io.File(queuePath)) } catch { case e ⇒ {} } val queue = new filequeue.PersistentQueue(queuePath, name, config) queue.setup // replays journal queue.discardExpired queue } catch { - case e: Exception => + case e: Exception ⇒ EventHandler.error(e, this, "Could not create a file-based mailbox") throw e } def enqueue(message: MessageInvocation) = { - EventHandler.debug(this, "\nENQUEUING message in file-based mailbox [%s]".format( message)) + EventHandler.debug(this, "\nENQUEUING message in file-based mailbox [%s]".format(message)) queue.add(serialize(message)) } @@ -47,8 +47,8 @@ class FileBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(own messageInvocation } else null } catch { - case e: java.util.NoSuchElementException => null - case e: Exception => + case e: java.util.NoSuchElementException ⇒ null + case e: Exception ⇒ EventHandler.error(e, this, "Couldn't dequeue from file-based mailbox") throw e } @@ -60,7 +60,7 @@ class FileBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(own queue.remove true } catch { - case e => false //review why catch Throwable? And swallow potential Errors? + case e ⇒ false //review why catch Throwable? And swallow potential Errors? } def size: Int = queue.length.toInt diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/Journal.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/Journal.scala index ecfce7ecc6..9b1973357f 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/Journal.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/Journal.scala @@ -18,7 +18,7 @@ package akka.actor.mailbox.filequeue import java.io._ -import java.nio.{ByteBuffer, ByteOrder} +import java.nio.{ ByteBuffer, ByteOrder } import java.nio.channels.FileChannel import akka.event.EventHandler @@ -35,11 +35,10 @@ object JournalItem { case object EndOfFile extends JournalItem } - /** * Codes for working with the journal file for a PersistentQueue. */ -class Journal(queuePath: String, syncJournal: => Boolean) { +class Journal(queuePath: String, syncJournal: ⇒ Boolean) { private val queueFile = new File(queuePath) @@ -63,7 +62,6 @@ class Journal(queuePath: String, syncJournal: => Boolean) { private val CMD_CONFIRM_REMOVE = 6 private val CMD_ADD_XID = 7 - private def open(file: File): Unit = { writer = new FileOutputStream(file, true).getChannel } @@ -77,12 +75,12 @@ class Journal(queuePath: String, syncJournal: => Boolean) { val tmpFile = new File(queuePath + "~~" + System.currentTimeMillis) open(tmpFile) size = 0 - for (item <- openItems) { + for (item ← openItems) { addWithXid(item) removeTentative(false) } saveXid(xid) - for (item <- queue) { + for (item ← queue) { add(false, item) } if (syncJournal) writer.force(false) @@ -93,7 +91,7 @@ class Journal(queuePath: String, syncJournal: => Boolean) { def close(): Unit = { writer.close - for (r <- reader) r.close + for (r ← reader) r.close reader = None } @@ -102,7 +100,7 @@ class Journal(queuePath: String, syncJournal: => Boolean) { close() queueFile.delete } catch { - case _ => + case _ ⇒ } } @@ -164,23 +162,23 @@ class Journal(queuePath: String, syncJournal: => Boolean) { reader = Some(rj) } - def fillReadBehind(f: QItem => Unit): Unit = { + def fillReadBehind(f: QItem ⇒ Unit): Unit = { val pos = if (replayer.isDefined) replayer.get.position else writer.position - for (rj <- reader) { + for (rj ← reader) { if (rj.position == pos) { // we've caught up. rj.close reader = None } else { readJournalEntry(rj) match { - case (JournalItem.Add(item), _) => f(item) - case (_, _) => + case (JournalItem.Add(item), _) ⇒ f(item) + case (_, _) ⇒ } } } } - def replay(name: String)(f: JournalItem => Unit): Unit = { + def replay(name: String)(f: JournalItem ⇒ Unit): Unit = { size = 0 var lastUpdate = 0L val TEN_MB = 10L * 1024 * 1024 @@ -191,8 +189,8 @@ class Journal(queuePath: String, syncJournal: => Boolean) { var done = false do { readJournalEntry(in) match { - case (JournalItem.EndOfFile, _) => done = true - case (x, itemsize) => + case (JournalItem.EndOfFile, _) ⇒ done = true + case (x, itemsize) ⇒ size += itemsize f(x) if (size / TEN_MB > lastUpdate) { @@ -203,17 +201,17 @@ class Journal(queuePath: String, syncJournal: => Boolean) { } } while (!done) } catch { - case e: BrokenItemException => + case e: BrokenItemException ⇒ EventHandler.error(e, this, "Exception replaying journal for '%s'".format(name)) truncateJournal(e.lastValidPosition) } } catch { - case e: FileNotFoundException => + case e: FileNotFoundException ⇒ EventHandler.info(this, "No transaction journal for '%s'; starting with empty queue.".format(name)) - case e: IOException => + case e: IOException ⇒ EventHandler.error(e, this, "Exception replaying journal for '%s'".format(name)) - // this can happen if the server hardware died abruptly in the middle - // of writing a journal. not awesome but we should recover. + // this can happen if the server hardware died abruptly in the middle + // of writing a journal. not awesome but we should recover. } replayer = None } @@ -241,36 +239,36 @@ class Journal(queuePath: String, syncJournal: => Boolean) { } else { try { buffer(0) match { - case CMD_ADD => + case CMD_ADD ⇒ val data = readBlock(in) (JournalItem.Add(QItem.unpackOldAdd(data)), 5 + data.length) - case CMD_REMOVE => + case CMD_REMOVE ⇒ (JournalItem.Remove, 1) - case CMD_ADDX => + case CMD_ADDX ⇒ val data = readBlock(in) (JournalItem.Add(QItem.unpack(data)), 5 + data.length) - case CMD_REMOVE_TENTATIVE => + case CMD_REMOVE_TENTATIVE ⇒ (JournalItem.RemoveTentative, 1) - case CMD_SAVE_XID => + case CMD_SAVE_XID ⇒ val xid = readInt(in) (JournalItem.SavedXid(xid), 5) - case CMD_UNREMOVE => + case CMD_UNREMOVE ⇒ val xid = readInt(in) (JournalItem.Unremove(xid), 5) - case CMD_CONFIRM_REMOVE => + case CMD_CONFIRM_REMOVE ⇒ val xid = readInt(in) (JournalItem.ConfirmRemove(xid), 5) - case CMD_ADD_XID => + case CMD_ADD_XID ⇒ val xid = readInt(in) val data = readBlock(in) val item = QItem.unpack(data) item.xid = xid (JournalItem.Add(item), 9 + data.length) - case n => + case n ⇒ throw new BrokenItemException(lastPosition, new IOException("invalid opcode in journal: " + n.toInt + " at position " + in.position)) } } catch { - case ex: IOException => + case ex: IOException ⇒ throw new BrokenItemException(lastPosition, ex) } } @@ -286,11 +284,11 @@ class Journal(queuePath: String, syncJournal: => Boolean) { false } else { nextItem = readJournalEntry(in) match { - case (JournalItem.EndOfFile, _) => + case (JournalItem.EndOfFile, _) ⇒ done = true in.close() None - case x => + case x ⇒ Some(x) } nextItem.isDefined @@ -332,9 +330,9 @@ class Journal(queuePath: String, syncJournal: => Boolean) { private def write(allowSync: Boolean, items: Any*): Int = { byteBuffer.clear - for (item <- items) item match { - case b: Byte => byteBuffer.put(b) - case i: Int => byteBuffer.putInt(i) + for (item ← items) item match { + case b: Byte ⇒ byteBuffer.put(b) + case i: Int ⇒ byteBuffer.putInt(i) } byteBuffer.flip while (byteBuffer.position < byteBuffer.limit) { 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 9414222276..3bc24e48d0 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 @@ -26,15 +26,15 @@ import akka.event.EventHandler import akka.config.Configuration // a config value that's backed by a global setting but may be locally overridden -class OverlaySetting[T](base: => T) { - @volatile private var local: Option[T] = None +class OverlaySetting[T](base: ⇒ T) { + @volatile + private var local: Option[T] = None def set(value: Option[T]) = local = value def apply() = local.getOrElse(base) } - class PersistentQueue(persistencePath: String, val name: String, val config: Configuration) { private case object ItemArrived @@ -66,7 +66,7 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con private var closed = false private var paused = false - def overlay[T](base: => T) = new OverlaySetting(base) + 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) @@ -145,8 +145,8 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con EventHandler.info(this, "Configuring queue %s: journal=%s, max-items=%s, max-size=%s, max-age=%s, max-journal-size=%s, max-memory-size=%s, max-journal-overflow=%s, max-journal-size-absolute=%s, discard-old-when-full=%s, sync-journal=%s" .format( - name, keepJournal(), maxItems(), maxSize(), maxAge(), maxJournalSize(), maxMemorySize(), - maxJournalOverflow(), maxJournalSizeAbsolute(), discardOldWhenFull(), syncJournal())) + name, keepJournal(), maxItems(), maxSize(), maxAge(), maxJournalSize(), maxMemorySize(), + maxJournalOverflow(), maxJournalSizeAbsolute(), discardOldWhenFull(), syncJournal())) if (!keepJournal()) journal.erase() } @@ -162,8 +162,7 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con "discard-old-when-full=" + discardOldWhenFull(), "journal=" + keepJournal(), "sync-journal=" + syncJournal(), - "move-expired-to" + expiredQueue().map { _.name }.getOrElse("(none)") - ) + "move-expired-to" + expiredQueue().map { _.name }.getOrElse("(none)")) } def dumpStats(): Array[(String, String)] = synchronized { @@ -177,8 +176,7 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con ("mem-bytes", memoryBytes.toString), ("age", currentAge.toString), ("discarded", totalDiscarded.toString), - ("open-transactions", openTransactionCount.toString) - ) + ("open-transactions", openTransactionCount.toString)) } private final def adjustExpiry(startingTime: Long, expiry: Long): Long = { @@ -291,7 +289,7 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con } def flush(): Unit = { - while (remove(false).isDefined) { } + while (remove(false).isDefined) {} } /** @@ -330,7 +328,7 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con // if we're in read-behind mode, scan forward in the journal to keep memory as full as // possible. this amortizes the disk overhead across all reads. while (keepJournal() && journal.inReadBehind && _memoryBytes < maxMemorySize()) { - journal.fillReadBehind { item => + journal.fillReadBehind { item ⇒ queue += item _memoryBytes += item.data.length } @@ -347,7 +345,7 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con xidCounter = 0 journal.replay(name) { - case JournalItem.Add(item) => + case JournalItem.Add(item) ⇒ _add(item) // when processing the journal, this has to happen after: if (!journal.inReadBehind && queueSize >= maxMemorySize()) { @@ -355,27 +353,27 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con "Dropping to read-behind for queue '%s' (%s bytes)".format(name, queueSize)) journal.startReadBehind } - case JournalItem.Remove => _remove(false) - case JournalItem.RemoveTentative => _remove(true) - case JournalItem.SavedXid(xid) => xidCounter = xid - case JournalItem.Unremove(xid) => _unremove(xid) - case JournalItem.ConfirmRemove(xid) => openTransactions.remove(xid) - case x => EventHandler.warning(this, "Unexpected item in journal: %s".format(x)) + case JournalItem.Remove ⇒ _remove(false) + case JournalItem.RemoveTentative ⇒ _remove(true) + case JournalItem.SavedXid(xid) ⇒ xidCounter = xid + case JournalItem.Unremove(xid) ⇒ _unremove(xid) + case JournalItem.ConfirmRemove(xid) ⇒ openTransactions.remove(xid) + case x ⇒ EventHandler.warning(this, "Unexpected item in journal: %s".format(x)) } EventHandler.debug(this, "Finished transaction journal for '%s' (%s items, %s bytes)" - .format(name, queueLength, journal.size)) + .format(name, queueLength, journal.size)) journal.open // now, any unfinished transactions must be backed out. - for (xid <- openTransactionIds) { + for (xid ← openTransactionIds) { journal.unremove(xid) _unremove(xid) } } - def toList(): List[QItem] = { + def toList(): List[QItem] = { discardExpired queue.toList } @@ -398,7 +396,7 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con if (queue.isEmpty) None else Some(queue.front) } - private def _remove(transaction: Boolean): Option[QItem] = { + private def _remove(transaction: Boolean): Option[QItem] = { discardExpired() if (queue.isEmpty) return None @@ -442,7 +440,7 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con } private def _unremove(xid: Int) = { - openTransactions.remove(xid) map { item => + openTransactions.remove(xid) map { item ⇒ queueLength += 1 queueSize += item.data.length queue unget item @@ -451,18 +449,29 @@ class PersistentQueue(persistencePath: String, val name: String, val config: Con } } - object PersistentQueue { - @volatile var maxItems: Int = Int.MaxValue - @volatile var maxSize: Long = Long.MaxValue - @volatile var maxItemSize: Long = Long.MaxValue - @volatile var maxAge: Int = 0 - @volatile var maxJournalSize: Long = 16 * 1024 * 1024 - @volatile var maxMemorySize: Long = 128 * 1024 * 1024 - @volatile var maxJournalOverflow: Int = 10 - @volatile var maxJournalSizeAbsolute: Long = Long.MaxValue - @volatile var discardOldWhenFull: Boolean = false - @volatile var keepJournal: Boolean = true - @volatile var syncJournal: Boolean = false - @volatile var expiredQueue: Option[PersistentQueue] = None + @volatile + var maxItems: Int = Int.MaxValue + @volatile + var maxSize: Long = Long.MaxValue + @volatile + var maxItemSize: Long = Long.MaxValue + @volatile + var maxAge: Int = 0 + @volatile + var maxJournalSize: Long = 16 * 1024 * 1024 + @volatile + var maxMemorySize: Long = 128 * 1024 * 1024 + @volatile + var maxJournalOverflow: Int = 10 + @volatile + var maxJournalSizeAbsolute: Long = Long.MaxValue + @volatile + var discardOldWhenFull: Boolean = false + @volatile + var keepJournal: Boolean = true + @volatile + var syncJournal: Boolean = false + @volatile + var expiredQueue: Option[PersistentQueue] = None } diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/QItem.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/QItem.scala index 449a3aa878..1a05070cb7 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/QItem.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/QItem.scala @@ -17,7 +17,7 @@ package akka.actor.mailbox.filequeue -import java.nio.{ByteBuffer, ByteOrder} +import java.nio.{ ByteBuffer, ByteOrder } case class QItem(addTime: Long, expiry: Long, data: Array[Byte], var xid: Int) { def pack(): Array[Byte] = { diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/QueueCollection.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/QueueCollection.scala index 9a2f6c879b..36ce3c6d8d 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/QueueCollection.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/QueueCollection.scala @@ -22,19 +22,18 @@ import java.util.concurrent.CountDownLatch import scala.collection.mutable -import akka.config.{Config, Configuration} +import akka.config.{ Config, Configuration } import akka.event.EventHandler class InaccessibleQueuePath extends Exception("Inaccessible queue path: Must be a directory and writable") - class QueueCollection(queueFolder: String, private var queueConfigs: Configuration) { private val path = new File(queueFolder) - if (! path.isDirectory) { + if (!path.isDirectory) { path.mkdirs() } - if (! path.isDirectory || ! path.canWrite) { + if (!path.isDirectory || !path.canWrite) { throw new InaccessibleQueuePath } @@ -58,7 +57,7 @@ class QueueCollection(queueFolder: String, private var queueConfigs: Configurati // preload any queues def loadQueues() { - path.list() filter { name => !(name contains "~~") } map { queue(_) } + path.list() filter { name ⇒ !(name contains "~~") } map { queue(_) } } def queueNames: List[String] = synchronized { @@ -101,13 +100,13 @@ class QueueCollection(queueFolder: String, private var queueConfigs: Configurati * down */ def add(key: String, item: Array[Byte], expiry: Int): Boolean = { - for (fanouts <- fanout_queues.get(key); name <- fanouts) { + for (fanouts ← fanout_queues.get(key); name ← fanouts) { add(name, item, expiry) } queue(key) match { - case None => false - case Some(q) => + case None ⇒ false + case Some(q) ⇒ val now = System.currentTimeMillis val normalizedExpiry: Long = if (expiry == 0) { 0 @@ -128,17 +127,17 @@ class QueueCollection(queueFolder: String, private var queueConfigs: Configurati * Retrieve an item from a queue and pass it to a continuation. If no item is available within * the requested time, or the server is shutting down, None is passed. */ - def remove(key: String, timeout: Int, transaction: Boolean, peek: Boolean)(f: Option[QItem] => Unit): Unit = { + def remove(key: String, timeout: Int, transaction: Boolean, peek: Boolean)(f: Option[QItem] ⇒ Unit): Unit = { queue(key) match { - case None => + case None ⇒ queueMisses.incr f(None) - case Some(q) => + case Some(q) ⇒ if (peek) { f(q.peek()) } else { q.remove -/* q.removeReact(if (timeout == 0) timeout else System.currentTimeMillis + timeout, transaction) { + /* q.removeReact(if (timeout == 0) timeout else System.currentTimeMillis + timeout, transaction) { case None => queueMisses.incr f(None) @@ -146,7 +145,7 @@ class QueueCollection(queueFolder: String, private var queueConfigs: Configurati queueHits.incr f(Some(item)) } -*/ } +*/ } } } @@ -155,10 +154,10 @@ class QueueCollection(queueFolder: String, private var queueConfigs: Configurati var rv: Option[Array[Byte]] = None val latch = new CountDownLatch(1) remove(key, 0, false, false) { - case None => + case None ⇒ rv = None latch.countDown - case Some(v) => + case Some(v) ⇒ rv = Some(v.data) latch.countDown } @@ -167,20 +166,20 @@ class QueueCollection(queueFolder: String, private var queueConfigs: Configurati } def unremove(key: String, xid: Int) { - queue(key) map { q => q.unremove(xid) } + queue(key) map { q ⇒ q.unremove(xid) } } def confirmRemove(key: String, xid: Int) { - queue(key) map { q => q.confirmRemove(xid) } + queue(key) map { q ⇒ q.confirmRemove(xid) } } def flush(key: String) { - queue(key) map { q => q.flush() } + queue(key) map { q ⇒ q.flush() } } def delete(name: String): Unit = synchronized { if (!shuttingDown) { - queues.get(name) map { q => + queues.get(name) map { q ⇒ q.close() q.destroyJournal() queues.remove(name) @@ -197,25 +196,25 @@ class QueueCollection(queueFolder: String, private var queueConfigs: Configurati if (shuttingDown) { 0 } else { - queue(name) map { q => q.discardExpired() } getOrElse(0) + queue(name) map { q ⇒ q.discardExpired() } getOrElse (0) } } def flushAllExpired(): Int = synchronized { - queueNames.foldLeft(0) { (sum, qName) => sum + flushExpired(qName) } + queueNames.foldLeft(0) { (sum, qName) ⇒ sum + flushExpired(qName) } } def stats(key: String): Array[(String, String)] = queue(key) match { - case None => Array[(String, String)]() - case Some(q) => + case None ⇒ Array[(String, String)]() + case Some(q) ⇒ q.dumpStats() ++ - fanout_queues.get(key).map { qset => ("children", qset.mkString(",")) }.toList + fanout_queues.get(key).map { qset ⇒ ("children", qset.mkString(",")) }.toList } def dumpConfig(key: String): Array[String] = { queue(key) match { - case None => Array() - case Some(q) => q.dumpConfig() + case None ⇒ Array() + case Some(q) ⇒ q.dumpConfig() } } @@ -228,7 +227,7 @@ class QueueCollection(queueFolder: String, private var queueConfigs: Configurati return } shuttingDown = true - for ((name, q) <- queues) { + for ((name, q) ← queues) { // synchronous, so the journals are all officially closed before we return. q.close } diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/tools/QDumper.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/tools/QDumper.scala index 9e99c9cd50..eb46028926 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/tools/QDumper.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/tools/QDumper.scala @@ -17,7 +17,7 @@ package akka.actor.mailbox.filequeue.tools -import java.io.{FileNotFoundException, IOException} +import java.io.{ FileNotFoundException, IOException } import scala.collection.mutable import akka.actor.mailbox.filequeue._ @@ -36,7 +36,7 @@ class QueueDumper(filename: String) { var lastDisplay = 0L try { - for ((item, itemsize) <- journal.walk()) { + for ((item, itemsize) ← journal.walk()) { operations += 1 dumpItem(item) offset += itemsize @@ -54,9 +54,9 @@ class QueueDumper(filename: String) { println("Journal size: %d bytes, with %d operations.".format(offset, operations)) println("%d items totalling %d bytes.".format(totalItems, totalBytes)) } catch { - case e: FileNotFoundException => + case e: FileNotFoundException ⇒ println("Can't open journal file: " + filename) - case e: IOException => + case e: IOException ⇒ println("Exception reading journal file: " + filename) e.printStackTrace() } @@ -66,7 +66,7 @@ class QueueDumper(filename: String) { val now = System.currentTimeMillis if (!QDumper.quiet) print("%08x ".format(offset & 0xffffffffL)) item match { - case JournalItem.Add(qitem) => + case JournalItem.Add(qitem) ⇒ if (!QDumper.quiet) { print("ADD %-6d".format(qitem.data.size)) if (qitem.xid > 0) { @@ -82,31 +82,30 @@ class QueueDumper(filename: String) { println() } queue += qitem.data.size - case JournalItem.Remove => + case JournalItem.Remove ⇒ if (!QDumper.quiet) println("REM") queue.dequeue - case JournalItem.RemoveTentative => + case JournalItem.RemoveTentative ⇒ do { currentXid += 1 } while (openTransactions contains currentXid) openTransactions(currentXid) = queue.dequeue if (!QDumper.quiet) println("RSV %d".format(currentXid)) - case JournalItem.SavedXid(xid) => + case JournalItem.SavedXid(xid) ⇒ if (!QDumper.quiet) println("XID %d".format(xid)) currentXid = xid - case JournalItem.Unremove(xid) => + case JournalItem.Unremove(xid) ⇒ queue.unget(openTransactions.remove(xid).get) if (!QDumper.quiet) println("CAN %d".format(xid)) - case JournalItem.ConfirmRemove(xid) => + case JournalItem.ConfirmRemove(xid) ⇒ if (!QDumper.quiet) println("ACK %d".format(xid)) openTransactions.remove(xid) - case x => + case x ⇒ if (!QDumper.quiet) println(x) } } } - object QDumper { val filenames = new mutable.ListBuffer[String] var quiet = false @@ -122,14 +121,14 @@ object QDumper { } def parseArgs(args: List[String]): Unit = args match { - case Nil => - case "--help" :: xs => + case Nil ⇒ + case "--help" :: xs ⇒ usage() System.exit(0) - case "-q" :: xs => + case "-q" :: xs ⇒ quiet = true parseArgs(xs) - case x :: xs => + case x :: xs ⇒ filenames += x parseArgs(xs) } @@ -141,7 +140,7 @@ object QDumper { System.exit(0) } - for (filename <- filenames) { + for (filename ← filenames) { println("Queue: " + filename) new QueueDumper(filename)() } diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/tools/Util.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/tools/Util.scala index de46bbede9..c1b9c35594 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/tools/Util.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/filequeue/tools/Util.scala @@ -43,10 +43,10 @@ object Util { } base.toString + (if (base < 100) ("." + dot) else "") + (divisor match { - case KILOBYTE => "K" - case MEGABYTE => "M" - case GIGABYTE => "G" - case _ => "" + case KILOBYTE ⇒ "K" + case MEGABYTE ⇒ "M" + case GIGABYTE ⇒ "G" + case _ ⇒ "" }) } } diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala index e1aab05c71..9adb6312b7 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala @@ -3,7 +3,7 @@ */ package akka.actor.mailbox -import akka.actor.{newUuid, ActorRef} +import akka.actor.{ newUuid, ActorRef } import akka.util.ReflectiveAccess import akka.dispatch._ import akka.config._ @@ -18,11 +18,11 @@ sealed abstract class DurableMailboxStorage(mailboxFQN: String) { val constructorSignature = Array[Class[_]](classOf[ActorRef]) val mailboxClass: Class[_] = ReflectiveAccess.getClassFor(mailboxFQN, classOf[ActorRef].getClassLoader) match { - case Right(clazz) => clazz - case Left(exception) => + case Right(clazz) ⇒ clazz + case Left(exception) ⇒ val cause = exception match { - case i: InvocationTargetException => i.getTargetException - case _ => exception + case i: InvocationTargetException ⇒ i.getTargetException + case _ ⇒ exception } throw new DurableMailboxException("Cannot find class [%s] due to: %s".format(mailboxFQN, cause.toString)) } @@ -31,22 +31,22 @@ sealed abstract class DurableMailboxStorage(mailboxFQN: String) { def createFor(actor: ActorRef): AnyRef = { EventHandler.debug(this, "Creating durable mailbox [%s] for [%s]".format(mailboxClass.getName, actor)) ReflectiveAccess.createInstance[AnyRef](mailboxClass, constructorSignature, Array[AnyRef](actor)) match { - case Right(instance) => instance - case Left(exception) => + case Right(instance) ⇒ instance + case Left(exception) ⇒ val cause = exception match { - case i: InvocationTargetException => i.getTargetException - case _ => exception + case i: InvocationTargetException ⇒ i.getTargetException + case _ ⇒ exception } throw new DurableMailboxException("Cannot instantiate [%s] due to: %s".format(mailboxClass.getName, cause.toString)) } } } -case object RedisDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.RedisBasedMailbox") +case object RedisDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.RedisBasedMailbox") case object MongoNaiveDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.MongoBasedNaiveMailbox") -case object BeanstalkDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.BeanstalkBasedMailbox") -case object FileDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.FileBasedMailbox") -case object ZooKeeperDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.ZooKeeperBasedMailbox") +case object BeanstalkDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.BeanstalkBasedMailbox") +case object FileDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.FileBasedMailbox") +case object ZooKeeperDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.ZooKeeperBasedMailbox") /** * The durable equivalent of Dispatcher @@ -67,7 +67,7 @@ case class DurableDispatcher( _config) { def this(_name: String, _storage: DurableMailboxStorage, throughput: Int, throughputDeadlineTime: Int, mailboxType: MailboxType) = - this(_name, _storage, throughput, throughputDeadlineTime, mailboxType,ThreadPoolConfig()) // Needed for Java API usage + this(_name, _storage, throughput, throughputDeadlineTime, mailboxType, ThreadPoolConfig()) // Needed for Java API usage def this(_name: String, _storage: DurableMailboxStorage, throughput: Int, mailboxType: MailboxType) = this(_name, _storage, throughput, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType) // Needed for Java API usage @@ -105,9 +105,9 @@ case class DurableDispatcher( * @author Jonas Bonér */ case class DurablePinnedDispatcher( - _actor: ActorRef, - _storage: DurableMailboxStorage, - _mailboxType: MailboxType) extends PinnedDispatcher(_actor,_mailboxType) { + _actor: ActorRef, + _storage: DurableMailboxStorage, + _mailboxType: MailboxType) extends PinnedDispatcher(_actor, _mailboxType) { def this(actor: ActorRef, _storage: DurableMailboxStorage) = this(actor, _storage, UnboundedMailbox()) // For Java API @@ -142,7 +142,7 @@ case class DurablePinnedDispatcher( */ class DurableDispatcherConfigurator extends MessageDispatcherConfigurator { def configure(config: Configuration): MessageDispatcher = { - configureThreadPool(config, threadPoolConfig => new DurableDispatcher( + configureThreadPool(config, threadPoolConfig ⇒ new DurableDispatcher( config.getString("name", newUuid.toString), getStorage(config), config.getInt("throughput", Dispatchers.THROUGHPUT), @@ -153,12 +153,12 @@ class DurableDispatcherConfigurator extends MessageDispatcherConfigurator { def getStorage(config: Configuration): DurableMailboxStorage = { val storage = config.getString("storage") map { - case "redis" => RedisDurableMailboxStorage - case "mongodb" => MongoNaiveDurableMailboxStorage - case "beanstalk" => BeanstalkDurableMailboxStorage - case "zookeeper" => ZooKeeperDurableMailboxStorage - case "file" => FileDurableMailboxStorage - case unknown => throw new IllegalArgumentException("[%s] is not a valid storage, valid options are [redis, beanstalk, zookeeper, file]" format unknown) + case "redis" ⇒ RedisDurableMailboxStorage + case "mongodb" ⇒ MongoNaiveDurableMailboxStorage + case "beanstalk" ⇒ BeanstalkDurableMailboxStorage + case "zookeeper" ⇒ ZooKeeperDurableMailboxStorage + case "file" ⇒ FileDurableMailboxStorage + case unknown ⇒ throw new IllegalArgumentException("[%s] is not a valid storage, valid options are [redis, beanstalk, zookeeper, file]" format unknown) } storage.getOrElse(throw new DurableMailboxException("No 'storage' defined for DurableDispatcherConfigurator")) 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 844023e445..381987e9f3 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -5,7 +5,7 @@ package akka.actor.mailbox import MailboxProtocol._ -import akka.actor.{Actor, ActorRef, NullChannel} +import akka.actor.{ Actor, ActorRef, NullChannel } import akka.dispatch._ import akka.event.EventHandler import akka.remote.MessageSerializer @@ -24,7 +24,7 @@ private[akka] object DurableExecutableMailboxConfig { val Name = "[\\.\\/\\$\\s]".r } -class DurableMailboxException private[akka](message: String, cause: Throwable = null) extends AkkaException(message, cause) +class DurableMailboxException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) /** * @author Jonas Bonér @@ -33,13 +33,13 @@ abstract class DurableExecutableMailbox(owner: ActorRef) extends MessageQueue wi import DurableExecutableMailboxConfig._ val ownerAddress = owner.address - val name = "mailbox_" + Name.replaceAllIn(ownerAddress, "_") + val name = "mailbox_" + Name.replaceAllIn(ownerAddress, "_") EventHandler.debug(this, "Creating %s mailbox [%s]".format(getClass.getName, name)) val dispatcher: Dispatcher = owner.dispatcher match { - case e: Dispatcher => e - case _ => null + case e: Dispatcher ⇒ e + case _ ⇒ null } //TODO: switch to RemoteProtocol @@ -49,28 +49,27 @@ abstract class DurableExecutableMailbox(owner: ActorRef) extends MessageQueue wi .setOwnerAddress(ownerAddress) .setMessage(message.toByteString) durableMessage.channel match { - case a : ActorRef => builder.setSenderAddress(a.address) - case _ => + case a: ActorRef ⇒ builder.setSenderAddress(a.address) + case _ ⇒ } builder.build.toByteArray } //TODO: switch to RemoteProtocol def deserialize(bytes: Array[Byte]) = { - val durableMessage = DurableMailboxMessageProtocol.parseFrom(bytes) + val durableMessage = DurableMailboxMessageProtocol.parseFrom(bytes) val messageProtocol = MessageProtocol.parseFrom(durableMessage.getMessage) - val message = MessageSerializer.deserialize(messageProtocol) - val ownerAddress = durableMessage.getOwnerAddress - val owner = Actor.registry.actorFor(ownerAddress).getOrElse( + val message = MessageSerializer.deserialize(messageProtocol) + val ownerAddress = durableMessage.getOwnerAddress + val owner = Actor.registry.actorFor(ownerAddress).getOrElse( throw new DurableMailboxException("No actor could be found for address [" + ownerAddress + "], could not deserialize message.")) - val senderOption = if (durableMessage.hasSenderAddress) { Actor.registry.actorFor(durableMessage.getSenderAddress) } else None val sender = senderOption match { - case Some(ref) => ref - case None => NullChannel + case Some(ref) ⇒ ref + case None ⇒ NullChannel } new MessageInvocation(owner, message, sender) diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala index 93d4226f40..6bece5925b 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 @@ -17,7 +17,7 @@ object DurableMailboxSpecActorFactory { class MailboxTestActor extends Actor { self.lifeCycle = Temporary def receive = { - case "sum" => self.reply("sum") + case "sum" ⇒ self.reply("sum") } } @@ -28,8 +28,7 @@ object DurableMailboxSpecActorFactory { } } -abstract class DurableMailboxSpec(val backendName: String, val storage: DurableMailboxStorage) extends - WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll { +abstract class DurableMailboxSpec(val backendName: String, val storage: DurableMailboxStorage) extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll { import DurableMailboxSpecActorFactory._ implicit val dispatcher = DurableDispatcher(backendName, storage, 1) @@ -39,23 +38,23 @@ abstract class DurableMailboxSpec(val backendName: String, val storage: DurableM "should handle reply to ! for 1 message" in { val latch = new CountDownLatch(1) val queueActor = createMailboxTestActor(backendName + " should handle reply to !") - val sender = localActorOf( new Actor { def receive = { case "sum" => latch.countDown } } ).start + val sender = localActorOf(new Actor { def receive = { case "sum" ⇒ latch.countDown } }).start queueActor.!("sum")(Some(sender)) - latch.await(10, TimeUnit.SECONDS) must be (true) + latch.await(10, TimeUnit.SECONDS) must be(true) } "should handle reply to ! for multiple messages" in { val latch = new CountDownLatch(5) val queueActor = createMailboxTestActor(backendName + " should handle reply to !") - val sender = localActorOf( new Actor { def receive = { case "sum" => latch.countDown } } ).start + val sender = localActorOf(new Actor { def receive = { case "sum" ⇒ latch.countDown } }).start queueActor.!("sum")(Some(sender)) queueActor.!("sum")(Some(sender)) queueActor.!("sum")(Some(sender)) queueActor.!("sum")(Some(sender)) queueActor.!("sum")(Some(sender)) - latch.await(10, TimeUnit.SECONDS) must be (true) + latch.await(10, TimeUnit.SECONDS) must be(true) } } diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala index 9f277ec1e5..63808addd5 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala @@ -3,7 +3,7 @@ */ package akka.actor.mailbox -import akka.actor.{Actor, ActorRef, NullChannel} +import akka.actor.{ Actor, ActorRef, NullChannel } import akka.config.Config.config import akka.dispatch._ import akka.event.EventHandler @@ -16,15 +16,15 @@ import MailboxProtocol._ import com.mongodb.async._ import org.bson.util._ -import org.bson.io.{BasicOutputBuffer, OutputBuffer} +import org.bson.io.{ BasicOutputBuffer, OutputBuffer } import org.bson.types.ObjectId -import java.io.{ByteArrayInputStream, InputStream} +import java.io.{ ByteArrayInputStream, InputStream } import org.bson._ import org.bson.collection._ object BSONSerializableMailbox extends SerializableBSONObject[MongoDurableMessage] with Logging { - + protected[akka] def serializeDurableMsg(msg: MongoDurableMessage)(implicit serializer: BSONSerializer) = { EventHandler.debug(this, "Serializing a durable message to MongoDB: %s".format(msg)) val msgData = MessageSerializer.serialize(msg.message.asInstanceOf[AnyRef]) @@ -36,12 +36,12 @@ object BSONSerializableMailbox extends SerializableBSONObject[MongoDurableMessag b += "ownerAddress" -> msg.ownerAddress msg.channel match { - case a : ActorRef => { b += "senderAddress" -> a.address } - case _ => + case a: ActorRef ⇒ { b += "senderAddress" -> a.address } + case _ ⇒ } - /** + /** * TODO - Figure out a way for custom serialization of the message instance - * TODO - Test if a serializer is registered for the message and if not, use toByteString + * TODO - Test if a serializer is registered for the message and if not, use toByteString */ b += "message" -> new org.bson.types.Binary(0, msgData.toByteArray) val doc = b.result @@ -51,7 +51,7 @@ object BSONSerializableMailbox extends SerializableBSONObject[MongoDurableMessag /* * TODO - Implement some object pooling for the Encoders/decoders - */ + */ def encode(msg: MongoDurableMessage, out: OutputBuffer) = { implicit val serializer = new DefaultBSONSerializer serializer.set(out) @@ -74,10 +74,10 @@ object BSONSerializableMailbox extends SerializableBSONObject[MongoDurableMessag // TODO - Skip the whole doc step for performance, fun, and profit! (Needs Salat / custom Deser) val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument] EventHandler.debug(this, "Deserializing a durable message from MongoDB: %s".format(doc)) - val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData) - val msg = MessageSerializer.deserialize(msgData) + val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData) + val msg = MessageSerializer.deserialize(msgData) val ownerAddress = doc.as[String]("ownerAddress") - val owner = Actor.registry.actorFor(ownerAddress).getOrElse( + val owner = Actor.registry.actorFor(ownerAddress).getOrElse( throw new DurableMailboxException("No actor could be found for address [" + ownerAddress + "], could not deserialize message.")) val senderOption = if (doc.contains("senderAddress")) { @@ -85,8 +85,8 @@ object BSONSerializableMailbox extends SerializableBSONObject[MongoDurableMessag } else None val sender = senderOption match { - case Some(ref) => ref - case None => NullChannel + case Some(ref) ⇒ ref + case None ⇒ NullChannel } MongoDurableMessage(ownerAddress, owner, msg, sender) @@ -99,11 +99,11 @@ object BSONSerializableMailbox extends SerializableBSONObject[MongoDurableMessag /** * Checks for an ID and generates one. * Not all implementers will need this, but it gets invoked nonetheless - * as a signal to BSONDocument, etc implementations to verify an id is there + * as a signal to BSONDocument, etc implementations to verify an id is there * and generate one if needed. */ def checkID(msg: MongoDurableMessage) = msg // OID already generated in wrapper message - + def _id(msg: MongoDurableMessage): Option[AnyRef] = Some(msg._id) } diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala index 15083d9a72..d5c31261ec 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala @@ -13,7 +13,7 @@ import MailboxProtocol._ import com.mongodb.async._ import com.mongodb.async.futures.RequestFutures -import org.bson.collection._ +import org.bson.collection._ class MongoBasedMailboxException(message: String) extends AkkaException(message) @@ -39,9 +39,9 @@ class MongoBasedNaiveMailbox(val owner: ActorRef) extends DurableExecutableMailb val mongoURI = config.getString(URI_CONFIG_KEY) val writeTimeout = config.getInt(WRITE_TIMEOUT_KEY, 3000) val readTimeout = config.getInt(READ_TIMEOUT_KEY, 3000) - - @volatile private var mongo = connect() + @volatile + private var mongo = connect() def enqueue(msg: MessageInvocation) = { EventHandler.debug(this, @@ -50,40 +50,45 @@ class MongoBasedNaiveMailbox(val owner: ActorRef) extends DurableExecutableMailb val durableMessage = MongoDurableMessage(ownerAddress, msg.receiver, msg.message, msg.channel) // todo - do we need to filter the actor name at all for safe collection naming? val result = new DefaultPromise[Boolean](writeTimeout) - mongo.insert(durableMessage, false)(RequestFutures.write { wr: Either[Throwable, (Option[AnyRef], WriteResult)] => wr match { - case Right((oid, wr)) => result.completeWithResult(true) - case Left(t) => result.completeWithException(t) - }}) + mongo.insert(durableMessage, false)(RequestFutures.write { wr: Either[Throwable, (Option[AnyRef], WriteResult)] ⇒ + wr match { + case Right((oid, wr)) ⇒ result.completeWithResult(true) + case Left(t) ⇒ result.completeWithException(t) + } + }) result.as[Boolean].orNull } def dequeue: MessageInvocation = withErrorHandling { - /** + /** * Retrieves first item in natural order (oldest first, assuming no modification/move) * Waits 3 seconds for now for a message, else pops back out. * TODO - How do we handle fetch, but sleep if nothing is in there cleanly? - * TODO - Should we have a specific query in place? Which way do we sort? + * TODO - Should we have a specific query in place? Which way do we sort? * TODO - Error handling version! */ val msgInvocation = new DefaultPromise[MessageInvocation](readTimeout) - mongo.findAndRemove(Document.empty) { doc: Option[MongoDurableMessage] => doc match { - case Some(msg) => { - EventHandler.debug(this, - "\nDEQUEUING message in mongo-based mailbox [%s]".format(msg)) - msgInvocation.completeWithResult(msg.messageInvocation()) - EventHandler.debug(this, - "\nDEQUEUING messageInvocation in mongo-based mailbox [%s]".format(msgInvocation)) + mongo.findAndRemove(Document.empty) { doc: Option[MongoDurableMessage] ⇒ + doc match { + case Some(msg) ⇒ { + EventHandler.debug(this, + "\nDEQUEUING message in mongo-based mailbox [%s]".format(msg)) + msgInvocation.completeWithResult(msg.messageInvocation()) + EventHandler.debug(this, + "\nDEQUEUING messageInvocation in mongo-based mailbox [%s]".format(msgInvocation)) + } + case None ⇒ + { + EventHandler.info(this, + "\nNo matching document found. Not an error, just an empty queue.") + msgInvocation.completeWithResult(null) + } + () } - case None => { - EventHandler.info(this, - "\nNo matching document found. Not an error, just an empty queue.") - msgInvocation.completeWithResult(null) - } - () - }} + } msgInvocation.as[MessageInvocation].orNull - } + } def size: Int = { val count = new DefaultPromise[Int](readTimeout) @@ -91,7 +96,6 @@ class MongoBasedNaiveMailbox(val owner: ActorRef) extends DurableExecutableMailb count.as[Int].getOrElse(-1) } - def isEmpty: Boolean = size == 0 //TODO review find other solution, this will be very expensive private[akka] def connect() = { @@ -99,33 +103,33 @@ class MongoBasedNaiveMailbox(val owner: ActorRef) extends DurableExecutableMailb EventHandler.info(this, "\nCONNECTING mongodb { uri : [%s] } ".format(mongoURI)) val _dbh = MongoConnection.fromURI(mongoURI.get) match { - case (conn, None, None) => { + case (conn, None, None) ⇒ { throw new UnsupportedOperationException("You must specify a database name to use with MongoDB; please see the MongoDB Connection URI Spec: 'http://www.mongodb.org/display/DOCS/Connections'") } - case (conn, Some(db), Some(coll)) => { + case (conn, Some(db), Some(coll)) ⇒ { EventHandler.warning(this, "\nCollection name (%s) specified in MongoURI Config will be used as a prefix for mailbox names".format(coll.name)) db("%s.%s".format(coll.name, name)) } - case (conn, Some(db), None) => { + case (conn, Some(db), None) ⇒ { db("mailbox.%s".format(name)) } - case default => throw new IllegalArgumentException("Illegal or unexpected response from Mongo Connection URI Parser: %s".format(default)) + case default ⇒ throw new IllegalArgumentException("Illegal or unexpected response from Mongo Connection URI Parser: %s".format(default)) } EventHandler.debug(this, "\nCONNECTED to mongodb { dbh: '%s | %s'} ".format(_dbh, _dbh.name)) _dbh } - private def withErrorHandling[T](body: => T): T = { + private def withErrorHandling[T](body: ⇒ T): T = { try { body } catch { - case e: Exception => { + case e: Exception ⇒ { mongo = connect() body } - case e => { + case e ⇒ { val error = new MongoBasedMailboxException("Could not connect to MongoDB server") EventHandler.error(error, this, "Could not connect to MongoDB server") throw error diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala index 01a666b16f..7cdca7f8b0 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala @@ -3,7 +3,7 @@ */ package akka.actor.mailbox -import akka.actor.{ActorRef, UntypedChannel, NullChannel} +import akka.actor.{ ActorRef, UntypedChannel, NullChannel } import akka.config.Config.config import akka.dispatch._ import akka.event.EventHandler @@ -21,9 +21,9 @@ import java.io.InputStream import org.bson.collection._ /** - * A container message for durable mailbox messages, which can be easily stuffed into + * A container message for durable mailbox messages, which can be easily stuffed into * and out of MongoDB. - * + * * Does not use the Protobuf protocol, instead using a pure Mongo based serialization for sanity * (and mongo-iness). * @@ -35,7 +35,7 @@ import org.bson.collection._ case class MongoDurableMessage(val ownerAddress: String, val receiver: ActorRef, val message: Any, - val channel: UntypedChannel, + val channel: UntypedChannel, val _id: ObjectId = new ObjectId) { def messageInvocation() = MessageInvocation(this.receiver, this.message, this.channel) diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/test/scala/akka/actor/mailbox/MongoBasedMailboxSpec.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/test/scala/akka/actor/mailbox/MongoBasedMailboxSpec.scala index bb34252035..b0b9c737fd 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/test/scala/akka/actor/mailbox/MongoBasedMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/test/scala/akka/actor/mailbox/MongoBasedMailboxSpec.scala @@ -12,13 +12,13 @@ import java.util.concurrent.CountDownLatch import akka.config.Supervision.Temporary import akka.dispatch.MessageDispatcher -class MongoBasedMailboxSpec extends DurableMailboxSpec("mongodb", MongoNaiveDurableMailboxStorage) { - import org.apache.log4j.{Logger, Level} +class MongoBasedMailboxSpec extends DurableMailboxSpec("mongodb", MongoNaiveDurableMailboxStorage) { + import org.apache.log4j.{ Logger, Level } import com.mongodb.async._ val mongo = MongoConnection("localhost", 27017)("akka") - mongo.dropDatabase(){ success => } + mongo.dropDatabase() { success ⇒ } Logger.getRootLogger.setLevel(Level.DEBUG) } diff --git a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala index 4a4ade1e38..ffccee269b 100644 --- a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala @@ -41,9 +41,10 @@ import CommonsCodec.Base64StringEncoder._ * @author Jonas Bonér */ class RedisBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(owner) { - val nodes = config.getList("akka.persistence.redis.cluster")// need an explicit definition in akka-conf + val nodes = config.getList("akka.persistence.redis.cluster") // need an explicit definition in akka-conf - @volatile private var db = connect() //review Is the Redis connection thread safe? + @volatile + private var db = connect() //review Is the Redis connection thread safe? def enqueue(message: MessageInvocation) = { EventHandler.debug(this, @@ -61,8 +62,8 @@ class RedisBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(ow "\nDEQUEUING message in redis-based mailbox [%s]".format(messageInvocation)) messageInvocation } catch { - case e: java.util.NoSuchElementException => null - case e => + case e: java.util.NoSuchElementException ⇒ null + case e ⇒ EventHandler.error(e, this, "Couldn't dequeue from Redis-based mailbox") throw e } @@ -76,13 +77,13 @@ class RedisBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(ow private[akka] def connect() = nodes match { - case Seq() => + case Seq() ⇒ // no cluster defined new RedisClient( config.getString("akka.actor.mailbox.redis.hostname", "127.0.0.1"), config.getInt("akka.actor.mailbox.redis.port", 6379)) - case s => + case s ⇒ // with cluster import com.redis.cluster._ EventHandler.info(this, "Running on Redis cluster") @@ -91,15 +92,15 @@ class RedisBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbox(ow } } - private def withErrorHandling[T](body: => T): T = { + private def withErrorHandling[T](body: ⇒ T): T = { try { body } catch { - case e: RedisConnectionException => { + case e: RedisConnectionException ⇒ { db = connect() body } - case e => + case e ⇒ val error = new RedisBasedMailboxException("Could not connect to Redis server") EventHandler.error(error, this, "Could not connect to Redis server") throw error diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala index 8d577a8aca..f924c21935 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala @@ -22,18 +22,18 @@ class ZooKeeperBasedMailboxException(message: String) extends AkkaException(mess */ private[akka] object ZooKeeperMailboxConfig { val zkServerAddresses = config.getString("akka.actor.mailbox.zookeeper.server-addresses", "localhost:2181") - val sessionTimeout = Duration(config.getInt("akka.actor.mailbox.zookeeper.session-timeout", 60), TIME_UNIT).toMillis.toInt + val sessionTimeout = Duration(config.getInt("akka.actor.mailbox.zookeeper.session-timeout", 60), TIME_UNIT).toMillis.toInt val connectionTimeout = Duration(config.getInt("akka.actor.mailbox.zookeeper.connection-timeout", 60), TIME_UNIT).toMillis.toInt - val blockingQueue = config.getBool("akka.actor.mailbox.zookeeper.blocking-queue", true) + val blockingQueue = config.getBool("akka.actor.mailbox.zookeeper.blocking-queue", true) - val queueNode = "/queues" + val queueNode = "/queues" val queuePathTemplate = queueNode + "/%s" object serializer extends ZkSerializer { def serialize(data: AnyRef): Array[Byte] = data match { - case d: DurableMailboxMessageProtocol => d.toByteArray - case null => throw new ZooKeeperBasedMailboxException("Expected a DurableMailboxMessageProtocol message, was null") - case _ => throw new ZooKeeperBasedMailboxException("Expected a DurableMailboxMessageProtocol message, was [" + data.getClass + "]") + case d: DurableMailboxMessageProtocol ⇒ d.toByteArray + case null ⇒ throw new ZooKeeperBasedMailboxException("Expected a DurableMailboxMessageProtocol message, was null") + case _ ⇒ throw new ZooKeeperBasedMailboxException("Expected a DurableMailboxMessageProtocol message, was [" + data.getClass + "]") } def deserialize(bytes: Array[Byte]): AnyRef = DurableMailboxMessageProtocol.parseFrom(bytes) @@ -61,9 +61,9 @@ class ZooKeeperBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbo "\nDEQUEUING message in zookeeper-based mailbox [%s]".format(messageInvocation)) messageInvocation } catch { - case e: java.util.NoSuchElementException => null - case e: InterruptedException => null - case e => + case e: java.util.NoSuchElementException ⇒ null + case e: InterruptedException ⇒ null + case e ⇒ EventHandler.error(e, this, "Couldn't dequeue from ZooKeeper-based mailbox") throw e } @@ -76,7 +76,7 @@ class ZooKeeperBasedMailbox(val owner: ActorRef) extends DurableExecutableMailbo queue.clear true } catch { - case e => false + case e ⇒ false } def close = zkClient.close diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala index b798d8fe8e..62ab97f750 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/scala/akka/actor/mailbox/ZooKeeperBasedMailboxSpec.scala @@ -18,8 +18,8 @@ class ZooKeeperBasedMailboxSpec extends DurableMailboxSpec("ZooKeeper", ZooKeepe override def afterEach() { Actor.registry.local.actors.foreach(_.mailbox match { - case zkm: ZooKeeperBasedMailbox => zkm.close - case _ => () + case zkm: ZooKeeperBasedMailbox ⇒ zkm.close + case _ ⇒ () }) super.afterEach } diff --git a/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Actors.scala b/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Actors.scala index 98e8462ec4..81efbd72bd 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Actors.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Actors.scala @@ -2,8 +2,8 @@ package sample.camel import org.apache.camel.Exchange -import akka.actor.{Actor, ActorRef, ActorRegistry} -import akka.camel.{Ack, Failure, Producer, Message, Consumer} +import akka.actor.{ Actor, ActorRef, ActorRegistry } +import akka.camel.{ Ack, Failure, Producer, Message, Consumer } /** * Client-initiated remote actor. @@ -12,7 +12,7 @@ class RemoteActor1 extends Actor with Consumer { def endpointUri = "jetty:http://localhost:6644/camel/remote-actor-1" protected def receive = { - case msg: Message => self.reply(Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote1"))) + case msg: Message ⇒ self.reply(Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote1"))) } } @@ -23,7 +23,7 @@ class RemoteActor2 extends Actor with Consumer { def endpointUri = "jetty:http://localhost:6644/camel/remote-actor-2" protected def receive = { - case msg: Message => self.reply(Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote2"))) + case msg: Message ⇒ self.reply(Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote2"))) } } @@ -36,7 +36,7 @@ class Consumer1 extends Actor with Consumer { def endpointUri = "file:data/input/actor" def receive = { - case msg: Message => println("received %s" format msg.bodyAs[String]) + case msg: Message ⇒ println("received %s" format msg.bodyAs[String]) } } @@ -44,7 +44,7 @@ class Consumer2 extends Actor with Consumer { def endpointUri = "jetty:http://0.0.0.0:8877/camel/default" def receive = { - case msg: Message => self.reply("Hello %s" format msg.bodyAs[String]) + case msg: Message ⇒ self.reply("Hello %s" format msg.bodyAs[String]) } } @@ -52,7 +52,7 @@ class Consumer3(transformer: ActorRef) extends Actor with Consumer { def endpointUri = "jetty:http://0.0.0.0:8877/camel/welcome" def receive = { - case msg: Message => transformer.forward(msg.setBodyAs[String]) + case msg: Message ⇒ transformer.forward(msg.setBodyAs[String]) } } @@ -60,12 +60,12 @@ class Consumer4 extends Actor with Consumer { def endpointUri = "jetty:http://0.0.0.0:8877/camel/stop" def receive = { - case msg: Message => msg.bodyAs[String] match { - case "stop" => { + case msg: Message ⇒ msg.bodyAs[String] match { + case "stop" ⇒ { self.reply("Consumer4 stopped") self.stop } - case body => self.reply(body) + case body ⇒ self.reply(body) } } } @@ -74,7 +74,7 @@ class Consumer5 extends Actor with Consumer { def endpointUri = "jetty:http://0.0.0.0:8877/camel/start" def receive = { - case _ => { + case _ ⇒ { Actor.actorOf[Consumer4].start self.reply("Consumer4 started") } @@ -83,15 +83,15 @@ class Consumer5 extends Actor with Consumer { class Transformer(producer: ActorRef) extends Actor { protected def receive = { - case msg: Message => producer.forward(msg.transformBody( (body: String) => "- %s -" format body)) + case msg: Message ⇒ producer.forward(msg.transformBody((body: String) ⇒ "- %s -" format body)) } } -class Subscriber(name:String, uri: String) extends Actor with Consumer { +class Subscriber(name: String, uri: String) extends Actor with Consumer { def endpointUri = uri protected def receive = { - case msg: Message => println("%s received: %s" format (name, msg.body)) + case msg: Message ⇒ println("%s received: %s" format (name, msg.body)) } } @@ -104,7 +104,7 @@ class PublisherBridge(uri: String, publisher: ActorRef) extends Actor with Consu def endpointUri = uri protected def receive = { - case msg: Message => { + case msg: Message ⇒ { publisher ! msg.bodyAs[String] self.reply("message published") } @@ -115,7 +115,7 @@ class HttpConsumer(producer: ActorRef) extends Actor with Consumer { def endpointUri = "jetty:http://0.0.0.0:8875/" protected def receive = { - case msg => producer forward msg + case msg ⇒ producer forward msg } } @@ -124,19 +124,19 @@ class HttpProducer(transformer: ActorRef) extends Actor with Producer { override protected def receiveBeforeProduce = { // only keep Exchange.HTTP_PATH message header (which needed by bridge endpoint) - case msg: Message => msg.setHeaders(msg.headers(Set(Exchange.HTTP_PATH))) + case msg: Message ⇒ msg.setHeaders(msg.headers(Set(Exchange.HTTP_PATH))) } override protected def receiveAfterProduce = { // do not reply but forward result to transformer - case msg => transformer forward msg + case msg ⇒ transformer forward msg } } class HttpTransformer extends Actor { protected def receive = { - case msg: Message => self.reply(msg.transformBody {body: String => body replaceAll ("Akka ", "AKKA ")}) - case msg: Failure => self.reply(msg) + case msg: Message ⇒ self.reply(msg.transformBody { body: String ⇒ body replaceAll ("Akka ", "AKKA ") }) + case msg: Failure ⇒ self.reply(msg) } } @@ -147,7 +147,7 @@ class FileConsumer extends Actor with Consumer { var counter = 0 def receive = { - case msg: Message => { + case msg: Message ⇒ { if (counter == 2) { println("received %s" format msg.bodyAs[String]) self.reply(Ack) diff --git a/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Boot.scala b/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Boot.scala index 8f3d16b318..32628ea04f 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Boot.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Boot.scala @@ -1,6 +1,6 @@ package sample.camel -import org.apache.camel.{Exchange, Processor} +import org.apache.camel.{ Exchange, Processor } import org.apache.camel.builder.RouteBuilder import org.apache.camel.impl.DefaultCamelContext import org.apache.camel.spring.spi.ApplicationContextRegistry @@ -74,7 +74,7 @@ class Boot { val jmsUri = "jms:topic:test" val jmsSubscriber1 = actorOf(new Subscriber("jms-subscriber-1", jmsUri)).start val jmsSubscriber2 = actorOf(new Subscriber("jms-subscriber-2", jmsUri)).start - val jmsPublisher = actorOf(new Publisher(jmsUri), "jms-publisher").start + val jmsPublisher = actorOf(new Publisher(jmsUri), "jms-publisher").start //val cometdPublisherBridge = actorOf(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/cometd", cometdPublisher)).start val jmsPublisherBridge = actorOf(new PublisherBridge("jetty:http://0.0.0.0:8877/camel/pub/jms", jmsPublisher)).start diff --git a/akka-samples/akka-sample-camel/src/main/scala/sample/camel/StandaloneApplication.scala b/akka-samples/akka-sample-camel/src/main/scala/sample/camel/StandaloneApplication.scala index 812d7f065a..9e252208af 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/sample/camel/StandaloneApplication.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/sample/camel/StandaloneApplication.scala @@ -1,11 +1,11 @@ package sample.camel -import org.apache.camel.impl.{DefaultCamelContext, SimpleRegistry} +import org.apache.camel.impl.{ DefaultCamelContext, SimpleRegistry } import org.apache.camel.builder.RouteBuilder import org.apache.camel.spring.spi.ApplicationContextRegistry import org.springframework.context.support.ClassPathXmlApplicationContext -import akka.actor.{Actor, TypedActor} +import akka.actor.{ Actor, TypedActor } import akka.actor.TypedActor.Configuration._ import akka.camel._ @@ -92,7 +92,7 @@ object StandaloneJmsApplication extends App { startCamelService val jmsUri = "jms:topic:test" - val jmsPublisher = Actor.actorOf(new Publisher(jmsUri), "jms-publisher").start + val jmsPublisher = Actor.actorOf(new Publisher(jmsUri), "jms-publisher").start mandatoryService.awaitEndpointActivation(2) { Actor.actorOf(new Subscriber("jms-subscriber-1", jmsUri)).start @@ -100,12 +100,12 @@ object StandaloneJmsApplication extends App { } // Send 10 messages to via publisher actor - for(i <- 1 to 10) { + for (i ← 1 to 10) { jmsPublisher ! ("Akka rocks (%d)" format i) } // Send 10 messages to JMS topic directly - for(i <- 1 to 10) { + for (i ← 1 to 10) { CamelContextManager.mandatoryTemplate.sendBody(jmsUri, "Camel rocks (%d)" format i) } diff --git a/akka-samples/akka-sample-camel/src/test/scala/sample/camel/HttpConcurrencyTestStress.scala b/akka-samples/akka-sample-camel/src/test/scala/sample/camel/HttpConcurrencyTestStress.scala index bbe039af22..0254e1e481 100644 --- a/akka-samples/akka-sample-camel/src/test/scala/sample/camel/HttpConcurrencyTestStress.scala +++ b/akka-samples/akka-sample-camel/src/test/scala/sample/camel/HttpConcurrencyTestStress.scala @@ -8,7 +8,7 @@ import org.junit._ import org.scalatest.junit.JUnitSuite import akka.actor.Actor._ -import akka.actor.{ActorRegistry, ActorRef, Actor} +import akka.actor.{ ActorRegistry, ActorRef, Actor } import akka.camel._ import akka.camel.CamelServiceManager._ import akka.routing.CyclicIterator @@ -20,7 +20,8 @@ import akka.routing.Routing._ class HttpConcurrencyTestStress extends JUnitSuite { import HttpConcurrencyTestStress._ - @Test def shouldProcessMessagesConcurrently = { + @Test + def shouldProcessMessagesConcurrently = { /* TODO: fix stress test val num = 50 @@ -49,7 +50,7 @@ object HttpConcurrencyTestStress { def beforeClass: Unit = { startCamelService - val workers = for (i <- 1 to 8) yield actorOf[HttpServerWorker].start + val workers = for (i ← 1 to 8) yield actorOf[HttpServerWorker].start val balancer = loadBalancerActor(new CyclicIterator(workers.toList)) //service.get.awaitEndpointActivation(1) { @@ -68,12 +69,12 @@ object HttpConcurrencyTestStress { var correlationIds = Set[Any]() override protected def receive = { - case "getCorrelationIdCount" => self.reply(correlationIds.size) - case msg => super.receive(msg) + case "getCorrelationIdCount" ⇒ self.reply(correlationIds.size) + case msg ⇒ super.receive(msg) } override protected def receiveAfterProduce = { - case msg: Message => { + case msg: Message ⇒ { val corr = msg.headers(Message.MessageExchangeId) val body = msg.bodyAs[String] correlationIds += corr @@ -89,13 +90,13 @@ object HttpConcurrencyTestStress { var counter = 0 def receive = { - case msg => balancer forward msg + case msg ⇒ balancer forward msg } } class HttpServerWorker extends Actor { protected def receive = { - case msg => self.reply(msg) + case msg ⇒ self.reply(msg) } } } diff --git a/akka-samples/akka-sample-camel/src/test/scala/sample/camel/RemoteConsumerTest.scala b/akka-samples/akka-sample-camel/src/test/scala/sample/camel/RemoteConsumerTest.scala index 92b204ec53..98c8ed74d7 100644 --- a/akka-samples/akka-sample-camel/src/test/scala/sample/camel/RemoteConsumerTest.scala +++ b/akka-samples/akka-sample-camel/src/test/scala/sample/camel/RemoteConsumerTest.scala @@ -1,6 +1,6 @@ package sample.camel -import org.scalatest.{GivenWhenThen, BeforeAndAfterAll, FeatureSpec} +import org.scalatest.{ GivenWhenThen, BeforeAndAfterAll, FeatureSpec } import akka.actor.Actor._ import akka.actor._ @@ -94,8 +94,8 @@ object RemoteConsumerTest { def endpointUri = "direct:remote-consumer" protected def receive = { - case "init" => self.reply("done") - case m: Message => self.reply("remote actor: %s" format m.body) + case "init" ⇒ self.reply("done") + case m: Message ⇒ self.reply("remote actor: %s" format m.body) } } } diff --git a/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala b/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala index a10528961f..77fb678a8a 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala @@ -2,7 +2,7 @@ package sample.fsm.buncher import scala.reflect.ClassManifest import akka.util.Duration -import akka.actor.{FSM, Actor, ActorRef} +import akka.actor.{ FSM, Actor, ActorRef } /* * generic typed object buncher. @@ -22,8 +22,8 @@ object GenericBuncher { case object Flush // send out current queue immediately case object Stop // poison pill - class MsgExtractor[A : Manifest] { - def unapply(m : AnyRef) : Option[A] = { + class MsgExtractor[A: Manifest] { + def unapply(m: AnyRef): Option[A] = { if (ClassManifest.fromClass(m.getClass) <:< manifest[A]) { Some(m.asInstanceOf[A]) } else { @@ -32,75 +32,75 @@ object GenericBuncher { } } } - -abstract class GenericBuncher[A : Manifest, B] (val singleTimeout : Duration, val multiTimeout : Duration) - extends Actor with FSM[GenericBuncher.State, B] { + +abstract class GenericBuncher[A: Manifest, B](val singleTimeout: Duration, val multiTimeout: Duration) + extends Actor with FSM[GenericBuncher.State, B] { import GenericBuncher._ import FSM._ - - protected def empty : B - protected def merge(acc : B, elem : A) : B - protected def send(acc : B) : Unit - - protected def flush(acc : B) = { + + protected def empty: B + protected def merge(acc: B, elem: A): B + protected def send(acc: B): Unit + + protected def flush(acc: B) = { send(acc) cancelTimer("multi") goto(Idle) using empty } - + val Msg = new MsgExtractor[A] - + startWith(Idle, empty) - + when(Idle) { - case Event(Msg(m), acc) => + case Event(Msg(m), acc) ⇒ setTimer("multi", StateTimeout, multiTimeout, false) goto(Active) using merge(acc, m) - case Event(Flush, _) => stay - case Event(Stop, _) => stop + case Event(Flush, _) ⇒ stay + case Event(Stop, _) ⇒ stop } - + when(Active, stateTimeout = Some(singleTimeout)) { - case Event(Msg(m), acc) => + case Event(Msg(m), acc) ⇒ stay using merge(acc, m) - case Event(StateTimeout, acc) => + case Event(StateTimeout, acc) ⇒ flush(acc) - case Event(Flush, acc) => + case Event(Flush, acc) ⇒ flush(acc) - case Event(Stop, acc) => + case Event(Stop, acc) ⇒ send(acc) cancelTimer("multi") stop } - + initialize } object Buncher { - case class Target(target : ActorRef) // for setting the target for default send action - + case class Target(target: ActorRef) // for setting the target for default send action + val Stop = GenericBuncher.Stop // make special message objects visible for Buncher clients val Flush = GenericBuncher.Flush - - def apply[A : Manifest](singleTimeout : Duration, - multiTimeout : Duration) = + + def apply[A: Manifest](singleTimeout: Duration, + multiTimeout: Duration) = Actor.actorOf(new Buncher[A](singleTimeout, multiTimeout)) } -class Buncher[A : Manifest](singleTimeout : Duration, multiTimeout : Duration) +class Buncher[A: Manifest](singleTimeout: Duration, multiTimeout: Duration) extends GenericBuncher[A, List[A]](singleTimeout, multiTimeout) { - + import Buncher._ - private var target : Option[ActorRef] = None - protected def send(acc : List[A]) : Unit = if (target.isDefined) target.get ! acc.reverse - - protected def empty : List[A] = Nil - - protected def merge(l : List[A], elem : A) = elem :: l + private var target: Option[ActorRef] = None + protected def send(acc: List[A]): Unit = if (target.isDefined) target.get ! acc.reverse + + protected def empty: List[A] = Nil + + protected def merge(l: List[A], elem: A) = elem :: l whenUnhandled { - case Event(Target(t), _) => + case Event(Target(t), _) ⇒ target = Some(t) stay } 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 156e852b37..7e313eb369 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala @@ -3,7 +3,7 @@ package sample.fsm.dining.become //Akka adaptation of //http://www.dalnefre.com/wp/2010/08/dining-philosophers-in-humus/ -import akka.actor.{Scheduler, ActorRef, Actor} +import akka.actor.{ Scheduler, ActorRef, Actor } import akka.actor.Actor._ import java.util.concurrent.TimeUnit @@ -27,15 +27,15 @@ class Chopstick(name: String) extends Actor { //It will refuse to be taken by other hakkers //But the owning hakker can put it back def takenBy(hakker: ActorRef): Receive = { - case Take(otherHakker) => + case Take(otherHakker) ⇒ otherHakker ! Busy(self) - case Put(`hakker`) => + case Put(`hakker`) ⇒ become(available) } //When a Chopstick is available, it can be taken by a hakker def available: Receive = { - case Take(hakker) => + case Take(hakker) ⇒ become(takenBy(hakker)) hakker ! Taken(self) } @@ -47,14 +47,14 @@ class Chopstick(name: String) extends Actor { /* * A hakker is an awesome dude or dudett who either thinks about hacking or has to eat ;-) */ -class Hakker(name: String,left: ActorRef, right: ActorRef) extends Actor { +class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor { //When a hakker is thinking it can become hungry //and try to pick up its chopsticks and eat def thinking: Receive = { - case Eat => + case Eat ⇒ become(hungry) - left ! Take(self) + left ! Take(self) right ! Take(self) } @@ -63,11 +63,11 @@ class Hakker(name: String,left: ActorRef, right: ActorRef) extends Actor { //If the hakkers first attempt at grabbing a chopstick fails, //it starts to wait for the response of the other grab def hungry: Receive = { - case Taken(`left`) => - become(waiting_for(right,left)) - case Taken(`right`) => - become(waiting_for(left,right)) - case Busy(chopstick) => + case Taken(`left`) ⇒ + become(waiting_for(right, left)) + case Taken(`right`) ⇒ + become(waiting_for(left, right)) + case Busy(chopstick) ⇒ become(denied_a_chopstick) } @@ -75,12 +75,12 @@ class Hakker(name: String,left: ActorRef, right: ActorRef) extends Actor { //and start eating, or the other chopstick was busy, and the hakker goes //back to think about how he should obtain his chopsticks :-) def waiting_for(chopstickToWaitFor: ActorRef, otherChopstick: ActorRef): Receive = { - case Taken(`chopstickToWaitFor`) => - println("%s has picked up %s and %s, and starts to eat",name,left.address,right.address) + case Taken(`chopstickToWaitFor`) ⇒ + println("%s has picked up %s and %s, and starts to eat", name, left.address, right.address) become(eating) - Scheduler.scheduleOnce(self,Think,5,TimeUnit.SECONDS) + Scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS) - case Busy(chopstick) => + case Busy(chopstick) ⇒ become(thinking) otherChopstick ! Put(self) self ! Eat @@ -90,11 +90,11 @@ class Hakker(name: String,left: ActorRef, right: ActorRef) extends Actor { //he needs to put it back if he got the other one. //Then go back and think and try to grab the chopsticks again def denied_a_chopstick: Receive = { - case Taken(chopstick) => + case Taken(chopstick) ⇒ become(thinking) chopstick ! Put(self) self ! Eat - case Busy(chopstick) => + case Busy(chopstick) ⇒ become(thinking) self ! Eat } @@ -102,20 +102,20 @@ class Hakker(name: String,left: ActorRef, right: ActorRef) extends Actor { //When a hakker is eating, he can decide to start to think, //then he puts down his chopsticks and starts to think def eating: Receive = { - case Think => + case Think ⇒ become(thinking) - left ! Put(self) + left ! Put(self) right ! Put(self) - println("%s puts down his chopsticks and starts to think",name) - Scheduler.scheduleOnce(self,Eat,5,TimeUnit.SECONDS) + println("%s puts down his chopsticks and starts to think", name) + Scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) } //All hakkers start in a non-eating state def receive = { - case Think => - println("%s starts to think",name) + case Think ⇒ + println("%s starts to think", name) become(thinking) - Scheduler.scheduleOnce(self,Eat,5,TimeUnit.SECONDS) + Scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) } } @@ -125,11 +125,11 @@ class Hakker(name: String,left: ActorRef, right: ActorRef) extends Actor { object DiningHakkers { def run { //Create 5 chopsticks - val chopsticks = for(i <- 1 to 5) yield actorOf(new Chopstick("Chopstick "+i)).start() + val chopsticks = for (i ← 1 to 5) yield actorOf(new Chopstick("Chopstick " + i)).start() //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 - } yield actorOf(new Hakker(name,chopsticks(i),chopsticks((i+1) % 5))).start() + (name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex + } yield actorOf(new Hakker(name, chopsticks(i), chopsticks((i + 1) % 5))).start() //Signal all hakkers that they should start thinking, and watch the show hakkers.foreach(_ ! Think) 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 ddca7c8fa0..7b02bc6ce7 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -1,6 +1,6 @@ package sample.fsm.dining.fsm -import akka.actor.{ActorRef, Actor, FSM} +import akka.actor.{ ActorRef, Actor, FSM } import akka.actor.FSM._ import Actor._ import akka.util.Duration @@ -37,7 +37,7 @@ class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] { // When a chopstick is available, it can be taken by a some hakker when(Available) { - case Event(Take, _) => + case Event(Take, _) ⇒ goto(Taken) using TakenBy(self.sender) replying Taken(self) } @@ -45,9 +45,9 @@ class Chopstick(name: String) extends Actor with FSM[ChopstickState, TakenBy] { // It will refuse to be taken by other hakkers // But the owning hakker can put it back when(Taken) { - case Event(Take, currentState) => + case Event(Take, currentState) ⇒ stay replying Busy(self) - case Event(Put, TakenBy(hakker)) if self.sender == hakker => + case Event(Put, TakenBy(hakker)) if self.sender == hakker ⇒ goto(Available) using TakenBy(None) } @@ -86,7 +86,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit startWith(Waiting, TakenChopsticks(None, None)) when(Waiting) { - case Event(Think, _) => + case Event(Think, _) ⇒ println("%s starts to think", name) startThinking(5 seconds) } @@ -94,7 +94,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit //When a hakker is thinking it can become hungry //and try to pick up its chopsticks and eat when(Thinking) { - case Event(StateTimeout, _) => + case Event(StateTimeout, _) ⇒ left ! Take right ! Take goto(Hungry) @@ -105,11 +105,11 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit // If the hakkers first attempt at grabbing a chopstick fails, // it starts to wait for the response of the other grab when(Hungry) { - case Event(Taken(`left`), _) => + case Event(Taken(`left`), _) ⇒ goto(WaitForOtherChopstick) using TakenChopsticks(Some(left), None) - case Event(Taken(`right`), _) => + case Event(Taken(`right`), _) ⇒ goto(WaitForOtherChopstick) using TakenChopsticks(None, Some(right)) - case Event(Busy(_), _) => + case Event(Busy(_), _) ⇒ goto(FirstChopstickDenied) } @@ -117,9 +117,9 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit // and start eating, or the other chopstick was busy, and the hakker goes // back to think about how he should obtain his chopsticks :-) when(WaitForOtherChopstick) { - case Event(Taken(`left`), TakenChopsticks(None, Some(right))) => startEating(left, right) - case Event(Taken(`right`), TakenChopsticks(Some(left), None)) => startEating(left, right) - case Event(Busy(chopstick), TakenChopsticks(leftOption, rightOption)) => + case Event(Taken(`left`), TakenChopsticks(None, Some(right))) ⇒ startEating(left, right) + case Event(Taken(`right`), TakenChopsticks(Some(left), None)) ⇒ startEating(left, right) + case Event(Busy(chopstick), TakenChopsticks(leftOption, rightOption)) ⇒ leftOption.foreach(_ ! Put) rightOption.foreach(_ ! Put) startThinking(10 milliseconds) @@ -134,17 +134,17 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit // he needs to put it back if he got the other one. // Then go back and think and try to grab the chopsticks again when(FirstChopstickDenied) { - case Event(Taken(secondChopstick), _) => + case Event(Taken(secondChopstick), _) ⇒ secondChopstick ! Put startThinking(10 milliseconds) - case Event(Busy(chopstick), _) => + case Event(Busy(chopstick), _) ⇒ startThinking(10 milliseconds) } // When a hakker is eating, he can decide to start to think, // then he puts down his chopsticks and starts to think when(Eating) { - case Event(StateTimeout, _) => + case Event(StateTimeout, _) ⇒ println("%s puts down his chopsticks and starts to think", name) left ! Put right ! Put @@ -166,10 +166,10 @@ object DiningHakkersOnFsm { def run = { // Create 5 chopsticks - val chopsticks = for (i <- 1 to 5) yield actorOf(new Chopstick("Chopstick " + i)).start() + val chopsticks = for (i ← 1 to 5) yield actorOf(new Chopstick("Chopstick " + i)).start() // 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 + val hakkers = for { + (name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex } yield actorOf(new FSMHakker(name, chopsticks(i), chopsticks((i + 1) % 5))).start() hakkers.foreach(_ ! Think) diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala index dca421c941..3c32bfa911 100644 --- a/akka-stm/src/main/scala/akka/agent/Agent.scala +++ b/akka-stm/src/main/scala/akka/agent/Agent.scala @@ -283,7 +283,7 @@ class AgentUpdater[T](agent: Agent[T]) extends Actor { def receive = { case update: Update[_] ⇒ - self.tryReply(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T => T] }) + self.tryReply(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T ⇒ T] }) case Get ⇒ self reply agent.get case _ ⇒ () } @@ -299,7 +299,7 @@ class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor { def receive = { case update: Update[_] ⇒ try { - self.tryReply(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T => T] }) + self.tryReply(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T ⇒ T] }) } finally { agent.resume self.stop() diff --git a/akka-stm/src/main/scala/akka/transactor/Coordinated.scala b/akka-stm/src/main/scala/akka/transactor/Coordinated.scala index bd50926d72..f52732294d 100644 --- a/akka-stm/src/main/scala/akka/transactor/Coordinated.scala +++ b/akka-stm/src/main/scala/akka/transactor/Coordinated.scala @@ -6,19 +6,19 @@ package akka.transactor import akka.AkkaException import akka.config.Config -import akka.stm.{Atomic, DefaultTransactionConfig, TransactionFactory} +import akka.stm.{ Atomic, DefaultTransactionConfig, TransactionFactory } import org.multiverse.commitbarriers.CountDownCommitBarrier import org.multiverse.templates.TransactionalCallable import akka.actor.ActorTimeoutException -import org.multiverse.api.{TransactionConfiguration, Transaction ⇒ MultiverseTransaction} +import org.multiverse.api.{ TransactionConfiguration, Transaction ⇒ MultiverseTransaction } import org.multiverse.api.exceptions.ControlFlowError /** * 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 = null) extends AkkaException(message, cause) { + def this(msg: String) = this(msg, null); } /** @@ -96,9 +96,9 @@ object Coordinated { class Coordinated(val message: Any, barrier: CountDownCommitBarrier) { // Java API constructors - def this(message: Any) = this (message, Coordinated.createBarrier) + def this(message: Any) = this(message, Coordinated.createBarrier) - def this() = this (null, Coordinated.createBarrier) + def this() = this(null, Coordinated.createBarrier) /** * Create a new Coordinated object and increment the number of parties by one. @@ -147,8 +147,8 @@ class Coordinated(val message: Any, barrier: CountDownCommitBarrier) { val result = try { body } catch { - case e: ControlFlowError => throw e - case e: Exception => { + case e: ControlFlowError ⇒ throw e + case e: Exception ⇒ { barrier.abort() throw e } @@ -158,7 +158,7 @@ class Coordinated(val message: Any, barrier: CountDownCommitBarrier) { val success = try { barrier.tryJoinCommit(mtx, timeout.length, timeout.unit) } catch { - case e: IllegalStateException => { + case e: IllegalStateException ⇒ { val config: TransactionConfiguration = mtx.getConfiguration throw new CoordinatedTransactionException("Coordinated transaction [" + config.getFamilyName + "] aborted", e) } diff --git a/akka-stm/src/main/scala/akka/transactor/Transactor.scala b/akka-stm/src/main/scala/akka/transactor/Transactor.scala index 5f3303405e..a3572eaa2a 100644 --- a/akka-stm/src/main/scala/akka/transactor/Transactor.scala +++ b/akka-stm/src/main/scala/akka/transactor/Transactor.scala @@ -105,7 +105,7 @@ trait Transactor extends Actor { * Implement a general pattern for using coordinated transactions. */ final def receive = { - case coordinated@Coordinated(message) ⇒ { + case coordinated @ Coordinated(message) ⇒ { val others = (coordinate orElse alone)(message) for (sendTo ← others) { sendTo.actor ! coordinated(sendTo.message.getOrElse(message)) diff --git a/akka-stm/src/main/scala/akka/transactor/UntypedTransactor.scala b/akka-stm/src/main/scala/akka/transactor/UntypedTransactor.scala index 850e80e120..aac2e55908 100644 --- a/akka-stm/src/main/scala/akka/transactor/UntypedTransactor.scala +++ b/akka-stm/src/main/scala/akka/transactor/UntypedTransactor.scala @@ -28,7 +28,7 @@ abstract class UntypedTransactor extends UntypedActor { @throws(classOf[Exception]) final def onReceive(message: Any): Unit = { message match { - case coordinated@Coordinated(message) ⇒ { + case coordinated @ Coordinated(message) ⇒ { val others = coordinate(message) for (sendTo ← others) { sendTo.actor.sendOneWay(coordinated(sendTo.message.getOrElse(message))) diff --git a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala index 7d568d03a3..c7c370e8bf 100644 --- a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala +++ b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala @@ -22,7 +22,7 @@ object CoordinatedIncrement { } def receive = { - case coordinated@Coordinated(Increment(friends)) ⇒ { + case coordinated @ Coordinated(Increment(friends)) ⇒ { if (friends.nonEmpty) { friends.head ! coordinated(Increment(friends.tail)) } @@ -39,7 +39,7 @@ object CoordinatedIncrement { val txFactory = TransactionFactory(timeout = 3 seconds) def receive = { - case coordinated@Coordinated(Increment(friends)) ⇒ { + case coordinated @ Coordinated(Increment(friends)) ⇒ { coordinated.atomic(txFactory) { throw new RuntimeException("Expected failure") } diff --git a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala index f7b0277656..67fc0fbcb2 100644 --- a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala +++ b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala @@ -72,7 +72,7 @@ object FickleFriends { } def receive = { - case coordinated@Coordinated(Increment(friends)) ⇒ { + case coordinated @ Coordinated(Increment(friends)) ⇒ { val failAt = random(8) failIf(failAt, 0) if (friends.nonEmpty) { diff --git a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala index ff09cf88ff..c3e1de6b6b 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala @@ -4,12 +4,12 @@ package akka.tutorial.first.scala -import akka.actor.{Actor, PoisonPill} +import akka.actor.{ Actor, PoisonPill } import Actor._ -import akka.routing.{Routing, CyclicIterator} +import akka.routing.{ Routing, CyclicIterator } import Routing._ -import System.{currentTimeMillis => now} +import System.{ currentTimeMillis ⇒ now } import java.util.concurrent.CountDownLatch object Pi extends App { @@ -32,13 +32,13 @@ object Pi extends App { // define the work def calculatePiFor(start: Int, nrOfElements: Int): Double = { var acc = 0.0 - for (i <- start until (start + nrOfElements)) + for (i ← start until (start + nrOfElements)) acc += 4.0 * (1 - (i % 2) * 2) / (2 * i + 1) acc } def receive = { - case Work(start, nrOfElements) => + case Work(start, nrOfElements) ⇒ self reply Result(calculatePiFor(start, nrOfElements)) // perform the work } } @@ -61,9 +61,9 @@ object Pi extends App { // message handler def receive = { - case Calculate => + case Calculate ⇒ // schedule work - for (i <- 0 until nrOfMessages) router ! Work(i * nrOfElements, nrOfElements) + for (i ← 0 until nrOfMessages) router ! Work(i * nrOfElements, nrOfElements) // send a PoisonPill to all workers telling them to shut down themselves router ! Broadcast(PoisonPill) @@ -71,7 +71,7 @@ object Pi extends App { // send a PoisonPill to the router, telling him to shut himself down router ! PoisonPill - case Result(value) => + case Result(value) ⇒ // handle result from the worker pi += value nrOfResults += 1 @@ -86,7 +86,7 @@ object Pi extends App { // tell the world that the calculation is complete println( "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis" - .format(pi, (System.currentTimeMillis - start))) + .format(pi, (System.currentTimeMillis - start))) latch.countDown() } } diff --git a/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala index 46f59cdabf..994cd7e106 100644 --- a/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala @@ -5,13 +5,13 @@ package akka.tutorial.second import akka.actor.Actor._ -import akka.routing.{Routing, CyclicIterator} +import akka.routing.{ Routing, CyclicIterator } import Routing._ import akka.event.EventHandler -import akka.actor.{Channel, Actor, PoisonPill} +import akka.actor.{ Channel, Actor, PoisonPill } import akka.dispatch.Future -import System.{currentTimeMillis => now} +import System.{ currentTimeMillis ⇒ now } object Pi extends App { @@ -30,16 +30,16 @@ object Pi extends App { // ================== class Worker() extends Actor { // define the work - val calculatePiFor = (arg: Int, nrOfElements: Int) => { + val calculatePiFor = (arg: Int, nrOfElements: Int) ⇒ { val range = (arg * nrOfElements) to ((arg + 1) * nrOfElements - 1) var acc = 0.0D - range foreach (i => acc += 4 * math.pow(-1, i) / (2 * i + 1)) + range foreach (i ⇒ acc += 4 * math.pow(-1, i) / (2 * i + 1)) acc //range map (j => 4 * math.pow(-1, j) / (2 * j + 1)) sum } def receive = { - case Work(arg, nrOfElements) => + case Work(arg, nrOfElements) ⇒ self reply Result(calculatePiFor(arg, nrOfElements)) // perform the work } } @@ -59,9 +59,9 @@ object Pi extends App { // phase 1, can accept a Calculate message def scatter: Receive = { - case Calculate => + case Calculate ⇒ // schedule work - for (arg <- 0 until nrOfMessages) router ! Work(arg, nrOfElements) + for (arg ← 0 until nrOfMessages) router ! Work(arg, nrOfElements) //Assume the gathering behavior this become gather(self.channel) @@ -69,7 +69,7 @@ object Pi extends App { // phase 2, aggregate the results of the Calculation def gather(recipient: Channel[Any]): Receive = { - case Result(value) => + case Result(value) ⇒ // handle result from the worker pi += value nrOfResults += 1 @@ -105,10 +105,10 @@ object Pi extends App { //send calculate message master.?(Calculate, Actor.Timeout(60000)). - await.resultOrException match {//wait for the result, with a 60 seconds timeout - case Some(pi) => + await.resultOrException match { //wait for the result, with a 60 seconds timeout + case Some(pi) ⇒ EventHandler.info(this, "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis".format(pi, (now - start))) - case None => + case None ⇒ EventHandler.error(this, "Pi calculation did not complete within the timeout.") } } From 0b1d5c78b9b175adf8bc6d6cac9c301973e8cc6d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 26 Jul 2011 11:46:30 +0200 Subject: [PATCH 5/6] Cleaning up some code --- akka-actor/src/main/scala/akka/actor/Deployer.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 3e98716722..c0e9d09430 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -307,15 +307,12 @@ object LocalDeployer { object Address { private val validAddressPattern = java.util.regex.Pattern.compile("[0-9a-zA-Z\\-\\_\\$\\.]+") - def validate(address: String) { - if (validAddressPattern.matcher(address).matches) true - else { - val e = new IllegalArgumentException( - "Address [" + address + "] is not valid, need to follow pattern [0-9a-zA-Z\\-\\_\\$]+") + def validate(address: String): Unit = + if (!validAddressPattern.matcher(address).matches) { + val e = new IllegalArgumentException("Address [" + address + "] is not valid, need to follow pattern: " + validAddressPattern.pattern) EventHandler.error(e, this, e.getMessage) throw e } - } } class DeploymentException private[akka] (message: String) extends AkkaException(message) From 0351858c9e29f757004106c2fc1a1c70fcd749e4 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 26 Jul 2011 17:12:00 +0200 Subject: [PATCH 6/6] Lots of code cleanup and bugfixes of Deployment, still not AOT/JIT separation --- .history | 2 + .../scala/akka/actor/actor/DeployerSpec.scala | 1 + .../src/main/scala/akka/actor/Actor.scala | 44 +++--- .../main/scala/akka/actor/ActorRegistry.scala | 2 - .../src/main/scala/akka/actor/Deployer.scala | 143 ++++++++---------- .../scala/akka/actor/DeploymentConfig.scala | 12 +- .../scala/akka/util/ReflectiveAccess.scala | 26 +--- .../src/main/scala/akka/cluster/Cluster.scala | 2 +- .../scala/akka/cluster/ClusterActorRef.scala | 7 +- .../scala/akka/cluster/ClusterDeployer.scala | 31 ++-- .../scala/akka/cluster/ClusterTestNode.scala | 1 + ...LogWriteBehindNoSnapshotMultiJvmSpec.scala | 21 +-- ...ogWriteThroughNoSnapshotMultiJvmNode1.conf | 1 - ...ogWriteThroughNoSnapshotMultiJvmSpec.scala | 23 +-- 14 files changed, 123 insertions(+), 193 deletions(-) create mode 100644 .history diff --git a/.history b/.history new file mode 100644 index 0000000000..209db6b195 --- /dev/null +++ b/.history @@ -0,0 +1,2 @@ +update +reload diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala index 423098a49a..b2e41a02c7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala @@ -18,6 +18,7 @@ class DeployerSpec extends WordSpec with MustMatchers { deployment must equal(Some( Deploy( "service-ping", + None, LeastCPU, Clustered( Vector(Node("node1")), diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 6015272a74..6c4b71ead1 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -356,7 +356,7 @@ object Actor extends ListenerManagement { * JAVA API */ def actorOf[T <: Actor](creator: Creator[T]): ActorRef = - actorOf(creator, new UUID().toString) + actorOf(creator, newUuid().toString) /** * Creates an ActorRef out of the Actor. Allows you to pass in a factory (Creator) @@ -379,7 +379,7 @@ object Actor extends ListenerManagement { } def localActorOf[T <: Actor](clazz: Class[T]): ActorRef = { - newLocalActorRef(clazz, new UUID().toString) + newLocalActorRef(clazz, newUuid().toString) } def localActorOf[T <: Actor](clazz: Class[T], address: String): ActorRef = { @@ -387,7 +387,7 @@ object Actor extends ListenerManagement { } def localActorOf[T <: Actor](factory: ⇒ T): ActorRef = { - new LocalActorRef(() ⇒ factory, new UUID().toString) + new LocalActorRef(() ⇒ factory, newUuid().toString) } def localActorOf[T <: Actor](factory: ⇒ T, address: String): ActorRef = { @@ -432,8 +432,8 @@ object Actor extends ListenerManagement { case None ⇒ // it is not -> create it try { Deployer.deploymentFor(address) match { - case Deploy(_, router, Local) ⇒ actorFactory() // create a local actor - case deploy ⇒ newClusterActorRef(actorFactory, address, deploy) + case Deploy(_, _, router, Local) ⇒ actorFactory() // create a local actor + case deploy ⇒ newClusterActorRef(actorFactory, address, deploy) } } catch { case e: DeploymentException ⇒ @@ -463,32 +463,26 @@ object Actor extends ListenerManagement { }, address) } - private def newClusterActorRef(factory: () ⇒ ActorRef, address: String, deploy: Deploy): ActorRef = { + private[akka] def newClusterActorRef(factory: () ⇒ ActorRef, address: String, deploy: Deploy): ActorRef = deploy match { - case Deploy( - configAdress, router, - Clustered( - preferredHomeNodes, - replicas, - replication)) ⇒ + case Deploy(configAddress, recipe, router, Clustered(preferredHomeNodes, replicas, replication)) ⇒ ClusterModule.ensureEnabled() - if (configAdress != address) throw new IllegalStateException( - "Deployment config for [" + address + "] is wrong [" + deploy + "]") - if (!Actor.remote.isRunning) throw new IllegalStateException( - "Remote server is not running") + if (configAddress != address) throw new IllegalStateException("Deployment config for [" + address + "] is wrong [" + deploy + "]") + if (!remote.isRunning) throw new IllegalStateException("Remote server is not running") val isHomeNode = DeploymentConfig.isHomeNode(preferredHomeNodes) - val nrOfReplicas = replicas.factor - val serializer: Serializer = - Serialization.serializerFor(this.getClass) + val serializer = recipe match { + case Some(r) ⇒ Serialization.serializerFor(r.implementationClass) + case None ⇒ Serialization.serializerFor(classOf[Actor]) //TODO revisit this decision of default + } def storeActorAndGetClusterRef(replicationScheme: ReplicationScheme, serializer: Serializer): ActorRef = { // add actor to cluster registry (if not already added) - if (!cluster.isClustered(address)) - cluster.store(address, factory, nrOfReplicas, replicationScheme, false, serializer) + if (!cluster.isClustered(address)) //WARNING!!!! Racy + cluster.store(address, factory, replicas.factor, replicationScheme, false, serializer) // remote node (not home node), check out as ClusterActorRef cluster.ref(address, DeploymentConfig.routerTypeFor(router)) @@ -503,21 +497,17 @@ object Actor extends ListenerManagement { "Can't replicate an actor [" + address + "] configured with another router than \"direct\" - found [" + router + "]") if (isHomeNode) { // stateful actor's home node - cluster - .use(address, serializer) + cluster.use(address, serializer) .getOrElse(throw new ConfigurationException( "Could not check out actor [" + address + "] from cluster registry as a \"local\" actor")) - } else { storeActorAndGetClusterRef(replication, serializer) } } case invalid ⇒ throw new IllegalActorStateException( - "Could not create actor with address [" + address + - "], not bound to a valid deployment scheme [" + invalid + "]") + "Could not create actor with address [" + address + "], not bound to a valid deployment scheme [" + invalid + "]") } - } } /** diff --git a/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala b/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala index e4fe387fcf..29da2fac04 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala @@ -32,8 +32,6 @@ case class ActorUnregistered(address: String, actor: ActorRef, typedActor: Optio * @author Jonas Bonér */ private[actor] final class ActorRegistry private[actor] () extends ListenerManagement { - - //private val isClusterEnabled = ReflectiveAccess.isClusterEnabled private val actorsByAddress = new ConcurrentHashMap[String, ActorRef] private val actorsByUuid = new ConcurrentHashMap[Uuid, ActorRef] private val typedActorsByUuid = new ConcurrentHashMap[Uuid, AnyRef] diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index c0e9d09430..55f1ab23a1 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -10,67 +10,53 @@ import java.util.concurrent.ConcurrentHashMap import akka.event.EventHandler import akka.actor.DeploymentConfig._ -import akka.config.{ ConfigurationException, Config } import akka.util.ReflectiveAccess._ import akka.AkkaException +import akka.serialization.{ Serializer, Serialization } +import akka.util.ReflectiveAccess +import akka.config.{ Configuration, ConfigurationException, Config } + +trait ActorDeployer { + private[akka] def init(deployments: Seq[Deploy]): Unit + private[akka] def shutdown(): Unit //TODO Why should we have "shutdown", should be crash only? + private[akka] def deploy(deployment: Deploy): Unit + private[akka] def lookupDeploymentFor(address: String): Option[Deploy] + private[akka] def deploy(deployment: Seq[Deploy]): Unit = deployment foreach (deploy(_)) +} /** * Deployer maps actor deployments to actor addresses. * * @author Jonas Bonér */ -object Deployer { +object Deployer extends ActorDeployer { val defaultAddress = Host(Config.hostname) - lazy val instance: ClusterModule.ClusterDeployer = { - val deployer = - if (ClusterModule.isEnabled) ClusterModule.clusterDeployer - else LocalDeployer + lazy val instance: ActorDeployer = { + val deployer = if (ClusterModule.isEnabled) ClusterModule.clusterDeployer else LocalDeployer deployer.init(deploymentsInConfig) deployer } - def start() { - instance.toString - } + def start(): Unit = instance.toString //Force evaluation - def shutdown() { - instance.shutdown() - } + private[akka] def init(deployments: Seq[Deploy]) = instance.init(deployments) - def deploy(deployment: Deploy) { - if (deployment eq null) throw new IllegalArgumentException("Deploy can not be null") - val address = deployment.address - Address.validate(address) - instance.deploy(deployment) - } + def shutdown(): Unit = instance.shutdown() //TODO Why should we have "shutdown", should be crash only? - def deploy(deployment: Seq[Deploy]) { - deployment foreach (deploy(_)) - } - - /** - * Undeploy is idemponent. E.g. safe to invoke multiple times. - */ - def undeploy(deployment: Deploy) { - instance.undeploy(deployment) - } - - def undeployAll() { - instance.undeployAll() - } + def deploy(deployment: Deploy): Unit = instance.deploy(deployment) def isLocal(deployment: Deploy): Boolean = deployment match { - case Deploy(_, _, Local) ⇒ true - case _ ⇒ false + case Deploy(_, _, _, Local) | Deploy(_, _, _, _: Local) ⇒ true + case _ ⇒ false } - def isClustered(deployment: Deploy): Boolean = isLocal(deployment) + def isClustered(deployment: Deploy): Boolean = !isLocal(deployment) - def isLocal(address: String): Boolean = isLocal(deploymentFor(address)) + def isLocal(address: String): Boolean = isLocal(deploymentFor(address)) //TODO Should this throw exception if address not found? - def isClustered(address: String): Boolean = !isLocal(address) + def isClustered(address: String): Boolean = !isLocal(address) //TODO Should this throw exception if address not found? /** * Same as 'lookupDeploymentFor' but throws an exception if no deployment is bound. @@ -87,15 +73,13 @@ object Deployer { if (deployment_?.isDefined && (deployment_?.get ne null)) deployment_? else { - - val newDeployment = - try { - lookupInConfig(address) - } catch { - case e: ConfigurationException ⇒ - EventHandler.error(e, this, e.getMessage) - throw e - } + val newDeployment = try { + lookupInConfig(address) + } catch { + case e: ConfigurationException ⇒ + EventHandler.error(e, this, e.getMessage) + throw e + } newDeployment foreach { d ⇒ if (d eq null) { @@ -131,20 +115,21 @@ object Deployer { /** * Lookup deployment in 'akka.conf' configuration file. */ - private[akka] def lookupInConfig(address: String): Option[Deploy] = { + private[akka] def lookupInConfig(address: String, configuration: Configuration = Config.config): Option[Deploy] = { + import akka.util.ReflectiveAccess.{ createInstance, emptyArguments, emptyParams, getClassFor } // -------------------------------- // akka.actor.deployment.
    // -------------------------------- val addressPath = "akka.actor.deployment." + address - Config.config.getSection(addressPath) match { - case None ⇒ Some(Deploy(address, Direct, Local)) + configuration.getSection(addressPath) match { + case None ⇒ Some(Deploy(address, None, Direct, Local)) case Some(addressConfig) ⇒ // -------------------------------- // akka.actor.deployment.
    .router // -------------------------------- - val router = addressConfig.getString("router", "direct") match { + val router: Routing = addressConfig.getString("router", "direct") match { case "direct" ⇒ Direct case "round-robin" ⇒ RoundRobin case "random" ⇒ Random @@ -152,14 +137,21 @@ object Deployer { case "least-ram" ⇒ LeastRAM case "least-messages" ⇒ LeastMessages case customRouterClassName ⇒ - val customRouter = try { - Class.forName(customRouterClassName).newInstance.asInstanceOf[AnyRef] - } catch { - case e ⇒ throw new ConfigurationException( + createInstance[AnyRef](customRouterClassName, emptyParams, emptyArguments).fold( + e ⇒ throw new ConfigurationException( "Config option [" + addressPath + ".router] needs to be one of " + - "[\"direct\", \"round-robin\", \"random\", \"least-cpu\", \"least-ram\", \"least-messages\" or FQN of router class]") - } - CustomRouter(customRouter) + "[\"direct\", \"round-robin\", \"random\", \"least-cpu\", \"least-ram\", \"least-messages\" or FQN of router class]", e), + CustomRouter(_)) + } + + val recipe: Option[ActorRecipe] = addressConfig.getSection("create-as") map { section ⇒ + val implementationClass = section.getString("implementation-class") match { + case Some(impl) ⇒ + getClassFor[Actor](impl).fold(e ⇒ throw new ConfigurationException("Config option [" + addressPath + ".create-as.implementation-class] load failed", e), identity) + case None ⇒ throw new ConfigurationException("Config option [" + addressPath + ".create-as.implementation-class] is missing") + } + + ActorRecipe(implementationClass) } // -------------------------------- @@ -167,7 +159,7 @@ object Deployer { // -------------------------------- addressConfig.getSection("clustered") match { case None ⇒ - Some(Deploy(address, router, Local)) // deploy locally + Some(Deploy(address, recipe, router, Local)) // deploy locally case Some(clusteredConfig) ⇒ @@ -227,7 +219,7 @@ object Deployer { // -------------------------------- clusteredConfig.getSection("replication") match { case None ⇒ - Some(Deploy(address, router, Clustered(preferredNodes, replicationFactor, Transient))) + Some(Deploy(address, recipe, router, Clustered(preferredNodes, replicationFactor, Transient))) case Some(replicationConfig) ⇒ val storage = replicationConfig.getString("storage", "transaction-log") match { @@ -246,17 +238,14 @@ object Deployer { ".clustered.replication.strategy] needs to be either [\"write-through\"] or [\"write-behind\"] - was [" + unknown + "]") } - Some(Deploy(address, router, Clustered(preferredNodes, replicationFactor, Replication(storage, strategy)))) + Some(Deploy(address, recipe, router, Clustered(preferredNodes, replicationFactor, Replication(storage, strategy)))) } } } } private[akka] def throwDeploymentBoundException(deployment: Deploy): Nothing = { - val e = new DeploymentAlreadyBoundException( - "Address [" + deployment.address + - "] already bound to [" + deployment + - "]. You have to invoke 'undeploy(deployment) first.") + val e = new DeploymentAlreadyBoundException("Address [" + deployment.address + "] already bound to [" + deployment + "]") EventHandler.error(e, this, e.getMessage) throw e } @@ -273,29 +262,30 @@ object Deployer { * * @author Jonas Bonér */ -object LocalDeployer { +object LocalDeployer extends ActorDeployer { private val deployments = new ConcurrentHashMap[String, Deploy] - private[akka] def init(deployments: List[Deploy]) { + private[akka] def init(deployments: Seq[Deploy]) { EventHandler.info(this, "Deploying actors locally [\n\t%s\n]" format deployments.mkString("\n\t")) deployments foreach (deploy(_)) // deploy } private[akka] def shutdown() { - undeployAll() - deployments.clear() + deployments.clear() //TODO do something else/more? } private[akka] def deploy(deployment: Deploy) { - if (deployments.putIfAbsent(deployment.address, deployment) != deployment) { - //Deployer.throwDeploymentBoundException(deployment) // FIXME uncomment this and fix the issue with multiple deployments - } + deployments.putIfAbsent(deployment.address, deployment) /* match { + case null ⇒ + deployment match { + case Deploy(address, Some(recipe), routing, _) ⇒ Actor.actorOf(recipe.implementationClass, address).start() //FIXME use routing? + case _ ⇒ + } + case `deployment` ⇒ //Already deployed TODO should it be like this? + case preexists ⇒ Deployer.throwDeploymentBoundException(deployment) + }*/ } - private[akka] def undeploy(deployment: Deploy): Unit = deployments.remove(deployment.address) - - private[akka] def undeployAll(): Unit = deployments.clear() - private[akka] def lookupDeploymentFor(address: String): Option[Deploy] = Option(deployments.get(address)) } @@ -307,12 +297,13 @@ object LocalDeployer { object Address { private val validAddressPattern = java.util.regex.Pattern.compile("[0-9a-zA-Z\\-\\_\\$\\.]+") - def validate(address: String): Unit = + def validate(address: String) { if (!validAddressPattern.matcher(address).matches) { val e = new IllegalArgumentException("Address [" + address + "] is not valid, need to follow pattern: " + validAddressPattern.pattern) EventHandler.error(e, this, e.getMessage) throw e } + } } class DeploymentException private[akka] (message: String) extends AkkaException(message) diff --git a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala index 43d895d9a5..c7dd94bbdb 100644 --- a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala +++ b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala @@ -21,8 +21,16 @@ object DeploymentConfig { // -------------------------------- case class Deploy( address: String, + recipe: Option[ActorRecipe], routing: Routing = Direct, - scope: Scope = Local) + scope: Scope = Local) { + Address.validate(address) + } + + // -------------------------------- + // --- Actor Recipe + // -------------------------------- + case class ActorRecipe(implementationClass: Class[_ <: Actor]) //TODO Add ActorConfiguration here // -------------------------------- // --- Routing @@ -158,7 +166,7 @@ object DeploymentConfig { } def replicationSchemeFor(deployment: Deploy): Option[ReplicationScheme] = deployment match { - case Deploy(_, _, Clustered(_, _, replicationScheme)) ⇒ Some(replicationScheme) + case Deploy(_, _, _, Clustered(_, _, replicationScheme)) ⇒ Some(replicationScheme) case _ ⇒ None } diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala index 3f0f33f01c..9f1311a349 100644 --- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala @@ -50,14 +50,7 @@ object ReflectiveAccess { None } - lazy val clusterDeployerInstance: Option[ClusterDeployer] = getObjectFor("akka.cluster.ClusterDeployer$") match { - case Right(value) ⇒ Some(value) - case Left(exception) ⇒ - EventHandler.debug(this, exception.toString) - None - } - - lazy val serializerClass: Option[Class[_]] = getClassFor("akka.serialization.Serializer") match { + lazy val clusterDeployerInstance: Option[ActorDeployer] = getObjectFor("akka.cluster.ClusterDeployer$") match { case Right(value) ⇒ Some(value) case Left(exception) ⇒ EventHandler.debug(this, exception.toString) @@ -76,7 +69,7 @@ object ReflectiveAccess { clusterInstance.get.node } - lazy val clusterDeployer: ClusterDeployer = { + lazy val clusterDeployer: ActorDeployer = { ensureEnabled() clusterDeployerInstance.get } @@ -86,15 +79,6 @@ object ReflectiveAccess { transactionLogInstance.get } - type ClusterDeployer = { - def init(deployments: List[Deploy]) - def shutdown() - def deploy(deployment: Deploy) - def undeploy(deployment: Deploy) - def undeployAll() - def lookupDeploymentFor(address: String): Option[Deploy] - } - type Cluster = { def node: ClusterNode } @@ -104,12 +88,6 @@ object ReflectiveAccess { def dequeue: MessageInvocation } - // FIXME: remove? - type Serializer = { - def toBinary(obj: AnyRef): Array[Byte] - def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef - } - type TransactionLogObject = { def newLogFor( id: String, diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 1d3a984ff1..0e7157fa5e 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -1306,7 +1306,7 @@ class DefaultClusterNode private[akka] ( if (actorAddress.isDefined) { // use 'preferred-nodes' in deployment config for the actor Deployer.deploymentFor(actorAddress.get) match { - case Deploy(_, _, Clustered(nodes, _, _)) ⇒ + case Deploy(_, _, _, Clustered(nodes, _, _)) ⇒ nodes map (node ⇒ DeploymentConfig.nodeNameFor(node)) take replicationFactor case _ ⇒ throw new ClusterException("Actor [" + actorAddress.get + "] is not configured as clustered") diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala index 9e136e50c4..cdfad82f7c 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala @@ -114,18 +114,21 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine } def start(): this.type = synchronized[this.type] { - _status = ActorRefInternals.RUNNING + if (_status == ActorRefInternals.UNSTARTED) { + _status = ActorRefInternals.RUNNING + //TODO add this? Actor.registry.register(this) + } this } def stop() { synchronized { if (_status == ActorRefInternals.RUNNING) { + //TODO add this? Actor.registry.unregister(this) _status = ActorRefInternals.SHUTDOWN postMessageToMailbox(RemoteActorSystemMessage.Stop, None) // FIXME here we need to fire off Actor.cluster.remove(address) (which needs to be properly implemented first, see ticket) - inetSocketAddressToActorRefMap.get.values foreach (_.stop()) // shut down all remote connections } } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterDeployer.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterDeployer.scala index a0a5fa40f2..63229b4770 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterDeployer.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterDeployer.scala @@ -4,8 +4,8 @@ package akka.cluster -import akka.actor.{ DeploymentConfig, Deployer, LocalDeployer, DeploymentException } -import DeploymentConfig._ +import akka.actor.DeploymentConfig._ +import akka.actor._ import akka.event.EventHandler import akka.config.Config import akka.util.Switch @@ -17,12 +17,10 @@ import org.apache.zookeeper.recipes.lock.{ WriteLock, LockListener } import org.I0Itec.zkclient.exception.{ ZkNoNodeException, ZkNodeExistsException } +import scala.collection.immutable.Seq import scala.collection.JavaConversions.collectionAsScalaIterable -import com.eaio.uuid.UUID - import java.util.concurrent.{ CountDownLatch, TimeUnit } -import java.util.concurrent.atomic.AtomicReference /** * A ClusterDeployer is responsible for deploying a Deploy. @@ -31,7 +29,7 @@ import java.util.concurrent.atomic.AtomicReference * * @author Jonas Bonér */ -object ClusterDeployer { +object ClusterDeployer extends ActorDeployer { val clusterName = Cluster.name val nodeName = Config.nodename val clusterPath = "/%s" format clusterName @@ -127,7 +125,7 @@ object ClusterDeployer { deployments } - private[akka] def init(deployments: List[Deploy]) { + private[akka] def init(deployments: Seq[Deploy]) { isConnected switchOn { EventHandler.info(this, "Initializing cluster deployer") @@ -143,7 +141,7 @@ object ClusterDeployer { } } - val allDeployments = deployments ::: systemDeployments + val allDeployments = deployments ++ systemDeployments if (!isDeploymentCompletedInCluster) { if (deploymentInProgressLock.lock()) { @@ -167,21 +165,20 @@ object ClusterDeployer { ensureRunning { LocalDeployer.deploy(deployment) deployment match { - case Deploy(_, _, Local) ⇒ {} // local deployment, do nothing here - case _ ⇒ // cluster deployment - val path = deploymentAddressPath.format(deployment.address) + case Deploy(_, _, _, Local) | Deploy(_, _, _, _: Local) ⇒ //TODO LocalDeployer.deploy(deployment)?? + case Deploy(address, recipe, routing, _) ⇒ // cluster deployment + /*TODO recipe foreach { r ⇒ + Deployer.newClusterActorRef(() ⇒ Actor.actorOf(r.implementationClass), address, deployment).start() + }*/ + val path = deploymentAddressPath.format(address) try { ignore[ZkNodeExistsException](zkClient.create(path, null, CreateMode.PERSISTENT)) zkClient.writeData(path, deployment) } catch { case e: NullPointerException ⇒ - handleError(new DeploymentException( - "Could not store deployment data [" + deployment + - "] in ZooKeeper since client session is closed")) + handleError(new DeploymentException("Could not store deployment data [" + deployment + "] in ZooKeeper since client session is closed")) case e: Exception ⇒ - handleError(new DeploymentException( - "Could not store deployment data [" + - deployment + "] in ZooKeeper due to: " + e)) + handleError(new DeploymentException("Could not store deployment data [" + deployment + "] in ZooKeeper due to: " + e)) } } } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala index 6393502377..bdc430ee6d 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala @@ -13,6 +13,7 @@ import akka.util.Duration import System.{ currentTimeMillis ⇒ now } import java.io.File +import akka.actor.Deployer trait MasterClusterTestNode extends WordSpec with MustMatchers with BeforeAndAfterAll { def testNodes: Int diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala index 23aaaec5a4..c40a06b404 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala @@ -44,26 +44,7 @@ class ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode1 extends Cluste val actorRef = Actor.actorOf[HelloWorld]("hello-world").start() node.isInUseOnNode("hello-world") must be(true) actorRef.address must be("hello-world") - var counter = 0 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") + for (i ← 0 until 10) (actorRef ? Count(i)).as[String] must be(Some("World from node [node1]")) } barrier("start-node2", NrOfNodes) { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1.conf index 42e57847b5..cc2fb1ef3b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1.conf @@ -2,7 +2,6 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" akka.actor.deployment.hello-world.router = "direct" akka.actor.deployment.hello-world.clustered.replication-factor = 1 - akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log" akka.actor.deployment.hello-world.clustered.replication.strategy = "write-through" akka.cluster.replication.snapshot-frequency = 1000 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala index 2d95ae6047..97fbb1c79b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala @@ -3,7 +3,6 @@ */ package akka.cluster.replication.transactionlog.writethrough.nosnapshot - import akka.actor._ import akka.cluster._ import Cluster._ @@ -44,26 +43,8 @@ class ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1 extends Clust val actorRef = Actor.actorOf[HelloWorld]("hello-world").start() node.isInUseOnNode("hello-world") must be(true) actorRef.address must be("hello-world") - var counter = 0 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") - counter += 1 - (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") + for (i ← 0 until 10) + (actorRef ? Count(i)).as[String] must be(Some("World from node [node1]")) } barrier("start-node2", NrOfNodes) {