Rename sbt akka modules
Co-authored-by: Sean Glover <sean@seanglover.com>
This commit is contained in:
parent
b92b749946
commit
24c03cde19
2930 changed files with 1466 additions and 1462 deletions
File diff suppressed because it is too large
Load diff
100
cluster-metrics/src/main/protobuf/ClusterMetricsMessages.proto
Normal file
100
cluster-metrics/src/main/protobuf/ClusterMetricsMessages.proto
Normal 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;
|
||||
}
|
||||
111
cluster-metrics/src/main/resources/reference.conf
Normal file
111
cluster-metrics/src/main/resources/reference.conf
Normal 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 sample’s 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"
|
||||
}
|
||||
}
|
||||
|
|
@ -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))
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
|
@ -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 don’t 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
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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")
|
||||
|
||||
}
|
||||
|
|
@ -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
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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 sample’s 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)
|
||||
}
|
||||
}
|
||||
|
|
@ -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 <= max if max is defined.
|
||||
* A memory allocation may fail if it attempts to increase the used memory such that used > committed
|
||||
* even if used <= max is true (e.g. when the system virtual memory is low).
|
||||
*
|
||||
* @param address [[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
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
@ -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]
|
||||
|
||||
}
|
||||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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
|
||||
}
|
||||
|
|
@ -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
|
||||
39
cluster-metrics/src/test/resources/logback-test.xml
Normal file
39
cluster-metrics/src/test/resources/logback-test.xml
Normal 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>
|
||||
|
|
@ -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()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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
|
||||
|
||||
}
|
||||
|
|
@ -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 }
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue