Rename sbt akka modules

Co-authored-by: Sean Glover <sean@seanglover.com>
This commit is contained in:
Matthew de Detrich 2023-01-05 11:10:50 +01:00 committed by GitHub
parent b92b749946
commit 24c03cde19
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
2930 changed files with 1466 additions and 1462 deletions

View file

@ -0,0 +1,100 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
syntax = "proto2";
option java_package = "org.apache.pekko.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;
}
/****************************************
* Router pool (for remote deployment
****************************************/
message AdaptiveLoadBalancingPool {
// omitted if default
optional MetricsSelector metricsSelector = 1;
required uint32 nrOfInstances = 2;
// omitted if default
optional string routerDispatcher = 3;
required bool usePoolDispatcher = 4;
}
// couldn't figure out how to import Payload
message MetricsSelector {
required uint32 serializerId = 1;
required string manifest = 2;
required bytes data = 3;
}
message MixMetricsSelector {
repeated MetricsSelector selectors = 1;
}

View file

@ -0,0 +1,111 @@
##############################################
# Pekko 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.
pekko.cluster.metrics {
# Full path of dispatcher configuration key.
dispatcher = "pekko.actor.default-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"
# Metrics supervisor actor.
supervisor {
# Actor name. Example name space: /system/cluster-metrics
name = "cluster-metrics"
# Supervision strategy.
strategy {
#
# FQCN of class providing `org.apache.pekko.actor.SupervisorStrategy`.
# Must have a constructor with signature `<init>(com.typesafe.config.Config)`.
# Default metrics strategy provider is a configurable extension of `OneForOneStrategy`.
provider = "org.apache.pekko.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: `org.apache.pekko.cluster.metrics.{CollectionStartMessage,CollectionStopMessage}`
enabled = on
# FQCN of the metrics collector implementation.
# It must implement `org.apache.pekko.cluster.metrics.MetricsCollector` and
# have public constructor with org.apache.pekko.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 https://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average
moving-average-half-life = 12s
}
}
# Cluster metrics extension serializers and routers.
pekko.actor {
# Protobuf serializer for remote cluster metrics messages.
serializers {
pekko-cluster-metrics = "org.apache.pekko.cluster.metrics.protobuf.MessageSerializer"
}
# Interface binding for remote cluster metrics messages.
serialization-bindings {
"org.apache.pekko.cluster.metrics.ClusterMetricsMessage" = pekko-cluster-metrics
"org.apache.pekko.cluster.metrics.AdaptiveLoadBalancingPool" = pekko-cluster-metrics
"org.apache.pekko.cluster.metrics.MixMetricsSelector" = pekko-cluster-metrics
"org.apache.pekko.cluster.metrics.CpuMetricsSelector$" = pekko-cluster-metrics
"org.apache.pekko.cluster.metrics.HeapMetricsSelector$" = pekko-cluster-metrics
"org.apache.pekko.cluster.metrics.SystemLoadAverageMetricsSelector$" = pekko-cluster-metrics
}
# Globally unique metrics extension serializer identifier.
serialization-identifiers {
"org.apache.pekko.cluster.metrics.protobuf.MessageSerializer" = 10
}
# Provide routing of messages based on cluster metrics.
router.type-mapping {
cluster-metrics-adaptive-pool = "org.apache.pekko.cluster.metrics.AdaptiveLoadBalancingPool"
cluster-metrics-adaptive-group = "org.apache.pekko.cluster.metrics.AdaptiveLoadBalancingGroup"
}
}

View file

@ -0,0 +1,279 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import java.util.concurrent.ThreadLocalRandom
import scala.collection.immutable
import scala.annotation.nowarn
import org.apache.pekko
import pekko.actor.Actor
import pekko.actor.ActorLogging
import pekko.actor.Address
import pekko.actor.DeadLetterSuppression
import pekko.actor.Props
import pekko.cluster.Cluster
import pekko.cluster.ClusterEvent
import pekko.cluster.Member
import pekko.cluster.MemberStatus
/**
* 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 {
val metrics = ClusterMetricsExtension(context.system)
import context._
import metrics.settings._
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 */
@nowarn("msg=deprecated")
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
with DeadLetterSuppression
/**
* INTERNAL API.
*/
private[metrics] object ClusterMetricsCollector {
case object MetricsTick
case object GossipTick
}
/**
* 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 ClusterEvent._
import ClusterMetricsCollector._
import Member.addressOrdering
import context.dispatcher
val cluster = Cluster(context.system)
import cluster.{ scheduler, selfAddress }
import cluster.ClusterLogger._
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.scheduleWithFixedDelay(
PeriodicTasksInitialDelay max CollectorGossipInterval,
CollectorGossipInterval,
self,
GossipTick)
/**
* Start periodic metrics collection
*/
val sampleTask = scheduler.scheduleWithFixedDelay(
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 MemberWeaklyUp(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 || m.status == MemberStatus.WeaklyUp)
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 [[pekko.cluster.MemberStatus]] `Up`.
*/
def receiveState(state: CurrentClusterState): Unit =
nodes = state.members.diff(state.unreachable).collect {
case m if m.status == MemberStatus.Up || m.status == MemberStatus.WeaklyUp => 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,97 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import scala.collection.immutable
import com.typesafe.config.Config
import org.apache.pekko
import pekko.actor.ActorRef
import pekko.actor.ActorSystem
import pekko.actor.ClassicActorSystemProvider
import pekko.actor.Deploy
import pekko.actor.ExtendedActorSystem
import pekko.actor.Extension
import pekko.actor.ExtensionId
import pekko.actor.ExtensionIdProvider
import pekko.actor.Props
import pekko.actor.SupervisorStrategy
import pekko.event.Logging
import pekko.event.LoggingAdapter
/**
* 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, classOf[ClusterMetricsExtension])
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 get(system: ClassicActorSystemProvider): ClusterMetricsExtension = super.get(system)
override def createExtension(system: ExtendedActorSystem): ClusterMetricsExtension =
new ClusterMetricsExtension(system)
}

View file

@ -0,0 +1,561 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import java.util.Arrays
import java.util.concurrent.ThreadLocalRandom
import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec
import scala.collection.immutable
import scala.annotation.nowarn
import com.typesafe.config.Config
import org.apache.pekko
import pekko.actor.Actor
import pekko.actor.ActorSystem
import pekko.actor.Address
import pekko.actor.DynamicAccess
import pekko.actor.NoSerializationVerificationNeeded
import pekko.actor.Props
import pekko.actor.SupervisorStrategy
import pekko.cluster.Cluster
import pekko.cluster.routing.ClusterRouterSettingsBase
import pekko.dispatch.Dispatchers
import pekko.japi.Util.immutableSeq
import pekko.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 = ClusterRouterSettingsBase.getMaxTotalNrOfInstances(config),
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 [[pekko.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,
val 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 [[pekko.actor.ActorSelection]] to these paths
*/
def this(metricsSelector: MetricsSelector, routeesPaths: java.lang.Iterable[String]) =
this(paths = immutableSeq(routeesPaths))
override def paths(system: ActorSystem): immutable.Iterable[String] = this.paths
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 pekko.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 pekko.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 pekko.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 [org.apache.pekko.cluster.routing.HeapMetricsSelector],
* [org.apache.pekko.cluster.routing.CpuMetricsSelector], and [org.apache.pekko.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 [org.apache.pekko.cluster.routing.HeapMetricsSelector],
* [org.apache.pekko.cluster.routing.CpuMetricsSelector], and [org.apache.pekko.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
val init: Map[Address, (Double, Int)] = Map.empty.withDefaultValue((0.0, 0))
combined
.foldLeft(init) {
case (acc, (address, capacity)) =>
val (sum, count) = acc(address)
acc + (address -> ((sum + capacity, count + 1)))
}
.map {
case (address, (sum, count)) => address -> (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 [org.apache.pekko.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.
*/
@nowarn("msg=@SerialVersionUID")
@SerialVersionUID(1L)
trait MetricsSelector extends Serializable {
/**
* The weights per address, based on 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.
*/
@nowarn("msg=deprecated")
@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 (address, c) => address -> 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
case _ => throw new RuntimeException()
}
a match {
case Address(_, _, None, None) => selfAddress
case a => a
}
}
val buckets = new Array[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,47 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
import com.typesafe.config.Config
import org.apache.pekko
import pekko.util.Helpers.ConfigOps
import pekko.util.Helpers.Requiring
/**
* Metrics extension settings. Documented in: `src/main/resources/reference.conf`.
*/
case class ClusterMetricsSettings(config: Config) {
private val cc = config.getConfig("pekko.cluster.metrics")
// Extension.
val MetricsDispatcher: String = cc.getString("dispatcher")
val PeriodicTasksInitialDelay: FiniteDuration = cc.getMillisDuration("periodic-tasks-initial-delay")
val NativeLibraryExtractFolder: String = cc.getString("native-library-extract-folder")
// 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,40 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import com.typesafe.config.Config
import org.apache.pekko
import pekko.actor.OneForOneStrategy
import pekko.util.Helpers.ConfigOps
/**
* Default [[ClusterMetricsSupervisor]] strategy:
* A configurable [[pekko.actor.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 pekko.actor._
import pekko.actor.SupervisorStrategy._
/**
* [[pekko.actor.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-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.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.
*
* https://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 https://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 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 [[org.apache.pekko.cluster.metrics.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,396 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import scala.util.Failure
import scala.util.Success
import scala.util.Try
import scala.annotation.nowarn
import org.apache.pekko
import pekko.actor.Address
/**
* 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 &lt;= max if max is defined.
* A memory allocation may fail if it attempts to increase the used memory such that used &gt; committed
* even if used &lt;= max is true (e.g. when the system virtual memory is low).
*
* @param address [[pekko.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 [[pekko.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 [[pekko.cluster.metrics.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 [[pekko.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 [[pekko.cluster.metrics.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
copy(metrics = that.metrics.union(metrics.diff(that.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]]
val merged =
updated.union(latestNode.metrics.diff(updated)).union(currentNode.metrics.diff(updated).diff(latestNode.metrics))
copy(metrics = merged, timestamp = latestNode.timestamp)
}
def metric(key: String): Option[Metric] = metrics.collectFirst { case m if m.name == key => m }
/**
* Java API
*/
@nowarn("msg=deprecated")
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 [[pekko.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,263 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import java.io.Closeable
import java.lang.System.{ currentTimeMillis => newTimestamp }
import java.lang.management.ManagementFactory
import java.lang.management.MemoryMXBean
import java.lang.management.MemoryUsage
import java.lang.management.OperatingSystemMXBean
import org.hyperic.sigar.SigarProxy
import org.apache.pekko
import pekko.ConfigurationException
import pekko.actor.ActorSystem
import pekko.actor.Address
import pekko.actor.ExtendedActorSystem
import pekko.cluster.Cluster
import pekko.event.Logging
/**
* 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, classOf[MetricsCollector])
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.getMessage}", e)
}.get
}
}
/**
* Loads JVM and system metrics through JMX monitoring beans.
*
* @param address The [[pekko.actor.Address]] of the node being sampled
* @param decayFactor 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 [[pekko.actor.Address]] of the node being sampled
* @param decayFactor 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().union(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 <a href="https://bugzilla.redhat.com/show_bug.cgi?id=749121">749121</a> 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: <a href="https://en.wikipedia.org/wiki/CPU_time#Subdivision">Wikipedia - CPU time subdivision</a> and
* <a href="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</a>
*
* 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,109 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import java.io.File
import scala.language.postfixOps
import scala.util.Failure
import scala.util.Success
import scala.util.Try
import kamon.sigar.SigarProvisioner
import org.hyperic.sigar.Sigar
import org.hyperic.sigar.SigarProxy
/**
* Provide sigar instance as `SigarProxy`.
*
* User can provision sigar classes and native library in one of the following ways:
*
* 1) Use <a href="https://github.com/kamon-io/sigar-loader">Kamon sigar-loader</a> 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 <a href="https://github.com/kamon-io/sigar-loader">Kamon sigar-loader</a> 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 _: 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,320 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics.protobuf
import java.{ lang => jl }
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import java.io.NotSerializableException
import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
import scala.annotation.tailrec
import scala.collection.immutable
import org.apache.pekko
import pekko.actor.{ Address, ExtendedActorSystem }
import pekko.cluster.metrics._
import pekko.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages => cm }
import pekko.dispatch.Dispatchers
import pekko.protobufv3.internal.MessageLite
import pekko.remote.ByteStringUtils
import pekko.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest, Serializers }
import pekko.util.ClassLoaderObjectInputStream
import pekko.util.ccompat._
import pekko.util.ccompat.JavaConverters._
/**
* Protobuf serializer for [[pekko.cluster.metrics.ClusterMetricsMessage]] types.
*/
@ccompatUsedUntil213
class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer {
private final val BufferSize = 4 * 1024
private val MetricsGossipEnvelopeManifest = "a"
private val AdaptiveLoadBalancingPoolManifest = "b"
private val MixMetricsSelectorManifest = "c"
private val CpuMetricsSelectorManifest = "d"
private val HeapMetricsSelectorManifest = "e"
private val SystemLoadAverageMetricsSelectorManifest = "f"
private lazy val serialization = SerializationExtension(system)
override def manifest(obj: AnyRef): String = obj match {
case _: MetricsGossipEnvelope => MetricsGossipEnvelopeManifest
case _: AdaptiveLoadBalancingPool => AdaptiveLoadBalancingPoolManifest
case _: MixMetricsSelector => MixMetricsSelectorManifest
case CpuMetricsSelector => CpuMetricsSelectorManifest
case HeapMetricsSelector => HeapMetricsSelectorManifest
case SystemLoadAverageMetricsSelector => SystemLoadAverageMetricsSelectorManifest
case _ =>
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
}
override def toBinary(obj: AnyRef): Array[Byte] = obj match {
case m: MetricsGossipEnvelope => compress(metricsGossipEnvelopeToProto(m))
case alb: AdaptiveLoadBalancingPool => adaptiveLoadBalancingPoolToBinary(alb)
case mms: MixMetricsSelector => mixMetricSelectorToBinary(mms)
case CpuMetricsSelector => Array.emptyByteArray
case HeapMetricsSelector => Array.emptyByteArray
case SystemLoadAverageMetricsSelector => Array.emptyByteArray
case _ =>
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
}
def compress(msg: MessageLite): Array[Byte] = {
val bos = new ByteArrayOutputStream(BufferSize)
val zip = new GZIPOutputStream(bos)
try msg.writeTo(zip)
finally 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()
}
try readChunk()
finally in.close()
out.toByteArray
}
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = manifest match {
case MetricsGossipEnvelopeManifest => metricsGossipEnvelopeFromBinary(bytes)
case AdaptiveLoadBalancingPoolManifest => adaptiveLoadBalancingPoolFromBinary(bytes)
case MixMetricsSelectorManifest => mixMetricSelectorFromBinary(bytes)
case CpuMetricsSelectorManifest => CpuMetricsSelector
case HeapMetricsSelectorManifest => HeapMetricsSelector
case SystemLoadAverageMetricsSelectorManifest => SystemLoadAverageMetricsSelector
case _ =>
throw new NotSerializableException(
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}")
}
private def addressToProto(address: Address): cm.Address.Builder = address match {
case Address(protocol, actorSystem, Some(host), Some(port)) =>
cm.Address.newBuilder().setSystem(actorSystem).setHostname(host).setPort(port).setProtocol(protocol)
case _ => throw new IllegalArgumentException(s"Address [$address] could not be serialized: host or port missing.")
}
def adaptiveLoadBalancingPoolToBinary(alb: AdaptiveLoadBalancingPool): Array[Byte] = {
val builder = cm.AdaptiveLoadBalancingPool.newBuilder()
if (alb.metricsSelector != MixMetricsSelector) {
builder.setMetricsSelector(metricsSelectorToProto(alb.metricsSelector))
}
if (alb.routerDispatcher != Dispatchers.DefaultDispatcherId) {
builder.setRouterDispatcher(alb.routerDispatcher)
}
builder.setNrOfInstances(alb.nrOfInstances)
builder.setUsePoolDispatcher(alb.usePoolDispatcher)
builder.build().toByteArray
}
private def metricsSelectorToProto(selector: MetricsSelector): cm.MetricsSelector = {
val builder = cm.MetricsSelector.newBuilder()
val serializer = serialization.findSerializerFor(selector)
builder
.setData(ByteStringUtils.toProtoByteStringUnsafe(serializer.toBinary(selector)))
.setSerializerId(serializer.identifier)
val manifest = Serializers.manifestFor(serializer, selector)
builder.setManifest(manifest)
builder.build()
}
private def mixMetricSelectorToBinary(mms: MixMetricsSelector): Array[Byte] = {
val builder = cm.MixMetricsSelector.newBuilder()
mms.selectors.foreach { selector =>
builder.addSelectors(metricsSelectorToProto(selector))
}
builder.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 allNodeMetrics = envelope.gossip.nodes
val allAddresses: Vector[Address] = allNodeMetrics.iterator.map(_.address).to(immutable.Vector)
val addressMapping = allAddresses.zipWithIndex.toMap
val allMetricNames: Vector[String] =
allNodeMetrics.foldLeft(Set.empty[String])((s, n) => s ++ n.metrics.iterator.map(_.name)).toVector
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(ByteStringUtils.toProtoByteStringUnsafe(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] = allNodeMetrics.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.iterator.map(addressFromProto).to(Vector)
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.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.iterator.map(metricFromProto).to(immutable.Set))
val nodeMetrics: Set[NodeMetrics] =
mgossip.getNodeMetricsList.asScala.iterator.map(nodeMetricsFromProto).to(immutable.Set)
MetricsGossipEnvelope(addressFromProto(envelope.getFrom), MetricsGossip(nodeMetrics), envelope.getReply)
}
def adaptiveLoadBalancingPoolFromBinary(bytes: Array[Byte]): AdaptiveLoadBalancingPool = {
val alb = cm.AdaptiveLoadBalancingPool.parseFrom(bytes)
val selector =
if (alb.hasMetricsSelector) {
val ms = alb.getMetricsSelector
serialization
.deserialize(ms.getData.toByteArray, ms.getSerializerId, ms.getManifest)
.get
.asInstanceOf[MetricsSelector]
} else MixMetricsSelector
AdaptiveLoadBalancingPool(
metricsSelector = selector,
nrOfInstances = alb.getNrOfInstances,
routerDispatcher =
if (alb.hasRouterDispatcher) alb.getRouterDispatcher
else Dispatchers.DefaultDispatcherId,
usePoolDispatcher = alb.getUsePoolDispatcher)
}
def mixMetricSelectorFromBinary(bytes: Array[Byte]): MixMetricsSelector = {
val mm = cm.MixMetricsSelector.parseFrom(bytes)
MixMetricsSelector(
mm.getSelectorsList.asScala
// should be safe because we serialized only the right subtypes of MetricsSelector
.map(s => metricSelectorFromProto(s).asInstanceOf[CapacityMetricsSelector])
.toIndexedSeq)
}
def metricSelectorFromProto(selector: cm.MetricsSelector): MetricsSelector =
serialization
.deserialize(selector.getData.toByteArray, selector.getSerializerId, selector.getManifest)
.get
.asInstanceOf[MetricsSelector]
}

View file

@ -0,0 +1,160 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import scala.concurrent.duration._
import scala.language.postfixOps
import com.typesafe.config.ConfigFactory
import org.apache.pekko
import pekko.cluster.MemberStatus
import pekko.cluster.MultiNodeClusterSpec
import pekko.remote.testkit.MultiNodeConfig
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(s"pekko.cluster.metrics.native-library-extract-folder=$${user.dir}/target/native/" + role.name)
}
}
// Enable metrics extension in akka-cluster-metrics.
def enableMetricsExtension = parseString("""
pekko.extensions=["org.apache.pekko.cluster.metrics.ClusterMetricsExtension"]
pekko.cluster.metrics.collector.enabled = on
""")
// Disable metrics extension in akka-cluster-metrics.
def disableMetricsExtension = parseString("""
pekko.extensions=["org.apache.pekko.cluster.metrics.ClusterMetricsExtension"]
pekko.cluster.metrics.collector.enabled = off
""")
// Activate slf4j logging along with test listener.
def customLogging = parseString("""
pekko.loggers=["org.apache.pekko.testkit.TestEventListener","org.apache.pekko.event.slf4j.Slf4jLogger"]
pekko.logger-startup-timeout = 15s
""")
}
object ClusterMetricsDisabledConfig extends ClusterMetricsCommonConfig {
commonConfig {
Seq(
customLogging,
disableMetricsExtension,
debugConfig(on = false),
MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet).reduceLeft(_.withFallback(_))
}
}
object ClusterMetricsEnabledConfig extends ClusterMetricsCommonConfig {
commonConfig {
Seq(
customLogging,
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 MultiNodeClusterSpec(ClusterMetricsEnabledConfig)
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" in within(60 seconds) {
awaitClusterUp(roles: _*)
enterBarrier("cluster-started")
awaitAssert(clusterView.members.count(_.status == MemberStatus.Up) should ===(roles.size))
// TODO ensure same contract
// awaitAssert(clusterView.clusterMetrics.size should ===(roles.size))
awaitAssert(metricsView.clusterMetrics.size should ===(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" 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 ===(roles.size - 1))
awaitAssert(metricsView.clusterMetrics.size should ===(roles.size - 1))
}
enterBarrier("finished")
}
}
}
class ClusterMetricsDisabledMultiJvmNode1 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode2 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode3 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode4 extends ClusterMetricsDisabledSpec
class ClusterMetricsDisabledMultiJvmNode5 extends ClusterMetricsDisabledSpec
abstract class ClusterMetricsDisabledSpec
extends MultiNodeClusterSpec(ClusterMetricsDisabledConfig)
with RedirectLogging {
val metricsView = new ClusterMetricsView(cluster.system)
"Cluster metrics" must {
"not collect metrics, not publish metrics events, and not gossip metrics" in {
awaitClusterUp(roles: _*)
// TODO ensure same contract
// clusterView.clusterMetrics.size should ===(0)
metricsView.clusterMetrics.size should ===(0)
ClusterMetricsExtension(system).subscribe(testActor)
expectNoMessage()
// TODO ensure same contract
// clusterView.clusterMetrics.size should ===(0)
metricsView.clusterMetrics.size should ===(0)
enterBarrier("after")
}
}
}

View file

@ -0,0 +1,261 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import java.lang.management.ManagementFactory
import scala.annotation.nowarn
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import language.postfixOps
import org.apache.pekko
import pekko.actor._
import pekko.cluster.Cluster
import pekko.cluster.MultiNodeClusterSpec
import pekko.cluster.routing.ClusterRouterPool
import pekko.cluster.routing.ClusterRouterPoolSettings
import pekko.pattern.ask
import pekko.remote.testkit.MultiNodeConfig
import pekko.routing.ActorRefRoutee
import pekko.routing.FromConfig
import pekko.routing.GetRoutees
import pekko.routing.Routees
import pekko.serialization.jackson.CborSerializable
import pekko.testkit.GHExcludeTest
import pekko.testkit.{ DefaultTimeout, ImplicitSender, LongRunningTest }
import pekko.util.unused
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.info("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.info("used heap after: [{}] bytes", ManagementFactory.getMemoryMXBean.getHeapMemoryUsage.getUsed)
sender() ! "done"
}
}
case object AllocateMemory
final case class Reply(address: Address) extends CborSerializable
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(
s"pekko.cluster.metrics.native-library-extract-folder=$${user.dir}/target/native/" + role.name)
}
}
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
# Enable metrics estension.
pekko.extensions=["org.apache.pekko.cluster.metrics.ClusterMetricsExtension"]
pekko.cluster.failure-detector.acceptable-heartbeat-pause = 10s
# Use rapid metrics collection.
pekko.cluster.metrics {
collector {
sample-interval = 1s
gossip-interval = 1s
moving-average-half-life = 2s
}
}
# Use metrics extension routing.
pekko.actor.deployment {
/router3 = {
router = cluster-metrics-adaptive-pool
metrics-selector = cpu
nr-of-instances = 9
}
/router4 = {
router = cluster-metrics-adaptive-pool
metrics-selector = "org.apache.pekko.cluster.metrics.TestCustomMetricsSelector"
nr-of-instances = 10
cluster {
enabled = on
max-nr-of-instances-per-node = 2
}
}
}
"""))
.withFallback(MultiNodeClusterSpec.clusterConfig))
}
class TestCustomMetricsSelector(@unused 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
@nowarn
abstract class AdaptiveLoadBalancingRouterSpec
extends MultiNodeClusterSpec(AdaptiveLoadBalancingRouterConfig)
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))
.props(Props[Echo]()),
name)
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router).size should ===(roles.size) }
val routees = currentRoutees(router)
routees.collect { case ActorRefRoutee(ref) => fullAddress(ref) }.toSet should ===(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 ===(iterationCount)
}
enterBarrier("after-2")
}
// Excluded on GH Actions: https://github.com/akka/akka/issues/30486
"prefer node with more free heap capacity" taggedAs (LongRunningTest, GHExcludeTest) 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 ===(iterationCount)
}
enterBarrier("after-3")
}
// Excluded on GH Actions: https://github.com/akka/akka/issues/30486
"create routees from configuration" taggedAs (LongRunningTest, GHExcludeTest) 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 ===(9) }
val routees = currentRoutees(router3)
routees.collect { case ActorRefRoutee(ref) => fullAddress(ref) }.toSet should ===(Set(address(node1)))
}
enterBarrier("after-4")
}
// Excluded on GH Actions: https://github.com/akka/akka/issues/30486
"create routees from cluster.enabled configuration" taggedAs (LongRunningTest, GHExcludeTest) 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 ===(6) }
val routees = currentRoutees(router4)
routees.collect { case ActorRefRoutee(ref) => fullAddress(ref) }.toSet should ===(
Set(address(node1), address(node2), address(node3)))
}
enterBarrier("after-5")
}
}
}

View file

@ -0,0 +1,13 @@
/*
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics.sample
import org.apache.pekko.serialization.jackson.CborSerializable
//#messages
final case class StatsJob(text: String) extends CborSerializable
final case class StatsResult(meanWordLength: Double) extends CborSerializable
final case class JobFailed(reason: String) extends CborSerializable
//#messages

View file

@ -0,0 +1,152 @@
/*
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics.sample
import org.apache.pekko
import pekko.actor.Props
import pekko.cluster.Cluster
import pekko.cluster.ClusterEvent.{ CurrentClusterState, MemberUp }
import scala.concurrent.duration._
import scala.language.postfixOps
//#MultiNodeConfig
import org.apache.pekko.remote.testkit.MultiNodeConfig
import com.typesafe.config.ConfigFactory
object StatsSampleSpecConfig extends MultiNodeConfig {
// register the named roles (nodes) of the test
val first = role("first")
val second = role("second")
val third = role("third")
def nodeList = Seq(first, second, third)
// Extract individual sigar library for every node.
nodeList.foreach { role =>
nodeConfig(role) {
ConfigFactory.parseString(s"""
# Enable metrics extension in pekko-cluster-metrics.
pekko.extensions=["org.apache.pekko.cluster.metrics.ClusterMetricsExtension"]
# Sigar native library extract location during tests.
pekko.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("""
pekko.actor.provider = cluster
pekko.remote.classic.log-remote-lifecycle-events = off
pekko.cluster.roles = [compute]
#//#router-lookup-config
pekko.actor.deployment {
/statsService/workerRouter {
router = consistent-hashing-group
routees.paths = ["/user/statsWorker"]
cluster {
enabled = on
allow-local-routees = on
use-roles = ["compute"]
}
}
}
#//#router-lookup-config
"""))
}
//#MultiNodeConfig
//#concrete-tests
// need one concrete test class per node
class StatsSampleSpecMultiJvmNode1 extends StatsSampleSpec
class StatsSampleSpecMultiJvmNode2 extends StatsSampleSpec
class StatsSampleSpecMultiJvmNode3 extends StatsSampleSpec
//#concrete-tests
//#abstract-test
import org.apache.pekko
import pekko.remote.testkit.MultiNodeSpec
import pekko.testkit.ImplicitSender
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
abstract class StatsSampleSpec
extends MultiNodeSpec(StatsSampleSpecConfig)
with AnyWordSpecLike
with Matchers
with BeforeAndAfterAll
with ImplicitSender {
import StatsSampleSpecConfig._
override def initialParticipants = roles.size
override def beforeAll() = multiNodeSpecBeforeAll()
override def afterAll() = multiNodeSpecAfterAll()
// #abstract-test
"The stats sample" must {
// #startup-cluster
"illustrate how to startup cluster" in within(15 seconds) {
Cluster(system).subscribe(testActor, classOf[MemberUp])
expectMsgClass(classOf[CurrentClusterState])
// #addresses
val firstAddress = node(first).address
val secondAddress = node(second).address
val thirdAddress = node(third).address
// #addresses
// #join
Cluster(system).join(firstAddress)
// #join
system.actorOf(Props[StatsWorker](), "statsWorker")
system.actorOf(Props[StatsService](), "statsService")
receiveN(3).collect { case MemberUp(m) => m.address }.toSet should be(
Set(firstAddress, secondAddress, thirdAddress))
Cluster(system).unsubscribe(testActor)
testConductor.enter("all-up")
}
// #startup-cluster
// #test-statsService
"show usage of the statsService from one node" in within(15 seconds) {
runOn(second) {
assertServiceOk()
}
testConductor.enter("done-2")
}
def assertServiceOk(): Unit = {
val service = system.actorSelection(node(third) / "user" / "statsService")
// eventually the service should be ok,
// first attempts might fail because worker actors not started yet
awaitAssert {
service ! StatsJob("this is the text that will be analyzed")
expectMsgType[StatsResult](1.second).meanWordLength should be(3.875 +- 0.001)
}
}
// #test-statsService
"show usage of the statsService from all nodes" in within(15 seconds) {
assertServiceOk()
testConductor.enter("done-3")
}
}
}

View file

@ -0,0 +1,85 @@
/*
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics.sample
import scala.concurrent.duration._
import org.apache.pekko
import pekko.actor.{ Actor, ActorRef, Props, ReceiveTimeout }
import pekko.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
import pekko.routing.FromConfig
//#service
class StatsService extends Actor {
// This router is used both with lookup and deploy of routees. If you
// have a router with only lookup of routees you can use Props.empty
// instead of Props[StatsWorker.class].
val workerRouter = context.actorOf(FromConfig.props(Props[StatsWorker]()), name = "workerRouter")
def receive = {
case StatsJob(text) if text != "" =>
val words = text.split(" ")
val replyTo = sender() // important to not close over sender()
// create actor that collects replies from workers
val aggregator = context.actorOf(Props(classOf[StatsAggregator], words.size, replyTo))
words.foreach { word =>
workerRouter.tell(ConsistentHashableEnvelope(word, word), aggregator)
}
}
}
class StatsAggregator(expectedResults: Int, replyTo: ActorRef) extends Actor {
var results = IndexedSeq.empty[Int]
context.setReceiveTimeout(3.seconds)
def receive = {
case wordCount: Int =>
results = results :+ wordCount
if (results.size == expectedResults) {
val meanWordLength = results.sum.toDouble / results.size
replyTo ! StatsResult(meanWordLength)
context.stop(self)
}
case ReceiveTimeout =>
replyTo ! JobFailed("Service unavailable, try again later")
context.stop(self)
}
}
//#service
// not used, only for documentation
abstract class StatsService2 extends Actor {
// #router-lookup-in-code
import org.apache.pekko
import pekko.cluster.routing.{ ClusterRouterGroup, ClusterRouterGroupSettings }
import pekko.routing.ConsistentHashingGroup
val workerRouter = context.actorOf(
ClusterRouterGroup(
ConsistentHashingGroup(Nil),
ClusterRouterGroupSettings(
totalInstances = 100,
routeesPaths = List("/user/statsWorker"),
allowLocalRoutees = true,
useRoles = Set("compute"))).props(),
name = "workerRouter2")
// #router-lookup-in-code
}
// not used, only for documentation
abstract class StatsService3 extends Actor {
// #router-deploy-in-code
import org.apache.pekko
import pekko.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
import pekko.routing.ConsistentHashingPool
val workerRouter = context.actorOf(
ClusterRouterPool(
ConsistentHashingPool(0),
ClusterRouterPoolSettings(totalInstances = 100, maxInstancesPerNode = 3, allowLocalRoutees = false))
.props(Props[StatsWorker]()),
name = "workerRouter3")
// #router-deploy-in-code
}

View file

@ -0,0 +1,25 @@
/*
* Copyright (C) 2018-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics.sample
import org.apache.pekko.actor.Actor
//#worker
class StatsWorker extends Actor {
var cache = Map.empty[String, Int]
def receive = {
case word: String =>
val length = cache.get(word) match {
case Some(x) => x
case None =>
val x = word.length
cache += (word -> x)
x
}
sender() ! length
}
}
//#worker

View file

@ -0,0 +1,39 @@
<?xml version="1.0" encoding="UTF-8"?>
<configuration>
<!-- Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> -->
<variable name="pattern" value="%date{ISO8601} %-5level %logger %marker - %msg MDC: {%mdc}%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,124 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import scala.concurrent.duration._
import scala.language.postfixOps
import org.apache.pekko
import pekko.cluster.Cluster
import pekko.cluster.metrics.StandardMetrics._
import pekko.testkit._
class ClusterMetricsExtensionSpec
extends PekkoSpec(MetricsConfig.clusterSigarMock)
with ImplicitSender
with RedirectLogging {
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 ===(nodeCount), 15 seconds)
}
"collect mock sample during a time window" in {
awaitAssert(metricsHistorySize should ===(sampleCount), 15 seconds)
extension.supervisor ! CollectionStopMessage
awaitSample()
metricsNodeCount should ===(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 ===(sampleCount)
history.zip(expected).foreach {
case (mockMetrics, expectedData) =>
(mockMetrics, expectedData) match {
case (
Cpu(_, _, loadAverageMock, cpuCombinedMock, cpuStolenMock, _),
(loadAverageEwma, cpuCombinedEwma, cpuStolenEwma)) =>
loadAverageMock.get should ===(loadAverageEwma +- epsilon)
cpuCombinedMock.get should ===(cpuCombinedEwma +- epsilon)
cpuStolenMock.get should ===(cpuStolenEwma +- epsilon)
case _ => fail()
}
}
}
"control collector on/off state" in {
def cycle() = {
val size1 = metricsHistorySize
awaitSample()
val size2 = metricsHistorySize
size1 should ===(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 ===(size4)
}
(1 to 3).foreach { _ =>
cycle()
}
}
}
}

View file

@ -0,0 +1,132 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
import org.apache.pekko
import pekko.actor.Address
import pekko.cluster.metrics.StandardMetrics._
class MetricsSelectorSpec extends AnyWordSpec 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", "sys", "a1", 2551)
val b1 = Address("akka", "sys", "b1", 2551)
val c1 = Address("akka", "sys", "c1", 2551)
val d1 = Address("akka", "sys", "d1", 2551)
val decayFactor = Some(0.18)
val nodeMetricsA = NodeMetrics(
a1,
System.currentTimeMillis,
Set(
Metric.create(HeapMemoryUsed, 128, decayFactor),
Metric.create(HeapMemoryCommitted, 256, decayFactor),
Metric.create(HeapMemoryMax, 512, None),
Metric.create(CpuCombined, 0.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 ===(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 ===(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 ===(0.75 +- 0.0001)
capacity(b1) should ===(0.75 +- 0.0001)
capacity(c1) should ===(0.0 +- 0.0001)
capacity(d1) should ===(0.001953125 +- 0.0001)
}
}
"CpuMetricsSelector" must {
"calculate capacity of cpuCombined metrics" in {
val capacity = CpuMetricsSelector.capacity(nodeMetrics)
capacity(a1) should ===(1.0 - 0.2 - 0.1 * (1.0 + factor) +- 0.0001)
capacity(b1) should ===(1.0 - 0.4 - 0.2 * (1.0 + factor) +- 0.0001)
capacity(c1) should ===(1.0 - 0.6 - 0.3 * (1.0 + factor) +- 0.0001)
capacity.contains(d1) should ===(false)
}
}
"SystemLoadAverageMetricsSelector" must {
"calculate capacity of systemLoadAverage metrics" in {
val capacity = SystemLoadAverageMetricsSelector.capacity(nodeMetrics)
capacity(a1) should ===(0.9375 +- 0.0001)
capacity(b1) should ===(0.9375 +- 0.0001)
capacity(c1) should ===(0.0 +- 0.0001)
capacity.contains(d1) should ===(false)
}
}
"MixMetricsSelector" must {
"aggregate capacity of all metrics" in {
val capacity = MixMetricsSelector.capacity(nodeMetrics)
capacity(a1) should ===((0.75 + 0.67 + 0.9375) / 3 +- 0.0001)
capacity(b1) should ===((0.75 + 0.34 + 0.9375) / 3 +- 0.0001)
capacity(c1) should ===((0.0 + 0.01 + 0.0) / 3 +- 0.0001)
capacity(d1) should ===(0.001953125 / 1 +- 0.0001)
}
}
}

View file

@ -0,0 +1,43 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import language.postfixOps
import org.apache.pekko
import pekko.dispatch.Dispatchers
import pekko.testkit.PekkoSpec
class ClusterMetricsSettingsSpec extends PekkoSpec {
"ClusterMetricsSettings" must {
"be able to parse generic metrics config elements" in {
val settings = new ClusterMetricsSettings(system.settings.config)
import settings._
// Extension.
MetricsDispatcher should ===(Dispatchers.DefaultDispatcherId)
PeriodicTasksInitialDelay should ===(1 second)
NativeLibraryExtractFolder should ===(System.getProperty("user.dir") + "/native")
// Supervisor.
SupervisorName should ===("cluster-metrics")
SupervisorStrategyProvider should ===(classOf[ClusterMetricsStrategy].getName)
SupervisorStrategyConfiguration should ===(
ConfigFactory.parseString("loggingEnabled=true,maxNrOfRetries=3,withinTimeRange=3s"))
// Collector.
CollectorEnabled should ===(true)
CollectorProvider should ===("")
CollectorSampleInterval should ===(3 seconds)
CollectorGossipInterval should ===(3 seconds)
CollectorMovingAverageHalfLife should ===(12 seconds)
}
}
}

View file

@ -0,0 +1,103 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import java.util.concurrent.ThreadLocalRandom
import scala.concurrent.duration._
import scala.annotation.nowarn
import org.apache.pekko.testkit.{ LongRunningTest, PekkoSpec }
@nowarn
class EWMASpec extends PekkoSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory {
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 ===(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 ===(1000.0 +- 0.01)
val d1 = d0 :+ 10.0
d1.value should ===(820.0 +- 0.01)
val d2 = d1 :+ 10.0
d2.value should ===(672.73 +- 0.01)
val d3 = d2 :+ 10.0
d3.value should ===(552.23 +- 0.01)
val d4 = d3 :+ 10.0
d4.value should ===(453.64 +- 0.01)
val dn = (1 to 100).foldLeft(d0)((d, _) => d :+ 10.0)
dn.value should ===(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 ===(100.0 +- 0.01)
val d1 = d0 :+ 1.0
d1.value should ===(1.0 +- 0.01)
val d2 = d1 :+ 57.0
d2.value should ===(57.0 +- 0.01)
val d3 = d2 :+ 10.0
d3.value should ===(10.0 +- 0.01)
}
"calculate alpha from half-life and collect interval" in {
// according to https://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 ===(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 ===(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 ===(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 ===(true)
updated.smoothValue should not be (previous.smoothValue)
Some(updated)
} else None
}
}
streamingDataSet ++= changes.map(m => m.name -> m)
}
}
}
}

View file

@ -0,0 +1,310 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import java.lang.System.{ currentTimeMillis => newTimestamp }
import scala.util.Failure
import scala.annotation.nowarn
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
import org.apache.pekko
import pekko.actor.Address
import pekko.cluster.metrics.StandardMetrics._
import pekko.testkit.PekkoSpec
import pekko.testkit.ImplicitSender
@nowarn
class MetricNumericConverterSpec extends AnyWordSpec with Matchers with MetricNumericConverter {
"MetricNumericConverter" must {
"convert" in {
convertNumber(0).isLeft should ===(true)
convertNumber(1).left.get should ===(1)
convertNumber(1L).isLeft should ===(true)
convertNumber(0.0).isRight should ===(true)
}
"define a new metric" in {
val Some(metric) = Metric.create(HeapMemoryUsed, 256L, decayFactor = Some(0.18))
metric.name should ===(HeapMemoryUsed)
metric.value should ===(256L)
metric.isSmooth should ===(true)
metric.smoothValue should ===(256.0 +- 0.0001)
}
"define an undefined value with a None " in {
Metric.create("x", -1, None).isDefined should ===(false)
Metric.create("x", java.lang.Double.NaN, None).isDefined should ===(false)
Metric.create("x", Failure(new RuntimeException), None).isDefined should ===(false)
}
"recognize whether a metric value is defined" in {
defined(0) should ===(true)
defined(0.0) should ===(true)
}
"recognize whether a metric value is not defined" in {
defined(-1) should ===(false)
defined(-1.0) should ===(false)
defined(Double.NaN) should ===(false)
}
}
}
@nowarn
class NodeMetricsSpec extends AnyWordSpec with Matchers {
val node1 = Address("akka", "sys", "a", 2554)
val node2 = Address("akka", "sys", "a", 2555)
"NodeMetrics must" must {
"return correct result for 2 'same' nodes" in {
(NodeMetrics(node1, 0).sameAs(NodeMetrics(node1, 0))) should ===(true)
}
"return correct result for 2 not 'same' nodes" in {
(NodeMetrics(node1, 0).sameAs(NodeMetrics(node2, 0))) should ===(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 ===(sample2.timestamp)
merged.metric("a").map(_.value) should ===(Some(11))
merged.metric("b").map(_.value) should ===(Some(20))
merged.metric("c").map(_.value) should ===(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 ===(sample1.timestamp)
merged.metrics should ===(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 ===(3)
updated.timestamp should ===(sample2.timestamp)
updated.metric("a").map(_.value) should ===(Some(11))
updated.metric("b").map(_.value) should ===(Some(20))
updated.metric("c").map(_.value) should ===(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 ===(4)
updated.timestamp should ===(sample3.timestamp)
updated.metric("a").map(_.value).get should ===(3)
updated.metric("b").map(_.value).get should ===(4)
updated.metric("c").map(_.value).get should ===(5)
updated.metric("d").map(_.value).get should ===(6)
updated.metric("a").map(_.smoothValue).get should ===(1.512 +- epsilon)
updated.metric("b").map(_.smoothValue).get should ===(4.000 +- epsilon)
updated.metric("c").map(_.smoothValue).get should ===(5.000 +- epsilon)
updated.metric("d").map(_.smoothValue).get should ===(6.000 +- epsilon)
}
}
}
class MetricsGossipSpec
extends PekkoSpec(MetricsConfig.defaultEnabled)
with ImplicitSender
with MetricsCollectorFactory {
val collector = createMetricsCollector
/**
* sometimes Sigar will not be able to return a valid value (NaN and such) so must ensure they
* have the same Metric types
*/
def newSample(previousSample: Set[Metric]): Set[Metric] = {
// Metric.equals is based on name equality
collector.sample().metrics.filter(previousSample.contains) ++ previousSample
}
"A MetricsGossip" must {
"add new NodeMetrics" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
m1.metrics.size should be > 3
m2.metrics.size should be > 3
val g1 = MetricsGossip.empty :+ m1
g1.nodes.size should ===(1)
g1.nodeMetricsFor(m1.address).map(_.metrics) should ===(Some(m1.metrics))
val g2 = g1 :+ m2
g2.nodes.size should ===(2)
g2.nodeMetricsFor(m1.address).map(_.metrics) should ===(Some(m1.metrics))
g2.nodeMetricsFor(m2.address).map(_.metrics) should ===(Some(m2.metrics))
}
"merge peer metrics" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size should ===(2)
val m2Updated = m2.copy(metrics = newSample(m2.metrics), timestamp = m2.timestamp + 1000)
val g2 = g1 :+ m2Updated // merge peers
g2.nodes.size should ===(2)
g2.nodeMetricsFor(m1.address).map(_.metrics) should ===(Some(m1.metrics))
g2.nodeMetricsFor(m2.address).map(_.metrics) should ===(Some(m2Updated.metrics))
g2.nodes.collect { case peer if peer.address == m2.address => peer.timestamp should ===(m2Updated.timestamp) }
}
"merge an existing metric set for a node and update node ring" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
val m3 = NodeMetrics(Address("akka", "sys", "a", 2556), newTimestamp, collector.sample().metrics)
val m2Updated = m2.copy(metrics = newSample(m2.metrics), timestamp = m2.timestamp + 1000)
val g1 = MetricsGossip.empty :+ m1 :+ m2
val g2 = MetricsGossip.empty :+ m3 :+ m2Updated
g1.nodes.map(_.address) should ===(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 ===(Set(m1.address, m2.address, m3.address))
mergedGossip.nodeMetricsFor(m1.address).map(_.metrics) should ===(Some(m1.metrics))
mergedGossip.nodeMetricsFor(m2.address).map(_.metrics) should ===(Some(m2Updated.metrics))
mergedGossip.nodeMetricsFor(m3.address).map(_.metrics) should ===(Some(m3.metrics))
mergedGossip.nodes.foreach(_.metrics.size should be > 3)
mergedGossip.nodeMetricsFor(m2.address).map(_.timestamp) should ===(Some(m2Updated.timestamp))
}
"get the current NodeMetrics if it exists in the local nodes" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
val g1 = MetricsGossip.empty :+ m1
g1.nodeMetricsFor(m1.address).map(_.metrics) should ===(Some(m1.metrics))
}
"remove a node if it is no longer Up" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size should ===(2)
val g2 = g1.remove(m1.address)
g2.nodes.size should ===(1)
g2.nodes.exists(_.address == m1.address) should ===(false)
g2.nodeMetricsFor(m1.address) should ===(None)
g2.nodeMetricsFor(m2.address).map(_.metrics) should ===(Some(m2.metrics))
}
"filter nodes" in {
val m1 = NodeMetrics(Address("akka", "sys", "a", 2554), newTimestamp, collector.sample().metrics)
val m2 = NodeMetrics(Address("akka", "sys", "a", 2555), newTimestamp, collector.sample().metrics)
val g1 = MetricsGossip.empty :+ m1 :+ m2
g1.nodes.size should ===(2)
val g2 = g1.filter(Set(m2.address))
g2.nodes.size should ===(1)
g2.nodes.exists(_.address == m1.address) should ===(false)
g2.nodeMetricsFor(m1.address) should ===(None)
g2.nodeMetricsFor(m2.address).map(_.metrics) should ===(Some(m2.metrics))
}
}
}
@nowarn
class MetricValuesSpec extends PekkoSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory {
import pekko.cluster.metrics.StandardMetrics._
val collector = createMetricsCollector
val node1 = NodeMetrics(Address("akka", "sys", "a", 2554), 1, collector.sample().metrics)
val node2 = NodeMetrics(Address("akka", "sys", "a", 2555), 1, collector.sample().metrics)
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(_, _, 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
case other =>
fail(other.toString)
}
node match {
case Cpu(_, _, 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
case other =>
fail(other.toString)
}
}
}
}
}

View file

@ -0,0 +1,105 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.Try
import scala.annotation.nowarn
import org.apache.pekko
import pekko.cluster.metrics.StandardMetrics._
import pekko.testkit._
@nowarn
class MetricsCollectorSpec
extends PekkoSpec(MetricsConfig.defaultEnabled)
with ImplicitSender
with MetricsCollectorFactory {
val collector = createMetricsCollector
"Metric must" must {
"merge 2 metrics that are tracking the same metric" in {
for (_ <- 1 to 20) {
val sample1 = collector.sample().metrics
val sample2 = collector.sample().metrics
sample2.flatMap(latest =>
sample1.collect {
case peer if latest.sameAs(peer) =>
val m = peer :+ latest
m.value should ===(latest.value)
m.isSmooth should ===(peer.isSmooth || latest.isSmooth)
m
})
val sample3 = collector.sample().metrics
val sample4 = collector.sample().metrics
sample4.flatMap(latest =>
sample3.collect {
case peer if latest.sameAs(peer) =>
val m = peer :+ latest
m.value should ===(latest.value)
m.isSmooth should ===(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
case unexpected =>
fail(s"Unexpected metric type $unexpected")
}
}
"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 ===(true)
c.heapCommitted(heap).isDefined should ===(true)
c.processors.isDefined should ===(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,237 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import java.io.Closeable
import java.util.logging.LogManager
import scala.language.postfixOps
import org.scalatestplus.mockito.MockitoSugar
import org.slf4j.bridge.SLF4JBridgeHandler
import org.apache.pekko
import pekko.actor.Actor
import pekko.actor.ActorLogging
import pekko.actor.ActorRef
import pekko.actor.ActorSystem
import pekko.actor.Address
import pekko.actor.Deploy
import pekko.actor.ExtendedActorSystem
import pekko.actor.PoisonPill
import pekko.actor.Props
import pekko.dispatch.Dispatchers
import pekko.dispatch.RequiresMessageQueue
import pekko.dispatch.UnboundedMessageQueueSemantics
import pekko.remote.RARP
import pekko.testkit.PekkoSpec
/**
* 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.0, increase(cpuCombined): _*)
when(cpuPerc.getStolen).thenReturn(0.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: PekkoSpec =>
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(if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" else "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 = """
pekko.cluster.metrics {
collector {
enabled = on
sample-interval = 1s
gossip-interval = 1s
}
}
pekko.actor.provider = remote
pekko.remote.classic.netty.tcp.port = 0
pekko.remote.artery.canonical.port = 0
"""
/** Test w/o cluster, with collection disabled. */
val defaultDisabled = """
pekko.cluster.metrics {
collector {
enabled = off
}
}
pekko.actor.provider = remote
pekko.remote.classic.netty.tcp.port = 0
pekko.remote.artery.canonical.port = 0
"""
/** Test in cluster, with manual collection activation, collector mock, fast. */
val clusterSigarMock = """
pekko.cluster.metrics {
periodic-tasks-initial-delay = 100ms
collector {
enabled = off
sample-interval = 200ms
gossip-interval = 200ms
provider = "org.apache.pekko.cluster.metrics.MockitoSigarMetricsCollector"
fallback = false
}
}
pekko.actor.provider = "cluster"
pekko.remote.classic.netty.tcp.port = 0
pekko.remote.artery.canonical.port = 0
"""
}
/**
* 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,106 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics
import scala.Vector
import com.typesafe.config.ConfigFactory
import org.apache.pekko
import pekko.actor.Address
import pekko.actor.RootActorPath
import pekko.remote.RARP
import pekko.routing.ActorRefRoutee
import pekko.routing.ActorSelectionRoutee
import pekko.testkit.PekkoSpec
class WeightedRouteesSpec extends PekkoSpec(ConfigFactory.parseString("""
pekko.actor.provider = "cluster"
pekko.remote.classic.netty.tcp.port = 0
pekko.remote.artery.canonical.port = 0
""")) {
val protocol =
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
else "akka.tcp"
val a1 = Address(protocol, "sys", "a1", 2551)
val b1 = Address(protocol, "sys", "b1", 2551)
val c1 = Address(protocol, "sys", "c1", 2551)
val d1 = Address(protocol, "sys", "d1", 2551)
val routeeA = ActorSelectionRoutee(system.actorSelection(RootActorPath(a1) / "user" / "a"))
val routeeB = ActorSelectionRoutee(system.actorSelection(RootActorPath(b1) / "user" / "b"))
val routeeC = ActorSelectionRoutee(system.actorSelection(RootActorPath(c1) / "user" / "c"))
val routees = Vector(routeeA, routeeB, routeeC)
val testActorRoutee = ActorRefRoutee(testActor)
"WeightedRoutees" must {
"allocate weighted routees" in {
val weights = Map(a1 -> 1, b1 -> 3, c1 -> 10)
val weighted = new WeightedRoutees(routees, a1, weights)
weighted(1) should ===(routeeA)
(2 to 4).foreach { weighted(_) should ===(routeeB) }
(5 to 14).foreach { weighted(_) should ===(routeeC) }
weighted.total should ===(14)
}
"check boundaries" in {
val empty = new WeightedRoutees(Vector(), a1, Map.empty)
empty.isEmpty should ===(true)
intercept[IllegalArgumentException] {
empty.total
}
val empty2 = new WeightedRoutees(Vector(routeeA), a1, Map(a1 -> 0))
empty2.isEmpty should ===(true)
intercept[IllegalArgumentException] {
empty2.total
}
intercept[IllegalArgumentException] {
empty2(0)
}
val weighted = new WeightedRoutees(routees, a1, Map.empty)
weighted.total should ===(3)
intercept[IllegalArgumentException] {
weighted(0)
}
intercept[IllegalArgumentException] {
weighted(4)
}
}
"allocate routees for undefined weight" in {
val weights = Map(a1 -> 1, b1 -> 7)
val weighted = new WeightedRoutees(routees, a1, weights)
weighted(1) should ===(routeeA)
(2 to 8).foreach { weighted(_) should ===(routeeB) }
// undefined, uses the mean of the weights, i.e. 4
(9 to 12).foreach { weighted(_) should ===(routeeC) }
weighted.total should ===(12)
}
"allocate weighted local routees" in {
val weights = Map(a1 -> 2, b1 -> 1, c1 -> 10)
val routees2 = Vector(testActorRoutee, routeeB, routeeC)
val weighted = new WeightedRoutees(routees2, a1, weights)
(1 to 2).foreach { weighted(_) should ===(testActorRoutee) }
(3 to weighted.total).foreach { weighted(_) should not be testActorRoutee }
}
"not allocate ref with weight zero" in {
val weights = Map(a1 -> 0, b1 -> 2, c1 -> 10)
val weighted = new WeightedRoutees(routees, a1, weights)
(1 to weighted.total).foreach { weighted(_) should not be routeeA }
}
}
}

View file

@ -0,0 +1,78 @@
/*
* Copyright (C) 2009-2022 Lightbend Inc. <https://www.lightbend.com>
*/
package org.apache.pekko.cluster.metrics.protobuf
import org.apache.pekko
import pekko.actor.{ Address, ExtendedActorSystem }
import pekko.cluster.MemberStatus
import pekko.cluster.TestMember
import pekko.cluster.metrics._
import pekko.testkit.PekkoSpec
class MessageSerializerSpec extends PekkoSpec("""
pekko.actor.provider = cluster
pekko.remote.classic.netty.tcp.port = 0
pekko.remote.artery.canonical.port = 0
""") {
val serializer = new MessageSerializer(system.asInstanceOf[ExtendedActorSystem])
def checkSerialization(obj: AnyRef): Unit = {
val blob = serializer.toBinary(obj)
val ref = serializer.fromBinary(blob, serializer.manifest(obj))
obj match {
case _ =>
ref should ===(obj)
}
}
import MemberStatus._
val a1 = TestMember(Address("akka", "sys", "a", 2552), Joining, Set.empty)
val b1 = TestMember(Address("akka", "sys", "b", 2552), Up, Set("r1"))
val c1 = TestMember(Address("akka", "sys", "c", 2552), Leaving, Set("r2"))
val d1 = TestMember(Address("akka", "sys", "d", 2552), Exiting, Set("r1", "r2"))
val e1 = TestMember(Address("akka", "sys", "e", 2552), Down, Set("r3"))
val f1 = TestMember(Address("akka", "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))
}
}
"AdaptiveLoadBalancingPool" must {
"be serializable" in {
val simplePool = AdaptiveLoadBalancingPool()
checkSerialization(simplePool)
val complicatedPool = AdaptiveLoadBalancingPool(
metricsSelector =
MixMetricsSelector(Vector(CpuMetricsSelector, HeapMetricsSelector, SystemLoadAverageMetricsSelector)),
nrOfInstances = 7,
routerDispatcher = "my-dispatcher",
usePoolDispatcher = true)
checkSerialization(complicatedPool)
}
}
}