diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 06d3b01a1b..39d1c47758 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -138,16 +138,24 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce } def parseConfig(key: String, config: Config): Option[Deploy] = { - val deployment = config.withFallback(default) + val router = createRouterConfig(deployment.getString("router"), key, config, deployment) + Some(Deploy(key, deployment, router, NoScopeGiven)) + } + /** + * Factory method for creating `RouterConfig` + * @param routerType the configured name of the router, or FQCN + * @param key the full configuration key of the deployment section + * @param config the user defined config of the deployment, without defaults + * @param deployment the deployment config, with defaults + */ + protected def createRouterConfig(routerType: String, key: String, config: Config, deployment: Config): RouterConfig = { val routees = Vector() ++ deployment.getStringList("routees.paths").asScala - val nrOfInstances = deployment.getInt("nr-of-instances") + val 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 { + routerType match { case "from-code" ⇒ NoRouter case "round-robin" ⇒ RoundRobinRouter(nrOfInstances, routees, resizer) case "random" ⇒ RandomRouter(nrOfInstances, routees, resizer) @@ -169,7 +177,6 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce .format(fqn, key), exception) }).get } - - Some(Deploy(key, deployment, router, NoScopeGiven)) } + } diff --git a/akka-cluster/src/main/resources/reference.conf b/akka-cluster/src/main/resources/reference.conf index 8539e56b1c..f24aa2dee8 100644 --- a/akka-cluster/src/main/resources/reference.conf +++ b/akka-cluster/src/main/resources/reference.conf @@ -70,7 +70,7 @@ akka { failure-detector { # FQCN of the failure detector implementation. - # It must implement akka.cluster.cluster.FailureDetector and + # It must implement akka.cluster.FailureDetector and # have constructor with akka.actor.ActorSystem and # akka.cluster.ClusterSettings parameters implementation-class = "akka.cluster.AccrualFailureDetector" @@ -106,14 +106,6 @@ akka { max-sample-size = 1000 } - # Uses JMX and Hyperic SIGAR, if SIGAR is on the classpath. - # Metrics that are only available with SIGAR for load balancing of nodes are: - # 1. Network latency - # 2. CPU - # A. Combined CPU (sum of User + Sys + Nice + Wait, in percentage) - # B. System load average: on some OS the JMX load average may not be available, - # however if SIGAR is on the classpath it is available on any OS. - # C. The number of cores per processor metrics { # Enable or disable metrics collector for load-balancing nodes. enabled = on @@ -121,6 +113,8 @@ akka { # FQCN of the metrics collector implementation. # It must implement akka.cluster.cluster.MetricsCollector and # have constructor with akka.actor.ActorSystem parameter. + # The default SigarMetricsCollector uses JMX and Hyperic SIGAR, if SIGAR + # is on the classpath, otherwise only JMX. collector-class = "akka.cluster.SigarMetricsCollector" # How often metrics is sampled on a node. @@ -131,10 +125,9 @@ akka { # How quickly the exponential weighting of past data is decayed compared to # new data. Set lower to increase the bias toward newer values. - # Corresponds to 'N time periods' as explained in - # http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average - # Must be >= 1 - rate-of-decay = 10 + # It takes about 4 half-life to drop below 10% contribution, and 7 to drop + # below 1%. + decay-half-life-duration = 12s } # If the tick-duration of the default scheduler is longer than the @@ -156,6 +149,16 @@ akka { } # Default configuration for routers + actor.deployment.default { + # MetricsSelector to use + # - available: "mix", "heap", "cpu", "load" + # - or: Fully qualified class name of the MetricsSelector class. + # The class must extend akka.cluster.routing.MetricsSelector + # and have a constructor with com.typesafe.config.Config + # parameter. + # - default is "mix" + metrics-selector = mix + } actor.deployment.default.cluster { # enable cluster aware router that deploys to nodes in the cluster enabled = off @@ -182,4 +185,5 @@ akka { routees-path = "" } + } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRefProvider.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRefProvider.scala index 024dfdc00c..eb58d6ef4a 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRefProvider.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRefProvider.scala @@ -3,6 +3,7 @@ */ package akka.cluster +import scala.collection.JavaConverters.iterableAsScalaIterableConverter import com.typesafe.config.Config import akka.ConfigurationException import akka.actor.Actor @@ -25,6 +26,14 @@ import akka.event.EventStream import akka.remote.RemoteActorRefProvider import akka.remote.RemoteDeployer import akka.remote.routing.RemoteRouterConfig +import akka.routing.RouterConfig +import akka.routing.DefaultResizer +import akka.cluster.routing.AdaptiveLoadBalancingRouter +import akka.cluster.routing.MixMetricsSelector +import akka.cluster.routing.HeapMetricsSelector +import akka.cluster.routing.SystemLoadAverageMetricsSelector +import akka.cluster.routing.CpuMetricsSelector +import akka.cluster.routing.MetricsSelector /** * INTERNAL API @@ -45,7 +54,7 @@ class ClusterActorRefProvider( remoteDeploymentWatcher = system.systemActorOf(Props[RemoteDeploymentWatcher], "RemoteDeploymentWatcher") } - override val deployer: ClusterDeployer = new ClusterDeployer(settings, dynamicAccess) + override lazy val deployer: ClusterDeployer = new ClusterDeployer(settings, dynamicAccess) /** * This method is overridden here to keep track of remote deployed actors to @@ -108,6 +117,36 @@ private[akka] class ClusterDeployer(_settings: ActorSystem.Settings, _pm: Dynami case None ⇒ None } } + + override protected def createRouterConfig(routerType: String, key: String, config: Config, deployment: Config): RouterConfig = { + val routees = Vector() ++ deployment.getStringList("routees.paths").asScala + val nrOfInstances = deployment.getInt("nr-of-instances") + val resizer = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None + + routerType match { + case "adaptive" ⇒ + val metricsSelector = deployment.getString("metrics-selector") match { + case "mix" ⇒ MixMetricsSelector() + case "heap" ⇒ HeapMetricsSelector + case "cpu" ⇒ CpuMetricsSelector + case "load" ⇒ SystemLoadAverageMetricsSelector + case fqn ⇒ + val args = Seq(classOf[Config] -> deployment) + dynamicAccess.createInstanceFor[MetricsSelector](fqn, args).recover({ + case exception ⇒ throw new IllegalArgumentException( + ("Cannot instantiate metrics-selector [%s], defined in [%s], " + + "make sure it extends [akka.cluster.routing.MetricsSelector] and " + + "has constructor with [com.typesafe.config.Config] parameter") + .format(fqn, key), exception) + }).get + } + + AdaptiveLoadBalancingRouter(metricsSelector, nrOfInstances, routees, resizer) + + case _ ⇒ super.createRouterConfig(routerType, key, config, deployment) + } + + } } @SerialVersionUID(1L) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala index 8d87f3fe53..feed89c24c 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterEvent.scala @@ -10,6 +10,8 @@ import akka.cluster.ClusterEvent._ import akka.cluster.MemberStatus._ import akka.event.EventStream import akka.actor.AddressTerminated +import java.lang.Iterable +import scala.collection.JavaConverters /** * Domain events published to the event bus. @@ -139,11 +141,18 @@ object ClusterEvent { } /** - * INTERNAL API * - * Current snapshot of cluster member metrics. Published to subscribers. + * Current snapshot of cluster node metrics. Published to subscribers. */ - case class ClusterMetricsChanged(nodes: Set[NodeMetrics]) extends ClusterDomainEvent + case class ClusterMetricsChanged(nodeMetrics: Set[NodeMetrics]) extends ClusterDomainEvent { + /** + * Java API + */ + def getNodeMetrics: java.lang.Iterable[NodeMetrics] = { + import scala.collection.JavaConverters._ + nodeMetrics.asJava + } + } /** * INTERNAL API diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala index ab3b6f5777..50f0fd81c6 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala @@ -9,13 +9,11 @@ import java.lang.System.{ currentTimeMillis ⇒ newTimestamp } import java.lang.management.{ OperatingSystemMXBean, MemoryMXBean, ManagementFactory } import java.lang.reflect.InvocationTargetException import java.lang.reflect.Method - import scala.collection.immutable.{ SortedSet, Map } import scala.concurrent.duration._ import scala.concurrent.forkjoin.ThreadLocalRandom import scala.runtime.{ RichLong, RichDouble, RichInt } import scala.util.{ Try, Success, Failure } - import akka.ConfigurationException import akka.actor.Actor import akka.actor.ActorLogging @@ -26,6 +24,7 @@ import akka.actor.DynamicAccess import akka.actor.ExtendedActorSystem import akka.cluster.MemberStatus.Up import akka.event.Logging +import java.lang.management.MemoryUsage /** * INTERNAL API. @@ -37,8 +36,8 @@ import akka.event.Logging * * Metrics sampling is delegated to the [[akka.cluster.MetricsCollector]]. * - * Calculation of statistical data for each monitored process is delegated to the - * [[akka.cluster.DataStream]] for exponential smoothing, with additional decay factor. + * Smoothing of the data for each monitored process is delegated to the + * [[akka.cluster.EWMA]] for exponential weighted moving average. */ private[cluster] class ClusterMetricsCollector(publisher: ActorRef) extends Actor with ActorLogging { @@ -58,7 +57,7 @@ private[cluster] class ClusterMetricsCollector(publisher: ActorRef) extends Acto /** * The latest metric values with their statistical data. */ - var latestGossip: MetricsGossip = MetricsGossip() + var latestGossip: MetricsGossip = MetricsGossip.empty /** * The metrics collector that samples data on the node. @@ -160,6 +159,13 @@ private[cluster] class ClusterMetricsCollector(publisher: ActorRef) extends Acto } +/** + * INTERNAL API + */ +private[cluster] object MetricsGossip { + val empty = MetricsGossip() +} + /** * INTERNAL API * @@ -195,11 +201,11 @@ private[cluster] case class MetricsGossip(nodes: Set[NodeMetrics] = Set.empty) { * Adds new local [[akka.cluster.NodeMetrics]] and initializes the data, or merges an existing. */ def :+(data: NodeMetrics): MetricsGossip = { - val previous = metricsFor(data) + val previous = metricsFor(data.address) val names = previous map (_.name) val (toMerge: Set[Metric], unseen: Set[Metric]) = data.metrics partition (a ⇒ names contains a.name) - val merged = toMerge flatMap (latest ⇒ previous.collect { case peer if latest same peer ⇒ peer :+ latest }) + val merged = toMerge flatMap (latest ⇒ previous.collect { case peer if latest sameAs peer ⇒ peer :+ latest }) val refreshed = nodes filterNot (_.address == data.address) copy(nodes = refreshed + data.copy(metrics = unseen ++ merged)) @@ -213,7 +219,9 @@ private[cluster] case class MetricsGossip(nodes: Set[NodeMetrics] = Set.empty) { /** * Returns metrics for a node if exists. */ - def metricsFor(node: NodeMetrics): Set[Metric] = nodes flatMap (n ⇒ if (n same node) n.metrics else Set.empty[Metric]) + def metricsFor(address: Address): Set[Metric] = nodes collectFirst { + case n if (n.address == address) ⇒ n.metrics + } getOrElse Set.empty[Metric] } @@ -223,155 +231,139 @@ private[cluster] case class MetricsGossip(nodes: Set[NodeMetrics] = Set.empty) { */ private[cluster] case class MetricsGossipEnvelope(from: Address, gossip: MetricsGossip) extends ClusterMessage +object EWMA { + /** + * Calculate the alpha (decay factor) used in [[akka.cluster.EWMA]] + * from specified half-life and interval between observations. + * It takes about 4 half-life to drop below 10% contribution, and 7 to drop + * below 1%. + */ + def alpha(halfLife: FiniteDuration, collectInterval: FiniteDuration): Double = { + val halfLifeMillis = halfLife.toMillis + require(halfLife.toMillis > 0, "halfLife must be > 0 s") + val decayRate = 0.69315 / halfLifeMillis + 1 - math.exp(-decayRate * collectInterval.toMillis) + } +} + /** * The exponentially weighted moving average (EWMA) approach captures short-term * movements in volatility for a conditional volatility forecasting model. By virtue * of its alpha, or decay factor, this provides a statistical streaming data model * that is exponentially biased towards newer entries. * + * http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average + * * An EWMA only needs the most recent forecast value to be kept, as opposed to a standard * moving average model. * * INTERNAL API * - * @param decay sets how quickly the exponential weighting decays for past data compared to new data - * Corresponds to 'N time periods' as explained in - * http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average + * @param alpha decay factor, sets how quickly the exponential weighting decays for past data compared to new data, + * see http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average * - * @param ewma the current exponentially weighted moving average, e.g. Y(n - 1), or, + * @param value the current exponentially weighted moving average, e.g. Y(n - 1), or, * the sampled value resulting from the previous smoothing iteration. * This value is always used as the previous EWMA to calculate the new EWMA. * - * @param timestamp the most recent time of sampling - * - * @param startTime the time of initial sampling for this data stream */ -private[cluster] case class DataStream(decay: Int, ewma: Double, startTime: Long, timestamp: Long) - extends ClusterMessage { +private[cluster] case class EWMA(value: Double, alpha: Double) extends ClusterMessage { - require(decay >= 1, "Rate of decay must be >= 1") - - /** - * The rate at which the weights of past observations - * decay as they become more distant. - */ - private val α = 2.0 / (decay + 1) + require(0.0 <= alpha && alpha <= 1.0, "alpha must be between 0.0 and 1.0") /** * Calculates the exponentially weighted moving average for a given monitored data set. * * @param xn the new data point - * - * @return a [[akka.cluster.DataStream]] with the updated yn and timestamp + * @return a new [[akka.cluster.EWMA]] with the updated value */ - def :+(xn: Double): DataStream = copy(ewma = (α * xn) + (1 - α) * ewma, timestamp = newTimestamp) - - /** - * The duration of observation for this data stream - */ - def duration: FiniteDuration = (timestamp - startTime).millis + def :+(xn: Double): EWMA = copy(value = (alpha * xn) + (1 - alpha) * value) } /** - * INTERNAL API - * * @param name the metric name - * * @param value the metric value, which may or may not be defined, it must be a valid numerical value, * see [[akka.cluster.MetricNumericConverter.defined()]] - * * @param average the data stream of the metric value, for trending over time. Metrics that are already - * averages (e.g. system load average) or finite (e.g. as total cores), are not trended. + * averages (e.g. system load average) or finite (e.g. as number of processors), are not trended. */ -private[cluster] case class Metric private (name: String, value: Option[Number], average: Option[DataStream], - dummy: Boolean) // dummy because of overloading clash with apply +case class Metric private (name: String, value: Number, private val average: Option[EWMA]) extends ClusterMessage with MetricNumericConverter { - require(value.isEmpty || defined(value.get), "Invalid Metric [%s] value [%]".format(name, value)) + require(defined(value), "Invalid Metric [%s] value [%]".format(name, value)) /** * If defined ( [[akka.cluster.MetricNumericConverter.defined()]] ), updates the new * data point, and if defined, updates the data stream. Returns the updated metric. */ - def :+(latest: Metric): Metric = latest.value match { - case Some(v) if this same latest ⇒ average match { - case Some(previous) ⇒ copy(value = latest.value, average = Some(previous :+ v.doubleValue)) - case None if latest.average.isDefined ⇒ copy(value = latest.value, average = latest.average) - case None if latest.average.isEmpty ⇒ copy(value = latest.value) - } - case None ⇒ this + def :+(latest: Metric): Metric = if (this sameAs latest) average match { + case Some(avg) ⇒ copy(value = latest.value, average = Some(avg :+ latest.value.doubleValue)) + case None if latest.average.isDefined ⇒ copy(value = latest.value, average = latest.average) + case _ ⇒ copy(value = latest.value) } - - def isDefined: Boolean = value.isDefined + else this /** - * The numerical value of the average, if defined, otherwise the latest value, - * if defined. + * The numerical value of the average, if defined, otherwise the latest value */ - def averageValue: Option[Double] = - average map (_.ewma) orElse value.map(_.doubleValue) + def smoothValue: Double = average match { + case Some(avg) ⇒ avg.value + case None ⇒ value.doubleValue + } + + /** + * @return true if this value is smoothed + */ + def isSmooth: Boolean = average.isDefined /** * Returns true if that is tracking the same metric as this. */ - def same(that: Metric): Boolean = name == that.name + def sameAs(that: Metric): Boolean = name == that.name } /** - * INTERNAL API - * - * Companion object of Metric class. + * Factory for creating valid Metric instances. */ -private[cluster] object Metric extends MetricNumericConverter { - import NodeMetrics.MetricValues._ - - private def definedValue(value: Option[Number]): Option[Number] = - if (value.isDefined && defined(value.get)) value else None +object Metric extends MetricNumericConverter { /** - * Evaluates validity of value based on whether it is available (SIGAR on classpath) - * or defined for the OS (JMX). If undefined we set the value option to None and do not modify - * the latest sampled metric to avoid skewing the statistical trend. - * - * @param decay rate of decay for values applicable for trending + * Creates a new Metric instance if the value is valid, otherwise None + * is returned. Invalid numeric values are negative Int/Long and NaN/Infinite + * Double values. */ - def apply(name: String, value: Option[Number], decay: Option[Int]): Metric = { - val dv = definedValue(value) - val average = - if (decay.isDefined && dv.isDefined) { - val now = newTimestamp - Some(DataStream(decay.get, dv.get.doubleValue, now, now)) - } else - None - new Metric(name, dv, average, true) + def create(name: String, value: Number, decayFactor: Option[Double]): Option[Metric] = + if (defined(value)) Some(new Metric(name, value, ceateEWMA(value.doubleValue, decayFactor))) + else None + + /** + * Creates a new Metric instance if the Try is successful and the value is valid, + * otherwise None is returned. Invalid numeric values are negative Int/Long and + * NaN/Infinite Double values. + */ + def create(name: String, value: Try[Number], decayFactor: Option[Double]): Option[Metric] = value match { + case Success(v) ⇒ create(name, v, decayFactor) + case Failure(_) ⇒ None + } + + private def ceateEWMA(value: Double, decayFactor: Option[Double]): Option[EWMA] = decayFactor match { + case Some(alpha) ⇒ Some(EWMA(value, alpha)) + case None ⇒ None } - def apply(name: String): Metric = new Metric(name, None, None, true) } /** - * INTERNAL API - * * The snapshot of current sampled health metrics for any monitored process. * Collected and gossipped at regular intervals for dynamic cluster management strategies. * - * For the JVM memory. The amount of used and committed memory will always be <= max if max is defined. - * A memory allocation may fail if it attempts to increase the used memory such that used > committed - * even if used <= max is true (e.g. when the system virtual memory is low). - * - * The system is possibly nearing a bottleneck if the system load average is nearing in cpus/cores. - * * @param address [[akka.actor.Address]] of the node the metrics are gathered at - * - * @param timestamp the time of sampling - * - * @param metrics the array of sampled [[akka.actor.Metric]] + * @param timestamp the time of sampling, in milliseconds since midnight, January 1, 1970 UTC + * @param metrics the set of sampled [[akka.actor.Metric]] */ -private[cluster] case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Metric] = Set.empty[Metric]) extends ClusterMessage { - import NodeMetrics._ - import NodeMetrics.MetricValues._ +case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Metric] = Set.empty[Metric]) extends ClusterMessage { /** * Returns the most recent data. @@ -381,101 +373,147 @@ private[cluster] case class NodeMetrics(address: Address, timestamp: Long, metri /** * Returns true if that address is the same as this and its metric set is more recent. */ - def updatable(that: NodeMetrics): Boolean = (this same that) && (that.timestamp > timestamp) + def updatable(that: NodeMetrics): Boolean = (this sameAs that) && (that.timestamp > timestamp) /** * Returns true if that address is the same as this */ - def same(that: NodeMetrics): Boolean = address == that.address + def sameAs(that: NodeMetrics): Boolean = address == that.address + + def metric(key: String): Option[Metric] = metrics.collectFirst { case m if m.name == key ⇒ m } /** - * Of all the data streams, this fluctuates the most. + * Java API */ - def heapMemory: HeapMemory = HeapMemory(metric(HeapMemoryUsed), metric(HeapMemoryCommitted), metric(HeapMemoryMax)) - - def networkLatency: NetworkLatency = NetworkLatency(metric(NetworkInboundRate), metric(NetworkOutboundRate)) - - def cpu: Cpu = Cpu(metric(SystemLoadAverage), metric(Processors), metric(CpuCombined), metric(TotalCores)) - - def metric(key: String): Metric = metrics.collectFirst { case m if m.name == key ⇒ m } getOrElse Metric(key) + def getMetrics: java.lang.Iterable[Metric] = { + import scala.collection.JavaConverters._ + metrics.asJava + } } /** - * INTERNAL API + * Definitions of the built-in standard metrics. * - * Companion object of Metric class - used by metrics consumers such as the load balancing routers. - * - * The following extractors and orderings hide the implementation from cluster metric consumers - * such as load balancers. + * The following extractors and data structures makes it easy to consume the + * [[akka.cluster.NodeMetrics]] in for example load balancers. */ -private[cluster] object NodeMetrics { +object StandardMetrics { - sealed trait MetricValues + object HeapMemory { + /** + * Constants for the heap related Metric names + */ + object Fields { + final val HeapMemoryUsed = "heap-memory-used" + final val HeapMemoryCommitted = "heap-memory-committed" + final val HeapMemoryMax = "heap-memory-max" + } + import Fields._ - object MetricValues { + /** + * Given a NodeMetrics it returns the HeapMemory data if the nodeMetrics contains + * necessary heap metrics. + * @return if possible a tuple matching the HeapMemory constructor parameters + */ + def unapply(nodeMetrics: NodeMetrics): Option[(Address, Long, Long, Long, Option[Long])] = { + for { + used ← nodeMetrics.metric(HeapMemoryUsed) + committed ← nodeMetrics.metric(HeapMemoryCommitted) + maxOption = nodeMetrics.metric(HeapMemoryMax).map(_.smoothValue.longValue) + } yield (nodeMetrics.address, nodeMetrics.timestamp, + used.smoothValue.longValue, committed.smoothValue.longValue, maxOption) + } - final val HeapMemoryUsed = "heap-memory-used" - final val HeapMemoryCommitted = "heap-memory-committed" - final val HeapMemoryMax = "heap-memory-max" - final val NetworkInboundRate = "network-max-inbound" - final val NetworkOutboundRate = "network-max-outbound" - final val SystemLoadAverage = "system-load-average" - final val Processors = "processors" - final val CpuCombined = "cpu-combined" - final val TotalCores = "total-cores" - - def unapply(v: HeapMemory): Tuple3[Long, Long, Option[Long]] = - (v.used.averageValue.get.longValue, - v.committed.averageValue.get.longValue, - v.max.averageValue map (_.longValue) orElse None) - - def unapply(v: NetworkLatency): Option[(Long, Long)] = - (v.inbound.averageValue, v.outbound.averageValue) match { - case (Some(a), Some(b)) ⇒ Some((a.longValue, b.longValue)) - case _ ⇒ None - } - - def unapply(v: Cpu): Tuple4[Option[Double], Int, Option[Double], Option[Int]] = - (v.systemLoadAverage.averageValue map (_.doubleValue), - v.processors.averageValue.get.intValue, - v.combinedCpu.averageValue map (_.doubleValue), - v.cores.averageValue map (_.intValue)) } /** + * Java API to extract HeapMemory data from nodeMetrics, if the nodeMetrics + * contains necessary heap metrics, otherwise it returns null. + */ + def extractHeapMemory(nodeMetrics: NodeMetrics): HeapMemory = nodeMetrics match { + case HeapMemory(address, timestamp, used, committed, max) ⇒ + HeapMemory(address, timestamp, used, committed, max) + case _ ⇒ null + } + + /** + * The amount of used and committed memory will always be <= max if max is defined. + * A memory allocation may fail if it attempts to increase the used memory such that used > committed + * even if used <= max is true (e.g. when the system virtual memory is low). + * + * @param address [[akka.actor.Address]] of the node the metrics are gathered at + * @param timestamp the time of sampling, in milliseconds since midnight, January 1, 1970 UTC * @param used the current sum of heap memory used from all heap memory pools (in bytes) - * * @param committed the current sum of heap memory guaranteed to be available to the JVM - * from all heap memory pools (in bytes). Committed will always be greater than or equal to used. - * + * from all heap memory pools (in bytes). Committed will always be greater than or equal to used. * @param max the maximum amount of memory (in bytes) that can be used for JVM memory management. - * Can be undefined on some OS. + * Can be undefined on some OS. */ - case class HeapMemory(used: Metric, committed: Metric, max: Metric) extends MetricValues { - require(used.isDefined, "used must be defined") - require(committed.isDefined, "committed must be defined") + case class HeapMemory(address: Address, timestamp: Long, used: Long, committed: Long, max: Option[Long]) { + require(committed > 0L, "committed heap expected to be > 0 bytes") + require(max.isEmpty || max.get > 0L, "max heap expected to be > 0 bytes") + } + + object Cpu { + /** + * Constants for the cpu related Metric names + */ + object Fields { + final val SystemLoadAverage = "system-load-average" + final val Processors = "processors" + final val CpuCombined = "cpu-combined" + } + import Fields._ + + /** + * Given a NodeMetrics it returns the Cpu data if the nodeMetrics contains + * necessary cpu metrics. + * @return if possible a tuple matching the Cpu constructor parameters + */ + def unapply(nodeMetrics: NodeMetrics): Option[(Address, Long, Option[Double], Option[Double], Int)] = { + for { + processors ← nodeMetrics.metric(Processors) + systemLoadAverageOption = nodeMetrics.metric(SystemLoadAverage).map(_.smoothValue) + cpuCombinedOption = nodeMetrics.metric(CpuCombined).map(_.smoothValue) + } yield (nodeMetrics.address, nodeMetrics.timestamp, + systemLoadAverageOption, cpuCombinedOption, processors.value.intValue) + } + } /** - * @param inbound the inbound network IO rate, in bytes - * - * @param outbound the outbound network IO rate, in bytes + * Java API to extract Cpu data from nodeMetrics, if the nodeMetrics + * contains necessary cpu metrics, otherwise it returns null. */ - case class NetworkLatency(inbound: Metric, outbound: Metric) extends MetricValues + def extractCpu(nodeMetrics: NodeMetrics): Cpu = nodeMetrics match { + case Cpu(address, timestamp, systemLoadAverage, cpuCombined, processors) ⇒ + Cpu(address, timestamp, systemLoadAverage, cpuCombined, processors) + case _ ⇒ null + } /** - * @param systemLoadAverage OS-specific average load on the CPUs in the system, for the past 1 minute - * + * @param address [[akka.actor.Address]] of the node the metrics are gathered at + * @param timestamp the time of sampling, in milliseconds since midnight, January 1, 1970 UTC + * @param systemLoadAverage OS-specific average load on the CPUs in the system, for the past 1 minute, + * The system is possibly nearing a bottleneck if the system load average is nearing number of cpus/cores. + * @param cpuCombined combined CPU sum of User + Sys + Nice + Wait, in percentage ([0.0 - 1.0]. This + * metric can describe the amount of time the CPU spent executing code during n-interval and how + * much more it could theoretically. * @param processors the number of available processors - * - * @param combinedCpu combined CPU sum of User + Sys + Nice + Wait, in percentage. This metric can describe - * the amount of time the CPU spent executing code during n-interval and how much more it could theoretically. - * - * @param cores the number of cores (multi-core: per processor) */ - private[cluster] case class Cpu(systemLoadAverage: Metric, processors: Metric, combinedCpu: Metric, cores: Metric) extends MetricValues { - require(processors.isDefined, "processors must be defined") + case class Cpu( + address: Address, + timestamp: Long, + systemLoadAverage: Option[Double], + cpuCombined: Option[Double], + processors: Int) { + + cpuCombined match { + case Some(x) ⇒ require(0.0 <= x && x <= 1.0, "cpuCombined must be between [0.0 - 1.0], was [%s]" format x) + case None ⇒ + } + } } @@ -527,22 +565,22 @@ private[cluster] trait MetricsCollector extends Closeable { } /** - * INTERNAL API - * - * Loads JVM metrics through JMX monitoring beans. + * Loads JVM and system metrics through JMX monitoring beans. * * @param address The [[akka.actor.Address]] of the node being sampled * @param decay how quickly the exponential weighting of past data is decayed */ -private[cluster] class JmxMetricsCollector(address: Address, decay: Int) extends MetricsCollector { - import NodeMetrics.MetricValues._ +class JmxMetricsCollector(address: Address, decayFactor: Double) extends MetricsCollector { + import StandardMetrics.HeapMemory.Fields._ + import StandardMetrics.Cpu.Fields._ private def this(cluster: Cluster) = - this(cluster.selfAddress, cluster.settings.MetricsRateOfDecay) + this(cluster.selfAddress, + EWMA.alpha(cluster.settings.MetricsDecayHalfLifeDuration, cluster.settings.MetricsInterval)) def this(system: ActorSystem) = this(Cluster(system)) - private val decayOption = Some(decay) + private val decayFactorOption = Some(decayFactor) private val memoryMBean: MemoryMXBean = ManagementFactory.getMemoryMXBean @@ -551,63 +589,68 @@ private[cluster] class JmxMetricsCollector(address: Address, decay: Int) extends /** * Samples and collects new data points. */ - def sample: NodeMetrics = NodeMetrics(address, newTimestamp, Set( - systemLoadAverage, heapUsed, heapCommitted, heapMax, processors)) + def sample: NodeMetrics = NodeMetrics(address, newTimestamp, metrics) + + def metrics: Set[Metric] = { + val heap = heapMemoryUsage + Set(systemLoadAverage, heapUsed(heap), heapCommitted(heap), heapMax(heap), processors).flatten + } /** * JMX Returns the OS-specific average load on the CPUs in the system, for the past 1 minute. - * On some systems the JMX OS system load average may not be available, in which case a -1 is returned, - * which means that the returned Metric is undefined. + * On some systems the JMX OS system load average may not be available, in which case a -1 is + * returned from JMX, and None is returned from this method. */ - def systemLoadAverage: Metric = Metric( + def systemLoadAverage: Option[Metric] = Metric.create( name = SystemLoadAverage, - value = Some(osMBean.getSystemLoadAverage), - decay = None) + value = osMBean.getSystemLoadAverage, + decayFactor = None) /** * (JMX) Returns the number of available processors */ - def processors: Metric = Metric( + def processors: Option[Metric] = Metric.create( name = Processors, - value = Some(osMBean.getAvailableProcessors), - decay = None) + value = osMBean.getAvailableProcessors, + decayFactor = None) - // FIXME those three heap calls should be done at once + /** + * Current heap to be passed in to heapUsed, heapCommitted and heapMax + */ + def heapMemoryUsage: MemoryUsage = memoryMBean.getHeapMemoryUsage /** * (JMX) Returns the current sum of heap memory used from all heap memory pools (in bytes). */ - def heapUsed: Metric = Metric( + def heapUsed(heap: MemoryUsage): Option[Metric] = Metric.create( name = HeapMemoryUsed, - value = Some(memoryMBean.getHeapMemoryUsage.getUsed), - decay = decayOption) + value = heap.getUsed, + decayFactor = decayFactorOption) /** * (JMX) Returns the current sum of heap memory guaranteed to be available to the JVM * from all heap memory pools (in bytes). */ - def heapCommitted: Metric = Metric( + def heapCommitted(heap: MemoryUsage): Option[Metric] = Metric.create( name = HeapMemoryCommitted, - value = Some(memoryMBean.getHeapMemoryUsage.getCommitted), - decay = decayOption) + value = heap.getCommitted, + decayFactor = decayFactorOption) /** * (JMX) Returns the maximum amount of memory (in bytes) that can be used * for JVM memory management. If not defined the metrics value is None, i.e. * never negative. */ - def heapMax: Metric = Metric( + def heapMax(heap: MemoryUsage): Option[Metric] = Metric.create( name = HeapMemoryMax, - value = Some(memoryMBean.getHeapMemoryUsage.getMax), - decay = None) + value = heap.getMax, + decayFactor = None) - def close(): Unit = () + override def close(): Unit = () } /** - * INTERNAL API - * * Loads metrics through Hyperic SIGAR and JMX monitoring beans. This * loads wider and more accurate range of metrics compared to JmxMetricsCollector * by using SIGAR's native OS library. @@ -621,32 +664,25 @@ private[cluster] class JmxMetricsCollector(address: Address, decay: Int) extends * @param decay how quickly the exponential weighting of past data is decayed * @param sigar the org.hyperic.Sigar instance */ -private[cluster] class SigarMetricsCollector(address: Address, decay: Int, sigar: AnyRef) - extends JmxMetricsCollector(address, decay) { - import NodeMetrics.MetricValues._ +class SigarMetricsCollector(address: Address, decayFactor: Double, sigar: AnyRef) + extends JmxMetricsCollector(address, decayFactor) { - def this(address: Address, decay: Int, dynamicAccess: DynamicAccess) = - this(address, decay, dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Seq.empty).get) + import StandardMetrics.HeapMemory.Fields._ + import StandardMetrics.Cpu.Fields._ private def this(cluster: Cluster) = - this(cluster.selfAddress, cluster.settings.MetricsRateOfDecay, cluster.system.dynamicAccess) + this(cluster.selfAddress, + EWMA.alpha(cluster.settings.MetricsDecayHalfLifeDuration, cluster.settings.MetricsInterval), + cluster.system.dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Seq.empty).get) def this(system: ActorSystem) = this(Cluster(system)) - private val decayOption = Some(decay) + private val decayFactorOption = Some(decayFactor) private val LoadAverage: Option[Method] = createMethodFrom(sigar, "getLoadAverage") - - private val CpuList: Option[Method] = createMethodFrom(sigar, "getCpuInfoList").map(m ⇒ m) - - private val NetInterfaces: Option[Method] = createMethodFrom(sigar, "getNetInterfaceList") - private val Cpu: Option[Method] = createMethodFrom(sigar, "getCpuPerc") - private val CombinedCpu: Option[Method] = Try(Cpu.get.getReturnType.getMethod("getCombined")).toOption - private val Pid: Option[Method] = createMethodFrom(sigar, "getPid") - // Do something initially, in constructor, to make sure that the native library can be loaded. // This will by design throw exception if sigar isn't usable val pid: Long = createMethodFrom(sigar, "getPid") match { @@ -661,25 +697,20 @@ private[cluster] class SigarMetricsCollector(address: Address, decay: Int, sigar case None ⇒ throw new IllegalArgumentException("Wrong version of Sigar, expected 'getPid' method") } - /** - * Samples and collects new data points. - */ - override def sample: NodeMetrics = NodeMetrics(address, newTimestamp, Set(cpuCombined, totalCores, - systemLoadAverage, heapUsed, heapCommitted, heapMax, processors, networkMaxRx, networkMaxTx)) + override def metrics: Set[Metric] = { + super.metrics.filterNot(_.name == SystemLoadAverage) ++ Set(systemLoadAverage, cpuCombined).flatten + } /** * (SIGAR / JMX) Returns the OS-specific average load on the CPUs in the system, for the past 1 minute. - * On some systems the JMX OS system load average may not be available, in which case a -1 is returned, - * which means that the returned Metric is undefined. + * On some systems the JMX OS system load average may not be available, in which case a -1 is returned + * from JMX, which means that None is returned from this method. * Hyperic SIGAR provides more precise values, thus, if the library is on the classpath, it is the default. */ - override def systemLoadAverage: Metric = { - val m = Metric( - name = SystemLoadAverage, - value = Try(LoadAverage.get.invoke(sigar).asInstanceOf[Array[AnyRef]].head.asInstanceOf[Number]).toOption, - decay = None) - if (m.isDefined) m else super.systemLoadAverage - } + override def systemLoadAverage: Option[Metric] = Metric.create( + name = SystemLoadAverage, + value = Try(LoadAverage.get.invoke(sigar).asInstanceOf[Array[AnyRef]](0).asInstanceOf[Number]), + decayFactor = None) orElse super.systemLoadAverage /** * (SIGAR) Returns the combined CPU sum of User + Sys + Nice + Wait, in percentage. This metric can describe @@ -689,59 +720,16 @@ private[cluster] class SigarMetricsCollector(address: Address, decay: Int, sigar * In the data stream, this will sometimes return with a valid metric value, and sometimes as a NaN or Infinite. * Documented bug https://bugzilla.redhat.com/show_bug.cgi?id=749121 and several others. */ - def cpuCombined: Metric = Metric( + def cpuCombined: Option[Metric] = Metric.create( name = CpuCombined, - value = Try(CombinedCpu.get.invoke(Cpu.get.invoke(sigar)).asInstanceOf[Number]).toOption, - decay = decayOption) - - /** - * FIXME: Array[Int].head - expose all if cores per processor might differ. - * (SIGAR) Returns the total number of cores. - * - * FIXME do we need this information, isn't it enough with jmx processors? - */ - def totalCores: Metric = Metric( - name = TotalCores, - value = Try(CpuList.get.invoke(sigar).asInstanceOf[Array[AnyRef]].map(cpu ⇒ - createMethodFrom(cpu, "getTotalCores").get.invoke(cpu).asInstanceOf[Number]).head).toOption, - decay = None) - - // FIXME those two network calls should be combined into one - - /** - * (SIGAR) Returns the max network IO read/write value, in bytes, for network latency evaluation. - */ - def networkMaxRx: Metric = networkFor("getRxBytes", NetworkInboundRate) - - /** - * (SIGAR) Returns the max network IO outbound value, in bytes. - */ - def networkMaxTx: Metric = networkFor("getTxBytes", NetworkOutboundRate) + value = Try(CombinedCpu.get.invoke(Cpu.get.invoke(sigar)).asInstanceOf[Number]), + decayFactor = decayFactorOption) /** * Releases any native resources associated with this instance. */ override def close(): Unit = Try(createMethodFrom(sigar, "close").get.invoke(sigar)) - // FIXME network metrics needs thought and refactoring - - /** - * Returns the max bytes for the given method in metric for metric from the network interface stats. - */ - private def networkFor(method: String, metric: String): Metric = Metric( - name = metric, - value = Try(networkStats.collect { - case (_, a) ⇒ - createMethodFrom(a, method).get.invoke(a).asInstanceOf[Long] - }.max.asInstanceOf[Number]).toOption.orElse(None), - decay = decayOption) - - /** - * Returns the network stats per interface. - */ - private def networkStats: Map[String, AnyRef] = Try(NetInterfaces.get.invoke(sigar).asInstanceOf[Array[String]].map(arg ⇒ - arg -> (createMethodFrom(sigar, "getNetInterfaceStat", Array(classOf[String])).get.invoke(sigar, arg))).toMap) getOrElse Map.empty[String, AnyRef] - private def createMethodFrom(ref: AnyRef, method: String, types: Array[(Class[_])] = Array.empty[(Class[_])]): Option[Method] = Try(ref.getClass.getMethod(method, types: _*)).toOption @@ -749,7 +737,9 @@ private[cluster] class SigarMetricsCollector(address: Address, decay: Int, sigar /** * INTERNAL API - * Companion object of MetricsCollector class. + * Factory to create configured MetricsCollector. + * If instantiation of SigarMetricsCollector fails (missing class or native library) + * it falls back to use JmxMetricsCollector. */ private[cluster] object MetricsCollector { def apply(system: ExtendedActorSystem, settings: ClusterSettings): MetricsCollector = { diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala index 1b2d11745e..cc87c1e112 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala @@ -77,9 +77,9 @@ class ClusterSettings(val config: Config, val systemName: String) { require(d > Duration.Zero, "metrics.collect-interval must be > 0"); d } final val MetricsGossipInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.metrics.gossip-interval"), MILLISECONDS) - final val MetricsRateOfDecay: Int = { - val n = getInt("akka.cluster.metrics.rate-of-decay") - require(n >= 1, "metrics.rate-of-decay must be >= 1"); n + final val MetricsDecayHalfLifeDuration: FiniteDuration = { + val d = Duration(getMilliseconds("akka.cluster.metrics.decay-half-life-duration"), MILLISECONDS) + require(d > Duration.Zero, "metrics.decay-half-life-duration must be > 0"); d } } diff --git a/akka-cluster/src/main/scala/akka/cluster/routing/AdaptiveLoadBalancingRouter.scala b/akka-cluster/src/main/scala/akka/cluster/routing/AdaptiveLoadBalancingRouter.scala new file mode 100644 index 0000000000..7208976926 --- /dev/null +++ b/akka-cluster/src/main/scala/akka/cluster/routing/AdaptiveLoadBalancingRouter.scala @@ -0,0 +1,414 @@ +/* + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster.routing + +import java.util.Arrays +import scala.concurrent.forkjoin.ThreadLocalRandom +import scala.collection.JavaConverters.iterableAsScalaIterableConverter +import akka.actor.Actor +import akka.actor.ActorRef +import akka.actor.Address +import akka.actor.OneForOneStrategy +import akka.actor.Props +import akka.actor.SupervisorStrategy +import akka.dispatch.Dispatchers +import akka.cluster.Cluster +import akka.cluster.ClusterEvent.ClusterMetricsChanged +import akka.cluster.ClusterEvent.CurrentClusterState +import akka.cluster.NodeMetrics +import akka.cluster.StandardMetrics.Cpu +import akka.cluster.StandardMetrics.HeapMemory +import akka.event.Logging +import akka.routing.Broadcast +import akka.routing.Destination +import akka.routing.FromConfig +import akka.routing.NoRouter +import akka.routing.Resizer +import akka.routing.Route +import akka.routing.RouteeProvider +import akka.routing.RouterConfig + +object AdaptiveLoadBalancingRouter { + private val defaultSupervisorStrategy: SupervisorStrategy = OneForOneStrategy() { + case _ ⇒ SupervisorStrategy.Escalate + } +} + +/** + * A Router that performs load balancing to cluster nodes based on + * cluster metric data. + * + * It uses random selection of routees based probabilities derived from + * the remaining capacity of corresponding node. + * + * Please note that providing both 'nrOfInstances' and 'routees' does not make logical + * sense as this means that the router should both create new actors and use the 'routees' + * actor(s). In this case the 'nrOfInstances' will be ignored and the 'routees' will be used. + *
+ * The configuration parameter trumps the constructor arguments. This means that + * if you provide either 'nrOfInstances' or 'routees' during instantiation they will + * be ignored if the router is defined in the configuration file for the actor being used. + * + *

Supervision Setup

+ * + * The router creates a “head” actor which supervises and/or monitors the + * routees. Instances are created as children of this actor, hence the + * children are not supervised by the parent of the router. Common choices are + * to always escalate (meaning that fault handling is always applied to all + * children simultaneously; this is the default) or use the parent’s strategy, + * which will result in routed children being treated individually, but it is + * possible as well to use Routers to give different supervisor strategies to + * different groups of children. + * + * @param metricsSelector decides what probability to use for selecting a routee, based + * on remaining capacity as indicated by the node metrics + * @param routees string representation of the actor paths of the routees that will be looked up + * using `actorFor` in [[akka.actor.ActorRefProvider]] + */ +@SerialVersionUID(1L) +case class AdaptiveLoadBalancingRouter( + metricsSelector: MetricsSelector = MixMetricsSelector(), + nrOfInstances: Int = 0, routees: Iterable[String] = Nil, + override val resizer: Option[Resizer] = None, + val routerDispatcher: String = Dispatchers.DefaultDispatcherId, + val supervisorStrategy: SupervisorStrategy = AdaptiveLoadBalancingRouter.defaultSupervisorStrategy) + extends RouterConfig with AdaptiveLoadBalancingRouterLike { + + /** + * Constructor that sets nrOfInstances to be created. + * Java API + * @param selector the selector is responsible for producing weighted mix of routees from the node metrics + * @param nr number of routees to create + */ + def this(selector: MetricsSelector, nr: Int) = this(metricsSelector = selector, nrOfInstances = nr) + + /** + * Constructor that sets the routees to be used. + * Java API + * @param selector the selector is responsible for producing weighted mix of routees from the node metrics + * @param routeePaths string representation of the actor paths of the routees that will be looked up + * using `actorFor` in [[akka.actor.ActorRefProvider]] + */ + def this(selector: MetricsSelector, routeePaths: java.lang.Iterable[String]) = + this(metricsSelector = selector, routees = routeePaths.asScala) + + /** + * Constructor that sets the resizer to be used. + * Java API + * @param selector the selector is responsible for producing weighted mix of routees from the node metrics + */ + def this(selector: MetricsSelector, resizer: Resizer) = + this(metricsSelector = selector, resizer = Some(resizer)) + + /** + * Java API for setting routerDispatcher + */ + def withDispatcher(dispatcherId: String): AdaptiveLoadBalancingRouter = + copy(routerDispatcher = dispatcherId) + + /** + * Java API for setting the supervisor strategy to be used for the “head” + * Router actor. + */ + def withSupervisorStrategy(strategy: SupervisorStrategy): AdaptiveLoadBalancingRouter = + copy(supervisorStrategy = strategy) + + /** + * Uses the resizer of the given RouterConfig if this RouterConfig + * doesn't have one, i.e. the resizer defined in code is used if + * resizer was not defined in config. + */ + override def withFallback(other: RouterConfig): RouterConfig = other match { + case _: FromConfig | _: NoRouter ⇒ this + case otherRouter: AdaptiveLoadBalancingRouter ⇒ + val useResizer = + if (this.resizer.isEmpty && otherRouter.resizer.isDefined) otherRouter.resizer + else this.resizer + copy(resizer = useResizer) + case _ ⇒ throw new IllegalArgumentException("Expected AdaptiveLoadBalancingRouter, got [%s]".format(other)) + } + +} + +/** + * INTERNAL API. + * + * This strategy is a metrics-aware router which performs load balancing of + * cluster nodes based on cluster metric data. It consumes [[akka.cluster.ClusterMetricsChanged]] + * events and the [[akka.cluster.routing.MetricsSelector]] creates an mix of + * weighted routees based on the node metrics. Messages are routed randomly to the + * weighted routees, i.e. nodes with lower load are more likely to be used than nodes with + * higher load + */ +trait AdaptiveLoadBalancingRouterLike { this: RouterConfig ⇒ + + def metricsSelector: MetricsSelector + + def nrOfInstances: Int + + def routees: Iterable[String] + + 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) + + // The current weighted routees, if any. Weights are produced by the metricsSelector + // via the metricsListener Actor. It's only updated by the actor, but accessed from + // the threads of the senders. + @volatile var weightedRoutees: Option[WeightedRoutees] = None + + // subscribe to ClusterMetricsChanged and update weightedRoutees + val metricsListener = routeeProvider.context.actorOf(Props(new Actor { + + val cluster = Cluster(context.system) + + override def preStart(): Unit = cluster.subscribe(self, classOf[ClusterMetricsChanged]) + override def postStop(): Unit = cluster.unsubscribe(self) + + def receive = { + case ClusterMetricsChanged(metrics) ⇒ receiveMetrics(metrics) + case _: CurrentClusterState ⇒ // ignore + } + + def receiveMetrics(metrics: Set[NodeMetrics]): Unit = { + // update the state outside of the actor, not a recommended practice, but works fine here + weightedRoutees = Some(new WeightedRoutees(routeeProvider.routees, cluster.selfAddress, + metricsSelector.weights(metrics))) + } + + }).withDispatcher(routerDispatcher), name = "metricsListener") + + def getNext(): ActorRef = weightedRoutees match { + case Some(weighted) ⇒ + if (weighted.total == 0) routeeProvider.context.system.deadLetters + else weighted(ThreadLocalRandom.current.nextInt(weighted.total) + 1) + case None ⇒ + val currentRoutees = routeeProvider.routees + if (currentRoutees.isEmpty) routeeProvider.context.system.deadLetters + else currentRoutees(ThreadLocalRandom.current.nextInt(currentRoutees.size)) + } + + { + case (sender, message) ⇒ + message match { + case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees) + case msg ⇒ List(Destination(sender, getNext())) + } + } + } +} + +/** + * MetricsSelector that uses the heap metrics. + * Low heap capacity => small weight. + */ +@SerialVersionUID(1L) +case object HeapMetricsSelector extends CapacityMetricsSelector { + /** + * Java API: get the singleton instance + */ + def getInstance = this + + override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = { + nodeMetrics.collect { + case HeapMemory(address, _, used, committed, max) ⇒ + val capacity = max match { + case None ⇒ (committed - used).toDouble / committed + case Some(m) ⇒ (m - used).toDouble / m + } + (address, capacity) + }.toMap + } +} + +/** + * MetricsSelector that uses the combined CPU metrics. + * Combined CPU is sum of User + Sys + Nice + Wait, in percentage. + * Low cpu capacity => small weight. + */ +@SerialVersionUID(1L) +case object CpuMetricsSelector extends CapacityMetricsSelector { + /** + * Java API: get the singleton instance + */ + def getInstance = this + + override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = { + nodeMetrics.collect { + case Cpu(address, _, _, Some(cpuCombined), _) ⇒ + val capacity = 1.0 - cpuCombined + (address, capacity) + }.toMap + } +} + +/** + * MetricsSelector that uses the system load average metrics. + * System load average is OS-specific average load on the CPUs in the system, + * for the past 1 minute. The system is possibly nearing a bottleneck if the + * system load average is nearing number of cpus/cores. + * Low load average capacity => small weight. + */ +@SerialVersionUID(1L) +case object SystemLoadAverageMetricsSelector extends CapacityMetricsSelector { + /** + * Java API: get the singleton instance + */ + def getInstance = this + + override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = { + nodeMetrics.collect { + case Cpu(address, _, Some(systemLoadAverage), _, processors) ⇒ + val capacity = 1.0 - math.min(1.0, systemLoadAverage / processors) + (address, capacity) + }.toMap + } +} + +/** + * MetricsSelector that combines other selectors and aggregates their capacity + * values. By default it uses [akka.cluster.routing.HeapMetricsSelector], + * [akka.cluster.routing.CpuMetricsSelector], and [akka.cluster.routing.SystemLoadAverageMetricsSelector] + */ +@SerialVersionUID(1L) +case class MixMetricsSelector( + selectors: IndexedSeq[CapacityMetricsSelector] = Vector( + HeapMetricsSelector, CpuMetricsSelector, SystemLoadAverageMetricsSelector)) + extends CapacityMetricsSelector { + + /** + * Java API + */ + def this(selectors: java.lang.Iterable[CapacityMetricsSelector]) = this(selectors.asScala.toIndexedSeq) + + override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = { + val combined: IndexedSeq[(Address, Double)] = selectors.flatMap(_.capacity(nodeMetrics).toSeq) + // aggregated average of the capacities by address + combined.foldLeft(Map.empty[Address, (Double, Int)].withDefaultValue((0.0, 0))) { + case (acc, (address, capacity)) ⇒ + val (sum, count) = acc(address) + acc + (address -> (sum + capacity, count + 1)) + }.mapValues { + case (sum, count) ⇒ sum / count + }.toMap + } + +} + +case object MixMetricsSelector { + /** + * Java API: get the default singleton instance + */ + def getInstance = MixMetricsSelector() +} + +/** + * A MetricsSelector is responsible for producing weights from the node metrics. + */ +@SerialVersionUID(1L) +trait MetricsSelector extends Serializable { + /** + * The weights per address, based on the the nodeMetrics. + */ + def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] +} + +/** + * A MetricsSelector producing weights from remaining capacity. + * The weights are typically proportional to the remaining capacity. + */ +abstract class CapacityMetricsSelector extends MetricsSelector { + + /** + * Remaining capacity for each node. The value is between + * 0.0 and 1.0, where 0.0 means no remaining capacity (full + * utilization) and 1.0 means full remaining capacity (zero + * utilization). + */ + def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] + + /** + * Converts the capacity values to weights. The node with lowest + * capacity gets weight 1 (lowest usable capacity is 1%) and other + * nodes gets weights proportional to their capacity compared to + * the node with lowest capacity. + */ + def weights(capacity: Map[Address, Double]): Map[Address, Int] = { + if (capacity.isEmpty) Map.empty[Address, Int] + else { + val (_, min) = capacity.minBy { case (_, c) ⇒ c } + // lowest usable capacity is 1% (>= 0.5% will be rounded to weight 1), also avoids div by zero + val divisor = math.max(0.01, min) + capacity mapValues { c ⇒ math.round((c) / divisor).toInt } + } + } + + /** + * The weights per address, based on the capacity produced by + * the nodeMetrics. + */ + override def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] = + weights(capacity(nodeMetrics)) + +} + +/** + * INTERNAL API + * + * Pick routee based on its weight. Higher weight, higher probability. + */ +private[cluster] class WeightedRoutees(refs: IndexedSeq[ActorRef], selfAddress: Address, weights: Map[Address, Int]) { + + // fill an array of same size as the refs with accumulated weights, + // binarySearch is used to pick the right bucket from a requested value + // from 1 to the total sum of the used weights. + private val buckets: Array[Int] = { + def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match { + case Address(_, _, None, None) ⇒ selfAddress + case a ⇒ a + } + val buckets = Array.ofDim[Int](refs.size) + val meanWeight = if (weights.isEmpty) 1 else weights.values.sum / weights.size + val w = weights.withDefaultValue(meanWeight) + var sum = 0 + refs.zipWithIndex foreach { + case (ref, i) ⇒ + sum += w(fullAddress(ref)) + buckets(i) = sum + } + buckets + } + + def total: Int = + if (buckets.length == 0) 0 + else buckets(buckets.length - 1) + + /** + * Pick the routee matching a value, from 1 to total. + */ + def apply(value: Int): ActorRef = { + // converts the result of Arrays.binarySearch into a index in the buckets array + // see documentation of Arrays.binarySearch for what it returns + def idx(i: Int): Int = { + if (i >= 0) i // exact match + else { + val j = math.abs(i + 1) + if (j >= buckets.length) throw new IndexOutOfBoundsException( + "Requested index [%s] is > max index [%s]".format(i, buckets.length)) + else j + } + } + + require(1 <= value && value <= total, "value must be between [1 - %s]" format total) + refs(idx(Arrays.binarySearch(buckets, value))) + + } +} \ No newline at end of file diff --git a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterLoadBalancingRouter.scala b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterLoadBalancingRouter.scala deleted file mode 100644 index 52b63ef5f6..0000000000 --- a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterLoadBalancingRouter.scala +++ /dev/null @@ -1,271 +0,0 @@ -/* - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.cluster.routing - -import scala.concurrent.forkjoin.ThreadLocalRandom -import scala.collection.JavaConverters.iterableAsScalaIterableConverter - -import akka.actor.Actor -import akka.actor.ActorRef -import akka.actor.Address -import akka.actor.OneForOneStrategy -import akka.actor.Props -import akka.actor.SupervisorStrategy -import akka.cluster.Cluster -import akka.cluster.ClusterEvent.ClusterMetricsChanged -import akka.cluster.ClusterEvent.CurrentClusterState -import akka.cluster.NodeMetrics -import akka.cluster.NodeMetrics.MetricValues -import akka.dispatch.Dispatchers -import akka.event.Logging -import akka.routing.Broadcast -import akka.routing.Destination -import akka.routing.Resizer -import akka.routing.Route -import akka.routing.RouteeProvider -import akka.routing.RouterConfig - -/** - * INTERNAL API - */ -private[cluster] object ClusterLoadBalancingRouter { - val defaultSupervisorStrategy: SupervisorStrategy = OneForOneStrategy() { - case _ ⇒ SupervisorStrategy.Escalate - } -} - -/** - * A Router that performs load balancing to cluster nodes based on - * cluster metric data. - * - * It uses random selection of routees based probabilities derived from - * the remaining capacity of corresponding node. - * - * Please note that providing both 'nrOfInstances' and 'routees' does not make logical - * sense as this means that the router should both create new actors and use the 'routees' - * actor(s). In this case the 'nrOfInstances' will be ignored and the 'routees' will be used. - *
- * The configuration parameter trumps the constructor arguments. This means that - * if you provide either 'nrOfInstances' or 'routees' during instantiation they will - * be ignored if the router is defined in the configuration file for the actor being used. - * - *

Supervision Setup

- * - * The router creates a “head” actor which supervises and/or monitors the - * routees. Instances are created as children of this actor, hence the - * children are not supervised by the parent of the router. Common choices are - * to always escalate (meaning that fault handling is always applied to all - * children simultaneously; this is the default) or use the parent’s strategy, - * which will result in routed children being treated individually, but it is - * possible as well to use Routers to give different supervisor strategies to - * different groups of children. - * - * @param metricsSelector decides what probability to use for selecting a routee, based - * on remaining capacity as indicated by the node metrics - * @param routees string representation of the actor paths of the routees that will be looked up - * using `actorFor` in [[akka.actor.ActorRefProvider]] - */ -@SerialVersionUID(1L) -case class ClusterLoadBalancingRouter( - metricsSelector: MetricsSelector, - 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 ClusterLoadBalancingRouterLike { - - /** - * Constructor that sets nrOfInstances to be created. - * Java API - * @param selector the selector is responsible for producing weighted mix of routees from the node metrics - * @param nr number of routees to create - */ - def this(selector: MetricsSelector, nr: Int) = this(metricsSelector = selector, nrOfInstances = nr) - - /** - * Constructor that sets the routees to be used. - * Java API - * @param selector the selector is responsible for producing weighted mix of routees from the node metrics - * @param routeePaths string representation of the actor paths of the routees that will be looked up - * using `actorFor` in [[akka.actor.ActorRefProvider]] - */ - def this(selector: MetricsSelector, routeePaths: java.lang.Iterable[String]) = - this(metricsSelector = selector, routees = routeePaths.asScala) - - /** - * Constructor that sets the resizer to be used. - * Java API - * @param selector the selector is responsible for producing weighted mix of routees from the node metrics - */ - def this(selector: MetricsSelector, resizer: Resizer) = - this(metricsSelector = selector, resizer = Some(resizer)) - - /** - * Java API for setting routerDispatcher - */ - def withDispatcher(dispatcherId: String): ClusterLoadBalancingRouter = - copy(routerDispatcher = dispatcherId) - - /** - * Java API for setting the supervisor strategy to be used for the “head” - * Router actor. - */ - def withSupervisorStrategy(strategy: SupervisorStrategy): ClusterLoadBalancingRouter = - copy(supervisorStrategy = strategy) - -} - -/** - * INTERNAL API. - * - * This strategy is a metrics-aware router which performs load balancing of - * cluster nodes based on cluster metric data. It consumes [[akka.cluster.ClusterMetricsChanged]] - * events and the [[akka.cluster.routing.MetricsSelector]] creates an mix of - * weighted routees based on the node metrics. Messages are routed randomly to the - * weighted routees, i.e. nodes with lower load are more likely to be used than nodes with - * higher load - */ -trait ClusterLoadBalancingRouterLike { this: RouterConfig ⇒ - - def metricsSelector: MetricsSelector - - def nrOfInstances: Int - - def routees: Iterable[String] - - 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) - - // Function that points to the routees to use, starts with the plain routees - // of the routeeProvider and then changes to the current weighted routees - // produced by the metricsSelector. The reason for using a function is that - // routeeProvider.routees can change. - @volatile var weightedRoutees: () ⇒ IndexedSeq[ActorRef] = () ⇒ routeeProvider.routees - - // subscribe to ClusterMetricsChanged and update weightedRoutees - val metricsListener = routeeProvider.context.actorOf(Props(new Actor { - - val cluster = Cluster(routeeProvider.context.system) - - override def preStart(): Unit = cluster.subscribe(self, classOf[ClusterMetricsChanged]) - override def postStop(): Unit = cluster.unsubscribe(self) - - def receive = { - case ClusterMetricsChanged(metrics) ⇒ receiveMetrics(metrics) - case _: CurrentClusterState ⇒ // ignore - } - - def receiveMetrics(metrics: Set[NodeMetrics]): Unit = { - val routees = metricsSelector.weightedRefs(routeeProvider.routees, cluster.selfAddress, metrics) - weightedRoutees = () ⇒ routees - } - - }).withDispatcher(routerDispatcher), name = "metricsListener") - - def getNext(): ActorRef = { - val currentRoutees = weightedRoutees.apply - if (currentRoutees.isEmpty) routeeProvider.context.system.deadLetters - else currentRoutees(ThreadLocalRandom.current.nextInt(currentRoutees.size)) - } - - { - case (sender, message) ⇒ - message match { - case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees) - case msg ⇒ List(Destination(sender, getNext())) - } - } - } -} - -/** - * MetricsSelector that uses the heap metrics. - * Low heap capacity => lower weight. - */ -@SerialVersionUID(1L) -case object HeapMetricsSelector extends MetricsSelector { - /** - * Java API: get the singleton instance - */ - def getInstance = this - - override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = { - nodeMetrics.map { n ⇒ - val (used, committed, max) = MetricValues.unapply(n.heapMemory) - val capacity = max match { - case None ⇒ (committed - used).toDouble / committed - case Some(m) ⇒ (m - used).toDouble / m - } - (n.address, capacity) - }.toMap - } -} - -// FIXME implement more MetricsSelectors, such as CpuMetricsSelector, -// LoadAverageMetricsSelector, NetworkMetricsSelector. -// Also a CompositeMetricsSelector which uses a mix of other -// selectors. - -/** - * A MetricsSelector is responsible for producing weighted mix of routees - * from the node metrics. The weights are typically proportional to the - * remaining capacity. - */ -abstract class MetricsSelector { - - /** - * Remaining capacity for each node. The value is between - * 0.0 and 1.0, where 0.0 means no remaining capacity (full - * utilization) and 1.0 means full remaining capacity (zero - * utilization). - */ - def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] - - /** - * Converts the capacity values to weights. The node with lowest - * capacity gets weight 1 (lowest usable capacity is 1%) and other - * nodes gets weights proportional to their capacity compared to - * the node with lowest capacity. - */ - def weights(capacity: Map[Address, Double]): Map[Address, Int] = { - if (capacity.isEmpty) Map.empty[Address, Int] - else { - val (_, min) = capacity.minBy { case (_, c) ⇒ c } - // lowest usable capacity is 1% (>= 0.5% will be rounded to weight 1), also avoids div by zero - val divisor = math.max(0.01, min) - capacity mapValues { c ⇒ math.round((c) / divisor).toInt } - } - } - - /** - * Allocates a list of actor refs according to the weight of their node, i.e. - * weight 3 of node A will allocate 3 slots for each ref with address A. - */ - def weightedRefs(refs: IndexedSeq[ActorRef], selfAddress: Address, weights: Map[Address, Int]): IndexedSeq[ActorRef] = { - def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match { - case Address(_, _, None, None) ⇒ selfAddress - case a ⇒ a - } - - val w = weights.withDefaultValue(1) - refs.foldLeft(IndexedSeq.empty[ActorRef]) { (acc, ref) ⇒ - acc ++ IndexedSeq.fill(w(fullAddress(ref)))(ref) - } - } - - /** - * Combines the different pieces to allocate a list of weighted actor refs - * based on the node metrics. - */ - def weightedRefs(refs: IndexedSeq[ActorRef], selfAddress: Address, nodeMetrics: Set[NodeMetrics]): IndexedSeq[ActorRef] = - weightedRefs(refs, selfAddress, weights(capacity(nodeMetrics))) -} \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterMetricsSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterMetricsSpec.scala index aa7c10b1ed..6712502312 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterMetricsSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterMetricsSpec.scala @@ -28,7 +28,7 @@ class ClusterMetricsMultiJvmNode3 extends ClusterMetricsSpec class ClusterMetricsMultiJvmNode4 extends ClusterMetricsSpec class ClusterMetricsMultiJvmNode5 extends ClusterMetricsSpec -abstract class ClusterMetricsSpec extends MultiNodeSpec(ClusterMetricsMultiJvmSpec) with MultiNodeClusterSpec with MetricSpec { +abstract class ClusterMetricsSpec extends MultiNodeSpec(ClusterMetricsMultiJvmSpec) with MultiNodeClusterSpec { import ClusterMetricsMultiJvmSpec._ def isSigar(collector: MetricsCollector): Boolean = collector.isInstanceOf[SigarMetricsCollector] diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterLoadBalancingRouterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/AdaptiveLoadBalancingRouterSpec.scala similarity index 63% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterLoadBalancingRouterSpec.scala rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/AdaptiveLoadBalancingRouterSpec.scala index 028693f04c..79143a6b4c 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterLoadBalancingRouterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/AdaptiveLoadBalancingRouterSpec.scala @@ -5,21 +5,25 @@ 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.duration._ import scala.concurrent.forkjoin.ThreadLocalRandom +import com.typesafe.config.Config import com.typesafe.config.ConfigFactory -object ClusterLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig { +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 { class Routee extends Actor { var usedMemory: Array[Byte] = _ @@ -51,18 +55,38 @@ object ClusterLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig { commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(""" akka.cluster.metrics.collect-interval = 1s akka.cluster.metrics.gossip-interval = 1s + 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 + } + } + } """)).withFallback(MultiNodeClusterSpec.clusterConfig)) } -class ClusterLoadBalancingRouterMultiJvmNode1 extends ClusterLoadBalancingRouterSpec -class ClusterLoadBalancingRouterMultiJvmNode2 extends ClusterLoadBalancingRouterSpec -class ClusterLoadBalancingRouterMultiJvmNode3 extends ClusterLoadBalancingRouterSpec +class TestCustomMetricsSelector(config: Config) extends MetricsSelector { + override def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] = Map.empty +} -abstract class ClusterLoadBalancingRouterSpec extends MultiNodeSpec(ClusterLoadBalancingRouterMultiJvmSpec) +class AdaptiveLoadBalancingRouterMultiJvmNode1 extends AdaptiveLoadBalancingRouterSpec +class AdaptiveLoadBalancingRouterMultiJvmNode2 extends AdaptiveLoadBalancingRouterSpec +class AdaptiveLoadBalancingRouterMultiJvmNode3 extends AdaptiveLoadBalancingRouterSpec + +abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoadBalancingRouterMultiJvmSpec) with MultiNodeClusterSpec with ImplicitSender with DefaultTimeout { - import ClusterLoadBalancingRouterMultiJvmSpec._ + import AdaptiveLoadBalancingRouterMultiJvmSpec._ def currentRoutees(router: ActorRef) = Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees @@ -86,7 +110,7 @@ abstract class ClusterLoadBalancingRouterSpec extends MultiNodeSpec(ClusterLoadB def startRouter(name: String): ActorRef = { val router = system.actorOf(Props[Routee].withRouter(ClusterRouterConfig( - local = ClusterLoadBalancingRouter(HeapMetricsSelector), + local = AdaptiveLoadBalancingRouter(HeapMetricsSelector), settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 1))), name) awaitCond { // it may take some time until router receives cluster member events @@ -96,7 +120,7 @@ abstract class ClusterLoadBalancingRouterSpec extends MultiNodeSpec(ClusterLoadB router } - "A cluster with a ClusterLoadBalancingRouter" must { + "A cluster with a AdaptiveLoadBalancingRouter" must { "start cluster nodes" taggedAs LongRunningTest in { awaitClusterUp(roles: _*) enterBarrier("after-1") @@ -112,6 +136,7 @@ abstract class ClusterLoadBalancingRouterSpec extends MultiNodeSpec(ClusterLoadB val iterationCount = 100 for (i ← 0 until iterationCount) { router1 ! "hit" + // wait a while between each message, since metrics is collected periodically Thread.sleep(10) } @@ -146,6 +171,7 @@ abstract class ClusterLoadBalancingRouterSpec extends MultiNodeSpec(ClusterLoadB val iterationCount = 100 for (i ← 0 until iterationCount) { router2 ! "hit" + // wait a while between each message, since metrics is collected periodically Thread.sleep(10) } @@ -158,5 +184,30 @@ abstract class ClusterLoadBalancingRouterSpec extends MultiNodeSpec(ClusterLoadB enterBarrier("after-3") } + + "create routees from configuration" taggedAs LongRunningTest in { + runOn(first) { + val router3 = system.actorOf(Props[Memory].withRouter(FromConfig()), "router3") + awaitCond { + // it may take some time until router receives cluster member events + currentRoutees(router3).size == 9 + } + 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") + awaitCond { + // it may take some time until router receives cluster member events + currentRoutees(router4).size == 6 + } + currentRoutees(router4).map(fullAddress).toSet must be(Set( + address(first), address(second), address(third))) + } + enterBarrier("after-5") + } } } diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala index 34513076c0..547cfb8211 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterConfigSpec.scala @@ -47,10 +47,10 @@ class ClusterConfigSpec extends AkkaSpec { callTimeout = 2 seconds, resetTimeout = 30 seconds)) MetricsEnabled must be(true) - MetricsCollectorClass must be("akka.cluster.SigarMetricsCollector") + MetricsCollectorClass must be(classOf[SigarMetricsCollector].getName) MetricsInterval must be(3 seconds) MetricsGossipInterval must be(3 seconds) - MetricsRateOfDecay must be(10) + MetricsDecayHalfLifeDuration must be(12 seconds) } } } diff --git a/akka-cluster/src/test/scala/akka/cluster/DataStreamSpec.scala b/akka-cluster/src/test/scala/akka/cluster/DataStreamSpec.scala deleted file mode 100644 index f6519f000f..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/DataStreamSpec.scala +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.cluster - -import language.postfixOps -import scala.concurrent.duration._ -import akka.testkit.{ LongRunningTest, AkkaSpec } -import scala.concurrent.forkjoin.ThreadLocalRandom -import System.currentTimeMillis - -@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class DataStreamSpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricSpec with MetricsCollectorFactory { - import system.dispatcher - - val collector = createMetricsCollector - - "DataStream" must { - - "calcualate same ewma for constant values" in { - val ds = DataStream(decay = 5, ewma = 100.0, currentTimeMillis, currentTimeMillis) :+ - 100.0 :+ 100.0 :+ 100.0 - ds.ewma must be(100.0 plusOrMinus 0.001) - } - - "calcualate correct ewma for normal decay" in { - val d0 = DataStream(decay = 10, ewma = 1000.0, currentTimeMillis, currentTimeMillis) - d0.ewma must be(1000.0 plusOrMinus 0.01) - val d1 = d0 :+ 10.0 - d1.ewma must be(820.0 plusOrMinus 0.01) - val d2 = d1 :+ 10.0 - d2.ewma must be(672.73 plusOrMinus 0.01) - val d3 = d2 :+ 10.0 - d3.ewma must be(552.23 plusOrMinus 0.01) - val d4 = d3 :+ 10.0 - d4.ewma must be(453.64 plusOrMinus 0.01) - - val dn = (1 to 100).foldLeft(d0)((d, _) ⇒ d :+ 10.0) - dn.ewma must be(10.0 plusOrMinus 0.1) - } - - "calculate ewma for decay 1" in { - val d0 = DataStream(decay = 1, ewma = 100.0, currentTimeMillis, currentTimeMillis) - d0.ewma must be(100.0 plusOrMinus 0.01) - val d1 = d0 :+ 1.0 - d1.ewma must be(1.0 plusOrMinus 0.01) - val d2 = d1 :+ 57.0 - d2.ewma must be(57.0 plusOrMinus 0.01) - val d3 = d2 :+ 10.0 - d3.ewma must be(10.0 plusOrMinus 0.01) - } - - "calculate the ewma for multiple, variable, data streams" taggedAs LongRunningTest in { - var streamingDataSet = Map.empty[String, Metric] - var usedMemory = Array.empty[Byte] - (1 to 50) foreach { _ ⇒ - Thread.sleep(100) - usedMemory = usedMemory ++ Array.fill(1024)(ThreadLocalRandom.current.nextInt(127).toByte) - val changes = collector.sample.metrics.flatMap { latest ⇒ - streamingDataSet.get(latest.name) match { - case None ⇒ Some(latest) - case Some(previous) ⇒ - if (latest.average.isDefined && latest.isDefined && latest.value.get != previous.value.get) { - val updated = previous :+ latest - updated.average.isDefined must be(true) - val updatedAverage = updated.average.get - updatedAverage.timestamp must be > (previous.average.get.timestamp) - updatedAverage.duration.length must be > (previous.average.get.duration.length) - updatedAverage.ewma must not be (previous.average.get.ewma) - (previous.value.get.longValue compare latest.value.get.longValue) must be( - previous.average.get.ewma.longValue compare updatedAverage.ewma.longValue) - Some(updated) - } else None - } - } - streamingDataSet ++= changes.map(m ⇒ m.name -> m).toMap - } - } - } -} diff --git a/akka-cluster/src/test/scala/akka/cluster/EWMASpec.scala b/akka-cluster/src/test/scala/akka/cluster/EWMASpec.scala new file mode 100644 index 0000000000..760f64206f --- /dev/null +++ b/akka-cluster/src/test/scala/akka/cluster/EWMASpec.scala @@ -0,0 +1,101 @@ +/* + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster + +import language.postfixOps +import scala.concurrent.duration._ +import akka.testkit.{ LongRunningTest, AkkaSpec } +import scala.concurrent.forkjoin.ThreadLocalRandom + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class EWMASpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricsCollectorFactory { + import system.dispatcher + + val collector = createMetricsCollector + + "DataStream" must { + + "calcualate same ewma for constant values" in { + val ds = EWMA(value = 100.0, alpha = 0.18) :+ + 100.0 :+ 100.0 :+ 100.0 + ds.value must be(100.0 plusOrMinus 0.001) + } + + "calcualate correct ewma for normal decay" in { + val d0 = EWMA(value = 1000.0, alpha = 2.0 / (1 + 10)) + d0.value must be(1000.0 plusOrMinus 0.01) + val d1 = d0 :+ 10.0 + d1.value must be(820.0 plusOrMinus 0.01) + val d2 = d1 :+ 10.0 + d2.value must be(672.73 plusOrMinus 0.01) + val d3 = d2 :+ 10.0 + d3.value must be(552.23 plusOrMinus 0.01) + val d4 = d3 :+ 10.0 + d4.value must be(453.64 plusOrMinus 0.01) + + val dn = (1 to 100).foldLeft(d0)((d, _) ⇒ d :+ 10.0) + dn.value must be(10.0 plusOrMinus 0.1) + } + + "calculate ewma for alpha 1.0, max bias towards latest value" in { + val d0 = EWMA(value = 100.0, alpha = 1.0) + d0.value must be(100.0 plusOrMinus 0.01) + val d1 = d0 :+ 1.0 + d1.value must be(1.0 plusOrMinus 0.01) + val d2 = d1 :+ 57.0 + d2.value must be(57.0 plusOrMinus 0.01) + val d3 = d2 :+ 10.0 + d3.value must be(10.0 plusOrMinus 0.01) + } + + "calculate alpha from half-life and collect interval" in { + // according to http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average + val expectedAlpha = 0.1 + // alpha = 2.0 / (1 + N) + val n = 19 + val halfLife = n.toDouble / 2.8854 + val collectInterval = 1.second + val halfLifeDuration = (halfLife * 1000).millis + EWMA.alpha(halfLifeDuration, collectInterval) must be(expectedAlpha plusOrMinus 0.001) + } + + "calculate sane alpha from short half-life" in { + val alpha = EWMA.alpha(1.millis, 3.seconds) + alpha must be <= (1.0) + alpha must be >= (0.0) + alpha must be(1.0 plusOrMinus 0.001) + } + + "calculate sane alpha from long half-life" in { + val alpha = EWMA.alpha(1.day, 3.seconds) + alpha must be <= (1.0) + alpha must be >= (0.0) + alpha must be(0.0 plusOrMinus 0.001) + } + + "calculate the ewma for multiple, variable, data streams" taggedAs LongRunningTest in { + var streamingDataSet = Map.empty[String, Metric] + var usedMemory = Array.empty[Byte] + (1 to 50) foreach { _ ⇒ + // wait a while between each message to give the metrics a chance to change + Thread.sleep(100) + usedMemory = usedMemory ++ Array.fill(1024)(ThreadLocalRandom.current.nextInt(127).toByte) + val changes = collector.sample.metrics.flatMap { latest ⇒ + streamingDataSet.get(latest.name) match { + case None ⇒ Some(latest) + case Some(previous) ⇒ + if (latest.isSmooth && latest.value != previous.value) { + val updated = previous :+ latest + updated.isSmooth must be(true) + updated.smoothValue must not be (previous.smoothValue) + Some(updated) + } else None + } + } + streamingDataSet ++= changes.map(m ⇒ m.name -> m).toMap + } + } + } +} diff --git a/akka-cluster/src/test/scala/akka/cluster/MetricNumericConverterSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MetricNumericConverterSpec.scala index f866056d9f..e20e670879 100644 --- a/akka-cluster/src/test/scala/akka/cluster/MetricNumericConverterSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/MetricNumericConverterSpec.scala @@ -5,10 +5,11 @@ package akka.cluster import akka.testkit.{ ImplicitSender, AkkaSpec } -import akka.cluster.NodeMetrics.MetricValues._ +import akka.cluster.StandardMetrics.HeapMemory.Fields._ +import scala.util.Try @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class MetricNumericConverterSpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricNumericConverter with ImplicitSender with MetricSpec +class MetricNumericConverterSpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricNumericConverter with ImplicitSender with MetricsCollectorFactory { "MetricNumericConverter" must { @@ -22,26 +23,17 @@ class MetricNumericConverterSpec extends AkkaSpec(MetricsEnabledSpec.config) wit } "define a new metric" in { - val metric = Metric(HeapMemoryUsed, Some(256L), decay = Some(10)) + val Some(metric) = Metric.create(HeapMemoryUsed, 256L, decayFactor = Some(0.18)) metric.name must be(HeapMemoryUsed) - metric.isDefined must be(true) - metric.value must be(Some(256L)) - metric.average.isDefined must be(true) - metric.average.get.ewma must be(256L) - - collector match { - case c: SigarMetricsCollector ⇒ - val cores = c.totalCores - cores.isDefined must be(true) - cores.value.get.intValue must be > (0) - case _ ⇒ - } + metric.value must be(256L) + metric.isSmooth must be(true) + metric.smoothValue must be(256.0 plusOrMinus 0.0001) } "define an undefined value with a None " in { - Metric("x", Some(-1), None).value.isDefined must be(false) - Metric("x", Some(java.lang.Double.NaN), None).value.isDefined must be(false) - Metric("x", None, None).isDefined must be(false) + Metric.create("x", -1, None).isDefined must be(false) + Metric.create("x", java.lang.Double.NaN, None).isDefined must be(false) + Metric.create("x", Try(throw new RuntimeException), None).isDefined must be(false) } "recognize whether a metric value is defined" in { diff --git a/akka-cluster/src/test/scala/akka/cluster/MetricValuesSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MetricValuesSpec.scala index 5910d0510f..b174375ece 100644 --- a/akka-cluster/src/test/scala/akka/cluster/MetricValuesSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/MetricValuesSpec.scala @@ -6,75 +6,69 @@ package akka.cluster import scala.util.Try import akka.actor.Address -import akka.cluster.NodeMetrics.MetricValues._ import akka.testkit.AkkaSpec +import akka.cluster.StandardMetrics.HeapMemory +import akka.cluster.StandardMetrics.Cpu @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class MetricValuesSpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricSpec - with MetricsCollectorFactory { - import NodeMetrics._ +class MetricValuesSpec extends AkkaSpec(MetricsEnabledSpec.config) with MetricsCollectorFactory { val collector = createMetricsCollector val node1 = NodeMetrics(Address("akka", "sys", "a", 2554), 1, collector.sample.metrics) val node2 = NodeMetrics(Address("akka", "sys", "a", 2555), 1, collector.sample.metrics) - var nodes: Seq[NodeMetrics] = Seq(node1, node2) - - // work up the data streams where applicable - for (i ← 1 to 100) { - nodes = nodes map { - n ⇒ + val nodes: Seq[NodeMetrics] = { + var nodes = Seq(node1, node2) + // work up the data streams where applicable + for (i ← 1 to 100) { + nodes = nodes map { n ⇒ n.copy(metrics = collector.sample.metrics.flatMap(latest ⇒ n.metrics.collect { - case streaming if latest same streaming ⇒ - streaming.average match { - case Some(e) ⇒ streaming.copy(value = latest.value, average = - if (latest.isDefined) Some(e :+ latest.value.get.doubleValue) else None) - case None ⇒ streaming.copy(value = latest.value) - } + case streaming if latest sameAs streaming ⇒ streaming :+ latest })) + } } + nodes } "NodeMetrics.MetricValues" must { "extract expected metrics for load balancing" in { - val stream1 = node2.metric(HeapMemoryCommitted).value.get.longValue - val stream2 = node1.metric(HeapMemoryUsed).value.get.longValue + import HeapMemory.Fields._ + val stream1 = node2.metric(HeapMemoryCommitted).get.value.longValue + val stream2 = node1.metric(HeapMemoryUsed).get.value.longValue stream1 must be >= (stream2) } "extract expected MetricValue types for load balancing" in { - nodes foreach { - node ⇒ - val (used, committed, max) = MetricValues.unapply(node.heapMemory) - committed must be >= (used) - max match { - case Some(m) ⇒ - used must be <= (m) - committed must be <= (m) - case None ⇒ - used must be > (0L) - committed must be > (0L) - } + nodes foreach { node ⇒ + node match { + case HeapMemory(address, _, used, committed, Some(max)) ⇒ + committed must be >= (used) + used must be <= (max) + committed must be <= (max) + // extract is the java api + StandardMetrics.extractHeapMemory(node) must not be (null) + case HeapMemory(address, _, used, committed, None) ⇒ + used must be > (0L) + committed must be > (0L) + // extract is the java api + StandardMetrics.extractCpu(node) must not be (null) + case _ ⇒ fail("no heap") + } - val network = MetricValues.unapply(node.networkLatency) - if (network.isDefined) { - network.get._1 must be > (0L) - network.get._2 must be > (0L) - } - - val (systemLoadAverage, processors, combinedCpu, cores) = MetricValues.unapply(node.cpu) - processors must be > (0) - if (systemLoadAverage.isDefined) - systemLoadAverage.get must be >= (0.0) - if (combinedCpu.isDefined) { - combinedCpu.get must be <= (1.0) - combinedCpu.get must be >= (0.0) - } - if (cores.isDefined) { - cores.get must be > (0) - cores.get must be >= (processors) - } + node match { + case Cpu(address, _, systemLoadAverageOption, cpuCombinedOption, processors) ⇒ + processors must be > (0) + if (systemLoadAverageOption.isDefined) + systemLoadAverageOption.get must be >= (0.0) + if (cpuCombinedOption.isDefined) { + cpuCombinedOption.get must be <= (1.0) + cpuCombinedOption.get must be >= (0.0) + } + // extract is the java api + StandardMetrics.extractCpu(node) must not be (null) + case _ ⇒ fail("no cpu") + } } } } diff --git a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala index ad836bf5f7..55c57b9775 100644 --- a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala @@ -1,4 +1,5 @@ /* + * Copyright (C) 2009-2012 Typesafe Inc. */ @@ -11,7 +12,8 @@ import scala.util.{ Success, Try, Failure } import akka.actor._ import akka.testkit._ -import akka.cluster.NodeMetrics.MetricValues._ +import akka.cluster.StandardMetrics.HeapMemory.Fields._ +import akka.cluster.StandardMetrics.Cpu.Fields._ import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers @@ -20,14 +22,12 @@ object MetricsEnabledSpec { akka.cluster.metrics.enabled = on akka.cluster.metrics.collect-interval = 1 s akka.cluster.metrics.gossip-interval = 1 s - akka.cluster.metrics.rate-of-decay = 10 akka.actor.provider = "akka.remote.RemoteActorRefProvider" """ } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with ImplicitSender with MetricSpec - with MetricsCollectorFactory { +class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with ImplicitSender with MetricsCollectorFactory { import system.dispatcher val collector = createMetricsCollector @@ -38,25 +38,25 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl for (i ← 1 to 20) { val sample1 = collector.sample.metrics val sample2 = collector.sample.metrics - var merged = sample2 flatMap (latest ⇒ sample1 collect { - case peer if latest same peer ⇒ { + val merged12 = sample2 flatMap (latest ⇒ sample1 collect { + case peer if latest sameAs peer ⇒ { val m = peer :+ latest - assertMerged(latest, peer, m) + m.value must be(latest.value) + m.isSmooth must be(peer.isSmooth || latest.isSmooth) m } }) val sample3 = collector.sample.metrics val sample4 = collector.sample.metrics - merged = sample4 flatMap (latest ⇒ sample3 collect { - case peer if latest same peer ⇒ { + val merged34 = sample4 flatMap (latest ⇒ sample3 collect { + case peer if latest sameAs peer ⇒ { val m = peer :+ latest - assertMerged(latest, peer, m) + m.value must be(latest.value) + m.isSmooth must be(peer.isSmooth || latest.isSmooth) m } }) - merged.size must be(sample3.size) - merged.size must be(sample4.size) } } } @@ -69,13 +69,10 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl "collect accurate metrics for a node" in { val sample = collector.sample - val metrics = sample.metrics.collect { case m if m.isDefined ⇒ (m.name, m.value.get) } + val metrics = sample.metrics.collect { case m ⇒ (m.name, m.value) } val used = metrics collectFirst { case (HeapMemoryUsed, b) ⇒ b } val committed = metrics collectFirst { case (HeapMemoryCommitted, b) ⇒ b } metrics foreach { - case (TotalCores, b) ⇒ b.intValue must be > (0) - case (NetworkInboundRate, b) ⇒ b.longValue must be > (0L) - case (NetworkOutboundRate, b) ⇒ b.longValue must be > (0L) case (SystemLoadAverage, b) ⇒ b.doubleValue must be >= (0.0) case (Processors, b) ⇒ b.intValue must be >= (0) case (HeapMemoryUsed, b) ⇒ b.longValue must be >= (0L) @@ -91,25 +88,14 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl } } - "collect SIGAR metrics if it is on the classpath" in { - collector match { - case c: SigarMetricsCollector ⇒ - // combined cpu may or may not be defined on a given sampling - // systemLoadAverage is not present on all platforms - c.networkMaxRx.isDefined must be(true) - c.networkMaxTx.isDefined must be(true) - c.totalCores.isDefined must be(true) - case _ ⇒ - } - } - "collect JMX metrics" in { // heap max may be undefined depending on the OS // systemLoadAverage is JMX when SIGAR not present, but // it's not present on all platforms val c = collector.asInstanceOf[JmxMetricsCollector] - c.heapUsed.isDefined must be(true) - c.heapCommitted.isDefined must be(true) + val heap = c.heapMemoryUsage + c.heapUsed(heap).isDefined must be(true) + c.heapCommitted(heap).isDefined must be(true) c.processors.isDefined must be(true) } @@ -123,47 +109,6 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl } } -trait MetricSpec extends WordSpec with MustMatchers { this: { def system: ActorSystem } ⇒ - - def assertMasterMetricsAgainstGossipMetrics(master: Set[NodeMetrics], gossip: MetricsGossip): Unit = { - val masterMetrics = collectNodeMetrics(master) - val gossipMetrics = collectNodeMetrics(gossip.nodes) - gossipMetrics.size must be(masterMetrics.size plusOrMinus 1) // combined cpu - } - - def assertExpectedNodeAddresses(gossip: MetricsGossip, nodes: Set[NodeMetrics]): Unit = - gossip.nodes.map(_.address) must be(nodes.map(_.address)) - - def assertMerged(latest: Metric, peer: Metric, merged: Metric): Unit = if (latest same peer) { - if (latest.isDefined) { - if (peer.isDefined) { - merged.isDefined must be(true) - merged.value.get must be(latest.value.get) - merged.average.isDefined must be(latest.average.isDefined) - } else { - merged.isDefined must be(true) - merged.value.get must be(latest.value.get) - merged.average.isDefined must be(latest.average.isDefined || peer.average.isDefined) - } - } else { - if (peer.isDefined) { - merged.isDefined must be(true) - merged.value.get must be(peer.value.get) - merged.average.isDefined must be(peer.average.isDefined) - } else { - merged.isDefined must be(false) - merged.average.isEmpty must be(true) - } - } - } - - def collectNodeMetrics(nodes: Set[NodeMetrics]): Seq[Metric] = { - var r: Seq[Metric] = Seq.empty - nodes.foreach(n ⇒ r ++= n.metrics.filter(_.isDefined)) - r - } -} - /** * Used when testing metrics without full cluster */ @@ -173,15 +118,15 @@ trait MetricsCollectorFactory { this: AkkaSpec ⇒ def selfAddress = extendedActorSystem.provider.rootPath.address - val defaultRateOfDecay = 10 + val defaultDecayFactor = 2.0 / (1 + 10) def createMetricsCollector: MetricsCollector = - Try(new SigarMetricsCollector(selfAddress, defaultRateOfDecay, extendedActorSystem.dynamicAccess)) match { + Try(new SigarMetricsCollector(selfAddress, defaultDecayFactor, + extendedActorSystem.dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Seq.empty).get)) match { case Success(sigarCollector) ⇒ sigarCollector case Failure(e) ⇒ - log.debug("Metrics will be retreived from MBeans, Sigar failed to load. Reason: " + - e.getMessage) - new JmxMetricsCollector(selfAddress, defaultRateOfDecay) + log.debug("Metrics will be retreived from MBeans, Sigar failed to load. Reason: " + e) + new JmxMetricsCollector(selfAddress, defaultDecayFactor) } def isSigar(collector: MetricsCollector): Boolean = collector.isInstanceOf[SigarMetricsCollector] diff --git a/akka-cluster/src/test/scala/akka/cluster/MetricsGossipSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MetricsGossipSpec.scala index 3ec10001f2..36470a4725 100644 --- a/akka-cluster/src/test/scala/akka/cluster/MetricsGossipSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/MetricsGossipSpec.scala @@ -12,8 +12,7 @@ import akka.actor.Address import java.lang.System.{ currentTimeMillis ⇒ newTimestamp } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class MetricsGossipSpec extends AkkaSpec(MetricsEnabledSpec.config) with ImplicitSender with MetricSpec - with MetricsCollectorFactory { +class MetricsGossipSpec extends AkkaSpec(MetricsEnabledSpec.config) with ImplicitSender with MetricsCollectorFactory { val collector = createMetricsCollector @@ -22,82 +21,76 @@ class MetricsGossipSpec extends AkkaSpec(MetricsEnabledSpec.config) with Implici val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics) val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics) - var localGossip = MetricsGossip() - localGossip :+= m1 - localGossip.nodes.size must be(1) - localGossip.nodeKeys.size must be(localGossip.nodes.size) - assertMasterMetricsAgainstGossipMetrics(Set(m1), localGossip) - collector.sample.metrics.size must be > (3) + m1.metrics.size must be > (3) + m2.metrics.size must be > (3) - localGossip :+= m2 - localGossip.nodes.size must be(2) - localGossip.nodeKeys.size must be(localGossip.nodes.size) - assertMasterMetricsAgainstGossipMetrics(Set(m1, m2), localGossip) - collector.sample.metrics.size must be > (3) + val g1 = MetricsGossip.empty :+ m1 + g1.nodes.size must be(1) + g1.nodeKeys.size must be(g1.nodes.size) + g1.metricsFor(m1.address).size must be(m1.metrics.size) + + val g2 = g1 :+ m2 + g2.nodes.size must be(2) + g2.nodeKeys.size must be(g2.nodes.size) + g2.metricsFor(m1.address).size must be(m1.metrics.size) + g2.metricsFor(m2.address).size must be(m2.metrics.size) } "merge peer metrics" in { val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics) val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics) - var remoteGossip = MetricsGossip() - remoteGossip :+= m1 - remoteGossip :+= m2 - remoteGossip.nodes.size must be(2) - val beforeMergeNodes = remoteGossip.nodes + val g1 = MetricsGossip.empty :+ m1 :+ m2 + g1.nodes.size must be(2) + val beforeMergeNodes = g1.nodes - val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = newTimestamp) - remoteGossip :+= m2Updated // merge peers - remoteGossip.nodes.size must be(2) - assertMasterMetricsAgainstGossipMetrics(beforeMergeNodes, remoteGossip) - remoteGossip.nodes.foreach(_.metrics.size must be > (3)) - remoteGossip.nodes collect { case peer if peer.address == m2.address ⇒ peer.timestamp must be(m2Updated.timestamp) } + val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = m2.timestamp + 1000) + val g2 = g1 :+ m2Updated // merge peers + g2.nodes.size must be(2) + g2.metricsFor(m1.address).size must be(m1.metrics.size) + g2.metricsFor(m2.address).size must be(m2Updated.metrics.size) + g2.nodes collect { case peer if peer.address == m2.address ⇒ peer.timestamp must be(m2Updated.timestamp) } } "merge an existing metric set for a node and update node ring" in { val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics) val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics) val m3 = NodeMetrics(Address("akka", "sys", "a", 2556), newTimestamp, collector.sample.metrics) - val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = newTimestamp) + val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = m2.timestamp + 1000) - var localGossip = MetricsGossip() - localGossip :+= m1 - localGossip :+= m2 + val g1 = MetricsGossip.empty :+ m1 :+ m2 + val g2 = MetricsGossip.empty :+ m3 :+ m2Updated - var remoteGossip = MetricsGossip() - remoteGossip :+= m3 - remoteGossip :+= m2Updated - - localGossip.nodeKeys.contains(m1.address) must be(true) - remoteGossip.nodeKeys.contains(m3.address) must be(true) + g1.nodeKeys.contains(m1.address) must be(true) + g2.nodeKeys.contains(m3.address) must be(true) // must contain nodes 1,3, and the most recent version of 2 - val mergedGossip = localGossip merge remoteGossip + val mergedGossip = g1 merge g2 mergedGossip.nodes.size must be(3) - assertExpectedNodeAddresses(mergedGossip, Set(m1, m2, m3)) + mergedGossip.metricsFor(m1.address).size must be(m1.metrics.size) + mergedGossip.metricsFor(m2.address).size must be(m2Updated.metrics.size) + mergedGossip.metricsFor(m3.address).size must be(m3.metrics.size) mergedGossip.nodes.foreach(_.metrics.size must be > (3)) mergedGossip.nodes.find(_.address == m2.address).get.timestamp must be(m2Updated.timestamp) } "get the current NodeMetrics if it exists in the local nodes" in { val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics) - var localGossip = MetricsGossip() - localGossip :+= m1 - localGossip.metricsFor(m1).nonEmpty must be(true) + val g1 = MetricsGossip.empty :+ m1 + g1.metricsFor(m1.address).nonEmpty must be(true) } "remove a node if it is no longer Up" in { val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample.metrics) val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample.metrics) - var localGossip = MetricsGossip() - localGossip :+= m1 - localGossip :+= m2 - - localGossip.nodes.size must be(2) - localGossip = localGossip remove m1.address - localGossip.nodes.size must be(1) - localGossip.nodes.exists(_.address == m1.address) must be(false) + val g1 = MetricsGossip.empty :+ m1 :+ m2 + g1.nodes.size must be(2) + val g2 = g1 remove m1.address + g2.nodes.size must be(1) + g2.nodes.exists(_.address == m1.address) must be(false) + g2.metricsFor(m1.address).size must be(0) + g2.metricsFor(m2.address).size must be(m2.metrics.size) } } } diff --git a/akka-cluster/src/test/scala/akka/cluster/NodeMetricsSpec.scala b/akka-cluster/src/test/scala/akka/cluster/NodeMetricsSpec.scala index d6a47db670..2c62bc5a41 100644 --- a/akka-cluster/src/test/scala/akka/cluster/NodeMetricsSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/NodeMetricsSpec.scala @@ -8,8 +8,7 @@ import akka.testkit.AkkaSpec import akka.actor.Address @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class NodeMetricsSpec extends AkkaSpec with MetricSpec - with MetricsCollectorFactory { +class NodeMetricsSpec extends AkkaSpec with MetricsCollectorFactory { val collector = createMetricsCollector @@ -27,11 +26,11 @@ class NodeMetricsSpec extends AkkaSpec with MetricSpec } "return correct result for 2 'same' nodes" in { - (NodeMetrics(node1, 0) same NodeMetrics(node1, 0)) must be(true) + (NodeMetrics(node1, 0) sameAs NodeMetrics(node1, 0)) must be(true) } "return correct result for 2 not 'same' nodes" in { - (NodeMetrics(node1, 0) same NodeMetrics(node2, 0)) must be(false) + (NodeMetrics(node1, 0) sameAs NodeMetrics(node2, 0)) must be(false) } "merge 2 NodeMetrics by most recent" in { diff --git a/akka-cluster/src/test/scala/akka/cluster/routing/HeapMetricsSelectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/routing/HeapMetricsSelectorSpec.scala deleted file mode 100644 index 5a8fecc2eb..0000000000 --- a/akka-cluster/src/test/scala/akka/cluster/routing/HeapMetricsSelectorSpec.scala +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.cluster.routing - -import akka.testkit.AkkaSpec -import akka.actor.Address -import akka.actor.RootActorPath -import akka.cluster.NodeMetrics -import akka.cluster.NodeMetrics.MetricValues._ -import akka.cluster.Metric -import com.typesafe.config.ConfigFactory - -@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class HeapMetricsSelectorSpec extends AkkaSpec(ConfigFactory.parseString(""" - akka.actor.provider = "akka.cluster.ClusterActorRefProvider" - akka.remote.netty.port = 0 - """)) { - - val selector = HeapMetricsSelector - - val a1 = Address("akka", "sys", "a1", 2551) - val b1 = Address("akka", "sys", "b1", 2551) - val c1 = Address("akka", "sys", "c1", 2551) - val d1 = Address("akka", "sys", "d1", 2551) - - val decay = Some(10) - - val nodeMetricsA = NodeMetrics(a1, System.currentTimeMillis, Set( - Metric(HeapMemoryUsed, Some(BigInt(128)), decay), - Metric(HeapMemoryCommitted, Some(BigInt(256)), decay), - Metric(HeapMemoryMax, Some(BigInt(512)), None))) - - val nodeMetricsB = NodeMetrics(b1, System.currentTimeMillis, Set( - Metric(HeapMemoryUsed, Some(BigInt(256)), decay), - Metric(HeapMemoryCommitted, Some(BigInt(512)), decay), - Metric(HeapMemoryMax, Some(BigInt(1024)), None))) - - val nodeMetricsC = NodeMetrics(c1, System.currentTimeMillis, Set( - Metric(HeapMemoryUsed, Some(BigInt(1024)), decay), - Metric(HeapMemoryCommitted, Some(BigInt(1024)), decay), - Metric(HeapMemoryMax, Some(BigInt(1024)), None))) - - val nodeMetrics = Set(nodeMetricsA, nodeMetricsB, nodeMetricsC) - - "MetricsAwareClusterNodeSelector" must { - - "calculate capacity of heap metrics" in { - val capacity = selector.capacity(nodeMetrics) - capacity(a1) must be(0.75 plusOrMinus 0.0001) - capacity(b1) must be(0.75 plusOrMinus 0.0001) - capacity(c1) must be(0.0 plusOrMinus 0.0001) - } - - "calculate weights from capacity" in { - val capacity = Map(a1 -> 0.6, b1 -> 0.3, c1 -> 0.1) - val weights = selector.weights(capacity) - weights must be(Map(c1 -> 1, b1 -> 3, a1 -> 6)) - } - - "handle low and zero capacity" in { - val capacity = Map(a1 -> 0.0, b1 -> 1.0, c1 -> 0.005, d1 -> 0.004) - val weights = selector.weights(capacity) - weights must be(Map(a1 -> 0, b1 -> 100, c1 -> 1, d1 -> 0)) - } - - "allocate weighted refs" in { - val weights = Map(a1 -> 1, b1 -> 3, c1 -> 10) - val refs = IndexedSeq( - system.actorFor(RootActorPath(a1) / "user" / "a"), - system.actorFor(RootActorPath(b1) / "user" / "b"), - system.actorFor(RootActorPath(c1) / "user" / "c")) - val result = selector.weightedRefs(refs, a1, weights) - val grouped = result.groupBy(_.path.address) - grouped(a1).size must be(1) - grouped(b1).size must be(3) - grouped(c1).size must be(10) - } - - "allocate refs for undefined weight" in { - val weights = Map(a1 -> 1, b1 -> 2) - val refs = IndexedSeq( - system.actorFor(RootActorPath(a1) / "user" / "a"), - system.actorFor(RootActorPath(b1) / "user" / "b"), - system.actorFor(RootActorPath(c1) / "user" / "c")) - val result = selector.weightedRefs(refs, a1, weights) - val grouped = result.groupBy(_.path.address) - grouped(a1).size must be(1) - grouped(b1).size must be(2) - grouped(c1).size must be(1) - } - - "allocate weighted local refs" in { - val weights = Map(a1 -> 2, b1 -> 1, c1 -> 10) - val refs = IndexedSeq( - testActor, - system.actorFor(RootActorPath(b1) / "user" / "b"), - system.actorFor(RootActorPath(c1) / "user" / "c")) - val result = selector.weightedRefs(refs, a1, weights) - result.filter(_ == testActor).size must be(2) - } - - "not allocate ref with weight zero" in { - val weights = Map(a1 -> 0, b1 -> 2, c1 -> 10) - val refs = IndexedSeq( - system.actorFor(RootActorPath(a1) / "user" / "a"), - system.actorFor(RootActorPath(b1) / "user" / "b"), - system.actorFor(RootActorPath(c1) / "user" / "c")) - val result = selector.weightedRefs(refs, a1, weights) - result.filter(_ == refs.head).size must be(0) - } - } -} \ No newline at end of file diff --git a/akka-cluster/src/test/scala/akka/cluster/routing/MetricsSelectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/routing/MetricsSelectorSpec.scala new file mode 100644 index 0000000000..851152413e --- /dev/null +++ b/akka-cluster/src/test/scala/akka/cluster/routing/MetricsSelectorSpec.scala @@ -0,0 +1,119 @@ +/* + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster.routing + +import org.scalatest.WordSpec +import org.scalatest.matchers.MustMatchers + +import akka.actor.Address +import akka.cluster.Metric +import akka.cluster.NodeMetrics +import akka.cluster.StandardMetrics.HeapMemory.Fields._ +import akka.cluster.StandardMetrics.Cpu.Fields._ + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class MetricsSelectorSpec extends WordSpec with MustMatchers { + + val abstractSelector = new CapacityMetricsSelector { + override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = Map.empty + } + + val a1 = Address("akka", "sys", "a1", 2551) + val b1 = Address("akka", "sys", "b1", 2551) + val c1 = Address("akka", "sys", "c1", 2551) + val d1 = Address("akka", "sys", "d1", 2551) + + val decayFactor = Some(0.18) + + val nodeMetricsA = NodeMetrics(a1, System.currentTimeMillis, Set( + Metric.create(HeapMemoryUsed, 128, decayFactor), + Metric.create(HeapMemoryCommitted, 256, decayFactor), + Metric.create(HeapMemoryMax, 512, None), + Metric.create(CpuCombined, 0.1, decayFactor), + Metric.create(SystemLoadAverage, 0.5, None), + Metric.create(Processors, 8, None)).flatten) + + val nodeMetricsB = NodeMetrics(b1, System.currentTimeMillis, Set( + Metric.create(HeapMemoryUsed, 256, decayFactor), + Metric.create(HeapMemoryCommitted, 512, decayFactor), + Metric.create(HeapMemoryMax, 1024, None), + Metric.create(CpuCombined, 0.5, decayFactor), + Metric.create(SystemLoadAverage, 1.0, None), + Metric.create(Processors, 16, None)).flatten) + + val nodeMetricsC = NodeMetrics(c1, System.currentTimeMillis, Set( + Metric.create(HeapMemoryUsed, 1024, decayFactor), + Metric.create(HeapMemoryCommitted, 1024, decayFactor), + Metric.create(HeapMemoryMax, 1024, None), + Metric.create(CpuCombined, 1.0, decayFactor), + Metric.create(SystemLoadAverage, 16.0, None), + Metric.create(Processors, 16, None)).flatten) + + val nodeMetricsD = NodeMetrics(d1, System.currentTimeMillis, Set( + Metric.create(HeapMemoryUsed, 511, decayFactor), + Metric.create(HeapMemoryCommitted, 512, decayFactor), + Metric.create(HeapMemoryMax, 512, None), + Metric.create(Processors, 2, decayFactor)).flatten) + + val nodeMetrics = Set(nodeMetricsA, nodeMetricsB, nodeMetricsC, nodeMetricsD) + + "CapacityMetricsSelector" must { + + "calculate weights from capacity" in { + val capacity = Map(a1 -> 0.6, b1 -> 0.3, c1 -> 0.1) + val weights = abstractSelector.weights(capacity) + weights must be(Map(c1 -> 1, b1 -> 3, a1 -> 6)) + } + + "handle low and zero capacity" in { + val capacity = Map(a1 -> 0.0, b1 -> 1.0, c1 -> 0.005, d1 -> 0.004) + val weights = abstractSelector.weights(capacity) + weights must be(Map(a1 -> 0, b1 -> 100, c1 -> 1, d1 -> 0)) + } + + } + + "HeapMetricsSelector" must { + "calculate capacity of heap metrics" in { + val capacity = HeapMetricsSelector.capacity(nodeMetrics) + capacity(a1) must be(0.75 plusOrMinus 0.0001) + capacity(b1) must be(0.75 plusOrMinus 0.0001) + capacity(c1) must be(0.0 plusOrMinus 0.0001) + capacity(d1) must be(0.001953125 plusOrMinus 0.0001) + } + } + + "CpuMetricsSelector" must { + "calculate capacity of cpuCombined metrics" in { + val capacity = CpuMetricsSelector.capacity(nodeMetrics) + capacity(a1) must be(0.9 plusOrMinus 0.0001) + capacity(b1) must be(0.5 plusOrMinus 0.0001) + capacity(c1) must be(0.0 plusOrMinus 0.0001) + capacity.contains(d1) must be(false) + } + } + + "SystemLoadAverageMetricsSelector" must { + "calculate capacity of systemLoadAverage metrics" in { + val capacity = SystemLoadAverageMetricsSelector.capacity(nodeMetrics) + capacity(a1) must be(0.9375 plusOrMinus 0.0001) + capacity(b1) must be(0.9375 plusOrMinus 0.0001) + capacity(c1) must be(0.0 plusOrMinus 0.0001) + capacity.contains(d1) must be(false) + } + } + + "MixMetricsSelector" must { + "aggregate capacity of all metrics" in { + val capacity = MixMetricsSelector().capacity(nodeMetrics) + capacity(a1) must be((0.75 + 0.9 + 0.9375) / 3 plusOrMinus 0.0001) + capacity(b1) must be((0.75 + 0.5 + 0.9375) / 3 plusOrMinus 0.0001) + capacity(c1) must be((0.0 + 0.0 + 0.0) / 3 plusOrMinus 0.0001) + capacity(d1) must be((0.001953125) / 1 plusOrMinus 0.0001) + } + } + +} + diff --git a/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala b/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala new file mode 100644 index 0000000000..3b5ad27c9e --- /dev/null +++ b/akka-cluster/src/test/scala/akka/cluster/routing/WeightedRouteesSpec.scala @@ -0,0 +1,84 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.cluster.routing + +import com.typesafe.config.ConfigFactory + +import akka.actor.Address +import akka.actor.RootActorPath +import akka.testkit.AkkaSpec + +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) +class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString(""" + akka.actor.provider = "akka.cluster.ClusterActorRefProvider" + akka.remote.netty.port = 0 + """)) { + + val a1 = Address("akka", "sys", "a1", 2551) + val b1 = Address("akka", "sys", "b1", 2551) + val c1 = Address("akka", "sys", "c1", 2551) + val d1 = Address("akka", "sys", "d1", 2551) + + val refA = system.actorFor(RootActorPath(a1) / "user" / "a") + val refB = system.actorFor(RootActorPath(b1) / "user" / "b") + val refC = system.actorFor(RootActorPath(c1) / "user" / "c") + + "WeightedRoutees" must { + + "allocate weighted refs" in { + val weights = Map(a1 -> 1, b1 -> 3, c1 -> 10) + val refs = IndexedSeq(refA, refB, refC) + val weighted = new WeightedRoutees(refs, a1, weights) + + weighted(1) must be(refA) + 2 to 4 foreach { weighted(_) must be(refB) } + 5 to 14 foreach { weighted(_) must be(refC) } + weighted.total must be(14) + } + + "check boundaries" in { + val empty = new WeightedRoutees(Vector(), a1, Map.empty) + empty.total must be(0) + val weighted = new WeightedRoutees(IndexedSeq(refA, refB, refC), a1, Map.empty) + weighted.total must be(3) + intercept[IllegalArgumentException] { + weighted(0) + } + intercept[IllegalArgumentException] { + weighted(4) + } + } + + "allocate refs for undefined weight" in { + val weights = Map(a1 -> 1, b1 -> 7) + val refs = IndexedSeq(refA, refB, refC) + val weighted = new WeightedRoutees(refs, a1, weights) + + weighted(1) must be(refA) + 2 to 8 foreach { weighted(_) must be(refB) } + // undefined, uses the mean of the weights, i.e. 4 + 9 to 12 foreach { weighted(_) must be(refC) } + weighted.total must be(12) + } + + "allocate weighted local refs" in { + val weights = Map(a1 -> 2, b1 -> 1, c1 -> 10) + val refs = IndexedSeq(testActor, refB, refC) + val weighted = new WeightedRoutees(refs, a1, weights) + + 1 to 2 foreach { weighted(_) must be(testActor) } + 3 to weighted.total foreach { weighted(_) must not be (testActor) } + } + + "not allocate ref with weight zero" in { + val weights = Map(a1 -> 0, b1 -> 2, c1 -> 10) + val refs = IndexedSeq(refA, refB, refC) + val weighted = new WeightedRoutees(refs, a1, weights) + + 1 to weighted.total foreach { weighted(_) must not be (refA) } + } + + } +} diff --git a/akka-docs/rst/cluster/cluster-usage-java.rst b/akka-docs/rst/cluster/cluster-usage-java.rst index c674bff3a6..b91b3d943d 100644 --- a/akka-docs/rst/cluster/cluster-usage-java.rst +++ b/akka-docs/rst/cluster/cluster-usage-java.rst @@ -38,8 +38,7 @@ Try it out: 1. Add the following ``application.conf`` in your project, place it in ``src/main/resources``: -.. literalinclude:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf - :language: none +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#cluster To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-java` settings, but with ``akka.cluster.ClusterActorRefProvider``. @@ -438,6 +437,105 @@ service nodes and 1 client:: .. note:: The above example, especially the last part, will be simplified when the cluster handles automatic actor partitioning. +Cluster Metrics +^^^^^^^^^^^^^^^ + +The member nodes of the cluster collects system health metrics and publish that to other nodes and to +registered subscribers. This information is primarily used for load-balancing of nodes. + +Hyperic Sigar +------------- + +The built-in metrics is gathered from JMX MBeans, and optionally you can use `Hyperic Sigar `_ +for a wider and more accurate range of metrics compared to what can be retrieved from ordinary MBeans. +Sigar is using a native OS library. To enable usage of Sigar you need to add the directory of the native library to +``-Djava.libarary.path=`` add the following dependency:: + + + org.hyperic + sigar + 1.6.4 + + + + +Adaptive Load Balancing +----------------------- + +The ``AdaptiveLoadBalancingRouter`` performs load balancing of messages to cluster nodes based on the cluster metrics data. +It uses random selection of routees with probabilities derived from the remaining capacity of corresponding node. +It can be configured to use a specific MetricsSelector to produce the probabilities, a.k.a. weights: + +* ``heap`` / ``HeapMetricsSelector`` - Used and max JVM heap memory. Weights based on remaining heap capacity; (max - used) / max +* ``load`` / ``SystemLoadAverageMetricsSelector`` - System load average for the past 1 minute, corresponding value can be found in ``top`` of Linux systems. The system is possibly nearing a bottleneck if the system load average is nearing number of cpus/cores. Weights based on remaining load capacity; 1 - (load / processors) +* ``cpu`` / ``CpuMetricsSelector`` - CPU utilization in percentage, sum of User + Sys + Nice + Wait. Weights based on remaining cpu capacity; 1 - utilization +* ``mix`` / ``MixMetricsSelector`` - Combines heap, cpu and load. Weights based on mean of remaining capacity of the combined selectors. +* Any custom implementation of ``akka.cluster.routing.MetricsSelector`` + +The collected metrics values are smoothed with `exponential weighted moving average `_. In the :ref:`cluster_configuration_java` you can adjust how quickly past data is decayed compared to new data. + +Let's take a look at this router in action. + +In this example the following imports are used: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackend.java#imports + +The backend worker that performs the factorial calculation: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackend.java#backend + +The frontend that receives user jobs and delegates to the backends via the router: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontend.java#frontend + + +As you can see, the router is defined in the same way as other routers, and in this case it's configured as follows: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#adaptive-router + +It's only router type ``adaptive`` and the ``metrics-selector`` that is specific to this router, other things work +in the same way as other routers. + +The same type of router could also have been defined in code: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontend.java#router-lookup-in-code + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontend.java#router-deploy-in-code + +This example is included in ``akka-samples/akka-sample-cluster`` and you can try it by copying the +`source <@github@/akka-samples/akka-sample-cluster>`_ to your +maven project, defined as in :ref:`cluster_simple_example_java`. +Run it by starting nodes in different terminal windows. For example, starting 3 backend nodes and +one frontend:: + + mvn exec:java \ + -Dexec.mainClass="sample.cluster.factorial.FactorialBackendMain" \ + -Dexec.args="2551" + + mvn exec:java \ + -Dexec.mainClass="sample.cluster.factorial.FactorialBackendMain" \ + -Dexec.args="2552" + + mvn exec:java \ + -Dexec.mainClass="sample.cluster.factorial.FactorialBackendMain" + + mvn exec:java \ + -Dexec.mainClass="sample.cluster.factorial.FactorialFrontendMain" + + +Subscribe to Metrics Events +--------------------------- + +It's possible to subscribe to the metrics events directly to implement other functionality. + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/MetricsListener.java#metrics-listener + +Custom Metrics Collector +------------------------ + +You can plug-in your own metrics collector instead of +``akka.cluster.SigarMetricsCollector`` or ``akka.cluster.JmxMetricsCollector``. Look at those two implementations +for inspiration. The implementation class can be defined in the :ref:`cluster_configuration_java`. .. _cluster_jmx_java: diff --git a/akka-docs/rst/cluster/cluster-usage-scala.rst b/akka-docs/rst/cluster/cluster-usage-scala.rst index 31ce7e7191..00369e2303 100644 --- a/akka-docs/rst/cluster/cluster-usage-scala.rst +++ b/akka-docs/rst/cluster/cluster-usage-scala.rst @@ -32,8 +32,7 @@ Try it out: 1. Add the following ``application.conf`` in your project, place it in ``src/main/resources``: -.. literalinclude:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf - :language: none +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#cluster To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-scala` settings, but with ``akka.cluster.ClusterActorRefProvider``. @@ -265,6 +264,8 @@ This is how the curve looks like for ``acceptable-heartbeat-pause`` configured t .. image:: images/phi3.png +.. _cluster_aware_routers_scala: + Cluster Aware Routers ^^^^^^^^^^^^^^^^^^^^^ @@ -397,6 +398,96 @@ service nodes and 1 client:: .. note:: The above example, especially the last part, will be simplified when the cluster handles automatic actor partitioning. +Cluster Metrics +^^^^^^^^^^^^^^^ + +The member nodes of the cluster collects system health metrics and publish that to other nodes and to +registered subscribers. This information is primarily used for load-balancing of nodes. + +Hyperic Sigar +------------- + +The built-in metrics is gathered from JMX MBeans, and optionally you can use `Hyperic Sigar `_ +for a wider and more accurate range of metrics compared to what can be retrieved from ordinary MBeans. +Sigar is using a native OS library. To enable usage of Sigar you need to add the directory of the native library to +``-Djava.libarary.path=`` add the following dependency:: + + "org.hyperic" % "sigar" % "1.6.4" + + +Adaptive Load Balancing +----------------------- + +The ``AdaptiveLoadBalancingRouter`` performs load balancing of messages to cluster nodes based on the cluster metrics data. +It uses random selection of routees with probabilities derived from the remaining capacity of corresponding node. +It can be configured to use a specific MetricsSelector to produce the probabilities, a.k.a. weights: + +* ``heap`` / ``HeapMetricsSelector`` - Used and max JVM heap memory. Weights based on remaining heap capacity; (max - used) / max +* ``load`` / ``SystemLoadAverageMetricsSelector`` - System load average for the past 1 minute, corresponding value can be found in ``top`` of Linux systems. The system is possibly nearing a bottleneck if the system load average is nearing number of cpus/cores. Weights based on remaining load capacity; 1 - (load / processors) +* ``cpu`` / ``CpuMetricsSelector`` - CPU utilization in percentage, sum of User + Sys + Nice + Wait. Weights based on remaining cpu capacity; 1 - utilization +* ``mix`` / ``MixMetricsSelector`` - Combines heap, cpu and load. Weights based on mean of remaining capacity of the combined selectors. +* Any custom implementation of ``akka.cluster.routing.MetricsSelector`` + +The collected metrics values are smoothed with `exponential weighted moving average `_. In the :ref:`cluster_configuration_scala` you can adjust how quickly past data is decayed compared to new data. + +Let's take a look at this router in action. + +In this example the following imports are used: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#imports + +The backend worker that performs the factorial calculation: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#backend + +The frontend that receives user jobs and delegates to the backends via the router: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#frontend + + +As you can see, the router is defined in the same way as other routers, and in this case it's configured as follows: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#adaptive-router + +It's only router type ``adaptive`` and the ``metrics-selector`` that is specific to this router, other things work +in the same way as other routers. + +The same type of router could also have been defined in code: + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#router-lookup-in-code + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#router-deploy-in-code + +This example is included in ``akka-samples/akka-sample-cluster`` +and you can try by starting nodes in different terminal windows. For example, starting 3 backend nodes and one frontend:: + + sbt + + project akka-sample-cluster-experimental + + run-main sample.cluster.factorial.FactorialBackend 2551 + + run-main sample.cluster.factorial.FactorialBackend 2552 + + run-main sample.cluster.factorial.FactorialBackend + + run-main sample.cluster.factorial.FactorialFrontend + + +Subscribe to Metrics Events +--------------------------- + +It's possible to subscribe to the metrics events directly to implement other functionality. + +.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#metrics-listener + +Custom Metrics Collector +------------------------ + +You can plug-in your own metrics collector instead of +``akka.cluster.SigarMetricsCollector`` or ``akka.cluster.JmxMetricsCollector``. Look at those two implementations +for inspiration. The implementation class can be defined in the :ref:`cluster_configuration_scala`. + How to Test ^^^^^^^^^^^ diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 4967978582..f7bdb47e0e 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -24,7 +24,8 @@ class RemoteActorRefProvider( val remoteSettings: RemoteSettings = new RemoteSettings(settings.config, systemName) - val deployer: RemoteDeployer = new RemoteDeployer(settings, dynamicAccess) + // this is lazy to be able to override it in subclass + lazy val deployer: RemoteDeployer = new RemoteDeployer(settings, dynamicAccess) private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, dynamicAccess, deployer) diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-amd64-freebsd-6.so b/akka-samples/akka-sample-cluster/sigar/libsigar-amd64-freebsd-6.so new file mode 100644 index 0000000000..3e94f0d2bf Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-amd64-freebsd-6.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-amd64-linux.so b/akka-samples/akka-sample-cluster/sigar/libsigar-amd64-linux.so new file mode 100644 index 0000000000..5a2e4c24fe Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-amd64-linux.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-amd64-solaris.so b/akka-samples/akka-sample-cluster/sigar/libsigar-amd64-solaris.so new file mode 100644 index 0000000000..6396482a43 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-amd64-solaris.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-ia64-hpux-11.sl b/akka-samples/akka-sample-cluster/sigar/libsigar-ia64-hpux-11.sl new file mode 100644 index 0000000000..d92ea4a96a Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-ia64-hpux-11.sl differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-ia64-linux.so b/akka-samples/akka-sample-cluster/sigar/libsigar-ia64-linux.so new file mode 100644 index 0000000000..2bd2fc8e32 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-ia64-linux.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-pa-hpux-11.sl b/akka-samples/akka-sample-cluster/sigar/libsigar-pa-hpux-11.sl new file mode 100644 index 0000000000..0dfd8a1122 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-pa-hpux-11.sl differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-ppc-aix-5.so b/akka-samples/akka-sample-cluster/sigar/libsigar-ppc-aix-5.so new file mode 100644 index 0000000000..7d4b519921 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-ppc-aix-5.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-ppc-linux.so b/akka-samples/akka-sample-cluster/sigar/libsigar-ppc-linux.so new file mode 100644 index 0000000000..4394b1b00f Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-ppc-linux.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-ppc64-aix-5.so b/akka-samples/akka-sample-cluster/sigar/libsigar-ppc64-aix-5.so new file mode 100644 index 0000000000..35fd828808 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-ppc64-aix-5.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-ppc64-linux.so b/akka-samples/akka-sample-cluster/sigar/libsigar-ppc64-linux.so new file mode 100644 index 0000000000..a1ba2529c9 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-ppc64-linux.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-s390x-linux.so b/akka-samples/akka-sample-cluster/sigar/libsigar-s390x-linux.so new file mode 100644 index 0000000000..c275f4ac69 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-s390x-linux.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-sparc-solaris.so b/akka-samples/akka-sample-cluster/sigar/libsigar-sparc-solaris.so new file mode 100644 index 0000000000..aa847d2b54 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-sparc-solaris.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-sparc64-solaris.so b/akka-samples/akka-sample-cluster/sigar/libsigar-sparc64-solaris.so new file mode 100644 index 0000000000..6c4fe809c5 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-sparc64-solaris.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-universal-macosx.dylib b/akka-samples/akka-sample-cluster/sigar/libsigar-universal-macosx.dylib new file mode 100644 index 0000000000..27ab107111 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-universal-macosx.dylib differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-universal64-macosx.dylib b/akka-samples/akka-sample-cluster/sigar/libsigar-universal64-macosx.dylib new file mode 100644 index 0000000000..0c721fecf3 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-universal64-macosx.dylib differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-x86-freebsd-5.so b/akka-samples/akka-sample-cluster/sigar/libsigar-x86-freebsd-5.so new file mode 100644 index 0000000000..8c50c6117a Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-x86-freebsd-5.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-x86-freebsd-6.so b/akka-samples/akka-sample-cluster/sigar/libsigar-x86-freebsd-6.so new file mode 100644 index 0000000000..f0800274a6 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-x86-freebsd-6.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-x86-linux.so b/akka-samples/akka-sample-cluster/sigar/libsigar-x86-linux.so new file mode 100644 index 0000000000..a0b64eddb0 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-x86-linux.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/libsigar-x86-solaris.so b/akka-samples/akka-sample-cluster/sigar/libsigar-x86-solaris.so new file mode 100644 index 0000000000..c6452e5655 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/libsigar-x86-solaris.so differ diff --git a/akka-samples/akka-sample-cluster/sigar/sigar-amd64-winnt.dll b/akka-samples/akka-sample-cluster/sigar/sigar-amd64-winnt.dll new file mode 100644 index 0000000000..1ec8a0353e Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/sigar-amd64-winnt.dll differ diff --git a/akka-samples/akka-sample-cluster/sigar/sigar-x86-winnt.dll b/akka-samples/akka-sample-cluster/sigar/sigar-x86-winnt.dll new file mode 100644 index 0000000000..6afdc0166c Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/sigar-x86-winnt.dll differ diff --git a/akka-samples/akka-sample-cluster/sigar/sigar-x86-winnt.lib b/akka-samples/akka-sample-cluster/sigar/sigar-x86-winnt.lib new file mode 100644 index 0000000000..04924a1fc1 Binary files /dev/null and b/akka-samples/akka-sample-cluster/sigar/sigar-x86-winnt.lib differ diff --git a/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackend.java b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackend.java new file mode 100644 index 0000000000..b1f813f684 --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackend.java @@ -0,0 +1,49 @@ +package sample.cluster.factorial.japi; + +//#imports +import java.math.BigInteger; +import java.util.concurrent.Callable; +import scala.concurrent.Future; +import akka.actor.UntypedActor; +import akka.dispatch.Mapper; +import static akka.dispatch.Futures.future; +import static akka.pattern.Patterns.pipe; +//#imports + +//#backend +public class FactorialBackend extends UntypedActor { + + @Override + public void onReceive(Object message) { + if (message instanceof Integer) { + final Integer n = (Integer) message; + Future f = future(new Callable() { + public BigInteger call() { + return factorial(n); + } + }, getContext().dispatcher()); + + Future result = f.map( + new Mapper() { + public FactorialResult apply(BigInteger factorial) { + return new FactorialResult(n, factorial); + } + }, getContext().dispatcher()); + + pipe(result, getContext().dispatcher()).to(getSender()); + + } else { + unhandled(message); + } + } + + BigInteger factorial(int n) { + BigInteger acc = BigInteger.ONE; + for (int i = 1; i <= n; ++i) { + acc = acc.multiply(BigInteger.valueOf(i)); + } + return acc; + } +} +//#backend + diff --git a/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackendMain.java b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackendMain.java new file mode 100644 index 0000000000..4bf907748d --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackendMain.java @@ -0,0 +1,22 @@ +package sample.cluster.factorial.japi; + +import akka.actor.ActorSystem; +import akka.actor.Props; + +public class FactorialBackendMain { + + public static void main(String[] args) throws Exception { + // Override the configuration of the port + // when specified as program argument + if (args.length > 0) + System.setProperty("akka.remote.netty.port", args[0]); + + ActorSystem system = ActorSystem.create("ClusterSystem"); + + system.actorOf(new Props(FactorialBackend.class), "factorialBackend"); + + system.actorOf(new Props(MetricsListener.class), "metricsListener"); + + } + +} diff --git a/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontend.java b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontend.java new file mode 100644 index 0000000000..63656a43ad --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontend.java @@ -0,0 +1,72 @@ +package sample.cluster.factorial.japi; + +import akka.actor.UntypedActor; +import akka.actor.ActorRef; +import akka.actor.Props; +import akka.event.Logging; +import akka.event.LoggingAdapter; +import akka.routing.FromConfig; +import akka.cluster.routing.AdaptiveLoadBalancingRouter; +import akka.cluster.routing.ClusterRouterConfig; +import akka.cluster.routing.ClusterRouterSettings; +import akka.cluster.routing.HeapMetricsSelector; +import akka.cluster.routing.SystemLoadAverageMetricsSelector; + +//#frontend +public class FactorialFrontend extends UntypedActor { + + LoggingAdapter log = Logging.getLogger(getContext().system(), this); + + ActorRef backend = getContext().actorOf( + new Props(FactorialBackend.class).withRouter(FromConfig.getInstance()), + "factorialBackendRouter"); + + @Override + public void onReceive(Object message) { + if (message instanceof Integer) { + Integer n = (Integer) message; + backend.tell(n, getSelf()); + + } else if (message instanceof FactorialResult) { + FactorialResult result = (FactorialResult) message; + log.info("{}! = {}", result.n, result.factorial); + + } else { + unhandled(message); + } + } + +} +//#frontend + + +//not used, only for documentation +abstract class FactorialFrontend2 extends UntypedActor { + //#router-lookup-in-code + int totalInstances = 100; + String routeesPath = "/user/statsWorker"; + boolean allowLocalRoutees = true; + ActorRef backend = getContext().actorOf( + new Props(FactorialBackend.class).withRouter(new ClusterRouterConfig( + new AdaptiveLoadBalancingRouter(HeapMetricsSelector.getInstance(), 0), + new ClusterRouterSettings( + totalInstances, routeesPath, allowLocalRoutees))), + "factorialBackendRouter2"); + //#router-lookup-in-code +} + +//not used, only for documentation +abstract class StatsService3 extends UntypedActor { + //#router-deploy-in-code + int totalInstances = 100; + int maxInstancesPerNode = 3; + boolean allowLocalRoutees = false; + ActorRef backend = getContext().actorOf( + new Props(FactorialBackend.class).withRouter(new ClusterRouterConfig( + new AdaptiveLoadBalancingRouter( + SystemLoadAverageMetricsSelector.getInstance(), 0), + new ClusterRouterSettings( + totalInstances, maxInstancesPerNode, allowLocalRoutees))), + "factorialBackendRouter3"); + //#router-deploy-in-code +} \ No newline at end of file diff --git a/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontendMain.java b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontendMain.java new file mode 100644 index 0000000000..31d0ff5f12 --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontendMain.java @@ -0,0 +1,38 @@ +package sample.cluster.factorial.japi; + +import java.util.concurrent.TimeUnit; + +import sample.cluster.transformation.japi.TransformationMessages.TransformationJob; +import scala.concurrent.ExecutionContext; +import scala.concurrent.duration.Duration; +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.Props; +import akka.dispatch.OnSuccess; +import akka.util.Timeout; +import static akka.pattern.Patterns.ask; + +public class FactorialFrontendMain { + + public static void main(String[] args) throws Exception { + int upToN = (args.length == 0 ? 200 : Integer.valueOf(args[0])); + + ActorSystem system = ActorSystem.create("ClusterSystem"); + + ActorRef frontend = system.actorOf(new Props( + FactorialFrontend.class), "factorialFrontend"); + + system.log().info("Starting up"); + // wait to let cluster converge and gather metrics + Thread.sleep(10000); + + system.log().info("Starting many factorials up to [{}]", upToN); + for (int i = 0; i < 1000; i++) { + for (int n = 1; n <= upToN; n++) { + frontend.tell(n, null); + } + } + + } + +} diff --git a/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialResult.java b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialResult.java new file mode 100644 index 0000000000..0cb74b6b54 --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialResult.java @@ -0,0 +1,14 @@ +package sample.cluster.factorial.japi; + +import java.math.BigInteger; +import java.io.Serializable; + +public class FactorialResult implements Serializable { + public final int n; + public final BigInteger factorial; + + FactorialResult(int n, BigInteger factorial) { + this.n = n; + this.factorial = factorial; + } +} \ No newline at end of file diff --git a/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/MetricsListener.java b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/MetricsListener.java new file mode 100644 index 0000000000..8931c3bed2 --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/MetricsListener.java @@ -0,0 +1,60 @@ +package sample.cluster.factorial.japi; + +//#metrics-listener +import akka.actor.UntypedActor; +import akka.cluster.Cluster; +import akka.cluster.ClusterEvent.ClusterMetricsChanged; +import akka.cluster.ClusterEvent.CurrentClusterState; +import akka.cluster.NodeMetrics; +import akka.cluster.StandardMetrics; +import akka.cluster.StandardMetrics.HeapMemory; +import akka.cluster.StandardMetrics.Cpu; +import akka.event.Logging; +import akka.event.LoggingAdapter; + +public class MetricsListener extends UntypedActor { + LoggingAdapter log = Logging.getLogger(getContext().system(), this); + + Cluster cluster = Cluster.get(getContext().system()); + + //subscribe to ClusterMetricsChanged + @Override + public void preStart() { + cluster.subscribe(getSelf(), ClusterMetricsChanged.class); + } + + //re-subscribe when restart + @Override + public void postStop() { + cluster.unsubscribe(getSelf()); + } + + + @Override + public void onReceive(Object message) { + if (message instanceof ClusterMetricsChanged) { + ClusterMetricsChanged clusterMetrics = (ClusterMetricsChanged) message; + for (NodeMetrics nodeMetrics : clusterMetrics.getNodeMetrics()) { + if (nodeMetrics.address().equals(cluster.selfAddress())) { + HeapMemory heap = StandardMetrics.extractHeapMemory(nodeMetrics); + if (heap != null) { + log.info("Used heap: {} MB", ((double) heap.used()) / 1024 / 1024); + } + Cpu cpu = StandardMetrics.extractCpu(nodeMetrics); + if (cpu != null && cpu.systemLoadAverage().isDefined()) { + log.info("Load: {} ({} processors)", cpu.systemLoadAverage().get(), + cpu.processors()); + } + } + } + + } else if (message instanceof CurrentClusterState) { + // ignore + + } else { + unhandled(message); + } + } + +} +//#metrics-listener \ No newline at end of file diff --git a/akka-samples/akka-sample-cluster/src/main/resources/application.conf b/akka-samples/akka-sample-cluster/src/main/resources/application.conf index 62554a65cf..09f6cd446d 100644 --- a/akka-samples/akka-sample-cluster/src/main/resources/application.conf +++ b/akka-samples/akka-sample-cluster/src/main/resources/application.conf @@ -1,3 +1,4 @@ +# //#cluster akka { actor { provider = "akka.cluster.ClusterActorRefProvider" @@ -20,4 +21,23 @@ akka { auto-down = on } -} \ No newline at end of file +} +# //#cluster + +# //#adaptive-router +akka.actor.deployment { + /factorialFrontend/factorialBackendRouter = { + router = adaptive + # metrics-selector = heap + # metrics-selector = load + # metrics-selector = cpu + metrics-selector = mix + nr-of-instances = 100 + cluster { + enabled = on + routees-path = "/user/factorialBackend" + allow-local-routees = off + } + } +} +# //#adaptive-router \ No newline at end of file diff --git a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala new file mode 100644 index 0000000000..6e55e03656 --- /dev/null +++ b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala @@ -0,0 +1,151 @@ +package sample.cluster.factorial + +//#imports +import scala.annotation.tailrec +import scala.concurrent.Future +import akka.actor.Actor +import akka.actor.ActorLogging +import akka.actor.ActorRef +import akka.actor.ActorSystem +import akka.actor.Props +import akka.pattern.pipe +import akka.routing.FromConfig + +//#imports + +object FactorialFrontend { + def main(args: Array[String]): Unit = { + val upToN = if (args.isEmpty) 200 else args(0).toInt + + val system = ActorSystem("ClusterSystem") + val frontend = system.actorOf(Props[FactorialFrontend], name = "factorialFrontend") + + system.log.info("Starting up") + // wait to let cluster converge and gather metrics + Thread.sleep(10000) + + system.log.info("Starting many factorials up to [{}]", upToN) + for (_ ← 1 to 1000; n ← 1 to upToN) { + frontend ! n + } + } +} + +//#frontend +class FactorialFrontend extends Actor with ActorLogging { + + val backend = context.actorOf(Props[FactorialBackend].withRouter(FromConfig), + name = "factorialBackendRouter") + + def receive = { + case n: Int ⇒ backend ! n + case (n: Int, factorial: BigInt) ⇒ + log.info("{}! = {}", n, factorial) + } +} +//#frontend + +object FactorialBackend { + def main(args: Array[String]): Unit = { + // Override the configuration of the port + // when specified as program argument + if (args.nonEmpty) System.setProperty("akka.remote.netty.port", args(0)) + + val system = ActorSystem("ClusterSystem") + system.actorOf(Props[FactorialBackend], name = "factorialBackend") + + system.actorOf(Props[MetricsListener], name = "metricsListener") + } +} + +//#backend +class FactorialBackend extends Actor with ActorLogging { + + import context.dispatcher + + def receive = { + case (n: Int) ⇒ + Future(factorial(n)) map { result ⇒ (n, result) } pipeTo sender + } + + def factorial(n: Int): BigInt = { + @tailrec def factorialAcc(acc: BigInt, n: Int): BigInt = { + if (n <= 1) acc + else factorialAcc(acc * n, n - 1) + } + factorialAcc(BigInt(1), n) + } + +} +//#backend + +//#metrics-listener +import akka.cluster.Cluster +import akka.cluster.ClusterEvent.ClusterMetricsChanged +import akka.cluster.ClusterEvent.CurrentClusterState +import akka.cluster.StandardMetrics.HeapMemory +import akka.cluster.StandardMetrics.Cpu + +class MetricsListener extends Actor with ActorLogging { + val selfAddress = Cluster(context.system).selfAddress + + // subscribe to ClusterMetricsChanged + // re-subscribe when restart + override def preStart(): Unit = + Cluster(context.system).subscribe(self, classOf[ClusterMetricsChanged]) + override def postStop(): Unit = + Cluster(context.system).unsubscribe(self) + + def receive = { + case ClusterMetricsChanged(nodeMetrics) ⇒ + nodeMetrics.filter(_.address == selfAddress) foreach { n ⇒ + n match { + case HeapMemory(address, timestamp, used, committed, max) ⇒ + log.info("Used heap: {} MB", used.doubleValue / 1024 / 1024) + case _ ⇒ // no heap info + } + n match { + case Cpu(address, timestamp, Some(systemLoadAverage), cpuCombined, processors) ⇒ + log.info("Load: {} ({} processors)", systemLoadAverage, processors) + case _ ⇒ // no cpu info + } + } + case state: CurrentClusterState ⇒ // ignore + } +} + +//#metrics-listener + +// not used, only for documentation +abstract class FactorialFrontend2 extends Actor { + //#router-lookup-in-code + import akka.cluster.routing.ClusterRouterConfig + import akka.cluster.routing.ClusterRouterSettings + import akka.cluster.routing.AdaptiveLoadBalancingRouter + import akka.cluster.routing.HeapMetricsSelector + + val backend = context.actorOf(Props[FactorialBackend].withRouter( + ClusterRouterConfig(AdaptiveLoadBalancingRouter(HeapMetricsSelector), + ClusterRouterSettings( + totalInstances = 100, routeesPath = "/user/statsWorker", + allowLocalRoutees = true))), + name = "factorialBackendRouter2") + //#router-lookup-in-code +} + +// not used, only for documentation +abstract class FactorialFrontend3 extends Actor { + //#router-deploy-in-code + import akka.cluster.routing.ClusterRouterConfig + import akka.cluster.routing.ClusterRouterSettings + import akka.cluster.routing.AdaptiveLoadBalancingRouter + import akka.cluster.routing.SystemLoadAverageMetricsSelector + + val backend = context.actorOf(Props[FactorialBackend].withRouter( + ClusterRouterConfig(AdaptiveLoadBalancingRouter( + SystemLoadAverageMetricsSelector), ClusterRouterSettings( + totalInstances = 100, maxInstancesPerNode = 3, + allowLocalRoutees = false))), + name = "factorialBackendRouter3") + //#router-deploy-in-code +} \ No newline at end of file diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala index ad678d377f..8df3a7fafb 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala @@ -33,6 +33,8 @@ object StatsSampleSingleMasterSpecConfig extends MultiNodeConfig { akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-join = off + # don't use sigar for tests, native lib not in path + akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector #//#router-deploy-config akka.actor.deployment { /statsFacade/statsService/workerRouter { diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala index 64c2e2994d..aff0d66f68 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala @@ -27,6 +27,8 @@ object StatsSampleSpecConfig extends MultiNodeConfig { akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-join = off + # don't use sigar for tests, native lib not in path + akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector #//#router-lookup-config akka.actor.deployment { /statsService/workerRouter { diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleJapiSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleJapiSpec.scala index 0712305b4b..0076807f30 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleJapiSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleJapiSpec.scala @@ -31,6 +31,8 @@ object StatsSampleJapiSpecConfig extends MultiNodeConfig { akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-join = off + # don't use sigar for tests, native lib not in path + akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector akka.actor.deployment { /statsService/workerRouter { router = consistent-hashing diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleSingleMasterJapiSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleSingleMasterJapiSpec.scala index 7a5275a7f8..6207a67fd6 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleSingleMasterJapiSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleSingleMasterJapiSpec.scala @@ -34,6 +34,8 @@ object StatsSampleSingleMasterJapiSpecConfig extends MultiNodeConfig { akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-join = off + # don't use sigar for tests, native lib not in path + akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector akka.actor.deployment { /statsFacade/statsService/workerRouter { router = consistent-hashing diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala index 131997730f..22bfdd6865 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala @@ -29,6 +29,8 @@ object TransformationSampleSpecConfig extends MultiNodeConfig { akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-join = off + # don't use sigar for tests, native lib not in path + akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector """)) } diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/japi/TransformationSampleJapiSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/japi/TransformationSampleJapiSpec.scala index d7d328f81d..40a541ff72 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/japi/TransformationSampleJapiSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/japi/TransformationSampleJapiSpec.scala @@ -30,6 +30,8 @@ object TransformationSampleJapiSpecConfig extends MultiNodeConfig { akka.actor.provider = "akka.cluster.ClusterActorRefProvider" akka.remote.log-remote-lifecycle-events = off akka.cluster.auto-join = off + # don't use sigar for tests, native lib not in path + akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector """)) } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 8c1a4a6335..fead85bb32 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -327,6 +327,10 @@ object AkkaBuild extends Build { dependencies = Seq(cluster, remoteTests % "test", testkit % "test"), settings = sampleSettings ++ multiJvmSettings ++ experimentalSettings ++ Seq( libraryDependencies ++= Dependencies.clusterSample, + javaOptions in run ++= Seq( + "-Djava.library.path=./sigar", + "-Xms128m", "-Xmx1024m"), + Keys.fork in run := true, // disable parallel tests parallelExecution in Test := false, extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => @@ -673,6 +677,9 @@ object Dependencies { // Camel Sample val camelJetty = "org.apache.camel" % "camel-jetty" % camelCore.revision // ApacheV2 + // Cluster Sample + val sigar = "org.hyperic" % "sigar" % "1.6.4" // ApacheV2 + // Test object Test { @@ -729,7 +736,7 @@ object Dependencies { val zeroMQ = Seq(protobuf, zeroMQClient, Test.scalatest, Test.junit) - val clusterSample = Seq(Test.scalatest) + val clusterSample = Seq(Test.scalatest, sigar) val contrib = Seq(Test.junitIntf)