2012-11-07 20:36:24 +01:00
|
|
|
/*
|
2013-01-09 01:47:48 +01:00
|
|
|
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
2012-11-07 20:36:24 +01:00
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
package akka.cluster.routing
|
|
|
|
|
|
|
|
|
|
import language.postfixOps
|
|
|
|
|
import java.lang.management.ManagementFactory
|
|
|
|
|
import scala.concurrent.Await
|
2012-11-08 18:49:54 +01:00
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
import com.typesafe.config.Config
|
2012-11-07 20:36:24 +01:00
|
|
|
import com.typesafe.config.ConfigFactory
|
|
|
|
|
|
2012-11-08 18:49:54 +01:00
|
|
|
import akka.actor._
|
|
|
|
|
import akka.cluster.Cluster
|
|
|
|
|
import akka.cluster.MultiNodeClusterSpec
|
|
|
|
|
import akka.cluster.NodeMetrics
|
|
|
|
|
import akka.pattern.ask
|
|
|
|
|
import akka.remote.testkit.{ MultiNodeSpec, MultiNodeConfig }
|
|
|
|
|
import akka.routing.CurrentRoutees
|
|
|
|
|
import akka.routing.FromConfig
|
|
|
|
|
import akka.routing.RouterRoutees
|
|
|
|
|
import akka.testkit.{ LongRunningTest, DefaultTimeout, ImplicitSender }
|
|
|
|
|
|
|
|
|
|
object AdaptiveLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig {
|
2012-11-07 20:36:24 +01:00
|
|
|
|
|
|
|
|
class Routee extends Actor {
|
|
|
|
|
def receive = {
|
|
|
|
|
case _ ⇒ sender ! Reply(Cluster(context.system).selfAddress)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class Memory extends Actor with ActorLogging {
|
2012-11-23 11:22:01 +01:00
|
|
|
var usedMemory: Array[Array[Int]] = _
|
2012-11-07 20:36:24 +01:00
|
|
|
def receive = {
|
|
|
|
|
case AllocateMemory ⇒
|
|
|
|
|
val heap = ManagementFactory.getMemoryMXBean.getHeapMemoryUsage
|
|
|
|
|
// getMax can be undefined (-1)
|
|
|
|
|
val max = math.max(heap.getMax, heap.getCommitted)
|
|
|
|
|
val used = heap.getUsed
|
2012-11-23 11:22:01 +01:00
|
|
|
log.debug("used heap before: [{}] bytes, of max [{}]", used, heap.getMax)
|
|
|
|
|
// allocate 70% of free space
|
|
|
|
|
val allocateBytes = (0.7 * (max - used)).toInt
|
|
|
|
|
val numberOfArrays = allocateBytes / 1024
|
|
|
|
|
usedMemory = Array.ofDim(numberOfArrays, 248) // each 248 element Int array will use ~ 1 kB
|
|
|
|
|
log.debug("used heap after: [{}] bytes", ManagementFactory.getMemoryMXBean.getHeapMemoryUsage.getUsed)
|
|
|
|
|
sender ! "done"
|
2012-11-07 20:36:24 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
case object AllocateMemory
|
|
|
|
|
case class Reply(address: Address)
|
|
|
|
|
|
|
|
|
|
val first = role("first")
|
|
|
|
|
val second = role("second")
|
|
|
|
|
val third = role("third")
|
|
|
|
|
|
|
|
|
|
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString("""
|
|
|
|
|
akka.cluster.metrics.collect-interval = 1s
|
|
|
|
|
akka.cluster.metrics.gossip-interval = 1s
|
2012-11-23 11:22:01 +01:00
|
|
|
akka.cluster.metrics.moving-average-half-life = 2s
|
2012-11-08 18:49:54 +01:00
|
|
|
akka.actor.deployment {
|
|
|
|
|
/router3 = {
|
|
|
|
|
router = adaptive
|
|
|
|
|
metrics-selector = cpu
|
|
|
|
|
nr-of-instances = 9
|
|
|
|
|
}
|
|
|
|
|
/router4 = {
|
|
|
|
|
router = adaptive
|
|
|
|
|
metrics-selector = "akka.cluster.routing.TestCustomMetricsSelector"
|
|
|
|
|
nr-of-instances = 10
|
|
|
|
|
cluster {
|
|
|
|
|
enabled = on
|
|
|
|
|
max-nr-of-instances-per-node = 2
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-11-07 20:36:24 +01:00
|
|
|
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2012-11-08 18:49:54 +01:00
|
|
|
class TestCustomMetricsSelector(config: Config) extends MetricsSelector {
|
|
|
|
|
override def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] = Map.empty
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class AdaptiveLoadBalancingRouterMultiJvmNode1 extends AdaptiveLoadBalancingRouterSpec
|
|
|
|
|
class AdaptiveLoadBalancingRouterMultiJvmNode2 extends AdaptiveLoadBalancingRouterSpec
|
|
|
|
|
class AdaptiveLoadBalancingRouterMultiJvmNode3 extends AdaptiveLoadBalancingRouterSpec
|
2012-11-07 20:36:24 +01:00
|
|
|
|
2012-11-08 18:49:54 +01:00
|
|
|
abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoadBalancingRouterMultiJvmSpec)
|
2012-11-07 20:36:24 +01:00
|
|
|
with MultiNodeClusterSpec
|
|
|
|
|
with ImplicitSender with DefaultTimeout {
|
2012-11-08 18:49:54 +01:00
|
|
|
import AdaptiveLoadBalancingRouterMultiJvmSpec._
|
2012-11-07 20:36:24 +01:00
|
|
|
|
|
|
|
|
def currentRoutees(router: ActorRef) =
|
|
|
|
|
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees
|
|
|
|
|
|
|
|
|
|
def receiveReplies(expectedReplies: Int): Map[Address, Int] = {
|
|
|
|
|
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
|
|
|
|
|
(receiveWhile(5 seconds, messages = expectedReplies) {
|
|
|
|
|
case Reply(address) ⇒ address
|
|
|
|
|
}).foldLeft(zero) {
|
|
|
|
|
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Fills in self address for local ActorRef
|
|
|
|
|
*/
|
|
|
|
|
def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
|
|
|
|
|
case Address(_, _, None, None) ⇒ cluster.selfAddress
|
|
|
|
|
case a ⇒ a
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def startRouter(name: String): ActorRef = {
|
|
|
|
|
val router = system.actorOf(Props[Routee].withRouter(ClusterRouterConfig(
|
2012-11-08 18:49:54 +01:00
|
|
|
local = AdaptiveLoadBalancingRouter(HeapMetricsSelector),
|
2013-03-14 20:32:43 +01:00
|
|
|
settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 1, useRole = None))), name)
|
2013-03-24 22:01:57 +01:00
|
|
|
// it may take some time until router receives cluster member events
|
|
|
|
|
awaitAssert { currentRoutees(router).size must be(roles.size) }
|
2012-11-07 20:36:24 +01:00
|
|
|
currentRoutees(router).map(fullAddress).toSet must be(roles.map(address).toSet)
|
|
|
|
|
router
|
|
|
|
|
}
|
|
|
|
|
|
2012-11-08 18:49:54 +01:00
|
|
|
"A cluster with a AdaptiveLoadBalancingRouter" must {
|
2012-11-07 20:36:24 +01:00
|
|
|
"start cluster nodes" taggedAs LongRunningTest in {
|
|
|
|
|
awaitClusterUp(roles: _*)
|
|
|
|
|
enterBarrier("after-1")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"use all nodes in the cluster when not overloaded" taggedAs LongRunningTest in {
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val router1 = startRouter("router1")
|
|
|
|
|
|
|
|
|
|
// collect some metrics before we start
|
2012-11-15 12:48:13 +01:00
|
|
|
Thread.sleep(cluster.settings.MetricsInterval.toMillis * 10)
|
2012-11-07 20:36:24 +01:00
|
|
|
|
|
|
|
|
val iterationCount = 100
|
2012-11-15 12:48:13 +01:00
|
|
|
1 to iterationCount foreach { _ ⇒
|
2012-11-07 20:36:24 +01:00
|
|
|
router1 ! "hit"
|
2012-11-08 18:49:54 +01:00
|
|
|
// wait a while between each message, since metrics is collected periodically
|
2012-11-07 20:36:24 +01:00
|
|
|
Thread.sleep(10)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val replies = receiveReplies(iterationCount)
|
|
|
|
|
|
|
|
|
|
replies(first) must be > (0)
|
|
|
|
|
replies(second) must be > (0)
|
|
|
|
|
replies(third) must be > (0)
|
|
|
|
|
replies.values.sum must be(iterationCount)
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-2")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"prefer node with more free heap capacity" taggedAs LongRunningTest in {
|
|
|
|
|
System.gc()
|
|
|
|
|
enterBarrier("gc")
|
|
|
|
|
|
|
|
|
|
runOn(second) {
|
2012-11-23 11:22:01 +01:00
|
|
|
within(20.seconds) {
|
|
|
|
|
system.actorOf(Props[Memory], "memory") ! AllocateMemory
|
|
|
|
|
expectMsg("done")
|
|
|
|
|
}
|
2012-11-07 20:36:24 +01:00
|
|
|
}
|
|
|
|
|
enterBarrier("heap-allocated")
|
|
|
|
|
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val router2 = startRouter("router2")
|
|
|
|
|
|
|
|
|
|
// collect some metrics before we start
|
2012-11-23 11:22:01 +01:00
|
|
|
Thread.sleep(cluster.settings.MetricsInterval.toMillis * 10)
|
2012-11-07 20:36:24 +01:00
|
|
|
|
2012-11-23 11:22:01 +01:00
|
|
|
val iterationCount = 3000
|
|
|
|
|
1 to iterationCount foreach { _ ⇒
|
2012-11-07 20:36:24 +01:00
|
|
|
router2 ! "hit"
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val replies = receiveReplies(iterationCount)
|
|
|
|
|
|
|
|
|
|
replies(third) must be > (replies(second))
|
|
|
|
|
replies.values.sum must be(iterationCount)
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-3")
|
|
|
|
|
}
|
2012-11-08 18:49:54 +01:00
|
|
|
|
|
|
|
|
"create routees from configuration" taggedAs LongRunningTest in {
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val router3 = system.actorOf(Props[Memory].withRouter(FromConfig()), "router3")
|
2013-03-24 22:01:57 +01:00
|
|
|
// it may take some time until router receives cluster member events
|
|
|
|
|
awaitAssert { currentRoutees(router3).size must be(9) }
|
2012-11-08 18:49:54 +01:00
|
|
|
currentRoutees(router3).map(fullAddress).toSet must be(Set(address(first)))
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("after-4")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"create routees from cluster.enabled configuration" taggedAs LongRunningTest in {
|
|
|
|
|
runOn(first) {
|
|
|
|
|
val router4 = system.actorOf(Props[Memory].withRouter(FromConfig()), "router4")
|
2013-03-24 22:01:57 +01:00
|
|
|
// it may take some time until router receives cluster member events
|
|
|
|
|
awaitAssert { currentRoutees(router4).size must be(6) }
|
2012-11-08 18:49:54 +01:00
|
|
|
currentRoutees(router4).map(fullAddress).toSet must be(Set(
|
|
|
|
|
address(first), address(second), address(third)))
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("after-5")
|
|
|
|
|
}
|
2012-11-07 20:36:24 +01:00
|
|
|
}
|
|
|
|
|
}
|