From 705c118ea279a6f886c270c917d3c60638927cef Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sat, 8 Sep 2012 20:54:16 +0200 Subject: [PATCH 01/46] Pick correct node at the ring boundary and change to TreeMap, see #944 * The small problem was that it picked node counter clockwise in the ring for all, except for crc < firstKey, where it picked firstKey instead of lastKey * Changed to clockwise selection and correct selection at the boundary, i.e. hash > lastKey => firstKey and hash < firstKey => firstKey * Changed to TreeMap instead of the 2 separate TreeSet and Map collections * Cleanup and documentation --- .../scala/akka/routing/ConsistentHash.scala | 137 +++++++++++------- 1 file changed, 84 insertions(+), 53 deletions(-) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index afa321d07d..ae14612e99 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -4,59 +4,7 @@ package akka.routing -import scala.collection.immutable.{ TreeSet, Seq } -import scala.collection.mutable.{ Buffer, Map } - -// ============================================================================================= -// Adapted from HashRing.scala in Debasish Ghosh's Redis Client, licensed under Apache 2 license -// ============================================================================================= - -/** - * Consistent Hashing node ring abstraction. - * - * Not thread-safe, to be used from within an Actor or protected some other way. - */ -class ConsistentHash[T](nodes: Seq[T], replicas: Int) { - private val cluster = Buffer[T]() - private var sortedKeys = TreeSet[Long]() - private var ring = Map[Long, T]() - - nodes.foreach(this += _) - - def +=(node: T): Unit = { - cluster += node - (1 to replicas) foreach { replica ⇒ - val key = hashFor((node + ":" + replica).getBytes("UTF-8")) - ring += (key -> node) - sortedKeys = sortedKeys + key - } - } - - def -=(node: T): Unit = { - cluster -= node - (1 to replicas) foreach { replica ⇒ - val key = hashFor((node + ":" + replica).getBytes("UTF-8")) - ring -= key - sortedKeys = sortedKeys - key - } - } - - def nodeFor(key: Array[Byte]): T = { - val hash = hashFor(key) - if (sortedKeys contains hash) ring(hash) - else { - if (hash < sortedKeys.firstKey) ring(sortedKeys.firstKey) - else if (hash > sortedKeys.lastKey) ring(sortedKeys.lastKey) - else ring(sortedKeys.rangeImpl(None, Some(hash)).lastKey) - } - } - - private def hashFor(bytes: Array[Byte]): Long = { - val hash = MurmurHash.arrayHash(bytes) - if (hash == Int.MinValue) hash + 1 - math.abs(hash) - } -} +import scala.collection.immutable.TreeMap /* __ *\ ** ________ ___ / / ___ Scala API ** @@ -80,6 +28,89 @@ class ConsistentHash[T](nodes: Seq[T], replicas: Int) { import java.lang.Integer.{ rotateLeft ⇒ rotl } +// ============================================================================================= +// Adapted from HashRing.scala in Debasish Ghosh's Redis Client, licensed under Apache 2 license +// ============================================================================================= + +/** + * Consistent Hashing node ring abstraction. + * + * A good explanation of Consistent Hashing: + * http://weblogs.java.net/blog/tomwhite/archive/2007/11/consistent_hash.html + * + * Note that toString of the ring nodes are used for the node + * hash, i.e. make sure it is different for different nodes. + * + * Not thread-safe, to be used from within an Actor or protected some other way. + */ +class ConsistentHash[T](nodes: Iterable[T], replicas: Int) { + private var ring = TreeMap[Int, T]() + + if (replicas < 1) throw new IllegalArgumentException("replicas must be >= 1") + + nodes.foreach(this += _) + + /** + * Adds a node to the ring. + */ + def +=(node: T): Unit = { + (1 to replicas) foreach { replica ⇒ + ring += (nodeHashFor(node, replica) -> node) + } + } + + /** + * Adds a node to the ring. + * JAVA API + */ + def add(node: T): Unit = this += node + + /** + * Removes a node from the ring. + */ + def -=(node: T): Unit = { + (1 to replicas) foreach { replica ⇒ + ring -= nodeHashFor(node, replica) + } + } + + /** + * Removes a node from the ring. + * JAVA API + */ + def remove(node: T): Unit = this -= node + + /** + * Get the node responsible for the data key. + * Can only be used if nodes exists in the ring, + * otherwise throws `IllegalStateException` + */ + def nodeFor(key: Array[Byte]): T = { + if (isEmpty) throw new IllegalStateException("Can't get node for [%s] from an empty ring" format key) + val hash = hashFor(key) + def nextClockwise: T = { + val (ringKey, node) = ring.rangeImpl(Some(hash), None).headOption.getOrElse(ring.head) + node + } + ring.getOrElse(hash, nextClockwise) + } + + /** + * Is the ring empty, i.e. no nodes added or all removed. + */ + def isEmpty: Boolean = ring.isEmpty + + private def nodeHashFor(node: T, replica: Int): Int = { + hashFor((node + ":" + replica).getBytes("UTF-8")) + } + + private def hashFor(bytes: Array[Byte]): Int = { + val hash = MurmurHash.arrayHash(bytes) + if (hash == Int.MinValue) hash + 1 + math.abs(hash) + } +} + /** * A class designed to generate well-distributed non-cryptographic * hashes. It is designed to be passed to a collection's foreach method, From 521d20ba73561c7682bafd31e652a0bce5602d9d Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sun, 9 Sep 2012 16:47:43 +0200 Subject: [PATCH 02/46] Make ConsistentHash immutable, see #944 * Makes it thread safe * Also changed two scary (mutable Array) fields in MurmurHash to private * Note in migration guide about changed api for ConsistentHash --- .../scala/akka/routing/ConsistentHash.scala | 58 ++++++++++++------- .../project/migration-guide-2.0.x-2.1.x.rst | 18 +++++- 2 files changed, 55 insertions(+), 21 deletions(-) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index ae14612e99..21f9bffbce 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -5,6 +5,7 @@ package akka.routing import scala.collection.immutable.TreeMap +import java.lang.Integer.{ rotateLeft ⇒ rotl } /* __ *\ ** ________ ___ / / ___ Scala API ** @@ -26,8 +27,6 @@ import scala.collection.immutable.TreeMap * @since 2.9 */ -import java.lang.Integer.{ rotateLeft ⇒ rotl } - // ============================================================================================= // Adapted from HashRing.scala in Debasish Ghosh's Redis Client, licensed under Apache 2 license // ============================================================================================= @@ -41,44 +40,46 @@ import java.lang.Integer.{ rotateLeft ⇒ rotl } * Note that toString of the ring nodes are used for the node * hash, i.e. make sure it is different for different nodes. * - * Not thread-safe, to be used from within an Actor or protected some other way. */ -class ConsistentHash[T](nodes: Iterable[T], replicas: Int) { - private var ring = TreeMap[Int, T]() +class ConsistentHash[T] private (replicas: Int, ring: TreeMap[Int, T]) { + + import ConsistentHash._ if (replicas < 1) throw new IllegalArgumentException("replicas must be >= 1") - nodes.foreach(this += _) - /** * Adds a node to the ring. + * Note that the instance is immutable and this + * operation returns a new instance. */ - def +=(node: T): Unit = { - (1 to replicas) foreach { replica ⇒ - ring += (nodeHashFor(node, replica) -> node) - } + def :+(node: T): ConsistentHash[T] = { + new ConsistentHash(replicas, ring ++ ((1 to replicas) map { r ⇒ (nodeHashFor(node, r) -> node) })) } /** * Adds a node to the ring. + * Note that the instance is immutable and this + * operation returns a new instance. * JAVA API */ - def add(node: T): Unit = this += node + def add(node: T): ConsistentHash[T] = this :+ node /** * Removes a node from the ring. + * Note that the instance is immutable and this + * operation returns a new instance. */ - def -=(node: T): Unit = { - (1 to replicas) foreach { replica ⇒ - ring -= nodeHashFor(node, replica) - } + def :-(node: T): ConsistentHash[T] = { + new ConsistentHash(replicas, ring -- ((1 to replicas) map { r ⇒ nodeHashFor(node, r) })) } /** * Removes a node from the ring. + * Note that the instance is immutable and this + * operation returns a new instance. * JAVA API */ - def remove(node: T): Unit = this -= node + def remove(node: T): ConsistentHash[T] = this :- node /** * Get the node responsible for the data key. @@ -100,7 +101,24 @@ class ConsistentHash[T](nodes: Iterable[T], replicas: Int) { */ def isEmpty: Boolean = ring.isEmpty - private def nodeHashFor(node: T, replica: Int): Int = { +} + +object ConsistentHash { + def apply[T](nodes: Iterable[T], replicas: Int) = { + new ConsistentHash(replicas, TreeMap.empty[Int, T] ++ + (for (node ← nodes; replica ← 1 to replicas) yield (nodeHashFor(node, replica) -> node))) + } + + /** + * Factory method to create a ConsistentHash + * JAVA API + */ + def create[T](nodes: java.lang.Iterable[T], replicas: Int) = { + import scala.collection.JavaConverters._ + apply(nodes.asScala, replicas) + } + + private def nodeHashFor(node: Any, replica: Int): Int = { hashFor((node + ":" + replica).getBytes("UTF-8")) } @@ -189,11 +207,11 @@ object MurmurHash { final private val seedArray: Int = 0x3c074a61 /** The first 23 magic integers from the first stream are stored here */ - val storedMagicA: Array[Int] = + private val storedMagicA: Array[Int] = Iterator.iterate(hiddenMagicA)(nextMagicA).take(23).toArray /** The first 23 magic integers from the second stream are stored here */ - val storedMagicB: Array[Int] = + private val storedMagicB: Array[Int] = Iterator.iterate(hiddenMagicB)(nextMagicB).take(23).toArray /** Begin a new hash with a seed value. */ diff --git a/akka-docs/project/migration-guide-2.0.x-2.1.x.rst b/akka-docs/project/migration-guide-2.0.x-2.1.x.rst index fd5629ea3b..2a9f925f7f 100644 --- a/akka-docs/project/migration-guide-2.0.x-2.1.x.rst +++ b/akka-docs/project/migration-guide-2.0.x-2.1.x.rst @@ -286,7 +286,6 @@ Both Actors and UntypedActors using ``Stash`` now overrides postStop to make sur stashed messages are put into the dead letters when the actor stops, make sure you call super.postStop if you override it. - Forward of Terminated message ============================= @@ -359,3 +358,20 @@ v2.1:: -requestedCapacity, stopDelay) +ConsistentHash +============== + +``akka.routing.ConsistentHash`` has been changed to an immutable data structure. + +v2.0:: + + val consistentHash = new ConsistentHash(Seq(a1, a2, a3), replicas = 10) + consistentHash += a4 + val a = consistentHash.nodeFor(data) + +v2.1:: + + var consistentHash = ConsistentHash(Seq(a1, a2, a3), replicas = 10) + consistentHash = consistentHash :+ a4 + val a = consistentHash.nodeFor(data) + From f6dcee423bb3e0bc9e79e88748c83d7d9e195858 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 11 Sep 2012 15:10:16 +0200 Subject: [PATCH 03/46] Implement ConsistentHashingRouter, see #944 * Added trait RouterEnvelope to handle Broadcast and ConsistentHashableEnvelope in same way, could also be useful for custom routers --- .../test/scala/akka/actor/DeployerSpec.scala | 7 + .../routing/ConsistentHashingRouterSpec.scala | 79 +++++++ .../src/main/scala/akka/actor/Deployer.scala | 13 +- .../routing/ConsistentHashingRouter.scala | 212 ++++++++++++++++++ .../src/main/scala/akka/routing/Routing.scala | 14 +- 5 files changed, 316 insertions(+), 9 deletions(-) create mode 100644 akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala create mode 100644 akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala index 1aef438627..37aa133583 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala @@ -35,6 +35,9 @@ object DeployerSpec { router = scatter-gather within = 2 seconds } + /service-consistent-hashing { + router = consistent-hashing + } /service-resizer { router = round-robin resizer { @@ -118,6 +121,10 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) { assertRouting("/service-scatter-gather", ScatterGatherFirstCompletedRouter(nrOfInstances = 1, within = 2 seconds), "/service-scatter-gather") } + "be able to parse 'akka.actor.deployment._' with consistent-hashing router" in { + assertRouting("/service-consistent-hashing", ConsistentHashingRouter(1), "/service-consistent-hashing") + } + "be able to parse 'akka.actor.deployment._' with router resizer" in { val resizer = DefaultResizer() assertRouting("/service-resizer", RoundRobinRouter(resizer = Some(resizer)), "/service-resizer") diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala new file mode 100644 index 0000000000..fd7a49e867 --- /dev/null +++ b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala @@ -0,0 +1,79 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.routing + +import scala.concurrent.Await + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Props +import akka.actor.actorRef2Scala +import akka.pattern.ask +import akka.routing.ConsistentHashingRouter.ConsistentHashable +import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope +import akka.testkit.AkkaSpec +import akka.testkit._ + +object ConsistentHashingRouterSpec { + + val config = """ + akka.actor.deployment { + /router1 { + router = consistent-hashing + nr-of-instances = 3 + } + } + """ + + class Echo extends Actor { + def receive = { + case _ ⇒ sender ! self + } + } + + case class Msg(key: Any, data: String) extends ConsistentHashable { + override def consistentHashKey = key + } + + case class MsgKey(name: String) +} + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.config) with DefaultTimeout with ImplicitSender { + import akka.routing.ConsistentHashingRouterSpec._ + implicit val ec = system.dispatcher + + val router1 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter()), "router1") + + "consistent hashing router" must { + "create routees from configuration" in { + val currentRoutees = Await.result(router1 ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees] + currentRoutees.routees.size must be(3) + } + + "select destination based on consistentHashKey of the message" in { + router1 ! Msg("a", "A") + val destinationA = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + router1 ! new ConsistentHashableEnvelope { + override def consistentHashKey = "a" + override def message = "AA" + } + expectMsg(destinationA) + + router1 ! Msg(17, "B") + val destinationB = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + router1 ! new ConsistentHashableEnvelope { + override def consistentHashKey = 17 + override def message = "BB" + } + expectMsg(destinationB) + + router1 ! Msg(MsgKey("c"), "C") + val destinationC = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + router1 ! Msg(MsgKey("c"), "CC") + expectMsg(destinationC) + } + } + +} diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 9ceebaacb3..be9da1a505 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -148,12 +148,13 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce val resizer: Option[Resizer] = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None val router: RouterConfig = deployment.getString("router") match { - case "from-code" ⇒ NoRouter - case "round-robin" ⇒ RoundRobinRouter(nrOfInstances, routees, resizer) - case "random" ⇒ RandomRouter(nrOfInstances, routees, resizer) - case "smallest-mailbox" ⇒ SmallestMailboxRouter(nrOfInstances, routees, resizer) - case "scatter-gather" ⇒ ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer) - case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer) + case "from-code" ⇒ NoRouter + case "round-robin" ⇒ RoundRobinRouter(nrOfInstances, routees, resizer) + case "random" ⇒ RandomRouter(nrOfInstances, routees, resizer) + case "smallest-mailbox" ⇒ SmallestMailboxRouter(nrOfInstances, routees, resizer) + case "scatter-gather" ⇒ ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer) + case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer) + case "consistent-hashing" ⇒ ConsistentHashingRouter(nrOfInstances, routees, resizer) case fqn ⇒ val args = Seq(classOf[Config] -> deployment) dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({ diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala new file mode 100644 index 0000000000..eba812ac60 --- /dev/null +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -0,0 +1,212 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.routing + +import scala.collection.JavaConversions.iterableAsScalaIterable +import scala.util.control.NonFatal + +import akka.actor.ActorRef +import akka.actor.SupervisorStrategy +import akka.actor.Props +import akka.dispatch.Dispatchers +import akka.event.Logging +import akka.serialization.SerializationExtension + +object ConsistentHashingRouter { + /** + * Creates a new ConsistentHashingRouter, routing to the specified routees + */ + def apply(routees: Iterable[ActorRef]): ConsistentHashingRouter = + new ConsistentHashingRouter(routees = routees map (_.path.toString)) + + /** + * Java API to create router with the supplied 'routees' actors. + */ + def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = { + import scala.collection.JavaConverters._ + apply(routees.asScala) + } + + /** + * Messages need to implement this interface to define what + * data to use for the consistent hash key. Note that it's not + * the hash, but the data to be hashed. If returning an + * `Array[Byte]` or String it will be used as is, otherwise the + * configured [[akka.akka.serialization.Serializer]] will + * be applied to the returned data. + * + * If messages can't implement this interface themselves, + * it's possible to wrap the messages in + * [[akka.routing.ConsistentHashableEnvelope]] + */ + trait ConsistentHashable { + def consistentHashKey: Any + } + + /** + * If messages can't implement [[akka.routing.ConsistentHashable]] + * themselves they can we wrapped by something implementing + * this interface instead. The router will only send the + * wrapped message to the destination, i.e. the envelope will + * be stripped off. + */ + trait ConsistentHashableEnvelope extends ConsistentHashable with RouterEnvelope { + def message: Any + } + + /** + * Default number of replicas (virtual nodes) used in [[akka.routing.ConsistantHash]] + */ + val DefaultReplicas: Int = 10 + +} +/** + * A Router that uses consistent hashing to select a connection based on the + * sent message. The messages must implement [[akka.routing.ConsistentHashable]] + * or be wrapped in a [[akka.routing.ConsistentHashableEnvelope]] to define what + * data to use for the consistent hash key. + * + * Please note that providing both 'nrOfInstances' and 'routees' does not make logical + * sense as this means that the router should both create new actors and use the 'routees' + * actor(s). In this case the 'nrOfInstances' will be ignored and the 'routees' will be used. + *
+ * The configuration parameter trumps the constructor arguments. This means that + * if you provide either 'nrOfInstances' or 'routees' during instantiation they will + * be ignored if the router is defined in the configuration file for the actor being used. + * + *

Supervision Setup

+ * + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to + * different groups of children. + * + * @param routees string representation of the actor paths of the routees that will be looked up + * using `actorFor` in [[akka.actor.ActorRefProvider]] + * @param replicas number of replicas (virtual nodes) used in [[akka.routing.ConsistantHash]] + */ +@SerialVersionUID(1L) +case class ConsistentHashingRouter( + nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, + val routerDispatcher: String = Dispatchers.DefaultDispatcherId, + val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy, + val replicas: Int = ConsistentHashingRouter.DefaultReplicas) + extends RouterConfig with ConsistentHashingLike { + + /** + * Constructor that sets nrOfInstances to be created. + * Java API + */ + def this(nr: Int) = this(nrOfInstances = nr) + + /** + * Constructor that sets the routees to be used. + * Java API + * @param routeePaths string representation of the actor paths of the routees that will be looked up + * using `actorFor` in [[akka.actor.ActorRefProvider]] + */ + def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) + + /** + * Constructor that sets the resizer to be used. + * Java API + */ + def this(resizer: Resizer) = this(resizer = Some(resizer)) + + /** + * Java API for setting routerDispatcher + */ + def withDispatcher(dispatcherId: String): ConsistentHashingRouter = copy(routerDispatcher = dispatcherId) + + /** + * Java API for setting the supervisor strategy to be used for the “head” + * Router actor. + */ + def withSupervisorStrategy(strategy: SupervisorStrategy): ConsistentHashingRouter = copy(supervisorStrategy = strategy) + + /** + * Java API for setting the number of replicas (virtual nodes) used in [[akka.routing.ConsistantHash]] + */ + def withReplicas(replicas: Int): ConsistentHashingRouter = copy(replicas = replicas) + + /** + * Uses the resizer of the given RouterConfig if this RouterConfig + * doesn't have one, i.e. the resizer defined in code is used if + * resizer was not defined in config. + */ + override def withFallback(other: RouterConfig): RouterConfig = { + if (this.resizer.isEmpty && other.resizer.isDefined) copy(resizer = other.resizer) + else this + } +} + +trait ConsistentHashingLike { this: RouterConfig ⇒ + + import ConsistentHashingRouter._ + + def nrOfInstances: Int + + def routees: Iterable[String] + + def replicas: Int + + override def createRoute(routeeProvider: RouteeProvider): Route = { + if (resizer.isEmpty) { + if (routees.isEmpty) routeeProvider.createRoutees(nrOfInstances) + else routeeProvider.registerRouteesFor(routees) + } + + val log = Logging(routeeProvider.context.system, routeeProvider.context.self) + + // consistentHashRoutees and consistentHash are updated together, synchronized on the consistentHashLock + val consistentHashLock = new Object + var consistentHashRoutees: IndexedSeq[ActorRef] = null + var consistentHash: ConsistentHash[ActorRef] = null + upateConsistentHash() + + // update consistentHash when routees has changed + // changes to routees are rare and when no changes this is a quick operation + def upateConsistentHash(): ConsistentHash[ActorRef] = consistentHashLock.synchronized { + val currentRoutees = routeeProvider.routees + if ((currentRoutees ne consistentHashRoutees) && currentRoutees != consistentHashRoutees) { + consistentHashRoutees = currentRoutees + consistentHash = ConsistentHash(currentRoutees, replicas) + } + consistentHash + } + + def target(hashData: Any): ActorRef = try { + val hash = hashData match { + case bytes: Array[Byte] ⇒ bytes + case str: String ⇒ str.getBytes("UTF-8") + case x: AnyRef ⇒ SerializationExtension(routeeProvider.context.system).serialize(x).get + } + val currentConsistenHash = upateConsistentHash() + if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters + else currentConsistenHash.nodeFor(hash) + } catch { + case NonFatal(e) ⇒ + // serialization failed + log.warning("Couldn't route message with consistentHashKey [%s] due to [%s]".format(hashData, e.getMessage)) + routeeProvider.context.system.deadLetters + } + + { + case (sender, message) ⇒ + message match { + case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees) + case hashable: ConsistentHashable ⇒ List(Destination(sender, target(hashable.consistentHashKey))) + case other ⇒ + log.warning("Message [{}] must implement [{}] or be wrapped in [{}]", + message.getClass.getName, classOf[ConsistentHashable].getName, + classOf[ConsistentHashableEnvelope].getName) + List(Destination(sender, routeeProvider.context.system.deadLetters)) + } + } + } +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 1d18e7ed2e..9e17bf2b9a 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -115,8 +115,8 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo val s = if (sender eq null) system.deadLetters else sender val msg = message match { - case Broadcast(m) ⇒ m - case m ⇒ m + case wrapped: RouterEnvelope ⇒ wrapped.message + case m ⇒ m } applyRoute(s, message) match { @@ -400,7 +400,15 @@ private object Router { * Router implementations may choose to handle this message differently. */ @SerialVersionUID(1L) -case class Broadcast(message: Any) +case class Broadcast(message: Any) extends RouterEnvelope + +/** + * Only the contained message will be forwarded to the + * destination, i.e. the envelope will be stripped off. + */ +trait RouterEnvelope { + def message: Any +} /** * Sending this message to a router will make it send back its currently used routees. From ea78e6d8052b1fbcd58452d79c1313c3c4195d1b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 11 Sep 2012 15:10:33 +0200 Subject: [PATCH 04/46] Documentation of constent hashing router, see #944 * Document sample to illustrate usage * Rewrote RouterViaConfig to real DocSpec * Added configuration snippets for all router types --- .../test/scala/akka/routing/RoutingSpec.scala | 2 - akka-docs/java/routing.rst | 50 +++++- .../ConsistentHashingRouterDocSpec.scala | 60 +++++++ .../docs/routing/RouterViaConfigDocSpec.scala | 157 ++++++++++++++++++ .../docs/routing/RouterViaConfigExample.scala | 52 ------ akka-docs/scala/routing.rst | 63 ++++++- 6 files changed, 323 insertions(+), 61 deletions(-) create mode 100644 akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala create mode 100644 akka-docs/scala/code/docs/routing/RouterViaConfigDocSpec.scala delete mode 100644 akka-docs/scala/code/docs/routing/RouterViaConfigExample.scala 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 9e126e75ed..2d4f2245f1 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -171,8 +171,6 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with } expectMsgType[ActorKilledException] - //#supervision - val router2 = system.actorOf(Props.empty.withRouter(RoundRobinRouter(1).withSupervisorStrategy(escalator))) router2 ! CurrentRoutees EventFilter[ActorKilledException](occurrences = 2) intercept { diff --git a/akka-docs/java/routing.rst b/akka-docs/java/routing.rst index 845e2825e3..561cbd9446 100644 --- a/akka-docs/java/routing.rst +++ b/akka-docs/java/routing.rst @@ -21,7 +21,7 @@ Routers In Action This is an example of how to create a router that is defined in configuration: -.. includecode:: ../scala/code/docs/routing/RouterViaConfigExample.scala#config +.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-round-robin .. includecode:: code/docs/jrouting/RouterViaConfigExample.java#configurableRouting @@ -177,6 +177,10 @@ is exactly what you would expect from a round-robin router to happen. (The name of an actor is automatically created in the format ``$letter`` unless you specify it - hence the names printed above.) +This is an example of how to define a round-robin router in configuration: + +.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-round-robin + RandomRouter ************ As the name implies this router type selects one of its routees randomly and forwards @@ -204,6 +208,10 @@ When run you should see a similar output to this: The result from running the random router should be different, or at least random, every time you run it. Try to run it a couple of times to verify its behavior if you don't trust us. +This is an example of how to define a random router in configuration: + +.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-random + SmallestMailboxRouter ********************* A Router that tries to send to the non-suspended routee with fewest messages in mailbox. @@ -219,6 +227,10 @@ Code example: .. includecode:: code/docs/jrouting/ParentActor.java#smallestMailboxRouter +This is an example of how to define a smallest-mailbox router in configuration: + +.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-smallest-mailbox + BroadcastRouter *************** A broadcast router forwards the message it receives to *all* its routees. @@ -238,6 +250,10 @@ When run you should see a similar output to this: As you can see here above each of the routees, five in total, received the broadcast message. +This is an example of how to define a broadcast router in configuration: + +.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-broadcast + ScatterGatherFirstCompletedRouter ********************************* The ScatterGatherFirstCompletedRouter will send the message on to all its routees as a future. @@ -255,6 +271,36 @@ When run you should see this: From the output above you can't really see that all the routees performed the calculation, but they did! The result you see is from the first routee that returned its calculation to the router. +This is an example of how to define a scatter-gather router in configuration: + +.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-scatter-gather + +ConsistentHashingRouter +*********************** + +The ConsistentHashingRouter uses `consistent hashing `_ +to select a connection based on the sent message. This +`article `_ gives good +insight into how consistent hashing is implemented. + +The messages sent to a ConsistentHashingRouter must implement +``akka.routing.ConsistentHashable`` or be wrapped in a ``akka.routing.ConsistentHashableEnvelope`` +to define what data to use for the consistent hash key. If returning a +byte array or String it will be used as is, otherwise the configured +:ref:`serializer ` will be applied to the returned data +to create a byte array that will be hashed. + +Code example: + +FIXME Java example of consistent routing + +In the above example you see that the ``Get`` message implements ``ConsistentHashable`` itself, +while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. + +This is an example of how to define a consistent-hashing router in configuration: + +.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-consistent-hashing + Broadcast Messages ^^^^^^^^^^^^^^^^^^ @@ -276,7 +322,7 @@ of routees dynamically. This is an example of how to create a resizable router that is defined in configuration: -.. includecode:: ../scala/code/docs/routing/RouterViaConfigExample.scala#config-resize +.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-resize .. includecode:: code/docs/jrouting/RouterViaConfigExample.java#configurableRoutingWithResizer diff --git a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala new file mode 100644 index 0000000000..8bb59a7bce --- /dev/null +++ b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala @@ -0,0 +1,60 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package docs.routing + +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender + +object ConsistentHashingRouterDocSpec { + + //#cache-actor + import akka.actor.Actor + import akka.routing.ConsistentHashingRouter.ConsistentHashable + import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope + + class Cache extends Actor { + var cache = Map.empty[String, String] + + def receive = { + case Entry(key, value) ⇒ cache += (key -> value) + case Get(key) ⇒ sender ! cache.get(key) + } + } + + case class Get(key: String) extends ConsistentHashable { + override def consistentHashKey: Any = key + } + + case class Entry(key: String, value: String) + case class EntryEnvelope(entry: Entry) extends ConsistentHashableEnvelope { + override def consistentHashKey: Any = entry.key + override def message: Any = entry + } + //#cache-actor + +} + +class ConsistentHashingRouterDocSpec extends AkkaSpec with ImplicitSender { + + import ConsistentHashingRouterDocSpec._ + + "demonstrate usage of ConsistentHashableRouter" in { + + //#consistent-hashing-router + import akka.actor.Props + import akka.routing.ConsistentHashingRouter + + val cache = system.actorOf(Props[Cache].withRouter(ConsistentHashingRouter(10)), "cache") + + cache ! EntryEnvelope(Entry("hello", "HELLO")) + cache ! EntryEnvelope(Entry("hi", "HI")) + + cache ! Get("hello") + cache ! Get("hi") + expectMsg(Some("HELLO")) + expectMsg(Some("HI")) + //#consistent-hashing-router + } + +} diff --git a/akka-docs/scala/code/docs/routing/RouterViaConfigDocSpec.scala b/akka-docs/scala/code/docs/routing/RouterViaConfigDocSpec.scala new file mode 100644 index 0000000000..265b0f8361 --- /dev/null +++ b/akka-docs/scala/code/docs/routing/RouterViaConfigDocSpec.scala @@ -0,0 +1,157 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package docs.routing + +import akka.actor.{ Actor, Props, ActorSystem, ActorLogging } +import com.typesafe.config.ConfigFactory +import akka.routing.FromConfig +import akka.routing.ConsistentHashingRouter.ConsistentHashable +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender + +object RouterWithConfigDocSpec { + + val config = ConfigFactory.parseString(""" + + //#config-round-robin + akka.actor.deployment { + /myrouter1 { + router = round-robin + nr-of-instances = 5 + } + } + //#config-round-robin + + //#config-resize + akka.actor.deployment { + /myrouter2 { + router = round-robin + resizer { + lower-bound = 2 + upper-bound = 15 + } + } + } + //#config-resize + + //#config-random + akka.actor.deployment { + /myrouter3 { + router = random + nr-of-instances = 5 + } + } + //#config-random + + //#config-smallest-mailbox + akka.actor.deployment { + /myrouter4 { + router = smallest-mailbox + nr-of-instances = 5 + } + } + //#config-smallest-mailbox + + //#config-broadcast + akka.actor.deployment { + /myrouter5 { + router = broadcast + nr-of-instances = 5 + } + } + //#config-broadcast + + //#config-scatter-gather + akka.actor.deployment { + /myrouter6 { + router = scatter-gather + nr-of-instances = 5 + within = 10 seconds + } + } + //#config-scatter-gather + + //#config-consistent-hashing + akka.actor.deployment { + /myrouter7 { + router = consistent-hashing + nr-of-instances = 5 + } + } + //#config-consistent-hashing + + """) + + case class Message(nbr: Int) extends ConsistentHashable { + override def consistentHashKey = nbr + } + + class ExampleActor extends Actor with ActorLogging { + def receive = { + case Message(nbr) ⇒ + log.debug("Received %s in router %s".format(nbr, self.path.name)) + sender ! nbr + } + } + +} + +class RouterWithConfigDocSpec extends AkkaSpec(RouterWithConfigDocSpec.config) with ImplicitSender { + + import RouterWithConfigDocSpec._ + + "demonstrate configured round-robin router" in { + //#configurableRouting + val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()), + "myrouter1") + //#configurableRouting + 1 to 10 foreach { i ⇒ router ! Message(i) } + receiveN(10) + } + + "demonstrate configured random router" in { + val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()), + "myrouter3") + 1 to 10 foreach { i ⇒ router ! Message(i) } + receiveN(10) + } + + "demonstrate configured smallest-mailbox router" in { + val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()), + "myrouter4") + 1 to 10 foreach { i ⇒ router ! Message(i) } + receiveN(10) + } + + "demonstrate configured broadcast router" in { + val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()), + "myrouter5") + 1 to 10 foreach { i ⇒ router ! Message(i) } + receiveN(5 * 10) + } + + "demonstrate configured scatter-gather router" in { + val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()), + "myrouter6") + 1 to 10 foreach { i ⇒ router ! Message(i) } + receiveN(10) + } + + "demonstrate configured consistent-hashing router" in { + val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()), + "myrouter7") + 1 to 10 foreach { i ⇒ router ! Message(i) } + receiveN(10) + } + + "demonstrate configured round-robin router with resizer" in { + //#configurableRoutingWithResizer + val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()), + "myrouter2") + //#configurableRoutingWithResizer + 1 to 10 foreach { i ⇒ router ! Message(i) } + receiveN(10) + } + +} \ No newline at end of file diff --git a/akka-docs/scala/code/docs/routing/RouterViaConfigExample.scala b/akka-docs/scala/code/docs/routing/RouterViaConfigExample.scala deleted file mode 100644 index 5d34e429bb..0000000000 --- a/akka-docs/scala/code/docs/routing/RouterViaConfigExample.scala +++ /dev/null @@ -1,52 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ -package docs.routing - -import akka.actor.{ Actor, Props, ActorSystem } -import com.typesafe.config.ConfigFactory -import akka.routing.FromConfig - -case class Message(nbr: Int) - -class ExampleActor extends Actor { - def receive = { - case Message(nbr) ⇒ println("Received %s in router %s".format(nbr, self.path.name)) - } -} - -object RouterWithConfigExample extends App { - val config = ConfigFactory.parseString(""" - //#config - akka.actor.deployment { - /router { - router = round-robin - nr-of-instances = 5 - } - } - //#config - //#config-resize - akka.actor.deployment { - /router2 { - router = round-robin - resizer { - lower-bound = 2 - upper-bound = 15 - } - } - } - //#config-resize - """) - val system = ActorSystem("Example", config) - //#configurableRouting - val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()), - "router") - //#configurableRouting - 1 to 10 foreach { i ⇒ router ! Message(i) } - - //#configurableRoutingWithResizer - val router2 = system.actorOf(Props[ExampleActor].withRouter(FromConfig()), - "router2") - //#configurableRoutingWithResizer - 1 to 10 foreach { i ⇒ router2 ! Message(i) } -} \ No newline at end of file diff --git a/akka-docs/scala/routing.rst b/akka-docs/scala/routing.rst index 353e82d277..3f384895c6 100644 --- a/akka-docs/scala/routing.rst +++ b/akka-docs/scala/routing.rst @@ -15,15 +15,16 @@ is really easy to create your own. The routers shipped with Akka are: * ``akka.routing.SmallestMailboxRouter`` * ``akka.routing.BroadcastRouter`` * ``akka.routing.ScatterGatherFirstCompletedRouter`` +* ``akka.routing.ConsistentHashingRouter`` Routers In Action ^^^^^^^^^^^^^^^^^ This is an example of how to create a router that is defined in configuration: -.. includecode:: code/docs/routing/RouterViaConfigExample.scala#config +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-round-robin -.. includecode:: code/docs/routing/RouterViaConfigExample.scala#configurableRouting +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#configurableRouting This is an example of how to programmatically create a router and set the number of routees it should create: @@ -125,7 +126,7 @@ not have an effect on the number of actors in the pool. Setting the strategy is easily done: -.. includecode:: ../../akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +.. includecode:: ../../akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala#supervision :include: supervision :exclude: custom-strategy @@ -179,6 +180,10 @@ is exactly what you would expect from a round-robin router to happen. (The name of an actor is automatically created in the format ``$letter`` unless you specify it - hence the names printed above.) +This is an example of how to define a round-robin router in configuration: + +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-round-robin + RandomRouter ************ As the name implies this router type selects one of its routees randomly and forwards @@ -206,6 +211,10 @@ When run you should see a similar output to this: The result from running the random router should be different, or at least random, every time you run it. Try to run it a couple of times to verify its behavior if you don't trust us. +This is an example of how to define a random router in configuration: + +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-random + SmallestMailboxRouter ********************* A Router that tries to send to the non-suspended routee with fewest messages in mailbox. @@ -221,6 +230,11 @@ Code example: .. includecode:: code/docs/routing/RouterTypeExample.scala#smallestMailboxRouter + +This is an example of how to define a smallest-mailbox router in configuration: + +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-smallest-mailbox + BroadcastRouter *************** A broadcast router forwards the message it receives to *all* its routees. @@ -240,6 +254,11 @@ When run you should see a similar output to this: As you can see here above each of the routees, five in total, received the broadcast message. +This is an example of how to define a broadcast router in configuration: + +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-broadcast + + ScatterGatherFirstCompletedRouter ********************************* The ScatterGatherFirstCompletedRouter will send the message on to all its routees as a future. @@ -257,6 +276,40 @@ When run you should see this: From the output above you can't really see that all the routees performed the calculation, but they did! The result you see is from the first routee that returned its calculation to the router. +This is an example of how to define a scatter-gather router in configuration: + +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-scatter-gather + + +ConsistentHashingRouter +*********************** + +The ConsistentHashingRouter uses `consistent hashing `_ +to select a connection based on the sent message. This +`article `_ gives good +insight into how consistent hashing is implemented. + +The messages sent to a ConsistentHashingRouter must implement +``akka.routing.ConsistentHashable`` or be wrapped in a ``akka.routing.ConsistentHashableEnvelope`` +to define what data to use for the consistent hash key. If returning a +byte array or String it will be used as is, otherwise the configured +:ref:`serializer ` will be applied to the returned data +to create a byte array that will be hashed. + +Code example: + +.. includecode:: code/docs/routing/ConsistentHashingRouterDocSpec.scala#cache-actor + +.. includecode:: code/docs/routing/ConsistentHashingRouterDocSpec.scala#consistent-hashing-router + +In the above example you see that the ``Get`` message implements ``ConsistentHashable`` itself, +while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. + +This is an example of how to define a consistent-hashing router in configuration: + +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-consistent-hashing + + Broadcast Messages ^^^^^^^^^^^^^^^^^^ @@ -278,9 +331,9 @@ of routees dynamically. This is an example of how to create a resizable router that is defined in configuration: -.. includecode:: code/docs/routing/RouterViaConfigExample.scala#config-resize +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-resize -.. includecode:: code/docs/routing/RouterViaConfigExample.scala#configurableRoutingWithResizer +.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#configurableRoutingWithResizer Several more configuration options are available and described in ``akka.actor.deployment.default.resizer`` section of the reference :ref:`configuration`. From adeb4fc8b868b00483dccd701e6d3c13a4739b52 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 13 Sep 2012 10:54:00 +0200 Subject: [PATCH 05/46] Handle empty availableNodes, see #2103 --- .../scala/akka/cluster/routing/ClusterRouterConfig.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala index 89622867af..e2d8b8b204 100644 --- a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala +++ b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala @@ -186,11 +186,12 @@ private[akka] class ClusterRouteeProvider( private def selectDeploymentTarget: Option[Address] = { val currentRoutees = routees - if (currentRoutees.size >= settings.totalInstances) { + val currentNodes = availbleNodes + if (currentRoutees.size >= settings.totalInstances || currentNodes.isEmpty) { None } else { val numberOfRouteesPerNode: Map[Address, Int] = - Map.empty[Address, Int] ++ availbleNodes.toSeq.map(_ -> 0) ++ + Map.empty[Address, Int] ++ currentNodes.toSeq.map(_ -> 0) ++ currentRoutees.groupBy(fullAddress).map { case (address, refs) ⇒ address -> refs.size } From 0524ba0a65d422f560c34530e5c6fccc11d228ab Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 13 Sep 2012 10:54:14 +0200 Subject: [PATCH 06/46] Cluster documentation * 2 Sample applications with main programs to play with, and multi-node tests to illustrate testing, and code snippets for inclusion in rst docs * TransformationSample illustratates subscription to cluster events * StatsSample illustrates usage of cluster aware routers, both lookup and deploy --- .../src/main/resources/reference.conf | 20 +- akka-docs/cluster/cluster-usage.rst | 220 ++++++++++++++- .../src/main/resources/application.conf | 5 +- .../SimpleClusterApp.scala} | 4 +- .../sample/cluster/stats/StatsSample.scala | 254 ++++++++++++++++++ .../transformation/TransformationSample.scala | 108 ++++++++ .../stats/StatsSampleSingleMasterSpec.scala | 101 +++++++ .../cluster/stats/StatsSampleSpec.scala | 92 +++++++ .../TransformationSampleSpec.scala | 115 ++++++++ project/AkkaBuild.scala | 16 +- 10 files changed, 913 insertions(+), 22 deletions(-) rename akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/{ClusterApp.scala => simple/SimpleClusterApp.scala} (94%) create mode 100644 akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala create mode 100644 akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala create mode 100644 akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala create mode 100644 akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala create mode 100644 akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala diff --git a/akka-cluster/src/main/resources/reference.conf b/akka-cluster/src/main/resources/reference.conf index 60e69c1984..141cbcbaec 100644 --- a/akka-cluster/src/main/resources/reference.conf +++ b/akka-cluster/src/main/resources/reference.conf @@ -9,7 +9,8 @@ akka { cluster { # Initial contact points of the cluster. Nodes to join at startup if auto-join = on. - # Comma separated full URIs defined by a string on the form of "akka://system@hostname:port" + # Comma separated full URIs defined by a string on the form of + # "akka://system@hostname:port" # Leave as empty if the node should be a singleton cluster. seed-nodes = [] @@ -20,9 +21,10 @@ akka { # If seed-nodes is empty it will join itself and become a single node cluster. auto-join = on - # Should the 'leader' in the cluster be allowed to automatically mark unreachable nodes as DOWN? - # Using auto-down implies that two separate clusters will automatically be formed in case of - # network partition. + # Should the 'leader' in the cluster be allowed to automatically mark unreachable + # nodes as DOWN? + # Using auto-down implies that two separate clusters will automatically be formed + # in case of network partition. auto-down = off # how long should the node wait before starting the periodic tasks maintenance tasks? @@ -37,18 +39,20 @@ akka { # how often should the leader perform maintenance tasks? leader-actions-interval = 1s - # how often should the node move nodes, marked as unreachable by the failure detector, out of the membership ring? + # how often should the node move nodes, marked as unreachable by the failure detector, + # out of the membership ring? unreachable-nodes-reaper-interval = 1s # How often the current internal stats should be published. # A value of 0 s can be used to always publish the stats, when it happens. publish-stats-interval = 10s - # A joining node stops sending heartbeats to the node to join if it hasn't become member - # of the cluster within this deadline. + # A joining node stops sending heartbeats to the node to join if it hasn't + # become member of the cluster within this deadline. join-timeout = 60s - # The id of the dispatcher to use for cluster actors. If not specified default dispatcher is used. + # The id of the dispatcher to use for cluster actors. If not specified + # default dispatcher is used. # If specified you need to define the settings of the actual dispatcher. use-dispatcher = "" diff --git a/akka-docs/cluster/cluster-usage.rst b/akka-docs/cluster/cluster-usage.rst index 9d922aea97..9e079796a7 100644 --- a/akka-docs/cluster/cluster-usage.rst +++ b/akka-docs/cluster/cluster-usage.rst @@ -18,7 +18,8 @@ The Akka cluster is a separate jar file. Make sure that you have the following d "com.typesafe.akka" %% "akka-cluster" % "2.1-SNAPSHOT" -If you are using the latest nightly build you should pick a timestamped Akka version from ``_. +If you are using the latest nightly build you should pick a timestamped Akka version from ``_. Don't use ``SNAPSHOT``. Note that the Scala version |scalaVersion| +is part of the artifactId. A Simple Cluster Example ^^^^^^^^^^^^^^^^^^^^^^^^ @@ -35,7 +36,9 @@ Try it out: :language: none To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-scala` -settings and the ``akka.cluster.seed-nodes`` to your ``application.conf`` file. +settings, but with ``ClusterActorRefProvider``. +The ``akka.cluster.seed-nodes`` and cluster extension should normally also be added to your +``application.conf`` file. The seed nodes are configured contact points for initial, automatic, join of the cluster. @@ -44,20 +47,20 @@ ip-addresses or host names of the machines in ``application.conf`` instead of `` 2. Add the following main program to your project, place it in ``src/main/scala``: -.. literalinclude:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/ClusterApp.scala +.. literalinclude:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala :language: scala 3. Start the first seed node. Open a sbt session in one terminal window and run:: - run-main sample.cluster.ClusterApp 2551 + run-main sample.cluster.simple.SimpleClusterApp 2551 2551 corresponds to the port of the first seed-nodes element in the configuration. In the log output you see that the cluster node has been started and changed status to 'Up'. 4. Start the second seed node. Open a sbt session in another terminal window and run:: - run-main sample.cluster.ClusterApp 2552 + run-main sample.cluster.simple.SimpleClusterApp 2552 2552 corresponds to the port of the second seed-nodes element in the configuration. @@ -68,7 +71,7 @@ Switch over to the first terminal window and see in the log output that the memb 5. Start another node. Open a sbt session in yet another terminal window and run:: - run-main sample.cluster.ClusterApp + run-main sample.cluster.simple.SimpleClusterApp Now you don't need to specify the port number, and it will use a random available port. It joins one of the configured seed nodes. Look at the log output in the different terminal @@ -82,7 +85,7 @@ output in the other terminals. Look at the source code of the program again. What it does is to create an actor and register it as subscriber of certain cluster events. It gets notified with -an snapshot event, 'CurrentClusterState' that holds full state information of +an snapshot event, ``CurrentClusterState`` that holds full state information of the cluster. After that it receives events for changes that happen in the cluster. Automatic vs. Manual Joining @@ -137,6 +140,206 @@ Be aware of that using auto-down implies that two separate clusters will automatically be formed in case of network partition. That might be desired by some applications but not by others. +Subscribe to Cluster Events +^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +You can subscribe to change notifications of the cluster membership by using +``Cluster(system).subscribe``. A snapshot of the full state, +``akka.cluster.ClusterEvent.CurrentClusterState``, is sent to the subscriber +as the first event, followed by events for incremental updates. + +There are several types of change events, consult the API documentation +of classes that extends ``akka.cluster.ClusterEvent.ClusterDomainEvent`` +for details about the events. + +Worker Dial-in Example +---------------------- + +Let's take a look at an example that illustrates how workers, here named *backend*, +can detect and register to new master nodes, here named *frontend*. + +The example application provides a service to transform text. When some text +is sent to one of the frontend services, it will be delegated to one of the +backend workers, which performs the transformation job, and sends the result back to +the original client. New backend nodes, as well as new frontend nodes, can be +added or removed to the cluster dynamically. + +In this example the following imports are used: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala#imports + +Messages: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala#messages + +The backend worker that performs the transformation job: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala#backend + +Note that the ``TransformationBackend`` actor subscribes to cluster events to detect new, +potential, frontend nodes, and send them a registration message so that they know +that they can use the backend worker. + +The frontend that receives user jobs and delegates to one of the registered backend workers: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala#frontend + +Note that the ``TransformationFrontend`` actor watch the registered backend +to be able to remove it from its list of availble backend workers. +Death watch uses the cluster failure detector for nodes in the cluster, i.e. it detects +network failures and JVM crashes, in addition to graceful termination of watched +actor. + +This example is included in ``akka-samples/akka-sample-cluster`` +and you can try by starting nodes in different terminal windows. For example, starting 2 +frontend nodes and 3 backend nodes:: + + sbt + + project akka-sample-cluster-experimental + + run-main sample.cluster.transformation.TransformationFrontend 2551 + + run-main sample.cluster.transformation.TransformationBackend 2552 + + run-main sample.cluster.transformation.TransformationBackend + + run-main sample.cluster.transformation.TransformationBackend + + run-main sample.cluster.transformation.TransformationFrontend + + +.. note:: The above example should probably be designed as two separate, frontend/backend, clusters, when there is a `cluster client for decoupling clusters `_. + +Cluster Aware Routers +^^^^^^^^^^^^^^^^^^^^^ + +All :ref:`routers ` can be made aware of member nodes in the cluster, i.e. +deploying new routees or looking up routees on nodes in the cluster. +When a node becomes unavailble or leaves the cluster the routees of that node are +automatically unregistered from the router. When new nodes join the cluster additional +routees are added to the router, according to the configuration. + +When using a router with routees looked up on the cluster member nodes, i.e. the routees +are already running, the configuration for a router looks like this: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala#router-lookup-config + +It's the relative actor path defined in ``routees-path`` that identify what actor to lookup. + +``nr-of-instances`` defines total number of routees in the cluster, but there will not be +more than one per node. Setting ``nr-of-instances`` to a high value will result in new routees +added to the router when nodes join the cluster. + +The same type of router could also have been defined in code: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala#router-lookup-in-code + +When using a router with routees created and deployed on the cluster member nodes +the configuration for a router looks like this: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala#router-deploy-config + + +``nr-of-instances`` defines total number of routees in the cluster, but the number of routees +per node, ``max-nr-of-instances-per-node``, will not be exceeded. Setting ``nr-of-instances`` +to a high value will result in creating and deploying additional routees when new nodes join +the cluster. + +The same type of router could also have been defined in code: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala#router-deploy-in-code + +See :ref:`cluster_configuration` section for further descriptions of the settings. + + +Router Example +-------------- + +Let's take a look at how to use cluster aware routers. + +The example application provides a service to calculate statistics for a text. +When some text is sent to the service it splits it into words, and delegates the task +to count number of characters in each word to a separate worker, a routee of a router. +The character count for each word is sent back to an aggregator that calculates +the average number of characters per word when all results have been collected. + +In this example we use the following imports: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala#imports + +Messages: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala#messages + +The worker that counts number of characters in each word: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala#worker + +The service that receives text from users and splits it up into words, delegates to workers and aggregates: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala#service + + +Note, nothing cluster specific so far, just plain actors. + +We can use these actors with two different types of router setup. Either with lookup of routees, +or with create and deploy of routees. Remember, routees are the workers in this case. + +We start with the router setup with lookup of routees. All nodes start ``StatsService`` and +``StatsWorker`` actors and the router is configured with ``routees-path``: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala#start-router-lookup + +This means that user requests can be sent to ``StatsService`` on any node and it will use +``StatsWorker`` on all nodes. There can only be one worker per node, but that worker could easily +fan out to local children if more parallelism is needed. + +This example is included in ``akka-samples/akka-sample-cluster`` +and you can try by starting nodes in different terminal windows. For example, starting 3 +service nodes and 1 client:: + + run-main sample.cluster.stats.StatsSample 2551 + + run-main sample.cluster.stats.StatsSample 2552 + + run-main sample.cluster.stats.StatsSampleClient + + run-main sample.cluster.stats.StatsSample + +The above setup is nice for this example, but we will also take a look at how to use +a single master node that creates and deploys workers. To keep track of a single +master we need one additional actor: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala#facade + +The ``StatsFacade`` receives text from users and delegates to the current ``StatsService``, the single +master. It listens to cluster events to create or lookup the ``StatsService`` depending on if +it is on the same same node or on another node. We run the master on the same node as the leader of +the cluster members, which is nothing more than the address currently sorted first in the member ring, +i.e. it can change when new nodes join or when current leader leaves. + +All nodes start ``StatsFacade`` and the router is now configured like this: + +.. includecode:: ../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala#start-router-deploy + + +This example is included in ``akka-samples/akka-sample-cluster`` +and you can try by starting nodes in different terminal windows. For example, starting 3 +service nodes and 1 client:: + + run-main sample.cluster.stats.StatsSampleOneMaster 2551 + + run-main sample.cluster.stats.StatsSampleOneMaster 2552 + + run-main sample.cluster.stats.StatsSampleOneMasterClient + + run-main sample.cluster.stats.StatsSampleOneMaster + +.. note:: The above example, especially the last part, will be simplified when the cluster handles automatic actor partitioning. + +.. _cluster_configuration: + Configuration ^^^^^^^^^^^^^ @@ -147,6 +350,9 @@ reference file for more information: .. literalinclude:: ../../akka-cluster/src/main/resources/reference.conf :language: none +Cluster Scheduler +----------------- + It is recommended that you change the ``tick-duration`` to 33 ms or less of the default scheduler when using cluster, if you don't need to have it configured to a longer duration for other reasons. If you don't do this diff --git a/akka-samples/akka-sample-cluster/src/main/resources/application.conf b/akka-samples/akka-sample-cluster/src/main/resources/application.conf index ee403ff23d..dc0f09445c 100644 --- a/akka-samples/akka-sample-cluster/src/main/resources/application.conf +++ b/akka-samples/akka-sample-cluster/src/main/resources/application.conf @@ -1,9 +1,10 @@ akka { actor { - provider = "akka.remote.RemoteActorRefProvider" + provider = "akka.cluster.ClusterActorRefProvider" } remote { transport = "akka.remote.netty.NettyRemoteTransport" + log-remote-lifecycle-events = off netty { hostname = "127.0.0.1" port = 0 @@ -16,5 +17,7 @@ akka { seed-nodes = [ "akka://ClusterSystem@127.0.0.1:2551", "akka://ClusterSystem@127.0.0.1:2552"] + + auto-down = on } } \ No newline at end of file diff --git a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/ClusterApp.scala b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala similarity index 94% rename from akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/ClusterApp.scala rename to akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala index 0fd396784d..4f69700835 100644 --- a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/ClusterApp.scala +++ b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/simple/SimpleClusterApp.scala @@ -1,10 +1,10 @@ -package sample.cluster +package sample.cluster.simple import akka.actor._ import akka.cluster.Cluster import akka.cluster.ClusterEvent._ -object ClusterApp { +object SimpleClusterApp { def main(args: Array[String]): Unit = { diff --git a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala new file mode 100644 index 0000000000..1f13d7fa7a --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala @@ -0,0 +1,254 @@ +package sample.cluster.stats + +//#imports +import language.postfixOps +import scala.concurrent.forkjoin.ThreadLocalRandom +import scala.concurrent.util.duration._ +import com.typesafe.config.ConfigFactory +import akka.actor.Actor +import akka.actor.ActorLogging +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.actor.Address +import akka.actor.Props +import akka.actor.ReceiveTimeout +import akka.actor.RelativeActorPath +import akka.actor.RootActorPath +import akka.cluster.Cluster +import akka.cluster.ClusterEvent.CurrentClusterState +import akka.cluster.ClusterEvent.LeaderChanged +import akka.cluster.ClusterEvent.MemberEvent +import akka.cluster.ClusterEvent.MemberUp +import akka.cluster.MemberStatus +import akka.routing.FromConfig +//#imports + +//#messages +case class StatsJob(text: String) +case class StatsResult(meanWordLength: Double) +case class JobFailed(reason: String) +//#messages + +//#service +class StatsService extends Actor { + val workerRouter = context.actorOf(Props[StatsWorker].withRouter(FromConfig), + name = "workerRouter") + + def receive = { + case StatsJob(text) if text != "" ⇒ + val words = text.split(" ") + val replyTo = sender // important to not close over sender + val aggregator = context.actorOf(Props(new StatsAggregator(words.size, replyTo))) + words foreach { word ⇒ workerRouter.tell(word, aggregator) } + } +} + +class StatsAggregator(expectedResults: Int, replyTo: ActorRef) extends Actor { + var results = IndexedSeq.empty[Int] + context.setReceiveTimeout(10 seconds) + + def receive = { + case wordCount: Int ⇒ + results = results :+ wordCount + if (results.size == expectedResults) { + val meanWordLength = results.sum.toDouble / results.size + replyTo ! StatsResult(meanWordLength) + context.stop(self) + } + case ReceiveTimeout ⇒ + replyTo ! JobFailed("Service unavailable, try again later") + context.stop(self) + } +} +//#service + +//#worker +class StatsWorker extends Actor { + // FIXME add a cache here to illustrate consistent hashing + def receive = { + case word: String ⇒ sender ! word.length + } +} +//#worker + +//#facade +class StatsFacade extends Actor with ActorLogging { + val cluster = Cluster(context.system) + + var currentMaster: Option[ActorRef] = None + var currentMasterCreatedByMe = false + + // subscribe to cluster changes, LeaderChanged + // re-subscribe when restart + override def preStart(): Unit = cluster.subscribe(self, classOf[LeaderChanged]) + override def postStop(): Unit = cluster.unsubscribe(self) + + def receive = { + case job: StatsJob if currentMaster.isEmpty ⇒ + sender ! JobFailed("Service unavailable, try again later") + case job: StatsJob ⇒ + currentMaster foreach { _ forward job } + case state: CurrentClusterState ⇒ + if (state.convergence) + state.leader foreach updateCurrentMaster + case LeaderChanged(Some(leaderAddress), true) ⇒ + updateCurrentMaster(leaderAddress) + case other: LeaderChanged ⇒ // ignore, not convergence + } + + def updateCurrentMaster(leaderAddress: Address): Unit = { + if (leaderAddress == cluster.selfAddress) { + if (!currentMasterCreatedByMe) { + log.info("Creating new statsService master at [{}]", leaderAddress) + currentMaster = Some(context.actorOf(Props[StatsService], name = "statsService")) + currentMasterCreatedByMe = true + } + } else { + if (currentMasterCreatedByMe) + currentMaster foreach { context.stop(_) } + log.info("Using statsService master at [{}]", leaderAddress) + currentMaster = Some(context.actorFor( + context.self.path.toStringWithAddress(leaderAddress) + "/statsService")) + currentMasterCreatedByMe = false + } + } + +} +//#facade + +object StatsSample { + def main(args: Array[String]): Unit = { + // Override the configuration of the port + // when specified as program argument + if (args.nonEmpty) System.setProperty("akka.remote.netty.port", args(0)) + + //#start-router-lookup + val system = ActorSystem("ClusterSystem", ConfigFactory.parseString(""" + akka.actor.deployment { + /statsService/workerRouter { + # FIXME use consistent hashing instead + router = round-robin + nr-of-instances = 100 + cluster { + enabled = on + routees-path = "/user/statsWorker" + allow-local-routees = on + } + } + } + """).withFallback(ConfigFactory.load())) + + system.actorOf(Props[StatsWorker], name = "statsWorker") + system.actorOf(Props[StatsService], name = "statsService") + //#start-router-lookup + + } +} + +object StatsSampleOneMaster { + def main(args: Array[String]): Unit = { + // Override the configuration of the port + // when specified as program argument + if (args.nonEmpty) System.setProperty("akka.remote.netty.port", args(0)) + + //#start-router-deploy + val system = ActorSystem("ClusterSystem", ConfigFactory.parseString(""" + akka.actor.deployment { + /statsFacade/statsService/workerRouter { + # FIXME use consistent hashing instead + router = round-robin + nr-of-instances = 100 + cluster { + enabled = on + max-nr-of-instances-per-node = 3 + allow-local-routees = off + } + } + } + """).withFallback(ConfigFactory.load())) + //#start-router-deploy + + system.actorOf(Props[StatsFacade], name = "statsFacade") + } +} + +object StatsSampleClient { + def main(args: Array[String]): Unit = { + val system = ActorSystem("ClusterSystem") + system.actorOf(Props(new StatsSampleClient("/user/statsService")), "client") + } +} + +object StatsSampleOneMasterClient { + def main(args: Array[String]): Unit = { + val system = ActorSystem("ClusterSystem") + system.actorOf(Props(new StatsSampleClient("/user/statsFacade")), "client") + } +} + +class StatsSampleClient(servicePath: String) extends Actor { + val cluster = Cluster(context.system) + val servicePathElements = servicePath match { + case RelativeActorPath(elements) ⇒ elements + case _ ⇒ throw new IllegalArgumentException( + "servicePath [%s] is not a valid relative actor path" format servicePath) + } + import context.dispatcher + val tickTask = context.system.scheduler.schedule(2 seconds, 2 seconds, self, "tick") + + var nodes = Set.empty[Address] + + override def preStart(): Unit = cluster.subscribe(self, classOf[MemberEvent]) + override def postStop(): Unit = { + cluster.unsubscribe(self) + tickTask.cancel() + } + + def receive = { + case "tick" if nodes.nonEmpty ⇒ + // just pick any one + val address = nodes.toIndexedSeq(ThreadLocalRandom.current.nextInt(nodes.size)) + context.actorFor(RootActorPath(address) / servicePathElements) ! + StatsJob("this is the text that will be analyzed") + case result: StatsResult ⇒ + println(result) + case failed: JobFailed ⇒ + println(failed) + case state: CurrentClusterState ⇒ + nodes = state.members.collect { case m if m.status == MemberStatus.Up ⇒ m.address } + case MemberUp(m) ⇒ nodes += m.address + case other: MemberEvent ⇒ nodes -= other.member.address + } + +} + +// not used, only for documentation +abstract class StatsService2 extends Actor { + //#router-lookup-in-code + import akka.cluster.routing.ClusterRouterConfig + import akka.cluster.routing.ClusterRouterSettings + import akka.routing.RoundRobinRouter + + val workerRouter = context.actorOf(Props[StatsWorker].withRouter( + ClusterRouterConfig(RoundRobinRouter(), ClusterRouterSettings( + totalInstances = 100, routeesPath = "/user/statsWorker", + allowLocalRoutees = true))), + name = "workerRouter2") + //#router-lookup-in-code +} + +// not used, only for documentation +abstract class StatsService3 extends Actor { + //#router-deploy-in-code + import akka.cluster.routing.ClusterRouterConfig + import akka.cluster.routing.ClusterRouterSettings + // FIXME use ConsistentHashingRouter instead + import akka.routing.RoundRobinRouter + + val workerRouter = context.actorOf(Props[StatsWorker].withRouter( + ClusterRouterConfig(RoundRobinRouter(), ClusterRouterSettings( + totalInstances = 100, maxInstancesPerNode = 3, + allowLocalRoutees = false))), + name = "workerRouter3") + //#router-deploy-in-code +} diff --git a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala new file mode 100644 index 0000000000..e26bcd245d --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala @@ -0,0 +1,108 @@ +package sample.cluster.transformation + +//#imports +import language.postfixOps +import scala.concurrent.util.duration._ + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.actor.Props +import akka.actor.RootActorPath +import akka.actor.Terminated +import akka.cluster.Cluster +import akka.cluster.ClusterEvent.CurrentClusterState +import akka.cluster.ClusterEvent.MemberEvent +import akka.cluster.ClusterEvent.MemberUp +import akka.cluster.Member +import akka.cluster.MemberStatus +import akka.pattern.ask +import akka.util.Timeout +//#imports + +//#messages +case class TransformationJob(text: String) +case class TransformationResult(text: String) +case class JobFailed(reason: String, job: TransformationJob) +case object BackendRegistration +//#messages + +object TransformationFrontend { + def main(args: Array[String]): Unit = { + // Override the configuration of the port + // when specified as program argument + if (args.nonEmpty) System.setProperty("akka.remote.netty.port", args(0)) + + val system = ActorSystem("ClusterSystem") + val frontend = system.actorOf(Props[TransformationFrontend], name = "frontend") + + import system.dispatcher + implicit val timeout = Timeout(5 seconds) + for (n ← 1 to 120) { + (frontend ? TransformationJob("hello-" + n)) onSuccess { + case result ⇒ println(result) + } + Thread.sleep(2000) + } + system.shutdown() + } +} + +//#frontend +class TransformationFrontend extends Actor { + + var backends = IndexedSeq.empty[ActorRef] + var jobCounter = 0 + + def receive = { + case job: TransformationJob if backends.isEmpty ⇒ + sender ! JobFailed("Service unavailable, try again later", job) + + case job: TransformationJob ⇒ + jobCounter += 1 + backends(jobCounter % backends.size) forward job + + case BackendRegistration if !backends.contains(sender) ⇒ + context watch sender + backends = backends :+ sender + + case Terminated(a) ⇒ backends.filterNot(_ == a) + } +} +//#frontend + +object TransformationBackend { + def main(args: Array[String]): Unit = { + // Override the configuration of the port + // when specified as program argument + if (args.nonEmpty) System.setProperty("akka.remote.netty.port", args(0)) + + val system = ActorSystem("ClusterSystem") + system.actorOf(Props[TransformationBackend], name = "backend") + } +} + +//#backend +class TransformationBackend extends Actor { + + val cluster = Cluster(context.system) + + // subscribe to cluster changes, MemberEvent + // re-subscribe when restart + override def preStart(): Unit = cluster.subscribe(self, classOf[MemberEvent]) + override def postStop(): Unit = cluster.unsubscribe(self) + + def receive = { + case TransformationJob(text) ⇒ sender ! TransformationResult(text.toUpperCase) + case state: CurrentClusterState ⇒ + state.members.filter(_.status == MemberStatus.Up) foreach register + case MemberUp(m) ⇒ register(m) + } + + // try to register to all nodes, even though there + // might not be any frontend on all nodes + def register(member: Member): Unit = + context.actorFor(RootActorPath(member.address) / "user" / "frontend") ! + BackendRegistration +} +//#backend \ No newline at end of file diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala new file mode 100644 index 0000000000..9c7315c902 --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala @@ -0,0 +1,101 @@ +package sample.cluster.stats + +import language.postfixOps +import scala.concurrent.util.duration._ + +import com.typesafe.config.ConfigFactory + +import StatsSampleSpec.first +import StatsSampleSpec.third +import akka.actor.Props +import akka.actor.RootActorPath +import akka.cluster.Cluster +import akka.cluster.Member +import akka.cluster.MemberStatus +import akka.cluster.ClusterEvent.CurrentClusterState +import akka.cluster.ClusterEvent.MemberUp +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit.ImplicitSender + +object StatsSampleSingleMasterSpec extends MultiNodeConfig { + // register the named roles (nodes) of the test + val first = role("first") + val second = role("second") + val third = role("thrid") + + // this configuration will be used for all nodes + // note that no fixed host names and ports are used + commonConfig(ConfigFactory.parseString(""" + akka.loglevel = INFO + akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.remote.log-remote-lifecycle-events = off + akka.cluster.auto-join = off + #//#router-deploy-config + akka.actor.deployment { + /statsFacade/statsService/workerRouter { + # FIXME use consistent hashing instead + router = round-robin + nr-of-instances = 100 + cluster { + enabled = on + max-nr-of-instances-per-node = 3 + allow-local-routees = off + } + } + } + #//#router-deploy-config + """)) + +} + +// need one concrete test class per node +class StatsSampleSingleMasterMultiJvmNode1 extends StatsSampleSingleMasterSpec +class StatsSampleSingleMasterMultiJvmNode2 extends StatsSampleSingleMasterSpec +class StatsSampleSingleMasterMultiJvmNode3 extends StatsSampleSingleMasterSpec + +abstract class StatsSampleSingleMasterSpec extends MultiNodeSpec(StatsSampleSingleMasterSpec) + with ImplicitSender { + + import StatsSampleSpec._ + + override def initialParticipants = roles.size + + "The stats sample with single master" must { + "illustrate how to startup cluster" in within(10 seconds) { + Cluster(system).subscribe(testActor, classOf[MemberUp]) + expectMsgClass(classOf[CurrentClusterState]) + + Cluster(system) join node(first).address + system.actorOf(Props[StatsFacade], "statsFacade") + + expectMsgAllOf( + MemberUp(Member(node(first).address, MemberStatus.Up)), + MemberUp(Member(node(second).address, MemberStatus.Up)), + MemberUp(Member(node(third).address, MemberStatus.Up))) + + Cluster(system).unsubscribe(testActor) + + testConductor.enter("all-up") + } + + "show usage of the statsFacade" in within(5 seconds) { + val facade = system.actorFor(RootActorPath(node(third).address) / "user" / "statsFacade") + + // eventually the service should be ok, + // worker nodes might not be up yet + awaitCond { + facade ! StatsJob("this is the text that will be analyzed") + expectMsgPF() { + case unavailble: JobFailed ⇒ false + case StatsResult(meanWordLength) ⇒ + meanWordLength must be(3.875 plusOrMinus 0.001) + true + } + } + + testConductor.enter("done") + } + } + +} \ No newline at end of file diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala new file mode 100644 index 0000000000..1b0a9e08b8 --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala @@ -0,0 +1,92 @@ +package sample.cluster.stats + +import language.postfixOps +import scala.concurrent.util.duration._ +import com.typesafe.config.ConfigFactory + +import akka.actor.Props +import akka.actor.RootActorPath +import akka.cluster.Cluster +import akka.cluster.Member +import akka.cluster.MemberStatus +import akka.cluster.ClusterEvent.CurrentClusterState +import akka.cluster.ClusterEvent.MemberUp +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit.ImplicitSender + +object StatsSampleSpec extends MultiNodeConfig { + // register the named roles (nodes) of the test + val first = role("first") + val second = role("second") + val third = role("thrid") + + // this configuration will be used for all nodes + // note that no fixed host names and ports are used + commonConfig(ConfigFactory.parseString(""" + akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.remote.log-remote-lifecycle-events = off + akka.cluster.auto-join = off + #//#router-lookup-config + akka.actor.deployment { + /statsService/workerRouter { + # FIXME use consistent hashing instead + router = round-robin + nr-of-instances = 100 + cluster { + enabled = on + routees-path = "/user/statsWorker" + allow-local-routees = on + } + } + } + #//#router-lookup-config + """)) + +} + +// need one concrete test class per node +class StatsSampleMultiJvmNode1 extends StatsSampleSpec +class StatsSampleMultiJvmNode2 extends StatsSampleSpec +class StatsSampleMultiJvmNode3 extends StatsSampleSpec + +abstract class StatsSampleSpec extends MultiNodeSpec(StatsSampleSpec) + with ImplicitSender { + + import StatsSampleSpec._ + + override def initialParticipants = roles.size + + "The stats sample" must { + "illustrate how to startup cluster" in within(10 seconds) { + Cluster(system).subscribe(testActor, classOf[MemberUp]) + expectMsgClass(classOf[CurrentClusterState]) + + Cluster(system) join node(first).address + system.actorOf(Props[StatsWorker], "statsWorker") + system.actorOf(Props[StatsService], "statsService") + + expectMsgAllOf( + MemberUp(Member(node(first).address, MemberStatus.Up)), + MemberUp(Member(node(second).address, MemberStatus.Up)), + MemberUp(Member(node(third).address, MemberStatus.Up))) + + Cluster(system).unsubscribe(testActor) + + testConductor.enter("all-up") + } + + "show usage of the statsService" in within(5 seconds) { + + val service = system.actorFor(RootActorPath(node(third).address) / "user" / "statsService") + service ! StatsJob("this is the text that will be analyzed") + val meanWordLength = expectMsgPF() { + case StatsResult(meanWordLength) ⇒ meanWordLength + } + meanWordLength must be(3.875 plusOrMinus 0.001) + + testConductor.enter("done") + } + } + +} \ No newline at end of file diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala new file mode 100644 index 0000000000..a18a4a4896 --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala @@ -0,0 +1,115 @@ +package sample.cluster.transformation + +import language.postfixOps +import scala.concurrent.util.duration._ + +import com.typesafe.config.ConfigFactory + +import akka.actor.Props +import akka.cluster.Cluster +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.testkit.ImplicitSender + +object TransformationSampleSpec extends MultiNodeConfig { + // register the named roles (nodes) of the test + val frontend1 = role("frontend1") + val frontend2 = role("frontend2") + val backend1 = role("backend1") + val backend2 = role("backend2") + val backend3 = role("backend3") + + // this configuration will be used for all nodes + // note that no fixed host names and ports are used + commonConfig(ConfigFactory.parseString(""" + akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.remote.log-remote-lifecycle-events = off + akka.cluster.auto-join = off + """)) + +} + +// need one concrete test class per node +class TransformationSampleMultiJvmNode1 extends TransformationSampleSpec +class TransformationSampleMultiJvmNode2 extends TransformationSampleSpec +class TransformationSampleMultiJvmNode3 extends TransformationSampleSpec +class TransformationSampleMultiJvmNode4 extends TransformationSampleSpec +class TransformationSampleMultiJvmNode5 extends TransformationSampleSpec + +abstract class TransformationSampleSpec extends MultiNodeSpec(TransformationSampleSpec) + with ImplicitSender { + + import TransformationSampleSpec._ + + override def initialParticipants = roles.size + + "The transformation sample" must { + "illustrate how to start first frontend" in { + runOn(frontend1) { + // this will only run on the 'first' node + Cluster(system) join node(frontend1).address + val transformationFrontend = system.actorOf(Props[TransformationFrontend], name = "frontend") + transformationFrontend ! TransformationJob("hello") + expectMsgPF() { + // no backends yet, service unavailble + case JobFailed(_, TransformationJob("hello")) ⇒ + } + } + + // this will run on all nodes + // use barrier to coordinate test steps + testConductor.enter("frontend1-started") + } + + "illustrate how a backend automatically registers" in within(15 seconds) { + runOn(backend1) { + Cluster(system) join node(frontend1).address + system.actorOf(Props[TransformationBackend], name = "backend") + } + testConductor.enter("backend1-started") + + runOn(frontend1) { + assertServiceOk + } + + testConductor.enter("frontend1-backend1-ok") + } + + "illustrate how more nodes registers" in within(15 seconds) { + runOn(frontend2) { + Cluster(system) join node(frontend1).address + system.actorOf(Props[TransformationFrontend], name = "frontend") + } + runOn(backend2, backend3) { + Cluster(system) join node(backend1).address + system.actorOf(Props[TransformationBackend], name = "backend") + } + + testConductor.enter("all-started") + + runOn(frontend1, frontend2) { + assertServiceOk + } + + testConductor.enter("all-ok") + + } + + } + + def assertServiceOk: Unit = { + val transformationFrontend = system.actorFor("akka://" + system.name + "/user/frontend") + // eventually the service should be ok, + // backends might not have registered initially + awaitCond { + transformationFrontend ! TransformationJob("hello") + expectMsgPF() { + case unavailble: JobFailed ⇒ false + case TransformationResult(result) ⇒ + result must be("HELLO") + true + } + } + } + +} \ No newline at end of file diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 8c102ec39f..7e5f6fb067 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -326,11 +326,19 @@ object AkkaBuild extends Build { ) lazy val clusterSample = Project( - id = "akka-sample-cluster", + id = "akka-sample-cluster-experimental", base = file("akka-samples/akka-sample-cluster"), - dependencies = Seq(cluster), - settings = defaultSettings - ) + dependencies = Seq(cluster, remoteTests % "compile;test->test;multi-jvm->multi-jvm", testkit % "test->test"), + settings = defaultSettings ++ multiJvmSettings ++ Seq( + // disable parallel tests + parallelExecution in Test := false, + extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => + (name: String) => (src ** (name + ".conf")).get.headOption.map("-Dakka.config=" + _.absolutePath).toSeq + }, + scalatestOptions in MultiJvm := defaultMultiJvmScalatestOptions, + jvmOptions in MultiJvm := defaultMultiJvmOptions + ) + ) configs (MultiJvm) lazy val docs = Project( id = "akka-docs", From d74464ba50ebe1f75a5720f9f0e7f76f9df4d53c Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 13 Sep 2012 14:25:25 +0200 Subject: [PATCH 07/46] Move MurmurHash to separate file due to license header, see #944 * Also removed unused 'class MurmurHash', we only use 'object MurmurHash' --- .../scala/akka/routing/ConsistentHash.scala | 197 ------------------ .../main/scala/akka/routing/MurmurHash.scala | 149 +++++++++++++ 2 files changed, 149 insertions(+), 197 deletions(-) create mode 100644 akka-actor/src/main/scala/akka/routing/MurmurHash.scala diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index 21f9bffbce..5dc23f1383 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -5,27 +5,6 @@ package akka.routing import scala.collection.immutable.TreeMap -import java.lang.Integer.{ rotateLeft ⇒ rotl } - -/* __ *\ -** ________ ___ / / ___ Scala API ** -** / __/ __// _ | / / / _ | (c) 2003-2011, LAMP/EPFL ** -** __\ \/ /__/ __ |/ /__/ __ | http://scala-lang.org/ ** -** /____/\___/_/ |_/____/_/ | | ** -** |/ ** -\* */ - -/** - * An implementation of Austin Appleby's MurmurHash 3.0 algorithm - * (32 bit version); reference: http://code.google.com/p/smhasher - * - * This is the hash used by collections and case classes (including - * tuples). - * - * @author Rex Kerr - * @version 2.9 - * @since 2.9 - */ // ============================================================================================= // Adapted from HashRing.scala in Debasish Ghosh's Redis Client, licensed under Apache 2 license @@ -129,179 +108,3 @@ object ConsistentHash { } } -/** - * A class designed to generate well-distributed non-cryptographic - * hashes. It is designed to be passed to a collection's foreach method, - * or can take individual hash values with append. Its own hash code is - * set equal to the hash code of whatever it is hashing. - */ -class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T ⇒ Unit) { - import MurmurHash._ - - private var h = startHash(seed) - private var c = hiddenMagicA - private var k = hiddenMagicB - private var hashed = false - private var hashvalue = h - - /** Begin a new hash using the same seed. */ - def reset(): Unit = { - h = startHash(seed) - c = hiddenMagicA - k = hiddenMagicB - hashed = false - } - - /** Incorporate the hash value of one item. */ - def apply(t: T): Unit = { - h = extendHash(h, t.##, c, k) - c = nextMagicA(c) - k = nextMagicB(k) - hashed = false - } - - /** Incorporate a known hash value. */ - def append(i: Int): Unit = { - h = extendHash(h, i, c, k) - c = nextMagicA(c) - k = nextMagicB(k) - hashed = false - } - - /** Retrieve the hash value */ - def hash: Int = { - if (!hashed) { - hashvalue = finalizeHash(h) - hashed = true - } - hashvalue - } - - override def hashCode: Int = hash -} - -/** - * An object designed to generate well-distributed non-cryptographic - * hashes. It is designed to hash a collection of integers; along with - * the integers to hash, it generates two magic streams of integers to - * increase the distribution of repetitive input sequences. Thus, - * three methods need to be called at each step (to start and to - * incorporate a new integer) to update the values. Only one method - * needs to be called to finalize the hash. - */ - -object MurmurHash { - // Magic values used for MurmurHash's 32 bit hash. - // Don't change these without consulting a hashing expert! - final private val visibleMagic: Int = 0x971e137b - final private val hiddenMagicA: Int = 0x95543787 - final private val hiddenMagicB: Int = 0x2ad7eb25 - final private val visibleMixer: Int = 0x52dce729 - final private val hiddenMixerA: Int = 0x7b7d159c - final private val hiddenMixerB: Int = 0x6bce6396 - final private val finalMixer1: Int = 0x85ebca6b - final private val finalMixer2: Int = 0xc2b2ae35 - - // Arbitrary values used for hashing certain classes - final private val seedString: Int = 0xf7ca7fd2 - final private val seedArray: Int = 0x3c074a61 - - /** The first 23 magic integers from the first stream are stored here */ - private val storedMagicA: Array[Int] = - Iterator.iterate(hiddenMagicA)(nextMagicA).take(23).toArray - - /** The first 23 magic integers from the second stream are stored here */ - private val storedMagicB: Array[Int] = - Iterator.iterate(hiddenMagicB)(nextMagicB).take(23).toArray - - /** Begin a new hash with a seed value. */ - def startHash(seed: Int): Int = seed ^ visibleMagic - - /** The initial magic integers in the first stream. */ - def startMagicA: Int = hiddenMagicA - - /** The initial magic integer in the second stream. */ - def startMagicB: Int = hiddenMagicB - - /** - * Incorporates a new value into an existing hash. - * - * @param hash the prior hash value - * @param value the new value to incorporate - * @param magicA a magic integer from the stream - * @param magicB a magic integer from a different stream - * @return the updated hash value - */ - def extendHash(hash: Int, value: Int, magicA: Int, magicB: Int): Int = - (hash ^ rotl(value * magicA, 11) * magicB) * 3 + visibleMixer - - /** Given a magic integer from the first stream, compute the next */ - def nextMagicA(magicA: Int): Int = magicA * 5 + hiddenMixerA - - /** Given a magic integer from the second stream, compute the next */ - def nextMagicB(magicB: Int): Int = magicB * 5 + hiddenMixerB - - /** Once all hashes have been incorporated, this performs a final mixing */ - def finalizeHash(hash: Int): Int = { - var i = (hash ^ (hash >>> 16)) - i *= finalMixer1 - i ^= (i >>> 13) - i *= finalMixer2 - i ^= (i >>> 16) - i - } - - /** Compute a high-quality hash of an array */ - def arrayHash[@specialized T](a: Array[T]): Int = { - var h = startHash(a.length * seedArray) - var c = hiddenMagicA - var k = hiddenMagicB - var j = 0 - while (j < a.length) { - h = extendHash(h, a(j).##, c, k) - c = nextMagicA(c) - k = nextMagicB(k) - j += 1 - } - finalizeHash(h) - } - - /** Compute a high-quality hash of a string */ - def stringHash(s: String): Int = { - var h = startHash(s.length * seedString) - var c = hiddenMagicA - var k = hiddenMagicB - var j = 0 - while (j + 1 < s.length) { - val i = (s.charAt(j) << 16) + s.charAt(j + 1); - h = extendHash(h, i, c, k) - c = nextMagicA(c) - k = nextMagicB(k) - j += 2 - } - if (j < s.length) h = extendHash(h, s.charAt(j), c, k) - finalizeHash(h) - } - - /** - * Compute a hash that is symmetric in its arguments--that is, - * where the order of appearance of elements does not matter. - * This is useful for hashing sets, for example. - */ - def symmetricHash[T](xs: TraversableOnce[T], seed: Int): Int = { - var a, b, n = 0 - var c = 1 - xs.foreach(i ⇒ { - val h = i.## - a += h - b ^= h - if (h != 0) c *= h - n += 1 - }) - var h = startHash(seed * n) - h = extendHash(h, a, storedMagicA(0), storedMagicB(0)) - h = extendHash(h, b, storedMagicA(1), storedMagicB(1)) - h = extendHash(h, c, storedMagicA(2), storedMagicB(2)) - finalizeHash(h) - } -} diff --git a/akka-actor/src/main/scala/akka/routing/MurmurHash.scala b/akka-actor/src/main/scala/akka/routing/MurmurHash.scala new file mode 100644 index 0000000000..fc67613a5d --- /dev/null +++ b/akka-actor/src/main/scala/akka/routing/MurmurHash.scala @@ -0,0 +1,149 @@ +/* __ *\ +** ________ ___ / / ___ Scala API ** +** / __/ __// _ | / / / _ | (c) 2003-2011, LAMP/EPFL ** +** __\ \/ /__/ __ |/ /__/ __ | http://scala-lang.org/ ** +** /____/\___/_/ |_/____/_/ | | ** +** |/ ** +\* */ + +/** + * An implementation of Austin Appleby's MurmurHash 3.0 algorithm + * (32 bit version); reference: http://code.google.com/p/smhasher + * + * This is the hash used by collections and case classes (including + * tuples). + * + * @author Rex Kerr + * @version 2.9 + * @since 2.9 + */ + +package akka.routing + +import java.lang.Integer.{ rotateLeft ⇒ rotl } + +/** + * An object designed to generate well-distributed non-cryptographic + * hashes. It is designed to hash a collection of integers; along with + * the integers to hash, it generates two magic streams of integers to + * increase the distribution of repetitive input sequences. Thus, + * three methods need to be called at each step (to start and to + * incorporate a new integer) to update the values. Only one method + * needs to be called to finalize the hash. + */ + +object MurmurHash { + // Magic values used for MurmurHash's 32 bit hash. + // Don't change these without consulting a hashing expert! + final private val visibleMagic: Int = 0x971e137b + final private val hiddenMagicA: Int = 0x95543787 + final private val hiddenMagicB: Int = 0x2ad7eb25 + final private val visibleMixer: Int = 0x52dce729 + final private val hiddenMixerA: Int = 0x7b7d159c + final private val hiddenMixerB: Int = 0x6bce6396 + final private val finalMixer1: Int = 0x85ebca6b + final private val finalMixer2: Int = 0xc2b2ae35 + + // Arbitrary values used for hashing certain classes + final private val seedString: Int = 0xf7ca7fd2 + final private val seedArray: Int = 0x3c074a61 + + /** The first 23 magic integers from the first stream are stored here */ + private val storedMagicA: Array[Int] = + Iterator.iterate(hiddenMagicA)(nextMagicA).take(23).toArray + + /** The first 23 magic integers from the second stream are stored here */ + private val storedMagicB: Array[Int] = + Iterator.iterate(hiddenMagicB)(nextMagicB).take(23).toArray + + /** Begin a new hash with a seed value. */ + def startHash(seed: Int): Int = seed ^ visibleMagic + + /** The initial magic integers in the first stream. */ + def startMagicA: Int = hiddenMagicA + + /** The initial magic integer in the second stream. */ + def startMagicB: Int = hiddenMagicB + + /** + * Incorporates a new value into an existing hash. + * + * @param hash the prior hash value + * @param value the new value to incorporate + * @param magicA a magic integer from the stream + * @param magicB a magic integer from a different stream + * @return the updated hash value + */ + def extendHash(hash: Int, value: Int, magicA: Int, magicB: Int): Int = + (hash ^ rotl(value * magicA, 11) * magicB) * 3 + visibleMixer + + /** Given a magic integer from the first stream, compute the next */ + def nextMagicA(magicA: Int): Int = magicA * 5 + hiddenMixerA + + /** Given a magic integer from the second stream, compute the next */ + def nextMagicB(magicB: Int): Int = magicB * 5 + hiddenMixerB + + /** Once all hashes have been incorporated, this performs a final mixing */ + def finalizeHash(hash: Int): Int = { + var i = (hash ^ (hash >>> 16)) + i *= finalMixer1 + i ^= (i >>> 13) + i *= finalMixer2 + i ^= (i >>> 16) + i + } + + /** Compute a high-quality hash of an array */ + def arrayHash[@specialized T](a: Array[T]): Int = { + var h = startHash(a.length * seedArray) + var c = hiddenMagicA + var k = hiddenMagicB + var j = 0 + while (j < a.length) { + h = extendHash(h, a(j).##, c, k) + c = nextMagicA(c) + k = nextMagicB(k) + j += 1 + } + finalizeHash(h) + } + + /** Compute a high-quality hash of a string */ + def stringHash(s: String): Int = { + var h = startHash(s.length * seedString) + var c = hiddenMagicA + var k = hiddenMagicB + var j = 0 + while (j + 1 < s.length) { + val i = (s.charAt(j) << 16) + s.charAt(j + 1); + h = extendHash(h, i, c, k) + c = nextMagicA(c) + k = nextMagicB(k) + j += 2 + } + if (j < s.length) h = extendHash(h, s.charAt(j), c, k) + finalizeHash(h) + } + + /** + * Compute a hash that is symmetric in its arguments--that is, + * where the order of appearance of elements does not matter. + * This is useful for hashing sets, for example. + */ + def symmetricHash[T](xs: TraversableOnce[T], seed: Int): Int = { + var a, b, n = 0 + var c = 1 + xs.foreach(i ⇒ { + val h = i.## + a += h + b ^= h + if (h != 0) c *= h + n += 1 + }) + var h = startHash(seed * n) + h = extendHash(h, a, storedMagicA(0), storedMagicB(0)) + h = extendHash(h, b, storedMagicA(1), storedMagicB(1)) + h = extendHash(h, c, storedMagicA(2), storedMagicB(2)) + finalizeHash(h) + } +} From 5a90d7198cea48e207ae5618971357a4ddb931ea Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 13 Sep 2012 18:06:35 +0200 Subject: [PATCH 08/46] Improvements based on review feedback, see #944 --- .../test/scala/akka/config/ConfigSpec.scala | 3 + .../routing/ConsistentHashingRouterSpec.scala | 11 +--- akka-actor/src/main/resources/reference.conf | 15 +++-- .../main/scala/akka/actor/ActorSystem.scala | 2 + .../src/main/scala/akka/actor/Deployer.scala | 20 +++--- .../scala/akka/routing/ConsistentHash.scala | 59 ++++++++---------- .../routing/ConsistentHashingRouter.scala | 61 ++++++++++--------- .../src/main/scala/akka/routing/Routing.scala | 20 ++++++ .../docs/routing/RouterViaConfigDocSpec.scala | 1 + 9 files changed, 106 insertions(+), 86 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala index 5fe3466b25..56f8cd45fc 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -43,6 +43,9 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin getBoolean("akka.jvm-exit-on-fatal-error") must be(true) settings.JvmExitOnFatalError must be(true) + + getInt("akka.actor.deployment.default.virtual-nodes-factor") must be(10) + settings.DefaultVirtualNodesFactor must be(10) } { diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala index fd7a49e867..7a28190523 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala @@ -22,6 +22,7 @@ object ConsistentHashingRouterSpec { /router1 { router = consistent-hashing nr-of-instances = 3 + virtual-nodes-factor = 17 } } """ @@ -55,18 +56,12 @@ class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.c "select destination based on consistentHashKey of the message" in { router1 ! Msg("a", "A") val destinationA = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } - router1 ! new ConsistentHashableEnvelope { - override def consistentHashKey = "a" - override def message = "AA" - } + router1 ! ConsistentHashableEnvelope(message = "AA", consistentHashKey = "a") expectMsg(destinationA) router1 ! Msg(17, "B") val destinationB = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } - router1 ! new ConsistentHashableEnvelope { - override def consistentHashKey = 17 - override def message = "BB" - } + router1 ! ConsistentHashableEnvelope(message = "BB", consistentHashKey = 17) expectMsg(destinationB) router1 ! Msg(MsgKey("c"), "C") diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index a585fe3406..163d2e83d9 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -14,7 +14,7 @@ akka { # Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT) event-handlers = ["akka.event.Logging$DefaultLogger"] - + # Event handlers are created and registered synchronously during ActorSystem # start-up, and since they are actors, this timeout is used to bound the # waiting time @@ -49,7 +49,7 @@ akka { # FQCN of the ActorRefProvider to be used; the below is the built-in default, # another one is akka.remote.RemoteActorRefProvider in the akka-remote bundle. provider = "akka.actor.LocalActorRefProvider" - + # The guardian "/user" will use this subclass of akka.actor.SupervisorStrategyConfigurator # to obtain its supervisorStrategy. Besides the default there is # akka.actor.StoppingSupervisorStrategy @@ -69,7 +69,7 @@ akka { # Serializes and deserializes creators (in Props) to ensure that they can be sent over the network, # this is only intended for testing. serialize-creators = off - + # Timeout for send operations to top-level actors which are in the process of being started. # This is only relevant if using a bounded mailbox or the CallingThreadDispatcher for a top-level actor. unstarted-push-timeout = 10s @@ -108,6 +108,9 @@ akka { # within is the timeout used for routers containing future calls within = 5 seconds + # number of virtual nodes per node for consistent-hashing router + virtual-nodes-factor = 10 + routees { # Alternatively to giving nr-of-instances you can specify the full # paths of those actors which should be routed to. This setting takes @@ -296,7 +299,7 @@ akka { # enable DEBUG logging of subscription changes on the eventStream event-stream = off - + # enable DEBUG logging of unhandled messages unhandled = off @@ -318,13 +321,13 @@ akka { serialization-bindings { "java.io.Serializable" = java } - + # Configuration items which are used by the akka.actor.ActorDSL._ methods dsl { # Maximum queue size of the actor created by newInbox(); this protects against # faulty programs which use select() and consistently miss messages inbox-size = 1000 - + # Default timeout to assume for operations like Inbox.receive et al default-timeout = 5s } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index bce966b99e..deec7697df 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -166,6 +166,8 @@ object ActorSystem { final val Daemonicity: Boolean = getBoolean("akka.daemonic") final val JvmExitOnFatalError: Boolean = getBoolean("akka.jvm-exit-on-fatal-error") + final val DefaultVirtualNodesFactor: Int = getInt("akka.actor.deployment.default.virtual-nodes-factor") + if (ConfigVersion != Version) throw new akka.ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]") diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index be9da1a505..3f5e2b1785 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -143,18 +143,20 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce val nrOfInstances = deployment.getInt("nr-of-instances") - val within = Duration(deployment.getMilliseconds("within"), TimeUnit.MILLISECONDS) - val resizer: Option[Resizer] = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None val router: RouterConfig = deployment.getString("router") match { - case "from-code" ⇒ NoRouter - case "round-robin" ⇒ RoundRobinRouter(nrOfInstances, routees, resizer) - case "random" ⇒ RandomRouter(nrOfInstances, routees, resizer) - case "smallest-mailbox" ⇒ SmallestMailboxRouter(nrOfInstances, routees, resizer) - case "scatter-gather" ⇒ ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer) - case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer) - case "consistent-hashing" ⇒ ConsistentHashingRouter(nrOfInstances, routees, resizer) + case "from-code" ⇒ NoRouter + case "round-robin" ⇒ RoundRobinRouter(nrOfInstances, routees, resizer) + case "random" ⇒ RandomRouter(nrOfInstances, routees, resizer) + case "smallest-mailbox" ⇒ SmallestMailboxRouter(nrOfInstances, routees, resizer) + case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer) + case "scatter-gather" ⇒ + val within = Duration(deployment.getMilliseconds("within"), TimeUnit.MILLISECONDS) + ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer) + case "consistent-hashing" ⇒ + val vnodes = deployment.getInt("virtual-nodes-factor") + ConsistentHashingRouter(nrOfInstances, routees, resizer, virtualNodesFactor = vnodes) case fqn ⇒ val args = Seq(classOf[Config] -> deployment) dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({ diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index 5dc23f1383..19e457a762 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -20,23 +20,22 @@ import scala.collection.immutable.TreeMap * hash, i.e. make sure it is different for different nodes. * */ -class ConsistentHash[T] private (replicas: Int, ring: TreeMap[Int, T]) { +class ConsistentHash[T] private (nodeRing: TreeMap[Int, T], virtualNodesFactor: Int) { import ConsistentHash._ - if (replicas < 1) throw new IllegalArgumentException("replicas must be >= 1") + if (virtualNodesFactor < 1) throw new IllegalArgumentException("virtualNodesFactor must be >= 1") /** - * Adds a node to the ring. + * Adds a node to the node ring. * Note that the instance is immutable and this * operation returns a new instance. */ - def :+(node: T): ConsistentHash[T] = { - new ConsistentHash(replicas, ring ++ ((1 to replicas) map { r ⇒ (nodeHashFor(node, r) -> node) })) - } + def :+(node: T): ConsistentHash[T] = + new ConsistentHash(nodeRing ++ ((1 to virtualNodesFactor) map { r ⇒ (nodeHashFor(node, r) -> node) }), virtualNodesFactor) /** - * Adds a node to the ring. + * Adds a node to the node ring. * Note that the instance is immutable and this * operation returns a new instance. * JAVA API @@ -44,16 +43,15 @@ class ConsistentHash[T] private (replicas: Int, ring: TreeMap[Int, T]) { def add(node: T): ConsistentHash[T] = this :+ node /** - * Removes a node from the ring. + * Removes a node from the node ring. * Note that the instance is immutable and this * operation returns a new instance. */ - def :-(node: T): ConsistentHash[T] = { - new ConsistentHash(replicas, ring -- ((1 to replicas) map { r ⇒ nodeHashFor(node, r) })) - } + def :-(node: T): ConsistentHash[T] = + new ConsistentHash(nodeRing -- ((1 to virtualNodesFactor) map { r ⇒ nodeHashFor(node, r) }), virtualNodesFactor) /** - * Removes a node from the ring. + * Removes a node from the node ring. * Note that the instance is immutable and this * operation returns a new instance. * JAVA API @@ -62,49 +60,42 @@ class ConsistentHash[T] private (replicas: Int, ring: TreeMap[Int, T]) { /** * Get the node responsible for the data key. - * Can only be used if nodes exists in the ring, + * Can only be used if nodes exists in the node ring, * otherwise throws `IllegalStateException` */ def nodeFor(key: Array[Byte]): T = { - if (isEmpty) throw new IllegalStateException("Can't get node for [%s] from an empty ring" format key) + if (isEmpty) throw new IllegalStateException("Can't get node for [%s] from an empty node ring" format key) val hash = hashFor(key) - def nextClockwise: T = { - val (ringKey, node) = ring.rangeImpl(Some(hash), None).headOption.getOrElse(ring.head) - node - } - ring.getOrElse(hash, nextClockwise) + // find the next node clockwise in the nodeRing, pick first if end of tree + nodeRing.rangeImpl(Some(hash), None).headOption.getOrElse(nodeRing.head)._2 } /** - * Is the ring empty, i.e. no nodes added or all removed. + * Is the node ring empty, i.e. no nodes added or all removed. */ - def isEmpty: Boolean = ring.isEmpty + def isEmpty: Boolean = nodeRing.isEmpty } object ConsistentHash { - def apply[T](nodes: Iterable[T], replicas: Int) = { - new ConsistentHash(replicas, TreeMap.empty[Int, T] ++ - (for (node ← nodes; replica ← 1 to replicas) yield (nodeHashFor(node, replica) -> node))) + def apply[T](nodes: Iterable[T], virtualNodesFactor: Int) = { + new ConsistentHash(TreeMap.empty[Int, T] ++ + (for (node ← nodes; vnode ← 1 to virtualNodesFactor) yield (nodeHashFor(node, vnode) -> node)), + virtualNodesFactor) } /** * Factory method to create a ConsistentHash * JAVA API */ - def create[T](nodes: java.lang.Iterable[T], replicas: Int) = { + def create[T](nodes: java.lang.Iterable[T], virtualNodesFactor: Int) = { import scala.collection.JavaConverters._ - apply(nodes.asScala, replicas) + apply(nodes.asScala, virtualNodesFactor) } - private def nodeHashFor(node: Any, replica: Int): Int = { - hashFor((node + ":" + replica).getBytes("UTF-8")) - } + private def nodeHashFor(node: Any, vnode: Int): Int = + hashFor((node + ":" + vnode).getBytes("UTF-8")) - private def hashFor(bytes: Array[Byte]): Int = { - val hash = MurmurHash.arrayHash(bytes) - if (hash == Int.MinValue) hash + 1 - math.abs(hash) - } + private def hashFor(bytes: Array[Byte]): Int = MurmurHash.arrayHash(bytes) } diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index eba812ac60..e177fc413a 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -5,13 +5,13 @@ package akka.routing import scala.collection.JavaConversions.iterableAsScalaIterable import scala.util.control.NonFatal - import akka.actor.ActorRef import akka.actor.SupervisorStrategy import akka.actor.Props import akka.dispatch.Dispatchers import akka.event.Logging import akka.serialization.SerializationExtension +import java.util.concurrent.atomic.AtomicReference object ConsistentHashingRouter { /** @@ -46,19 +46,13 @@ object ConsistentHashingRouter { /** * If messages can't implement [[akka.routing.ConsistentHashable]] - * themselves they can we wrapped by something implementing - * this interface instead. The router will only send the - * wrapped message to the destination, i.e. the envelope will - * be stripped off. + * themselves they can we wrapped by this envelope instead. The + * router will only send the wrapped message to the destination, + * i.e. the envelope will be stripped off. */ - trait ConsistentHashableEnvelope extends ConsistentHashable with RouterEnvelope { - def message: Any - } - - /** - * Default number of replicas (virtual nodes) used in [[akka.routing.ConsistantHash]] - */ - val DefaultReplicas: Int = 10 + @SerialVersionUID(1L) + case class ConsistentHashableEnvelope(message: Any, consistentHashKey: Any) + extends ConsistentHashable with RouterEnvelope } /** @@ -88,14 +82,14 @@ object ConsistentHashingRouter { * * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] - * @param replicas number of replicas (virtual nodes) used in [[akka.routing.ConsistantHash]] + * @param virtualNodesFactor number of virtual nodes per node, used in [[akka.routing.ConsistantHash]] */ @SerialVersionUID(1L) case class ConsistentHashingRouter( nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy, - val replicas: Int = ConsistentHashingRouter.DefaultReplicas) + val virtualNodesFactor: Int = 0) extends RouterConfig with ConsistentHashingLike { /** @@ -130,9 +124,9 @@ case class ConsistentHashingRouter( def withSupervisorStrategy(strategy: SupervisorStrategy): ConsistentHashingRouter = copy(supervisorStrategy = strategy) /** - * Java API for setting the number of replicas (virtual nodes) used in [[akka.routing.ConsistantHash]] + * Java API for setting the number of virtual nodes per node, used in [[akka.routing.ConsistantHash]] */ - def withReplicas(replicas: Int): ConsistentHashingRouter = copy(replicas = replicas) + def withVirtualNodesFactor(vnodes: Int): ConsistentHashingRouter = copy(virtualNodesFactor = vnodes) /** * Uses the resizer of the given RouterConfig if this RouterConfig @@ -145,6 +139,10 @@ case class ConsistentHashingRouter( } } +/** + * The core pieces of the routing logic is located in this + * trait to be able to extend. + */ trait ConsistentHashingLike { this: RouterConfig ⇒ import ConsistentHashingRouter._ @@ -153,7 +151,7 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ def routees: Iterable[String] - def replicas: Int + def virtualNodesFactor: Int override def createRoute(routeeProvider: RouteeProvider): Route = { if (resizer.isEmpty) { @@ -162,20 +160,25 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ } val log = Logging(routeeProvider.context.system, routeeProvider.context.self) + val vnodes = + if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor + else virtualNodesFactor - // consistentHashRoutees and consistentHash are updated together, synchronized on the consistentHashLock - val consistentHashLock = new Object - var consistentHashRoutees: IndexedSeq[ActorRef] = null - var consistentHash: ConsistentHash[ActorRef] = null - upateConsistentHash() + var consistentHashRoutees = new AtomicReference[IndexedSeq[ActorRef]] + @volatile var consistentHash: ConsistentHash[ActorRef] = null + updateConsistentHash() // update consistentHash when routees has changed // changes to routees are rare and when no changes this is a quick operation - def upateConsistentHash(): ConsistentHash[ActorRef] = consistentHashLock.synchronized { + def updateConsistentHash(): ConsistentHash[ActorRef] = { val currentRoutees = routeeProvider.routees - if ((currentRoutees ne consistentHashRoutees) && currentRoutees != consistentHashRoutees) { - consistentHashRoutees = currentRoutees - consistentHash = ConsistentHash(currentRoutees, replicas) + if (currentRoutees ne consistentHashRoutees.get) { + val oldConsistentHashRoutees = consistentHashRoutees.get + val rehash = currentRoutees != oldConsistentHashRoutees + // when other instance, same content, no need to re-hash, but try to set currentRoutees + // ignore, don't update, in case of CAS failure + if (consistentHashRoutees.compareAndSet(oldConsistentHashRoutees, currentRoutees) && rehash) + consistentHash = ConsistentHash(currentRoutees, vnodes) } consistentHash } @@ -186,13 +189,13 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ case str: String ⇒ str.getBytes("UTF-8") case x: AnyRef ⇒ SerializationExtension(routeeProvider.context.system).serialize(x).get } - val currentConsistenHash = upateConsistentHash() + val currentConsistenHash = updateConsistentHash() if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters else currentConsistenHash.nodeFor(hash) } catch { case NonFatal(e) ⇒ // serialization failed - log.warning("Couldn't route message with consistentHashKey [%s] due to [%s]".format(hashData, e.getMessage)) + log.warning("Couldn't route message with consistentHashKey [{}] due to [{}]", hashData, e.getMessage) routeeProvider.context.system.deadLetters } diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 9e17bf2b9a..da3c439c4d 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -596,6 +596,10 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = } } +/** + * The core pieces of the routing logic is located in this + * trait to be able to extend. + */ trait RoundRobinLike { this: RouterConfig ⇒ def nrOfInstances: Int @@ -729,6 +733,10 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, } } +/** + * The core pieces of the routing logic is located in this + * trait to be able to extend. + */ trait RandomLike { this: RouterConfig ⇒ def nrOfInstances: Int @@ -869,6 +877,10 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin } } +/** + * The core pieces of the routing logic is located in this + * trait to be able to extend. + */ trait SmallestMailboxLike { this: RouterConfig ⇒ def nrOfInstances: Int @@ -1084,6 +1096,10 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N } } +/** + * The core pieces of the routing logic is located in this + * trait to be able to extend. + */ trait BroadcastLike { this: RouterConfig ⇒ def nrOfInstances: Int @@ -1213,6 +1229,10 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It } } +/** + * The core pieces of the routing logic is located in this + * trait to be able to extend. + */ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒ def nrOfInstances: Int diff --git a/akka-docs/scala/code/docs/routing/RouterViaConfigDocSpec.scala b/akka-docs/scala/code/docs/routing/RouterViaConfigDocSpec.scala index 265b0f8361..24dc291087 100644 --- a/akka-docs/scala/code/docs/routing/RouterViaConfigDocSpec.scala +++ b/akka-docs/scala/code/docs/routing/RouterViaConfigDocSpec.scala @@ -77,6 +77,7 @@ object RouterWithConfigDocSpec { /myrouter7 { router = consistent-hashing nr-of-instances = 5 + virtual-nodes-factor = 10 } } //#config-consistent-hashing From 5c20c07199bc35a99098e2a4c43729930a3e631a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 13 Sep 2012 20:44:17 +0200 Subject: [PATCH 09/46] Use tuple to update ConsistentHash and routees together, see #944 --- .../routing/ConsistentHashingRouter.scala | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index e177fc413a..1e78025350 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -164,23 +164,25 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor else virtualNodesFactor - var consistentHashRoutees = new AtomicReference[IndexedSeq[ActorRef]] - @volatile var consistentHash: ConsistentHash[ActorRef] = null + // tuple of routees and the ConsistentHash, updated together in updateConsistentHash + val consistentHashRef = new AtomicReference[(IndexedSeq[ActorRef], ConsistentHash[ActorRef])]((null, null)) updateConsistentHash() // update consistentHash when routees has changed // changes to routees are rare and when no changes this is a quick operation def updateConsistentHash(): ConsistentHash[ActorRef] = { val currentRoutees = routeeProvider.routees - if (currentRoutees ne consistentHashRoutees.get) { - val oldConsistentHashRoutees = consistentHashRoutees.get - val rehash = currentRoutees != oldConsistentHashRoutees - // when other instance, same content, no need to re-hash, but try to set currentRoutees + val oldConsistentHashTuple = consistentHashRef.get + val (oldConsistentHashRoutees, oldConsistentHash) = oldConsistentHashTuple + if (currentRoutees ne oldConsistentHashRoutees) { + // when other instance, same content, no need to re-hash, but try to set routees + val consistentHash = + if (currentRoutees == oldConsistentHashRoutees) oldConsistentHash + else ConsistentHash(currentRoutees, vnodes) // re-hash // ignore, don't update, in case of CAS failure - if (consistentHashRoutees.compareAndSet(oldConsistentHashRoutees, currentRoutees) && rehash) - consistentHash = ConsistentHash(currentRoutees, vnodes) - } - consistentHash + consistentHashRef.compareAndSet(oldConsistentHashTuple, (currentRoutees, consistentHash)) + consistentHash + } else oldConsistentHash } def target(hashData: Any): ActorRef = try { From a515c28e931651c75a6648a568958e84845c1cde Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 13 Sep 2012 22:04:22 +0200 Subject: [PATCH 10/46] Correction, get from ref first, see #944 --- .../src/main/scala/akka/routing/ConsistentHashingRouter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index 1e78025350..b0a9ccb113 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -171,9 +171,9 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ // update consistentHash when routees has changed // changes to routees are rare and when no changes this is a quick operation def updateConsistentHash(): ConsistentHash[ActorRef] = { - val currentRoutees = routeeProvider.routees val oldConsistentHashTuple = consistentHashRef.get val (oldConsistentHashRoutees, oldConsistentHash) = oldConsistentHashTuple + val currentRoutees = routeeProvider.routees if (currentRoutees ne oldConsistentHashRoutees) { // when other instance, same content, no need to re-hash, but try to set routees val consistentHash = From e3bd02b82cbf098f809a81f81a575f77e11c2239 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 14 Sep 2012 11:06:55 +0200 Subject: [PATCH 11/46] Optimized ConsistentHash with array and binarySearch, see #944 * Benchmarked with https://gist.github.com/3720666 * Instead of using TreeMap and rangeImpl this uses an sorted array of hash values of the nodes, and uses binarySearch in java.util.Arrays to find next node clockwise * Benchmarked with https://gist.github.com/3720666 * Benchmark results show improvement from 500 kmsg/s to 700 kmsg/s, round-robin handles 1300 kmsg/s --- .../scala/akka/routing/ConsistentHash.scala | 32 +++++++++++++++---- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index 19e457a762..51f593ad40 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -5,6 +5,7 @@ package akka.routing import scala.collection.immutable.TreeMap +import java.util.Arrays // ============================================================================================= // Adapted from HashRing.scala in Debasish Ghosh's Redis Client, licensed under Apache 2 license @@ -20,19 +21,26 @@ import scala.collection.immutable.TreeMap * hash, i.e. make sure it is different for different nodes. * */ -class ConsistentHash[T] private (nodeRing: TreeMap[Int, T], virtualNodesFactor: Int) { +class ConsistentHash[T] private (nodes: Map[Int, T], virtualNodesFactor: Int) { import ConsistentHash._ if (virtualNodesFactor < 1) throw new IllegalArgumentException("virtualNodesFactor must be >= 1") + // sorted hash values of the nodes + private val nodeRing: Array[Int] = { + val nodeRing = nodes.keys.toArray + Arrays.sort(nodeRing) + nodeRing + } + /** * Adds a node to the node ring. * Note that the instance is immutable and this * operation returns a new instance. */ def :+(node: T): ConsistentHash[T] = - new ConsistentHash(nodeRing ++ ((1 to virtualNodesFactor) map { r ⇒ (nodeHashFor(node, r) -> node) }), virtualNodesFactor) + new ConsistentHash(nodes ++ ((1 to virtualNodesFactor) map { r ⇒ (nodeHashFor(node, r) -> node) }), virtualNodesFactor) /** * Adds a node to the node ring. @@ -48,7 +56,7 @@ class ConsistentHash[T] private (nodeRing: TreeMap[Int, T], virtualNodesFactor: * operation returns a new instance. */ def :-(node: T): ConsistentHash[T] = - new ConsistentHash(nodeRing -- ((1 to virtualNodesFactor) map { r ⇒ nodeHashFor(node, r) }), virtualNodesFactor) + new ConsistentHash(nodes -- ((1 to virtualNodesFactor) map { r ⇒ nodeHashFor(node, r) }), virtualNodesFactor) /** * Removes a node from the node ring. @@ -65,15 +73,25 @@ class ConsistentHash[T] private (nodeRing: TreeMap[Int, T], virtualNodesFactor: */ def nodeFor(key: Array[Byte]): T = { if (isEmpty) throw new IllegalStateException("Can't get node for [%s] from an empty node ring" format key) - val hash = hashFor(key) - // find the next node clockwise in the nodeRing, pick first if end of tree - nodeRing.rangeImpl(Some(hash), None).headOption.getOrElse(nodeRing.head)._2 + + // converts the result of Arrays.binarySearch into a index in the nodeRing array + // see documentation of Arrays.binarySearch for what it returns + def idx(i: Int): Int = { + if (i >= 0) i // exact match + else { + val j = math.abs(i + 1) + if (j >= nodeRing.length) 0 // after last, use first + else j // next node clockwise + } + } + val nodeRingIndex = idx(Arrays.binarySearch(nodeRing, hashFor(key))) + nodes(nodeRing(nodeRingIndex)) } /** * Is the node ring empty, i.e. no nodes added or all removed. */ - def isEmpty: Boolean = nodeRing.isEmpty + def isEmpty: Boolean = nodes.isEmpty } From a4dd6b754754fbdd61e1fcfa76497664380985df Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 14 Sep 2012 13:47:58 +0200 Subject: [PATCH 12/46] Support partial function to map message to hash key, see #944 * Partial function, ConsistentHashRoute, for Scala API * withConsistentHashMapping ConsistentHashMapping for Java API * Updated documentation --- .../routing/ConsistentHashingRouterSpec.scala | 31 ++++- .../src/test/scala/perf/RouterPerf.scala | 79 +++++++++++++ .../routing/ConsistentHashingRouter.scala | 106 +++++++++++++++--- akka-docs/java/routing.rst | 23 +++- .../ConsistentHashingRouterDocSpec.scala | 3 + akka-docs/scala/routing.rst | 22 +++- 6 files changed, 235 insertions(+), 29 deletions(-) create mode 100644 akka-actor-tests/src/test/scala/perf/RouterPerf.scala diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala index 7a28190523..b4ef639067 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala @@ -24,6 +24,10 @@ object ConsistentHashingRouterSpec { nr-of-instances = 3 virtual-nodes-factor = 17 } + /router2 { + router = consistent-hashing + nr-of-instances = 5 + } } """ @@ -38,6 +42,8 @@ object ConsistentHashingRouterSpec { } case class MsgKey(name: String) + + case class Msg2(key: Any, data: String) } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @@ -66,7 +72,30 @@ class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.c router1 ! Msg(MsgKey("c"), "C") val destinationC = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } - router1 ! Msg(MsgKey("c"), "CC") + router1 ! ConsistentHashableEnvelope(message = "CC", consistentHashKey = MsgKey("c")) + expectMsg(destinationC) + } + + "select destination with defined consistentHashRoute" in { + def consistentHashRoute: ConsistentHashingRouter.ConsistentHashRoute = { + case Msg2(key, data) ⇒ key + } + val router2 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter( + consistentHashRoute = consistentHashRoute)), "router2") + + router2 ! Msg2("a", "A") + val destinationA = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + router2 ! ConsistentHashableEnvelope(message = "AA", consistentHashKey = "a") + expectMsg(destinationA) + + router2 ! Msg2(17, "B") + val destinationB = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + router2 ! ConsistentHashableEnvelope(message = "BB", consistentHashKey = 17) + expectMsg(destinationB) + + router2 ! Msg2(MsgKey("c"), "C") + val destinationC = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + router2 ! ConsistentHashableEnvelope(message = "CC", consistentHashKey = MsgKey("c")) expectMsg(destinationC) } } diff --git a/akka-actor-tests/src/test/scala/perf/RouterPerf.scala b/akka-actor-tests/src/test/scala/perf/RouterPerf.scala new file mode 100644 index 0000000000..20f9554540 --- /dev/null +++ b/akka-actor-tests/src/test/scala/perf/RouterPerf.scala @@ -0,0 +1,79 @@ +package perf + +import akka.actor.ActorSystem +import akka.actor.Actor +import akka.actor.Props +import scala.concurrent.forkjoin.ThreadLocalRandom +import akka.routing.ConsistentHashingRouter +import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope +import scala.util.Random +import akka.routing.Broadcast +import akka.actor.ActorLogging +import scala.concurrent.util.Duration +import akka.routing.RoundRobinRouter + +object RouterPerf extends App { + val system = ActorSystem("PerfApp") + var perf = new RouterPerf(system) + // Thread.sleep(20000) // hook up profiler here + perf.run() +} + +class RouterPerf(system: ActorSystem) { + + def run(): Unit = { + // nbrMessages = 10000000 + val sender = system.actorOf(Props(new Sender( + nbrMessages = 10000000, nbrRoutees = 10, nbrIterations = 10)), name = "sender") + sender ! "start" + } + +} + +class Sender(nbrMessages: Int, nbrRoutees: Int, nbrIterations: Int) extends Actor with ActorLogging { + val router = context.actorOf(Props[Destination].withRouter(ConsistentHashingRouter(nbrRoutees, + virtualNodesFactor = 10)), "router") + // val router = context.actorOf(Props[Destination].withRouter(RoundRobinRouter(nbrRoutees)), "router") + val rnd = new Random + val messages = Vector.fill(1000)(ConsistentHashableEnvelope("msg", rnd.nextString(10))) + var startTime = 0L + var doneCounter = 0 + var iterationCounter = 0 + + def receive = { + case "start" ⇒ + iterationCounter += 1 + doneCounter = 0 + startTime = System.nanoTime + val messgesSize = messages.size + for (n ← 1 to nbrMessages) { router ! messages(n % messgesSize) } + router ! Broadcast("done") + + case "done" ⇒ + doneCounter += 1 + if (doneCounter == nbrRoutees) { + val duration = Duration.fromNanos(System.nanoTime - startTime) + val mps = (nbrMessages.toDouble * 1000 / duration.toMillis).toInt + // log.info("Processed [{}] messages in [{} millis], i.e. [{}] msg/s", + // nbrMessages, duration.toMillis, mps) + println("Processed [%s] messages in [%s millis], i.e. [%s] msg/s".format( + nbrMessages, duration.toMillis, mps)) + if (iterationCounter < nbrIterations) + self ! "start" + else + context.system.shutdown() + } + } +} + +class Destination extends Actor with ActorLogging { + var count = 0 + def receive = { + case "done" ⇒ + log.info("Handled [{}] messages", count) + count = 0 + sender ! "done" + case msg ⇒ count += 1 + + } +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index b0a9ccb113..a08bbdd481 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -29,23 +29,29 @@ object ConsistentHashingRouter { } /** - * Messages need to implement this interface to define what + * If you don't define the consistentHashRoute when + * constructing the [[akka.routing.ConsistentHashingRouter]] + * the messages need to implement this interface to define what * data to use for the consistent hash key. Note that it's not - * the hash, but the data to be hashed. If returning an - * `Array[Byte]` or String it will be used as is, otherwise the - * configured [[akka.akka.serialization.Serializer]] will - * be applied to the returned data. + * the hash, but the data to be hashed. + * + * If returning an `Array[Byte]` or String it will be used as is, + * otherwise the configured [[akka.akka.serialization.Serializer]] + * will be applied to the returned data. * * If messages can't implement this interface themselves, * it's possible to wrap the messages in - * [[akka.routing.ConsistentHashableEnvelope]] + * [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]], + * or use [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]] */ trait ConsistentHashable { def consistentHashKey: Any } /** - * If messages can't implement [[akka.routing.ConsistentHashable]] + * If you don't define the consistentHashRoute when + * constructing the [[akka.routing.ConsistentHashingRouter]] + * and messages can't implement [[akka.routing.ConsistentHashingRouter.ConsistentHashable]] * themselves they can we wrapped by this envelope instead. The * router will only send the wrapped message to the destination, * i.e. the envelope will be stripped off. @@ -54,12 +60,57 @@ object ConsistentHashingRouter { case class ConsistentHashableEnvelope(message: Any, consistentHashKey: Any) extends ConsistentHashable with RouterEnvelope + /** + * Partial function from message to the data to + * use for the consistent hash key. Note that it's not + * the hash that is to be returned, but the data to be hashed. + * + * If returning an `Array[Byte]` or String it will be used as is, + * otherwise the configured [[akka.akka.serialization.Serializer]] + * will be applied to the returned data. + */ + type ConsistentHashRoute = PartialFunction[Any, Any] + + @SerialVersionUID(1L) + object emptyConsistentHashRoute extends ConsistentHashRoute { + def isDefinedAt(x: Any) = false + def apply(x: Any) = throw new UnsupportedOperationException("Empty ConsistentHashRoute apply()") + } + + /** + * JAVA API + * Mapping from message to the data to use for the consistent hash key. + * Note that it's not the hash that is to be returned, but the data to be + * hashed. + * + * If returning an `Array[Byte]` or String it will be used as is, + * otherwise the configured [[akka.akka.serialization.Serializer]] + * will be applied to the returned data. + */ + trait ConsistentHashMapping { + def consistentHashKey(message: Any): Any + } } /** * A Router that uses consistent hashing to select a connection based on the - * sent message. The messages must implement [[akka.routing.ConsistentHashable]] - * or be wrapped in a [[akka.routing.ConsistentHashableEnvelope]] to define what - * data to use for the consistent hash key. + * sent message. + * + * There is 3 ways to define what data to use for the consistent hash key. + * + * 1. You can define `consistentHashRoute` / `withConsistentHashMapping` + * of the router to map incoming messages to their consistent hash key. + * This makes the makes the decision transparent for the sender. + * + * 2. The messages may implement [[akka.routing.ConsistentHashable]]. + * The key is part of the message and it's convenient to define it together + * with the message definition. + * + * 3. The messages can be be wrapped in a [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]] + * to define what data to use for the consistent hash key. The sender knows + * the key to use. + * + * These ways to define the consistent hash key can be use together and at + * the same time for one router. The `consistentHashRoute` is tried first. * * Please note that providing both 'nrOfInstances' and 'routees' does not make logical * sense as this means that the router should both create new actors and use the 'routees' @@ -83,13 +134,16 @@ object ConsistentHashingRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] * @param virtualNodesFactor number of virtual nodes per node, used in [[akka.routing.ConsistantHash]] + * @param consistentHashRoute partial function from message to the data to + * use for the consistent hash key */ @SerialVersionUID(1L) case class ConsistentHashingRouter( nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy, - val virtualNodesFactor: Int = 0) + val virtualNodesFactor: Int = 0, + val consistentHashRoute: ConsistentHashingRouter.ConsistentHashRoute = ConsistentHashingRouter.emptyConsistentHashRoute) extends RouterConfig with ConsistentHashingLike { /** @@ -128,14 +182,29 @@ case class ConsistentHashingRouter( */ def withVirtualNodesFactor(vnodes: Int): ConsistentHashingRouter = copy(virtualNodesFactor = vnodes) + /** + * Java API for setting the mapping from message to the data to use for the consistent hash key + */ + def withConsistentHashMapping(mapping: ConsistentHashingRouter.ConsistentHashMapping) = { + copy(consistentHashRoute = { + case message ⇒ mapping.consistentHashKey(message) + }) + } + /** * Uses the resizer of the given RouterConfig if this RouterConfig * doesn't have one, i.e. the resizer defined in code is used if * resizer was not defined in config. + * Uses the the consistentHashRoute defined in code, since + * that can't be defined in configuration. */ - override def withFallback(other: RouterConfig): RouterConfig = { - if (this.resizer.isEmpty && other.resizer.isDefined) copy(resizer = other.resizer) - else this + override def withFallback(other: RouterConfig): RouterConfig = other match { + case otherRouter: ConsistentHashingRouter ⇒ + val useResizer = + if (this.resizer.isEmpty && otherRouter.resizer.isDefined) otherRouter.resizer + else this.resizer + copy(resizer = useResizer, consistentHashRoute = otherRouter.consistentHashRoute) + case _ ⇒ throw new IllegalArgumentException("Expected ConsistentHashingRouter, got [%s]".format(other)) } } @@ -153,6 +222,8 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ def virtualNodesFactor: Int + def consistentHashRoute: ConsistentHashRoute + override def createRoute(routeeProvider: RouteeProvider): Route = { if (resizer.isEmpty) { if (routees.isEmpty) routeeProvider.createRoutees(nrOfInstances) @@ -204,14 +275,17 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ { case (sender, message) ⇒ message match { - case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees) + case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees) + case _ if consistentHashRoute.isDefinedAt(message) ⇒ + List(Destination(sender, target(consistentHashRoute(message)))) case hashable: ConsistentHashable ⇒ List(Destination(sender, target(hashable.consistentHashKey))) case other ⇒ - log.warning("Message [{}] must implement [{}] or be wrapped in [{}]", + log.warning("Message [{}] must be handled by consistentHashRoute, or implement [{}] or be wrapped in [{}]", message.getClass.getName, classOf[ConsistentHashable].getName, classOf[ConsistentHashableEnvelope].getName) List(Destination(sender, routeeProvider.context.system.deadLetters)) } + } } } \ No newline at end of file diff --git a/akka-docs/java/routing.rst b/akka-docs/java/routing.rst index 561cbd9446..0096c9e3e8 100644 --- a/akka-docs/java/routing.rst +++ b/akka-docs/java/routing.rst @@ -15,6 +15,7 @@ is really easy to create your own. The routers shipped with Akka are: * ``akka.routing.SmallestMailboxRouter`` * ``akka.routing.BroadcastRouter`` * ``akka.routing.ScatterGatherFirstCompletedRouter`` +* ``akka.routing.ConsistentHashingRouter`` Routers In Action ^^^^^^^^^^^^^^^^^ @@ -283,12 +284,22 @@ to select a connection based on the sent message. This `article `_ gives good insight into how consistent hashing is implemented. -The messages sent to a ConsistentHashingRouter must implement -``akka.routing.ConsistentHashable`` or be wrapped in a ``akka.routing.ConsistentHashableEnvelope`` -to define what data to use for the consistent hash key. If returning a -byte array or String it will be used as is, otherwise the configured -:ref:`serializer ` will be applied to the returned data -to create a byte array that will be hashed. +There is 3 ways to define what data to use for the consistent hash key. + +* You can define ``withConsistentHashMapping`` of the router to map incoming + messages to their consistent hash key. This makes the makes the decision + transparent for the sender. + +* The messages may implement ``akka.routing.ConsistentHashable``. + The key is part of the message and it's convenient to define it together + with the message definition. + +* The messages can be be wrapped in a ``akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope`` + to define what data to use for the consistent hash key. The sender knows + the key to use. + +These ways to define the consistent hash key can be use together and at +the same time for one router. The ``withConsistentHashMapping`` is tried first. Code example: diff --git a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala index 8bb59a7bce..42af05c049 100644 --- a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala +++ b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala @@ -19,9 +19,12 @@ object ConsistentHashingRouterDocSpec { def receive = { case Entry(key, value) ⇒ cache += (key -> value) case Get(key) ⇒ sender ! cache.get(key) + case Evict(key) => cache -= key } } + case class Evict(key: String) + case class Get(key: String) extends ConsistentHashable { override def consistentHashKey: Any = key } diff --git a/akka-docs/scala/routing.rst b/akka-docs/scala/routing.rst index 3f384895c6..0cffa3c1a7 100644 --- a/akka-docs/scala/routing.rst +++ b/akka-docs/scala/routing.rst @@ -289,12 +289,22 @@ to select a connection based on the sent message. This `article `_ gives good insight into how consistent hashing is implemented. -The messages sent to a ConsistentHashingRouter must implement -``akka.routing.ConsistentHashable`` or be wrapped in a ``akka.routing.ConsistentHashableEnvelope`` -to define what data to use for the consistent hash key. If returning a -byte array or String it will be used as is, otherwise the configured -:ref:`serializer ` will be applied to the returned data -to create a byte array that will be hashed. +There is 3 ways to define what data to use for the consistent hash key. + +* You can define ``consistentHashRoute`` of the router to map incoming + messages to their consistent hash key. This makes the makes the decision + transparent for the sender. + +* The messages may implement ``akka.routing.ConsistentHashable``. + The key is part of the message and it's convenient to define it together + with the message definition. + +* The messages can be be wrapped in a ``akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope`` + to define what data to use for the consistent hash key. The sender knows + the key to use. + +These ways to define the consistent hash key can be use together and at +the same time for one router. The ``consistentHashRoute`` is tried first. Code example: From 48d8a09075fd3dd000882fb4692e28e3e4b85839 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 14 Sep 2012 13:53:04 +0200 Subject: [PATCH 13/46] Remove note about redis client, completely different impl, see #944 --- akka-actor/src/main/scala/akka/routing/ConsistentHash.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index 51f593ad40..bc86f5f82d 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -7,12 +7,8 @@ package akka.routing import scala.collection.immutable.TreeMap import java.util.Arrays -// ============================================================================================= -// Adapted from HashRing.scala in Debasish Ghosh's Redis Client, licensed under Apache 2 license -// ============================================================================================= - /** - * Consistent Hashing node ring abstraction. + * Consistent Hashing node ring implementation. * * A good explanation of Consistent Hashing: * http://weblogs.java.net/blog/tomwhite/archive/2007/11/consistent_hash.html From 50fc5a03a2adc35ca0bd3d20e5db1b68decd50d1 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 14 Sep 2012 14:28:47 +0200 Subject: [PATCH 14/46] Complete unfinished ConsistentHashingRouterDocSpec, see #944 --- akka-docs/java/routing.rst | 3 +- .../ConsistentHashingRouterDocSpec.scala | 29 +++++++++++++------ akka-docs/scala/routing.rst | 3 +- 3 files changed, 24 insertions(+), 11 deletions(-) diff --git a/akka-docs/java/routing.rst b/akka-docs/java/routing.rst index 0096c9e3e8..dad68ecf30 100644 --- a/akka-docs/java/routing.rst +++ b/akka-docs/java/routing.rst @@ -306,7 +306,8 @@ Code example: FIXME Java example of consistent routing In the above example you see that the ``Get`` message implements ``ConsistentHashable`` itself, -while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. +while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. The ``Evict`` +message is handled by the ``withConsistentHashMapping``. This is an example of how to define a consistent-hashing router in configuration: diff --git a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala index 42af05c049..89a329c743 100644 --- a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala +++ b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala @@ -19,7 +19,7 @@ object ConsistentHashingRouterDocSpec { def receive = { case Entry(key, value) ⇒ cache += (key -> value) case Get(key) ⇒ sender ! cache.get(key) - case Evict(key) => cache -= key + case Evict(key) ⇒ cache -= key } } @@ -30,10 +30,6 @@ object ConsistentHashingRouterDocSpec { } case class Entry(key: String, value: String) - case class EntryEnvelope(entry: Entry) extends ConsistentHashableEnvelope { - override def consistentHashKey: Any = entry.key - override def message: Any = entry - } //#cache-actor } @@ -47,16 +43,31 @@ class ConsistentHashingRouterDocSpec extends AkkaSpec with ImplicitSender { //#consistent-hashing-router import akka.actor.Props import akka.routing.ConsistentHashingRouter + import akka.routing.ConsistentHashingRouter.ConsistentHashRoute + import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope - val cache = system.actorOf(Props[Cache].withRouter(ConsistentHashingRouter(10)), "cache") + def consistentHashRoute: ConsistentHashingRouter.ConsistentHashRoute = { + case Evict(key) ⇒ key + } - cache ! EntryEnvelope(Entry("hello", "HELLO")) - cache ! EntryEnvelope(Entry("hi", "HI")) + val cache = system.actorOf(Props[Cache].withRouter(ConsistentHashingRouter(10, + consistentHashRoute = consistentHashRoute)), name = "cache") + + cache ! ConsistentHashableEnvelope( + message = Entry("hello", "HELLO"), consistentHashKey = "hello") + cache ! ConsistentHashableEnvelope( + message = Entry("hi", "HI"), consistentHashKey = "hi") cache ! Get("hello") - cache ! Get("hi") expectMsg(Some("HELLO")) + + cache ! Get("hi") expectMsg(Some("HI")) + + cache ! Evict("hi") + cache ! Get("hi") + expectMsg(None) + //#consistent-hashing-router } diff --git a/akka-docs/scala/routing.rst b/akka-docs/scala/routing.rst index 0cffa3c1a7..e10a353988 100644 --- a/akka-docs/scala/routing.rst +++ b/akka-docs/scala/routing.rst @@ -313,7 +313,8 @@ Code example: .. includecode:: code/docs/routing/ConsistentHashingRouterDocSpec.scala#consistent-hashing-router In the above example you see that the ``Get`` message implements ``ConsistentHashable`` itself, -while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. +while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. The ``Evict`` +message is handled by the ``consistentHashRoute`` partial function. This is an example of how to define a consistent-hashing router in configuration: From da3f08cf41dc280267717892ef0ad94bc3f8174d Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 17 Sep 2012 08:42:47 +0200 Subject: [PATCH 15/46] Simplify selectDeploymentTarget * Break the loop immediately when no more targets --- .../cluster/routing/ClusterRouterConfig.scala | 43 ++++++++++++------- 1 file changed, 28 insertions(+), 15 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala index e2d8b8b204..c99ae68704 100644 --- a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala +++ b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala @@ -32,6 +32,7 @@ import akka.routing.RemoteRouterConfig import akka.actor.RootActorPath import akka.actor.ActorCell import akka.actor.RelativeActorPath +import scala.annotation.tailrec /** * [[akka.routing.RouterConfig]] implementation for deployment on cluster nodes. @@ -168,18 +169,26 @@ private[akka] class ClusterRouteeProvider( * to use for cluster routers. */ override def createRoutees(nrOfInstances: Int): Unit = { - for (i ← 1 to settings.totalInstances; target ← selectDeploymentTarget) { - val ref = - if (settings.isRouteesPathDefined) { - context.actorFor(RootActorPath(target) / settings.routeesPathElements) - } else { - val name = "c" + childNameCounter.incrementAndGet - val deploy = Deploy("", ConfigFactory.empty(), routeeProps.routerConfig, RemoteScope(target)) - context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false) - } - // must register each one, since registered routees are used in selectDeploymentTarget - registerRoutees(Some(ref)) + @tailrec + def doCreateRoutees(): Unit = selectDeploymentTarget match { + case None ⇒ // done + case Some(target) ⇒ + val ref = + if (settings.isRouteesPathDefined) { + context.actorFor(RootActorPath(target) / settings.routeesPathElements) + } else { + val name = "c" + childNameCounter.incrementAndGet + val deploy = Deploy("", ConfigFactory.empty(), routeeProps.routerConfig, RemoteScope(target)) + context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false) + } + // must register each one, since registered routees are used in selectDeploymentTarget + registerRoutees(Some(ref)) + + // recursion until all created + doCreateRoutees() } + + doCreateRoutees() } private[routing] def createRoutees(): Unit = createRoutees(settings.totalInstances) @@ -187,14 +196,18 @@ private[akka] class ClusterRouteeProvider( private def selectDeploymentTarget: Option[Address] = { val currentRoutees = routees val currentNodes = availbleNodes - if (currentRoutees.size >= settings.totalInstances || currentNodes.isEmpty) { + if (currentNodes.isEmpty || currentRoutees.size >= settings.totalInstances) { None } else { + // find the node with least routees val numberOfRouteesPerNode: Map[Address, Int] = - Map.empty[Address, Int] ++ currentNodes.toSeq.map(_ -> 0) ++ - currentRoutees.groupBy(fullAddress).map { - case (address, refs) ⇒ address -> refs.size + currentRoutees.foldLeft(currentNodes.map(_ -> 0).toMap) { (acc, x) ⇒ + val address = fullAddress(x) + acc.get(address) match { + case Some(count) ⇒ acc + (address -> (count + 1)) + case None ⇒ acc + (address -> 1) } + } val (address, count) = numberOfRouteesPerNode.minBy(_._2) if (count < settings.maxInstancesPerNode) Some(address) else None From 08dbc4d23500a0c0efff816bcf1ffa04dd1b758f Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 17 Sep 2012 09:30:52 +0200 Subject: [PATCH 16/46] Incorporate minor review feedback, see #944 --- .../akka/routing/ConsistentHashingRouterSpec.scala | 12 ++++++------ .../routing/ConsistentHashingRouterDocSpec.scala | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala index b4ef639067..5ac3ea5566 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala @@ -61,17 +61,17 @@ class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.c "select destination based on consistentHashKey of the message" in { router1 ! Msg("a", "A") - val destinationA = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + val destinationA = expectMsgType[ActorRef] router1 ! ConsistentHashableEnvelope(message = "AA", consistentHashKey = "a") expectMsg(destinationA) router1 ! Msg(17, "B") - val destinationB = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + val destinationB = expectMsgType[ActorRef] router1 ! ConsistentHashableEnvelope(message = "BB", consistentHashKey = 17) expectMsg(destinationB) router1 ! Msg(MsgKey("c"), "C") - val destinationC = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + val destinationC = expectMsgType[ActorRef] router1 ! ConsistentHashableEnvelope(message = "CC", consistentHashKey = MsgKey("c")) expectMsg(destinationC) } @@ -84,17 +84,17 @@ class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.c consistentHashRoute = consistentHashRoute)), "router2") router2 ! Msg2("a", "A") - val destinationA = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + val destinationA = expectMsgType[ActorRef] router2 ! ConsistentHashableEnvelope(message = "AA", consistentHashKey = "a") expectMsg(destinationA) router2 ! Msg2(17, "B") - val destinationB = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + val destinationB = expectMsgType[ActorRef] router2 ! ConsistentHashableEnvelope(message = "BB", consistentHashKey = 17) expectMsg(destinationB) router2 ! Msg2(MsgKey("c"), "C") - val destinationC = expectMsgPF(remaining) { case ref: ActorRef ⇒ ref } + val destinationC = expectMsgType[ActorRef] router2 ! ConsistentHashableEnvelope(message = "CC", consistentHashKey = MsgKey("c")) expectMsg(destinationC) } diff --git a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala index 89a329c743..945a78e1eb 100644 --- a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala +++ b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala @@ -46,7 +46,7 @@ class ConsistentHashingRouterDocSpec extends AkkaSpec with ImplicitSender { import akka.routing.ConsistentHashingRouter.ConsistentHashRoute import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope - def consistentHashRoute: ConsistentHashingRouter.ConsistentHashRoute = { + def consistentHashRoute: ConsistentHashRoute = { case Evict(key) ⇒ key } From d59612609fec5e8e20dc40ce360b49dd0088d90a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 17 Sep 2012 09:39:35 +0200 Subject: [PATCH 17/46] Handle FromConfig in withFallback also, see #944 --- .../test/scala/akka/routing/ConsistentHashingRouterSpec.scala | 2 +- .../src/main/scala/akka/routing/ConsistentHashingRouter.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala index 5ac3ea5566..0024ceea44 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala @@ -51,7 +51,7 @@ class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.c import akka.routing.ConsistentHashingRouterSpec._ implicit val ec = system.dispatcher - val router1 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter()), "router1") + val router1 = system.actorOf(Props[Echo].withRouter(FromConfig()), "router1") "consistent hashing router" must { "create routees from configuration" in { diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index a08bbdd481..d8d384c5c0 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -199,6 +199,7 @@ case class ConsistentHashingRouter( * that can't be defined in configuration. */ override def withFallback(other: RouterConfig): RouterConfig = other match { + case fromConfig: FromConfig ⇒ this case otherRouter: ConsistentHashingRouter ⇒ val useResizer = if (this.resizer.isEmpty && otherRouter.resizer.isDefined) otherRouter.resizer From 48a0fda9915802d5516e476c489b01edc840a0a1 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 17 Sep 2012 11:40:06 +0200 Subject: [PATCH 18/46] Java doc sample for consistent hashing router, see #944 * Allow return null as undefined ConsistentHashMapping --- .../routing/ConsistentHashingRouter.scala | 14 +- .../ConsistentHashingRouterDocTest.scala | 5 + .../ConsistentHashingRouterDocTestBase.java | 136 ++++++++++++++++++ akka-docs/java/routing.rst | 10 +- .../ConsistentHashingRouterDocSpec.scala | 1 - akka-docs/scala/routing.rst | 4 +- 6 files changed, 159 insertions(+), 11 deletions(-) create mode 100644 akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTest.scala create mode 100644 akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTestBase.java diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index d8d384c5c0..9650a038e9 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -83,6 +83,9 @@ object ConsistentHashingRouter { * Note that it's not the hash that is to be returned, but the data to be * hashed. * + * May return `null` to indicate that the message is not handled by + * this mapping. + * * If returning an `Array[Byte]` or String it will be used as is, * otherwise the configured [[akka.akka.serialization.Serializer]] * will be applied to the returned data. @@ -99,9 +102,9 @@ object ConsistentHashingRouter { * * 1. You can define `consistentHashRoute` / `withConsistentHashMapping` * of the router to map incoming messages to their consistent hash key. - * This makes the makes the decision transparent for the sender. + * This makes the decision transparent for the sender. * - * 2. The messages may implement [[akka.routing.ConsistentHashable]]. + * 2. The messages may implement [[akka.routing.ConsistentHashingRouter.ConsistentHashable]]. * The key is part of the message and it's convenient to define it together * with the message definition. * @@ -183,11 +186,12 @@ case class ConsistentHashingRouter( def withVirtualNodesFactor(vnodes: Int): ConsistentHashingRouter = copy(virtualNodesFactor = vnodes) /** - * Java API for setting the mapping from message to the data to use for the consistent hash key + * Java API for setting the mapping from message to the data to use for the consistent hash key. */ def withConsistentHashMapping(mapping: ConsistentHashingRouter.ConsistentHashMapping) = { copy(consistentHashRoute = { - case message ⇒ mapping.consistentHashKey(message) + case message if (mapping.consistentHashKey(message).asInstanceOf[AnyRef] ne null) ⇒ + mapping.consistentHashKey(message) }) } @@ -199,7 +203,7 @@ case class ConsistentHashingRouter( * that can't be defined in configuration. */ override def withFallback(other: RouterConfig): RouterConfig = other match { - case fromConfig: FromConfig ⇒ this + case _: FromConfig ⇒ this case otherRouter: ConsistentHashingRouter ⇒ val useResizer = if (this.resizer.isEmpty && otherRouter.resizer.isDefined) otherRouter.resizer diff --git a/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTest.scala b/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTest.scala new file mode 100644 index 0000000000..a043f9cb73 --- /dev/null +++ b/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTest.scala @@ -0,0 +1,5 @@ +package docs.jrouting; + +import org.scalatest.junit.JUnitSuite + +class ConsistentHashingRouterDocTest extends ConsistentHashingRouterDocTestBase with JUnitSuite diff --git a/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTestBase.java b/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTestBase.java new file mode 100644 index 0000000000..6f00ed1509 --- /dev/null +++ b/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTestBase.java @@ -0,0 +1,136 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package docs.jrouting; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import akka.testkit.JavaTestKit; +import akka.actor.ActorSystem; + +//#imports1 +import akka.actor.UntypedActor; +import akka.routing.ConsistentHashingRouter.ConsistentHashable; +import java.util.Map; +import java.util.HashMap; +import java.io.Serializable; +//#imports1 + +//#imports2 +import akka.actor.Props; +import akka.actor.ActorRef; +import akka.routing.ConsistentHashingRouter; +import akka.routing.ConsistentHashingRouter.ConsistentHashMapping; +import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope; +//#imports2 + +public class ConsistentHashingRouterDocTestBase { + + static ActorSystem system; + + @BeforeClass + public static void setup() { + system = ActorSystem.create(); + } + + @AfterClass + public static void teardown() { + system.shutdown(); + } + + //#cache-actor + + public static class Cache extends UntypedActor { + Map cache = new HashMap(); + + public void onReceive(Object msg) { + if (msg instanceof Entry) { + Entry entry = (Entry) msg; + cache.put(entry.key, entry.value); + } else if (msg instanceof Get) { + Get get = (Get) msg; + Object value = cache.get(get.key); + getSender().tell(value == null ? NOT_FOUND : value, + getContext().self()); + } else if (msg instanceof Evict) { + Evict evict = (Evict) msg; + cache.remove(evict.key); + } else { + unhandled(msg); + } + } + } + + public static class Evict implements Serializable { + public final String key; + public Evict(String key) { + this.key = key; + } + } + + public static class Get implements Serializable, ConsistentHashable { + public final String key; + public Get(String key) { + this.key = key; + } + public Object consistentHashKey() { + return key; + } + } + + public static class Entry implements Serializable { + public final String key; + public final String value; + public Entry(String key, String value) { + this.key = key; + this.value = value; + } + } + + public static final String NOT_FOUND = "NOT_FOUND"; + //#cache-actor + + + @Test + public void demonstrateUsageOfConsistentHashableRouter() { + + new JavaTestKit(system) {{ + + //#consistent-hashing-router + + final ConsistentHashMapping consistentHashMapping = new ConsistentHashMapping() { + @Override + public Object consistentHashKey(Object message) { + if (message instanceof Evict) { + return ((Evict) message).key; + } else { + return null; + } + } + }; + + ActorRef cache = system.actorOf(new Props(Cache.class).withRouter( + new ConsistentHashingRouter(10).withConsistentHashMapping(consistentHashMapping)), + "cache"); + + cache.tell(new ConsistentHashableEnvelope( + new Entry("hello", "HELLO"), "hello"), getRef()); + cache.tell(new ConsistentHashableEnvelope( + new Entry("hi", "HI"), "hi"), getRef()); + + cache.tell(new Get("hello"), getRef()); + expectMsgEquals("HELLO"); + + cache.tell(new Get("hi"), getRef()); + expectMsgEquals("HI"); + + cache.tell(new Evict("hi"), getRef()); + cache.tell(new Get("hi"), getRef()); + expectMsgEquals(NOT_FOUND); + + //#consistent-hashing-router + }}; + } + +} diff --git a/akka-docs/java/routing.rst b/akka-docs/java/routing.rst index dad68ecf30..149a3a315b 100644 --- a/akka-docs/java/routing.rst +++ b/akka-docs/java/routing.rst @@ -287,10 +287,10 @@ insight into how consistent hashing is implemented. There is 3 ways to define what data to use for the consistent hash key. * You can define ``withConsistentHashMapping`` of the router to map incoming - messages to their consistent hash key. This makes the makes the decision + messages to their consistent hash key. This makes the the decision transparent for the sender. -* The messages may implement ``akka.routing.ConsistentHashable``. +* The messages may implement ``akka.routing.ConsistentHashingRouter.ConsistentHashable``. The key is part of the message and it's convenient to define it together with the message definition. @@ -303,7 +303,11 @@ the same time for one router. The ``withConsistentHashMapping`` is tried first. Code example: -FIXME Java example of consistent routing +.. includecode:: code/docs/jrouting/ConsistentHashingRouterDocTestBase.java + :include: imports1,cache-actor + +.. includecode:: code/docs/jrouting/ConsistentHashingRouterDocTestBase.java + :include: imports2,consistent-hashing-router In the above example you see that the ``Get`` message implements ``ConsistentHashable`` itself, while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. The ``Evict`` diff --git a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala index 945a78e1eb..7041db6049 100644 --- a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala +++ b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala @@ -11,7 +11,6 @@ object ConsistentHashingRouterDocSpec { //#cache-actor import akka.actor.Actor import akka.routing.ConsistentHashingRouter.ConsistentHashable - import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope class Cache extends Actor { var cache = Map.empty[String, String] diff --git a/akka-docs/scala/routing.rst b/akka-docs/scala/routing.rst index e10a353988..4766b2e6ca 100644 --- a/akka-docs/scala/routing.rst +++ b/akka-docs/scala/routing.rst @@ -292,10 +292,10 @@ insight into how consistent hashing is implemented. There is 3 ways to define what data to use for the consistent hash key. * You can define ``consistentHashRoute`` of the router to map incoming - messages to their consistent hash key. This makes the makes the decision + messages to their consistent hash key. This makes the decision transparent for the sender. -* The messages may implement ``akka.routing.ConsistentHashable``. +* The messages may implement ``akka.routing.ConsistentHashingRouter.ConsistentHashable``. The key is part of the message and it's convenient to define it together with the message definition. From b4f59483400bfeb9d515b6557b23628ae53bd8a5 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 17 Sep 2012 12:12:19 +0200 Subject: [PATCH 19/46] Remove router benchmark, see #944 --- .../src/test/scala/perf/RouterPerf.scala | 79 ------------------- 1 file changed, 79 deletions(-) delete mode 100644 akka-actor-tests/src/test/scala/perf/RouterPerf.scala diff --git a/akka-actor-tests/src/test/scala/perf/RouterPerf.scala b/akka-actor-tests/src/test/scala/perf/RouterPerf.scala deleted file mode 100644 index 20f9554540..0000000000 --- a/akka-actor-tests/src/test/scala/perf/RouterPerf.scala +++ /dev/null @@ -1,79 +0,0 @@ -package perf - -import akka.actor.ActorSystem -import akka.actor.Actor -import akka.actor.Props -import scala.concurrent.forkjoin.ThreadLocalRandom -import akka.routing.ConsistentHashingRouter -import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope -import scala.util.Random -import akka.routing.Broadcast -import akka.actor.ActorLogging -import scala.concurrent.util.Duration -import akka.routing.RoundRobinRouter - -object RouterPerf extends App { - val system = ActorSystem("PerfApp") - var perf = new RouterPerf(system) - // Thread.sleep(20000) // hook up profiler here - perf.run() -} - -class RouterPerf(system: ActorSystem) { - - def run(): Unit = { - // nbrMessages = 10000000 - val sender = system.actorOf(Props(new Sender( - nbrMessages = 10000000, nbrRoutees = 10, nbrIterations = 10)), name = "sender") - sender ! "start" - } - -} - -class Sender(nbrMessages: Int, nbrRoutees: Int, nbrIterations: Int) extends Actor with ActorLogging { - val router = context.actorOf(Props[Destination].withRouter(ConsistentHashingRouter(nbrRoutees, - virtualNodesFactor = 10)), "router") - // val router = context.actorOf(Props[Destination].withRouter(RoundRobinRouter(nbrRoutees)), "router") - val rnd = new Random - val messages = Vector.fill(1000)(ConsistentHashableEnvelope("msg", rnd.nextString(10))) - var startTime = 0L - var doneCounter = 0 - var iterationCounter = 0 - - def receive = { - case "start" ⇒ - iterationCounter += 1 - doneCounter = 0 - startTime = System.nanoTime - val messgesSize = messages.size - for (n ← 1 to nbrMessages) { router ! messages(n % messgesSize) } - router ! Broadcast("done") - - case "done" ⇒ - doneCounter += 1 - if (doneCounter == nbrRoutees) { - val duration = Duration.fromNanos(System.nanoTime - startTime) - val mps = (nbrMessages.toDouble * 1000 / duration.toMillis).toInt - // log.info("Processed [{}] messages in [{} millis], i.e. [{}] msg/s", - // nbrMessages, duration.toMillis, mps) - println("Processed [%s] messages in [%s millis], i.e. [%s] msg/s".format( - nbrMessages, duration.toMillis, mps)) - if (iterationCounter < nbrIterations) - self ! "start" - else - context.system.shutdown() - } - } -} - -class Destination extends Actor with ActorLogging { - var count = 0 - def receive = { - case "done" ⇒ - log.info("Handled [{}] messages", count) - count = 0 - sender ! "done" - case msg ⇒ count += 1 - - } -} \ No newline at end of file From 64a1fb023582f40de222e1fc8ee2f24250f43064 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 17 Sep 2012 13:24:13 +0200 Subject: [PATCH 20/46] Rename a few things, see #944 * hashKey * hashMapping ConsistentHashMapping * withHashMapper ConsistentHashMapper --- .../routing/ConsistentHashingRouterSpec.scala | 17 ++++--- .../routing/ConsistentHashingRouter.scala | 50 ++++++++++--------- .../ConsistentHashingRouterDocTestBase.java | 16 +++--- akka-docs/java/routing.rst | 6 +-- .../ConsistentHashingRouterDocSpec.scala | 10 ++-- akka-docs/scala/routing.rst | 6 +-- 6 files changed, 54 insertions(+), 51 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala index 0024ceea44..867da83bd7 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConsistentHashingRouterSpec.scala @@ -12,6 +12,7 @@ import akka.actor.actorRef2Scala import akka.pattern.ask import akka.routing.ConsistentHashingRouter.ConsistentHashable import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope +import akka.routing.ConsistentHashingRouter.ConsistentHashMapping import akka.testkit.AkkaSpec import akka.testkit._ @@ -62,40 +63,40 @@ class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.c "select destination based on consistentHashKey of the message" in { router1 ! Msg("a", "A") val destinationA = expectMsgType[ActorRef] - router1 ! ConsistentHashableEnvelope(message = "AA", consistentHashKey = "a") + router1 ! ConsistentHashableEnvelope(message = "AA", hashKey = "a") expectMsg(destinationA) router1 ! Msg(17, "B") val destinationB = expectMsgType[ActorRef] - router1 ! ConsistentHashableEnvelope(message = "BB", consistentHashKey = 17) + router1 ! ConsistentHashableEnvelope(message = "BB", hashKey = 17) expectMsg(destinationB) router1 ! Msg(MsgKey("c"), "C") val destinationC = expectMsgType[ActorRef] - router1 ! ConsistentHashableEnvelope(message = "CC", consistentHashKey = MsgKey("c")) + router1 ! ConsistentHashableEnvelope(message = "CC", hashKey = MsgKey("c")) expectMsg(destinationC) } "select destination with defined consistentHashRoute" in { - def consistentHashRoute: ConsistentHashingRouter.ConsistentHashRoute = { + def hashMapping: ConsistentHashMapping = { case Msg2(key, data) ⇒ key } val router2 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter( - consistentHashRoute = consistentHashRoute)), "router2") + hashMapping = hashMapping)), "router2") router2 ! Msg2("a", "A") val destinationA = expectMsgType[ActorRef] - router2 ! ConsistentHashableEnvelope(message = "AA", consistentHashKey = "a") + router2 ! ConsistentHashableEnvelope(message = "AA", hashKey = "a") expectMsg(destinationA) router2 ! Msg2(17, "B") val destinationB = expectMsgType[ActorRef] - router2 ! ConsistentHashableEnvelope(message = "BB", consistentHashKey = 17) + router2 ! ConsistentHashableEnvelope(message = "BB", hashKey = 17) expectMsg(destinationB) router2 ! Msg2(MsgKey("c"), "C") val destinationC = expectMsgType[ActorRef] - router2 ! ConsistentHashableEnvelope(message = "CC", consistentHashKey = MsgKey("c")) + router2 ! ConsistentHashableEnvelope(message = "CC", hashKey = MsgKey("c")) expectMsg(destinationC) } } diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index 9650a038e9..4937691d88 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -29,7 +29,7 @@ object ConsistentHashingRouter { } /** - * If you don't define the consistentHashRoute when + * If you don't define the `hashMapping` when * constructing the [[akka.routing.ConsistentHashingRouter]] * the messages need to implement this interface to define what * data to use for the consistent hash key. Note that it's not @@ -49,7 +49,7 @@ object ConsistentHashingRouter { } /** - * If you don't define the consistentHashRoute when + * If you don't define the `hashMapping` when * constructing the [[akka.routing.ConsistentHashingRouter]] * and messages can't implement [[akka.routing.ConsistentHashingRouter.ConsistentHashable]] * themselves they can we wrapped by this envelope instead. The @@ -57,8 +57,10 @@ object ConsistentHashingRouter { * i.e. the envelope will be stripped off. */ @SerialVersionUID(1L) - case class ConsistentHashableEnvelope(message: Any, consistentHashKey: Any) - extends ConsistentHashable with RouterEnvelope + final case class ConsistentHashableEnvelope(message: Any, hashKey: Any) + extends ConsistentHashable with RouterEnvelope { + override def consistentHashKey: Any = hashKey + } /** * Partial function from message to the data to @@ -69,12 +71,12 @@ object ConsistentHashingRouter { * otherwise the configured [[akka.akka.serialization.Serializer]] * will be applied to the returned data. */ - type ConsistentHashRoute = PartialFunction[Any, Any] + type ConsistentHashMapping = PartialFunction[Any, Any] @SerialVersionUID(1L) - object emptyConsistentHashRoute extends ConsistentHashRoute { + object emptyConsistentHashMapping extends ConsistentHashMapping { def isDefinedAt(x: Any) = false - def apply(x: Any) = throw new UnsupportedOperationException("Empty ConsistentHashRoute apply()") + def apply(x: Any) = throw new UnsupportedOperationException("Empty ConsistentHashMapping apply()") } /** @@ -90,8 +92,8 @@ object ConsistentHashingRouter { * otherwise the configured [[akka.akka.serialization.Serializer]] * will be applied to the returned data. */ - trait ConsistentHashMapping { - def consistentHashKey(message: Any): Any + trait ConsistentHashMapper { + def hashKey(message: Any): Any } } /** @@ -100,7 +102,7 @@ object ConsistentHashingRouter { * * There is 3 ways to define what data to use for the consistent hash key. * - * 1. You can define `consistentHashRoute` / `withConsistentHashMapping` + * 1. You can define `hashMapping` / `withHashMapper` * of the router to map incoming messages to their consistent hash key. * This makes the decision transparent for the sender. * @@ -113,7 +115,7 @@ object ConsistentHashingRouter { * the key to use. * * These ways to define the consistent hash key can be use together and at - * the same time for one router. The `consistentHashRoute` is tried first. + * the same time for one router. The `hashMapping` is tried first. * * Please note that providing both 'nrOfInstances' and 'routees' does not make logical * sense as this means that the router should both create new actors and use the 'routees' @@ -137,7 +139,7 @@ object ConsistentHashingRouter { * @param routees string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] * @param virtualNodesFactor number of virtual nodes per node, used in [[akka.routing.ConsistantHash]] - * @param consistentHashRoute partial function from message to the data to + * @param hashMapping partial function from message to the data to * use for the consistent hash key */ @SerialVersionUID(1L) @@ -146,7 +148,7 @@ case class ConsistentHashingRouter( val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy, val virtualNodesFactor: Int = 0, - val consistentHashRoute: ConsistentHashingRouter.ConsistentHashRoute = ConsistentHashingRouter.emptyConsistentHashRoute) + val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping) extends RouterConfig with ConsistentHashingLike { /** @@ -188,10 +190,10 @@ case class ConsistentHashingRouter( /** * Java API for setting the mapping from message to the data to use for the consistent hash key. */ - def withConsistentHashMapping(mapping: ConsistentHashingRouter.ConsistentHashMapping) = { - copy(consistentHashRoute = { - case message if (mapping.consistentHashKey(message).asInstanceOf[AnyRef] ne null) ⇒ - mapping.consistentHashKey(message) + def withHashMapper(mapping: ConsistentHashingRouter.ConsistentHashMapper) = { + copy(hashMapping = { + case message if (mapping.hashKey(message).asInstanceOf[AnyRef] ne null) ⇒ + mapping.hashKey(message) }) } @@ -199,7 +201,7 @@ case class ConsistentHashingRouter( * Uses the resizer of the given RouterConfig if this RouterConfig * doesn't have one, i.e. the resizer defined in code is used if * resizer was not defined in config. - * Uses the the consistentHashRoute defined in code, since + * Uses the the `hashMapping` defined in code, since * that can't be defined in configuration. */ override def withFallback(other: RouterConfig): RouterConfig = other match { @@ -208,7 +210,7 @@ case class ConsistentHashingRouter( val useResizer = if (this.resizer.isEmpty && otherRouter.resizer.isDefined) otherRouter.resizer else this.resizer - copy(resizer = useResizer, consistentHashRoute = otherRouter.consistentHashRoute) + copy(resizer = useResizer, hashMapping = otherRouter.hashMapping) case _ ⇒ throw new IllegalArgumentException("Expected ConsistentHashingRouter, got [%s]".format(other)) } } @@ -227,7 +229,7 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ def virtualNodesFactor: Int - def consistentHashRoute: ConsistentHashRoute + def hashMapping: ConsistentHashMapping override def createRoute(routeeProvider: RouteeProvider): Route = { if (resizer.isEmpty) { @@ -273,7 +275,7 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ } catch { case NonFatal(e) ⇒ // serialization failed - log.warning("Couldn't route message with consistentHashKey [{}] due to [{}]", hashData, e.getMessage) + log.warning("Couldn't route message with consistent hash key [{}] due to [{}]", hashData, e.getMessage) routeeProvider.context.system.deadLetters } @@ -281,11 +283,11 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ case (sender, message) ⇒ message match { case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees) - case _ if consistentHashRoute.isDefinedAt(message) ⇒ - List(Destination(sender, target(consistentHashRoute(message)))) + case _ if hashMapping.isDefinedAt(message) ⇒ + List(Destination(sender, target(hashMapping(message)))) case hashable: ConsistentHashable ⇒ List(Destination(sender, target(hashable.consistentHashKey))) case other ⇒ - log.warning("Message [{}] must be handled by consistentHashRoute, or implement [{}] or be wrapped in [{}]", + log.warning("Message [{}] must be handled by hashMapping, or implement [{}] or be wrapped in [{}]", message.getClass.getName, classOf[ConsistentHashable].getName, classOf[ConsistentHashableEnvelope].getName) List(Destination(sender, routeeProvider.context.system.deadLetters)) diff --git a/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTestBase.java b/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTestBase.java index 6f00ed1509..378049578f 100644 --- a/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTestBase.java +++ b/akka-docs/java/code/docs/jrouting/ConsistentHashingRouterDocTestBase.java @@ -21,7 +21,7 @@ import java.io.Serializable; import akka.actor.Props; import akka.actor.ActorRef; import akka.routing.ConsistentHashingRouter; -import akka.routing.ConsistentHashingRouter.ConsistentHashMapping; +import akka.routing.ConsistentHashingRouter.ConsistentHashMapper; import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope; //#imports2 @@ -40,7 +40,7 @@ public class ConsistentHashingRouterDocTestBase { } //#cache-actor - + public static class Cache extends UntypedActor { Map cache = new HashMap(); @@ -62,14 +62,14 @@ public class ConsistentHashingRouterDocTestBase { } } - public static class Evict implements Serializable { + public static final class Evict implements Serializable { public final String key; public Evict(String key) { this.key = key; } } - public static class Get implements Serializable, ConsistentHashable { + public static final class Get implements Serializable, ConsistentHashable { public final String key; public Get(String key) { this.key = key; @@ -79,7 +79,7 @@ public class ConsistentHashingRouterDocTestBase { } } - public static class Entry implements Serializable { + public static final class Entry implements Serializable { public final String key; public final String value; public Entry(String key, String value) { @@ -99,9 +99,9 @@ public class ConsistentHashingRouterDocTestBase { //#consistent-hashing-router - final ConsistentHashMapping consistentHashMapping = new ConsistentHashMapping() { + final ConsistentHashMapper hashMapper = new ConsistentHashMapper() { @Override - public Object consistentHashKey(Object message) { + public Object hashKey(Object message) { if (message instanceof Evict) { return ((Evict) message).key; } else { @@ -111,7 +111,7 @@ public class ConsistentHashingRouterDocTestBase { }; ActorRef cache = system.actorOf(new Props(Cache.class).withRouter( - new ConsistentHashingRouter(10).withConsistentHashMapping(consistentHashMapping)), + new ConsistentHashingRouter(10).withHashMapper(hashMapper)), "cache"); cache.tell(new ConsistentHashableEnvelope( diff --git a/akka-docs/java/routing.rst b/akka-docs/java/routing.rst index 149a3a315b..4653a07692 100644 --- a/akka-docs/java/routing.rst +++ b/akka-docs/java/routing.rst @@ -286,7 +286,7 @@ insight into how consistent hashing is implemented. There is 3 ways to define what data to use for the consistent hash key. -* You can define ``withConsistentHashMapping`` of the router to map incoming +* You can define ``withHashMapper`` of the router to map incoming messages to their consistent hash key. This makes the the decision transparent for the sender. @@ -299,7 +299,7 @@ There is 3 ways to define what data to use for the consistent hash key. the key to use. These ways to define the consistent hash key can be use together and at -the same time for one router. The ``withConsistentHashMapping`` is tried first. +the same time for one router. The ``withHashMapper`` is tried first. Code example: @@ -311,7 +311,7 @@ Code example: In the above example you see that the ``Get`` message implements ``ConsistentHashable`` itself, while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. The ``Evict`` -message is handled by the ``withConsistentHashMapping``. +message is handled by the ``withHashMapper``. This is an example of how to define a consistent-hashing router in configuration: diff --git a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala index 7041db6049..dcb40b0c67 100644 --- a/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala +++ b/akka-docs/scala/code/docs/routing/ConsistentHashingRouterDocSpec.scala @@ -42,20 +42,20 @@ class ConsistentHashingRouterDocSpec extends AkkaSpec with ImplicitSender { //#consistent-hashing-router import akka.actor.Props import akka.routing.ConsistentHashingRouter - import akka.routing.ConsistentHashingRouter.ConsistentHashRoute + import akka.routing.ConsistentHashingRouter.ConsistentHashMapping import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope - def consistentHashRoute: ConsistentHashRoute = { + def hashMapping: ConsistentHashMapping = { case Evict(key) ⇒ key } val cache = system.actorOf(Props[Cache].withRouter(ConsistentHashingRouter(10, - consistentHashRoute = consistentHashRoute)), name = "cache") + hashMapping = hashMapping)), name = "cache") cache ! ConsistentHashableEnvelope( - message = Entry("hello", "HELLO"), consistentHashKey = "hello") + message = Entry("hello", "HELLO"), hashKey = "hello") cache ! ConsistentHashableEnvelope( - message = Entry("hi", "HI"), consistentHashKey = "hi") + message = Entry("hi", "HI"), hashKey = "hi") cache ! Get("hello") expectMsg(Some("HELLO")) diff --git a/akka-docs/scala/routing.rst b/akka-docs/scala/routing.rst index 4766b2e6ca..c1fa21b23f 100644 --- a/akka-docs/scala/routing.rst +++ b/akka-docs/scala/routing.rst @@ -291,7 +291,7 @@ insight into how consistent hashing is implemented. There is 3 ways to define what data to use for the consistent hash key. -* You can define ``consistentHashRoute`` of the router to map incoming +* You can define ``hashMapping`` of the router to map incoming messages to their consistent hash key. This makes the decision transparent for the sender. @@ -304,7 +304,7 @@ There is 3 ways to define what data to use for the consistent hash key. the key to use. These ways to define the consistent hash key can be use together and at -the same time for one router. The ``consistentHashRoute`` is tried first. +the same time for one router. The ``hashMapping`` is tried first. Code example: @@ -314,7 +314,7 @@ Code example: In the above example you see that the ``Get`` message implements ``ConsistentHashable`` itself, while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. The ``Evict`` -message is handled by the ``consistentHashRoute`` partial function. +message is handled by the ``hashMapping`` partial function. This is an example of how to define a consistent-hashing router in configuration: From 44f4fdb003dbf8014bef8bc2bc22c3e974815eaf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Antonsson?= Date: Mon, 17 Sep 2012 09:51:15 +0200 Subject: [PATCH 21/46] Some optimizations for the ConsistentHashingRouter --- .../scala/akka/routing/ConsistentHash.scala | 45 ++++++++++++------- .../routing/ConsistentHashingRouter.scala | 11 +++-- 2 files changed, 33 insertions(+), 23 deletions(-) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index bc86f5f82d..fca0837662 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -24,10 +24,9 @@ class ConsistentHash[T] private (nodes: Map[Int, T], virtualNodesFactor: Int) { if (virtualNodesFactor < 1) throw new IllegalArgumentException("virtualNodesFactor must be >= 1") // sorted hash values of the nodes - private val nodeRing: Array[Int] = { - val nodeRing = nodes.keys.toArray - Arrays.sort(nodeRing) - nodeRing + private val (nodeHashRing: Array[Int], nodeRing: Vector[T]) = { + val (nhr: IndexedSeq[Int], nr: IndexedSeq[AnyRef]) = nodes.toArray.sortBy(_._1).unzip + (nhr.toArray, Vector[T]() ++ nr) } /** @@ -62,6 +61,17 @@ class ConsistentHash[T] private (nodes: Map[Int, T], virtualNodesFactor: Int) { */ def remove(node: T): ConsistentHash[T] = this :- node + // converts the result of Arrays.binarySearch into a index in the nodeRing array + // see documentation of Arrays.binarySearch for what it returns + private def idx(i: Int): Int = { + if (i >= 0) i // exact match + else { + val j = math.abs(i + 1) + if (j >= nodeHashRing.length) 0 // after last, use first + else j // next node clockwise + } + } + /** * Get the node responsible for the data key. * Can only be used if nodes exists in the node ring, @@ -70,18 +80,18 @@ class ConsistentHash[T] private (nodes: Map[Int, T], virtualNodesFactor: Int) { def nodeFor(key: Array[Byte]): T = { if (isEmpty) throw new IllegalStateException("Can't get node for [%s] from an empty node ring" format key) - // converts the result of Arrays.binarySearch into a index in the nodeRing array - // see documentation of Arrays.binarySearch for what it returns - def idx(i: Int): Int = { - if (i >= 0) i // exact match - else { - val j = math.abs(i + 1) - if (j >= nodeRing.length) 0 // after last, use first - else j // next node clockwise - } - } - val nodeRingIndex = idx(Arrays.binarySearch(nodeRing, hashFor(key))) - nodes(nodeRing(nodeRingIndex)) + nodeRing(idx(Arrays.binarySearch(nodeHashRing, hashFor(key)))) + } + + /** + * Get the node responsible for the data key. + * Can only be used if nodes exists in the node ring, + * otherwise throws `IllegalStateException` + */ + def nodeFor(key: String): T = { + if (isEmpty) throw new IllegalStateException("Can't get node for [%s] from an empty node ring" format key) + + nodeRing(idx(Arrays.binarySearch(nodeHashRing, hashFor(key)))) } /** @@ -111,5 +121,6 @@ object ConsistentHash { hashFor((node + ":" + vnode).getBytes("UTF-8")) private def hashFor(bytes: Array[Byte]): Int = MurmurHash.arrayHash(bytes) -} + private def hashFor(string: String): Int = MurmurHash.stringHash(string) +} diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index 4937691d88..3b9802d7fd 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -264,14 +264,13 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ } def target(hashData: Any): ActorRef = try { - val hash = hashData match { - case bytes: Array[Byte] ⇒ bytes - case str: String ⇒ str.getBytes("UTF-8") - case x: AnyRef ⇒ SerializationExtension(routeeProvider.context.system).serialize(x).get - } val currentConsistenHash = updateConsistentHash() if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters - else currentConsistenHash.nodeFor(hash) + else hashData match { + case bytes: Array[Byte] ⇒ currentConsistenHash.nodeFor(bytes) + case str: String ⇒ currentConsistenHash.nodeFor(str) + case x: AnyRef ⇒ currentConsistenHash.nodeFor(SerializationExtension(routeeProvider.context.system).serialize(x).get) + } } catch { case NonFatal(e) ⇒ // serialization failed From 90d2fb46fca24ab908b068c5893bdb9cfabf4e6c Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 17 Sep 2012 20:46:53 +0200 Subject: [PATCH 22/46] ClassTag to construct arrays, see #944 * array instead of Vector --- .../scala/akka/routing/ConsistentHash.scala | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index fca0837662..79c31cda33 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -4,7 +4,8 @@ package akka.routing -import scala.collection.immutable.TreeMap +import scala.collection.immutable.SortedMap +import scala.reflect.ClassTag import java.util.Arrays /** @@ -17,16 +18,18 @@ import java.util.Arrays * hash, i.e. make sure it is different for different nodes. * */ -class ConsistentHash[T] private (nodes: Map[Int, T], virtualNodesFactor: Int) { +class ConsistentHash[T: ClassTag] private (nodes: SortedMap[Int, T], virtualNodesFactor: Int) { import ConsistentHash._ if (virtualNodesFactor < 1) throw new IllegalArgumentException("virtualNodesFactor must be >= 1") - // sorted hash values of the nodes - private val (nodeHashRing: Array[Int], nodeRing: Vector[T]) = { - val (nhr: IndexedSeq[Int], nr: IndexedSeq[AnyRef]) = nodes.toArray.sortBy(_._1).unzip - (nhr.toArray, Vector[T]() ++ nr) + // arrays for fast binary search and access + // nodeHashRing is the sorted hash values of the nodes + // nodeRing is the nodes sorted in the same order as nodeHashRing, i.e. same index + private val (nodeHashRing: Array[Int], nodeRing: Array[T]) = { + val (nhr: Seq[Int], nr: Seq[T]) = nodes.toSeq.unzip + (nhr.toArray, nr.toArray) } /** @@ -102,8 +105,8 @@ class ConsistentHash[T] private (nodes: Map[Int, T], virtualNodesFactor: Int) { } object ConsistentHash { - def apply[T](nodes: Iterable[T], virtualNodesFactor: Int) = { - new ConsistentHash(TreeMap.empty[Int, T] ++ + def apply[T: ClassTag](nodes: Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = { + new ConsistentHash(SortedMap.empty[Int, T] ++ (for (node ← nodes; vnode ← 1 to virtualNodesFactor) yield (nodeHashFor(node, vnode) -> node)), virtualNodesFactor) } @@ -112,9 +115,9 @@ object ConsistentHash { * Factory method to create a ConsistentHash * JAVA API */ - def create[T](nodes: java.lang.Iterable[T], virtualNodesFactor: Int) = { + def create[T](nodes: java.lang.Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = { import scala.collection.JavaConverters._ - apply(nodes.asScala, virtualNodesFactor) + apply(nodes.asScala, virtualNodesFactor)(ClassTag(classOf[Any].asInstanceOf[Class[T]])) } private def nodeHashFor(node: Any, vnode: Int): Int = From 35b7a9e3384813deac53adb337aee94f990c4cc3 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 18 Sep 2012 09:58:30 +0200 Subject: [PATCH 23/46] second round of FiniteDuration business, including cluster fixes - make Scheduler only accept FiniteDuration, which has quite some knock-on effects --- .../test/scala/akka/actor/FSMActorSpec.scala | 4 +-- .../src/test/scala/akka/actor/IOActor.scala | 7 ++-- .../akka/util/internal/HashedWheelTimer.java | 3 +- .../main/java/akka/util/internal/Timer.java | 6 ++-- .../src/main/scala/akka/actor/FSM.scala | 19 +++++----- .../src/main/scala/akka/actor/Scheduler.scala | 35 ++++++++++--------- .../akka/actor/cell/ReceiveTimeout.scala | 33 +++++++++-------- .../src/main/scala/akka/actor/dsl/Inbox.scala | 4 +-- .../akka/dispatch/AbstractDispatcher.scala | 3 +- .../akka/dispatch/BalancingDispatcher.scala | 3 +- .../main/scala/akka/dispatch/Dispatcher.scala | 3 +- .../akka/dispatch/PinnedDispatcher.scala | 3 +- .../akka/pattern/FutureTimeoutSupport.scala | 3 +- .../main/scala/akka/pattern/Patterns.scala | 4 +-- .../src/main/scala/akka/routing/Routing.scala | 6 ++-- .../src/main/scala/akka/cluster/Cluster.scala | 16 ++++----- .../scala/akka/cluster/ClusterDaemon.scala | 11 +++--- .../scala/akka/cluster/FixedRateTask.scala | 9 ++--- .../scala/akka/cluster/LargeClusterSpec.scala | 8 +++-- .../akka/cluster/MultiNodeClusterSpec.scala | 4 +-- .../actor/japi/FaultHandlingDocSample.java | 4 +-- .../code/docs/future/FutureDocTestBase.java | 2 +- .../code/docs/zeromq/ZeromqDocTestBase.java | 2 +- .../akka/remote/testconductor/Conductor.scala | 5 +-- .../NetworkFailureInjector.scala | 13 +++---- .../akka/remote/testconductor/Player.scala | 2 +- .../src/main/scala/Buncher.scala | 5 +-- .../src/main/scala/DiningHakkersOnFsm.scala | 4 +-- .../main/scala/akka/testkit/TestFSMRef.scala | 5 +-- .../main/scala/akka/zeromq/SocketOption.scala | 3 +- 30 files changed, 125 insertions(+), 104 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index 23a48d5417..bae6d2f6fe 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -5,7 +5,6 @@ package akka.actor import language.postfixOps - import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import akka.testkit._ import TestEvent.Mute @@ -15,6 +14,7 @@ import com.typesafe.config.ConfigFactory import scala.concurrent.Await import akka.util.Timeout import scala.concurrent.util.Duration +import scala.concurrent.util.FiniteDuration object FSMActorSpec { val timeout = Timeout(2 seconds) @@ -33,7 +33,7 @@ object FSMActorSpec { case object Locked extends LockState case object Open extends LockState - class Lock(code: String, timeout: Duration, latches: Latches) extends Actor with FSM[LockState, CodeState] { + class Lock(code: String, timeout: FiniteDuration, latches: Latches) extends Actor with FSM[LockState, CodeState] { import latches._ diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 72efaef0d4..ccb18e60ea 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -5,7 +5,6 @@ package akka.actor import language.postfixOps - import akka.util.ByteString import scala.concurrent.{ ExecutionContext, Await, Future, Promise } import scala.concurrent.util.{ Duration, Deadline } @@ -17,6 +16,7 @@ import akka.pattern.ask import java.net.{ Socket, InetSocketAddress, InetAddress, SocketAddress } import scala.util.Failure import annotation.tailrec +import scala.concurrent.util.FiniteDuration object IOActorSpec { @@ -244,7 +244,10 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { * @param filter determines which exceptions should be retried * @return a future containing the result or the last exception before a limit was hit. */ - def retry[T](count: Option[Int] = None, timeout: Option[Duration] = None, delay: Option[Duration] = Some(100 millis), filter: Option[Throwable ⇒ Boolean] = None)(future: ⇒ Future[T])(implicit executor: ExecutionContext): Future[T] = { + def retry[T](count: Option[Int] = None, + timeout: Option[FiniteDuration] = None, + delay: Option[FiniteDuration] = Some(100 millis), + filter: Option[Throwable ⇒ Boolean] = None)(future: ⇒ Future[T])(implicit executor: ExecutionContext): Future[T] = { val promise = Promise[T]() diff --git a/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java b/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java index cc4328d763..1630f599ee 100644 --- a/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java +++ b/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java @@ -25,6 +25,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import scala.concurrent.util.Duration; +import scala.concurrent.util.FiniteDuration; import akka.event.LoggingAdapter; import akka.util.Unsafe; @@ -241,7 +242,7 @@ public class HashedWheelTimer implements Timer { return new HashedWheelTimeout(this, task, time); } - public Timeout newTimeout(TimerTask task, Duration delay) { + public Timeout newTimeout(TimerTask task, FiniteDuration delay) { final long currentTime = System.nanoTime(); if (task == null) { diff --git a/akka-actor/src/main/java/akka/util/internal/Timer.java b/akka-actor/src/main/java/akka/util/internal/Timer.java index 7086aef9c6..be7656ec6c 100644 --- a/akka-actor/src/main/java/akka/util/internal/Timer.java +++ b/akka-actor/src/main/java/akka/util/internal/Timer.java @@ -15,9 +15,9 @@ */ package akka.util.internal; -import scala.concurrent.util.Duration; import java.util.Set; -import java.util.concurrent.TimeUnit; + +import scala.concurrent.util.FiniteDuration; /** * Schedules {@link TimerTask}s for one-time future execution in a background @@ -42,7 +42,7 @@ public interface Timer { * @throws IllegalStateException if this timer has been * {@linkplain #stop() stopped} already */ - Timeout newTimeout(TimerTask task, Duration delay); + Timeout newTimeout(TimerTask task, FiniteDuration delay); /** * Releases all resources acquired by this {@link Timer} and cancels all diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index 51d0c290dc..a58abb0ac3 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -4,11 +4,11 @@ package akka.actor import language.implicitConversions - import akka.util._ import scala.concurrent.util.Duration import scala.collection.mutable import akka.routing.{ Deafen, Listen, Listeners } +import scala.concurrent.util.FiniteDuration object FSM { @@ -92,7 +92,7 @@ object FSM { private val scheduler = context.system.scheduler private implicit val executionContext = context.dispatcher - def schedule(actor: ActorRef, timeout: Duration): Unit = + def schedule(actor: ActorRef, timeout: FiniteDuration): Unit = ref = Some( if (repeat) scheduler.schedule(timeout, timeout, actor, this) else scheduler.scheduleOnce(timeout, actor, this)) @@ -121,15 +121,18 @@ object FSM { * name, the state data, possibly custom timeout, stop reason and replies * accumulated while processing the last message. */ - case class State[S, D](stateName: S, stateData: D, timeout: Option[Duration] = None, stopReason: Option[Reason] = None, replies: List[Any] = Nil) { + case class State[S, D](stateName: S, stateData: D, timeout: Option[FiniteDuration] = None, stopReason: Option[Reason] = None, replies: List[Any] = Nil) { /** * Modify state transition descriptor to include a state timeout for the * next state. This timeout overrides any default timeout set for the next * state. + * + * Use Duration.Inf to deactivate an existing timeout. */ - def forMax(timeout: Duration): State[S, D] = { - copy(timeout = Some(timeout)) + def forMax(timeout: Duration): State[S, D] = timeout match { + case f: FiniteDuration ⇒ copy(timeout = Some(f)) + case _ ⇒ copy(timeout = None) } /** @@ -245,7 +248,7 @@ trait FSM[S, D] extends Listeners with ActorLogging { type State = FSM.State[S, D] type StateFunction = scala.PartialFunction[Event, State] - type Timeout = Option[Duration] + type Timeout = Option[FiniteDuration] type TransitionHandler = PartialFunction[(S, S), Unit] /* @@ -279,7 +282,7 @@ trait FSM[S, D] extends Listeners with ActorLogging { * @param stateTimeout default state timeout for this state * @param stateFunction partial function describing response to input */ - final def when(stateName: S, stateTimeout: Duration = null)(stateFunction: StateFunction): Unit = + final def when(stateName: S, stateTimeout: FiniteDuration = null)(stateFunction: StateFunction): Unit = register(stateName, stateFunction, Option(stateTimeout)) /** @@ -339,7 +342,7 @@ trait FSM[S, D] extends Listeners with ActorLogging { * @param repeat send once if false, scheduleAtFixedRate if true * @return current state descriptor */ - final def setTimer(name: String, msg: Any, timeout: Duration, repeat: Boolean): State = { + final def setTimer(name: String, msg: Any, timeout: FiniteDuration, repeat: Boolean): State = { if (debugEvent) log.debug("setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg) if (timers contains name) { diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index 42f2a10604..a3822d90ce 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -13,6 +13,7 @@ import java.util.concurrent.atomic.AtomicReference import scala.annotation.tailrec import akka.util.internal._ import concurrent.ExecutionContext +import scala.concurrent.util.FiniteDuration //#scheduler /** @@ -34,8 +35,8 @@ trait Scheduler { * Java & Scala API */ def schedule( - initialDelay: Duration, - frequency: Duration, + initialDelay: FiniteDuration, + frequency: FiniteDuration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable @@ -48,7 +49,7 @@ trait Scheduler { * Scala API */ def schedule( - initialDelay: Duration, frequency: Duration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable + initialDelay: FiniteDuration, frequency: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable /** * Schedules a function to be run repeatedly with an initial delay and @@ -59,7 +60,7 @@ trait Scheduler { * Java API */ def schedule( - initialDelay: Duration, frequency: Duration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable + initialDelay: FiniteDuration, frequency: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable /** * Schedules a Runnable to be run once with a delay, i.e. a time period that @@ -67,7 +68,7 @@ trait Scheduler { * * Java & Scala API */ - def scheduleOnce(delay: Duration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable + def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable /** * Schedules a message to be sent once with a delay, i.e. a time period that has @@ -75,7 +76,7 @@ trait Scheduler { * * Java & Scala API */ - def scheduleOnce(delay: Duration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable + def scheduleOnce(delay: FiniteDuration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable /** * Schedules a function to be run once with a delay, i.e. a time period that has @@ -83,7 +84,7 @@ trait Scheduler { * * Scala API */ - def scheduleOnce(delay: Duration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable + def scheduleOnce(delay: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable } //#scheduler @@ -120,8 +121,8 @@ trait Cancellable { * returned from stop(). */ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter) extends Scheduler with Closeable { - override def schedule(initialDelay: Duration, - delay: Duration, + override def schedule(initialDelay: FiniteDuration, + delay: FiniteDuration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable = { val continuousCancellable = new ContinuousCancellable @@ -142,12 +143,12 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter) initialDelay)) } - override def schedule(initialDelay: Duration, - delay: Duration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = + override def schedule(initialDelay: FiniteDuration, + delay: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = schedule(initialDelay, delay, new Runnable { override def run = f }) - override def schedule(initialDelay: Duration, - delay: Duration, + override def schedule(initialDelay: FiniteDuration, + delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = { val continuousCancellable = new ContinuousCancellable continuousCancellable.init( @@ -163,20 +164,20 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter) initialDelay)) } - override def scheduleOnce(delay: Duration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = + override def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = new DefaultCancellable( hashedWheelTimer.newTimeout( new TimerTask() { def run(timeout: HWTimeout): Unit = executor.execute(runnable) }, delay)) - override def scheduleOnce(delay: Duration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable = + override def scheduleOnce(delay: FiniteDuration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable = scheduleOnce(delay, new Runnable { override def run = receiver ! message }) - override def scheduleOnce(delay: Duration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = + override def scheduleOnce(delay: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = scheduleOnce(delay, new Runnable { override def run = f }) private trait ContinuousScheduling { this: TimerTask ⇒ - def scheduleNext(timeout: HWTimeout, delay: Duration, delegator: ContinuousCancellable) { + def scheduleNext(timeout: HWTimeout, delay: FiniteDuration, delegator: ContinuousCancellable) { try delegator.swap(timeout.getTimer.newTimeout(this, delay)) catch { case _: IllegalStateException ⇒ } // stop recurring if timer is stopped } } diff --git a/akka-actor/src/main/scala/akka/actor/cell/ReceiveTimeout.scala b/akka-actor/src/main/scala/akka/actor/cell/ReceiveTimeout.scala index c04d485262..6ff6a39dea 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/ReceiveTimeout.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/ReceiveTimeout.scala @@ -9,9 +9,10 @@ import akka.actor.ActorCell import akka.actor.ActorCell.emptyCancellable import akka.actor.Cancellable import scala.concurrent.util.Duration +import scala.concurrent.util.FiniteDuration private[akka] object ReceiveTimeout { - final val emptyReceiveTimeoutData: (Duration, Cancellable) = (Duration.Undefined, ActorCell.emptyCancellable) + final val emptyReceiveTimeoutData: (Option[FiniteDuration], Cancellable) = (None, ActorCell.emptyCancellable) } private[akka] trait ReceiveTimeout { this: ActorCell ⇒ @@ -19,28 +20,32 @@ private[akka] trait ReceiveTimeout { this: ActorCell ⇒ import ReceiveTimeout._ import ActorCell._ - private var receiveTimeoutData: (Duration, Cancellable) = emptyReceiveTimeoutData + private var receiveTimeoutData: (Option[FiniteDuration], Cancellable) = emptyReceiveTimeoutData - final def receiveTimeout: Option[Duration] = receiveTimeoutData._1 match { - case Duration.Undefined ⇒ None - case duration ⇒ Some(duration) + final def receiveTimeout: Option[FiniteDuration] = receiveTimeoutData._1 + + final def setReceiveTimeout(timeout: Option[FiniteDuration]): Unit = + receiveTimeoutData = receiveTimeoutData.copy(_1 = timeout) + + final def setReceiveTimeout(timeout: Duration): Unit = { + import Duration._ + setReceiveTimeout(timeout match { + case x if x eq Undefined ⇒ None + case Inf | MinusInf ⇒ throw new IllegalArgumentException("receiveTimeout cannot be infinite") + case f: FiniteDuration ⇒ + if (f < Zero) throw new IllegalArgumentException("receiveTimeout cannot be negative") + else Some(f) + }) } - final def setReceiveTimeout(timeout: Option[Duration]): Unit = setReceiveTimeout(timeout.getOrElse(Duration.Undefined)) - - final def setReceiveTimeout(timeout: Duration): Unit = - receiveTimeoutData = ( - if (Duration.Undefined == timeout || timeout.toMillis < 1) Duration.Undefined else timeout, - receiveTimeoutData._2) - final def resetReceiveTimeout(): Unit = setReceiveTimeout(None) final def checkReceiveTimeout() { val recvtimeout = receiveTimeoutData - if (Duration.Undefined != recvtimeout._1 && !mailbox.hasMessages) { + if (recvtimeout._1.isDefined && !mailbox.hasMessages) { recvtimeout._2.cancel() //Cancel any ongoing future //Only reschedule if desired and there are currently no more messages to be processed - receiveTimeoutData = (recvtimeout._1, system.scheduler.scheduleOnce(recvtimeout._1, self, akka.actor.ReceiveTimeout)(this.dispatcher)) + receiveTimeoutData = (recvtimeout._1, system.scheduler.scheduleOnce(recvtimeout._1.get, self, akka.actor.ReceiveTimeout)(this.dispatcher)) } else cancelReceiveTimeout() } diff --git a/akka-actor/src/main/scala/akka/actor/dsl/Inbox.scala b/akka-actor/src/main/scala/akka/actor/dsl/Inbox.scala index e01e87262d..7b1a77bc71 100644 --- a/akka-actor/src/main/scala/akka/actor/dsl/Inbox.scala +++ b/akka-actor/src/main/scala/akka/actor/dsl/Inbox.scala @@ -129,10 +129,10 @@ trait Inbox { this: ActorDSL.type ⇒ val next = clientsByTimeout.head.deadline import context.dispatcher if (currentDeadline.isEmpty) { - currentDeadline = Some((next, context.system.scheduler.scheduleOnce(next.timeLeft, self, Kick))) + currentDeadline = Some((next, context.system.scheduler.scheduleOnce(next.timeLeft.asInstanceOf[FiniteDuration], self, Kick))) } else if (currentDeadline.get._1 != next) { currentDeadline.get._2.cancel() - currentDeadline = Some((next, context.system.scheduler.scheduleOnce(next.timeLeft, self, Kick))) + currentDeadline = Some((next, context.system.scheduler.scheduleOnce(next.timeLeft.asInstanceOf[FiniteDuration], self, Kick))) } } } diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 340195d1a6..eeff39f2e6 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -16,6 +16,7 @@ import scala.concurrent.forkjoin.{ ForkJoinTask, ForkJoinPool } import scala.concurrent.util.Duration import scala.concurrent.{ ExecutionContext, Await, Awaitable } import scala.util.control.NonFatal +import scala.concurrent.util.FiniteDuration final case class Envelope private (val message: Any, val sender: ActorRef) @@ -316,7 +317,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext * * INTERNAL API */ - protected[akka] def shutdownTimeout: Duration + protected[akka] def shutdownTimeout: FiniteDuration /** * After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 3897027d9b..c90048c80b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -11,6 +11,7 @@ import akka.util.Helpers import java.util.{ Comparator, Iterator } import java.util.concurrent.{ Executor, LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet } import akka.actor.ActorSystemImpl +import scala.concurrent.util.FiniteDuration /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -33,7 +34,7 @@ class BalancingDispatcher( throughputDeadlineTime: Duration, mailboxType: MailboxType, _executorServiceFactoryProvider: ExecutorServiceFactoryProvider, - _shutdownTimeout: Duration, + _shutdownTimeout: FiniteDuration, attemptTeamWork: Boolean) extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) { diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 21f4612750..96166022f8 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -12,6 +12,7 @@ import java.util.concurrent.{ ExecutorService, RejectedExecutionException } import scala.concurrent.forkjoin.ForkJoinPool import scala.concurrent.util.Duration import scala.concurrent.Awaitable +import scala.concurrent.util.FiniteDuration /** * The event-based ``Dispatcher`` binds a set of Actors to a thread pool backed up by a @@ -32,7 +33,7 @@ class Dispatcher( val throughputDeadlineTime: Duration, val mailboxType: MailboxType, executorServiceFactoryProvider: ExecutorServiceFactoryProvider, - val shutdownTimeout: Duration) + val shutdownTimeout: FiniteDuration) extends MessageDispatcher(_prerequisites) { private class LazyExecutorServiceDelegate(factory: ExecutorServiceFactory) extends ExecutorServiceDelegate { diff --git a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala index 8fdde39cb3..af421ddb96 100644 --- a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala @@ -6,6 +6,7 @@ package akka.dispatch import akka.actor.ActorCell import scala.concurrent.util.Duration +import scala.concurrent.util.FiniteDuration /** * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. @@ -18,7 +19,7 @@ class PinnedDispatcher( _actor: ActorCell, _id: String, _mailboxType: MailboxType, - _shutdownTimeout: Duration, + _shutdownTimeout: FiniteDuration, _threadPoolConfig: ThreadPoolConfig = ThreadPoolConfig()) extends Dispatcher(_prerequisites, _id, diff --git a/akka-actor/src/main/scala/akka/pattern/FutureTimeoutSupport.scala b/akka-actor/src/main/scala/akka/pattern/FutureTimeoutSupport.scala index 72335d810b..dc398e7fa2 100644 --- a/akka-actor/src/main/scala/akka/pattern/FutureTimeoutSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/FutureTimeoutSupport.scala @@ -8,13 +8,14 @@ import scala.concurrent.util.Duration import scala.concurrent.{ ExecutionContext, Promise, Future } import akka.actor._ import scala.util.control.NonFatal +import scala.concurrent.util.FiniteDuration trait FutureTimeoutSupport { /** * Returns a [[scala.concurrent.Future]] that will be completed with the success or failure of the provided value * after the specified duration. */ - def after[T](duration: Duration, using: Scheduler)(value: ⇒ Future[T])(implicit ec: ExecutionContext): Future[T] = + def after[T](duration: FiniteDuration, using: Scheduler)(value: ⇒ Future[T])(implicit ec: ExecutionContext): Future[T] = if (duration.isFinite() && duration.length < 1) { try value catch { case NonFatal(t) ⇒ Future.failed(t) } } else { diff --git a/akka-actor/src/main/scala/akka/pattern/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala index bd86cc4930..c4440f4723 100644 --- a/akka-actor/src/main/scala/akka/pattern/Patterns.scala +++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala @@ -111,13 +111,13 @@ object Patterns { * Returns a [[scala.concurrent.Future]] that will be completed with the success or failure of the provided Callable * after the specified duration. */ - def after[T](duration: Duration, scheduler: Scheduler, context: ExecutionContext, value: Callable[Future[T]]): Future[T] = + def after[T](duration: FiniteDuration, scheduler: Scheduler, context: ExecutionContext, value: Callable[Future[T]]): Future[T] = scalaAfter(duration, scheduler)(value.call())(context) /** * Returns a [[scala.concurrent.Future]] that will be completed with the success or failure of the provided value * after the specified duration. */ - def after[T](duration: Duration, scheduler: Scheduler, context: ExecutionContext, value: Future[T]): Future[T] = + def after[T](duration: FiniteDuration, scheduler: Scheduler, context: ExecutionContext, value: Future[T]): Future[T] = scalaAfter(duration, scheduler)(value)(context) } diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 95fe9d9db2..cb6f37eb8d 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -283,7 +283,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi * The reason for the delay is to give concurrent messages a chance to be * placed in mailbox before sending PoisonPill. */ - def removeRoutees(nrOfInstances: Int, stopDelay: Duration): Unit = { + def removeRoutees(nrOfInstances: Int, stopDelay: FiniteDuration): Unit = { if (nrOfInstances <= 0) { throw new IllegalArgumentException("Expected positive nrOfInstances, got [%s]".format(nrOfInstances)) } else if (nrOfInstances > 0) { @@ -298,7 +298,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi * Give concurrent messages a chance to be placed in mailbox before * sending PoisonPill. */ - protected def delayedStop(scheduler: Scheduler, abandon: Iterable[ActorRef], stopDelay: Duration): Unit = { + protected def delayedStop(scheduler: Scheduler, abandon: Iterable[ActorRef], stopDelay: FiniteDuration): Unit = { if (abandon.nonEmpty) { if (stopDelay <= Duration.Zero) { abandon foreach (_ ! PoisonPill) @@ -1332,7 +1332,7 @@ case class DefaultResizer( * messages a chance to be placed in mailbox before sending PoisonPill. * Use 0 seconds to skip delay. */ - stopDelay: Duration = 1.second, + stopDelay: FiniteDuration = 1.second, /** * Number of messages between resize operation. * Use 1 to resize before each message. diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 9a4dcaa62b..9d9e955272 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -5,7 +5,6 @@ package akka.cluster import language.implicitConversions - import akka.actor._ import akka.actor.Status._ import akka.ConfigurationException @@ -20,13 +19,12 @@ import scala.concurrent.util.{ Duration, Deadline } import scala.concurrent.forkjoin.ThreadLocalRandom import scala.annotation.tailrec import scala.collection.immutable.SortedSet - import java.io.Closeable import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicReference - import akka.util.internal.HashedWheelTimer import concurrent.{ ExecutionContext, Await } +import scala.concurrent.util.FiniteDuration /** * Cluster Extension Id and factory for creating Cluster extension. @@ -111,26 +109,26 @@ class Cluster(val system: ExtendedActorSystem) extends Extension { new Scheduler with Closeable { override def close(): Unit = () // we are using system.scheduler, which we are not responsible for closing - override def schedule(initialDelay: Duration, frequency: Duration, + override def schedule(initialDelay: FiniteDuration, frequency: FiniteDuration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable = systemScheduler.schedule(initialDelay, frequency, receiver, message) - override def schedule(initialDelay: Duration, frequency: Duration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = + override def schedule(initialDelay: FiniteDuration, frequency: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = systemScheduler.schedule(initialDelay, frequency)(f) - override def schedule(initialDelay: Duration, frequency: Duration, + override def schedule(initialDelay: FiniteDuration, frequency: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = systemScheduler.schedule(initialDelay, frequency, runnable) - override def scheduleOnce(delay: Duration, + override def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = systemScheduler.scheduleOnce(delay, runnable) - override def scheduleOnce(delay: Duration, receiver: ActorRef, + override def scheduleOnce(delay: FiniteDuration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable = systemScheduler.scheduleOnce(delay, receiver, message) - override def scheduleOnce(delay: Duration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = + override def scheduleOnce(delay: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = systemScheduler.scheduleOnce(delay)(f) } } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala index 9557a20f7a..3d198572c9 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala @@ -16,6 +16,7 @@ import akka.cluster.MemberStatus._ import akka.cluster.ClusterEvent._ import language.existentials import language.postfixOps +import scala.concurrent.util.FiniteDuration /** * Base trait for all cluster messages. All ClusterMessage's are serializable. @@ -191,32 +192,32 @@ private[cluster] final class ClusterCoreDaemon extends Actor with ActorLogging { // start periodic gossip to random nodes in cluster val gossipTask = - FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(GossipInterval), GossipInterval) { + FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(GossipInterval).asInstanceOf[FiniteDuration], GossipInterval) { self ! GossipTick } // start periodic heartbeat to all nodes in cluster val heartbeatTask = - FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(HeartbeatInterval), HeartbeatInterval) { + FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(HeartbeatInterval).asInstanceOf[FiniteDuration], HeartbeatInterval) { self ! HeartbeatTick } // start periodic cluster failure detector reaping (moving nodes condemned by the failure detector to unreachable list) val failureDetectorReaperTask = - FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(UnreachableNodesReaperInterval), UnreachableNodesReaperInterval) { + FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(UnreachableNodesReaperInterval).asInstanceOf[FiniteDuration], UnreachableNodesReaperInterval) { self ! ReapUnreachableTick } // start periodic leader action management (only applies for the current leader) private val leaderActionsTask = - FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(LeaderActionsInterval), LeaderActionsInterval) { + FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(LeaderActionsInterval).asInstanceOf[FiniteDuration], LeaderActionsInterval) { self ! LeaderActionsTick } // start periodic publish of current state private val publishStateTask: Option[Cancellable] = if (PublishStatsInterval == Duration.Zero) None - else Some(FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(PublishStatsInterval), PublishStatsInterval) { + else Some(FixedRateTask(scheduler, PeriodicTasksInitialDelay.max(PublishStatsInterval).asInstanceOf[FiniteDuration], PublishStatsInterval) { self ! PublishStatsTick }) diff --git a/akka-cluster/src/main/scala/akka/cluster/FixedRateTask.scala b/akka-cluster/src/main/scala/akka/cluster/FixedRateTask.scala index 396f5127ad..9e6eedf659 100644 --- a/akka-cluster/src/main/scala/akka/cluster/FixedRateTask.scala +++ b/akka-cluster/src/main/scala/akka/cluster/FixedRateTask.scala @@ -9,14 +9,15 @@ import java.util.concurrent.atomic.{ AtomicBoolean, AtomicLong } import akka.actor.{ Scheduler, Cancellable } import scala.concurrent.util.Duration import concurrent.ExecutionContext +import scala.concurrent.util.FiniteDuration /** * INTERNAL API */ private[akka] object FixedRateTask { def apply(scheduler: Scheduler, - initalDelay: Duration, - delay: Duration)(f: ⇒ Unit)(implicit executor: ExecutionContext): FixedRateTask = + initalDelay: FiniteDuration, + delay: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): FixedRateTask = new FixedRateTask(scheduler, initalDelay, delay, new Runnable { def run(): Unit = f }) } @@ -28,8 +29,8 @@ private[akka] object FixedRateTask { * initialDelay. */ private[akka] class FixedRateTask(scheduler: Scheduler, - initalDelay: Duration, - delay: Duration, + initalDelay: FiniteDuration, + delay: FiniteDuration, task: Runnable)(implicit executor: ExecutionContext) extends Runnable with Cancellable { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala index c92ff0eafb..ea35249303 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala @@ -19,6 +19,7 @@ import akka.remote.testconductor.RoleName import akka.actor.Props import akka.actor.Actor import akka.cluster.MemberStatus._ +import scala.concurrent.util.FiniteDuration object LargeClusterMultiJvmSpec extends MultiNodeConfig { // each jvm simulates a datacenter with many nodes @@ -122,8 +123,9 @@ abstract class LargeClusterSpec systems foreach { Cluster(_) } } - def expectedMaxDuration(totalNodes: Int): Duration = - 5.seconds + (2.seconds * totalNodes) + def expectedMaxDuration(totalNodes: Int): FiniteDuration = + // this cast will always succeed, but the compiler does not know about it ... + (5.seconds + (2.seconds * totalNodes)).asInstanceOf[FiniteDuration] def joinAll(from: RoleName, to: RoleName, totalNodes: Int, runOnRoles: RoleName*): Unit = { val joiningClusters = systems.map(Cluster(_)).toSet @@ -271,7 +273,7 @@ abstract class LargeClusterSpec val unreachableNodes = nodesPerDatacenter val liveNodes = nodesPerDatacenter * 4 - within(30.seconds + (3.seconds * liveNodes)) { + within((30.seconds + (3.seconds * liveNodes)).asInstanceOf[FiniteDuration]) { val startGossipCounts = Map.empty[Cluster, Long] ++ systems.map(sys ⇒ (Cluster(sys) -> Cluster(sys).readView.latestStats.receivedGossipCount)) def gossipCount(c: Cluster): Long = { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala index 47475fa80f..489c5415ea 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala @@ -4,7 +4,6 @@ package akka.cluster import language.implicitConversions - import com.typesafe.config.Config import com.typesafe.config.ConfigFactory import akka.actor.{ Address, ExtendedActorSystem } @@ -18,6 +17,7 @@ import org.scalatest.exceptions.TestFailedException import java.util.concurrent.ConcurrentHashMap import akka.actor.ActorPath import akka.actor.RootActorPath +import scala.concurrent.util.FiniteDuration object MultiNodeClusterSpec { @@ -174,7 +174,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS def awaitUpConvergence( numberOfMembers: Int, canNotBePartOfMemberRing: Seq[Address] = Seq.empty[Address], - timeout: Duration = 20.seconds): Unit = { + timeout: FiniteDuration = 20.seconds): Unit = { within(timeout) { awaitCond(clusterView.members.size == numberOfMembers) awaitCond(clusterView.members.forall(_.status == MemberStatus.Up)) diff --git a/akka-docs/java/code/docs/actor/japi/FaultHandlingDocSample.java b/akka-docs/java/code/docs/actor/japi/FaultHandlingDocSample.java index 75f102192e..70d3c35142 100644 --- a/akka-docs/java/code/docs/actor/japi/FaultHandlingDocSample.java +++ b/akka-docs/java/code/docs/actor/japi/FaultHandlingDocSample.java @@ -139,7 +139,7 @@ public class FaultHandlingDocSample { if (msg.equals(Start) && progressListener == null) { progressListener = getSender(); getContext().system().scheduler().schedule( - Duration.Zero(), Duration.parse("1 second"), getSelf(), Do, getContext().dispatcher() + Duration.Zero(), Duration.create(1, "second"), getSelf(), Do, getContext().dispatcher() ); } else if (msg.equals(Do)) { counterService.tell(new Increment(1), getSelf()); @@ -299,7 +299,7 @@ public class FaultHandlingDocSample { counter.tell(new UseStorage(null), getSelf()); // Try to re-establish storage after while getContext().system().scheduler().scheduleOnce( - Duration.parse("10 seconds"), getSelf(), Reconnect, getContext().dispatcher() + Duration.create(10, "seconds"), getSelf(), Reconnect, getContext().dispatcher() ); } else if (msg.equals(Reconnect)) { // Re-establish storage after the scheduled delay diff --git a/akka-docs/java/code/docs/future/FutureDocTestBase.java b/akka-docs/java/code/docs/future/FutureDocTestBase.java index 5cc84f9935..ca23065661 100644 --- a/akka-docs/java/code/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/docs/future/FutureDocTestBase.java @@ -518,7 +518,7 @@ public class FutureDocTestBase { //#after final ExecutionContext ec = system.dispatcher(); Future failExc = Futures.failed(new IllegalStateException("OHNOES1")); - Future delayed = Patterns.after(Duration.parse("500 millis"), + Future delayed = Patterns.after(Duration.create(500, "millis"), system.scheduler(), ec, failExc); Future future = future(new Callable() { public String call() throws InterruptedException { diff --git a/akka-docs/java/code/docs/zeromq/ZeromqDocTestBase.java b/akka-docs/java/code/docs/zeromq/ZeromqDocTestBase.java index f7b246bc57..c392cf131f 100644 --- a/akka-docs/java/code/docs/zeromq/ZeromqDocTestBase.java +++ b/akka-docs/java/code/docs/zeromq/ZeromqDocTestBase.java @@ -187,7 +187,7 @@ public class ZeromqDocTestBase { @Override public void preStart() { getContext().system().scheduler() - .schedule(Duration.parse("1 second"), Duration.parse("1 second"), getSelf(), TICK, getContext().dispatcher()); + .schedule(Duration.create(1, "second"), Duration.create(1, "second"), getSelf(), TICK, getContext().dispatcher()); } @Override diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala index 3822647ce8..84b2f17173 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit.MILLISECONDS import akka.util.{ Timeout } import scala.concurrent.util.{ Deadline, Duration } import scala.reflect.classTag +import scala.concurrent.util.FiniteDuration sealed trait Direction { def includes(other: Direction): Boolean @@ -559,7 +560,7 @@ private[akka] class BarrierCoordinator extends Actor with LoggingFSM[BarrierCoor } onTransition { - case Idle -> Waiting ⇒ setTimer("Timeout", StateTimeout, nextStateData.deadline.timeLeft, false) + case Idle -> Waiting ⇒ setTimer("Timeout", StateTimeout, nextStateData.deadline.timeLeft.asInstanceOf[FiniteDuration], false) case Waiting -> Idle ⇒ cancelTimer("Timeout") } @@ -570,7 +571,7 @@ private[akka] class BarrierCoordinator extends Actor with LoggingFSM[BarrierCoor val enterDeadline = getDeadline(timeout) // we only allow the deadlines to get shorter if (enterDeadline.timeLeft < deadline.timeLeft) { - setTimer("Timeout", StateTimeout, enterDeadline.timeLeft, false) + setTimer("Timeout", StateTimeout, enterDeadline.timeLeft.asInstanceOf[FiniteDuration], false) handleBarrier(d.copy(arrived = together, deadline = enterDeadline)) } else handleBarrier(d.copy(arrived = together)) diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala index 6c8352d880..e1d5fb0854 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala @@ -4,20 +4,17 @@ package akka.remote.testconductor import language.postfixOps - import java.net.InetSocketAddress - import scala.annotation.tailrec import scala.collection.immutable.Queue - import org.jboss.netty.buffer.ChannelBuffer import org.jboss.netty.channel.{ SimpleChannelHandler, MessageEvent, Channels, ChannelStateEvent, ChannelHandlerContext, ChannelFutureListener, ChannelFuture } - import akka.actor.{ Props, LoggingFSM, Address, ActorSystem, ActorRef, ActorLogging, Actor, FSM } import akka.event.Logging import akka.remote.netty.ChannelAddress import scala.concurrent.util.Duration import scala.concurrent.util.duration._ +import scala.concurrent.util.FiniteDuration /** * INTERNAL API. @@ -331,20 +328,20 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext) * lead to the correct rate on average, with increased latency of the order of * HWT granularity. */ - private def schedule(d: Data): (Data, Seq[Send], Option[Duration]) = { + private def schedule(d: Data): (Data, Seq[Send], Option[FiniteDuration]) = { val now = System.nanoTime - @tailrec def rec(d: Data, toSend: Seq[Send]): (Data, Seq[Send], Option[Duration]) = { + @tailrec def rec(d: Data, toSend: Seq[Send]): (Data, Seq[Send], Option[FiniteDuration]) = { if (d.queue.isEmpty) (d, toSend, None) else { val timeForPacket = d.lastSent + (1000 * size(d.queue.head.msg) / d.rateMBit).toLong if (timeForPacket <= now) rec(Data(timeForPacket, d.rateMBit, d.queue.tail), toSend :+ d.queue.head) else { val splitThreshold = d.lastSent + packetSplitThreshold.toNanos - if (now < splitThreshold) (d, toSend, Some((timeForPacket - now).nanos min (splitThreshold - now).nanos)) + if (now < splitThreshold) (d, toSend, Some(((timeForPacket - now).nanos min (splitThreshold - now).nanos).asInstanceOf[FiniteDuration])) else { val microsToSend = (now - d.lastSent) / 1000 val (s1, s2) = split(d.queue.head, (microsToSend * d.rateMBit / 8).toInt) - (d.copy(queue = s2 +: d.queue.tail), toSend :+ s1, Some((timeForPacket - now).nanos min packetSplitThreshold)) + (d.copy(queue = s2 +: d.queue.tail), toSend :+ s1, Some(((timeForPacket - now).nanos min packetSplitThreshold).asInstanceOf[FiniteDuration])) } } } diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala index 2b18bdbabb..03b07486f0 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala @@ -278,7 +278,7 @@ private[akka] class PlayerHandler( event.getCause match { case c: ConnectException if reconnects > 0 ⇒ reconnects -= 1 - scheduler.scheduleOnce(nextAttempt.timeLeft)(reconnect()) + scheduler.scheduleOnce(nextAttempt.timeLeft.asInstanceOf[FiniteDuration])(reconnect()) case e ⇒ fsm ! ConnectionFailure(e.getMessage) } } 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 fe1b31ce45..64dc611396 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala @@ -7,6 +7,7 @@ import akka.actor.ActorRefFactory import scala.reflect.ClassTag import scala.concurrent.util.Duration import akka.actor.{ FSM, Actor, ActorRef } +import scala.concurrent.util.FiniteDuration /* * generic typed object buncher. @@ -35,7 +36,7 @@ object GenericBuncher { } } -abstract class GenericBuncher[A: ClassTag, B](val singleTimeout: Duration, val multiTimeout: Duration) +abstract class GenericBuncher[A: ClassTag, B](val singleTimeout: FiniteDuration, val multiTimeout: FiniteDuration) extends Actor with FSM[GenericBuncher.State, B] { import GenericBuncher._ import FSM._ @@ -85,7 +86,7 @@ object Buncher { val Flush = GenericBuncher.Flush } -class Buncher[A: ClassTag](singleTimeout: Duration, multiTimeout: Duration) +class Buncher[A: ClassTag](singleTimeout: FiniteDuration, multiTimeout: FiniteDuration) extends GenericBuncher[A, List[A]](singleTimeout, multiTimeout) { import Buncher._ 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 e97c76e6af..902eb797d2 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -4,11 +4,11 @@ package sample.fsm.dining.fsm import language.postfixOps - import akka.actor._ import akka.actor.FSM._ import scala.concurrent.util.Duration import scala.concurrent.util.duration._ +import scala.concurrent.util.FiniteDuration /* * Some messages for the chopstick @@ -159,7 +159,7 @@ class FSMHakker(name: String, left: ActorRef, right: ActorRef) extends Actor wit // Initialize the hakker initialize - private def startThinking(duration: Duration): State = { + private def startThinking(duration: FiniteDuration): State = { goto(Thinking) using TakenChopsticks(None, None) forMax duration } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index bab4601587..90f3b2f545 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -7,6 +7,7 @@ package akka.testkit import akka.actor._ import scala.concurrent.util.Duration import akka.dispatch.DispatcherPrerequisites +import scala.concurrent.util.FiniteDuration /** * This is a specialised form of the TestActorRef with support for querying and @@ -59,14 +60,14 @@ class TestFSMRef[S, D, T <: Actor]( * corresponding transition initiated from within the FSM, including timeout * and stop handling. */ - def setState(stateName: S = fsm.stateName, stateData: D = fsm.stateData, timeout: Duration = null, stopReason: Option[FSM.Reason] = None) { + def setState(stateName: S = fsm.stateName, stateData: D = fsm.stateData, timeout: FiniteDuration = null, stopReason: Option[FSM.Reason] = None) { fsm.applyState(FSM.State(stateName, stateData, Option(timeout), stopReason)) } /** * Proxy for FSM.setTimer. */ - def setTimer(name: String, msg: Any, timeout: Duration, repeat: Boolean) { + def setTimer(name: String, msg: Any, timeout: FiniteDuration, repeat: Boolean) { fsm.setTimer(name, msg, timeout, repeat) } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala index 1d393afef0..ea7fb82d07 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala @@ -9,6 +9,7 @@ import akka.actor.ActorRef import scala.concurrent.util.duration._ import scala.concurrent.util.Duration import org.zeromq.ZMQ.{ Poller, Socket } +import scala.concurrent.util.FiniteDuration /** * Marker trait representing request messages for zeromq @@ -152,7 +153,7 @@ case class PollDispatcher(name: String) extends SocketMeta * An option containing the duration a poll cycle should wait for a message before it loops * @param duration */ -case class PollTimeoutDuration(duration: Duration = 100 millis) extends SocketMeta +case class PollTimeoutDuration(duration: FiniteDuration = 100 millis) extends SocketMeta /** * Start listening with this server socket on the specified address From 175dd4c547471a47858eb1e11105c2fd7c280659 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 18 Sep 2012 10:49:41 +0200 Subject: [PATCH 24/46] Minor cleanup --- .../src/main/scala/sample/cluster/stats/StatsSample.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala index 1f13d7fa7a..0ba5a19b3e 100644 --- a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala +++ b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala @@ -208,8 +208,8 @@ class StatsSampleClient(servicePath: String) extends Actor { case "tick" if nodes.nonEmpty ⇒ // just pick any one val address = nodes.toIndexedSeq(ThreadLocalRandom.current.nextInt(nodes.size)) - context.actorFor(RootActorPath(address) / servicePathElements) ! - StatsJob("this is the text that will be analyzed") + val service = context.actorFor(RootActorPath(address) / servicePathElements) + service ! StatsJob("this is the text that will be analyzed") case result: StatsResult ⇒ println(result) case failed: JobFailed ⇒ From c0c6cc3931fe4b40176f4a9c04e52649914eef84 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 18 Sep 2012 14:19:38 +0200 Subject: [PATCH 25/46] Publish cluster LeaderChanged only when convergence, see #2518 --- .../scala/akka/cluster/ClusterEvent.scala | 28 ++++-- .../scala/akka/cluster/ClusterReadView.scala | 10 +- .../ClusterDomainEventPublisherSpec.scala | 96 +++++++++++++++++++ .../akka/cluster/ClusterDomainEventSpec.scala | 9 +- .../sample/cluster/stats/StatsSample.scala | 6 +- 5 files changed, 127 insertions(+), 22 deletions(-) create mode 100644 akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala index 4bb0105413..fd7bfa0de9 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala @@ -94,9 +94,9 @@ object ClusterEvent { case class ConvergenceChanged(convergence: Boolean) extends ClusterDomainEvent /** - * Leader of the cluster members changed, and/or convergence status. + * Leader of the cluster members changed. Only published after convergence. */ - case class LeaderChanged(leader: Option[Address], convergence: Boolean) extends ClusterDomainEvent + case class LeaderChanged(leader: Option[Address]) extends ClusterDomainEvent /** * INTERNAL API @@ -150,7 +150,7 @@ object ClusterEvent { val convergenceEvents = if (convergenceChanged) Seq(ConvergenceChanged(newConvergence)) else Seq.empty val leaderEvents = - if (convergenceChanged || newGossip.leader != oldGossip.leader) Seq(LeaderChanged(newGossip.leader, newConvergence)) + if (newGossip.leader != oldGossip.leader) Seq(LeaderChanged(newGossip.leader)) else Seq.empty val newSeenBy = newGossip.seenBy @@ -159,7 +159,7 @@ object ClusterEvent { else Seq.empty memberEvents.toIndexedSeq ++ unreachableEvents ++ downedEvents ++ unreachableDownedEvents ++ removedEvents ++ - convergenceEvents ++ leaderEvents ++ seenEvents + leaderEvents ++ convergenceEvents ++ seenEvents } } @@ -173,6 +173,7 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto import InternalClusterAction._ var latestGossip: Gossip = Gossip() + var stashedLeaderChanged: Option[LeaderChanged] = None def receive = { case PublishChanges(oldGossip, newGossip) ⇒ publishChanges(oldGossip, newGossip) @@ -201,11 +202,22 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto // keep the latestGossip to be sent to new subscribers latestGossip = newGossip diff(oldGossip, newGossip) foreach { event ⇒ - eventStream publish event - // notify DeathWatch about unreachable node event match { - case MemberUnreachable(m) ⇒ eventStream publish AddressTerminated(m.address) - case _ ⇒ + case LeaderChanged(_) if oldGossip.convergence && newGossip.convergence ⇒ + stashedLeaderChanged = None + eventStream publish event + case x: LeaderChanged ⇒ + // publish later, when convergence + stashedLeaderChanged = Some(x) + case ConvergenceChanged(true) ⇒ + stashedLeaderChanged foreach { eventStream publish _ } + stashedLeaderChanged = None + eventStream publish event + case MemberUnreachable(m) ⇒ + eventStream publish event + // notify DeathWatch about unreachable node + eventStream publish AddressTerminated(m.address) + case _ ⇒ eventStream publish event } } } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterReadView.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterReadView.scala index ff827574d7..0aa9e6997e 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterReadView.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterReadView.scala @@ -52,11 +52,11 @@ private[akka] class ClusterReadView(cluster: Cluster) extends Closeable { case event: MemberEvent ⇒ // replace current member with new member (might have different status, only address is used in equals) state = state.copy(members = state.members - event.member + event.member) - case LeaderChanged(leader, convergence) ⇒ state = state.copy(leader = leader, convergence = convergence) - case ConvergenceChanged(convergence) ⇒ state = state.copy(convergence = convergence) - case s: CurrentClusterState ⇒ state = s - case CurrentInternalStats(stats) ⇒ _latestStats = stats - case _ ⇒ // ignore, not interesting + case LeaderChanged(leader) ⇒ state = state.copy(leader = leader) + case ConvergenceChanged(convergence) ⇒ state = state.copy(convergence = convergence) + case s: CurrentClusterState ⇒ state = s + case CurrentInternalStats(stats) ⇒ _latestStats = stats + case _ ⇒ // ignore, not interesting } }).withDispatcher(cluster.settings.UseDispatcher), name = "clusterEventBusListener") } diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala new file mode 100644 index 0000000000..d7c76270f3 --- /dev/null +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala @@ -0,0 +1,96 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import language.postfixOps +import scala.collection.immutable.SortedSet +import scala.concurrent.util.duration._ +import akka.actor.Address +import akka.actor.Props +import akka.cluster.MemberStatus._ +import akka.cluster.InternalClusterAction._ +import akka.cluster.ClusterEvent._ +import akka.testkit.AkkaSpec +import akka.testkit.ImplicitSender + +object ClusterDomainEventPublisherSpec { + val config = """ + akka.cluster.auto-join = off + akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.remote.log-remote-lifecycle-events = off + akka.remote.netty.port = 0 + """ + + case class GossipTo(address: Address) +} + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublisherSpec.config) with ImplicitSender { + import ClusterDomainEventPublisherSpec._ + + val publisher = system.actorOf(Props[ClusterDomainEventPublisher], name = "test-publisher") + val a1 = Member(Address("akka", "sys", "a", 2552), Up) + val b1 = Member(Address("akka", "sys", "b", 2552), Up) + val c1 = Member(Address("akka", "sys", "c", 2552), Joining) + val c2 = Member(Address("akka", "sys", "c", 2552), Up) + val d1 = Member(Address("akka", "sys", "a", 2551), Up) + + val g0 = Gossip(members = SortedSet(a1)).seen(a1.address) + val g1 = Gossip(members = SortedSet(a1, b1, c1)).seen(a1.address).seen(b1.address).seen(c1.address) + val g2 = Gossip(members = SortedSet(a1, b1, c2)).seen(a1.address) + val g3 = g2.seen(b1.address).seen(c2.address) + val g4 = Gossip(members = SortedSet(d1, a1, b1, c2)).seen(a1.address) + val g5 = Gossip(members = SortedSet(d1, a1, b1, c2)).seen(a1.address).seen(b1.address).seen(c2.address).seen(d1.address) + + "ClusterDomainEventPublisher" must { + + "send snapshot when starting subscription" in { + publisher ! PublishChanges(g0, g1) + publisher ! Subscribe(testActor, classOf[ClusterDomainEvent]) + val state = expectMsgType[CurrentClusterState] + state.members must be(g1.members) + state.convergence must be(true) + } + + "publish MemberUp when member status changed to Up" in { + publisher ! PublishChanges(g1, g2) + expectMsg(MemberUp(c2)) + expectMsg(ConvergenceChanged(false)) + expectMsgType[SeenChanged] + } + + "publish convergence true when all seen it" in { + publisher ! PublishChanges(g2, g3) + expectMsg(ConvergenceChanged(true)) + expectMsgType[SeenChanged] + } + + "publish leader changed when new leader and after convergence" in { + publisher ! PublishChanges(g3, g4) + expectMsg(MemberUp(d1)) + expectMsg(ConvergenceChanged(false)) + expectMsgType[SeenChanged] + + publisher ! PublishChanges(g4, g5) + expectMsg(LeaderChanged(Some(d1.address))) + expectMsg(ConvergenceChanged(true)) + expectMsgType[SeenChanged] + + // convergence both before and after + publisher ! PublishChanges(g3, g5) + expectMsg(MemberUp(d1)) + expectMsg(LeaderChanged(Some(d1.address))) + expectMsgType[SeenChanged] + expectNoMsg(1 second) + + // not convergence + publisher ! PublishChanges(g2, g4) + expectMsg(MemberUp(d1)) + expectNoMsg(1 second) + } + + } + +} diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventSpec.scala index 1bbffca3c2..3a4e3ee3a4 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventSpec.scala @@ -69,10 +69,9 @@ class ClusterDomainEventSpec extends WordSpec with MustMatchers { val g1 = Gossip(members = SortedSet(a1, b1, e1)).seen(a1.address).seen(b1.address).seen(e1.address) val g2 = Gossip(members = SortedSet(a1, b1, e1)).seen(a1.address).seen(b1.address) - // LeaderChanged is also published when convergence changed - diff(g1, g2) must be(Seq(ConvergenceChanged(false), LeaderChanged(Some(a1.address), convergence = false), + diff(g1, g2) must be(Seq(ConvergenceChanged(false), SeenChanged(convergence = false, seenBy = Set(a1.address, b1.address)))) - diff(g2, g1) must be(Seq(ConvergenceChanged(true), LeaderChanged(Some(a1.address), convergence = true), + diff(g2, g1) must be(Seq(ConvergenceChanged(true), SeenChanged(convergence = true, seenBy = Set(a1.address, b1.address, e1.address)))) } @@ -81,8 +80,8 @@ class ClusterDomainEventSpec extends WordSpec with MustMatchers { val g2 = Gossip(members = SortedSet(b1, e1), overview = GossipOverview(unreachable = Set(a1))) val g3 = g2.copy(overview = GossipOverview()).seen(b1.address).seen(e1.address) - diff(g1, g2) must be(Seq(MemberUnreachable(a1), LeaderChanged(Some(b1.address), convergence = false))) - diff(g2, g3) must be(Seq(ConvergenceChanged(true), LeaderChanged(Some(b1.address), convergence = true), + diff(g1, g2) must be(Seq(MemberUnreachable(a1), LeaderChanged(Some(b1.address)))) + diff(g2, g3) must be(Seq(ConvergenceChanged(true), SeenChanged(convergence = true, seenBy = Set(b1.address, e1.address)))) } } diff --git a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala index 0ba5a19b3e..351b5d0a48 100644 --- a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala +++ b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala @@ -89,11 +89,9 @@ class StatsFacade extends Actor with ActorLogging { case job: StatsJob ⇒ currentMaster foreach { _ forward job } case state: CurrentClusterState ⇒ - if (state.convergence) - state.leader foreach updateCurrentMaster - case LeaderChanged(Some(leaderAddress), true) ⇒ + state.leader foreach updateCurrentMaster + case LeaderChanged(Some(leaderAddress)) ⇒ updateCurrentMaster(leaderAddress) - case other: LeaderChanged ⇒ // ignore, not convergence } def updateCurrentMaster(leaderAddress: Address): Unit = { From 9ec764561733de6ca805c46772718097f9ed822a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 18 Sep 2012 15:48:25 +0200 Subject: [PATCH 26/46] Support load of extensions without specifying trailing $, see #2496 * Backwards compatible, i.e. still possible to use $ * DynamicAccess tries first with $ in getObjectFor, then falls back to fqcn as is --- akka-actor-tests/src/test/java/akka/actor/JavaExtension.java | 1 + .../src/test/scala/akka/actor/ActorSystemSpec.scala | 5 +++-- akka-actor/src/main/scala/akka/actor/DynamicAccess.scala | 5 ++++- akka-docs/scala/code/docs/extension/ExtensionDocSpec.scala | 2 +- akka-docs/scala/extending-akka.rst | 2 -- .../akka-sample-cluster/src/main/resources/application.conf | 2 +- 6 files changed, 10 insertions(+), 7 deletions(-) diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java index 616675eb77..b87ace5849 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java @@ -64,6 +64,7 @@ public class JavaExtension { @Test public void mustBeAccessible() { + assertTrue(system.hasExtension((TestExtensionId.TestExtensionProvider))); assertSame(system.extension(TestExtensionId.TestExtensionProvider).system, system); assertSame(TestExtensionId.TestExtensionProvider.apply(system).system, system); } diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index 28485c9bad..9c5dd95dc5 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -70,7 +70,7 @@ object ActorSystemSpec { } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension$"]""") with ImplicitSender { +class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension"]""") with ImplicitSender { "An ActorSystem" must { @@ -95,9 +95,10 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt } "support extensions" in { + // TestExtension is configured and should be loaded at startup + system.hasExtension(TestExtension) must be(true) TestExtension(system).system must be === system system.extension(TestExtension).system must be === system - system.hasExtension(TestExtension) must be(true) } "run termination callbacks in order" in { diff --git a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala index 50f5d26177..473b0297fc 100644 --- a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala +++ b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala @@ -85,7 +85,10 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces getClassFor(fqcn) flatMap { c ⇒ createInstanceFor(c, args) } override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = { - getClassFor(fqcn) flatMap { c ⇒ + val classTry = + if (fqcn.endsWith("$")) getClassFor(fqcn) + else getClassFor(fqcn + "$") recoverWith { case NonFatal(_) ⇒ getClassFor(fqcn) } + classTry flatMap { c ⇒ Try { val module = c.getDeclaredField("MODULE$") module.setAccessible(true) diff --git a/akka-docs/scala/code/docs/extension/ExtensionDocSpec.scala b/akka-docs/scala/code/docs/extension/ExtensionDocSpec.scala index c2558fb4f1..a7973826da 100644 --- a/akka-docs/scala/code/docs/extension/ExtensionDocSpec.scala +++ b/akka-docs/scala/code/docs/extension/ExtensionDocSpec.scala @@ -45,7 +45,7 @@ object ExtensionDocSpec { val config = """ //#config akka { - extensions = ["docs.extension.CountExtension$"] + extensions = ["docs.extension.CountExtension"] } //#config """ diff --git a/akka-docs/scala/extending-akka.rst b/akka-docs/scala/extending-akka.rst index 9c890de252..52b7a715c5 100644 --- a/akka-docs/scala/extending-akka.rst +++ b/akka-docs/scala/extending-akka.rst @@ -58,8 +58,6 @@ in the ``akka.extensions`` section of the config you provide to your ``ActorSyst .. includecode:: code/docs/extension/ExtensionDocSpec.scala :include: config -Note that in this case ``CountExtension`` is an object and therefore the class name ends with ``$``. - Applicability ============= diff --git a/akka-samples/akka-sample-cluster/src/main/resources/application.conf b/akka-samples/akka-sample-cluster/src/main/resources/application.conf index ee403ff23d..a54c87510c 100644 --- a/akka-samples/akka-sample-cluster/src/main/resources/application.conf +++ b/akka-samples/akka-sample-cluster/src/main/resources/application.conf @@ -10,7 +10,7 @@ akka { } } - extensions = ["akka.cluster.Cluster$"] + extensions = ["akka.cluster.Cluster"] cluster { seed-nodes = [ From 947cea3a8c321f081b19331650d063eba4151e9f Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 18 Sep 2012 18:17:44 +0200 Subject: [PATCH 27/46] more conversion to FiniteDuration, plus fix the build, see #2521 - also remove resetReceiveTimeout and change receiveTimeout() to return a Duration (which may be Undefined) - also move akka.actor.{cell -> dungeon}, because that is + an adequate name + and does not coincide with the lowercase version of a class name --- .../src/test/scala/akka/actor/IOActor.scala | 6 +- .../scala/akka/actor/ReceiveTimeoutSpec.scala | 5 +- .../{cell => dungeon}/AbstractActorCell.java | 8 +-- .../src/main/scala/akka/actor/ActorCell.scala | 32 +++++----- .../main/scala/akka/actor/ActorSystem.scala | 2 +- .../akka/actor/RepointableActorRef.scala | 2 +- .../src/main/scala/akka/actor/Scheduler.scala | 12 ++-- .../akka/actor/cell/ReceiveTimeout.scala | 59 ------------------- .../actor/{cell => dungeon}/Children.scala | 2 +- .../{cell => dungeon}/ChildrenContainer.scala | 2 +- .../actor/{cell => dungeon}/DeathWatch.scala | 2 +- .../actor/{cell => dungeon}/Dispatch.scala | 2 +- .../{cell => dungeon}/FaultHandling.scala | 7 ++- .../akka/actor/dungeon/ReceiveTimeout.scala | 49 +++++++++++++++ .../src/main/scala/akka/cluster/Cluster.scala | 12 ++-- .../project/migration-guide-2.0.x-2.1.x.rst | 21 +++++++ .../scala/akka/testkit/TestActorRefSpec.scala | 4 +- 17 files changed, 120 insertions(+), 107 deletions(-) rename akka-actor/src/main/java/akka/actor/{cell => dungeon}/AbstractActorCell.java (65%) delete mode 100644 akka-actor/src/main/scala/akka/actor/cell/ReceiveTimeout.scala rename akka-actor/src/main/scala/akka/actor/{cell => dungeon}/Children.scala (99%) rename akka-actor/src/main/scala/akka/actor/{cell => dungeon}/ChildrenContainer.scala (99%) rename akka-actor/src/main/scala/akka/actor/{cell => dungeon}/DeathWatch.scala (99%) rename akka-actor/src/main/scala/akka/actor/{cell => dungeon}/Dispatch.scala (99%) rename akka-actor/src/main/scala/akka/actor/{cell => dungeon}/FaultHandling.scala (98%) create mode 100644 akka-actor/src/main/scala/akka/actor/dungeon/ReceiveTimeout.scala diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index ccb18e60ea..58ffb9c602 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -245,9 +245,9 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { * @return a future containing the result or the last exception before a limit was hit. */ def retry[T](count: Option[Int] = None, - timeout: Option[FiniteDuration] = None, - delay: Option[FiniteDuration] = Some(100 millis), - filter: Option[Throwable ⇒ Boolean] = None)(future: ⇒ Future[T])(implicit executor: ExecutionContext): Future[T] = { + timeout: Option[FiniteDuration] = None, + delay: Option[FiniteDuration] = Some(100 millis), + filter: Option[Throwable ⇒ Boolean] = None)(future: ⇒ Future[T])(implicit executor: ExecutionContext): Future[T] = { val promise = Promise[T]() diff --git a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala index dc5229ae41..a74cbc9839 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala @@ -5,13 +5,12 @@ package akka.actor import language.postfixOps - import akka.testkit._ import scala.concurrent.util.duration._ - import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.Await import java.util.concurrent.TimeoutException +import scala.concurrent.util.Duration @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ReceiveTimeoutSpec extends AkkaSpec { @@ -65,7 +64,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { case ReceiveTimeout ⇒ count.incrementAndGet timeoutLatch.open - context.resetReceiveTimeout() + context.setReceiveTimeout(Duration.Undefined) } })) diff --git a/akka-actor/src/main/java/akka/actor/cell/AbstractActorCell.java b/akka-actor/src/main/java/akka/actor/dungeon/AbstractActorCell.java similarity index 65% rename from akka-actor/src/main/java/akka/actor/cell/AbstractActorCell.java rename to akka-actor/src/main/java/akka/actor/dungeon/AbstractActorCell.java index 2d8c4fbc1e..6735b6e2cb 100644 --- a/akka-actor/src/main/java/akka/actor/cell/AbstractActorCell.java +++ b/akka-actor/src/main/java/akka/actor/dungeon/AbstractActorCell.java @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.actor.cell; +package akka.actor.dungeon; import akka.actor.ActorCell; import akka.util.Unsafe; @@ -14,9 +14,9 @@ final class AbstractActorCell { static { try { - mailboxOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$cell$Dispatch$$_mailboxDoNotCallMeDirectly")); - childrenOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$cell$Children$$_childrenRefsDoNotCallMeDirectly")); - nextNameOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$cell$Children$$_nextNameDoNotCallMeDirectly")); + mailboxOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$dungeon$Dispatch$$_mailboxDoNotCallMeDirectly")); + childrenOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$dungeon$Children$$_childrenRefsDoNotCallMeDirectly")); + nextNameOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("akka$actor$dungeon$Children$$_nextNameDoNotCallMeDirectly")); } catch(Throwable t){ throw new ExceptionInInitializerError(t); } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index c34b9ac05e..2c12dae8e4 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -4,13 +4,13 @@ package akka.actor -import cell.ChildrenContainer.{ WaitingForChildren } import java.io.{ ObjectOutputStream, NotSerializableException } import scala.annotation.tailrec import scala.collection.immutable.TreeSet import scala.concurrent.util.Duration import scala.util.control.NonFatal -import akka.actor.cell.ChildrenContainer +import akka.actor.dungeon.ChildrenContainer +import akka.actor.dungeon.ChildrenContainer.WaitingForChildren import akka.dispatch.{ Watch, Unwatch, Terminate, SystemMessage, Suspend, Supervise, Resume, Recreate, NoMessage, MessageDispatcher, Envelope, Create, ChildTerminated } import akka.event.Logging.{ LogEvent, Debug, Error } import akka.japi.Procedure @@ -55,23 +55,25 @@ trait ActorContext extends ActorRefFactory { def props: Props /** - * Gets the current receive timeout + * Gets the current receive timeout. * When specified, the receive method should be able to handle a 'ReceiveTimeout' message. */ - def receiveTimeout: Option[Duration] + def receiveTimeout: Duration /** - * Defines the default timeout for an initial receive invocation. + * Defines the inactivity timeout after which the sending of a `ReceiveTimeout` message is triggered. * When specified, the receive function should be able to handle a 'ReceiveTimeout' message. * 1 millisecond is the minimum supported timeout. + * + * Please note that the receive timeout might fire and enqueue the `ReceiveTimeout` message right after + * another message was enqueued; hence it is '''not guaranteed''' that upon reception of the receive + * timeout there must have been an idle period beforehand as configured via this method. + * + * Once set, the receive timeout stays in effect (i.e. continues firing repeatedly after inactivity + * periods). Pass in `Duration.Undefined` to switch off this feature. */ def setReceiveTimeout(timeout: Duration): Unit - /** - * Clears the receive timeout, i.e. deactivates this feature. - */ - def resetReceiveTimeout(): Unit - /** * Changes the Actor's behavior to become the new 'Receive' (PartialFunction[Any, Unit]) handler. * Puts the behavior on top of the hotswap stack. @@ -290,11 +292,11 @@ private[akka] class ActorCell( val props: Props, val parent: InternalActorRef) extends UntypedActorContext with Cell - with cell.ReceiveTimeout - with cell.Children - with cell.Dispatch - with cell.DeathWatch - with cell.FaultHandling { + with dungeon.ReceiveTimeout + with dungeon.Children + with dungeon.Dispatch + with dungeon.DeathWatch + with dungeon.FaultHandling { import ActorCell._ diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index bce966b99e..3d3c4b83d4 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -17,7 +17,7 @@ import akka.util._ import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap } import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException } import java.util.concurrent.TimeUnit.MILLISECONDS -import akka.actor.cell.ChildrenContainer +import akka.actor.dungeon.ChildrenContainer import scala.concurrent.util.FiniteDuration import util.{ Failure, Success } diff --git a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala index 3f0539f123..8ca8ab5cb7 100644 --- a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala @@ -12,7 +12,7 @@ import scala.annotation.tailrec import scala.collection.mutable.Queue import scala.concurrent.forkjoin.ThreadLocalRandom -import akka.actor.cell.ChildrenContainer +import akka.actor.dungeon.ChildrenContainer import akka.dispatch.{ Envelope, Supervise, SystemMessage, Terminate } import akka.event.Logging.Warning import akka.util.Unsafe diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index a3822d90ce..c210bc0976 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -30,13 +30,13 @@ trait Scheduler { * Schedules a message to be sent repeatedly with an initial delay and * frequency. E.g. if you would like a message to be sent immediately and * thereafter every 500ms you would set delay=Duration.Zero and - * frequency=Duration(500, TimeUnit.MILLISECONDS) + * interval=Duration(500, TimeUnit.MILLISECONDS) * * Java & Scala API */ def schedule( initialDelay: FiniteDuration, - frequency: FiniteDuration, + interval: FiniteDuration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable @@ -44,23 +44,23 @@ trait Scheduler { * Schedules a function to be run repeatedly with an initial delay and a * frequency. E.g. if you would like the function to be run after 2 seconds * and thereafter every 100ms you would set delay = Duration(2, TimeUnit.SECONDS) - * and frequency = Duration(100, TimeUnit.MILLISECONDS) + * and interval = Duration(100, TimeUnit.MILLISECONDS) * * Scala API */ def schedule( - initialDelay: FiniteDuration, frequency: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable + initialDelay: FiniteDuration, interval: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable /** * Schedules a function to be run repeatedly with an initial delay and * a frequency. E.g. if you would like the function to be run after 2 * seconds and thereafter every 100ms you would set delay = Duration(2, - * TimeUnit.SECONDS) and frequency = Duration(100, TimeUnit.MILLISECONDS) + * TimeUnit.SECONDS) and interval = Duration(100, TimeUnit.MILLISECONDS) * * Java API */ def schedule( - initialDelay: FiniteDuration, frequency: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable + initialDelay: FiniteDuration, interval: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable /** * Schedules a Runnable to be run once with a delay, i.e. a time period that diff --git a/akka-actor/src/main/scala/akka/actor/cell/ReceiveTimeout.scala b/akka-actor/src/main/scala/akka/actor/cell/ReceiveTimeout.scala deleted file mode 100644 index 6ff6a39dea..0000000000 --- a/akka-actor/src/main/scala/akka/actor/cell/ReceiveTimeout.scala +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.actor.cell - -import ReceiveTimeout.emptyReceiveTimeoutData -import akka.actor.ActorCell -import akka.actor.ActorCell.emptyCancellable -import akka.actor.Cancellable -import scala.concurrent.util.Duration -import scala.concurrent.util.FiniteDuration - -private[akka] object ReceiveTimeout { - final val emptyReceiveTimeoutData: (Option[FiniteDuration], Cancellable) = (None, ActorCell.emptyCancellable) -} - -private[akka] trait ReceiveTimeout { this: ActorCell ⇒ - - import ReceiveTimeout._ - import ActorCell._ - - private var receiveTimeoutData: (Option[FiniteDuration], Cancellable) = emptyReceiveTimeoutData - - final def receiveTimeout: Option[FiniteDuration] = receiveTimeoutData._1 - - final def setReceiveTimeout(timeout: Option[FiniteDuration]): Unit = - receiveTimeoutData = receiveTimeoutData.copy(_1 = timeout) - - final def setReceiveTimeout(timeout: Duration): Unit = { - import Duration._ - setReceiveTimeout(timeout match { - case x if x eq Undefined ⇒ None - case Inf | MinusInf ⇒ throw new IllegalArgumentException("receiveTimeout cannot be infinite") - case f: FiniteDuration ⇒ - if (f < Zero) throw new IllegalArgumentException("receiveTimeout cannot be negative") - else Some(f) - }) - } - - final def resetReceiveTimeout(): Unit = setReceiveTimeout(None) - - final def checkReceiveTimeout() { - val recvtimeout = receiveTimeoutData - if (recvtimeout._1.isDefined && !mailbox.hasMessages) { - recvtimeout._2.cancel() //Cancel any ongoing future - //Only reschedule if desired and there are currently no more messages to be processed - receiveTimeoutData = (recvtimeout._1, system.scheduler.scheduleOnce(recvtimeout._1.get, self, akka.actor.ReceiveTimeout)(this.dispatcher)) - } else cancelReceiveTimeout() - - } - - final def cancelReceiveTimeout(): Unit = - if (receiveTimeoutData._2 ne emptyCancellable) { - receiveTimeoutData._2.cancel() - receiveTimeoutData = (receiveTimeoutData._1, emptyCancellable) - } - -} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/cell/Children.scala b/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala similarity index 99% rename from akka-actor/src/main/scala/akka/actor/cell/Children.scala rename to akka-actor/src/main/scala/akka/actor/dungeon/Children.scala index 7fa8eceece..85dfd7095a 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/Children.scala +++ b/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.actor.cell +package akka.actor.dungeon import scala.annotation.tailrec import scala.collection.JavaConverters.asJavaIterableConverter diff --git a/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala b/akka-actor/src/main/scala/akka/actor/dungeon/ChildrenContainer.scala similarity index 99% rename from akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala rename to akka-actor/src/main/scala/akka/actor/dungeon/ChildrenContainer.scala index d0bf76953a..eeb28cf018 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala +++ b/akka-actor/src/main/scala/akka/actor/dungeon/ChildrenContainer.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.actor.cell +package akka.actor.dungeon import scala.collection.immutable.TreeMap diff --git a/akka-actor/src/main/scala/akka/actor/cell/DeathWatch.scala b/akka-actor/src/main/scala/akka/actor/dungeon/DeathWatch.scala similarity index 99% rename from akka-actor/src/main/scala/akka/actor/cell/DeathWatch.scala rename to akka-actor/src/main/scala/akka/actor/dungeon/DeathWatch.scala index f994e956c6..5407afc2c8 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/DeathWatch.scala +++ b/akka-actor/src/main/scala/akka/actor/dungeon/DeathWatch.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.actor.cell +package akka.actor.dungeon import akka.actor.{ Terminated, InternalActorRef, ActorRef, ActorCell, Actor, Address, AddressTerminated } import akka.dispatch.{ Watch, Unwatch } diff --git a/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala b/akka-actor/src/main/scala/akka/actor/dungeon/Dispatch.scala similarity index 99% rename from akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala rename to akka-actor/src/main/scala/akka/actor/dungeon/Dispatch.scala index 0f3619e208..e071c1605d 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala +++ b/akka-actor/src/main/scala/akka/actor/dungeon/Dispatch.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.actor.cell +package akka.actor.dungeon import scala.annotation.tailrec import akka.actor.{ ActorRef, ActorCell } diff --git a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala similarity index 98% rename from akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala rename to akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala index 383db82b70..f7c06032c8 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2012 Typesafe Inc. */ -package akka.actor.cell +package akka.actor.dungeon import scala.annotation.tailrec import akka.actor.{ PreRestartException, PostRestartException, InternalActorRef, Failed, ActorRef, ActorInterruptedException, ActorCell, Actor } @@ -16,6 +16,7 @@ import akka.actor.PreRestartException import akka.actor.Failed import akka.actor.PostRestartException import akka.event.Logging.Debug +import scala.concurrent.util.Duration private[akka] trait FaultHandling { this: ActorCell ⇒ @@ -121,7 +122,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ assert(mailbox.isSuspended, "mailbox must be suspended during failed creation, status=" + mailbox.status) assert(perpetrator == self) - setReceiveTimeout(None) + setReceiveTimeout(Duration.Undefined) cancelReceiveTimeout // stop all children, which will turn childrenRefs into TerminatingChildrenContainer (if there are children) @@ -137,7 +138,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ } protected def terminate() { - setReceiveTimeout(None) + setReceiveTimeout(Duration.Undefined) cancelReceiveTimeout // stop all children, which will turn childrenRefs into TerminatingChildrenContainer (if there are children) diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/ReceiveTimeout.scala b/akka-actor/src/main/scala/akka/actor/dungeon/ReceiveTimeout.scala new file mode 100644 index 0000000000..0c3661b59a --- /dev/null +++ b/akka-actor/src/main/scala/akka/actor/dungeon/ReceiveTimeout.scala @@ -0,0 +1,49 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.actor.dungeon + +import ReceiveTimeout.emptyReceiveTimeoutData +import akka.actor.ActorCell +import akka.actor.ActorCell.emptyCancellable +import akka.actor.Cancellable +import scala.concurrent.util.Duration +import scala.concurrent.util.FiniteDuration + +private[akka] object ReceiveTimeout { + final val emptyReceiveTimeoutData: (Duration, Cancellable) = (Duration.Undefined, ActorCell.emptyCancellable) +} + +private[akka] trait ReceiveTimeout { this: ActorCell ⇒ + + import ReceiveTimeout._ + import ActorCell._ + + private var receiveTimeoutData: (Duration, Cancellable) = emptyReceiveTimeoutData + + final def receiveTimeout: Duration = receiveTimeoutData._1 + + final def setReceiveTimeout(timeout: Duration): Unit = receiveTimeoutData = receiveTimeoutData.copy(_1 = timeout) + + final def checkReceiveTimeout() { + val recvtimeout = receiveTimeoutData + //Only reschedule if desired and there are currently no more messages to be processed + if (!mailbox.hasMessages) recvtimeout._1 match { + case f: FiniteDuration ⇒ + recvtimeout._2.cancel() //Cancel any ongoing future + val task = system.scheduler.scheduleOnce(f, self, akka.actor.ReceiveTimeout)(this.dispatcher) + receiveTimeoutData = (f, task) + case _ ⇒ cancelReceiveTimeout() + } + else cancelReceiveTimeout() + + } + + final def cancelReceiveTimeout(): Unit = + if (receiveTimeoutData._2 ne emptyCancellable) { + receiveTimeoutData._2.cancel() + receiveTimeoutData = (receiveTimeoutData._1, emptyCancellable) + } + +} \ No newline at end of file diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 9d9e955272..f924dc20de 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -109,16 +109,16 @@ class Cluster(val system: ExtendedActorSystem) extends Extension { new Scheduler with Closeable { override def close(): Unit = () // we are using system.scheduler, which we are not responsible for closing - override def schedule(initialDelay: FiniteDuration, frequency: FiniteDuration, + override def schedule(initialDelay: FiniteDuration, interval: FiniteDuration, receiver: ActorRef, message: Any)(implicit executor: ExecutionContext): Cancellable = - systemScheduler.schedule(initialDelay, frequency, receiver, message) + systemScheduler.schedule(initialDelay, interval, receiver, message) - override def schedule(initialDelay: FiniteDuration, frequency: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = - systemScheduler.schedule(initialDelay, frequency)(f) + override def schedule(initialDelay: FiniteDuration, interval: FiniteDuration)(f: ⇒ Unit)(implicit executor: ExecutionContext): Cancellable = + systemScheduler.schedule(initialDelay, interval)(f) - override def schedule(initialDelay: FiniteDuration, frequency: FiniteDuration, + override def schedule(initialDelay: FiniteDuration, interval: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = - systemScheduler.schedule(initialDelay, frequency, runnable) + systemScheduler.schedule(initialDelay, interval, runnable) override def scheduleOnce(delay: FiniteDuration, runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = diff --git a/akka-docs/project/migration-guide-2.0.x-2.1.x.rst b/akka-docs/project/migration-guide-2.0.x-2.1.x.rst index 2444e4c791..94f3d65b7f 100644 --- a/akka-docs/project/migration-guide-2.0.x-2.1.x.rst +++ b/akka-docs/project/migration-guide-2.0.x-2.1.x.rst @@ -385,3 +385,24 @@ v2.1:: final FiniteDuration d = Duration.create("1 second"); final Timeout t = new Timeout(d); // always required finite duration, now also in type +Actor Receive Timeout +===================== + +The API for setting and querying the receive timeout has been made more +consisten in always taking and returning a ``Duration``, the wrapping in +``Option`` has been removed. + +v2.0:: + + getContext().setReceiveTimeout(Duration.create(10, SECONDS)); + final Option timeout = getContext().receiveTimeout(); + final isSet = timeout.isDefined(); + resetReceiveTimeout(); + +v2.1:: + + getContext().setReceiveTimeout(Duration.create(10, SECONDS)); + final Duration timeout = getContext().receiveTimeout(); + final isSet = timeout.isFinite(); + getContext().setReceiveTimeout(Duration.Undefined()); + diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index f033df302f..2e835b408f 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -4,7 +4,6 @@ package akka.testkit import language.{ postfixOps, reflectiveCalls } - import org.scalatest.matchers.MustMatchers import org.scalatest.{ BeforeAndAfterEach, WordSpec } import akka.actor._ @@ -14,6 +13,7 @@ import scala.concurrent.util.duration._ import akka.actor.ActorSystem import akka.pattern.ask import akka.dispatch.Dispatcher +import scala.concurrent.util.Duration /** * Test whether TestActorRef behaves as an ActorRef should, besides its own spec. @@ -244,7 +244,7 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA "set receiveTimeout to None" in { val a = TestActorRef[WorkerActor] - a.underlyingActor.context.receiveTimeout must be(None) + a.underlyingActor.context.receiveTimeout must be theSameInstanceAs Duration.Undefined } "set CallingThreadDispatcher" in { From 63607d4914d9d05952800915ff5a8e6a7d522500 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 18 Sep 2012 23:57:25 +0200 Subject: [PATCH 28/46] Minor Serialization performance improvement. --- .../src/main/scala/akka/serialization/Serialization.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index b9d6298784..003c9de2b1 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -9,7 +9,6 @@ import com.typesafe.config.Config import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess } import akka.event.Logging import java.util.concurrent.ConcurrentHashMap -import scala.util.control.NonFatal import scala.collection.mutable.ArrayBuffer import java.io.NotSerializableException import util.{ Try, DynamicVariable } @@ -100,8 +99,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { // bindings are ordered from most specific to least specific def unique(possibilities: Seq[(Class[_], Serializer)]): Boolean = possibilities.size == 1 || - (possibilities map (_._1) forall (_ isAssignableFrom possibilities(0)._1)) || - (possibilities map (_._2) forall (_ == possibilities(0)._2)) + (possibilities forall (_._1 isAssignableFrom possibilities(0)._1)) || + (possibilities forall (_._2 == possibilities(0)._2)) val ser = bindings filter { _._1 isAssignableFrom clazz } match { case Seq() ⇒ From 374c65d1e36f32ff557cf796ae4b701e07331179 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 19 Sep 2012 08:33:48 +0200 Subject: [PATCH 29/46] NonFatal not needed in Try recoverWith, see #2496 --- akka-actor/src/main/scala/akka/actor/DynamicAccess.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala index 473b0297fc..2929032dc5 100644 --- a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala +++ b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala @@ -87,7 +87,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = { val classTry = if (fqcn.endsWith("$")) getClassFor(fqcn) - else getClassFor(fqcn + "$") recoverWith { case NonFatal(_) ⇒ getClassFor(fqcn) } + else getClassFor(fqcn + "$") recoverWith { case _ ⇒ getClassFor(fqcn) } classTry flatMap { c ⇒ Try { val module = c.getDeclaredField("MODULE$") From 718686e2f2d266dff3ae595ad230684867868934 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 19 Sep 2012 10:18:55 +0200 Subject: [PATCH 30/46] Add another test case for publish of LeaderChanged, see #2518 * It didn't handle convergence changes with same leader correctly --- .../scala/akka/cluster/ClusterEvent.scala | 19 +++++-- .../ClusterDomainEventPublisherSpec.scala | 51 ++++++++++++++----- 2 files changed, 53 insertions(+), 17 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala index fd7bfa0de9..17988fd7ab 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala @@ -174,6 +174,7 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto var latestGossip: Gossip = Gossip() var stashedLeaderChanged: Option[LeaderChanged] = None + var publishedLeaderChanged: Option[LeaderChanged] = None def receive = { case PublishChanges(oldGossip, newGossip) ⇒ publishChanges(oldGossip, newGossip) @@ -203,16 +204,26 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto latestGossip = newGossip diff(oldGossip, newGossip) foreach { event ⇒ event match { - case LeaderChanged(_) if oldGossip.convergence && newGossip.convergence ⇒ + case x @ LeaderChanged(_) if Some(x) == publishedLeaderChanged ⇒ + // skip, this leader has already been published + + case x @ LeaderChanged(_) if oldGossip.convergence && newGossip.convergence ⇒ stashedLeaderChanged = None - eventStream publish event + publishedLeaderChanged = Some(x) + eventStream publish x + case x: LeaderChanged ⇒ // publish later, when convergence stashedLeaderChanged = Some(x) + case ConvergenceChanged(true) ⇒ - stashedLeaderChanged foreach { eventStream publish _ } - stashedLeaderChanged = None + stashedLeaderChanged foreach { + publishedLeaderChanged = stashedLeaderChanged + stashedLeaderChanged = None + eventStream publish _ + } eventStream publish event + case MemberUnreachable(m) ⇒ eventStream publish event // notify DeathWatch about unreachable node diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala index d7c76270f3..c29f237be7 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterDomainEventPublisherSpec.scala @@ -7,6 +7,7 @@ package akka.cluster import language.postfixOps import scala.collection.immutable.SortedSet import scala.concurrent.util.duration._ +import org.scalatest.BeforeAndAfterEach import akka.actor.Address import akka.actor.Props import akka.cluster.MemberStatus._ @@ -14,6 +15,7 @@ import akka.cluster.InternalClusterAction._ import akka.cluster.ClusterEvent._ import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender +import akka.actor.ActorRef object ClusterDomainEventPublisherSpec { val config = """ @@ -27,10 +29,11 @@ object ClusterDomainEventPublisherSpec { } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublisherSpec.config) with ImplicitSender { +class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublisherSpec.config) + with BeforeAndAfterEach with ImplicitSender { import ClusterDomainEventPublisherSpec._ - val publisher = system.actorOf(Props[ClusterDomainEventPublisher], name = "test-publisher") + var publisher: ActorRef = _ val a1 = Member(Address("akka", "sys", "a", 2552), Up) val b1 = Member(Address("akka", "sys", "b", 2552), Up) val c1 = Member(Address("akka", "sys", "c", 2552), Joining) @@ -44,15 +47,18 @@ class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublish val g4 = Gossip(members = SortedSet(d1, a1, b1, c2)).seen(a1.address) val g5 = Gossip(members = SortedSet(d1, a1, b1, c2)).seen(a1.address).seen(b1.address).seen(c2.address).seen(d1.address) - "ClusterDomainEventPublisher" must { + override def beforeEach(): Unit = { + publisher = system.actorOf(Props[ClusterDomainEventPublisher]) + publisher ! Subscribe(testActor, classOf[ClusterDomainEvent]) + expectMsgType[CurrentClusterState] + } - "send snapshot when starting subscription" in { - publisher ! PublishChanges(g0, g1) - publisher ! Subscribe(testActor, classOf[ClusterDomainEvent]) - val state = expectMsgType[CurrentClusterState] - state.members must be(g1.members) - state.convergence must be(true) - } + override def afterEach(): Unit = { + publisher ! Unsubscribe(testActor) + system.stop(publisher) + } + + "ClusterDomainEventPublisher" must { "publish MemberUp when member status changed to Up" in { publisher ! PublishChanges(g1, g2) @@ -67,30 +73,49 @@ class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublish expectMsgType[SeenChanged] } - "publish leader changed when new leader and after convergence" in { + "publish leader changed when new leader after convergence" in { publisher ! PublishChanges(g3, g4) expectMsg(MemberUp(d1)) expectMsg(ConvergenceChanged(false)) expectMsgType[SeenChanged] + expectNoMsg(1 second) publisher ! PublishChanges(g4, g5) expectMsg(LeaderChanged(Some(d1.address))) expectMsg(ConvergenceChanged(true)) expectMsgType[SeenChanged] + } + "publish leader changed when new leader and convergence both before and after" in { // convergence both before and after publisher ! PublishChanges(g3, g5) expectMsg(MemberUp(d1)) expectMsg(LeaderChanged(Some(d1.address))) expectMsgType[SeenChanged] - expectNoMsg(1 second) + } - // not convergence + "not publish leader changed when not convergence" in { publisher ! PublishChanges(g2, g4) expectMsg(MemberUp(d1)) expectNoMsg(1 second) } + "not publish leader changed when changed convergence but still same leader" in { + publisher ! PublishChanges(g2, g5) + expectMsg(MemberUp(d1)) + expectMsg(LeaderChanged(Some(d1.address))) + expectMsg(ConvergenceChanged(true)) + expectMsgType[SeenChanged] + + publisher ! PublishChanges(g5, g4) + expectMsg(ConvergenceChanged(false)) + expectMsgType[SeenChanged] + + publisher ! PublishChanges(g4, g5) + expectMsg(ConvergenceChanged(true)) + expectMsgType[SeenChanged] + } + } } From d0014c06bc44d3047949174264a8d3f70c6c565b Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 19 Sep 2012 14:42:45 +0200 Subject: [PATCH 31/46] clarify migration doc re: java samples --- akka-docs/project/migration-guide-2.0.x-2.1.x.rst | 2 ++ 1 file changed, 2 insertions(+) diff --git a/akka-docs/project/migration-guide-2.0.x-2.1.x.rst b/akka-docs/project/migration-guide-2.0.x-2.1.x.rst index 94f3d65b7f..06393bdc83 100644 --- a/akka-docs/project/migration-guide-2.0.x-2.1.x.rst +++ b/akka-docs/project/migration-guide-2.0.x-2.1.x.rst @@ -392,6 +392,8 @@ The API for setting and querying the receive timeout has been made more consisten in always taking and returning a ``Duration``, the wrapping in ``Option`` has been removed. +(Samples for Java, Scala sources are affected in exactly the same way.) + v2.0:: getContext().setReceiveTimeout(Duration.create(10, SECONDS)); From db17e5c5be2219c71f41ff30cb084870ac858dc1 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Sep 2012 15:04:27 +0200 Subject: [PATCH 32/46] #2402 - Cleaning up repositories used by the AkkaBuild --- project/AkkaBuild.scala | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 5cf2c076a0..895f21ed5a 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -349,11 +349,17 @@ object AkkaBuild extends Build { // Settings - override lazy val settings = super.settings ++ buildSettings ++ Seq( - resolvers += "Scala Community 2.10.0-SNAPSHOT" at "https://scala-webapps.epfl.ch/jenkins/job/community-nightly/ws/target/repositories/8e83577d99af1d718fe369c4a4ee92737b9cf669", - resolvers += "Sonatype Snapshot Repo" at "https://oss.sonatype.org/content/repositories/snapshots/", - resolvers += "Sonatype Releases Repo" at "https://oss.sonatype.org/content/repositories/releases/", - shellPrompt := { s => Project.extract(s).currentProject.id + " > " } + override lazy val settings = + super.settings ++ + buildSettings ++ + Seq( + shellPrompt := { s => Project.extract(s).currentProject.id + " > " }, + resolvers <<= (resolvers, scalaVersion) apply { + case (res, "2.10.0-SNAPSHOT") => + res :+ ("Scala Community 2.10.0-SNAPSHOT" at "https://scala-webapps.epfl.ch/jenkins/job/community-nightly/ws/target/repositories/8e83577d99af1d718fe369c4a4ee92737b9cf669") + case (res, _) => + res + } ) lazy val baseSettings = Defaults.defaultSettings ++ Publish.settings @@ -418,8 +424,6 @@ object AkkaBuild extends Build { } lazy val defaultSettings = baseSettings ++ formatSettings ++ mimaSettings ++ Seq( - resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/", - // compile options scalacOptions in Compile ++= Seq("-encoding", "UTF-8", "-target:jvm-1.6", "-deprecation", "-feature", "-unchecked", "-Xlog-reflective-calls", "-Ywarn-adapted-args"), javacOptions in Compile ++= Seq("-source", "1.6", "-target", "1.6", "-Xlint:unchecked", "-Xlint:deprecation"), From 8ee48e7a051933a669daa1858ebbeb55d80acf45 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 19 Sep 2012 15:50:33 +0200 Subject: [PATCH 33/46] Remove duplicated multi-node-log-replace, it's in scripts/ --- project/scripts/multi-node-log-replace | 11 ----------- 1 file changed, 11 deletions(-) delete mode 100755 project/scripts/multi-node-log-replace diff --git a/project/scripts/multi-node-log-replace b/project/scripts/multi-node-log-replace deleted file mode 100755 index 83f1b8a136..0000000000 --- a/project/scripts/multi-node-log-replace +++ /dev/null @@ -1,11 +0,0 @@ -#!/usr/bin/env bash -# -# Utility to make log files from multi-node tests easier to analyze. -# Replaces jvm names and host:port with corresponding logical role name. -# - - -# check for an sbt command -type -P sbt &> /dev/null || fail "sbt command not found" - -sbt "project akka-remote-tests" "test:run-main akka.remote.testkit.LogRoleReplace $1 $2" \ No newline at end of file From d0c71d563e84f81164db660e92da65a094eeed2b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Sep 2012 17:32:54 +0200 Subject: [PATCH 34/46] Removing compilation warnings. --- .../internal/component/DurationConverterTest.scala | 2 ++ .../test/scala/akka/dataflow/DataflowSpec.scala | 14 +++++++------- .../docs/actor/mailbox/DurableMailboxDocSpec.scala | 3 +-- .../scala/code/docs/testkit/TestkitDocSpec.scala | 4 +--- .../akka/remote/RemoteCommunicationSpec.scala | 3 ++- .../akka/remote/Ticket1978CommunicationSpec.scala | 3 ++- 6 files changed, 15 insertions(+), 14 deletions(-) diff --git a/akka-camel/src/test/scala/akka/camel/internal/component/DurationConverterTest.scala b/akka-camel/src/test/scala/akka/camel/internal/component/DurationConverterTest.scala index 4134ed35bc..c66a77c287 100644 --- a/akka-camel/src/test/scala/akka/camel/internal/component/DurationConverterTest.scala +++ b/akka-camel/src/test/scala/akka/camel/internal/component/DurationConverterTest.scala @@ -4,6 +4,8 @@ package akka.camel.internal.component +import language.postfixOps + import org.scalatest.matchers.MustMatchers import scala.concurrent.util.duration._ import scala.concurrent.util.Duration diff --git a/akka-dataflow/src/test/scala/akka/dataflow/DataflowSpec.scala b/akka-dataflow/src/test/scala/akka/dataflow/DataflowSpec.scala index 5024e911b6..302d245930 100644 --- a/akka-dataflow/src/test/scala/akka/dataflow/DataflowSpec.scala +++ b/akka-dataflow/src/test/scala/akka/dataflow/DataflowSpec.scala @@ -5,7 +5,7 @@ package akka.dataflow import language.postfixOps -import scala.reflect.ClassTag +import scala.reflect.{ ClassTag, classTag } import akka.actor.{ Actor, Status, Props } import akka.actor.Status._ import akka.pattern.ask @@ -45,7 +45,7 @@ class DataflowSpec extends AkkaSpec with DefaultTimeout { val actor = system.actorOf(Props[TestActor]) val x = Future("Hello") - val y = x flatMap (actor ? _) mapTo manifest[String] + val y = x flatMap (actor ? _) mapTo classTag[String] val r = flow(x() + " " + y() + "!") @@ -268,11 +268,11 @@ class DataflowSpec extends AkkaSpec with DefaultTimeout { x.length + y() } - assert(checkType(rString, manifest[String])) - assert(checkType(rInt, manifest[Int])) - assert(!checkType(rInt, manifest[String])) - assert(!checkType(rInt, manifest[Nothing])) - assert(!checkType(rInt, manifest[Any])) + assert(checkType(rString, classTag[String])) + assert(checkType(rInt, classTag[Int])) + assert(!checkType(rInt, classTag[String])) + //assert(!checkType(rInt, classTag[Nothing])) This test does not work with classTags, but works with Manifests + assert(!checkType(rInt, classTag[Any])) Await.result(rString, timeout.duration) Await.result(rInt, timeout.duration) diff --git a/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocSpec.scala b/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocSpec.scala index f93a213a15..479236b9fe 100644 --- a/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocSpec.scala +++ b/akka-docs/modules/code/docs/actor/mailbox/DurableMailboxDocSpec.scala @@ -60,8 +60,7 @@ class MyMailboxType(systemSettings: ActorSystem.Settings, config: Config) override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = (owner zip system) headOption match { case Some((o, s: ExtendedActorSystem)) ⇒ new MyMessageQueue(o, s) - case None ⇒ throw new IllegalArgumentException( - "requires an owner (i.e. does not work with BalancingDispatcher)") + case _ ⇒ throw new IllegalArgumentException("requires an owner (i.e. does not work with BalancingDispatcher)") } } diff --git a/akka-docs/scala/code/docs/testkit/TestkitDocSpec.scala b/akka-docs/scala/code/docs/testkit/TestkitDocSpec.scala index fcb9360559..8a78fc8c7a 100644 --- a/akka-docs/scala/code/docs/testkit/TestkitDocSpec.scala +++ b/akka-docs/scala/code/docs/testkit/TestkitDocSpec.scala @@ -129,9 +129,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { val actorRef = TestActorRef(new MyActor) // hypothetical message stimulating a '42' answer val future = actorRef ? Say42 - val result = future.value.get match { - case Success(x: Int) ⇒ x - } + val Success(result: Int) = future.value.get result must be(42) //#test-behavior } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index 1f0badebd2..5a43e90443 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -8,6 +8,7 @@ import akka.actor._ import com.typesafe.config._ import scala.concurrent.Future import scala.concurrent.Await +import scala.reflect.classTag import akka.pattern.ask object RemoteCommunicationSpec { @@ -133,7 +134,7 @@ akka { "not fail ask across node boundaries" in { import system.dispatcher - val f = for (_ ← 1 to 1000) yield here ? "ping" mapTo manifest[(String, ActorRef)] + val f = for (_ ← 1 to 1000) yield here ? "ping" mapTo classTag[(String, ActorRef)] Await.result(Future.sequence(f), remaining).map(_._1).toSet must be(Set("pong")) } diff --git a/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala index dfa47665de..b6d2bed02a 100644 --- a/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/Ticket1978CommunicationSpec.scala @@ -9,6 +9,7 @@ import akka.testkit._ import akka.actor._ import com.typesafe.config._ import scala.concurrent.Future +import scala.reflect.classTag import akka.pattern.ask import java.io.File import java.security.{ NoSuchAlgorithmException, SecureRandom, PrivilegedAction, AccessController } @@ -144,7 +145,7 @@ abstract class Ticket1978CommunicationSpec(val cipherConfig: CipherConfig) exten import system.dispatcher val here = system.actorFor(otherAddress.toString + "/user/echo") - val f = for (i ← 1 to 1000) yield here ? (("ping", i)) mapTo manifest[((String, Int), ActorRef)] + val f = for (i ← 1 to 1000) yield here ? (("ping", i)) mapTo classTag[((String, Int), ActorRef)] Await.result(Future.sequence(f), timeout.duration).map(_._1._1).toSet must be(Set("pong")) } From 7b06ee57057f6e2a2c6bfe3eba6e92019c0684f1 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Wed, 19 Sep 2012 16:46:39 +0200 Subject: [PATCH 35/46] Cluster doc of JMX and admin script, see #2014 --- akka-docs/cluster/cluster-usage.rst | 64 +++++++++++++++++++++- akka-kernel/src/main/dist/bin/akka-cluster | 32 +++++------ 2 files changed, 78 insertions(+), 18 deletions(-) diff --git a/akka-docs/cluster/cluster-usage.rst b/akka-docs/cluster/cluster-usage.rst index 9e079796a7..6087e1b637 100644 --- a/akka-docs/cluster/cluster-usage.rst +++ b/akka-docs/cluster/cluster-usage.rst @@ -113,7 +113,7 @@ You can disable automatic joining with configuration: akka.cluster.auto-join = off -Then you need to join manually, using JMX or the provided script. +Then you need to join manually, using :ref:`cluster_jmx` or :ref:`cluster_command_line`. You can join to any node in the cluster. It doesn't have to be configured as seed node. If you are not using auto-join there is no need to configure seed nodes at all. @@ -128,7 +128,8 @@ When a member is considered by the failure detector to be unreachable the leader is not allowed to perform its duties, such as changing status of new joining members to 'Up'. The status of the unreachable member must be changed to 'Down'. This can be performed automatically or manually. By -default it must be done manually, using using JMX or the provided script. +default it must be done manually, using using :ref:`cluster_jmx` or +:ref:`cluster_command_line`. It can also be performed programatically with ``Cluster(system).down``. @@ -338,6 +339,65 @@ service nodes and 1 client:: .. note:: The above example, especially the last part, will be simplified when the cluster handles automatic actor partitioning. +.. _cluster_jmx: + +JMX +^^^ + +Information and management of the cluster is available as JMX MBeans with the root name ``akka.Cluster``. +The JMX information can be displayed with an ordinary JMX console such as JConsole or JVisualVM. + +From JMX you can: + +* see what members that are part of the cluster +* see status of this node +* join this node to another node in cluster +* mark any node in the cluster as down +* tell any node in the cluster to leave + +Member nodes are identified with their address, in format `akka://actor-system-name@hostname:port`. + +.. _cluster_command_line: + +Command Line Management +^^^^^^^^^^^^^^^^^^^^^^^ + +The cluster can be managed with the script `bin/akka-cluster` provided in the +Akka distribution. + +Run it without parameters to see instructions about how to use the script:: + + Usage: bin/akka-cluster ... + + Supported commands are: + join - Sends request a JOIN node with the specified URL + leave - Sends a request for node with URL to LEAVE the cluster + down - Sends a request for marking node with URL as DOWN + member-status - Asks the member node for its current status + cluster-status - Asks the cluster for its current status (member ring, + unavailable nodes, meta data etc.) + leader - Asks the cluster who the current leader is + is-singleton - Checks if the cluster is a singleton cluster (single + node cluster) + is-available - Checks if the member node is available + is-running - Checks if the member node is running + has-convergence - Checks if there is a cluster convergence + Where the should be on the format of 'akka://actor-system-name@hostname:port' + + Examples: bin/akka-cluster localhost:9999 is-available + bin/akka-cluster localhost:9999 join akka://MySystem@darkstar:2552 + bin/akka-cluster localhost:9999 cluster-status + + +To be able to use the script you must enable remote monitoring and management when starting the JVMs of the cluster nodes, +as described in `Monitoring and Management Using JMX Technology `_ + +Example of system properties to enable remote monitoring and management:: + + java -Dcom.sun.management.jmxremote.port=9999 \ + -Dcom.sun.management.jmxremote.authenticate=false \ + -Dcom.sun.management.jmxremote.ssl=false + .. _cluster_configuration: Configuration diff --git a/akka-kernel/src/main/dist/bin/akka-cluster b/akka-kernel/src/main/dist/bin/akka-cluster index fe3af38449..0cbff520dd 100755 --- a/akka-kernel/src/main/dist/bin/akka-cluster +++ b/akka-kernel/src/main/dist/bin/akka-cluster @@ -27,7 +27,7 @@ HOST=$1 # cluster node:port to talk to through JMX function ensureNodeIsRunningAndAvailable { REPLY=$($JMX_CLIENT $HOST akka:type=Cluster Available 2>&1 >/dev/null) # redirects STDERR to STDOUT before capturing it if [[ "$REPLY" != *true ]]; then - echo "Akka cluster node is not available on $HOST" + echo "Akka cluster node is not available on $HOST, due to $REPLY" exit 1 fi } @@ -37,7 +37,7 @@ case "$2" in join) if [ $# -ne 3 ]; then - echo "Usage: $SELF join " + echo "Usage: $SELF join " exit 1 fi @@ -51,7 +51,7 @@ case "$2" in leave) if [ $# -ne 3 ]; then - echo "Usage: $SELF leave " + echo "Usage: $SELF leave " exit 1 fi @@ -65,7 +65,7 @@ case "$2" in down) if [ $# -ne 3 ]; then - echo "Usage: $SELF down " + echo "Usage: $SELF down " exit 1 fi @@ -164,7 +164,7 @@ case "$2" in ensureNodeIsRunningAndAvailable shift - echo "Checking if member node on $HOST is AVAILABLE" + echo "Checking if member node on $HOST is RUNNING" $JMX_CLIENT $HOST akka:type=Cluster Running ;; @@ -172,17 +172,17 @@ case "$2" in printf "Usage: bin/$SELF ...\n" printf "\n" printf "Supported commands are:\n" - printf "%26s - %s\n" "join " "Sends request a JOIN node with the specified URL" - printf "%26s - %s\n" "leave " "Sends a request for node with URL to LEAVE the cluster" - printf "%26s - %s\n" "down " "Sends a request for marking node with URL as DOWN" - printf "%26s - %s\n" member-status "Asks the member node for its current status" - printf "%26s - %s\n" cluster-status "Asks the cluster for its current status (member ring, unavailable nodes, meta data etc.)" - printf "%26s - %s\n" leader "Asks the cluster who the current leader is" - printf "%26s - %s\n" is-singleton "Checks if the cluster is a singleton cluster (single node cluster)" - printf "%26s - %s\n" is-available "Checks if the member node is available" - printf "%26s - %s\n" is-running "Checks if the member node is running" - printf "%26s - %s\n" has-convergence "Checks if there is a cluster convergence" - printf "Where the should be on the format of 'akka://actor-system-name@hostname:port'\n" + printf "%26s - %s\n" "join " "Sends request a JOIN node with the specified URL" + printf "%26s - %s\n" "leave " "Sends a request for node with URL to LEAVE the cluster" + printf "%26s - %s\n" "down " "Sends a request for marking node with URL as DOWN" + printf "%26s - %s\n" member-status "Asks the member node for its current status" + printf "%26s - %s\n" cluster-status "Asks the cluster for its current status (member ring, unavailable nodes, meta data etc.)" + printf "%26s - %s\n" leader "Asks the cluster who the current leader is" + printf "%26s - %s\n" is-singleton "Checks if the cluster is a singleton cluster (single node cluster)" + printf "%26s - %s\n" is-available "Checks if the member node is available" + printf "%26s - %s\n" is-running "Checks if the member node is running" + printf "%26s - %s\n" has-convergence "Checks if there is a cluster convergence" + printf "Where the should be on the format of 'akka://actor-system-name@hostname:port'\n" printf "\n" printf "Examples: bin/$SELF localhost:9999 is-available\n" printf " bin/$SELF localhost:9999 join akka://MySystem@darkstar:2552\n" From 068335789cd6762bd4f148f9dae6f40a7d362af2 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 20 Sep 2012 08:09:01 +0200 Subject: [PATCH 36/46] Cluster config setting to disable jmx, see #2531 --- akka-cluster/src/main/resources/reference.conf | 3 +++ akka-cluster/src/main/scala/akka/cluster/Cluster.scala | 9 ++++++--- .../src/main/scala/akka/cluster/ClusterSettings.scala | 1 + .../scala/akka/cluster/MultiNodeClusterSpec.scala | 1 + .../src/test/scala/akka/cluster/ClusterConfigSpec.scala | 1 + 5 files changed, 12 insertions(+), 3 deletions(-) diff --git a/akka-cluster/src/main/resources/reference.conf b/akka-cluster/src/main/resources/reference.conf index 141cbcbaec..fa6860a1a8 100644 --- a/akka-cluster/src/main/resources/reference.conf +++ b/akka-cluster/src/main/resources/reference.conf @@ -27,6 +27,9 @@ akka { # in case of network partition. auto-down = off + # Enable or disable JMX MBeans for management of the cluster + jmx.enabled = on + # how long should the node wait before starting the periodic tasks maintenance tasks? periodic-tasks-initial-delay = 1s diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 647f335b8c..09feaeb656 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -160,8 +160,11 @@ class Cluster(val system: ExtendedActorSystem) extends Extension { system.registerOnTermination(shutdown()) - private val clusterJmx = new ClusterJmx(this, log) - clusterJmx.createMBean() + private val clusterJmx: Option[ClusterJmx] = { + val jmx = new ClusterJmx(this, log) + jmx.createMBean() + Some(jmx) + } log.info("Cluster Node [{}] - has started up successfully", selfAddress) @@ -237,7 +240,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension { scheduler.close() - clusterJmx.unregisterMBean() + clusterJmx foreach { _.unregisterMBean() } log.info("Cluster Node [{}] - Cluster node successfully shut down", selfAddress) } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala index e37d4abc72..4212e59c1c 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala @@ -36,6 +36,7 @@ class ClusterSettings(val config: Config, val systemName: String) { final val PublishStatsInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.publish-stats-interval"), MILLISECONDS) final val AutoJoin: Boolean = getBoolean("akka.cluster.auto-join") final val AutoDown: Boolean = getBoolean("akka.cluster.auto-down") + final val JmxEnabled: Boolean = getBoolean("akka.cluster.jmx.enabled") final val JoinTimeout: FiniteDuration = Duration(getMilliseconds("akka.cluster.join-timeout"), MILLISECONDS) final val UseDispatcher: String = getString("akka.cluster.use-dispatcher") match { case "" ⇒ Dispatchers.DefaultDispatcherId diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala index af47d869dc..f4594543e6 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala @@ -33,6 +33,7 @@ object MultiNodeClusterSpec { akka.cluster { auto-join = on auto-down = off + jmx.enabled = off gossip-interval = 200 ms heartbeat-interval = 400 ms leader-actions-interval = 200 ms diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala index 2d7565f5f5..f8c5571a57 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala @@ -35,6 +35,7 @@ class ClusterConfigSpec extends AkkaSpec { JoinTimeout must be(60 seconds) AutoJoin must be(true) AutoDown must be(false) + JmxEnabled must be(true) UseDispatcher must be(Dispatchers.DefaultDispatcherId) GossipDifferentViewProbability must be(0.8 plusOrMinus 0.0001) MaxGossipMergeRate must be(5.0 plusOrMinus 0.0001) From 6cf638815f8d65052cdb5362a91699e3f9cf3901 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 20 Sep 2012 08:09:46 +0200 Subject: [PATCH 37/46] Disable remote logging --- .../src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala | 1 + .../src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala index c92ff0eafb..0d122938ab 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala @@ -52,6 +52,7 @@ object LargeClusterMultiJvmSpec extends MultiNodeConfig { parallelism-max = 13 } akka.scheduler.tick-duration = 33 ms + akka.remote.log-remote-lifecycle-events = off akka.remote.netty.execution-pool-size = 4 #akka.remote.netty.reconnection-time-window = 1s akka.remote.netty.backoff-timeout = 500ms diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala index 768546a532..40f46ffbbc 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SunnyWeatherSpec.scala @@ -29,6 +29,7 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig { auto-join = off } akka.loglevel = INFO + akka.remote.log-remote-lifecycle-events = off """)) } From ab8a690c65fa67d5183bb8615f2fda1dc417d6a1 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 20 Sep 2012 08:44:44 +0200 Subject: [PATCH 38/46] Use Either for LeaderChanged state, see #2518 --- .../scala/akka/cluster/ClusterEvent.scala | 34 +++++++++++++------ 1 file changed, 23 insertions(+), 11 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala index 17988fd7ab..60172ed6a4 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala @@ -173,8 +173,14 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto import InternalClusterAction._ var latestGossip: Gossip = Gossip() - var stashedLeaderChanged: Option[LeaderChanged] = None - var publishedLeaderChanged: Option[LeaderChanged] = None + + // Keep track of LeaderChanged event. Should not be published until + // convergence, and it should only be published when leader actually + // changed to another node. 3 states: + // - None: No LeaderChanged detected yet, nothing published yet + // - Some(Left): Stashed LeaderChanged to be published later, when convergence + // - Some(Right): Latest published LeaderChanged + var leaderChangedState: Option[Either[LeaderChanged, LeaderChanged]] = None def receive = { case PublishChanges(oldGossip, newGossip) ⇒ publishChanges(oldGossip, newGossip) @@ -204,23 +210,26 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto latestGossip = newGossip diff(oldGossip, newGossip) foreach { event ⇒ event match { - case x @ LeaderChanged(_) if Some(x) == publishedLeaderChanged ⇒ + case x @ LeaderChanged(_) if leaderChangedState == Some(Right(x)) ⇒ // skip, this leader has already been published case x @ LeaderChanged(_) if oldGossip.convergence && newGossip.convergence ⇒ - stashedLeaderChanged = None - publishedLeaderChanged = Some(x) + // leader changed and immediate convergence + leaderChangedState = Some(Right(x)) eventStream publish x case x: LeaderChanged ⇒ // publish later, when convergence - stashedLeaderChanged = Some(x) + leaderChangedState = Some(Left(x)) case ConvergenceChanged(true) ⇒ - stashedLeaderChanged foreach { - publishedLeaderChanged = stashedLeaderChanged - stashedLeaderChanged = None - eventStream publish _ + // now it's convergence, publish eventual stashed LeaderChanged event + leaderChangedState match { + case Some(Left(x)) ⇒ + leaderChangedState = Some(Right(x)) + eventStream publish x + + case _ ⇒ // nothing stashed } eventStream publish event @@ -228,7 +237,10 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto eventStream publish event // notify DeathWatch about unreachable node eventStream publish AddressTerminated(m.address) - case _ ⇒ eventStream publish event + + case _ ⇒ + // all other events + eventStream publish event } } } From a790f5bb32f18e7b5668f8db50b090a743e5d20b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 20 Sep 2012 08:50:12 +0200 Subject: [PATCH 39/46] Use named parameters to Deploy, avoid strange path param --- .../main/scala/akka/cluster/routing/ClusterRouterConfig.scala | 3 ++- .../src/main/scala/akka/routing/RemoteRouterConfig.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala index c99ae68704..99415e9838 100644 --- a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala +++ b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala @@ -178,7 +178,8 @@ private[akka] class ClusterRouteeProvider( context.actorFor(RootActorPath(target) / settings.routeesPathElements) } else { val name = "c" + childNameCounter.incrementAndGet - val deploy = Deploy("", ConfigFactory.empty(), routeeProps.routerConfig, RemoteScope(target)) + val deploy = Deploy(config = ConfigFactory.empty(), routerConfig = routeeProps.routerConfig, + scope = RemoteScope(target)) context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false) } // must register each one, since registered routees are used in selectDeploymentTarget diff --git a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala index 7075aa5ea7..1d05dae3bc 100644 --- a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala +++ b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala @@ -78,7 +78,8 @@ final class RemoteRouteeProvider(nodes: Iterable[Address], _context: ActorContex override def createRoutees(nrOfInstances: Int): Unit = { val refs = IndexedSeq.fill(nrOfInstances) { val name = "c" + childNameCounter.incrementAndGet - val deploy = Deploy("", ConfigFactory.empty(), routeeProps.routerConfig, RemoteScope(nodeAddressIter.next)) + val deploy = Deploy(config = ConfigFactory.empty(), routerConfig = routeeProps.routerConfig, + scope = RemoteScope(nodeAddressIter.next)) // attachChild means that the provider will treat this call as if possibly done out of the wrong // context and use RepointableActorRef instead of LocalActorRef. Seems like a slightly sub-optimal From 6dbe1c00c49800455edee4fb6aa091b1a75531de Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 20 Sep 2012 09:24:06 +0200 Subject: [PATCH 40/46] Further improvement of selectDeploymentTarget based on feedback --- .../scala/akka/cluster/routing/ClusterRouterConfig.scala | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala index 99415e9838..d3a0b9cb88 100644 --- a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala +++ b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala @@ -202,12 +202,9 @@ private[akka] class ClusterRouteeProvider( } else { // find the node with least routees val numberOfRouteesPerNode: Map[Address, Int] = - currentRoutees.foldLeft(currentNodes.map(_ -> 0).toMap) { (acc, x) ⇒ + currentRoutees.foldLeft(currentNodes.map(_ -> 0).toMap.withDefault(_ ⇒ 0)) { (acc, x) ⇒ val address = fullAddress(x) - acc.get(address) match { - case Some(count) ⇒ acc + (address -> (count + 1)) - case None ⇒ acc + (address -> 1) - } + acc + (address -> (acc(address) + 1)) } val (address, count) = numberOfRouteesPerNode.minBy(_._2) From 2893b0f4671c3c49ff3b84f3d46cfefd15a2e8c1 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 20 Sep 2012 10:22:52 +0200 Subject: [PATCH 41/46] Update to the latest community build. --- project/AkkaBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 895f21ed5a..2bbf27f3d2 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -356,7 +356,7 @@ object AkkaBuild extends Build { shellPrompt := { s => Project.extract(s).currentProject.id + " > " }, resolvers <<= (resolvers, scalaVersion) apply { case (res, "2.10.0-SNAPSHOT") => - res :+ ("Scala Community 2.10.0-SNAPSHOT" at "https://scala-webapps.epfl.ch/jenkins/job/community-nightly/ws/target/repositories/8e83577d99af1d718fe369c4a4ee92737b9cf669") + res :+ ("Scala Community 2.10.0-SNAPSHOT" at "https://scala-webapps.epfl.ch/jenkins/job/community-nightly/ws/target/repositories/fc24ea43b17664f020e43379e800c34be09700bd") case (res, _) => res } From b8242ed1dab36c99fdb44eca2a09d203921cd663 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 20 Sep 2012 10:51:52 +0200 Subject: [PATCH 42/46] Adding regression test for DataflowSpec --- akka-dataflow/src/test/scala/akka/dataflow/DataflowSpec.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/akka-dataflow/src/test/scala/akka/dataflow/DataflowSpec.scala b/akka-dataflow/src/test/scala/akka/dataflow/DataflowSpec.scala index 302d245930..2bc616881b 100644 --- a/akka-dataflow/src/test/scala/akka/dataflow/DataflowSpec.scala +++ b/akka-dataflow/src/test/scala/akka/dataflow/DataflowSpec.scala @@ -271,7 +271,9 @@ class DataflowSpec extends AkkaSpec with DefaultTimeout { assert(checkType(rString, classTag[String])) assert(checkType(rInt, classTag[Int])) assert(!checkType(rInt, classTag[String])) - //assert(!checkType(rInt, classTag[Nothing])) This test does not work with classTags, but works with Manifests + assert(intercept[java.lang.Exception] { + assert(!checkType(rInt, classTag[Nothing])) + }.getMessage == "Nothing is a bottom type, therefore its erasure does not return a value") // When this fails, remove the intercept assert(!checkType(rInt, classTag[Any])) Await.result(rString, timeout.duration) From bc34adf6246c2368974a31e97f824022cee7dff4 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 20 Sep 2012 12:35:03 +0200 Subject: [PATCH 43/46] Test consistent hashing router with cluster router, see #944 * Found and fixed issue with NoRouter which is used in the combination of FromConfig and cluster.enabled=true * Multi-node test that tests several of the possible cominations --- .../routing/ConsistentHashingRouter.scala | 2 +- .../ClusterConsistentHashingRouterSpec.scala | 182 ++++++++++++++++++ 2 files changed, 183 insertions(+), 1 deletion(-) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index 3b9802d7fd..cdfd040ace 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -205,7 +205,7 @@ case class ConsistentHashingRouter( * that can't be defined in configuration. */ override def withFallback(other: RouterConfig): RouterConfig = other match { - case _: FromConfig ⇒ this + case _: FromConfig | _: NoRouter ⇒ this case otherRouter: ConsistentHashingRouter ⇒ val useResizer = if (this.resizer.isEmpty && otherRouter.resizer.isDefined) otherRouter.resizer diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala new file mode 100644 index 0000000000..c39edd8a13 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala @@ -0,0 +1,182 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.cluster.routing + +import scala.concurrent.Await +import scala.concurrent.util.duration._ + +import com.typesafe.config.ConfigFactory + +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Address +import akka.actor.Props +import akka.cluster.MultiNodeClusterSpec +import akka.pattern.ask +import akka.remote.testkit.MultiNodeConfig +import akka.remote.testkit.MultiNodeSpec +import akka.routing.ConsistentHashingRouter +import akka.routing.ConsistentHashingRouter.ConsistentHashMapping +import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope +import akka.routing.CurrentRoutees +import akka.routing.FromConfig +import akka.routing.RouterRoutees +import akka.testkit._ + +object ClusterConsistentHashingRouterMultiJvmSpec extends MultiNodeConfig { + + class Echo extends Actor { + def receive = { + case _ ⇒ sender ! self + } + } + + val first = role("first") + val second = role("second") + val third = role("third") + + commonConfig(debugConfig(on = false). + withFallback(ConfigFactory.parseString(""" + common-router-settings = { + router = consistent-hashing + nr-of-instances = 10 + cluster { + enabled = on + max-nr-of-instances-per-node = 2 + } + } + + akka.actor.deployment { + /router1 = ${common-router-settings} + /router3 = ${common-router-settings} + /router4 = ${common-router-settings} + } + """)). + withFallback(MultiNodeClusterSpec.clusterConfig)) + +} + +class ClusterConsistentHashingRouterMultiJvmNode1 extends ClusterConsistentHashingRouterSpec +class ClusterConsistentHashingRouterMultiJvmNode2 extends ClusterConsistentHashingRouterSpec +class ClusterConsistentHashingRouterMultiJvmNode3 extends ClusterConsistentHashingRouterSpec + +abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterConsistentHashingRouterMultiJvmSpec) + with MultiNodeClusterSpec + with ImplicitSender with DefaultTimeout { + import ClusterConsistentHashingRouterMultiJvmSpec._ + + lazy val router1 = system.actorOf(Props[Echo].withRouter(FromConfig()), "router1") + + def currentRoutees(router: ActorRef) = + Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees + + /** + * Fills in self address for local ActorRef + */ + private def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match { + case Address(_, _, None, None) ⇒ cluster.selfAddress + case a ⇒ a + } + + "A cluster router with a consistent hashing router" must { + "start cluster with 2 nodes" taggedAs LongRunningTest in { + awaitClusterUp(first, second) + enterBarrier("after-1") + } + + "create routees from configuration" in { + runOn(first) { + awaitCond { + // it may take some time until router receives cluster member events + currentRoutees(router1).size == 4 + } + currentRoutees(router1).map(fullAddress).toSet must be(Set(address(first), address(second))) + } + enterBarrier("after-2") + } + + "select destination based on hashKey" in { + runOn(first) { + router1 ! ConsistentHashableEnvelope(message = "A", hashKey = "a") + val destinationA = expectMsgType[ActorRef] + router1 ! ConsistentHashableEnvelope(message = "AA", hashKey = "a") + expectMsg(destinationA) + } + enterBarrier("after-2") + } + + "deploy routees to new member nodes in the cluster" taggedAs LongRunningTest in { + + awaitClusterUp(first, second, third) + + runOn(first) { + awaitCond { + // it may take some time until router receives cluster member events + currentRoutees(router1).size == 6 + } + currentRoutees(router1).map(fullAddress).toSet must be(roles.map(address).toSet) + } + + enterBarrier("after-3") + } + + "deploy programatically defined routees to the member nodes in the cluster" taggedAs LongRunningTest in { + runOn(first) { + val router2 = system.actorOf(Props[Echo].withRouter(ClusterRouterConfig(local = ConsistentHashingRouter(), + settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 2))), "router2") + awaitCond { + // it may take some time until router receives cluster member events + currentRoutees(router2).size == 6 + } + currentRoutees(router2).map(fullAddress).toSet must be(roles.map(address).toSet) + } + + enterBarrier("after-4") + } + + "handle combination of configured router and programatically defined hashMapping" taggedAs LongRunningTest in { + runOn(first) { + def hashMapping: ConsistentHashMapping = { + case s: String ⇒ s + } + + val router3 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter(hashMapping = hashMapping)), "router3") + + assertHashMapping(router3) + } + + enterBarrier("after-5") + } + + "handle combination of configured router and programatically defined hashMapping and ClusterRouterConfig" taggedAs LongRunningTest in { + runOn(first) { + def hashMapping: ConsistentHashMapping = { + case s: String ⇒ s + } + + val router4 = system.actorOf(Props[Echo].withRouter(ClusterRouterConfig( + local = ConsistentHashingRouter(hashMapping = hashMapping), + settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 1))), "router4") + + assertHashMapping(router4) + } + + enterBarrier("after-6") + } + + def assertHashMapping(router: ActorRef): Unit = { + awaitCond { + // it may take some time until router receives cluster member events + currentRoutees(router).size == 6 + } + currentRoutees(router).map(fullAddress).toSet must be(roles.map(address).toSet) + + router ! "a" + val destinationA = expectMsgType[ActorRef] + router ! "a" + expectMsg(destinationA) + } + + } +} From 2c3bc310a3b4d6f92bc91b3acd1aa5ec2fbc450c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 20 Sep 2012 12:54:17 +0200 Subject: [PATCH 44/46] #2532 - Fixing PostRestart prematurely clearing out the fancy bits out of the InvocationHandler --- .../test/scala/akka/actor/TypedActorSpec.scala | 18 ++++++++++++++++++ .../src/main/scala/akka/actor/TypedActor.scala | 11 ++--------- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index 84367f7ec0..fc7be182f7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -193,6 +193,9 @@ object TypedActorSpec { }) } } + + trait F { def f(pow: Boolean): Int } + class FI extends F { def f(pow: Boolean): Int = if (pow) throw new IllegalStateException("expected") else 1 } } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @@ -351,6 +354,21 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) } } + "be restarted on failure" in { + filterEvents(EventFilter[IllegalStateException]("expected")) { + val t = newFooBar(Duration(2, "s")) + intercept[IllegalStateException] { t.failingOptionPigdog() }.getMessage must be === "expected" + t.optionPigdog() must be === Some("Pigdog") + mustStop(t) + + val ta: F = TypedActor(system).typedActorOf(TypedProps[FI]()) + intercept[IllegalStateException] { ta.f(true) }.getMessage must be === "expected" + ta.f(false) must be === 1 + + mustStop(ta) + } + } + "be able to support stacked traits for the interface part" in { val t = newStacked() t.notOverriddenStacked must be("foobar") diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index e10e5350bc..69a3707f48 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -235,14 +235,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi * INTERNAL USE ONLY */ private[akka] class TypedActor[R <: AnyRef, T <: R](val proxyVar: AtomVar[R], createInstance: ⇒ T) extends Actor { - val me = try { - TypedActor.selfReference set proxyVar.get - TypedActor.currentContext set context - createInstance - } finally { - TypedActor.selfReference set null - TypedActor.currentContext set null - } + val me = withContext[T](createInstance) override def supervisorStrategy(): SupervisorStrategy = me match { case l: Supervisor ⇒ l.supervisorStrategy @@ -275,7 +268,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi override def preRestart(reason: Throwable, message: Option[Any]): Unit = withContext { me match { case l: PreRestart ⇒ l.preRestart(reason, message) - case _ ⇒ super.preRestart(reason, message) + case _ ⇒ context.children foreach context.stop //Can't be super.preRestart(reason, message) since that would invoke postStop which would set the actorVar to DL and proxyVar to null } } From c7b966b4e70d65e012fc76a1d649eacffc4f485b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 20 Sep 2012 12:58:51 +0200 Subject: [PATCH 45/46] Switch to ConistentHashingRouter in cluster sample --- .../sample/cluster/stats/StatsSample.scala | 34 ++++++++++++------- .../stats/StatsSampleSingleMasterSpec.scala | 3 +- .../cluster/stats/StatsSampleSpec.scala | 3 +- 3 files changed, 24 insertions(+), 16 deletions(-) diff --git a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala index 351b5d0a48..8a847d0b05 100644 --- a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala +++ b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/stats/StatsSample.scala @@ -21,6 +21,7 @@ import akka.cluster.ClusterEvent.MemberEvent import akka.cluster.ClusterEvent.MemberUp import akka.cluster.MemberStatus import akka.routing.FromConfig +import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope //#imports //#messages @@ -39,7 +40,10 @@ class StatsService extends Actor { val words = text.split(" ") val replyTo = sender // important to not close over sender val aggregator = context.actorOf(Props(new StatsAggregator(words.size, replyTo))) - words foreach { word ⇒ workerRouter.tell(word, aggregator) } + words foreach { word ⇒ + workerRouter.tell( + ConsistentHashableEnvelope(word, word), aggregator) + } } } @@ -64,9 +68,18 @@ class StatsAggregator(expectedResults: Int, replyTo: ActorRef) extends Actor { //#worker class StatsWorker extends Actor { - // FIXME add a cache here to illustrate consistent hashing + var cache = Map.empty[String, Int] def receive = { - case word: String ⇒ sender ! word.length + case word: String ⇒ + val length = cache.get(word) match { + case Some(x) ⇒ x + case None ⇒ + val x = word.length + cache += (word -> x) + x + } + + sender ! length } } //#worker @@ -124,8 +137,7 @@ object StatsSample { val system = ActorSystem("ClusterSystem", ConfigFactory.parseString(""" akka.actor.deployment { /statsService/workerRouter { - # FIXME use consistent hashing instead - router = round-robin + router = consistent-hashing nr-of-instances = 100 cluster { enabled = on @@ -153,8 +165,7 @@ object StatsSampleOneMaster { val system = ActorSystem("ClusterSystem", ConfigFactory.parseString(""" akka.actor.deployment { /statsFacade/statsService/workerRouter { - # FIXME use consistent hashing instead - router = round-robin + router = consistent-hashing nr-of-instances = 100 cluster { enabled = on @@ -225,10 +236,10 @@ abstract class StatsService2 extends Actor { //#router-lookup-in-code import akka.cluster.routing.ClusterRouterConfig import akka.cluster.routing.ClusterRouterSettings - import akka.routing.RoundRobinRouter + import akka.routing.ConsistentHashingRouter val workerRouter = context.actorOf(Props[StatsWorker].withRouter( - ClusterRouterConfig(RoundRobinRouter(), ClusterRouterSettings( + ClusterRouterConfig(ConsistentHashingRouter(), ClusterRouterSettings( totalInstances = 100, routeesPath = "/user/statsWorker", allowLocalRoutees = true))), name = "workerRouter2") @@ -240,11 +251,10 @@ abstract class StatsService3 extends Actor { //#router-deploy-in-code import akka.cluster.routing.ClusterRouterConfig import akka.cluster.routing.ClusterRouterSettings - // FIXME use ConsistentHashingRouter instead - import akka.routing.RoundRobinRouter + import akka.routing.ConsistentHashingRouter val workerRouter = context.actorOf(Props[StatsWorker].withRouter( - ClusterRouterConfig(RoundRobinRouter(), ClusterRouterSettings( + ClusterRouterConfig(ConsistentHashingRouter(), ClusterRouterSettings( totalInstances = 100, maxInstancesPerNode = 3, allowLocalRoutees = false))), name = "workerRouter3") diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala index e23504d084..b1d27cd7a3 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala @@ -35,8 +35,7 @@ object StatsSampleSingleMasterSpec extends MultiNodeConfig { #//#router-deploy-config akka.actor.deployment { /statsFacade/statsService/workerRouter { - # FIXME use consistent hashing instead - router = round-robin + router = consistent-hashing nr-of-instances = 100 cluster { enabled = on diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala index 7398aa025b..9f88597051 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala @@ -31,8 +31,7 @@ object StatsSampleSpec extends MultiNodeConfig { #//#router-lookup-config akka.actor.deployment { /statsService/workerRouter { - # FIXME use consistent hashing instead - router = round-robin + router = consistent-hashing nr-of-instances = 100 cluster { enabled = on From 08121e4ae3e89f0c188ad664b9ca53c68cbbc338 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 20 Sep 2012 13:02:24 +0200 Subject: [PATCH 46/46] Minor corr, fqcn of CARP --- akka-docs/cluster/cluster-usage.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-docs/cluster/cluster-usage.rst b/akka-docs/cluster/cluster-usage.rst index 6087e1b637..b0ec8f08b7 100644 --- a/akka-docs/cluster/cluster-usage.rst +++ b/akka-docs/cluster/cluster-usage.rst @@ -36,7 +36,7 @@ Try it out: :language: none To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-scala` -settings, but with ``ClusterActorRefProvider``. +settings, but with ``akka.cluster.ClusterActorRefProvider``. The ``akka.cluster.seed-nodes`` and cluster extension should normally also be added to your ``application.conf`` file.