2012-08-28 08:36:14 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
package akka.cluster.routing
|
|
|
|
|
|
2012-09-07 16:57:49 +02:00
|
|
|
import language.postfixOps
|
2012-08-30 13:52:47 +02:00
|
|
|
import scala.concurrent.Await
|
|
|
|
|
import scala.concurrent.util.duration._
|
|
|
|
|
import com.typesafe.config.ConfigFactory
|
|
|
|
|
|
2012-08-28 08:36:14 +02:00
|
|
|
import akka.actor.Actor
|
|
|
|
|
import akka.actor.ActorRef
|
|
|
|
|
import akka.actor.Address
|
2012-08-30 13:52:47 +02:00
|
|
|
import akka.actor.Props
|
|
|
|
|
import akka.cluster.MultiNodeClusterSpec
|
2012-08-28 08:36:14 +02:00
|
|
|
import akka.pattern.ask
|
|
|
|
|
import akka.remote.testkit.MultiNodeConfig
|
|
|
|
|
import akka.remote.testkit.MultiNodeSpec
|
2012-08-30 13:52:47 +02:00
|
|
|
import akka.routing.CurrentRoutees
|
2012-08-28 08:36:14 +02:00
|
|
|
import akka.routing.RoundRobinRouter
|
|
|
|
|
import akka.routing.RoutedActorRef
|
2012-08-30 10:54:58 +02:00
|
|
|
import akka.routing.RouterRoutees
|
2012-08-30 13:52:47 +02:00
|
|
|
import akka.testkit._
|
2012-08-28 08:36:14 +02:00
|
|
|
|
|
|
|
|
object ClusterRoundRobinRoutedActorMultiJvmSpec extends MultiNodeConfig {
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
class SomeActor(routeeType: RouteeType) extends Actor {
|
|
|
|
|
def this() = this(DeployRoutee)
|
|
|
|
|
|
2012-08-28 08:36:14 +02:00
|
|
|
def receive = {
|
2012-09-07 14:54:53 +02:00
|
|
|
case "hit" ⇒ sender ! Reply(routeeType, self)
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
case class Reply(routeeType: RouteeType, ref: ActorRef)
|
|
|
|
|
|
|
|
|
|
sealed trait RouteeType extends Serializable
|
|
|
|
|
object DeployRoutee extends RouteeType
|
|
|
|
|
object LookupRoutee extends RouteeType
|
|
|
|
|
|
2012-08-28 08:36:14 +02:00
|
|
|
val first = role("first")
|
|
|
|
|
val second = role("second")
|
|
|
|
|
val third = role("third")
|
|
|
|
|
val fourth = role("fourth")
|
|
|
|
|
|
|
|
|
|
commonConfig(debugConfig(on = false).
|
|
|
|
|
withFallback(ConfigFactory.parseString("""
|
|
|
|
|
akka.actor.deployment {
|
2012-08-30 10:54:58 +02:00
|
|
|
/router1 {
|
2012-08-28 08:36:14 +02:00
|
|
|
router = round-robin
|
2012-08-29 19:33:19 +02:00
|
|
|
nr-of-instances = 10
|
2012-09-07 12:07:41 +02:00
|
|
|
cluster {
|
|
|
|
|
enabled = on
|
|
|
|
|
max-nr-of-instances-per-node = 2
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
/router3 {
|
|
|
|
|
router = round-robin
|
|
|
|
|
nr-of-instances = 10
|
|
|
|
|
cluster {
|
|
|
|
|
enabled = on
|
|
|
|
|
max-nr-of-instances-per-node = 1
|
2012-09-11 19:11:20 +02:00
|
|
|
allow-local-routees = off
|
2012-09-07 12:07:41 +02:00
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|
2012-09-07 14:54:53 +02:00
|
|
|
/router4 {
|
|
|
|
|
router = round-robin
|
|
|
|
|
nr-of-instances = 10
|
|
|
|
|
cluster {
|
|
|
|
|
enabled = on
|
|
|
|
|
routees-path = "/user/myservice"
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|
|
|
|
|
""")).
|
|
|
|
|
withFallback(MultiNodeClusterSpec.clusterConfig))
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2012-08-28 16:38:05 +02:00
|
|
|
class ClusterRoundRobinRoutedActorMultiJvmNode1 extends ClusterRoundRobinRoutedActorSpec
|
|
|
|
|
class ClusterRoundRobinRoutedActorMultiJvmNode2 extends ClusterRoundRobinRoutedActorSpec
|
|
|
|
|
class ClusterRoundRobinRoutedActorMultiJvmNode3 extends ClusterRoundRobinRoutedActorSpec
|
|
|
|
|
class ClusterRoundRobinRoutedActorMultiJvmNode4 extends ClusterRoundRobinRoutedActorSpec
|
2012-08-28 08:36:14 +02:00
|
|
|
|
|
|
|
|
abstract class ClusterRoundRobinRoutedActorSpec extends MultiNodeSpec(ClusterRoundRobinRoutedActorMultiJvmSpec)
|
|
|
|
|
with MultiNodeClusterSpec
|
|
|
|
|
with ImplicitSender with DefaultTimeout {
|
|
|
|
|
import ClusterRoundRobinRoutedActorMultiJvmSpec._
|
|
|
|
|
|
2012-08-30 10:54:58 +02:00
|
|
|
lazy val router1 = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "router1")
|
2012-09-10 14:11:03 +02:00
|
|
|
lazy val router2 = system.actorOf(Props[SomeActor].withRouter(ClusterRouterConfig(RoundRobinRouter(),
|
|
|
|
|
ClusterRouterSettings(totalInstances = 3, maxInstancesPerNode = 1))), "router2")
|
2012-09-07 12:07:41 +02:00
|
|
|
lazy val router3 = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "router3")
|
2012-09-07 14:54:53 +02:00
|
|
|
lazy val router4 = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "router4")
|
2012-08-29 19:33:19 +02:00
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
def receiveReplies(routeeType: RouteeType, expectedReplies: Int): Map[Address, Int] = {
|
2012-08-29 19:33:19 +02:00
|
|
|
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
|
|
|
|
|
(receiveWhile(5 seconds, messages = expectedReplies) {
|
2012-09-07 14:54:53 +02:00
|
|
|
case Reply(`routeeType`, ref) ⇒ fullAddress(ref)
|
2012-08-29 19:33:19 +02:00
|
|
|
}).foldLeft(zero) {
|
|
|
|
|
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2012-08-30 10:54:58 +02:00
|
|
|
/**
|
|
|
|
|
* 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
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-07 12:07:41 +02:00
|
|
|
"A cluster router with a RoundRobin router" must {
|
2012-08-29 19:33:19 +02:00
|
|
|
"start cluster with 2 nodes" taggedAs LongRunningTest in {
|
|
|
|
|
awaitClusterUp(first, second)
|
2012-08-28 08:36:14 +02:00
|
|
|
enterBarrier("after-1")
|
|
|
|
|
}
|
|
|
|
|
|
2012-08-29 19:33:19 +02:00
|
|
|
"deploy routees to the member nodes in the cluster" taggedAs LongRunningTest in {
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2012-08-29 19:33:19 +02:00
|
|
|
runOn(first) {
|
2012-08-30 10:54:58 +02:00
|
|
|
router1.isInstanceOf[RoutedActorRef] must be(true)
|
2012-08-28 08:36:14 +02:00
|
|
|
|
|
|
|
|
val iterationCount = 10
|
2012-08-29 19:33:19 +02:00
|
|
|
for (i ← 0 until iterationCount) {
|
2012-08-30 10:54:58 +02:00
|
|
|
router1 ! "hit"
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
val replies = receiveReplies(DeployRoutee, iterationCount)
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2012-08-29 19:33:19 +02:00
|
|
|
replies(first) must be > (0)
|
|
|
|
|
replies(second) must be > (0)
|
|
|
|
|
replies(third) must be(0)
|
|
|
|
|
replies(fourth) must be(0)
|
|
|
|
|
replies.values.sum must be(iterationCount)
|
|
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2012-08-29 19:33:19 +02:00
|
|
|
enterBarrier("after-2")
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
"lookup routees on the member nodes in the cluster" taggedAs LongRunningTest in {
|
|
|
|
|
|
|
|
|
|
// cluster consists of first and second
|
|
|
|
|
|
|
|
|
|
system.actorOf(Props(new SomeActor(LookupRoutee)), "myservice")
|
|
|
|
|
enterBarrier("myservice-started")
|
|
|
|
|
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val iterationCount = 10
|
|
|
|
|
for (i ← 0 until iterationCount) {
|
|
|
|
|
router4 ! "hit"
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val replies = receiveReplies(LookupRoutee, iterationCount)
|
|
|
|
|
|
|
|
|
|
replies(first) must be > (0)
|
|
|
|
|
replies(second) must be > (0)
|
|
|
|
|
replies(third) must be(0)
|
|
|
|
|
replies(fourth) must be(0)
|
|
|
|
|
replies.values.sum must be(iterationCount)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-3")
|
|
|
|
|
}
|
|
|
|
|
|
2012-08-29 19:33:19 +02:00
|
|
|
"deploy routees to new nodes in the cluster" taggedAs LongRunningTest in {
|
|
|
|
|
|
|
|
|
|
// add third and fourth
|
|
|
|
|
awaitClusterUp(first, second, third, fourth)
|
|
|
|
|
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val iterationCount = 10
|
|
|
|
|
for (i ← 0 until iterationCount) {
|
2012-08-30 10:54:58 +02:00
|
|
|
router1 ! "hit"
|
2012-08-29 19:33:19 +02:00
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
val replies = receiveReplies(DeployRoutee, iterationCount)
|
2012-08-29 19:33:19 +02:00
|
|
|
|
|
|
|
|
replies.values.foreach { _ must be > (0) }
|
|
|
|
|
replies.values.sum must be(iterationCount)
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
enterBarrier("after-4")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"lookup routees on new nodes in the cluster" taggedAs LongRunningTest in {
|
|
|
|
|
|
|
|
|
|
// cluster consists of first, second, third and fourth
|
|
|
|
|
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val iterationCount = 10
|
|
|
|
|
for (i ← 0 until iterationCount) {
|
|
|
|
|
router4 ! "hit"
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val replies = receiveReplies(LookupRoutee, iterationCount)
|
|
|
|
|
|
|
|
|
|
replies.values.foreach { _ must be > (0) }
|
|
|
|
|
replies.values.sum must be(iterationCount)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-5")
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|
2012-08-30 10:54:58 +02:00
|
|
|
|
2012-09-11 19:11:20 +02:00
|
|
|
"deploy routees to only remote nodes when allow-local-routees = off" taggedAs LongRunningTest in {
|
2012-09-07 12:07:41 +02:00
|
|
|
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val iterationCount = 10
|
|
|
|
|
for (i ← 0 until iterationCount) {
|
|
|
|
|
router3 ! "hit"
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
val replies = receiveReplies(DeployRoutee, iterationCount)
|
2012-09-07 12:07:41 +02:00
|
|
|
|
|
|
|
|
replies(first) must be(0)
|
|
|
|
|
replies(second) must be > (0)
|
|
|
|
|
replies(third) must be > (0)
|
|
|
|
|
replies(fourth) must be > (0)
|
|
|
|
|
replies.values.sum must be(iterationCount)
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
enterBarrier("after-6")
|
2012-09-07 12:07:41 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"deploy programatically defined routees to the member nodes in the cluster" taggedAs LongRunningTest in {
|
2012-08-30 10:54:58 +02:00
|
|
|
|
|
|
|
|
runOn(first) {
|
|
|
|
|
router2.isInstanceOf[RoutedActorRef] must be(true)
|
|
|
|
|
|
|
|
|
|
val iterationCount = 10
|
|
|
|
|
for (i ← 0 until iterationCount) {
|
|
|
|
|
router2 ! "hit"
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
val replies = receiveReplies(DeployRoutee, iterationCount)
|
2012-08-30 10:54:58 +02:00
|
|
|
|
|
|
|
|
// note that router2 has totalInstances = 3, maxInstancesPerNode = 1
|
|
|
|
|
val currentRoutees = Await.result(router2 ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees
|
|
|
|
|
val routeeAddresses = currentRoutees map fullAddress
|
|
|
|
|
|
|
|
|
|
routeeAddresses.size must be(3)
|
|
|
|
|
replies.values.sum must be(iterationCount)
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
enterBarrier("after-7")
|
2012-08-30 10:54:58 +02:00
|
|
|
}
|
|
|
|
|
|
2012-09-07 12:07:41 +02:00
|
|
|
"deploy programatically defined routees to other node when a node becomes down" taggedAs LongRunningTest in {
|
2012-08-30 10:54:58 +02:00
|
|
|
|
|
|
|
|
runOn(first) {
|
|
|
|
|
def currentRoutees = Await.result(router2 ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees
|
|
|
|
|
def routeeAddresses = (currentRoutees map fullAddress).toSet
|
|
|
|
|
|
|
|
|
|
val notUsedAddress = ((roles map address).toSet -- routeeAddresses).head
|
|
|
|
|
|
|
|
|
|
val downAddress = routeeAddresses.find(_ != address(first)).get
|
|
|
|
|
cluster.down(downAddress)
|
|
|
|
|
awaitCond {
|
|
|
|
|
routeeAddresses.contains(notUsedAddress) && !routeeAddresses.contains(downAddress)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val iterationCount = 10
|
|
|
|
|
for (i ← 0 until iterationCount) {
|
|
|
|
|
router2 ! "hit"
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
val replies = receiveReplies(DeployRoutee, iterationCount)
|
2012-08-30 10:54:58 +02:00
|
|
|
|
|
|
|
|
routeeAddresses.size must be(3)
|
|
|
|
|
replies.values.sum must be(iterationCount)
|
|
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
enterBarrier("after-8")
|
2012-08-30 10:54:58 +02:00
|
|
|
}
|
2012-09-10 14:11:03 +02:00
|
|
|
|
2012-08-30 10:54:58 +02:00
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|