Merge pull request #698 from akka/wip-944-consistent-hashing-patriknw

Consistent hashing router, see #944
This commit is contained in:
Patrik Nordwall 2012-09-19 07:57:27 -07:00
commit 888f81df8d
19 changed files with 1236 additions and 309 deletions

View file

@ -35,6 +35,9 @@ object DeployerSpec {
router = scatter-gather router = scatter-gather
within = 2 seconds within = 2 seconds
} }
/service-consistent-hashing {
router = consistent-hashing
}
/service-resizer { /service-resizer {
router = round-robin router = round-robin
resizer { resizer {
@ -118,6 +121,10 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
assertRouting("/service-scatter-gather", ScatterGatherFirstCompletedRouter(nrOfInstances = 1, within = 2 seconds), "/service-scatter-gather") 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 { "be able to parse 'akka.actor.deployment._' with router resizer" in {
val resizer = DefaultResizer() val resizer = DefaultResizer()
assertRouting("/service-resizer", RoundRobinRouter(resizer = Some(resizer)), "/service-resizer") assertRouting("/service-resizer", RoundRobinRouter(resizer = Some(resizer)), "/service-resizer")

View file

@ -43,6 +43,9 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin
getBoolean("akka.jvm-exit-on-fatal-error") must be(true) getBoolean("akka.jvm-exit-on-fatal-error") must be(true)
settings.JvmExitOnFatalError must be(true) settings.JvmExitOnFatalError must be(true)
getInt("akka.actor.deployment.default.virtual-nodes-factor") must be(10)
settings.DefaultVirtualNodesFactor must be(10)
} }
{ {

View file

@ -0,0 +1,104 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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.routing.ConsistentHashingRouter.ConsistentHashMapping
import akka.testkit.AkkaSpec
import akka.testkit._
object ConsistentHashingRouterSpec {
val config = """
akka.actor.deployment {
/router1 {
router = consistent-hashing
nr-of-instances = 3
virtual-nodes-factor = 17
}
/router2 {
router = consistent-hashing
nr-of-instances = 5
}
}
"""
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)
case class Msg2(key: Any, data: 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(FromConfig()), "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 = expectMsgType[ActorRef]
router1 ! ConsistentHashableEnvelope(message = "AA", hashKey = "a")
expectMsg(destinationA)
router1 ! Msg(17, "B")
val destinationB = expectMsgType[ActorRef]
router1 ! ConsistentHashableEnvelope(message = "BB", hashKey = 17)
expectMsg(destinationB)
router1 ! Msg(MsgKey("c"), "C")
val destinationC = expectMsgType[ActorRef]
router1 ! ConsistentHashableEnvelope(message = "CC", hashKey = MsgKey("c"))
expectMsg(destinationC)
}
"select destination with defined consistentHashRoute" in {
def hashMapping: ConsistentHashMapping = {
case Msg2(key, data) key
}
val router2 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter(
hashMapping = hashMapping)), "router2")
router2 ! Msg2("a", "A")
val destinationA = expectMsgType[ActorRef]
router2 ! ConsistentHashableEnvelope(message = "AA", hashKey = "a")
expectMsg(destinationA)
router2 ! Msg2(17, "B")
val destinationB = expectMsgType[ActorRef]
router2 ! ConsistentHashableEnvelope(message = "BB", hashKey = 17)
expectMsg(destinationB)
router2 ! Msg2(MsgKey("c"), "C")
val destinationC = expectMsgType[ActorRef]
router2 ! ConsistentHashableEnvelope(message = "CC", hashKey = MsgKey("c"))
expectMsg(destinationC)
}
}
}

View file

@ -171,8 +171,6 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
} }
expectMsgType[ActorKilledException] expectMsgType[ActorKilledException]
//#supervision
val router2 = system.actorOf(Props.empty.withRouter(RoundRobinRouter(1).withSupervisorStrategy(escalator))) val router2 = system.actorOf(Props.empty.withRouter(RoundRobinRouter(1).withSupervisorStrategy(escalator)))
router2 ! CurrentRoutees router2 ! CurrentRoutees
EventFilter[ActorKilledException](occurrences = 2) intercept { EventFilter[ActorKilledException](occurrences = 2) intercept {

View file

@ -108,6 +108,9 @@ akka {
# within is the timeout used for routers containing future calls # within is the timeout used for routers containing future calls
within = 5 seconds within = 5 seconds
# number of virtual nodes per node for consistent-hashing router
virtual-nodes-factor = 10
routees { routees {
# Alternatively to giving nr-of-instances you can specify the full # Alternatively to giving nr-of-instances you can specify the full
# paths of those actors which should be routed to. This setting takes # paths of those actors which should be routed to. This setting takes

View file

@ -166,6 +166,8 @@ object ActorSystem {
final val Daemonicity: Boolean = getBoolean("akka.daemonic") final val Daemonicity: Boolean = getBoolean("akka.daemonic")
final val JvmExitOnFatalError: Boolean = getBoolean("akka.jvm-exit-on-fatal-error") 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) if (ConfigVersion != Version)
throw new akka.ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]") throw new akka.ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]")

View file

@ -143,8 +143,6 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
val nrOfInstances = deployment.getInt("nr-of-instances") 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 resizer: Option[Resizer] = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None
val router: RouterConfig = deployment.getString("router") match { val router: RouterConfig = deployment.getString("router") match {
@ -152,8 +150,13 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
case "round-robin" RoundRobinRouter(nrOfInstances, routees, resizer) case "round-robin" RoundRobinRouter(nrOfInstances, routees, resizer)
case "random" RandomRouter(nrOfInstances, routees, resizer) case "random" RandomRouter(nrOfInstances, routees, resizer)
case "smallest-mailbox" SmallestMailboxRouter(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 "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 case fqn
val args = Seq(classOf[Config] -> deployment) val args = Seq(classOf[Config] -> deployment)
dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({ dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({

View file

@ -4,255 +4,126 @@
package akka.routing package akka.routing
import scala.collection.immutable.{ TreeSet, Seq } import scala.collection.immutable.SortedMap
import scala.collection.mutable.{ Buffer, Map } import scala.reflect.ClassTag
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
*
* 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: Seq[T], replicas: Int) { class ConsistentHash[T: ClassTag] private (nodes: SortedMap[Int, T], virtualNodesFactor: Int) {
private val cluster = Buffer[T]()
private var sortedKeys = TreeSet[Long]()
private var ring = Map[Long, T]()
nodes.foreach(this += _) import ConsistentHash._
def +=(node: T): Unit = { if (virtualNodesFactor < 1) throw new IllegalArgumentException("virtualNodesFactor must be >= 1")
cluster += node
(1 to replicas) foreach { replica // arrays for fast binary search and access
val key = hashFor((node + ":" + replica).getBytes("UTF-8")) // nodeHashRing is the sorted hash values of the nodes
ring += (key -> node) // nodeRing is the nodes sorted in the same order as nodeHashRing, i.e. same index
sortedKeys = sortedKeys + key private val (nodeHashRing: Array[Int], nodeRing: Array[T]) = {
} val (nhr: Seq[Int], nr: Seq[T]) = nodes.toSeq.unzip
(nhr.toArray, nr.toArray)
} }
def -=(node: T): Unit = { /**
cluster -= node * Adds a node to the node ring.
(1 to replicas) foreach { replica * Note that the instance is immutable and this
val key = hashFor((node + ":" + replica).getBytes("UTF-8")) * operation returns a new instance.
ring -= key */
sortedKeys = sortedKeys - key def :+(node: T): ConsistentHash[T] =
} new ConsistentHash(nodes ++ ((1 to virtualNodesFactor) map { r (nodeHashFor(node, r) -> node) }), virtualNodesFactor)
}
def nodeFor(key: Array[Byte]): T = { /**
val hash = hashFor(key) * Adds a node to the node ring.
if (sortedKeys contains hash) ring(hash) * Note that the instance is immutable and this
* operation returns a new instance.
* JAVA API
*/
def add(node: T): ConsistentHash[T] = this :+ node
/**
* 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(nodes -- ((1 to virtualNodesFactor) map { r nodeHashFor(node, r) }), virtualNodesFactor)
/**
* Removes a node from the node ring.
* Note that the instance is immutable and this
* operation returns a new instance.
* JAVA API
*/
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 { else {
if (hash < sortedKeys.firstKey) ring(sortedKeys.firstKey) val j = math.abs(i + 1)
else if (hash > sortedKeys.lastKey) ring(sortedKeys.lastKey) if (j >= nodeHashRing.length) 0 // after last, use first
else ring(sortedKeys.rangeImpl(None, Some(hash)).lastKey) else j // next node clockwise
} }
} }
private def hashFor(bytes: Array[Byte]): Long = {
val hash = MurmurHash.arrayHash(bytes)
if (hash == Int.MinValue) hash + 1
math.abs(hash)
}
}
/* __ *\
** ________ ___ / / ___ 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
*/
import java.lang.Integer.{ rotateLeft rotl }
/**
* 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 */
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] =
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, * Get the node responsible for the data key.
* where the order of appearance of elements does not matter. * Can only be used if nodes exists in the node ring,
* This is useful for hashing sets, for example. * otherwise throws `IllegalStateException`
*/ */
def symmetricHash[T](xs: TraversableOnce[T], seed: Int): Int = { def nodeFor(key: Array[Byte]): T = {
var a, b, n = 0 if (isEmpty) throw new IllegalStateException("Can't get node for [%s] from an empty node ring" format key)
var c = 1
xs.foreach(i { nodeRing(idx(Arrays.binarySearch(nodeHashRing, hashFor(key))))
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)
} }
/**
* 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))))
}
/**
* Is the node ring empty, i.e. no nodes added or all removed.
*/
def isEmpty: Boolean = nodes.isEmpty
}
object ConsistentHash {
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)
}
/**
* Factory method to create a ConsistentHash
* JAVA API
*/
def create[T](nodes: java.lang.Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = {
import scala.collection.JavaConverters._
apply(nodes.asScala, virtualNodesFactor)(ClassTag(classOf[Any].asInstanceOf[Class[T]]))
}
private def nodeHashFor(node: Any, vnode: Int): Int =
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)
} }

View file

@ -0,0 +1,297 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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 {
/**
* 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)
}
/**
* 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
* 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.ConsistentHashingRouter.ConsistentHashableEnvelope]],
* or use [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]]
*/
trait ConsistentHashable {
def consistentHashKey: Any
}
/**
* 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
* router will only send the wrapped message to the destination,
* i.e. the envelope will be stripped off.
*/
@SerialVersionUID(1L)
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
* 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 ConsistentHashMapping = PartialFunction[Any, Any]
@SerialVersionUID(1L)
object emptyConsistentHashMapping extends ConsistentHashMapping {
def isDefinedAt(x: Any) = false
def apply(x: Any) = throw new UnsupportedOperationException("Empty ConsistentHashMapping 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.
*
* 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.
*/
trait ConsistentHashMapper {
def hashKey(message: Any): Any
}
}
/**
* A Router that uses consistent hashing to select a connection based on the
* sent message.
*
* There is 3 ways to define what data to use for the consistent hash key.
*
* 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.
*
* 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.
*
* 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 `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'
* actor(s). In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
* <br>
* <b>The</b> 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.
*
* <h1>Supervision Setup</h1>
*
* 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 parents 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 virtualNodesFactor number of virtual nodes per node, used in [[akka.routing.ConsistantHash]]
* @param hashMapping 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 hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping)
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 virtual nodes per node, used in [[akka.routing.ConsistantHash]]
*/
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 withHashMapper(mapping: ConsistentHashingRouter.ConsistentHashMapper) = {
copy(hashMapping = {
case message if (mapping.hashKey(message).asInstanceOf[AnyRef] ne null)
mapping.hashKey(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 `hashMapping` defined in code, since
* that can't be defined in configuration.
*/
override def withFallback(other: RouterConfig): RouterConfig = other match {
case _: FromConfig this
case otherRouter: ConsistentHashingRouter
val useResizer =
if (this.resizer.isEmpty && otherRouter.resizer.isDefined) otherRouter.resizer
else this.resizer
copy(resizer = useResizer, hashMapping = otherRouter.hashMapping)
case _ throw new IllegalArgumentException("Expected ConsistentHashingRouter, got [%s]".format(other))
}
}
/**
* The core pieces of the routing logic is located in this
* trait to be able to extend.
*/
trait ConsistentHashingLike { this: RouterConfig
import ConsistentHashingRouter._
def nrOfInstances: Int
def routees: Iterable[String]
def virtualNodesFactor: Int
def hashMapping: ConsistentHashMapping
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)
val vnodes =
if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor
else virtualNodesFactor
// 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 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 =
if (currentRoutees == oldConsistentHashRoutees) oldConsistentHash
else ConsistentHash(currentRoutees, vnodes) // re-hash
// ignore, don't update, in case of CAS failure
consistentHashRef.compareAndSet(oldConsistentHashTuple, (currentRoutees, consistentHash))
consistentHash
} else oldConsistentHash
}
def target(hashData: Any): ActorRef = try {
val currentConsistenHash = updateConsistentHash()
if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters
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
log.warning("Couldn't route message with consistent hash key [{}] due to [{}]", hashData, e.getMessage)
routeeProvider.context.system.deadLetters
}
{
case (sender, message)
message match {
case Broadcast(msg) toAll(sender, routeeProvider.routees)
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 hashMapping, or implement [{}] or be wrapped in [{}]",
message.getClass.getName, classOf[ConsistentHashable].getName,
classOf[ConsistentHashableEnvelope].getName)
List(Destination(sender, routeeProvider.context.system.deadLetters))
}
}
}
}

View file

@ -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)
}
}

View file

@ -115,7 +115,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
val s = if (sender eq null) system.deadLetters else sender val s = if (sender eq null) system.deadLetters else sender
val msg = message match { val msg = message match {
case Broadcast(m) m case wrapped: RouterEnvelope wrapped.message
case m m case m m
} }
@ -400,7 +400,15 @@ private object Router {
* Router implementations may choose to handle this message differently. * Router implementations may choose to handle this message differently.
*/ */
@SerialVersionUID(1L) @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. * Sending this message to a router will make it send back its currently used routees.
@ -588,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 trait RoundRobinLike { this: RouterConfig
def nrOfInstances: Int def nrOfInstances: Int
@ -721,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 trait RandomLike { this: RouterConfig
def nrOfInstances: Int def nrOfInstances: Int
@ -861,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 trait SmallestMailboxLike { this: RouterConfig
def nrOfInstances: Int def nrOfInstances: Int
@ -1076,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 trait BroadcastLike { this: RouterConfig
def nrOfInstances: Int def nrOfInstances: Int
@ -1205,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 trait ScatterGatherFirstCompletedLike { this: RouterConfig
def nrOfInstances: Int def nrOfInstances: Int

View file

@ -0,0 +1,5 @@
package docs.jrouting;
import org.scalatest.junit.JUnitSuite
class ConsistentHashingRouterDocTest extends ConsistentHashingRouterDocTestBase with JUnitSuite

View file

@ -0,0 +1,136 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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.ConsistentHashMapper;
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<String, String> cache = new HashMap<String, String>();
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 final class Evict implements Serializable {
public final String key;
public Evict(String key) {
this.key = key;
}
}
public static final class Get implements Serializable, ConsistentHashable {
public final String key;
public Get(String key) {
this.key = key;
}
public Object consistentHashKey() {
return key;
}
}
public static final 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 ConsistentHashMapper hashMapper = new ConsistentHashMapper() {
@Override
public Object hashKey(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).withHashMapper(hashMapper)),
"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
}};
}
}

View file

@ -15,13 +15,14 @@ is really easy to create your own. The routers shipped with Akka are:
* ``akka.routing.SmallestMailboxRouter`` * ``akka.routing.SmallestMailboxRouter``
* ``akka.routing.BroadcastRouter`` * ``akka.routing.BroadcastRouter``
* ``akka.routing.ScatterGatherFirstCompletedRouter`` * ``akka.routing.ScatterGatherFirstCompletedRouter``
* ``akka.routing.ConsistentHashingRouter``
Routers In Action Routers In Action
^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^
This is an example of how to create a router that is defined in configuration: 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 .. includecode:: code/docs/jrouting/RouterViaConfigExample.java#configurableRouting
@ -177,6 +178,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 - (The name of an actor is automatically created in the format ``$letter`` unless you specify it -
hence the names printed above.) 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 RandomRouter
************ ************
As the name implies this router type selects one of its routees randomly and forwards As the name implies this router type selects one of its routees randomly and forwards
@ -204,6 +209,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. 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. 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 SmallestMailboxRouter
********************* *********************
A Router that tries to send to the non-suspended routee with fewest messages in mailbox. A Router that tries to send to the non-suspended routee with fewest messages in mailbox.
@ -219,6 +228,10 @@ Code example:
.. includecode:: code/docs/jrouting/ParentActor.java#smallestMailboxRouter .. 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 BroadcastRouter
*************** ***************
A broadcast router forwards the message it receives to *all* its routees. A broadcast router forwards the message it receives to *all* its routees.
@ -238,6 +251,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. 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 ScatterGatherFirstCompletedRouter
********************************* *********************************
The ScatterGatherFirstCompletedRouter will send the message on to all its routees as a future. The ScatterGatherFirstCompletedRouter will send the message on to all its routees as a future.
@ -255,6 +272,51 @@ 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! 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. 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 <http://en.wikipedia.org/wiki/Consistent_hashing>`_
to select a connection based on the sent message. This
`article <http://weblogs.java.net/blog/tomwhite/archive/2007/11/consistent_hash.html>`_ gives good
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 ``withHashMapper`` of the router to map incoming
messages to their consistent hash key. This makes the the decision
transparent for the sender.
* 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.
* 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 ``withHashMapper`` is tried first.
Code example:
.. 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``
message is handled by the ``withHashMapper``.
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 Broadcast Messages
^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^
@ -276,7 +338,7 @@ of routees dynamically.
This is an example of how to create a resizable router that is defined in configuration: 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 .. includecode:: code/docs/jrouting/RouterViaConfigExample.java#configurableRoutingWithResizer

View file

@ -287,7 +287,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 stashed messages are put into the dead letters when the actor stops, make sure you call
super.postStop if you override it. super.postStop if you override it.
Forward of Terminated message Forward of Terminated message
============================= =============================
@ -442,3 +441,20 @@ v2.1::
final isSet = timeout.isFinite(); final isSet = timeout.isFinite();
getContext().setReceiveTimeout(Duration.Undefined()); getContext().setReceiveTimeout(Duration.Undefined());
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)

View file

@ -0,0 +1,73 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.routing
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
object ConsistentHashingRouterDocSpec {
//#cache-actor
import akka.actor.Actor
import akka.routing.ConsistentHashingRouter.ConsistentHashable
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 Evict(key) cache -= key
}
}
case class Evict(key: String)
case class Get(key: String) extends ConsistentHashable {
override def consistentHashKey: Any = key
}
case class Entry(key: String, value: String)
//#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
import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
def hashMapping: ConsistentHashMapping = {
case Evict(key) key
}
val cache = system.actorOf(Props[Cache].withRouter(ConsistentHashingRouter(10,
hashMapping = hashMapping)), name = "cache")
cache ! ConsistentHashableEnvelope(
message = Entry("hello", "HELLO"), hashKey = "hello")
cache ! ConsistentHashableEnvelope(
message = Entry("hi", "HI"), hashKey = "hi")
cache ! Get("hello")
expectMsg(Some("HELLO"))
cache ! Get("hi")
expectMsg(Some("HI"))
cache ! Evict("hi")
cache ! Get("hi")
expectMsg(None)
//#consistent-hashing-router
}
}

View file

@ -0,0 +1,158 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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
virtual-nodes-factor = 10
}
}
//#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)
}
}

View file

@ -1,52 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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) }
}

View file

@ -15,15 +15,16 @@ is really easy to create your own. The routers shipped with Akka are:
* ``akka.routing.SmallestMailboxRouter`` * ``akka.routing.SmallestMailboxRouter``
* ``akka.routing.BroadcastRouter`` * ``akka.routing.BroadcastRouter``
* ``akka.routing.ScatterGatherFirstCompletedRouter`` * ``akka.routing.ScatterGatherFirstCompletedRouter``
* ``akka.routing.ConsistentHashingRouter``
Routers In Action Routers In Action
^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^
This is an example of how to create a router that is defined in configuration: 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: 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: 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 :include: supervision
:exclude: custom-strategy :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 - (The name of an actor is automatically created in the format ``$letter`` unless you specify it -
hence the names printed above.) 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 RandomRouter
************ ************
As the name implies this router type selects one of its routees randomly and forwards 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. 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. 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 SmallestMailboxRouter
********************* *********************
A Router that tries to send to the non-suspended routee with fewest messages in mailbox. 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 .. 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 BroadcastRouter
*************** ***************
A broadcast router forwards the message it receives to *all* its routees. 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. 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 ScatterGatherFirstCompletedRouter
********************************* *********************************
The ScatterGatherFirstCompletedRouter will send the message on to all its routees as a future. The ScatterGatherFirstCompletedRouter will send the message on to all its routees as a future.
@ -257,6 +276,51 @@ 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! 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. 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 <http://en.wikipedia.org/wiki/Consistent_hashing>`_
to select a connection based on the sent message. This
`article <http://weblogs.java.net/blog/tomwhite/archive/2007/11/consistent_hash.html>`_ gives good
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 ``hashMapping`` of the router to map incoming
messages to their consistent hash key. This makes the decision
transparent for the sender.
* 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.
* 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 ``hashMapping`` is tried first.
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``. The ``Evict``
message is handled by the ``hashMapping`` partial function.
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 Broadcast Messages
^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^
@ -278,9 +342,9 @@ of routees dynamically.
This is an example of how to create a resizable router that is defined in configuration: 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`` Several more configuration options are available and described in ``akka.actor.deployment.default.resizer``
section of the reference :ref:`configuration`. section of the reference :ref:`configuration`.