ClusterLoadBalancingRouter and refactoring of metrics, see #2547
* MetricsSelector, calculate capacity, weights and allocate weighted routee refs * ClusterLoadBalancingRouterSpec * Optional heap max * Constants for the metric fields * Refactoring of Metric and decay * Rewrite of DataStreamSpec * Correction of EWMA and removal of BigInt, BigDecimal * Separation of MetricsCollector into trait and two classes, SigarMetricsCollector and JmxMetricsCollector * This will reduce cost when sigar is not installed, such as avoiding throwing and catching exc for every call * Improved error handling for loading sigar * Made MetricsCollector implementation configurable * Tested with sigar
This commit is contained in:
parent
f306964fca
commit
c9d206764a
18 changed files with 1065 additions and 733 deletions
|
|
@ -1,61 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.routing
|
||||
|
||||
import akka.remote.testkit.{MultiNodeSpec, MultiNodeConfig}
|
||||
import akka.testkit.{LongRunningTest, DefaultTimeout, ImplicitSender}
|
||||
import akka.actor._
|
||||
import akka.cluster.{ MemberStatus, MultiNodeClusterSpec }
|
||||
import akka.cluster.routing.ClusterRoundRobinRoutedActorMultiJvmSpec.SomeActor
|
||||
|
||||
|
||||
object ClusterAdaptiveLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
val fifth = role("fifth")
|
||||
|
||||
// TODO - config
|
||||
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
}
|
||||
|
||||
class ClusterAdaptiveLoadBalancingRouterMultiJvmNode1 extends ClusterAdaptiveLoadBalancingRouterSpec
|
||||
class ClusterAdaptiveLoadBalancingRouterMultiJvmNode2 extends ClusterAdaptiveLoadBalancingRouterSpec
|
||||
class ClusterAdaptiveLoadBalancingRouterMultiJvmNode3 extends ClusterAdaptiveLoadBalancingRouterSpec
|
||||
class ClusterAdaptiveLoadBalancingRouterMultiJvmNode4 extends ClusterAdaptiveLoadBalancingRouterSpec
|
||||
class ClusterAdaptiveLoadBalancingRouterMultiJvmNode5 extends ClusterAdaptiveLoadBalancingRouterSpec
|
||||
|
||||
abstract class ClusterAdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(ClusterAdaptiveLoadBalancingRouterMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with DefaultTimeout {
|
||||
import ClusterAdaptiveLoadBalancingRouterMultiJvmSpec._
|
||||
|
||||
// TODO configure properly and leverage the other pending load balancing routers
|
||||
lazy val router1 = system.actorOf(Props[SomeActor].withRouter(ClusterRouterConfig(MemoryLoadBalancingRouter(),
|
||||
ClusterRouterSettings(totalInstances = 3, maxInstancesPerNode = 1))), "router1")
|
||||
lazy val router2 = system.actorOf(Props[SomeActor].withRouter(ClusterRouterConfig(MemoryLoadBalancingRouter(),
|
||||
ClusterRouterSettings(totalInstances = 3, maxInstancesPerNode = 1))), "router2")
|
||||
lazy val router3 = system.actorOf(Props[SomeActor].withRouter(ClusterRouterConfig(MemoryLoadBalancingRouter(),
|
||||
ClusterRouterSettings(totalInstances = 3, maxInstancesPerNode = 1))), "router3")
|
||||
lazy val router4 = system.actorOf(Props[SomeActor].withRouter(ClusterRouterConfig(MemoryLoadBalancingRouter(),
|
||||
ClusterRouterSettings(totalInstances = 3, maxInstancesPerNode = 1))), "router4")
|
||||
lazy val router5 = system.actorOf(Props[SomeActor].withRouter(ClusterRouterConfig(MemoryLoadBalancingRouter(),
|
||||
ClusterRouterSettings(totalInstances = 3, maxInstancesPerNode = 1))), "router5")
|
||||
|
||||
"A cluster with a ClusterAdaptiveLoadBalancingRouter" must {
|
||||
"start cluster with 5 nodes" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(roles: _*)
|
||||
enterBarrier("cluster-started")
|
||||
awaitCond(clusterView.members.filter(_.status == MemberStatus.Up).size == roles.size)
|
||||
awaitCond(clusterView.clusterMetrics.size == roles.size)
|
||||
enterBarrier("cluster-metrics-consumer-ready")
|
||||
}
|
||||
// TODO the rest of the necessary testing. All the work needed for consumption and extraction
|
||||
// of the data needed is in ClusterMetricsCollector._
|
||||
}
|
||||
}
|
||||
|
|
@ -1,155 +0,0 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.routing
|
||||
|
||||
import language.implicitConversions
|
||||
import language.postfixOps
|
||||
import akka.actor._
|
||||
import akka.cluster._
|
||||
import akka.routing._
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.event.Logging
|
||||
import java.util.concurrent.atomic.{ AtomicLong, AtomicReference }
|
||||
import akka.routing.Destination
|
||||
import akka.cluster.NodeMetrics
|
||||
import akka.routing.Broadcast
|
||||
import akka.actor.OneForOneStrategy
|
||||
import akka.cluster.ClusterEvent.ClusterMetricsChanged
|
||||
import akka.cluster.ClusterEvent.CurrentClusterState
|
||||
import util.Try
|
||||
import akka.cluster.NodeMetrics.{ NodeMetricsComparator, MetricValues }
|
||||
import NodeMetricsComparator._
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*
|
||||
* Trait that embodies the contract for all load balancing implementations.
|
||||
*/
|
||||
private[cluster] trait LoadBalancer {
|
||||
|
||||
/**
|
||||
* Compares only those nodes that are deemed 'available' by the
|
||||
* [[akka.routing.RouteeProvider]]
|
||||
*/
|
||||
def selectNodeByHealth(availableNodes: Set[NodeMetrics]): Option[Address]
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[cluster] object ClusterLoadBalancingRouter {
|
||||
val defaultSupervisorStrategy: SupervisorStrategy = OneForOneStrategy() {
|
||||
case _ ⇒ SupervisorStrategy.Escalate
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*
|
||||
* The abstract consumer of [[akka.cluster.ClusterMetricsChanged]] events and the primary consumer
|
||||
* of cluster metric data. This strategy is a metrics-aware router which performs load balancing of
|
||||
* cluster nodes with a fallback strategy of a [[akka.routing.RoundRobinRouter]].
|
||||
*
|
||||
* Load balancing of nodes is based on .. etc etc desc forthcoming
|
||||
*/
|
||||
trait ClusterAdaptiveLoadBalancingRouterLike extends RoundRobinLike with LoadBalancer { this: RouterConfig ⇒
|
||||
|
||||
def routerDispatcher: String
|
||||
|
||||
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 next = new AtomicLong(0)
|
||||
|
||||
val nodeMetrics = new AtomicReference[Set[NodeMetrics]](Set.empty)
|
||||
|
||||
val metricsListener = routeeProvider.context.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case ClusterMetricsChanged(metrics) ⇒ receiveMetrics(metrics)
|
||||
case _: CurrentClusterState ⇒ // ignore
|
||||
}
|
||||
def receiveMetrics(metrics: Set[NodeMetrics]): Unit = {
|
||||
val availableNodes = routeeProvider.routees.map(_.path.address).toSet
|
||||
val updated: Set[NodeMetrics] = nodeMetrics.get.collect { case node if availableNodes contains node.address ⇒ node }
|
||||
nodeMetrics.set(updated)
|
||||
}
|
||||
override def postStop(): Unit = Cluster(routeeProvider.context.system) unsubscribe self
|
||||
}).withDispatcher(routerDispatcher), name = "metricsListener")
|
||||
Cluster(routeeProvider.context.system).subscribe(metricsListener, classOf[ClusterMetricsChanged])
|
||||
|
||||
def getNext(): ActorRef = {
|
||||
// TODO use as/where you will... selects by health category based on the implementation
|
||||
val address: Option[Address] = selectNodeByHealth(nodeMetrics.get)
|
||||
// TODO actual routee selection. defaults to round robin.
|
||||
routeeProvider.routees((next.getAndIncrement % routees.size).asInstanceOf[Int])
|
||||
}
|
||||
|
||||
def routeTo(): ActorRef = if (routeeProvider.routees.isEmpty) routeeProvider.context.system.deadLetters else getNext()
|
||||
|
||||
{
|
||||
case (sender, message) ⇒
|
||||
message match {
|
||||
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
||||
case msg ⇒ List(Destination(sender, routeTo()))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Selects by all monitored metric types (memory, network latency, cpu...) and
|
||||
* chooses the healthiest node to route to.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
private[cluster] case class ClusterAdaptiveMetricsLoadBalancingRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = ClusterLoadBalancingRouter.defaultSupervisorStrategy)
|
||||
extends RouterConfig with ClusterAdaptiveLoadBalancingRouterLike with MetricsAwareClusterNodeSelector {
|
||||
|
||||
// TODO
|
||||
def selectNodeByHealth(nodes: Set[NodeMetrics]): Option[Address] = {
|
||||
val s = Set(selectByMemory(nodes), selectByNetworkLatency(nodes), selectByCpu(nodes))
|
||||
s.head // TODO select the Address that appears with the highest or lowest frequency
|
||||
}
|
||||
}
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
private[cluster] case class MemoryLoadBalancingRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = ClusterLoadBalancingRouter.defaultSupervisorStrategy)
|
||||
extends RouterConfig with ClusterAdaptiveLoadBalancingRouterLike with MetricsAwareClusterNodeSelector {
|
||||
|
||||
def selectNodeByHealth(nodes: Set[NodeMetrics]): Option[Address] = selectByMemory(nodes)
|
||||
}
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
private[cluster] case class CpuLoadBalancer(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = ClusterLoadBalancingRouter.defaultSupervisorStrategy)
|
||||
extends RouterConfig with ClusterAdaptiveLoadBalancingRouterLike with MetricsAwareClusterNodeSelector {
|
||||
|
||||
// TODO
|
||||
def selectNodeByHealth(nodes: Set[NodeMetrics]): Option[Address] = None
|
||||
}
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
private[cluster] case class NetworkLatencyLoadBalancer(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = ClusterLoadBalancingRouter.defaultSupervisorStrategy)
|
||||
extends RouterConfig with ClusterAdaptiveLoadBalancingRouterLike with MetricsAwareClusterNodeSelector {
|
||||
|
||||
// TODO
|
||||
def selectNodeByHealth(nodes: Set[NodeMetrics]): Option[Address] = None
|
||||
}
|
||||
|
|
@ -0,0 +1,162 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.routing
|
||||
|
||||
import language.postfixOps
|
||||
import scala.concurrent.duration._
|
||||
import java.lang.management.ManagementFactory
|
||||
import akka.remote.testkit.{ MultiNodeSpec, MultiNodeConfig }
|
||||
import akka.testkit.{ LongRunningTest, DefaultTimeout, ImplicitSender }
|
||||
import akka.actor._
|
||||
import akka.cluster.{ MemberStatus, MultiNodeClusterSpec }
|
||||
import scala.concurrent.Await
|
||||
import akka.routing.RouterRoutees
|
||||
import akka.pattern.ask
|
||||
import akka.routing.CurrentRoutees
|
||||
import akka.cluster.Cluster
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
object ClusterLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class Routee extends Actor {
|
||||
var usedMemory: Array[Byte] = _
|
||||
def receive = {
|
||||
case _ ⇒ sender ! Reply(Cluster(context.system).selfAddress)
|
||||
}
|
||||
}
|
||||
|
||||
class Memory extends Actor with ActorLogging {
|
||||
var usedMemory: Array[Byte] = _
|
||||
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
|
||||
val allocate = (0.8 * (max - used)).toInt
|
||||
usedMemory = Array.fill(allocate)(ThreadLocalRandom.current.nextInt(127).toByte)
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
}
|
||||
|
||||
class ClusterLoadBalancingRouterMultiJvmNode1 extends ClusterLoadBalancingRouterSpec
|
||||
class ClusterLoadBalancingRouterMultiJvmNode2 extends ClusterLoadBalancingRouterSpec
|
||||
class ClusterLoadBalancingRouterMultiJvmNode3 extends ClusterLoadBalancingRouterSpec
|
||||
|
||||
abstract class ClusterLoadBalancingRouterSpec extends MultiNodeSpec(ClusterLoadBalancingRouterMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with DefaultTimeout {
|
||||
import ClusterLoadBalancingRouterMultiJvmSpec._
|
||||
|
||||
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(
|
||||
local = ClusterLoadBalancingRouter(HeapMetricsSelector),
|
||||
settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 1))), name)
|
||||
awaitCond {
|
||||
// it may take some time until router receives cluster member events
|
||||
currentRoutees(router).size == roles.size
|
||||
}
|
||||
currentRoutees(router).map(fullAddress).toSet must be(roles.map(address).toSet)
|
||||
router
|
||||
}
|
||||
|
||||
"A cluster with a ClusterLoadBalancingRouter" must {
|
||||
"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
|
||||
Thread.sleep(10000)
|
||||
|
||||
val iterationCount = 100
|
||||
for (i ← 0 until iterationCount) {
|
||||
router1 ! "hit"
|
||||
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) {
|
||||
system.actorOf(Props[Memory], "memory") ! AllocateMemory
|
||||
}
|
||||
enterBarrier("heap-allocated")
|
||||
|
||||
runOn(first) {
|
||||
val router2 = startRouter("router2")
|
||||
router2
|
||||
|
||||
// collect some metrics before we start
|
||||
Thread.sleep(10000)
|
||||
|
||||
val iterationCount = 100
|
||||
for (i ← 0 until iterationCount) {
|
||||
router2 ! "hit"
|
||||
Thread.sleep(10)
|
||||
}
|
||||
|
||||
val replies = receiveReplies(iterationCount)
|
||||
|
||||
replies(third) must be > (replies(second))
|
||||
replies.values.sum must be(iterationCount)
|
||||
|
||||
}
|
||||
|
||||
enterBarrier("after-3")
|
||||
}
|
||||
}
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue