+ akka-cluster-metrics: new akka module

* new akka module split from akka-cluster
* provide sigar provisioning
* fix ewma usage
* resolve #16121
* see #16354
This commit is contained in:
Andrei Pozolotin 2014-12-12 11:49:32 -06:00
parent baca3644e2
commit 7b9f77a073
121 changed files with 10462 additions and 215 deletions

3
.gitignore vendored
View file

@ -74,3 +74,6 @@ tm.out
worker*.log
*-shim.sbt
test-output
# Default sigar library extract location.
native/

View file

@ -0,0 +1,29 @@
import akka.{ AkkaBuild, Dependencies, Formatting, OSGi, MultiNode, Unidoc, SigarLoader }
import com.typesafe.sbt.SbtMultiJvm.MultiJvmKeys._
import com.typesafe.tools.mima.plugin.MimaKeys
AkkaBuild.defaultSettings
Formatting.formatSettings
Unidoc.scaladocSettings
Unidoc.javadocSettings
MultiNode.multiJvmSettings
SigarLoader.sigarSettings
OSGi.clusterMetrics
libraryDependencies ++= Dependencies.clusterMetrics
//MimaKeys.previousArtifact := akkaPreviousArtifact("akka-cluster-metrics").value
parallelExecution in Test := false
extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src =>
(name: String) => (src ** (name + ".conf")).get.headOption.map("-Dakka.config=" + _.absolutePath).toSeq
}
scalatestOptions in MultiJvm := MultiNode.defaultMultiJvmScalatestOptions.value

View file

@ -0,0 +1,73 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
option java_package = "akka.cluster.metrics.protobuf.msg";
option optimize_for = SPEED;
/****************************************
* Metrics Gossip Messages
****************************************/
/**
* Metrics Gossip Envelope
*/
message MetricsGossipEnvelope {
required Address from = 1;
required MetricsGossip gossip = 2;
required bool reply = 3;
}
/**
* Metrics Gossip
*/
message MetricsGossip {
repeated Address allAddresses = 1;
repeated string allMetricNames = 2;
repeated NodeMetrics nodeMetrics = 3;
}
/**
* Node Metrics
*/
message NodeMetrics {
enum NumberType {
Serialized = 0;
Double = 1;
Float = 2;
Integer = 3;
Long = 4;
}
message Number {
required NumberType type = 1;
optional uint32 value32 = 2;
optional uint64 value64 = 3;
optional bytes serialized = 4;
}
message EWMA {
required double value = 1;
required double alpha = 2;
}
message Metric {
required int32 nameIndex = 1;
required Number number = 2;
optional EWMA ewma = 3;
}
required int32 addressIndex = 1;
required int64 timestamp = 2;
repeated Metric metrics = 3;
}
/****************************************
* Common Datatypes and Messages
****************************************/
/**
* Defines a remote address.
*/
message Address {
required string system = 1;
required string hostname = 2;
required uint32 port = 3;
optional string protocol = 4;
}

View file

@ -0,0 +1,105 @@
##############################################
# Akka Cluster Metrics Reference Config File #
##############################################
# This is the reference config file that contains all the default settings.
# Make your edits in your application.conf in order to override these settings.
# Sigar provisioning:
#
# User can provision sigar classes and native library in one of the following ways:
#
# 1) Use https://github.com/kamon-io/sigar-loader Kamon sigar-loader as a project dependency for the user project.
# Metrics extension will extract and load sigar library on demand with help of Kamon sigar provisioner.
#
# 2) Use https://github.com/kamon-io/sigar-loader Kamon sigar-loader as java agent: `java -javaagent:/path/to/sigar-loader.jar`
# Kamon sigar loader agent will extract and load sigar library during JVM start.
#
# 3) Place `sigar.jar` on the `classpath` and sigar native library for the o/s on the `java.library.path`
# User is required to manage both project dependency and library deployment manually.
# Cluster metrics extension.
# Provides periodic statistics collection and publication throughout the cluster.
akka.cluster.metrics {
# Full path of dispatcher configuration key.
# Use "" for default key `akka.actor.default-dispatcher`.
dispatcher = ""
# How long should any actor wait before starting the periodic tasks.
periodic-tasks-initial-delay = 1s
# Sigar native library extract location.
# Use per-application-instance scoped location, such as program working directory.
native-library-extract-folder = ${user.dir}"/native"
# Unique serialization identifier. Must not conflict with any other in an akka system.
serializer-identifier = 10
# Metrics supervisor actor.
supervisor {
# Actor name. Example name space: /system/cluster-metrics
name = "cluster-metrics"
# Supervision strategy.
strategy {
#
# FQCN of class providing `akka.actor.SupervisorStrategy`.
# Must have a constructor with signature `<init>(com.typesafe.config.Config)`.
# Default metrics strategy provider is a configurable extension of `OneForOneStrategy`.
provider = "akka.cluster.metrics.ClusterMetricsStrategy"
#
# Configuration of the default strategy provider.
# Replace with custom settings when overriding the provider.
configuration = {
# Log restart attempts.
loggingEnabled = true
# Child actor restart-on-failure window.
withinTimeRange = 3s
# Maximum number of restart attempts before child actor is stopped.
maxNrOfRetries = 3
}
}
}
# Metrics collector actor.
collector {
# Enable or disable metrics collector for load-balancing nodes.
# Metrics collection can also be controlled at runtime by sending control messages
# to /system/cluster-metrics actor: `akka.cluster.metrics.{CollectionStartMessage,CollectionStopMessage}`
enabled = on
# FQCN of the metrics collector implementation.
# It must implement `akka.cluster.metrics.MetricsCollector` and
# have public constructor with akka.actor.ActorSystem parameter.
# Will try to load in the following order of priority:
# 1) configured custom collector 2) internal `SigarMetricsCollector` 3) internal `JmxMetricsCollector`
provider = ""
# Try all 3 available collector providers, or else fail on the configured custom collector provider.
fallback = true
# How often metrics are sampled on a node.
# Shorter interval will collect the metrics more often.
# Also controls frequency of the metrics publication to the node system event bus.
sample-interval = 3s
# How often a node publishes metrics information to the other nodes in the cluster.
# Shorter interval will publish the metrics gossip more often.
gossip-interval = 3s
# How quickly the exponential weighting of past data is decayed compared to
# new data. Set lower to increase the bias toward newer values.
# The relevance of each data sample is halved for every passing half-life
# duration, i.e. after 4 times the half-life, a data samples relevance is
# reduced to 6% of its original relevance. The initial relevance of a data
# sample is given by 1 0.5 ^ (collect-interval / half-life).
# See http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average
moving-average-half-life = 12s
}
}
# Cluster metrics extension serializers and routers.
akka.actor {
# Protobuf serializer for remote cluster metrics messages.
serializers {
akka-cluster-metrics = "akka.cluster.metrics.protobuf.MessageSerializer"
}
# Interface binding for remote cluster metrics messages.
serialization-bindings {
"akka.cluster.metrics.ClusterMetricsMessage" = akka-cluster-metrics
}
# Provide routing of messages based on cluster metrics.
router.type-mapping {
cluster-metrics-adaptive-pool = "akka.cluster.metrics.AdaptiveLoadBalancingPool"
cluster-metrics-adaptive-group = "akka.cluster.metrics.AdaptiveLoadBalancingGroup"
}
}

View file

@ -0,0 +1,257 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.Props
import akka.actor.Address
import akka.cluster.InternalClusterAction
import akka.cluster.ClusterEvent
import akka.cluster.Member
import akka.cluster.Cluster
import scala.collection.immutable
import akka.cluster.MemberStatus
import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.actor.Terminated
/**
* Runtime collection management commands.
*/
sealed abstract class CollectionControlMessage extends Serializable
/**
* Command for [[ClusterMetricsSupervisor]] to start metrics collection.
*/
@SerialVersionUID(1L)
case object CollectionStartMessage extends CollectionControlMessage {
/** Java API */
def getInstance = CollectionStartMessage
}
/**
* Command for [[ClusterMetricsSupervisor]] to stop metrics collection.
*/
@SerialVersionUID(1L)
case object CollectionStopMessage extends CollectionControlMessage {
/** Java API */
def getInstance = CollectionStopMessage
}
/**
* INTERNAL API.
*
* Actor providing customizable metrics collection supervision.
*/
private[metrics] class ClusterMetricsSupervisor extends Actor with ActorLogging {
import ClusterMetricsExtension._
val metrics = ClusterMetricsExtension(context.system)
import metrics.settings._
import context._
override val supervisorStrategy = metrics.strategy
var collectorInstance = 0
def collectorName = s"collector-${collectorInstance}"
override def preStart() = {
if (CollectorEnabled) {
self ! CollectionStartMessage
} else {
log.warning(s"Metrics collection is disabled in configuration. Use subtypes of ${classOf[CollectionControlMessage].getName} to manage collection at runtime.")
}
}
override def receive = {
case CollectionStartMessage
children.foreach(stop)
collectorInstance += 1
actorOf(Props(classOf[ClusterMetricsCollector]), collectorName)
log.debug(s"Collection started.")
case CollectionStopMessage
children.foreach(stop)
log.debug(s"Collection stopped.")
}
}
/**
* Local cluster metrics extension events.
*
* Published to local event bus subscribers by [[ClusterMetricsCollector]].
*/
trait ClusterMetricsEvent
/**
* Current snapshot of cluster node metrics.
*/
final case class ClusterMetricsChanged(nodeMetrics: Set[NodeMetrics]) extends ClusterMetricsEvent {
/** Java API */
def getNodeMetrics: java.lang.Iterable[NodeMetrics] =
scala.collection.JavaConverters.asJavaIterableConverter(nodeMetrics).asJava
}
/**
* INTERNAL API.
*
* Remote cluster metrics extension messages.
*
* Published to cluster members with metrics extension.
*/
private[metrics] trait ClusterMetricsMessage extends Serializable
/**
* INTERNAL API.
*
* Envelope adding a sender address to the cluster metrics gossip.
*/
@SerialVersionUID(1L)
private[metrics] final case class MetricsGossipEnvelope(from: Address, gossip: MetricsGossip, reply: Boolean) extends ClusterMetricsMessage
/**
* INTERNAL API.
*
* Actor responsible for periodic data sampling in the node and publication to the cluster.
*/
private[metrics] class ClusterMetricsCollector extends Actor with ActorLogging {
import InternalClusterAction._
// TODO collapse to ClusterEvent._ after akka-cluster metrics is gone
import ClusterEvent.MemberEvent
import ClusterEvent.MemberUp
import ClusterEvent.MemberRemoved
import ClusterEvent.MemberExited
import ClusterEvent.ReachabilityEvent
import ClusterEvent.ReachableMember
import ClusterEvent.UnreachableMember
import ClusterEvent.CurrentClusterState
import Member.addressOrdering
import context.dispatcher
val cluster = Cluster(context.system)
import cluster.{ selfAddress, scheduler, settings }
import cluster.InfoLogger._
val metrics = ClusterMetricsExtension(context.system)
import metrics.settings._
/**
* The node ring gossipped that contains only members that are Up.
*/
var nodes: immutable.SortedSet[Address] = immutable.SortedSet.empty
/**
* The latest metric values with their statistical data.
*/
var latestGossip: MetricsGossip = MetricsGossip.empty
/**
* The metrics collector that samples data on the node.
*/
val collector: MetricsCollector = MetricsCollector(context.system)
/**
* Start periodic gossip to random nodes in cluster
*/
val gossipTask = scheduler.schedule(PeriodicTasksInitialDelay max CollectorGossipInterval,
CollectorGossipInterval, self, GossipTick)
/**
* Start periodic metrics collection
*/
val sampleTask = scheduler.schedule(PeriodicTasksInitialDelay max CollectorSampleInterval,
CollectorSampleInterval, self, MetricsTick)
override def preStart(): Unit = {
cluster.subscribe(self, classOf[MemberEvent], classOf[ReachabilityEvent])
logInfo("Metrics collection has started successfully")
}
def receive = {
case GossipTick gossip()
case MetricsTick sample()
case msg: MetricsGossipEnvelope receiveGossip(msg)
case state: CurrentClusterState receiveState(state)
case MemberUp(m) addMember(m)
case MemberRemoved(m, _) removeMember(m)
case MemberExited(m) removeMember(m)
case UnreachableMember(m) removeMember(m)
case ReachableMember(m) if (m.status == MemberStatus.Up) addMember(m)
case _: MemberEvent // not interested in other types of MemberEvent
}
override def postStop: Unit = {
cluster unsubscribe self
gossipTask.cancel()
sampleTask.cancel()
collector.close()
}
/**
* Adds a member to the node ring.
*/
def addMember(member: Member): Unit = nodes += member.address
/**
* Removes a member from the member node ring.
*/
def removeMember(member: Member): Unit = {
nodes -= member.address
latestGossip = latestGossip remove member.address
publish()
}
/**
* Updates the initial node ring for those nodes that are [[akka.cluster.MemberStatus.Up]].
*/
def receiveState(state: CurrentClusterState): Unit =
nodes = (state.members -- state.unreachable) collect { case m if m.status == MemberStatus.Up m.address }
/**
* Samples the latest metrics for the node, updates metrics statistics in
* [[MetricsGossip]], and publishes the change to the event bus.
*
* @see [[MetricsCollector]]
*/
def sample(): Unit = {
latestGossip :+= collector.sample()
publish()
}
/**
* Receives changes from peer nodes, merges remote with local gossip nodes, then publishes
* changes to the event stream for load balancing router consumption, and gossip back.
*/
def receiveGossip(envelope: MetricsGossipEnvelope): Unit = {
// remote node might not have same view of member nodes, this side should only care
// about nodes that are known here, otherwise removed nodes can come back
val otherGossip = envelope.gossip.filter(nodes)
latestGossip = latestGossip merge otherGossip
// changes will be published in the period collect task
if (!envelope.reply)
replyGossipTo(envelope.from)
}
/**
* Gossip to peer nodes.
*/
def gossip(): Unit = selectRandomNode((nodes - selfAddress).toVector) foreach gossipTo
def gossipTo(address: Address): Unit =
sendGossip(address, MetricsGossipEnvelope(selfAddress, latestGossip, reply = false))
def replyGossipTo(address: Address): Unit =
sendGossip(address, MetricsGossipEnvelope(selfAddress, latestGossip, reply = true))
def sendGossip(address: Address, envelope: MetricsGossipEnvelope): Unit =
context.actorSelection(self.path.toStringWithAddress(address)) ! envelope
def selectRandomNode(addresses: immutable.IndexedSeq[Address]): Option[Address] =
if (addresses.isEmpty) None else Some(addresses(ThreadLocalRandom.current nextInt addresses.size))
/**
* Publishes to the event stream.
*/
def publish(): Unit = context.system.eventStream publish ClusterMetricsChanged(latestGossip.nodes)
}

View file

@ -0,0 +1,87 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import akka.actor.ExtendedActorSystem
import akka.actor.Extension
import akka.actor.SupervisorStrategy
import akka.event.LoggingAdapter
import akka.event.Logging
import com.typesafe.config.Config
import scala.collection.immutable
import akka.actor.Props
import akka.actor.Deploy
import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider
import akka.actor.ActorSystem
import akka.actor.ActorRef
/**
* Cluster metrics extension.
*
* Cluster metrics is primarily for load-balancing of nodes. It controls metrics sampling
* at a regular frequency, prepares highly variable data for further analysis by other entities,
* and publishes the latest cluster metrics data around the node ring and local eventStream
* to assist in determining the need to redirect traffic to the least-loaded nodes.
*
* Metrics sampling is delegated to the [[MetricsCollector]].
*
* Smoothing of the data for each monitored process is delegated to the
* [[EWMA]] for exponential weighted moving average.
*/
class ClusterMetricsExtension(system: ExtendedActorSystem) extends Extension {
/**
* Metrics extension configuration.
*/
val settings = ClusterMetricsSettings(system.settings.config)
import settings._
/**
* INTERNAL API
*
* Supervision strategy.
*/
private[metrics] val strategy = system.dynamicAccess.createInstanceFor[SupervisorStrategy](
SupervisorStrategyProvider, immutable.Seq(classOf[Config] -> SupervisorStrategyConfiguration))
.getOrElse {
val log: LoggingAdapter = Logging(system, getClass.getName)
log.error(s"Configured strategy provider ${SupervisorStrategyProvider} failed to load, using default ${classOf[ClusterMetricsStrategy].getName}.")
new ClusterMetricsStrategy(SupervisorStrategyConfiguration)
}
/**
* Supervisor actor.
* Accepts subtypes of [[CollectionControlMessage]]s to manage metrics collection at runtime.
*/
val supervisor = system.systemActorOf(
Props(classOf[ClusterMetricsSupervisor]).withDispatcher(MetricsDispatcher).withDeploy(Deploy.local),
SupervisorName)
/**
* Subscribe user metrics listener actor unto [[ClusterMetricsEvent]]
* events published by extension on the system event bus.
*/
def subscribe(metricsListener: ActorRef): Unit = {
system.eventStream.subscribe(metricsListener, classOf[ClusterMetricsEvent])
}
/**
* Unsubscribe user metrics listener actor from [[ClusterMetricsEvent]]
* events published by extension on the system event bus.
*/
def unsubscribe(metricsListenter: ActorRef): Unit = {
system.eventStream.unsubscribe(metricsListenter, classOf[ClusterMetricsEvent])
}
}
/**
* Cluster metrics extension provider.
*/
object ClusterMetricsExtension extends ExtensionId[ClusterMetricsExtension] with ExtensionIdProvider {
override def lookup = ClusterMetricsExtension
override def get(system: ActorSystem): ClusterMetricsExtension = super.get(system)
override def createExtension(system: ExtendedActorSystem): ClusterMetricsExtension = new ClusterMetricsExtension(system)
}

View file

@ -0,0 +1,525 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import java.util.Arrays
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.forkjoin.ThreadLocalRandom
import com.typesafe.config.Config
import akka.actor.Actor
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.DynamicAccess
import akka.actor.NoSerializationVerificationNeeded
import akka.actor.Props
import akka.actor.SupervisorStrategy
import akka.cluster.Cluster
import akka.cluster.ClusterEvent.CurrentClusterState
import akka.dispatch.Dispatchers
import akka.japi.Util.immutableSeq
import akka.routing._
/**
* Load balancing of messages to cluster nodes based on cluster metric data.
*
* It uses random selection of routees based on probabilities derived from
* the remaining capacity of corresponding node.
*
* @param system the actor system hosting this router
*
* @param metricsSelector decides what probability to use for selecting a routee, based
* on remaining capacity as indicated by the node metrics
*/
final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsSelector: MetricsSelector = MixMetricsSelector)
extends RoutingLogic with NoSerializationVerificationNeeded {
private val cluster = Cluster(system)
// 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 sender()s.
private val weightedRouteesRef =
new AtomicReference[(immutable.IndexedSeq[Routee], Set[NodeMetrics], Option[WeightedRoutees])](
(Vector.empty, Set.empty, None))
@tailrec final def metricsChanged(event: ClusterMetricsChanged): Unit = {
val oldValue = weightedRouteesRef.get
val (routees, _, _) = oldValue
val weightedRoutees = Some(new WeightedRoutees(routees, cluster.selfAddress,
metricsSelector.weights(event.nodeMetrics)))
// retry when CAS failure
if (!weightedRouteesRef.compareAndSet(oldValue, (routees, event.nodeMetrics, weightedRoutees)))
metricsChanged(event)
}
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
if (routees.isEmpty) NoRoutee
else {
def updateWeightedRoutees(): Option[WeightedRoutees] = {
val oldValue = weightedRouteesRef.get
val (oldRoutees, oldMetrics, oldWeightedRoutees) = oldValue
if (routees ne oldRoutees) {
val weightedRoutees = Some(new WeightedRoutees(routees, cluster.selfAddress,
metricsSelector.weights(oldMetrics)))
// ignore, don't update, in case of CAS failure
weightedRouteesRef.compareAndSet(oldValue, (routees, oldMetrics, weightedRoutees))
weightedRoutees
} else oldWeightedRoutees
}
updateWeightedRoutees() match {
case Some(weighted)
if (weighted.isEmpty) NoRoutee
else weighted(ThreadLocalRandom.current.nextInt(weighted.total) + 1)
case None
routees(ThreadLocalRandom.current.nextInt(routees.size))
}
}
}
/**
* A router pool that performs load balancing of messages to cluster nodes based on
* cluster metric data.
*
* It uses random selection of routees based on probabilities derived from
* the remaining capacity of corresponding node.
*
* The configuration parameter trumps the constructor arguments. This means that
* if you provide `nrOfInstances` during instantiation they will be ignored if
* the router is defined in the configuration file for the actor being used.
*
* <h1>Supervision Setup</h1>
*
* Any routees that are created by a router will be created as the router's children.
* The router is therefore also the children's supervisor.
*
* The supervision strategy of the router actor can be configured with
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
* a strategy of always escalate. This means that errors are passed up to the
* router's supervisor for handling.
*
* The router's supervisor will treat the error as an error with the router itself.
* Therefore a directive to stop or restart will cause the router itself to stop or
* restart. The router, in turn, will cause its children to stop and restart.
*
* @param metricsSelector decides what probability to use for selecting a routee, based
* on remaining capacity as indicated by the node metrics
*
* @param nrOfInstances initial number of routees in the pool
*
* @param supervisorStrategy strategy for supervising the routees, see 'Supervision Setup'
*
* @param routerDispatcher dispatcher to use for the router head actor, which handles
* supervision, death watch and router management messages
*/
@SerialVersionUID(1L)
final case class AdaptiveLoadBalancingPool(
metricsSelector: MetricsSelector = MixMetricsSelector,
val nrOfInstances: Int = 0,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
extends Pool {
def this(config: Config, dynamicAccess: DynamicAccess) =
this(nrOfInstances = config.getInt("nr-of-instances"),
metricsSelector = MetricsSelector.fromConfig(config, dynamicAccess),
usePoolDispatcher = config.hasPath("pool-dispatcher"))
/**
* Java API
* @param metricsSelector decides what probability to use for selecting a routee, based
* on remaining capacity as indicated by the node metrics
* @param nr initial number of routees in the pool
*/
def this(metricsSelector: MetricsSelector, nr: Int) = this(nrOfInstances = nr)
override def resizer: Option[Resizer] = None
override def nrOfInstances(sys: ActorSystem) = this.nrOfInstances
override def createRouter(system: ActorSystem): Router =
new Router(AdaptiveLoadBalancingRoutingLogic(system, metricsSelector))
override def routingLogicController(routingLogic: RoutingLogic): Option[Props] =
Some(Props(classOf[AdaptiveLoadBalancingMetricsListener],
routingLogic.asInstanceOf[AdaptiveLoadBalancingRoutingLogic]))
/**
* Setting the supervisor strategy to be used for the head Router actor.
*/
def withSupervisorStrategy(strategy: SupervisorStrategy): AdaptiveLoadBalancingPool = copy(supervisorStrategy = strategy)
/**
* Setting the dispatcher to be used for the router head actor, which handles
* supervision, death watch and router management messages.
*/
def withDispatcher(dispatcherId: String): AdaptiveLoadBalancingPool = copy(routerDispatcher = dispatcherId)
/**
* Uses the supervisor strategy of the given Routerconfig
* if this RouterConfig doesn't have one
*/
override def withFallback(other: RouterConfig): RouterConfig =
if (this.supervisorStrategy ne Pool.defaultSupervisorStrategy) this
else other match {
case _: FromConfig | _: NoRouter this // NoRouter is the default, hence neutral
case otherRouter: AdaptiveLoadBalancingPool
if (otherRouter.supervisorStrategy eq Pool.defaultSupervisorStrategy) this
else this.withSupervisorStrategy(otherRouter.supervisorStrategy)
case _ throw new IllegalArgumentException("Expected AdaptiveLoadBalancingPool, got [%s]".format(other))
}
}
/**
* A router group that performs load balancing of messages to cluster nodes based on
* cluster metric data.
*
* It uses random selection of routees based on probabilities derived from
* the remaining capacity of corresponding node.
*
* The configuration parameter trumps the constructor arguments. This means that
* if you provide `paths` during instantiation they will be ignored if
* the router is defined in the configuration file for the actor being used.
*
* @param metricsSelector decides what probability to use for selecting a routee, based
* on remaining capacity as indicated by the node metrics
*
* @param paths string representation of the actor paths of the routees, messages are
* sent with [[akka.actor.ActorSelection]] to these paths
*
* @param routerDispatcher dispatcher to use for the router head actor, which handles
* router management messages
*/
@SerialVersionUID(1L)
final case class AdaptiveLoadBalancingGroup(
metricsSelector: MetricsSelector = MixMetricsSelector,
paths: immutable.Iterable[String] = Nil,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends Group {
def this(config: Config, dynamicAccess: DynamicAccess) =
this(metricsSelector = MetricsSelector.fromConfig(config, dynamicAccess),
paths = immutableSeq(config.getStringList("routees.paths")))
/**
* Java API
* @param metricsSelector decides what probability to use for selecting a routee, based
* on remaining capacity as indicated by the node metrics
* @param routeesPaths string representation of the actor paths of the routees, messages are
* sent with [[akka.actor.ActorSelection]] to these paths
*/
def this(metricsSelector: MetricsSelector,
routeesPaths: java.lang.Iterable[String]) = this(paths = immutableSeq(routeesPaths))
override def createRouter(system: ActorSystem): Router =
new Router(AdaptiveLoadBalancingRoutingLogic(system, metricsSelector))
override def routingLogicController(routingLogic: RoutingLogic): Option[Props] =
Some(Props(classOf[AdaptiveLoadBalancingMetricsListener],
routingLogic.asInstanceOf[AdaptiveLoadBalancingRoutingLogic]))
/**
* Setting the dispatcher to be used for the router head actor, which handles
* router management messages
*/
def withDispatcher(dispatcherId: String): AdaptiveLoadBalancingGroup = copy(routerDispatcher = dispatcherId)
}
/**
* MetricsSelector that uses the heap metrics.
* Low heap capacity => small weight.
*/
@SerialVersionUID(1L)
case object HeapMetricsSelector extends CapacityMetricsSelector {
import akka.cluster.metrics.StandardMetrics.HeapMemory
/**
* 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 time metrics and stolen CPU time metrics.
* In modern Linux kernels: CpuCombined + CpuStolen + CpuIdle = 1.0 or 100%.
* Combined CPU is sum of User + Sys + Nice + Wait times, as percentage.
* Stolen CPU is the amount of CPU taken away from this virtual machine by the hypervisor, as percentage.
*
* Low CPU capacity => small node weight.
*/
@SerialVersionUID(1L)
case object CpuMetricsSelector extends CapacityMetricsSelector {
import akka.cluster.metrics.StandardMetrics.Cpu
/**
* Java API: get the singleton instance
*/
def getInstance = this
// Notes from reading around:
// In modern Linux kernels: CpuCombined + CpuStolen + CpuIdle = 1.0 or 100%. More convoluted for other o/s.
// We could use CpuIdle as the only capacity measure: http://axibase.com/news/ec2-monitoring-the-case-of-stolen-cpu/
// But not all "idle time"s are created equal: https://docs.newrelic.com/docs/servers/new-relic-servers-linux/maintenance/servers-linux-faq
// Example: assume that combined+stolen=70%, idle=30%. Then 50/20/30 system will be more responsive then 20/50/30 system (combined/stolen/idle ratio).
// Current approach: "The more stolen resources there are, the less active the virtual machine needs to be to generate a high load rating."
// TODO read factor from reference.conf
/** How much extra weight to give to the stolen time. */
val factor = 0.3
require(0.0 <= factor, s"factor must be non negative: ${factor}")
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
nodeMetrics.collect {
case Cpu(address, _, _, Some(cpuCombined), Some(cpuStolen), _)
// Arbitrary load rating function which skews in favor of stolen time.
val load = cpuCombined + cpuStolen * (1.0 + factor)
val capacity = if (load >= 1.0) 0.0 else 1.0 - load
(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 {
import akka.cluster.metrics.StandardMetrics.Cpu
/**
* 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
}
}
/**
* Singleton instance of the default MixMetricsSelector, which uses [akka.cluster.routing.HeapMetricsSelector],
* [akka.cluster.routing.CpuMetricsSelector], and [akka.cluster.routing.SystemLoadAverageMetricsSelector]
*/
@SerialVersionUID(1L)
object MixMetricsSelector extends MixMetricsSelectorBase(
Vector(HeapMetricsSelector, CpuMetricsSelector, SystemLoadAverageMetricsSelector)) {
/**
* Java API: get the default singleton instance
*/
def getInstance = this
}
/**
* 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)
final case class MixMetricsSelector(
selectors: immutable.IndexedSeq[CapacityMetricsSelector])
extends MixMetricsSelectorBase(selectors)
/**
* Base class for MetricsSelector that combines other selectors and aggregates their capacity.
*/
@SerialVersionUID(1L)
abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMetricsSelector])
extends CapacityMetricsSelector {
/**
* Java API: construct a mix-selector from a sequence of selectors
*/
def this(selectors: java.lang.Iterable[CapacityMetricsSelector]) = this(immutableSeq(selectors).toVector)
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
val combined: immutable.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)))
}.map {
case (addr, (sum, count)) (addr -> sum / count)
}
}
}
object MetricsSelector {
def fromConfig(config: Config, dynamicAccess: DynamicAccess) =
config.getString("metrics-selector") match {
case "mix" MixMetricsSelector
case "heap" HeapMetricsSelector
case "cpu" CpuMetricsSelector
case "load" SystemLoadAverageMetricsSelector
case fqn
val args = List(classOf[Config] -> config)
dynamicAccess.createInstanceFor[MetricsSelector](fqn, args).recover({
case exception throw new IllegalArgumentException(
(s"Cannot instantiate metrics-selector [$fqn], " +
"make sure it extends [akka.cluster.routing.MetricsSelector] and " +
"has constructor with [com.typesafe.config.Config] parameter"), exception)
}).get
}
}
/**
* 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.
*/
@SerialVersionUID(1L)
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 map { case (addr, c) (addr -> 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[metrics] class WeightedRoutees(routees: immutable.IndexedSeq[Routee], 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(routee: Routee): Address = {
val a = routee match {
case ActorRefRoutee(ref) ref.path.address
case ActorSelectionRoutee(sel) sel.anchor.path.address
}
a match {
case Address(_, _, None, None) selfAddress
case a a
}
}
val buckets = Array.ofDim[Int](routees.size)
val meanWeight = if (weights.isEmpty) 1 else weights.values.sum / weights.size
val w = weights.withDefaultValue(meanWeight) // we dont necessarily have metrics for all addresses
var i = 0
var sum = 0
routees foreach { r
sum += w(fullAddress(r))
buckets(i) = sum
i += 1
}
buckets
}
def isEmpty: Boolean = buckets.length == 0 || buckets(buckets.length - 1) == 0
def total: Int = {
require(!isEmpty, "WeightedRoutees must not be used when empty")
buckets(buckets.length - 1)
}
/**
* Pick the routee matching a value, from 1 to total.
*/
def apply(value: Int): Routee = {
require(1 <= value && value <= total, "value must be between [1 - %s]" format total)
routees(idx(Arrays.binarySearch(buckets, value)))
}
/**
* Converts the result of Arrays.binarySearch into a index in the buckets array
* see documentation of Arrays.binarySearch for what it returns
*/
private 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
}
}
}
/**
* INTERNAL API
* Subscribe to [[ClusterMetricsEvent]]s and update routing logic depending on the events.
*/
private[metrics] class AdaptiveLoadBalancingMetricsListener(routingLogic: AdaptiveLoadBalancingRoutingLogic)
extends Actor {
def extension = ClusterMetricsExtension(context.system)
override def preStart(): Unit = extension.subscribe(self)
override def postStop(): Unit = extension.unsubscribe(self)
def receive = {
case event: ClusterMetricsChanged routingLogic.metricsChanged(event)
case _ // ignore
}
}

View file

@ -0,0 +1,49 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import com.typesafe.config.Config
import akka.dispatch.Dispatchers
import scala.concurrent.duration.FiniteDuration
import akka.util.Helpers.Requiring
import akka.util.Helpers.ConfigOps
import scala.concurrent.duration.Duration
/**
* Metrics extension settings. Documented in: `src/main/resources/reference.conf`.
*/
case class ClusterMetricsSettings(config: Config) {
private val cc = config.getConfig("akka.cluster.metrics")
// Extension.
val MetricsDispatcher: String = cc.getString("dispatcher") match {
case "" Dispatchers.DefaultDispatcherId
case id id
}
val PeriodicTasksInitialDelay: FiniteDuration = cc.getMillisDuration("periodic-tasks-initial-delay")
val NativeLibraryExtractFolder: String = cc.getString("native-library-extract-folder")
val SerializerIdentifier: Int = cc.getInt("serializer-identifier")
// Supervisor.
val SupervisorName: String = cc.getString("supervisor.name")
val SupervisorStrategyProvider: String = cc.getString("supervisor.strategy.provider")
val SupervisorStrategyConfiguration: Config = cc.getConfig("supervisor.strategy.configuration")
// Collector.
val CollectorEnabled: Boolean = cc.getBoolean("collector.enabled")
val CollectorProvider: String = cc.getString("collector.provider")
val CollectorFallback: Boolean = cc.getBoolean("collector.fallback")
val CollectorSampleInterval: FiniteDuration = {
cc.getMillisDuration("collector.sample-interval")
} requiring (_ > Duration.Zero, "collector.sample-interval must be > 0")
val CollectorGossipInterval: FiniteDuration = {
cc.getMillisDuration("collector.gossip-interval")
} requiring (_ > Duration.Zero, "collector.gossip-interval must be > 0")
val CollectorMovingAverageHalfLife: FiniteDuration = {
cc.getMillisDuration("collector.moving-average-half-life")
} requiring (_ > Duration.Zero, "collector.moving-average-half-life must be > 0")
}

View file

@ -0,0 +1,37 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import com.typesafe.config.Config
import akka.actor.OneForOneStrategy
import akka.util.Helpers.ConfigOps
/**
* Default [[ClusterMetricsSupervisor]] strategy:
* A configurable [[OneForOneStrategy]] with restart-on-throwable decider.
*/
class ClusterMetricsStrategy(config: Config) extends OneForOneStrategy(
maxNrOfRetries = config.getInt("maxNrOfRetries"),
withinTimeRange = config.getMillisDuration("withinTimeRange"),
loggingEnabled = config.getBoolean("loggingEnabled"))(ClusterMetricsStrategy.metricsDecider)
/**
* Provide custom metrics strategy resources.
*/
object ClusterMetricsStrategy {
import akka.actor._
import akka.actor.SupervisorStrategy._
/**
* [[SupervisorStrategy.Decider]] which allows to survive intermittent Sigar native method calls failures.
*/
val metricsDecider: SupervisorStrategy.Decider = {
case _: ActorInitializationException Stop
case _: ActorKilledException Stop
case _: DeathPactException Stop
case _: Throwable Restart
}
}

View file

@ -0,0 +1,68 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import scala.concurrent.duration.FiniteDuration
/**
* 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.
*
* @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 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.
*
*/
@SerialVersionUID(1L)
final case class EWMA(value: Double, alpha: Double) {
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 new [[akka.cluster.EWMA]] with the updated value
*/
def :+(xn: Double): EWMA = {
val newValue = (alpha * xn) + (1 - alpha) * value
if (newValue == value) this // no change
else copy(value = newValue)
}
}
object EWMA {
/**
* math.log(2)
*/
private val LogOf2 = 0.69315
/**
* Calculate the alpha (decay factor) used in [[akka.cluster.EWMA]]
* from specified half-life and interval between observations.
* Half-life is the interval over which the weights decrease by a factor of two.
* The relevance of each data sample is halved for every passing half-life duration,
* i.e. after 4 times the half-life, a data samples relevance is reduced to 6% of
* its original relevance. The initial relevance of a data sample is given by
* 1 0.5 ^ (collect-interval / half-life).
*/
def alpha(halfLife: FiniteDuration, collectInterval: FiniteDuration): Double = {
val halfLifeMillis = halfLife.toMillis
require(halfLife.toMillis > 0, "halfLife must be > 0 s")
val decayRate = LogOf2 / halfLifeMillis
1 - math.exp(-decayRate * collectInterval.toMillis)
}
}

View file

@ -0,0 +1,377 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import akka.actor.Address
import scala.util.Success
import scala.util.Failure
import scala.util.Try
/**
* Metrics key/value.
*
* Equality of Metric is based on its name.
*
* @param name the metric name
* @param value the metric value, which must be a valid numerical value,
* a valid value is neither negative nor NaN/Infinite.
* @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 number of processors), are not trended.
*/
@SerialVersionUID(1L)
final case class Metric private[metrics] (name: String, value: Number, average: Option[EWMA])
extends MetricNumericConverter {
require(defined(value), s"Invalid Metric [$name] value [$value]")
/**
* Updates the data point, and if defined, updates the data stream (average).
* Returns the updated metric.
*/
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)
}
else this
/**
* The numerical value of the average, if defined, otherwise the latest value
*/
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 <code>that</code> is tracking the same metric as this.
*/
def sameAs(that: Metric): Boolean = name == that.name
override def hashCode = name.##
override def equals(obj: Any) = obj match {
case other: Metric sameAs(other)
case _ false
}
}
/**
* Factory for creating valid Metric instances.
*/
object Metric extends MetricNumericConverter {
/**
* Creates a new Metric instance if the value is valid, otherwise None
* is returned. Invalid numeric values are negative and NaN/Infinite.
*/
def create(name: String, value: Number, decayFactor: Option[Double]): Option[Metric] =
if (defined(value)) Some(new Metric(name, value, createEWMA(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 and NaN/Infinite.
*/
def create(name: String, value: Try[Number], decayFactor: Option[Double]): Option[Metric] = value match {
case Success(v) create(name, v, decayFactor)
case Failure(_) None
}
def createEWMA(value: Double, decayFactor: Option[Double]): Option[EWMA] = decayFactor match {
case Some(alpha) Some(EWMA(value, alpha))
case None None
}
}
/**
* Definitions of the built-in standard metrics.
*
* The following extractors and data structures makes it easy to consume the
* [[NodeMetrics]] in for example load balancers.
*/
object StandardMetrics {
// Constants for the heap related Metric names
final val HeapMemoryUsed = "heap-memory-used"
final val HeapMemoryCommitted = "heap-memory-committed"
final val HeapMemoryMax = "heap-memory-max"
// Constants for the cpu related Metric names
final val SystemLoadAverage = "system-load-average"
final val Processors = "processors"
// In latest Linux kernels: CpuCombined + CpuStolen + CpuIdle = 1.0 or 100%.
/** Sum of User + Sys + Nice + Wait. See [[org.hyperic.sigar.CpuPerc]] */
final val CpuCombined = "cpu-combined"
/** The amount of CPU 'stolen' from this virtual machine by the hypervisor for other tasks (such as running another virtual machine). */
final val CpuStolen = "cpu-stolen"
/** Amount of CPU time left after combined and stolen are removed. */
final val CpuIdle = "cpu-idle"
object HeapMemory {
/**
* 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)
} yield (nodeMetrics.address, nodeMetrics.timestamp,
used.smoothValue.longValue, committed.smoothValue.longValue,
nodeMetrics.metric(HeapMemoryMax).map(_.smoothValue.longValue))
}
}
/**
* 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)
// note that above extractor returns tuple
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.
* @param max the maximum amount of memory (in bytes) that can be used for JVM memory management.
* Can be undefined on some OS.
*/
@SerialVersionUID(1L)
final 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 {
/**
* 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], Option[Double], Int)] = {
for {
processors nodeMetrics.metric(Processors)
} yield (nodeMetrics.address, nodeMetrics.timestamp,
nodeMetrics.metric(SystemLoadAverage).map(_.smoothValue),
nodeMetrics.metric(CpuCombined).map(_.smoothValue),
nodeMetrics.metric(CpuStolen).map(_.smoothValue),
processors.value.intValue)
}
}
/**
* Java API to extract Cpu data from nodeMetrics, if the nodeMetrics
* contains necessary cpu metrics, otherwise it returns null.
*/
def extractCpu(nodeMetrics: NodeMetrics): Cpu = nodeMetrics match {
case Cpu(address, timestamp, systemLoadAverage, cpuCombined, cpuStolen, processors)
// note that above extractor returns tuple
Cpu(address, timestamp, systemLoadAverage, cpuCombined, cpuStolen, processors)
case _ null
}
/**
* @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 cpuStolen stolen CPU time, in percentage ([0.0 - 1.0].
* @param processors the number of available processors
*/
@SerialVersionUID(1L)
final case class Cpu(
address: Address,
timestamp: Long,
systemLoadAverage: Option[Double],
cpuCombined: Option[Double],
cpuStolen: Option[Double],
processors: Int) {
cpuCombined match {
case Some(x) require(0.0 <= x && x <= 1.0, s"cpuCombined must be between [0.0 - 1.0], was [$x]")
case None
}
cpuStolen match {
case Some(x) require(0.0 <= x && x <= 1.0, s"cpuStolen must be between [0.0 - 1.0], was [$x]")
case None
}
}
}
/**
* INTERNAL API
*
* Encapsulates evaluation of validity of metric values, conversion of an actual metric value to
* a [[akka.cluster.Metric]] for consumption by subscribed cluster entities.
*/
private[metrics] trait MetricNumericConverter {
/**
* An defined value is neither negative nor NaN/Infinite:
* <ul><li>JMX system load average and max heap can be 'undefined' for certain OS, in which case a -1 is returned</li>
* <li>SIGAR combined CPU can occasionally return a NaN or Infinite (known bug)</li></ul>
*/
def defined(value: Number): Boolean = convertNumber(value) match {
case Left(a) a >= 0
case Right(b) !(b < 0.0 || b.isNaN || b.isInfinite)
}
/**
* May involve rounding or truncation.
*/
def convertNumber(from: Any): Either[Long, Double] = from match {
case n: Int Left(n)
case n: Long Left(n)
case n: Double Right(n)
case n: Float Right(n)
case n: BigInt Left(n.longValue)
case n: BigDecimal Right(n.doubleValue)
case x throw new IllegalArgumentException(s"Not a number [$x]")
}
}
/**
* The snapshot of current sampled health metrics for any monitored process.
* Collected and gossipped at regular intervals for dynamic cluster management strategies.
*
* Equality of NodeMetrics is based on its address.
*
* @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 metrics the set of sampled [[akka.actor.Metric]]
*/
@SerialVersionUID(1L)
final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Metric] = Set.empty[Metric]) {
/**
* Returns the most recent data.
*/
def merge(that: NodeMetrics): NodeMetrics = {
require(address == that.address, s"merge only allowed for same address, [$address] != [$that.address]")
if (timestamp >= that.timestamp) this // that is older
else {
// equality is based on the name of the Metric and Set doesn't replace existing element
copy(metrics = that.metrics ++ metrics, timestamp = that.timestamp)
}
}
/**
* Returns the most recent data with [[EWMA]] averaging.
*/
def update(that: NodeMetrics): NodeMetrics = {
require(address == that.address, s"update only allowed for same address, [$address] != [$that.address]")
// Apply sample ordering.
val (latestNode, currentNode) = if (this.timestamp >= that.timestamp) (this, that) else (that, this)
// Average metrics present in both latest and current.
val updated = for {
latest latestNode.metrics
current currentNode.metrics
if (latest sameAs current)
} yield {
current :+ latest
}
// Append metrics missing from either latest or current.
// Equality is based on the [[Metric.name]] and [[Set]] doesn't replace existing elements.
val merged = updated ++ latestNode.metrics ++ currentNode.metrics
copy(metrics = merged, timestamp = latestNode.timestamp)
}
def metric(key: String): Option[Metric] = metrics.collectFirst { case m if m.name == key m }
/**
* Java API
*/
def getMetrics: java.lang.Iterable[Metric] =
scala.collection.JavaConverters.asJavaIterableConverter(metrics).asJava
/**
* Returns true if <code>that</code> address is the same as this
*/
def sameAs(that: NodeMetrics): Boolean = address == that.address
override def hashCode = address.##
override def equals(obj: Any) = obj match {
case other: NodeMetrics sameAs(other)
case _ false
}
}
/**
* INTERNAL API
*/
private[metrics] object MetricsGossip {
val empty = MetricsGossip(Set.empty[NodeMetrics])
}
/**
* INTERNAL API
*
* @param nodes metrics per node
*/
@SerialVersionUID(1L)
private[metrics] final case class MetricsGossip(nodes: Set[NodeMetrics]) {
/**
* Removes nodes if their correlating node ring members are not [[akka.cluster.MemberStatus.Up]]
*/
def remove(node: Address): MetricsGossip = copy(nodes = nodes filterNot (_.address == node))
/**
* Only the nodes that are in the `includeNodes` Set.
*/
def filter(includeNodes: Set[Address]): MetricsGossip =
copy(nodes = nodes filter { includeNodes contains _.address })
/**
* Adds new remote [[NodeMetrics]] and merges existing from a remote gossip.
*/
def merge(otherGossip: MetricsGossip): MetricsGossip =
otherGossip.nodes.foldLeft(this) { (gossip, nodeMetrics) gossip :+ nodeMetrics }
/**
* Adds new local [[NodeMetrics]], or merges an existing.
*/
def :+(newNodeMetrics: NodeMetrics): MetricsGossip = nodeMetricsFor(newNodeMetrics.address) match {
case Some(existingNodeMetrics)
copy(nodes = nodes - existingNodeMetrics + (existingNodeMetrics update newNodeMetrics))
case None copy(nodes = nodes + newNodeMetrics)
}
/**
* Returns [[NodeMetrics]] for a node if exists.
*/
def nodeMetricsFor(address: Address): Option[NodeMetrics] = nodes find { n n.address == address }
}

View file

@ -0,0 +1,277 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import akka.actor.ActorSystem
import akka.actor.ExtendedActorSystem
import akka.event.Logging
import akka.event.LoggingAdapter
import akka.ConfigurationException
import akka.actor.Address
import java.lang.management.MemoryMXBean
import java.lang.management.ManagementFactory
import java.lang.management.OperatingSystemMXBean
import java.lang.management.MemoryUsage
import java.lang.System.{ currentTimeMillis newTimestamp }
import akka.cluster.Cluster
import java.io.Closeable
import org.hyperic.sigar.SigarProxy
/**
* Metrics sampler.
*
* Implementations of cluster system metrics collectors extend this trait.
*/
trait MetricsCollector extends Closeable {
/**
* Samples and collects new data points.
* This method is invoked periodically and should return
* current metrics for this node.
*/
def sample(): NodeMetrics
}
/**
* INTERNAL API
*
* Factory to create configured [[MetricsCollector]].
*
* Metrics collector instantiation priority order:
* 1) Provided custom collector
* 2) Internal [[SigarMetricsCollector]]
* 3) Internal [[JmxMetricsCollector]]
*/
private[metrics] object MetricsCollector {
/** Try to create collector instance in the order of priority. */
def apply(system: ActorSystem): MetricsCollector = {
val log = Logging(system, getClass.getName)
val settings = ClusterMetricsSettings(system.settings.config)
import settings._
val collectorCustom = CollectorProvider
val collectorSigar = classOf[SigarMetricsCollector].getName
val collectorJMX = classOf[JmxMetricsCollector].getName
val useCustom = !CollectorFallback
val useInternal = CollectorFallback && CollectorProvider == ""
def create(provider: String) = TryNative {
log.debug(s"Trying ${provider}.")
system.asInstanceOf[ExtendedActorSystem].dynamicAccess
.createInstanceFor[MetricsCollector](provider, List(classOf[ActorSystem] -> system)).get
}
val collector = if (useCustom)
create(collectorCustom)
else if (useInternal)
create(collectorSigar) orElse create(collectorJMX)
else // Use complete fall back chain.
create(collectorCustom) orElse create(collectorSigar) orElse create(collectorJMX)
collector.recover {
case e throw new ConfigurationException(s"Could not create metrics collector: ${e}")
}.get
}
}
/**
* 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
*/
class JmxMetricsCollector(address: Address, decayFactor: Double) extends MetricsCollector {
import StandardMetrics._
private def this(address: Address, settings: ClusterMetricsSettings) =
this(address,
EWMA.alpha(settings.CollectorMovingAverageHalfLife, settings.CollectorSampleInterval))
/**
* This constructor is used when creating an instance from configured FQCN
*/
def this(system: ActorSystem) = this(Cluster(system).selfAddress, ClusterMetricsExtension(system).settings)
private val decayFactorOption = Some(decayFactor)
private val memoryMBean: MemoryMXBean = ManagementFactory.getMemoryMXBean
private val osMBean: OperatingSystemMXBean = ManagementFactory.getOperatingSystemMXBean
/**
* Samples and collects new data points.
* Creates a new instance each time.
*/
def sample(): NodeMetrics = NodeMetrics(address, newTimestamp, metrics)
/**
* Generate metrics set.
* Creates a new instance each time.
*/
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 from JMX, and None is returned from this method.
* Creates a new instance each time.
*/
def systemLoadAverage: Option[Metric] = Metric.create(
name = SystemLoadAverage,
value = osMBean.getSystemLoadAverage,
decayFactor = None)
/**
* (JMX) Returns the number of available processors
* Creates a new instance each time.
*/
def processors: Option[Metric] = Metric.create(
name = Processors,
value = osMBean.getAvailableProcessors,
decayFactor = None)
/**
* 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).
* Creates a new instance each time.
*/
def heapUsed(heap: MemoryUsage): Option[Metric] = Metric.create(
name = HeapMemoryUsed,
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).
* Creates a new instance each time.
*/
def heapCommitted(heap: MemoryUsage): Option[Metric] = Metric.create(
name = HeapMemoryCommitted,
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.
* Creates a new instance each time.
*/
def heapMax(heap: MemoryUsage): Option[Metric] = Metric.create(
name = HeapMemoryMax,
value = heap.getMax,
decayFactor = None)
override def close(): Unit = ()
}
/**
* 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.
*
* The constructor will by design throw exception if org.hyperic.sigar.Sigar can't be loaded, due
* to missing classes or native libraries.
*
* @param address The [[akka.actor.Address]] of the node being sampled
* @param decay how quickly the exponential weighting of past data is decayed
* @param sigar the org.hyperic.Sigar instance
*/
class SigarMetricsCollector(address: Address, decayFactor: Double, sigar: SigarProxy)
extends JmxMetricsCollector(address, decayFactor) {
import StandardMetrics._
import org.hyperic.sigar.CpuPerc
def this(address: Address, settings: ClusterMetricsSettings, sigar: SigarProxy) =
this(address,
EWMA.alpha(settings.CollectorMovingAverageHalfLife, settings.CollectorSampleInterval),
sigar)
def this(address: Address, settings: ClusterMetricsSettings) =
this(address, settings, DefaultSigarProvider(settings).createSigarInstance)
/**
* This constructor is used when creating an instance from configured FQCN
*/
def this(system: ActorSystem) = this(Cluster(system).selfAddress, ClusterMetricsExtension(system).settings)
private val decayFactorOption = Some(decayFactor)
/**
* Verify at the end of construction that Sigar is operational.
*/
metrics()
// Construction complete.
override def metrics(): Set[Metric] = {
// Must obtain cpuPerc in one shot. See https://github.com/akka/akka/issues/16121
val cpuPerc = sigar.getCpuPerc
super.metrics ++ Set(cpuCombined(cpuPerc), cpuStolen(cpuPerc)).flatten
}
/**
* (SIGAR) Returns the OS-specific average load on the CPUs in the system, for the past 1 minute.
*
* Creates a new instance each time.
*/
override def systemLoadAverage: Option[Metric] = Metric.create(
name = SystemLoadAverage,
value = sigar.getLoadAverage()(0).asInstanceOf[Number],
decayFactor = None)
/**
* (SIGAR) Returns the 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. Note that 99% CPU utilization can be optimal or indicative of failure.
*
* 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.
*
* Creates a new instance each time.
*/
def cpuCombined(cpuPerc: CpuPerc): Option[Metric] = Metric.create(
name = CpuCombined,
value = cpuPerc.getCombined.asInstanceOf[Number],
decayFactor = decayFactorOption)
/**
* (SIGAR) Returns the stolen CPU time. Relevant to virtual hosting environments.
* For details please see: [[http://en.wikipedia.org/wiki/CPU_time#Subdivision Wikipedia - CPU time subdivision]] and
* [[https://www.datadoghq.com/2013/08/understanding-aws-stolen-cpu-and-how-it-affects-your-apps/ Understanding AWS stolen CPU and how it affects your apps]]
*
* Creates a new instance each time.
*/
def cpuStolen(cpuPerc: CpuPerc): Option[Metric] = Metric.create(
name = CpuStolen,
value = cpuPerc.getStolen.asInstanceOf[Number],
decayFactor = decayFactorOption)
/**
* (SIGAR) Returns the idle CPU time.
* Amount of CPU time left after combined and stolen are removed.
*
* Creates a new instance each time.
*/
def cpuIdle(cpuPerc: CpuPerc): Option[Metric] = Metric.create(
name = CpuIdle,
value = cpuPerc.getIdle.asInstanceOf[Number],
decayFactor = decayFactorOption)
/**
* Releases any native resources associated with this instance.
*/
override def close(): Unit = SigarProvider.close(sigar)
}

View file

@ -0,0 +1,105 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import java.io.File
import kamon.sigar.SigarProvisioner
import org.hyperic.sigar.Sigar
import org.hyperic.sigar.SigarProxy
import org.hyperic.sigar.SigarException
import scala.language.postfixOps
import scala.util.Success
import scala.util.Failure
import scala.util.Try
/**
* Provide sigar instance as [[SigarProxy]].
*
* User can provision sigar classes and native library in one of the following ways:
*
* 1) Use [[https://github.com/kamon-io/sigar-loader Kamon sigar-loader]] as a project dependency for the user project.
* Metrics extension will extract and load sigar library on demand with help of Kamon sigar provisioner.
*
* 2) Use [[https://github.com/kamon-io/sigar-loader Kamon sigar-loader]] as java agent: `java -javaagent:/path/to/sigar-loader.jar`
* Kamon sigar loader agent will extract and load sigar library during JVM start.
*
* 3) Place `sigar.jar` on the `classpath` and sigar native library for the o/s on the `java.library.path`
* User is required to manage both project dependency and library deployment manually.
*/
trait SigarProvider {
/** Library extract location. */
def extractFolder: String
/** Verify if sigar native library is loaded and operational. */
def isNativeLoaded: Boolean =
try {
val sigar = verifiedSigarInstance
SigarProvider.close(sigar)
true
} catch {
case e: Throwable false
}
/** Create sigar and verify it works. */
def verifiedSigarInstance: SigarProxy = {
val sigar = new Sigar()
sigar.getPid
sigar.getLoadAverage
sigar.getCpuPerc
sigar
}
/** Extract and load sigar native library. */
def provisionSigarLibrary(): Unit = {
SigarProvisioner.provision(new File(extractFolder))
}
/**
* Create sigar instance with 2-phase sigar library loading.
* 1) Assume that library is already provisioned.
* 2) Attempt to provision library via sigar-loader.
*/
def createSigarInstance: SigarProxy = {
TryNative {
verifiedSigarInstance
} orElse TryNative {
provisionSigarLibrary()
verifiedSigarInstance
} recover {
case e: Throwable throw new RuntimeException("Failed to load sigar:", e)
} get
}
}
object SigarProvider {
/**
* Release underlying sigar proxy resources.
*
* Note: [[SigarProxy]] is not [[Sigar]] during tests.
*/
def close(sigar: SigarProxy) = {
if (sigar.isInstanceOf[Sigar]) sigar.asInstanceOf[Sigar].close()
}
}
/**
* Provide sigar instance as [[SigarProxy]] with configured location via [[ClusterMetricsSettings]].
*/
case class DefaultSigarProvider(settings: ClusterMetricsSettings) extends SigarProvider {
def extractFolder = settings.NativeLibraryExtractFolder
}
/**
* INTERNAL API
*/
private[metrics] object TryNative {
def apply[T](r: T): Try[T] =
try Success(r) catch {
// catching all, for example java.lang.LinkageError that are not caught by `NonFatal` in `Try`
case e: Throwable Failure(e)
}
}

View file

@ -0,0 +1,219 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics.protobuf
import java.io.ByteArrayInputStream
import java.io.ByteArrayOutputStream
import java.io.ObjectOutputStream
import java.{ lang jl }
import java.util.zip.GZIPInputStream
import java.util.zip.GZIPOutputStream
import scala.annotation.tailrec
import scala.collection.JavaConverters.asJavaIterableConverter
import scala.collection.JavaConverters.asScalaBufferConverter
import scala.collection.JavaConverters.setAsJavaSetConverter
import scala.collection.breakOut
import com.google.protobuf.ByteString
import com.google.protobuf.MessageLite
import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.cluster.metrics.EWMA
import akka.cluster.metrics.Metric
import akka.cluster.metrics.MetricsGossip
import akka.cluster.metrics.NodeMetrics
import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages cm }
import akka.serialization.Serializer
import akka.util.ClassLoaderObjectInputStream
import akka.cluster.metrics.ClusterMetricsMessage
import akka.cluster.metrics.MetricsGossipEnvelope
import akka.cluster.metrics.ClusterMetricsSettings
/**
* Protobuf serializer for [[ClusterMetricsMessage]] types.
*/
class MessageSerializer(val system: ExtendedActorSystem) extends Serializer {
private final val BufferSize = 4 * 1024
private val fromBinaryMap = collection.immutable.HashMap[Class[_ <: ClusterMetricsMessage], Array[Byte] AnyRef](
classOf[MetricsGossipEnvelope] -> metricsGossipEnvelopeFromBinary)
override val includeManifest: Boolean = true
override val identifier = ClusterMetricsSettings(system.settings.config).SerializerIdentifier
override def toBinary(obj: AnyRef): Array[Byte] = obj match {
case m: MetricsGossipEnvelope
compress(metricsGossipEnvelopeToProto(m))
case _
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass}")
}
def compress(msg: MessageLite): Array[Byte] = {
val bos = new ByteArrayOutputStream(BufferSize)
val zip = new GZIPOutputStream(bos)
msg.writeTo(zip)
zip.close()
bos.toByteArray
}
def decompress(bytes: Array[Byte]): Array[Byte] = {
val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
val out = new ByteArrayOutputStream()
val buffer = new Array[Byte](BufferSize)
@tailrec def readChunk(): Unit = in.read(buffer) match {
case -1 ()
case n
out.write(buffer, 0, n)
readChunk()
}
readChunk()
out.toByteArray
}
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = clazz match {
case Some(c) fromBinaryMap.get(c.asInstanceOf[Class[ClusterMetricsMessage]]) match {
case Some(f) f(bytes)
case None throw new IllegalArgumentException(s"Unimplemented deserialization of message class $c in metrics")
}
case _ throw new IllegalArgumentException("Need a metrics message class to be able to deserialize bytes in metrics")
}
private def addressFromBinary(bytes: Array[Byte]): Address =
addressFromProto(cm.Address.parseFrom(bytes))
private def addressToProto(address: Address): cm.Address.Builder = address match {
case Address(protocol, system, Some(host), Some(port))
cm.Address.newBuilder().setSystem(system).setHostname(host).setPort(port).setProtocol(protocol)
case _ throw new IllegalArgumentException(s"Address [${address}] could not be serialized: host or port missing.")
}
private def addressToProtoByteArray(address: Address): Array[Byte] = addressToProto(address).build.toByteArray
@volatile
private var protocolCache: String = null
@volatile
private var systemCache: String = null
private def getProtocol(address: cm.Address): String = {
val p = address.getProtocol
val pc = protocolCache
if (pc == p) pc
else {
protocolCache = p
p
}
}
private def getSystem(address: cm.Address): String = {
val s = address.getSystem
val sc = systemCache
if (sc == s) sc
else {
systemCache = s
s
}
}
private def addressFromProto(address: cm.Address): Address =
Address(getProtocol(address), getSystem(address), address.getHostname, address.getPort)
private def mapWithErrorMessage[T](map: Map[T, Int], value: T, unknown: String): Int = map.get(value) match {
case Some(x) x
case _ throw new IllegalArgumentException(s"Unknown ${unknown} [${value}] in cluster message")
}
private def metricsGossipEnvelopeToProto(envelope: MetricsGossipEnvelope): cm.MetricsGossipEnvelope = {
val mgossip = envelope.gossip
val allAddresses = mgossip.nodes.foldLeft(Set.empty[Address])((s, n) s + n.address)
val addressMapping = allAddresses.zipWithIndex.toMap
val allMetricNames = mgossip.nodes.foldLeft(Set.empty[String])((s, n) s ++ n.metrics.iterator.map(_.name))
val metricNamesMapping = allMetricNames.zipWithIndex.toMap
def mapAddress(address: Address) = mapWithErrorMessage(addressMapping, address, "address")
def mapName(name: String) = mapWithErrorMessage(metricNamesMapping, name, "address")
def ewmaToProto(ewma: Option[EWMA]): Option[cm.NodeMetrics.EWMA.Builder] = ewma.map {
x cm.NodeMetrics.EWMA.newBuilder().setValue(x.value).setAlpha(x.alpha)
}
def numberToProto(number: Number): cm.NodeMetrics.Number.Builder = {
import cm.NodeMetrics.Number
import cm.NodeMetrics.NumberType
number match {
case n: jl.Double Number.newBuilder().setType(NumberType.Double).setValue64(jl.Double.doubleToLongBits(n))
case n: jl.Long Number.newBuilder().setType(NumberType.Long).setValue64(n)
case n: jl.Float Number.newBuilder().setType(NumberType.Float).setValue32(jl.Float.floatToIntBits(n))
case n: jl.Integer Number.newBuilder().setType(NumberType.Integer) setValue32 (n)
case _
val bos = new ByteArrayOutputStream
val out = new ObjectOutputStream(bos)
out.writeObject(number)
out.close()
Number.newBuilder().setType(NumberType.Serialized).setSerialized(ByteString.copyFrom(bos.toByteArray))
}
}
def metricToProto(metric: Metric): cm.NodeMetrics.Metric.Builder = {
val builder = cm.NodeMetrics.Metric.newBuilder().setNameIndex(mapName(metric.name)).setNumber(numberToProto(metric.value))
ewmaToProto(metric.average).map(builder.setEwma(_)).getOrElse(builder)
}
def nodeMetricsToProto(nodeMetrics: NodeMetrics): cm.NodeMetrics.Builder =
cm.NodeMetrics.newBuilder().setAddressIndex(mapAddress(nodeMetrics.address)).setTimestamp(nodeMetrics.timestamp).
addAllMetrics(nodeMetrics.metrics.map(metricToProto(_).build).asJava)
val nodeMetrics: Iterable[cm.NodeMetrics] = mgossip.nodes.map(nodeMetricsToProto(_).build)
cm.MetricsGossipEnvelope.newBuilder().setFrom(addressToProto(envelope.from)).setGossip(
cm.MetricsGossip.newBuilder().addAllAllAddresses(allAddresses.map(addressToProto(_).build()).asJava).
addAllAllMetricNames(allMetricNames.asJava).addAllNodeMetrics(nodeMetrics.asJava)).
setReply(envelope.reply).build
}
private def metricsGossipEnvelopeFromBinary(bytes: Array[Byte]): MetricsGossipEnvelope =
metricsGossipEnvelopeFromProto(cm.MetricsGossipEnvelope.parseFrom(decompress(bytes)))
private def metricsGossipEnvelopeFromProto(envelope: cm.MetricsGossipEnvelope): MetricsGossipEnvelope = {
val mgossip = envelope.getGossip
val addressMapping: Vector[Address] = mgossip.getAllAddressesList.asScala.map(addressFromProto)(breakOut)
val metricNameMapping: Vector[String] = mgossip.getAllMetricNamesList.asScala.toVector
def ewmaFromProto(ewma: cm.NodeMetrics.EWMA): Option[EWMA] =
Some(EWMA(ewma.getValue, ewma.getAlpha))
def numberFromProto(number: cm.NodeMetrics.Number): Number = {
import cm.NodeMetrics.Number
import cm.NodeMetrics.NumberType
number.getType.getNumber match {
case NumberType.Double_VALUE jl.Double.longBitsToDouble(number.getValue64)
case NumberType.Long_VALUE number.getValue64
case NumberType.Float_VALUE jl.Float.intBitsToFloat(number.getValue32)
case NumberType.Integer_VALUE number.getValue32
case NumberType.Serialized_VALUE
val in = new ClassLoaderObjectInputStream(system.dynamicAccess.classLoader,
new ByteArrayInputStream(number.getSerialized.toByteArray))
val obj = in.readObject
in.close()
obj.asInstanceOf[jl.Number]
}
}
def metricFromProto(metric: cm.NodeMetrics.Metric): Metric =
Metric(metricNameMapping(metric.getNameIndex), numberFromProto(metric.getNumber),
if (metric.hasEwma) ewmaFromProto(metric.getEwma) else None)
def nodeMetricsFromProto(nodeMetrics: cm.NodeMetrics): NodeMetrics =
NodeMetrics(addressMapping(nodeMetrics.getAddressIndex), nodeMetrics.getTimestamp,
nodeMetrics.getMetricsList.asScala.map(metricFromProto)(breakOut))
val nodeMetrics: Set[NodeMetrics] = mgossip.getNodeMetricsList.asScala.map(nodeMetricsFromProto)(breakOut)
MetricsGossipEnvelope(addressFromProto(envelope.getFrom), MetricsGossip(nodeMetrics), envelope.getReply)
}
}

View file

@ -0,0 +1,163 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import scala.language.postfixOps
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.actor.ExtendedActorSystem
import akka.cluster.MultiNodeClusterSpec
import akka.testkit.LongRunningTest
import akka.cluster.MemberStatus
trait ClusterMetricsCommonConfig extends MultiNodeConfig {
import ConfigFactory._
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
val node4 = role("node-4")
val node5 = role("node-5")
def nodeList = Seq(node1, node2, node3, node4, node5)
// Extract individual sigar library for every node.
nodeList foreach { role
nodeConfig(role) {
parseString("akka.cluster.metrics.native-library-extract-folder=${user.dir}/target/native/" + role.name)
}
}
// Disable legacy metrics in akka-cluster.
def disableMetricsLegacy = parseString("""akka.cluster.metrics.enabled=off""")
// Enable metrics extension in akka-cluster-metrics.
def enableMetricsExtension = parseString("""
akka.extensions=["akka.cluster.metrics.ClusterMetricsExtension"]
akka.cluster.metrics.collector.enabled = on
""")
// Disable metrics extension in akka-cluster-metrics.
def disableMetricsExtension = parseString("""
akka.extensions=["akka.cluster.metrics.ClusterMetricsExtension"]
akka.cluster.metrics.collector.enabled = off
""")
// Activate slf4j logging along with test listener.
def customLogging = parseString("""akka.loggers=["akka.testkit.TestEventListener","akka.event.slf4j.Slf4jLogger"]""")
}
object ClusterMetricsDisabledConfig extends ClusterMetricsCommonConfig {
commonConfig {
Seq(
customLogging,
disableMetricsLegacy,
disableMetricsExtension,
debugConfig(on = false),
MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet)
.reduceLeft(_ withFallback _)
}
}
object ClusterMetricsEnabledConfig extends ClusterMetricsCommonConfig {
import ConfigFactory._
commonConfig {
Seq(
customLogging,
disableMetricsLegacy,
enableMetricsExtension,
debugConfig(on = false),
MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet)
.reduceLeft(_ withFallback _)
}
}
class ClusterMetricsEnabledMultiJvmNode1 extends ClusterMetricsEnabledSpec
class ClusterMetricsEnabledMultiJvmNode2 extends ClusterMetricsEnabledSpec
class ClusterMetricsEnabledMultiJvmNode3 extends ClusterMetricsEnabledSpec
class ClusterMetricsEnabledMultiJvmNode4 extends ClusterMetricsEnabledSpec
class ClusterMetricsEnabledMultiJvmNode5 extends ClusterMetricsEnabledSpec
abstract class ClusterMetricsEnabledSpec extends MultiNodeSpec(ClusterMetricsEnabledConfig)
with MultiNodeClusterSpec with RedirectLogging {
import ClusterMetricsEnabledConfig._
def isSigar(collector: MetricsCollector): Boolean = collector.isInstanceOf[SigarMetricsCollector]
def saveApplicationConf(): Unit = {
import java.io.File
import java.io.PrintWriter
val conf = cluster.system.settings.config
val text = conf.root.render
val file = new File(s"target/${myself.name}_application.conf")
Some(new PrintWriter(file)) map { p p.write(text); p.close }
}
saveApplicationConf()
val metricsView = new ClusterMetricsView(cluster.system)
"Cluster metrics" must {
"periodically collect metrics on each node, publish to the event stream, " +
"and gossip metrics around the node ring" taggedAs LongRunningTest in within(60 seconds) {
awaitClusterUp(roles: _*)
enterBarrier("cluster-started")
awaitAssert(clusterView.members.count(_.status == MemberStatus.Up) should be(roles.size))
// TODO ensure same contract
//awaitAssert(clusterView.clusterMetrics.size should be(roles.size))
awaitAssert(metricsView.clusterMetrics.size should be(roles.size))
val collector = MetricsCollector(cluster.system)
collector.sample.metrics.size should be > (3)
enterBarrier("after")
}
"reflect the correct number of node metrics in cluster view" taggedAs LongRunningTest in within(30 seconds) {
runOn(node2) {
cluster.leave(node1)
}
enterBarrier("first-left")
runOn(node2, node3, node4, node5) {
markNodeAsUnavailable(node1)
// TODO ensure same contract
//awaitAssert(clusterView.clusterMetrics.size should be(roles.size - 1))
awaitAssert(metricsView.clusterMetrics.size should be(roles.size - 1))
}
enterBarrier("finished")
}
}
}
class ClusterMetricsDisabledMultiJvmNode1 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode2 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNodv3 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode4 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode5 extends ClusterMetricsDisabledSpec
abstract class ClusterMetricsDisabledSpec extends MultiNodeSpec(ClusterMetricsDisabledConfig)
with MultiNodeClusterSpec with RedirectLogging {
import akka.cluster.ClusterEvent.CurrentClusterState
val metricsView = new ClusterMetricsView(cluster.system)
"Cluster metrics" must {
"not collect metrics, not publish metrics events, and not gossip metrics" taggedAs LongRunningTest in {
awaitClusterUp(roles: _*)
// TODO ensure same contract
//clusterView.clusterMetrics.size should be(0)
metricsView.clusterMetrics.size should be(0)
cluster.subscribe(testActor, classOf[ClusterMetricsChanged])
expectMsgType[CurrentClusterState]
expectNoMsg
// TODO ensure same contract
//clusterView.clusterMetrics.size should be(0)
metricsView.clusterMetrics.size should be(0)
enterBarrier("after")
}
}
}

View file

@ -0,0 +1,244 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import language.postfixOps
import java.lang.management.ManagementFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import akka.actor._
import akka.cluster.Cluster
import akka.cluster.MultiNodeClusterSpec
import akka.pattern.ask
import akka.remote.testkit.{ MultiNodeSpec, MultiNodeConfig }
import akka.routing.GetRoutees
import akka.routing.FromConfig
import akka.testkit.{ LongRunningTest, DefaultTimeout, ImplicitSender }
import akka.routing.ActorRefRoutee
import akka.routing.Routees
import akka.cluster.routing.ClusterRouterPool
import akka.cluster.routing.ClusterRouterPoolSettings
object AdaptiveLoadBalancingRouterConfig extends MultiNodeConfig {
class Echo extends Actor {
def receive = {
case _ sender() ! Reply(Cluster(context.system).selfAddress)
}
}
class Memory extends Actor with ActorLogging {
var usedMemory: Array[Array[Int]] = _
def receive = {
case AllocateMemory
val heap = ManagementFactory.getMemoryMXBean.getHeapMemoryUsage
// getMax can be undefined (-1)
val max = math.max(heap.getMax, heap.getCommitted)
val used = heap.getUsed
log.debug("used heap before: [{}] bytes, of max [{}]", used, heap.getMax)
// allocate 70% of free space
val allocateBytes = (0.7 * (max - used)).toInt
val numberOfArrays = allocateBytes / 1024
usedMemory = Array.ofDim(numberOfArrays, 248) // each 248 element Int array will use ~ 1 kB
log.debug("used heap after: [{}] bytes", ManagementFactory.getMemoryMXBean.getHeapMemoryUsage.getUsed)
sender() ! "done"
}
}
case object AllocateMemory
final case class Reply(address: Address)
val node1 = role("node-1")
val node2 = role("node-2")
val node3 = role("node-3")
def nodeList = Seq(node1, node2, node3)
// Extract individual sigar library for every node.
nodeList foreach { role
nodeConfig(role) {
ConfigFactory.parseString("akka.cluster.metrics.native-library-extract-folder=${user.dir}/target/native/" + role.name)
}
}
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString("""
# Disable legacy metrics.
akka.cluster.metrics.enabled=off
# Enable metrics estension.
akka.extensions=["akka.cluster.metrics.ClusterMetricsExtension"]
# Use rapid metrics collection.
akka.cluster.metrics {
collector {
sample-interval = 1s
gossip-interval = 1s
moving-average-half-life = 2s
}
}
# Use metrics extension routing.
akka.actor.deployment {
/router3 = {
router = cluster-metrics-adaptive-pool
metrics-selector = cpu
nr-of-instances = 9
}
/router4 = {
router = cluster-metrics-adaptive-pool
metrics-selector = "akka.cluster.metrics.TestCustomMetricsSelector"
nr-of-instances = 10
cluster {
enabled = on
max-nr-of-instances-per-node = 2
}
}
}
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
}
class TestCustomMetricsSelector(config: Config) extends MetricsSelector {
override def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] = Map.empty
}
class AdaptiveLoadBalancingRouterMultiJvmNode1 extends AdaptiveLoadBalancingRouterSpec
class AdaptiveLoadBalancingRouterMultiJvmNode2 extends AdaptiveLoadBalancingRouterSpec
class AdaptiveLoadBalancingRouterMultiJvmNode3 extends AdaptiveLoadBalancingRouterSpec
abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoadBalancingRouterConfig)
with MultiNodeClusterSpec with RedirectLogging
with ImplicitSender with DefaultTimeout {
import AdaptiveLoadBalancingRouterConfig._
def currentRoutees(router: ActorRef) =
Await.result(router ? GetRoutees, timeout.duration).asInstanceOf[Routees].routees
def receiveReplies(expectedReplies: Int): Map[Address, Int] = {
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
(receiveWhile(5 seconds, messages = expectedReplies) {
case Reply(address) address
}).foldLeft(zero) {
case (replyMap, address) replyMap + (address -> (replyMap(address) + 1))
}
}
/**
* Fills in self address for local ActorRef
*/
def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
case Address(_, _, None, None) cluster.selfAddress
case a a
}
def startRouter(name: String): ActorRef = {
val router = system.actorOf(ClusterRouterPool(
local = AdaptiveLoadBalancingPool(HeapMetricsSelector),
settings = ClusterRouterPoolSettings(totalInstances = 10, maxInstancesPerNode = 1, allowLocalRoutees = true, useRole = None)).
props(Props[Echo]),
name)
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router).size should be(roles.size) }
val routees = currentRoutees(router)
routees.map { case ActorRefRoutee(ref) fullAddress(ref) }.toSet should be(roles.map(address).toSet)
router
}
val metricsSettings = ClusterMetricsSettings(cluster.system.settings.config)
def metricsAwait(factor: Int = 10): Unit = Thread.sleep(metricsSettings.CollectorSampleInterval.toMillis * factor)
"A cluster with a AdaptiveLoadBalancingRouter" must {
"start cluster nodes" taggedAs LongRunningTest in {
awaitClusterUp(roles: _*)
enterBarrier("after-1")
}
"use all nodes in the cluster when not overloaded" taggedAs LongRunningTest in {
runOn(node1) {
val router1 = startRouter("router1")
// collect some metrics before we start
metricsAwait()
val iterationCount = 100
1 to iterationCount foreach { _
router1 ! "hit"
// wait a while between each message, since metrics is collected periodically
Thread.sleep(10)
}
val replies = receiveReplies(iterationCount)
replies(node1) should be > (0)
replies(node2) should be > (0)
replies(node3) should be > (0)
replies.values.sum should be(iterationCount)
}
enterBarrier("after-2")
}
"prefer node with more free heap capacity" taggedAs LongRunningTest in {
System.gc()
enterBarrier("gc")
runOn(node2) {
within(20.seconds) {
system.actorOf(Props[Memory], "memory") ! AllocateMemory
expectMsg("done")
}
}
enterBarrier("heap-allocated")
runOn(node1) {
val router2 = startRouter("router2")
// collect some metrics before we start
metricsAwait()
val iterationCount = 3000
1 to iterationCount foreach { _
router2 ! "hit"
}
val replies = receiveReplies(iterationCount)
replies(node3) should be > (replies(node2))
replies.values.sum should be(iterationCount)
}
enterBarrier("after-3")
}
"create routees from configuration" taggedAs LongRunningTest in {
runOn(node1) {
val router3 = system.actorOf(FromConfig.props(Props[Memory]), "router3")
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router3).size should be(9) }
val routees = currentRoutees(router3)
routees.map { case ActorRefRoutee(ref) fullAddress(ref) }.toSet should be(Set(address(node1)))
}
enterBarrier("after-4")
}
"create routees from cluster.enabled configuration" taggedAs LongRunningTest in {
runOn(node1) {
val router4 = system.actorOf(FromConfig.props(Props[Memory]), "router4")
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router4).size should be(6) }
val routees = currentRoutees(router4)
routees.map { case ActorRefRoutee(ref) fullAddress(ref) }.toSet should be(Set(
address(node1), address(node2), address(node3)))
}
enterBarrier("after-5")
}
}
}

View file

@ -0,0 +1,39 @@
<?xml version="1.0" encoding="UTF-8"?>
<configuration>
<!-- Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> -->
<variable name="pattern" value="%date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n" />
<variable name="folder" value="${user.dir}/target/metrics" />
<appender name="stdout" class="ch.qos.logback.core.ConsoleAppender">
<encoder>
<pattern>${pattern}</pattern>
</encoder>
</appender>
<appender name="filing" class="ch.qos.logback.core.FileAppender">
<file>${folder}/test.log</file>
<append>false</append>
<encoder>
<pattern>${pattern}</pattern>
</encoder>
</appender>
<!-- Log at INFO level to suppress sigar library loading errors. -->
<root level="info">
<appender-ref ref="stdout" />
<appender-ref ref="filing" />
</root>
<!-- org.hyperic.Sigar logs missing library exception via Log4J at DEBUG level. -->
<logger name="org.hyperic" level="debug" />
<!-- Kamon sigar-loader is using JUL. -->
<!-- Propagate level changes made to a logback logger into the equivalent logger in JUL. -->
<contextListener class="ch.qos.logback.classic.jul.LevelChangePropagator">
<resetJUL>true</resetJUL>
</contextListener>
</configuration>

View file

@ -0,0 +1,6 @@
akka {
actor {
serialize-creators = on
serialize-messages = on
}
}

View file

@ -0,0 +1,124 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import scala.language.postfixOps
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.Await
import scala.util.{ Success, Try, Failure }
import akka.actor._
import akka.testkit._
import akka.cluster.metrics.StandardMetrics._
import org.scalatest.WordSpec
import org.scalatest.Matchers
import akka.cluster.Cluster
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricsExtensionSpec extends AkkaSpec(MetricsConfig.clusterSigarMock)
with ImplicitSender with RedirectLogging {
import system.dispatcher
val cluster = Cluster(system)
val extension = ClusterMetricsExtension(system)
val metricsView = new ClusterMetricsView(cluster.system)
val sampleInterval = extension.settings.CollectorSampleInterval
def metricsNodeCount = metricsView.clusterMetrics.size
def metricsHistorySize = metricsView.metricsHistory.size
// This is a single node test.
val nodeCount = 1
// Limit collector sample count.
val sampleCount = 10
// Metrics verification precision.
val epsilon = 0.001
// Sleep longer then single sample.
def awaitSample(time: Long = 3 * sampleInterval.toMillis) = Thread.sleep(time)
"Metrics Extension" must {
"collect metrics after start command" in {
extension.supervisor ! CollectionStartMessage
awaitAssert(metricsNodeCount should be(nodeCount), 15 seconds)
}
"collect mock sample during a time window" in {
awaitAssert(metricsHistorySize should be(sampleCount), 15 seconds)
extension.supervisor ! CollectionStopMessage
awaitSample()
metricsNodeCount should be(nodeCount)
metricsHistorySize should be >= (sampleCount)
}
"verify sigar mock data matches expected ewma data" in {
val history = metricsView.metricsHistory.reverse.map { _.head }
val expected = List(
(0.700, 0.000, 0.000),
(0.700, 0.018, 0.007),
(0.700, 0.051, 0.020),
(0.700, 0.096, 0.038),
(0.700, 0.151, 0.060),
(0.700, 0.214, 0.085),
(0.700, 0.266, 0.106),
(0.700, 0.309, 0.123),
(0.700, 0.343, 0.137),
(0.700, 0.372, 0.148))
expected.size should be(sampleCount)
history.zip(expected) foreach {
case (mockMetrics, expectedData)
(mockMetrics, expectedData) match {
case (Cpu(_, _, loadAverageMock, cpuCombinedMock, cpuStolenMock, _),
(loadAverageEwma, cpuCombinedEwma, cpuStolenEwma))
loadAverageMock.get should be(loadAverageEwma +- epsilon)
cpuCombinedMock.get should be(cpuCombinedEwma +- epsilon)
cpuStolenMock.get should be(cpuStolenEwma +- epsilon)
}
}
}
"control collector on/off state" in {
def cycle() = {
val size1 = metricsHistorySize
awaitSample()
val size2 = metricsHistorySize
size1 should be(size2)
extension.supervisor ! CollectionStartMessage
awaitSample()
val size3 = metricsHistorySize
size3 should be > (size2)
extension.supervisor ! CollectionStopMessage
awaitSample()
val size4 = metricsHistorySize
size4 should be >= (size3)
awaitSample()
val size5 = metricsHistorySize
size5 should be(size4)
}
(1 to 3) foreach { step cycle() }
}
}
}

View file

@ -0,0 +1,119 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import org.scalatest.WordSpec
import org.scalatest.Matchers
import akka.actor.Address
import akka.cluster.metrics.StandardMetrics._
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricsSelectorSpec extends WordSpec with Matchers {
val factor = 0.3 // TODO read from reference.conf
val abstractSelector = new CapacityMetricsSelector {
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = Map.empty
}
val a1 = Address("akka.tcp", "sys", "a1", 2551)
val b1 = Address("akka.tcp", "sys", "b1", 2551)
val c1 = Address("akka.tcp", "sys", "c1", 2551)
val d1 = Address("akka.tcp", "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.2, decayFactor),
Metric.create(CpuStolen, 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.4, decayFactor),
Metric.create(CpuStolen, 0.2, 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, 0.6, decayFactor),
Metric.create(CpuStolen, 0.3, 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 should 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 should 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) should be(0.75 +- 0.0001)
capacity(b1) should be(0.75 +- 0.0001)
capacity(c1) should be(0.0 +- 0.0001)
capacity(d1) should be(0.001953125 +- 0.0001)
}
}
"CpuMetricsSelector" must {
"calculate capacity of cpuCombined metrics" in {
val capacity = CpuMetricsSelector.capacity(nodeMetrics)
capacity(a1) should be(1.0 - 0.2 - 0.1 * (1.0 + factor) +- 0.0001)
capacity(b1) should be(1.0 - 0.4 - 0.2 * (1.0 + factor) +- 0.0001)
capacity(c1) should be(1.0 - 0.6 - 0.3 * (1.0 + factor) +- 0.0001)
capacity.contains(d1) should be(false)
}
}
"SystemLoadAverageMetricsSelector" must {
"calculate capacity of systemLoadAverage metrics" in {
val capacity = SystemLoadAverageMetricsSelector.capacity(nodeMetrics)
capacity(a1) should be(0.9375 +- 0.0001)
capacity(b1) should be(0.9375 +- 0.0001)
capacity(c1) should be(0.0 +- 0.0001)
capacity.contains(d1) should be(false)
}
}
"MixMetricsSelector" must {
"aggregate capacity of all metrics" in {
val capacity = MixMetricsSelector.capacity(nodeMetrics)
capacity(a1) should be((0.75 + 0.67 + 0.9375) / 3 +- 0.0001)
capacity(b1) should be((0.75 + 0.34 + 0.9375) / 3 +- 0.0001)
capacity(c1) should be((0.0 + 0.01 + 0.0) / 3 +- 0.0001)
capacity(d1) should be((0.001953125) / 1 +- 0.0001)
}
}
}

View file

@ -0,0 +1,44 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import language.postfixOps
import akka.testkit.AkkaSpec
import akka.dispatch.Dispatchers
import scala.concurrent.duration._
import akka.remote.PhiAccrualFailureDetector
import akka.util.Helpers.ConfigOps
import com.typesafe.config.ConfigFactory
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ClusterMetricsSettingsSpec extends AkkaSpec {
"ClusterMetricsSettings" must {
"be able to parse generic metrics config elements" in {
val settings = new ClusterMetricsSettings(system.settings.config)
import settings._
// Extension.
MetricsDispatcher should be(Dispatchers.DefaultDispatcherId)
PeriodicTasksInitialDelay should be(1 second)
NativeLibraryExtractFolder should be(System.getProperty("user.dir") + "/native")
SerializerIdentifier should be(10)
// Supervisor.
SupervisorName should be("cluster-metrics")
SupervisorStrategyProvider should be(classOf[ClusterMetricsStrategy].getName)
SupervisorStrategyConfiguration should be(
ConfigFactory.parseString("loggingEnabled=true,maxNrOfRetries=3,withinTimeRange=3s"))
// Collector.
CollectorEnabled should be(true)
CollectorProvider should be("")
CollectorSampleInterval should be(3 seconds)
CollectorGossipInterval should be(3 seconds)
CollectorMovingAverageHalfLife should be(12 seconds)
}
}
}

View file

@ -0,0 +1,101 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
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(MetricsConfig.defaultEnabled) 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 should be(100.0 +- 0.001)
}
"calcualate correct ewma for normal decay" in {
val d0 = EWMA(value = 1000.0, alpha = 2.0 / (1 + 10))
d0.value should be(1000.0 +- 0.01)
val d1 = d0 :+ 10.0
d1.value should be(820.0 +- 0.01)
val d2 = d1 :+ 10.0
d2.value should be(672.73 +- 0.01)
val d3 = d2 :+ 10.0
d3.value should be(552.23 +- 0.01)
val d4 = d3 :+ 10.0
d4.value should be(453.64 +- 0.01)
val dn = (1 to 100).foldLeft(d0)((d, _) d :+ 10.0)
dn.value should be(10.0 +- 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 should be(100.0 +- 0.01)
val d1 = d0 :+ 1.0
d1.value should be(1.0 +- 0.01)
val d2 = d1 :+ 57.0
d2.value should be(57.0 +- 0.01)
val d3 = d2 :+ 10.0
d3.value should be(10.0 +- 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) should be(expectedAlpha +- 0.001)
}
"calculate sane alpha from short half-life" in {
val alpha = EWMA.alpha(1.millis, 3.seconds)
alpha should be <= (1.0)
alpha should be >= (0.0)
alpha should be(1.0 +- 0.001)
}
"calculate sane alpha from long half-life" in {
val alpha = EWMA.alpha(1.day, 3.seconds)
alpha should be <= (1.0)
alpha should be >= (0.0)
alpha should be(0.0 +- 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 should be(true)
updated.smoothValue should not be (previous.smoothValue)
Some(updated)
} else None
}
}
streamingDataSet ++= changes.map(m m.name -> m)
}
}
}
}

View file

@ -0,0 +1,286 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import org.scalatest.WordSpec
import org.scalatest.Matchers
import akka.cluster.metrics.StandardMetrics._
import scala.util.Failure
import akka.actor.Address
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import java.lang.System.{ currentTimeMillis newTimestamp }
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricNumericConverterSpec extends WordSpec with Matchers with MetricNumericConverter {
"MetricNumericConverter" must {
"convert" in {
convertNumber(0).isLeft should be(true)
convertNumber(1).left.get should be(1)
convertNumber(1L).isLeft should be(true)
convertNumber(0.0).isRight should be(true)
}
"define a new metric" in {
val Some(metric) = Metric.create(HeapMemoryUsed, 256L, decayFactor = Some(0.18))
metric.name should be(HeapMemoryUsed)
metric.value should be(256L)
metric.isSmooth should be(true)
metric.smoothValue should be(256.0 +- 0.0001)
}
"define an undefined value with a None " in {
Metric.create("x", -1, None).isDefined should be(false)
Metric.create("x", java.lang.Double.NaN, None).isDefined should be(false)
Metric.create("x", Failure(new RuntimeException), None).isDefined should be(false)
}
"recognize whether a metric value is defined" in {
defined(0) should be(true)
defined(0.0) should be(true)
}
"recognize whether a metric value is not defined" in {
defined(-1) should be(false)
defined(-1.0) should be(false)
defined(Double.NaN) should be(false)
}
}
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class NodeMetricsSpec extends WordSpec with Matchers {
val node1 = Address("akka.tcp", "sys", "a", 2554)
val node2 = Address("akka.tcp", "sys", "a", 2555)
"NodeMetrics must" must {
"return correct result for 2 'same' nodes" in {
(NodeMetrics(node1, 0) sameAs NodeMetrics(node1, 0)) should be(true)
}
"return correct result for 2 not 'same' nodes" in {
(NodeMetrics(node1, 0) sameAs NodeMetrics(node2, 0)) should be(false)
}
"merge 2 NodeMetrics by most recent" in {
val sample1 = NodeMetrics(node1, 1, Set(Metric.create("a", 10, None), Metric.create("b", 20, None)).flatten)
val sample2 = NodeMetrics(node1, 2, Set(Metric.create("a", 11, None), Metric.create("c", 30, None)).flatten)
val merged = sample1 merge sample2
merged.timestamp should be(sample2.timestamp)
merged.metric("a").map(_.value) should be(Some(11))
merged.metric("b").map(_.value) should be(Some(20))
merged.metric("c").map(_.value) should be(Some(30))
}
"not merge 2 NodeMetrics if master is more recent" in {
val sample1 = NodeMetrics(node1, 1, Set(Metric.create("a", 10, None), Metric.create("b", 20, None)).flatten)
val sample2 = NodeMetrics(node1, 0, Set(Metric.create("a", 11, None), Metric.create("c", 30, None)).flatten)
val merged = sample1 merge sample2 // older and not same
merged.timestamp should be(sample1.timestamp)
merged.metrics should be(sample1.metrics)
}
"update 2 NodeMetrics by most recent" in {
val sample1 = NodeMetrics(node1, 1, Set(Metric.create("a", 10, None), Metric.create("b", 20, None)).flatten)
val sample2 = NodeMetrics(node1, 2, Set(Metric.create("a", 11, None), Metric.create("c", 30, None)).flatten)
val updated = sample1 update sample2
updated.metrics.size should be(3)
updated.timestamp should be(sample2.timestamp)
updated.metric("a").map(_.value) should be(Some(11))
updated.metric("b").map(_.value) should be(Some(20))
updated.metric("c").map(_.value) should be(Some(30))
}
"update 3 NodeMetrics with ewma applied" in {
import MetricsConfig._
val decay = Some(defaultDecayFactor)
val epsilon = 0.001
val sample1 = NodeMetrics(node1, 1, Set(Metric.create("a", 1, decay), Metric.create("b", 4, decay)).flatten)
val sample2 = NodeMetrics(node1, 2, Set(Metric.create("a", 2, decay), Metric.create("c", 5, decay)).flatten)
val sample3 = NodeMetrics(node1, 3, Set(Metric.create("a", 3, decay), Metric.create("d", 6, decay)).flatten)
val updated = sample1 update sample2 update sample3
updated.metrics.size should be(4)
updated.timestamp should be(sample3.timestamp)
updated.metric("a").map(_.value).get should be(3)
updated.metric("b").map(_.value).get should be(4)
updated.metric("c").map(_.value).get should be(5)
updated.metric("d").map(_.value).get should be(6)
updated.metric("a").map(_.smoothValue).get should be(1.512 +- epsilon)
updated.metric("b").map(_.smoothValue).get should be(4.000 +- epsilon)
updated.metric("c").map(_.smoothValue).get should be(5.000 +- epsilon)
updated.metric("d").map(_.smoothValue).get should be(6.000 +- epsilon)
}
}
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricsGossipSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with ImplicitSender with MetricsCollectorFactory {
val collector = createMetricsCollector
"A MetricsGossip" must {
"add new NodeMetrics" in {
val m1 = NodeMetrics(Address("akka.tcp", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka.tcp", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
m1.metrics.size should be > (3)
m2.metrics.size should be > (3)
val g1 = MetricsGossip.empty :+ m1
g1.nodes.size should be(1)
g1.nodeMetricsFor(m1.address).map(_.metrics) should be(Some(m1.metrics))
val g2 = g1 :+ m2
g2.nodes.size should be(2)
g2.nodeMetricsFor(m1.address).map(_.metrics) should be(Some(m1.metrics))
g2.nodeMetricsFor(m2.address).map(_.metrics) should be(Some(m2.metrics))
}
"merge peer metrics" in {
val m1 = NodeMetrics(Address("akka.tcp", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka.tcp", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size should be(2)
val beforeMergeNodes = g1.nodes
val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = m2.timestamp + 1000)
val g2 = g1 :+ m2Updated // merge peers
g2.nodes.size should be(2)
g2.nodeMetricsFor(m1.address).map(_.metrics) should be(Some(m1.metrics))
g2.nodeMetricsFor(m2.address).map(_.metrics) should be(Some(m2Updated.metrics))
g2.nodes collect { case peer if peer.address == m2.address peer.timestamp should be(m2Updated.timestamp) }
}
"merge an existing metric set for a node and update node ring" in {
val m1 = NodeMetrics(Address("akka.tcp", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka.tcp", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
val m3 = NodeMetrics(Address("akka.tcp", "sys", "a", 2556), newTimestamp, collector.sample.metrics)
val m2Updated = m2 copy (metrics = collector.sample.metrics, timestamp = m2.timestamp + 1000)
val g1 = MetricsGossip.empty :+ m1 :+ m2
val g2 = MetricsGossip.empty :+ m3 :+ m2Updated
g1.nodes.map(_.address) should be(Set(m1.address, m2.address))
// should contain nodes 1,3, and the most recent version of 2
val mergedGossip = g1 merge g2
mergedGossip.nodes.map(_.address) should be(Set(m1.address, m2.address, m3.address))
mergedGossip.nodeMetricsFor(m1.address).map(_.metrics) should be(Some(m1.metrics))
mergedGossip.nodeMetricsFor(m2.address).map(_.metrics) should be(Some(m2Updated.metrics))
mergedGossip.nodeMetricsFor(m3.address).map(_.metrics) should be(Some(m3.metrics))
mergedGossip.nodes.foreach(_.metrics.size should be > (3))
mergedGossip.nodeMetricsFor(m2.address).map(_.timestamp) should be(Some(m2Updated.timestamp))
}
"get the current NodeMetrics if it exists in the local nodes" in {
val m1 = NodeMetrics(Address("akka.tcp", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val g1 = MetricsGossip.empty :+ m1
g1.nodeMetricsFor(m1.address).map(_.metrics) should be(Some(m1.metrics))
}
"remove a node if it is no longer Up" in {
val m1 = NodeMetrics(Address("akka.tcp", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka.tcp", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size should be(2)
val g2 = g1 remove m1.address
g2.nodes.size should be(1)
g2.nodes.exists(_.address == m1.address) should be(false)
g2.nodeMetricsFor(m1.address) should be(None)
g2.nodeMetricsFor(m2.address).map(_.metrics) should be(Some(m2.metrics))
}
"filter nodes" in {
val m1 = NodeMetrics(Address("akka.tcp", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
val m2 = NodeMetrics(Address("akka.tcp", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size should be(2)
val g2 = g1 filter Set(m2.address)
g2.nodes.size should be(1)
g2.nodes.exists(_.address == m1.address) should be(false)
g2.nodeMetricsFor(m1.address) should be(None)
g2.nodeMetricsFor(m2.address).map(_.metrics) should be(Some(m2.metrics))
}
}
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricValuesSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory {
import akka.cluster.metrics.StandardMetrics._
val collector = createMetricsCollector
val node1 = NodeMetrics(Address("akka.tcp", "sys", "a", 2554), 1, collector.sample.metrics)
val node2 = NodeMetrics(Address("akka.tcp", "sys", "a", 2555), 1, collector.sample.metrics)
val nodes: Seq[NodeMetrics] = {
(1 to 100).foldLeft(List(node1, node2)) { (nodes, _)
nodes map { n
n.copy(metrics = collector.sample.metrics.flatMap(latest n.metrics.collect {
case streaming if latest sameAs streaming streaming :+ latest
}))
}
}
}
"NodeMetrics.MetricValues" must {
"extract expected metrics for load balancing" in {
val stream1 = node2.metric(HeapMemoryCommitted).get.value.longValue
val stream2 = node1.metric(HeapMemoryUsed).get.value.longValue
stream1 should be >= (stream2)
}
"extract expected MetricValue types for load balancing" in {
nodes foreach { node
node match {
case HeapMemory(address, _, used, committed, _)
used should be > (0L)
committed should be >= (used)
// Documentation java.lang.management.MemoryUsage says that committed <= max,
// but in practice that is not always true (we have seen it happen). Therefore
// we don't check the heap max value in this test.
// extract is the java api
StandardMetrics.extractHeapMemory(node) should not be (null)
}
node match {
case Cpu(address, _, systemLoadAverageOption, cpuCombinedOption, cpuStolenOption, processors)
processors should be > (0)
if (systemLoadAverageOption.isDefined)
systemLoadAverageOption.get should be >= (0.0)
if (cpuCombinedOption.isDefined) {
cpuCombinedOption.get should be <= (1.0)
cpuCombinedOption.get should be >= (0.0)
}
if (cpuStolenOption.isDefined) {
cpuStolenOption.get should be <= (1.0)
cpuStolenOption.get should be >= (0.0)
}
// extract is the java api
StandardMetrics.extractCpu(node) should not be (null)
}
}
}
}
}

View file

@ -0,0 +1,102 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import scala.language.postfixOps
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.Await
import scala.util.{ Success, Try, Failure }
import akka.actor._
import akka.testkit._
import akka.cluster.metrics.StandardMetrics._
import org.scalatest.WordSpec
import org.scalatest.Matchers
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MetricsCollectorSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with ImplicitSender with MetricsCollectorFactory {
import system.dispatcher
val collector = createMetricsCollector
"Metric must" must {
"merge 2 metrics that are tracking the same metric" in {
for (i 1 to 20) {
val sample1 = collector.sample.metrics
val sample2 = collector.sample.metrics
val merged12 = sample2 flatMap (latest sample1 collect {
case peer if latest sameAs peer
val m = peer :+ latest
m.value should be(latest.value)
m.isSmooth should be(peer.isSmooth || latest.isSmooth)
m
})
val sample3 = collector.sample.metrics
val sample4 = collector.sample.metrics
val merged34 = sample4 flatMap (latest sample3 collect {
case peer if latest sameAs peer
val m = peer :+ latest
m.value should be(latest.value)
m.isSmooth should be(peer.isSmooth || latest.isSmooth)
m
})
}
}
}
"MetricsCollector" must {
"not raise errors when attempting reflective code in apply" in {
Try(createMetricsCollector).get should not be null
}
"collect accurate metrics for a node" in {
val sample = collector.sample
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 (SystemLoadAverage, b) b.doubleValue should be >= (0.0)
case (Processors, b) b.intValue should be >= (0)
case (HeapMemoryUsed, b) b.longValue should be >= (0L)
case (HeapMemoryCommitted, b) b.longValue should be > (0L)
case (HeapMemoryMax, b)
b.longValue should be > (0L)
used.get.longValue should be <= (b.longValue)
committed.get.longValue should be <= (b.longValue)
case (CpuCombined, b)
b.doubleValue should be <= (1.0)
b.doubleValue should be >= (0.0)
case (CpuStolen, b)
b.doubleValue should be <= (1.0)
b.doubleValue should be >= (0.0)
}
}
"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]
val heap = c.heapMemoryUsage
c.heapUsed(heap).isDefined should be(true)
c.heapCommitted(heap).isDefined should be(true)
c.processors.isDefined should be(true)
}
"collect 50 node metrics samples in an acceptable duration" taggedAs LongRunningTest in within(10 seconds) {
(1 to 50) foreach { _
val sample = collector.sample
sample.metrics.size should be >= (3)
Thread.sleep(100)
}
}
}
}

View file

@ -0,0 +1,228 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics
import scala.language.postfixOps
import java.util.logging.LogManager
import org.slf4j.bridge.SLF4JBridgeHandler
import akka.testkit.AkkaSpec
import akka.actor.ExtendedActorSystem
import akka.actor.Address
import akka.cluster.MemberStatus
import akka.cluster.Member
import akka.cluster.UniqueAddress
import akka.cluster.Cluster
import java.io.Closeable
import akka.actor.ActorRef
import akka.actor.Props
import akka.actor.Actor
import akka.dispatch.RequiresMessageQueue
import akka.actor.Deploy
import akka.dispatch.UnboundedMessageQueueSemantics
import akka.actor.PoisonPill
import akka.actor.ActorLogging
import org.scalatest.mock.MockitoSugar
import akka.actor.ActorSystem
import akka.dispatch.Dispatchers
/**
* Redirect different logging sources to SLF4J.
*/
trait RedirectLogging {
def redirectLogging(): Unit = {
// Redirect JUL to SLF4J.
LogManager.getLogManager().reset()
SLF4JBridgeHandler.install()
}
redirectLogging()
}
/**
* Provide sigar library from `project/target` location.
*/
case class SimpleSigarProvider(location: String = "native") extends SigarProvider {
def extractFolder = s"${System.getProperty("user.dir")}/target/${location}"
}
/**
* Provide sigar library as static mock.
*/
case class MockitoSigarProvider(
pid: Long = 123,
loadAverage: Array[Double] = Array(0.7, 0.3, 0.1),
cpuCombined: Double = 0.5,
cpuStolen: Double = 0.2,
steps: Int = 5) extends SigarProvider with MockitoSugar {
import org.hyperic.sigar._
import org.mockito.Mockito._
/** Not used. */
override def extractFolder = ???
/** Generate monotonic array from 0 to value. */
def increase(value: Double): Array[Double] = {
val delta = value / steps
(0 to steps) map { _ * delta } toArray
}
/** Sigar mock instance. */
override def verifiedSigarInstance = {
// Note "thenReturn(0)" invocation is consumed in collector construction.
val cpuPerc = mock[CpuPerc]
when(cpuPerc.getCombined) thenReturn (0, increase(cpuCombined): _*)
when(cpuPerc.getStolen) thenReturn (0, increase(cpuStolen): _*)
val sigar = mock[SigarProxy]
when(sigar.getPid) thenReturn pid
when(sigar.getLoadAverage) thenReturn loadAverage // Constant.
when(sigar.getCpuPerc) thenReturn cpuPerc // Increasing.
sigar
}
}
/**
* Used when testing metrics without full cluster
*
* TODO change factory after https://github.com/akka/akka/issues/16369
*/
trait MetricsCollectorFactory { this: AkkaSpec
import MetricsConfig._
import org.hyperic.sigar.Sigar
private def extendedActorSystem = system.asInstanceOf[ExtendedActorSystem]
def selfAddress = extendedActorSystem.provider.rootPath.address
def createMetricsCollector: MetricsCollector =
try {
new SigarMetricsCollector(selfAddress, defaultDecayFactor, new Sigar())
//new SigarMetricsCollector(selfAddress, defaultDecayFactor, SimpleSigarProvider().createSigarInstance)
} catch {
case e: Throwable
log.warning("Sigar failed to load. Using JMX. Reason: " + e.toString)
new JmxMetricsCollector(selfAddress, defaultDecayFactor)
}
/** Create JMX collector. */
def collectorJMX: MetricsCollector =
new JmxMetricsCollector(selfAddress, defaultDecayFactor)
/** Create Sigar collector. Rely on java agent injection. */
def collectorSigarDefault: MetricsCollector =
new SigarMetricsCollector(selfAddress, defaultDecayFactor, new Sigar())
/** Create Sigar collector. Rely on sigar-loader provisioner. */
def collectorSigarProvision: MetricsCollector =
new SigarMetricsCollector(selfAddress, defaultDecayFactor, SimpleSigarProvider().createSigarInstance)
/** Create Sigar collector. Rely on static sigar library mock. */
def collectorSigarMockito: MetricsCollector =
new SigarMetricsCollector(selfAddress, defaultDecayFactor, MockitoSigarProvider().createSigarInstance)
def isSigar(collector: MetricsCollector): Boolean = collector.isInstanceOf[SigarMetricsCollector]
}
/**
*
*/
class MockitoSigarMetricsCollector(system: ActorSystem)
extends SigarMetricsCollector(
Address("akka.tcp", system.name),
MetricsConfig.defaultDecayFactor,
MockitoSigarProvider().createSigarInstance) {
}
/**
* Metrics test configurations.
*/
object MetricsConfig {
val defaultDecayFactor = 2.0 / (1 + 10)
/** Test w/o cluster, with collection enabled. */
val defaultEnabled = """
akka.cluster.metrics {
collector {
enabled = on
sample-interval = 1s
gossip-interval = 1s
}
}
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
"""
/** Test w/o cluster, with collection disabled. */
val defaultDisabled = """
akka.cluster.metrics {
collector {
enabled = off
}
}
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
"""
/** Test in cluster, with manual collection activation, collector mock, fast. */
val clusterSigarMock = """
akka.cluster.metrics {
periodic-tasks-initial-delay = 100ms
collector {
enabled = off
sample-interval = 200ms
gossip-interval = 200ms
provider = "akka.cluster.metrics.MockitoSigarMetricsCollector"
fallback = false
}
}
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
"""
}
/**
* Current cluster metrics, updated periodically via event bus.
*/
class ClusterMetricsView(system: ExtendedActorSystem) extends Closeable {
val extension = ClusterMetricsExtension(system)
/** Current cluster metrics, updated periodically via event bus. */
@volatile
private var currentMetricsSet: Set[NodeMetrics] = Set.empty
/** Collected cluster metrics history. */
@volatile
private var collectedMetricsList: List[Set[NodeMetrics]] = List.empty
/** Create actor that subscribes to the cluster eventBus to update current read view state. */
private val eventBusListener: ActorRef = {
system.systemActorOf(Props(new Actor with ActorLogging with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
override def preStart(): Unit = extension.subscribe(self)
override def postStop(): Unit = extension.unsubscribe(self)
def receive = {
case ClusterMetricsChanged(nodes)
currentMetricsSet = nodes
collectedMetricsList = nodes :: collectedMetricsList
case _
// Ignore.
}
}).withDispatcher(Dispatchers.DefaultDispatcherId).withDeploy(Deploy.local), name = "metrics-event-bus-listener")
}
/** Current cluster metrics. */
def clusterMetrics: Set[NodeMetrics] = currentMetricsSet
/** Collected cluster metrics history. */
def metricsHistory: List[Set[NodeMetrics]] = collectedMetricsList
/** Unsubscribe from cluster events. */
def close(): Unit = eventBusListener ! PoisonPill
}

View file

@ -0,0 +1,60 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.cluster.metrics.protobuf
import akka.actor.{ ExtendedActorSystem, Address }
import collection.immutable.SortedSet
import akka.testkit.AkkaSpec
import java.math.BigInteger
import akka.cluster.MemberStatus
import akka.cluster.metrics.MetricsGossip
import akka.cluster.metrics.NodeMetrics
import akka.cluster.metrics.Metric
import akka.cluster.metrics.EWMA
import akka.cluster.TestMember
import akka.cluster.metrics.MetricsGossipEnvelope
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class MessageSerializerSpec extends AkkaSpec(
"akka.actor.provider = akka.cluster.ClusterActorRefProvider") {
val serializer = new MessageSerializer(system.asInstanceOf[ExtendedActorSystem])
def checkSerialization(obj: AnyRef): Unit = {
val blob = serializer.toBinary(obj)
val ref = serializer.fromBinary(blob, obj.getClass)
obj match {
case _
ref should be(obj)
}
}
import MemberStatus._
val a1 = TestMember(Address("akka.tcp", "sys", "a", 2552), Joining, Set.empty)
val b1 = TestMember(Address("akka.tcp", "sys", "b", 2552), Up, Set("r1"))
val c1 = TestMember(Address("akka.tcp", "sys", "c", 2552), Leaving, Set("r2"))
val d1 = TestMember(Address("akka.tcp", "sys", "d", 2552), Exiting, Set("r1", "r2"))
val e1 = TestMember(Address("akka.tcp", "sys", "e", 2552), Down, Set("r3"))
val f1 = TestMember(Address("akka.tcp", "sys", "f", 2552), Removed, Set("r2", "r3"))
"ClusterMessages" must {
"be serializable" in {
val metricsGossip = MetricsGossip(Set(NodeMetrics(a1.address, 4711, Set(Metric("foo", 1.2, None))),
NodeMetrics(b1.address, 4712, Set(Metric("foo", 2.1, Some(EWMA(value = 100.0, alpha = 0.18))),
Metric("bar1", Double.MinPositiveValue, None),
Metric("bar2", Float.MaxValue, None),
Metric("bar3", Int.MaxValue, None),
Metric("bar4", Long.MaxValue, None),
Metric("bar5", BigInt(Long.MaxValue), None)))))
checkSerialization(MetricsGossipEnvelope(a1.address, metricsGossip, true))
}
}
}

View file

@ -3,6 +3,8 @@
*/
package akka.cluster
// TODO remove metrics
import language.postfixOps
import scala.collection.immutable
import scala.collection.immutable.VectorBuilder
@ -190,6 +192,7 @@ object ClusterEvent {
/**
* Current snapshot of cluster node metrics. Published to subscribers.
*/
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
final case class ClusterMetricsChanged(nodeMetrics: Set[NodeMetrics]) extends ClusterDomainEvent {
/**
* Java API

View file

@ -4,6 +4,8 @@
package akka.cluster
// TODO remove metrics
import java.io.Closeable
import java.lang.System.{ currentTimeMillis newTimestamp }
import java.lang.management.{ OperatingSystemMXBean, MemoryMXBean, ManagementFactory }
@ -302,6 +304,7 @@ private[cluster] final case class EWMA(value: Double, alpha: Double) {
* averages (e.g. system load average) or finite (e.g. as number of processors), are not trended.
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
final case class Metric private[cluster] (name: String, value: Number, private[cluster] val average: Option[EWMA])
extends MetricNumericConverter {
@ -348,6 +351,7 @@ final case class Metric private[cluster] (name: String, value: Number, private[c
/**
* Factory for creating valid Metric instances.
*/
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
object Metric extends MetricNumericConverter {
/**
@ -385,6 +389,7 @@ object Metric extends MetricNumericConverter {
* @param metrics the set of sampled [[akka.actor.Metric]]
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Metric] = Set.empty[Metric]) {
/**
@ -426,6 +431,7 @@ final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Met
* The following extractors and data structures makes it easy to consume the
* [[akka.cluster.NodeMetrics]] in for example load balancers.
*/
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
object StandardMetrics {
// Constants for the heap related Metric names
@ -577,6 +583,7 @@ private[cluster] trait MetricNumericConverter {
/**
* Implementations of cluster system metrics extends this trait.
*/
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
trait MetricsCollector extends Closeable {
/**
* Samples and collects new data points.
@ -592,6 +599,7 @@ trait MetricsCollector extends Closeable {
* @param address The [[akka.actor.Address]] of the node being sampled
* @param decay how quickly the exponential weighting of past data is decayed
*/
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
class JmxMetricsCollector(address: Address, decayFactor: Double) extends MetricsCollector {
import StandardMetrics._
@ -692,6 +700,7 @@ class JmxMetricsCollector(address: Address, decayFactor: Double) extends Metrics
* @param decay how quickly the exponential weighting of past data is decayed
* @param sigar the org.hyperic.Sigar instance
*/
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
class SigarMetricsCollector(address: Address, decayFactor: Double, sigar: AnyRef)
extends JmxMetricsCollector(address, decayFactor) {

View file

@ -4,6 +4,8 @@
package akka.cluster
// TODO remove metrics
import java.io.Closeable
import scala.collection.immutable
import akka.actor.{ Actor, ActorRef, ActorSystemImpl, Address, Props }

View file

@ -3,6 +3,8 @@
*/
package akka.cluster.routing
// TODO remove metrics
import java.util.Arrays
import java.util.concurrent.atomic.AtomicReference
@ -40,6 +42,7 @@ import akka.routing._
* @param metricsSelector decides what probability to use for selecting a routee, based
* on remaining capacity as indicated by the node metrics
*/
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsSelector: MetricsSelector = MixMetricsSelector)
extends RoutingLogic with NoSerializationVerificationNeeded {
@ -126,6 +129,7 @@ final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsS
* supervision, death watch and router management messages
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
final case class AdaptiveLoadBalancingPool(
metricsSelector: MetricsSelector = MixMetricsSelector,
val nrOfInstances: Int = 0,
@ -206,6 +210,7 @@ final case class AdaptiveLoadBalancingPool(
* router management messages
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
final case class AdaptiveLoadBalancingGroup(
metricsSelector: MetricsSelector = MixMetricsSelector,
paths: immutable.Iterable[String] = Nil,
@ -246,6 +251,7 @@ final case class AdaptiveLoadBalancingGroup(
* Low heap capacity => small weight.
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
case object HeapMetricsSelector extends CapacityMetricsSelector {
/**
* Java API: get the singleton instance
@ -270,6 +276,7 @@ case object HeapMetricsSelector extends CapacityMetricsSelector {
* Low cpu capacity => small weight.
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
case object CpuMetricsSelector extends CapacityMetricsSelector {
/**
* Java API: get the singleton instance
@ -293,6 +300,7 @@ case object CpuMetricsSelector extends CapacityMetricsSelector {
* Low load average capacity => small weight.
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
case object SystemLoadAverageMetricsSelector extends CapacityMetricsSelector {
/**
* Java API: get the singleton instance
@ -313,6 +321,7 @@ case object SystemLoadAverageMetricsSelector extends CapacityMetricsSelector {
* [akka.cluster.routing.CpuMetricsSelector], and [akka.cluster.routing.SystemLoadAverageMetricsSelector]
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
object MixMetricsSelector extends MixMetricsSelectorBase(
Vector(HeapMetricsSelector, CpuMetricsSelector, SystemLoadAverageMetricsSelector)) {
@ -328,6 +337,7 @@ object MixMetricsSelector extends MixMetricsSelectorBase(
* [akka.cluster.routing.CpuMetricsSelector], and [akka.cluster.routing.SystemLoadAverageMetricsSelector]
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
final case class MixMetricsSelector(
selectors: immutable.IndexedSeq[CapacityMetricsSelector])
extends MixMetricsSelectorBase(selectors)
@ -336,6 +346,7 @@ final case class MixMetricsSelector(
* Base class for MetricsSelector that combines other selectors and aggregates their capacity.
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMetricsSelector])
extends CapacityMetricsSelector {
@ -358,6 +369,7 @@ abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMe
}
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
object MetricsSelector {
def fromConfig(config: Config, dynamicAccess: DynamicAccess) =
config.getString("metrics-selector") match {
@ -380,6 +392,7 @@ object MetricsSelector {
* A MetricsSelector is responsible for producing weights from the node metrics.
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
trait MetricsSelector extends Serializable {
/**
* The weights per address, based on the the nodeMetrics.
@ -392,6 +405,7 @@ trait MetricsSelector extends Serializable {
* The weights are typically proportional to the remaining capacity.
*/
@SerialVersionUID(1L)
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
abstract class CapacityMetricsSelector extends MetricsSelector {
/**

View file

@ -4,6 +4,8 @@
package akka.cluster
// TODO remove metrics
import akka.remote.testkit.{ MultiNodeSpec, MultiNodeConfig }
import com.typesafe.config.ConfigFactory
import akka.testkit.LongRunningTest

View file

@ -4,6 +4,8 @@
package akka.cluster
// TODO remove metrics
import scala.language.postfixOps
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory

View file

@ -3,6 +3,8 @@
*/
package akka.cluster
// TODO remove metrics
import language.implicitConversions
import org.scalatest.{ Suite, Outcome, Canceled }
import org.scalatest.exceptions.TestCanceledException

View file

@ -3,6 +3,9 @@
*/
package akka.cluster
// TODO remove metrics
// FIXME this test is not migrated to metrics extension
import language.postfixOps
import scala.annotation.tailrec
import scala.collection.immutable

View file

@ -4,6 +4,8 @@
package akka.cluster.routing
// TODO remove metrics
import language.postfixOps
import java.lang.management.ManagementFactory
import scala.concurrent.Await

View file

@ -47,6 +47,7 @@ class ClusterConfigSpec extends AkkaSpec {
ReduceGossipDifferentViewProbability should be(400)
SchedulerTickDuration should be(33 millis)
SchedulerTicksPerWheel should be(512)
// TODO remove metrics
MetricsEnabled should be(true)
MetricsCollectorClass should be(classOf[SigarMetricsCollector].getName)
MetricsInterval should be(3 seconds)

View file

@ -4,6 +4,8 @@
package akka.cluster
// TODO remove metrics
import language.postfixOps
import scala.concurrent.duration._
import akka.testkit.{ LongRunningTest, AkkaSpec }

View file

@ -4,6 +4,8 @@
package akka.cluster
// TODO remove metrics
import org.scalatest.WordSpec
import org.scalatest.Matchers
import akka.cluster.StandardMetrics._

View file

@ -4,6 +4,8 @@
package akka.cluster
// TODO remove metrics
import scala.util.Try
import akka.actor.Address
import akka.testkit.AkkaSpec

View file

@ -5,6 +5,8 @@
package akka.cluster
// TODO remove metrics
import scala.language.postfixOps
import scala.collection.immutable

View file

@ -4,6 +4,8 @@
package akka.cluster
// TODO remove metrics
import scala.concurrent.duration._
import akka.testkit.{ ImplicitSender, AkkaSpec }

View file

@ -4,6 +4,8 @@
package akka.cluster
// TODO remove metrics
import org.scalatest.WordSpec
import org.scalatest.Matchers
import akka.actor.Address

View file

@ -3,6 +3,8 @@
*/
package akka.cluster.protobuf
// TODO remove metrics
import akka.cluster._
import akka.actor.{ ExtendedActorSystem, Address }
import collection.immutable.SortedSet

View file

@ -4,6 +4,8 @@
package akka.cluster.routing
// TODO remove metrics
import org.scalatest.WordSpec
import org.scalatest.Matchers

View file

@ -0,0 +1,162 @@
.. _cluster_metrics_java:
Cluster Metrics Extension
=========================
Introduction
------------
The member nodes of the cluster can collect system health metrics and publish that to other cluster nodes
and to the registered subscribers on the system event bus with the help of Cluster Metrics Extension.
Cluster metrics information is primarily used for load-balancing routers,
and can also be used to implement advanced metrics-based node life cycles,
such as "Node Let-it-crash" when CPU steal time becomes excessive.
Cluster Metrics Extension is a separate akka module delivered in ``akka-cluster-metrics`` jar.
To enable usage of the extension you need to add the following dependency to your project:
::
<dependency>
<groupId>com.typesafe.akka</groupId>
<artifactId>akka-cluster-metrics_@binVersion@</artifactId>
<version>@version@</version>
</dependency>
and add the following configuration stanza to your ``application.conf``
::
akka.extensions = [ "akka.cluster.metrics.ClusterMetricsExtension" ]
Make sure to disable legacy metrics in akka-cluster: ``akka.cluster.metrics.enabled=off``,
since it is still enabled in akka-cluster by default (for compatibility with past releases).
Metrics Collector
-----------------
Metrics collection is delegated to an implementation of ``akka.cluster.metrics.MetricsCollector``.
Different collector implementations provide different subsets of metrics published to the cluster.
Certain message routing and let-it-crash functions may not work when Sigar is not provisioned.
Cluster metrics extension comes with two built-in collector implementations:
#. ``akka.cluster.metrics.SigarMetricsCollector``, which requires Sigar provisioning, and is more rich/precise
#. ``akka.cluster.metrics.JmxMetricsCollector``, which is used as fall back, and is less rich/precise
You can also plug-in your own metrics collector implementation.
By default, metrics extension will use collector provider fall back and will try to load them in this order:
#. configured user-provided collector
#. built-in ``akka.cluster.metrics.SigarMetricsCollector``
#. and finally ``akka.cluster.metrics.JmxMetricsCollector``
Metrics Events
--------------
Metrics extension periodically publishes current snapshot of the cluster metrics to the node system event bus.
The publication period is controlled by the ``akka.cluster.metrics.collector.sample-period`` setting.
The payload of the ``akka.cluster.metris.ClusterMetricsChanged`` event will contain
latest metrics of the node as well as other cluster member nodes metrics gossip
which was received during the collector sample period.
You can subscribe your metrics listener actors to these events in order to implement custom node lifecycle
::
ClusterMetricsExtension.get(system).subscribe(metricsListenerActor);
Hyperic Sigar Provisioning
--------------------------
Both user-provided and built-in metrics collectors can optionally use `Hyperic Sigar <http://www.hyperic.com/products/sigar>`_
for a wider and more accurate range of metrics compared to what can be retrieved from ordinary JMX MBeans.
Sigar is using a native o/s library, and requires library provisioning, i.e.
deployment, extraction and loading of the o/s native library into JVM at runtime.
User can provision Sigar classes and native library in one of the following ways:
#. Use `Kamon sigar-loader <https://github.com/kamon-io/sigar-loader>`_ as a project dependency for the user project.
Metrics extension will extract and load sigar library on demand with help of Kamon sigar provisioner.
#. Use `Kamon sigar-loader <https://github.com/kamon-io/sigar-loader>`_ as java agent: ``java -javaagent:/path/to/sigar-loader.jar``.
Kamon sigar loader agent will extract and load sigar library during JVM start.
#. Place ``sigar.jar`` on the ``classpath`` and Sigar native library for the o/s on the ``java.library.path``.
User is required to manage both project dependency and library deployment manually.
To enable usage of Sigar you can add the following dependency to the user project
::
<dependency>
<groupId>io.kamon</groupId>
<artifactId>sigar-loader</artifactId>
<version>@sigarLoaderVersion@</version>
</dependency>
You can download Kamon sigar-loader from `Maven Central <http://search.maven.org/#search%7Cga%7C1%7Csigar-loader>`_
Adaptive Load Balancing
-----------------------
The ``AdaptiveLoadBalancingPool`` / ``AdaptiveLoadBalancingGroup`` 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 the 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.metrics.MetricsSelector``
The collected metrics values are smoothed with `exponential weighted moving average <http://en.wikipedia.org/wiki/Moving_average#Exponential_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. What can be more demanding than calculating factorials?
The backend worker that performs the factorial calculation:
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/java/sample/cluster/factorial/FactorialBackend.java#backend
The frontend that receives user jobs and delegates to the backends via the router:
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/java/sample/cluster/factorial/FactorialFrontend.java#frontend
As you can see, the router is defined in the same way as other routers, and in this case it is configured as follows:
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/resources/factorial.conf#adaptive-router
It is only ``router`` type and the ``metrics-selector`` parameter 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-java/src/main/java/sample/cluster/factorial/Extra.java#router-lookup-in-code
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/java/sample/cluster/factorial/Extra.java#router-deploy-in-code
The `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_ tutorial named
`Akka Cluster Samples with Java <http://www.typesafe.com/activator/template/akka-sample-cluster-java>`_.
contains the full source code and instructions of how to run the **Adaptive Load Balancing** sample.
Subscribe to Metrics Events
---------------------------
It is possible to subscribe to the metrics events directly to implement other functionality.
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/java/sample/cluster/factorial/MetricsListener.java#metrics-listener
Custom Metrics Collector
------------------------
Metrics collection is delegated to the implementation of ``akka.cluster.metrics.MetricsCollector``
You can plug-in your own metrics collector instead of built-in
``akka.cluster.metrics.SigarMetricsCollector`` or ``akka.cluster.metrics.JmxMetricsCollector``.
Look at those two implementations for inspiration.
Custom metrics collector implementation class must be specified in the :ref:`cluster_metrics_configuration_java`.

View file

@ -536,81 +536,9 @@ contains the full source code and instructions of how to run the **Router Exampl
Cluster Metrics
^^^^^^^^^^^^^^^
The member nodes of the cluster collects system health metrics and publishes that to other nodes and to
registered subscribers. This information is primarily used for load-balancing routers.
The member nodes of the cluster can collect system health metrics and publish that to other cluster nodes
and to the registered subscribers on the system event bus with the help of :doc:`cluster-metrics`.
Hyperic Sigar
-------------
The built-in metrics is gathered from JMX MBeans, and optionally you can use `Hyperic Sigar <http://www.hyperic.com/products/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=<path_of_sigar_libs>`` add the following dependency::
<dependency>
<groupId>org.fusesource</groupId>
<artifactId>sigar</artifactId>
<version>@sigarVersion@</version>
</dependency>
Download the native Sigar libraries from `Maven Central <http://repo1.maven.org/maven2/org/fusesource/sigar/@sigarVersion@/>`_
Adaptive Load Balancing
-----------------------
The ``AdaptiveLoadBalancingPool`` / ``AdaptiveLoadBalancingGroup`` 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 the 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 <http://en.wikipedia.org/wiki/Moving_average#Exponential_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. What can be more demanding than calculating factorials?
The backend worker that performs the factorial calculation:
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/java/sample/cluster/factorial/FactorialBackend.java#backend
The frontend that receives user jobs and delegates to the backends via the router:
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/java/sample/cluster/factorial/FactorialFrontend.java#frontend
As you can see, the router is defined in the same way as other routers, and in this case it is configured as follows:
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/resources/factorial.conf#adaptive-router
It is 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-java/src/main/java/sample/cluster/factorial/Extra.java#router-lookup-in-code
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/java/sample/cluster/factorial/Extra.java#router-deploy-in-code
The `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_ tutorial named
`Akka Cluster Samples with Java <http://www.typesafe.com/activator/template/akka-sample-cluster-java>`_.
contains the full source code and instructions of how to run the **Adaptive Load Balancing** sample.
Subscribe to Metrics Events
---------------------------
It is possible to subscribe to the metrics events directly to implement other functionality.
.. includecode:: ../../../akka-samples/akka-sample-cluster-java/src/main/java/sample/cluster/factorial/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:

View file

@ -6,6 +6,7 @@ Networking
../common/cluster
cluster-usage
cluster-metrics
remoting
serialization
io

View file

@ -165,3 +165,17 @@ Secure Cookies
`Secure cookies` feature was deprecated.
New Cluster Metrics Extension
=============================
Previously, cluster metrics functionality was located in the ``akka-cluster`` jar.
Now it is split out and moved into a separate akka module: ``akka-cluster-metrics`` jar.
The module comes with few enhancements, such as use of Kamon sigar-loader
for native library provisioning as well as use of statistical averaging of metrics data.
Note that both old and new metrics configuration entries in the ``reference.conf``
are still in the same name space ``akka.cluster.metrics`` but are not compatible.
Make sure to disable legacy metrics in akka-cluster: ``akka.cluster.metrics.enabled=off``,
since it is still enabled in akka-cluster by default (for compatibility with past releases).
Router configuration entries have also changed for the module, they use prefix ``cluster-metrics-``:
``cluster-metrics-adaptive-pool`` and ``cluster-metrics-adaptive-group``
Metrics extension classes and objects are located in the new package ``akka.cluster.metrics``.
Please see :ref:`Scala <cluster-metrics-scala>`, :ref:`Java <cluster-metrics-java>` for more information.

View file

@ -0,0 +1,155 @@
.. _cluster_metrics_scala:
Cluster Metrics Extension
=========================
Introduction
------------
The member nodes of the cluster can collect system health metrics and publish that to other cluster nodes
and to the registered subscribers on the system event bus with the help of Cluster Metrics Extension.
Cluster metrics information is primarily used for load-balancing routers,
and can also be used to implement advanced metrics-based node life cycles,
such as "Node Let-it-crash" when CPU steal time becomes excessive.
Cluster Metrics Extension is a separate akka module delivered in ``akka-cluster-metrics`` jar.
To enable usage of the extension you need to add the following dependency to your project:
::
"com.typesafe.akka" % "akka-cluster-metrics_@binVersion@" % "@version@"
and add the following configuration stanza to your ``application.conf``
::
akka.extensions = [ "akka.cluster.metrics.ClusterMetricsExtension" ]
Make sure to disable legacy metrics in akka-cluster: ``akka.cluster.metrics.enabled=off``,
since it is still enabled in akka-cluster by default (for compatibility with past releases).
Metrics Collector
-----------------
Metrics collection is delegated to an implementation of ``akka.cluster.metrics.MetricsCollector``.
Different collector implementations provide different subsets of metrics published to the cluster.
Certain message routing and let-it-crash functions may not work when Sigar is not provisioned.
Cluster metrics extension comes with two built-in collector implementations:
#. ``akka.cluster.metrics.SigarMetricsCollector``, which requires Sigar provisioning, and is more rich/precise
#. ``akka.cluster.metrics.JmxMetricsCollector``, which is used as fall back, and is less rich/precise
You can also plug-in your own metrics collector implementation.
By default, metrics extension will use collector provider fall back and will try to load them in this order:
#. configured user-provided collector
#. built-in ``akka.cluster.metrics.SigarMetricsCollector``
#. and finally ``akka.cluster.metrics.JmxMetricsCollector``
Metrics Events
--------------
Metrics extension periodically publishes current snapshot of the cluster metrics to the node system event bus.
The publication period is controlled by the ``akka.cluster.metrics.collector.sample-period`` setting.
The payload of the ``akka.cluster.metris.ClusterMetricsChanged`` event will contain
latest metrics of the node as well as other cluster member nodes metrics gossip
which was received during the collector sample period.
You can subscribe your metrics listener actors to these events in order to implement custom node lifecycle
::
ClusterMetricsExtension(system).subscribe(metricsListenerActor)
Hyperic Sigar Provisioning
--------------------------
Both user-provided and built-in metrics collectors can optionally use `Hyperic Sigar <http://www.hyperic.com/products/sigar>`_
for a wider and more accurate range of metrics compared to what can be retrieved from ordinary JMX MBeans.
Sigar is using a native o/s library, and requires library provisioning, i.e.
deployment, extraction and loading of the o/s native library into JVM at runtime.
User can provision Sigar classes and native library in one of the following ways:
#. Use `Kamon sigar-loader <https://github.com/kamon-io/sigar-loader>`_ as a project dependency for the user project.
Metrics extension will extract and load sigar library on demand with help of Kamon sigar provisioner.
#. Use `Kamon sigar-loader <https://github.com/kamon-io/sigar-loader>`_ as java agent: ``java -javaagent:/path/to/sigar-loader.jar``.
Kamon sigar loader agent will extract and load sigar library during JVM start.
#. Place ``sigar.jar`` on the ``classpath`` and Sigar native library for the o/s on the ``java.library.path``.
User is required to manage both project dependency and library deployment manually.
To enable usage of Sigar you can add the following dependency to the user project
::
"io.kamon" % "sigar-loader" % "@sigarLoaderVersion@"
You can download Kamon sigar-loader from `Maven Central <http://search.maven.org/#search%7Cga%7C1%7Csigar-loader>`_
Adaptive Load Balancing
-----------------------
The ``AdaptiveLoadBalancingPool`` / ``AdaptiveLoadBalancingGroup`` 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 the 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.metrics.MetricsSelector``
The collected metrics values are smoothed with `exponential weighted moving average <http://en.wikipedia.org/wiki/Moving_average#Exponential_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. What can be more demanding than calculating factorials?
The backend worker that performs the factorial calculation:
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala#backend
The frontend that receives user jobs and delegates to the backends via the router:
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialFrontend.scala#frontend
As you can see, the router is defined in the same way as other routers, and in this case it is configured as follows:
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/resources/factorial.conf#adaptive-router
It is only ``router`` type and the ``metrics-selector`` parameter 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-scala/src/main/scala/sample/cluster/factorial/Extra.scala#router-lookup-in-code
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/Extra.scala#router-deploy-in-code
The `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_ tutorial named
`Akka Cluster Samples with Scala <http://www.typesafe.com/activator/template/akka-sample-cluster-scala>`_.
contains the full source code and instructions of how to run the **Adaptive Load Balancing** sample.
Subscribe to Metrics Events
---------------------------
It is possible to subscribe to the metrics events directly to implement other functionality.
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/MetricsListener.scala#metrics-listener
Custom Metrics Collector
------------------------
Metrics collection is delegated to the implementation of ``akka.cluster.metrics.MetricsCollector``
You can plug-in your own metrics collector instead of built-in
``akka.cluster.metrics.SigarMetricsCollector`` or ``akka.cluster.metrics.JmxMetricsCollector``.
Look at those two implementations for inspiration.
Custom metrics collector implementation class must be specified in the :ref:`cluster_metrics_configuration_scala`.

View file

@ -532,77 +532,9 @@ contains the full source code and instructions of how to run the **Router Exampl
Cluster Metrics
^^^^^^^^^^^^^^^
The member nodes of the cluster collects system health metrics and publishes that to other nodes and to
registered subscribers. This information is primarily used for load-balancing routers.
The member nodes of the cluster can collect system health metrics and publish that to other cluster nodes
and to the registered subscribers on the system event bus with the help of :doc:`cluster-metrics`.
Hyperic Sigar
-------------
The built-in metrics is gathered from JMX MBeans, and optionally you can use `Hyperic Sigar <http://www.hyperic.com/products/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=<path_of_sigar_libs>`` add the following dependency::
"org.fusesource" % "sigar" % "@sigarVersion@"
Download the native Sigar libraries from `Maven Central <http://repo1.maven.org/maven2/org/fusesource/sigar/@sigarVersion@/>`_
Adaptive Load Balancing
-----------------------
The ``AdaptiveLoadBalancingPool`` / ``AdaptiveLoadBalancingGroup`` 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 the 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 <http://en.wikipedia.org/wiki/Moving_average#Exponential_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. What can be more demanding than calculating factorials?
The backend worker that performs the factorial calculation:
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialBackend.scala#backend
The frontend that receives user jobs and delegates to the backends via the router:
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/FactorialFrontend.scala#frontend
As you can see, the router is defined in the same way as other routers, and in this case it is configured as follows:
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/resources/factorial.conf#adaptive-router
It is 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-scala/src/main/scala/sample/cluster/factorial/Extra.scala#router-lookup-in-code
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/Extra.scala#router-deploy-in-code
The `Typesafe Activator <http://www.typesafe.com/platform/getstarted>`_ tutorial named
`Akka Cluster Samples with Scala <http://www.typesafe.com/activator/template/akka-sample-cluster-scala>`_.
contains the full source code and instructions of how to run the **Adaptive Load Balancing** sample.
Subscribe to Metrics Events
---------------------------
It is possible to subscribe to the metrics events directly to implement other functionality.
.. includecode:: ../../../akka-samples/akka-sample-cluster-scala/src/main/scala/sample/cluster/factorial/MetricsListener.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
^^^^^^^^^^^

View file

@ -291,4 +291,4 @@ trait PersistenceDocSpec {
//#view-update
}
}
}

View file

@ -6,6 +6,7 @@ Networking
../common/cluster
cluster-usage
cluster-metrics
remoting
serialization
io

View file

@ -14,4 +14,5 @@ target/
.cache
.classpath
.project
.settings
.settings
native/

View file

@ -9,7 +9,7 @@ val project = Project(
settings = Project.defaultSettings ++ SbtMultiJvm.multiJvmSettings ++ Seq(
name := "akka-sample-cluster-java",
version := "2.4-SNAPSHOT",
scalaVersion := "2.10.4",
scalaVersion := "2.11.5",
scalacOptions in Compile ++= Seq("-encoding", "UTF-8", "-target:jvm-1.6", "-deprecation", "-feature", "-unchecked", "-Xlog-reflective-calls", "-Xlint"),
javacOptions in Compile ++= Seq("-source", "1.6", "-target", "1.6", "-Xlint:unchecked", "-Xlint:deprecation"),
javacOptions in doc in Compile := Seq("-source", "1.6"), // javadoc does not support -target and -Xlint flags
@ -17,12 +17,12 @@ val project = Project(
"com.typesafe.akka" %% "akka-actor" % akkaVersion,
"com.typesafe.akka" %% "akka-remote" % akkaVersion,
"com.typesafe.akka" %% "akka-cluster" % akkaVersion,
"com.typesafe.akka" %% "akka-cluster-metrics" % akkaVersion,
"com.typesafe.akka" %% "akka-contrib" % akkaVersion,
"com.typesafe.akka" %% "akka-multi-node-testkit" % akkaVersion,
"org.scalatest" %% "scalatest" % "2.2.1" % "test",
"org.fusesource" % "sigar" % "1.6.4"),
"io.kamon" % "sigar-loader" % "1.6.5-rev001"),
javaOptions in run ++= Seq(
"-Djava.library.path=./sigar",
"-Xms128m", "-Xmx1024m"),
Keys.fork in run := true,
mainClass in (Compile, run) := Some("sample.cluster.simple.SimpleClusterApp"),

View file

@ -6,14 +6,14 @@ import java.util.Collections;
import akka.actor.ActorRef;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.cluster.routing.AdaptiveLoadBalancingGroup;
import akka.cluster.routing.AdaptiveLoadBalancingPool;
import akka.cluster.metrics.AdaptiveLoadBalancingGroup;
import akka.cluster.metrics.AdaptiveLoadBalancingPool;
import akka.cluster.routing.ClusterRouterGroup;
import akka.cluster.routing.ClusterRouterGroupSettings;
import akka.cluster.routing.ClusterRouterPool;
import akka.cluster.routing.ClusterRouterPoolSettings;
import akka.cluster.routing.HeapMetricsSelector;
import akka.cluster.routing.SystemLoadAverageMetricsSelector;
import akka.cluster.metrics.HeapMetricsSelector;
import akka.cluster.metrics.SystemLoadAverageMetricsSelector;
//not used, only for documentation
abstract class FactorialFrontend2 extends UntypedActor {

View file

@ -4,11 +4,12 @@ import java.math.BigInteger;
import java.io.Serializable;
public class FactorialResult implements Serializable {
public final int n;
public final BigInteger factorial;
private static final long serialVersionUID = 1L;
public final int n;
public final BigInteger factorial;
FactorialResult(int n, BigInteger factorial) {
this.n = n;
this.factorial = factorial;
}
}
FactorialResult(int n, BigInteger factorial) {
this.n = n;
this.factorial = factorial;
}
}

View file

@ -3,12 +3,13 @@ package sample.cluster.factorial;
//#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.cluster.metrics.ClusterMetricsChanged;
import akka.cluster.metrics.NodeMetrics;
import akka.cluster.metrics.StandardMetrics;
import akka.cluster.metrics.StandardMetrics.HeapMemory;
import akka.cluster.metrics.StandardMetrics.Cpu;
import akka.cluster.metrics.ClusterMetricsExtension;
import akka.event.Logging;
import akka.event.LoggingAdapter;
@ -16,24 +17,27 @@ public class MetricsListener extends UntypedActor {
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
Cluster cluster = Cluster.get(getContext().system());
ClusterMetricsExtension extension = ClusterMetricsExtension.get(getContext().system());
//subscribe to ClusterMetricsChanged
// Subscribe unto ClusterMetricsEvent events.
@Override
public void preStart() {
cluster.subscribe(getSelf(), ClusterMetricsChanged.class);
extension.subscribe(getSelf());
}
//re-subscribe when restart
// Unsubscribe from ClusterMetricsEvent events.
@Override
public void postStop() {
cluster.unsubscribe(getSelf());
extension.unsubscribe(getSelf());
}
@Override
public void onReceive(Object message) {
if (message instanceof ClusterMetricsChanged) {
ClusterMetricsChanged clusterMetrics = (ClusterMetricsChanged) message;
ClusterMetricsChanged clusterMetrics = (ClusterMetricsChanged) message;
for (NodeMetrics nodeMetrics : clusterMetrics.getNodeMetrics()) {
if (nodeMetrics.address().equals(cluster.selfAddress())) {
logHeap(nodeMetrics);
@ -42,8 +46,7 @@ public class MetricsListener extends UntypedActor {
}
} else if (message instanceof CurrentClusterState) {
// ignore
// Ignore.
} else {
unhandled(message);
}

View file

@ -18,3 +18,13 @@ akka {
auto-down-unreachable-after = 10s
}
}
# Disable legacy metrics in akka-cluster.
akka.cluster.metrics.enabled=off
# Enable metrics extension in akka-cluster-metrics.
akka.extensions=["akka.cluster.metrics.ClusterMetricsExtension"]
# Sigar native library extract location during tests.
# Note: use per-jvm-instance folder when running multiple jvm on one host.
akka.cluster.metrics.native-library-extract-folder=${user.dir}/target/native

View file

@ -14,11 +14,14 @@ akka.cluster.role {
# //#adaptive-router
akka.actor.deployment {
/factorialFrontend/factorialBackendRouter = {
router = adaptive-group
# Router type provided by metrics extension.
router = cluster-metrics-adaptive-group
# Router parameter specific for metrics extension.
# metrics-selector = heap
# metrics-selector = load
# metrics-selector = cpu
metrics-selector = mix
#
nr-of-instances = 100
routees.paths = ["/user/factorialBackend"]
cluster {

View file

@ -27,6 +27,22 @@ object StatsSampleSingleMasterSpecConfig extends MultiNodeConfig {
val second = role("second")
val third = role("thrid")
def nodeList = Seq(first, second, third)
// Extract individual sigar library for every node.
nodeList foreach { role
nodeConfig(role) {
ConfigFactory.parseString(s"""
# Disable legacy metrics in akka-cluster.
akka.cluster.metrics.enabled=off
# Enable metrics extension in akka-cluster-metrics.
akka.extensions=["akka.cluster.metrics.ClusterMetricsExtension"]
# Sigar native library extract location during tests.
akka.cluster.metrics.native-library-extract-folder=target/native/${role.name}
""")
}
}
// this configuration will be used for all nodes
// note that no fixed host names and ports are used
commonConfig(ConfigFactory.parseString("""
@ -34,8 +50,6 @@ object StatsSampleSingleMasterSpecConfig extends MultiNodeConfig {
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off
akka.cluster.roles = [compute]
# 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 {
/singleton/statsService/workerRouter {

View file

@ -25,14 +25,28 @@ object StatsSampleSpecConfig extends MultiNodeConfig {
val second = role("second")
val third = role("thrid")
def nodeList = Seq(first, second, third)
// Extract individual sigar library for every node.
nodeList foreach { role
nodeConfig(role) {
ConfigFactory.parseString(s"""
# Disable legacy metrics in akka-cluster.
akka.cluster.metrics.enabled=off
# Enable metrics extension in akka-cluster-metrics.
akka.extensions=["akka.cluster.metrics.ClusterMetricsExtension"]
# Sigar native library extract location during tests.
akka.cluster.metrics.native-library-extract-folder=target/native/${role.name}
""")
}
}
// this configuration will be used for all nodes
// note that no fixed host names and ports are used
commonConfig(ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off
akka.cluster.roles = [compute]
# 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 {

View file

@ -24,13 +24,27 @@ object TransformationSampleSpecConfig extends MultiNodeConfig {
val backend2 = role("backend2")
val backend3 = role("backend3")
def nodeList = Seq(frontend1, frontend2, backend1, backend2, backend3)
// Extract individual sigar library for every node.
nodeList foreach { role
nodeConfig(role) {
ConfigFactory.parseString(s"""
# Disable legacy metrics in akka-cluster.
akka.cluster.metrics.enabled=off
# Enable metrics extension in akka-cluster-metrics.
akka.extensions=["akka.cluster.metrics.ClusterMetricsExtension"]
# Sigar native library extract location during tests.
akka.cluster.metrics.native-library-extract-folder=target/native/${role.name}
""")
}
}
// this configuration will be used for all nodes
// note that no fixed host names and ports are used
commonConfig(ConfigFactory.parseString("""
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
akka.remote.log-remote-lifecycle-events = off
# don't use sigar for tests, native lib not in path
akka.cluster.metrics.collector-class = akka.cluster.JmxMetricsCollector
"""))
nodeConfig(frontend1, frontend2)(

View file

@ -9,19 +9,19 @@ val project = Project(
settings = Project.defaultSettings ++ SbtMultiJvm.multiJvmSettings ++ Seq(
name := "akka-sample-cluster-scala",
version := "2.4-SNAPSHOT",
scalaVersion := "2.10.4",
scalaVersion := "2.11.5",
scalacOptions in Compile ++= Seq("-encoding", "UTF-8", "-target:jvm-1.6", "-deprecation", "-feature", "-unchecked", "-Xlog-reflective-calls", "-Xlint"),
javacOptions in Compile ++= Seq("-source", "1.6", "-target", "1.6", "-Xlint:unchecked", "-Xlint:deprecation"),
libraryDependencies ++= Seq(
"com.typesafe.akka" %% "akka-actor" % akkaVersion,
"com.typesafe.akka" %% "akka-remote" % akkaVersion,
"com.typesafe.akka" %% "akka-cluster" % akkaVersion,
"com.typesafe.akka" %% "akka-cluster-metrics" % akkaVersion,
"com.typesafe.akka" %% "akka-contrib" % akkaVersion,
"com.typesafe.akka" %% "akka-multi-node-testkit" % akkaVersion,
"org.scalatest" %% "scalatest" % "2.2.1" % "test",
"org.fusesource" % "sigar" % "1.6.4"),
"io.kamon" % "sigar-loader" % "1.6.5-rev001"),
javaOptions in run ++= Seq(
"-Djava.library.path=./sigar",
"-Xms128m", "-Xmx1024m"),
Keys.fork in run := true,
mainClass in (Compile, run) := Some("sample.cluster.simple.SimpleClusterApp"),

Some files were not shown because too many files have changed in this diff Show more