replace unicode arrows

* ⇒, →, ←
* because we don't want to show them in documentation snippets and
  then it's complicated to avoid that when snippets are
  located in src/test/scala in individual modules
* dont replace object `→` in FSM.scala and PersistentFSM.scala
This commit is contained in:
Patrik Nordwall 2019-02-09 15:25:39 +01:00
parent e4d38f92a4
commit 5c96a5f556
1521 changed files with 18846 additions and 18786 deletions

View file

@ -64,12 +64,12 @@ private[metrics] class ClusterMetricsSupervisor extends Actor with ActorLogging
}
override def receive = {
case CollectionStartMessage
case CollectionStartMessage =>
children.foreach(stop)
collectorInstance += 1
actorOf(Props(classOf[ClusterMetricsCollector]), collectorName)
log.debug(s"Collection started.")
case CollectionStopMessage
case CollectionStopMessage =>
children.foreach(stop)
log.debug(s"Collection stopped.")
}
@ -169,19 +169,19 @@ private[metrics] class ClusterMetricsCollector extends Actor with ActorLogging {
}
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)
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
case _: MemberEvent => // not interested in other types of MemberEvent
}
@ -211,7 +211,7 @@ private[metrics] class ClusterMetricsCollector extends Actor with ActorLogging {
*/
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
case m if m.status == MemberStatus.Up || m.status == MemberStatus.WeaklyUp => m.address
}
/**

View file

@ -45,7 +45,7 @@ class ClusterMetricsExtension(system: ExtendedActorSystem) extends Extension {
* Supervision strategy.
*/
private[metrics] val strategy = system.dynamicAccess.createInstanceFor[SupervisorStrategy](
SupervisorStrategyProvider, immutable.Seq(classOf[Config] SupervisorStrategyConfiguration))
SupervisorStrategyProvider, immutable.Seq(classOf[Config] -> SupervisorStrategyConfiguration))
.getOrElse {
val log: LoggingAdapter = Logging(system, getClass.getName)
log.error(s"Configured strategy provider ${SupervisorStrategyProvider} failed to load, using default ${classOf[ClusterMetricsStrategy].getName}.")

View file

@ -74,10 +74,10 @@ final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsS
}
updateWeightedRoutees() match {
case Some(weighted)
case Some(weighted) =>
if (weighted.isEmpty) NoRoutee
else weighted(ThreadLocalRandom.current.nextInt(weighted.total) + 1)
case None
case None =>
routees(ThreadLocalRandom.current.nextInt(routees.size))
}
@ -172,11 +172,11 @@ final case class AdaptiveLoadBalancingPool(
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
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))
case _ => throw new IllegalArgumentException("Expected AdaptiveLoadBalancingPool, got [%s]".format(other))
}
}
@ -256,10 +256,10 @@ case object HeapMetricsSelector extends CapacityMetricsSelector {
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
nodeMetrics.collect {
case HeapMemory(address, _, used, committed, max)
case HeapMemory(address, _, used, committed, max) =>
val capacity = max match {
case None (committed - used).toDouble / committed
case Some(m) (m - used).toDouble / m
case None => (committed - used).toDouble / committed
case Some(m) => (m - used).toDouble / m
}
(address, capacity)
}.toMap
@ -296,7 +296,7 @@ case object CpuMetricsSelector extends CapacityMetricsSelector {
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
nodeMetrics.collect {
case Cpu(address, _, _, Some(cpuCombined), Some(cpuStolen), _)
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
@ -322,7 +322,7 @@ case object SystemLoadAverageMetricsSelector extends CapacityMetricsSelector {
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
nodeMetrics.collect {
case Cpu(address, _, Some(systemLoadAverage), _, _, processors)
case Cpu(address, _, Some(systemLoadAverage), _, _, processors) =>
val capacity = 1.0 - math.min(1.0, systemLoadAverage / processors)
(address, capacity)
}.toMap
@ -370,11 +370,11 @@ abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMe
// 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))
case (acc, (address, capacity)) =>
val (sum, count) = acc(address)
acc + (address ((sum + capacity, count + 1)))
acc + (address -> ((sum + capacity, count + 1)))
}.map {
case (address, (sum, count)) address (sum / count)
case (address, (sum, count)) => address -> (sum / count)
}
}
@ -383,14 +383,14 @@ abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMe
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)
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(
case exception => throw new IllegalArgumentException(
(s"Cannot instantiate metrics-selector [$fqn], " +
"make sure it extends [akka.cluster.routing.MetricsSelector] and " +
"has constructor with [com.typesafe.config.Config] parameter"), exception)
@ -433,10 +433,10 @@ abstract class CapacityMetricsSelector extends MetricsSelector {
def weights(capacity: Map[Address, Double]): Map[Address, Int] = {
if (capacity.isEmpty) Map.empty[Address, Int]
else {
val (_, min) = capacity.minBy { case (_, c) c }
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) }
capacity map { case (address, c) => (address -> math.round((c) / divisor).toInt) }
}
}
@ -462,12 +462,12 @@ private[metrics] class WeightedRoutees(routees: immutable.IndexedSeq[Routee], se
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 ActorRefRoutee(ref) => ref.path.address
case ActorSelectionRoutee(sel) => sel.anchor.path.address
}
a match {
case Address(_, _, None, None) selfAddress
case a a
case Address(_, _, None, None) => selfAddress
case a => a
}
}
val buckets = new Array[Int](routees.size)
@ -475,7 +475,7 @@ private[metrics] class WeightedRoutees(routees: immutable.IndexedSeq[Routee], se
val w = weights.withDefaultValue(meanWeight) // we dont necessarily have metrics for all addresses
var i = 0
var sum = 0
routees foreach { r
routees foreach { r =>
sum += w(fullAddress(r))
buckets(i) = sum
i += 1
@ -527,8 +527,8 @@ private[metrics] class AdaptiveLoadBalancingMetricsListener(routingLogic: Adapti
override def postStop(): Unit = extension.unsubscribe(self)
def receive = {
case event: ClusterMetricsChanged routingLogic.metricsChanged(event)
case _ // ignore
case event: ClusterMetricsChanged => routingLogic.metricsChanged(event)
case _ => // ignore
}
}

View file

@ -20,8 +20,8 @@ case class ClusterMetricsSettings(config: Config) {
// Extension.
val MetricsDispatcher: String = cc.getString("dispatcher") match {
case "" Dispatchers.DefaultDispatcherId
case id id
case "" => Dispatchers.DefaultDispatcherId
case id => id
}
val PeriodicTasksInitialDelay: FiniteDuration = cc.getMillisDuration("periodic-tasks-initial-delay")
val NativeLibraryExtractFolder: String = cc.getString("native-library-extract-folder")

View file

@ -28,10 +28,10 @@ object ClusterMetricsStrategy {
* [[akka.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
case _: ActorInitializationException => Stop
case _: ActorKilledException => Stop
case _: DeathPactException => Stop
case _: Throwable => Restart
}
}

View file

@ -32,9 +32,9 @@ final case class Metric private[metrics] (name: String, value: Number, average:
*/
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)
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
@ -42,8 +42,8 @@ final case class Metric private[metrics] (name: String, value: Number, average:
* 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
case Some(avg) => avg.value
case None => value.doubleValue
}
/**
@ -58,8 +58,8 @@ final case class Metric private[metrics] (name: String, value: Number, average:
override def hashCode = name.##
override def equals(obj: Any) = obj match {
case other: Metric sameAs(other)
case _ false
case other: Metric => sameAs(other)
case _ => false
}
}
@ -82,13 +82,13 @@ object Metric extends MetricNumericConverter {
* 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
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
case Some(alpha) => Some(EWMA(value, alpha))
case None => None
}
}
@ -126,8 +126,8 @@ object StandardMetrics {
*/
def unapply(nodeMetrics: NodeMetrics): Option[(Address, Long, Long, Long, Option[Long])] = {
for {
used nodeMetrics.metric(HeapMemoryUsed)
committed nodeMetrics.metric(HeapMemoryCommitted)
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))
@ -140,10 +140,10 @@ object StandardMetrics {
* contains necessary heap metrics, otherwise it returns null.
*/
def extractHeapMemory(nodeMetrics: NodeMetrics): HeapMemory = nodeMetrics match {
case HeapMemory(address, timestamp, used, committed, max)
case HeapMemory(address, timestamp, used, committed, max) =>
// note that above extractor returns tuple
HeapMemory(address, timestamp, used, committed, max)
case _ null
case _ => null
}
/**
@ -174,7 +174,7 @@ object StandardMetrics {
*/
def unapply(nodeMetrics: NodeMetrics): Option[(Address, Long, Option[Double], Option[Double], Option[Double], Int)] = {
for {
processors nodeMetrics.metric(Processors)
processors <- nodeMetrics.metric(Processors)
} yield (nodeMetrics.address, nodeMetrics.timestamp,
nodeMetrics.metric(SystemLoadAverage).map(_.smoothValue),
nodeMetrics.metric(CpuCombined).map(_.smoothValue),
@ -189,10 +189,10 @@ object StandardMetrics {
* contains necessary cpu metrics, otherwise it returns null.
*/
def extractCpu(nodeMetrics: NodeMetrics): Cpu = nodeMetrics match {
case Cpu(address, timestamp, systemLoadAverage, cpuCombined, cpuStolen, processors)
case Cpu(address, timestamp, systemLoadAverage, cpuCombined, cpuStolen, processors) =>
// note that above extractor returns tuple
Cpu(address, timestamp, systemLoadAverage, cpuCombined, cpuStolen, processors)
case _ null
case _ => null
}
/**
@ -216,13 +216,13 @@ object StandardMetrics {
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
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
case Some(x) => require(0.0 <= x && x <= 1.0, s"cpuStolen must be between [0.0 - 1.0], was [$x]")
case None =>
}
}
@ -243,21 +243,21 @@ private[metrics] trait MetricNumericConverter {
* <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)
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]")
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]")
}
}
@ -296,8 +296,8 @@ final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Met
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
latest <- latestNode.metrics
current <- currentNode.metrics
if (latest sameAs current)
} yield {
current :+ latest
@ -308,7 +308,7 @@ final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Met
copy(metrics = merged, timestamp = latestNode.timestamp)
}
def metric(key: String): Option[Metric] = metrics.collectFirst { case m if m.name == key m }
def metric(key: String): Option[Metric] = metrics.collectFirst { case m if m.name == key => m }
/**
* Java API
@ -323,8 +323,8 @@ final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Met
override def hashCode = address.##
override def equals(obj: Any) = obj match {
case other: NodeMetrics sameAs(other)
case _ false
case other: NodeMetrics => sameAs(other)
case _ => false
}
}
@ -359,20 +359,20 @@ private[metrics] final case class MetricsGossip(nodes: Set[NodeMetrics]) {
* Adds new remote [[NodeMetrics]] and merges existing from a remote gossip.
*/
def merge(otherGossip: MetricsGossip): MetricsGossip =
otherGossip.nodes.foldLeft(this) { (gossip, nodeMetrics) gossip :+ nodeMetrics }
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)
case Some(existingNodeMetrics) =>
copy(nodes = nodes - existingNodeMetrics + (existingNodeMetrics update newNodeMetrics))
case None copy(nodes = nodes + 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 }
def nodeMetricsFor(address: Address): Option[NodeMetrics] = nodes find { n => n.address == address }
}

View file

@ -13,7 +13,7 @@ import java.lang.management.MemoryMXBean
import java.lang.management.ManagementFactory
import java.lang.management.OperatingSystemMXBean
import java.lang.management.MemoryUsage
import java.lang.System.{ currentTimeMillis newTimestamp }
import java.lang.System.{ currentTimeMillis => newTimestamp }
import akka.cluster.Cluster
import java.io.Closeable
import org.hyperic.sigar.SigarProxy
@ -60,7 +60,7 @@ private[metrics] object MetricsCollector {
def create(provider: String) = TryNative {
log.debug(s"Trying ${provider}.")
system.asInstanceOf[ExtendedActorSystem].dynamicAccess
.createInstanceFor[MetricsCollector](provider, List(classOf[ActorSystem] system)).get
.createInstanceFor[MetricsCollector](provider, List(classOf[ActorSystem] -> system)).get
}
val collector = if (useCustom)
@ -71,7 +71,7 @@ private[metrics] object MetricsCollector {
create(collectorCustom) orElse create(collectorSigar) orElse create(collectorJMX)
collector.recover {
case e throw new ConfigurationException(s"Could not create metrics collector: ${e}")
case e => throw new ConfigurationException(s"Could not create metrics collector: ${e}")
}.get
}
}

View file

@ -39,7 +39,7 @@ trait SigarProvider {
SigarProvider.close(sigar)
true
} catch {
case e: Throwable false
case e: Throwable => false
}
/** Create sigar and verify it works. */
@ -68,7 +68,7 @@ trait SigarProvider {
provisionSigarLibrary()
verifiedSigarInstance
} recover {
case e: Throwable throw new RuntimeException("Failed to load sigar:", e)
case e: Throwable => throw new RuntimeException("Failed to load sigar:", e)
} get
}
@ -96,9 +96,9 @@ case class DefaultSigarProvider(settings: ClusterMetricsSettings) extends SigarP
* INTERNAL API
*/
private[metrics] object TryNative {
def apply[T](r: T): Try[T] =
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)
case e: Throwable => Failure(e)
}
}

View file

@ -6,10 +6,10 @@ package akka.cluster.metrics.protobuf
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
import java.{ lang jl }
import java.{ lang => jl }
import akka.actor.{ Address, ExtendedActorSystem }
import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages cm }
import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages => cm }
import akka.cluster.metrics._
import akka.serialization.{ BaseSerializer, SerializationExtension, Serializers, SerializerWithStringManifest }
import akka.util.ClassLoaderObjectInputStream
@ -40,24 +40,24 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
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 _
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 _
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}]")
}
@ -75,8 +75,8 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
val buffer = new Array[Byte](BufferSize)
@tailrec def readChunk(): Unit = in.read(buffer) match {
case -1 ()
case n
case -1 => ()
case n =>
out.write(buffer, 0, n)
readChunk()
}
@ -87,20 +87,20 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
}
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(
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))
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.")
case _ => throw new IllegalArgumentException(s"Address [$address] could not be serialized: host or port missing.")
}
def adaptiveLoadBalancingPoolToBinary(alb: AdaptiveLoadBalancingPool): Array[Byte] = {
@ -132,7 +132,7 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
private def mixMetricSelectorToBinary(mms: MixMetricsSelector): Array[Byte] = {
val builder = cm.MixMetricsSelector.newBuilder()
mms.selectors.foreach { selector
mms.selectors.foreach { selector =>
builder.addSelectors(metricsSelectorToProto(selector))
}
builder.build().toByteArray
@ -167,32 +167,32 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
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")
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 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)
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 _
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)
@ -232,11 +232,11 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
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
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))
@ -284,7 +284,7 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
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)
.map(s => metricSelectorFromProto(s).asInstanceOf[CapacityMetricsSelector]).toIndexedSeq)
}
def metricSelectorFromProto(selector: cm.MetricsSelector): MetricsSelector =

View file

@ -24,7 +24,7 @@ trait ClusterMetricsCommonConfig extends MultiNodeConfig {
def nodeList = Seq(node1, node2, node3, node4, node5)
// Extract individual sigar library for every node.
nodeList foreach { role
nodeList foreach { role =>
nodeConfig(role) {
parseString(s"akka.cluster.metrics.native-library-extract-folder=$${user.dir}/target/native/" + role.name)
}
@ -89,7 +89,7 @@ abstract class ClusterMetricsEnabledSpec extends MultiNodeSpec(ClusterMetricsEna
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 }
Some(new PrintWriter(file)) map { p => p.write(text); p.close }
}
saveApplicationConf()

View file

@ -27,14 +27,14 @@ object AdaptiveLoadBalancingRouterConfig extends MultiNodeConfig {
class Echo extends Actor {
def receive = {
case _ sender() ! Reply(Cluster(context.system).selfAddress)
case _ => sender() ! Reply(Cluster(context.system).selfAddress)
}
}
class Memory extends Actor with ActorLogging {
var usedMemory: Array[Array[Int]] = _
def receive = {
case AllocateMemory
case AllocateMemory =>
val heap = ManagementFactory.getMemoryMXBean.getHeapMemoryUsage
// getMax can be undefined (-1)
val max = math.max(heap.getMax, heap.getCommitted)
@ -59,7 +59,7 @@ object AdaptiveLoadBalancingRouterConfig extends MultiNodeConfig {
def nodeList = Seq(node1, node2, node3)
// Extract individual sigar library for every node.
nodeList foreach { role
nodeList foreach { role =>
nodeConfig(role) {
ConfigFactory.parseString(s"akka.cluster.metrics.native-library-extract-folder=$${user.dir}/target/native/" + role.name)
}
@ -118,11 +118,11 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
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)
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
(receiveWhile(5 seconds, messages = expectedReplies) {
case Reply(address) address
case Reply(address) => address
}).foldLeft(zero) {
case (replyMap, address) replyMap + (address (replyMap(address) + 1))
case (replyMap, address) => replyMap + (address -> (replyMap(address) + 1))
}
}
@ -130,8 +130,8 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
* 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
case Address(_, _, None, None) => cluster.selfAddress
case a => a
}
def startRouter(name: String): ActorRef = {
@ -144,7 +144,7 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router).size should ===(roles.size) }
val routees = currentRoutees(router)
routees.map { case ActorRefRoutee(ref) fullAddress(ref) }.toSet should ===(roles.map(address).toSet)
routees.map { case ActorRefRoutee(ref) => fullAddress(ref) }.toSet should ===(roles.map(address).toSet)
router
}
@ -166,7 +166,7 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
metricsAwait()
val iterationCount = 100
1 to iterationCount foreach { _
1 to iterationCount foreach { _ =>
router1 ! "hit"
// wait a while between each message, since metrics is collected periodically
Thread.sleep(10)
@ -203,7 +203,7 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
metricsAwait()
val iterationCount = 3000
1 to iterationCount foreach { _
1 to iterationCount foreach { _ =>
router2 ! "hit"
}
@ -223,7 +223,7 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router3).size should ===(9) }
val routees = currentRoutees(router3)
routees.map { case ActorRefRoutee(ref) fullAddress(ref) }.toSet should ===(Set(address(node1)))
routees.map { case ActorRefRoutee(ref) => fullAddress(ref) }.toSet should ===(Set(address(node1)))
}
enterBarrier("after-4")
}
@ -234,7 +234,7 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
// it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router4).size should ===(6) }
val routees = currentRoutees(router4)
routees.map { case ActorRefRoutee(ref) fullAddress(ref) }.toSet should ===(Set(
routees.map { case ActorRefRoutee(ref) => fullAddress(ref) }.toSet should ===(Set(
address(node1), address(node2), address(node3)))
}
enterBarrier("after-5")

View file

@ -24,7 +24,7 @@ object StatsSampleSpecConfig extends MultiNodeConfig {
def nodeList = Seq(first, second, third)
// Extract individual sigar library for every node.
nodeList foreach { role
nodeList foreach { role =>
nodeConfig(role) {
ConfigFactory.parseString(s"""
# Enable metrics extension in akka-cluster-metrics.
@ -105,7 +105,7 @@ abstract class StatsSampleSpec extends MultiNodeSpec(StatsSampleSpecConfig)
system.actorOf(Props[StatsWorker], "statsWorker")
system.actorOf(Props[StatsService], "statsService")
receiveN(3).collect { case MemberUp(m) m.address }.toSet should be(
receiveN(3).collect { case MemberUp(m) => m.address }.toSet should be(
Set(firstAddress, secondAddress, thirdAddress))
Cluster(system).unsubscribe(testActor)

View file

@ -20,13 +20,13 @@ class StatsService extends Actor {
name = "workerRouter")
def receive = {
case StatsJob(text) if text != ""
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
words foreach { word =>
workerRouter.tell(
ConsistentHashableEnvelope(word, word), aggregator)
}
@ -38,14 +38,14 @@ class StatsAggregator(expectedResults: Int, replyTo: ActorRef) extends Actor {
context.setReceiveTimeout(3.seconds)
def receive = {
case wordCount: Int
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
case ReceiveTimeout =>
replyTo ! JobFailed("Service unavailable, try again later")
context.stop(self)
}

View file

@ -10,12 +10,12 @@ import akka.actor.Actor
class StatsWorker extends Actor {
var cache = Map.empty[String, Int]
def receive = {
case word: String
case word: String =>
val length = cache.get(word) match {
case Some(x) x
case None
case Some(x) => x
case None =>
val x = word.length
cache += (word x)
cache += (word -> x)
x
}

View file

@ -72,10 +72,10 @@ class MetricsExtensionSpec extends AkkaSpec(MetricsConfig.clusterSigarMock)
expected.size should ===(sampleCount)
history.zip(expected) foreach {
case (mockMetrics, expectedData)
case (mockMetrics, expectedData) =>
(mockMetrics, expectedData) match {
case (Cpu(_, _, loadAverageMock, cpuCombinedMock, cpuStolenMock, _),
(loadAverageEwma, cpuCombinedEwma, cpuStolenEwma))
(loadAverageEwma, cpuCombinedEwma, cpuStolenEwma)) =>
loadAverageMock.get should ===(loadAverageEwma +- epsilon)
cpuCombinedMock.get should ===(cpuCombinedEwma +- epsilon)
cpuStolenMock.get should ===(cpuStolenEwma +- epsilon)
@ -108,7 +108,7 @@ class MetricsExtensionSpec extends AkkaSpec(MetricsConfig.clusterSigarMock)
}
(1 to 3) foreach { step cycle() }
(1 to 3) foreach { step => cycle() }
}

View file

@ -63,15 +63,15 @@ class MetricsSelectorSpec extends WordSpec with Matchers {
"CapacityMetricsSelector" must {
"calculate weights from capacity" in {
val capacity = Map(a1 0.6, b1 0.3, c1 0.1)
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))
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 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))
weights should ===(Map(a1 -> 0, b1 -> 100, c1 -> 1, d1 -> 0))
}
}

View file

@ -32,7 +32,7 @@ class EWMASpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollec
val d4 = d3 :+ 10.0
d4.value should ===(453.64 +- 0.01)
val dn = (1 to 100).foldLeft(d0)((d, _) d :+ 10.0)
val dn = (1 to 100).foldLeft(d0)((d, _) => d :+ 10.0)
dn.value should ===(10.0 +- 0.1)
}
@ -75,14 +75,14 @@ class EWMASpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollec
"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 { _
(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
val changes = collector.sample.metrics.flatMap { latest =>
streamingDataSet.get(latest.name) match {
case None Some(latest)
case Some(previous)
case None => Some(latest)
case Some(previous) =>
if (latest.isSmooth && latest.value != previous.value) {
val updated = previous :+ latest
updated.isSmooth should ===(true)
@ -91,7 +91,7 @@ class EWMASpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollec
} else None
}
}
streamingDataSet ++= changes.map(m m.name m)
streamingDataSet ++= changes.map(m => m.name -> m)
}
}
}

View file

@ -11,7 +11,7 @@ import scala.util.Failure
import akka.actor.Address
import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender
import java.lang.System.{ currentTimeMillis newTimestamp }
import java.lang.System.{ currentTimeMillis => newTimestamp }
class MetricNumericConverterSpec extends WordSpec with Matchers with MetricNumericConverter {
@ -172,7 +172,7 @@ class MetricsGossipSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with Impl
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) }
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 {
@ -239,10 +239,10 @@ class MetricValuesSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with Metri
val node2 = NodeMetrics(Address("akka.tcp", "sys", "a", 2555), 1, collector.sample.metrics)
val nodes: Seq[NodeMetrics] = {
(1 to 100).foldLeft(List(node1, node2)) { (nodes, _)
nodes map { n
n.copy(metrics = collector.sample.metrics.flatMap(latest n.metrics.collect {
case streaming if latest sameAs streaming streaming :+ latest
(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
}))
}
}
@ -256,9 +256,9 @@ class MetricValuesSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with Metri
}
"extract expected MetricValue types for load balancing" in {
nodes foreach { node
nodes foreach { node =>
node match {
case HeapMemory(address, _, used, committed, _)
case HeapMemory(address, _, used, committed, _) =>
used should be > (0L)
committed should be >= (used)
// Documentation java.lang.management.MemoryUsage says that committed <= max,
@ -269,7 +269,7 @@ class MetricValuesSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with Metri
}
node match {
case Cpu(address, _, systemLoadAverageOption, cpuCombinedOption, cpuStolenOption, processors)
case Cpu(address, _, systemLoadAverageOption, cpuCombinedOption, cpuStolenOption, processors) =>
processors should be > (0)
if (systemLoadAverageOption.isDefined)
systemLoadAverageOption.get should be >= (0.0)

View file

@ -19,11 +19,11 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with I
"Metric must" must {
"merge 2 metrics that are tracking the same metric" in {
for (i 1 to 20) {
for (i <- 1 to 20) {
val sample1 = collector.sample.metrics
val sample2 = collector.sample.metrics
val merged12 = sample2 flatMap (latest sample1 collect {
case peer if latest sameAs peer
val merged12 = 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)
@ -32,8 +32,8 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with I
val sample3 = collector.sample.metrics
val sample4 = collector.sample.metrics
val merged34 = sample4 flatMap (latest sample3 collect {
case peer if latest sameAs peer
val merged34 = 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)
@ -51,22 +51,22 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with I
"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 }
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)
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)
case (CpuCombined, b) =>
b.doubleValue should be <= (1.0)
b.doubleValue should be >= (0.0)
case (CpuStolen, b)
case (CpuStolen, b) =>
b.doubleValue should be <= (1.0)
b.doubleValue should be >= (0.0)
@ -85,7 +85,7 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with I
}
"collect 50 node metrics samples in an acceptable duration" taggedAs LongRunningTest in within(10 seconds) {
(1 to 50) foreach { _
(1 to 50) foreach { _ =>
val sample = collector.sample
sample.metrics.size should be >= (3)
Thread.sleep(100)

View file

@ -93,7 +93,7 @@ case class MockitoSigarProvider(
*
* TODO change factory after https://github.com/akka/akka/issues/16369
*/
trait MetricsCollectorFactory { this: AkkaSpec
trait MetricsCollectorFactory { this: AkkaSpec =>
import MetricsConfig._
import org.hyperic.sigar.Sigar
@ -106,7 +106,7 @@ trait MetricsCollectorFactory { this: AkkaSpec ⇒
new SigarMetricsCollector(selfAddress, defaultDecayFactor, new Sigar())
//new SigarMetricsCollector(selfAddress, defaultDecayFactor, SimpleSigarProvider().createSigarInstance)
} catch {
case e: Throwable
case e: Throwable =>
log.warning("Sigar failed to load. Using JMX. Reason: " + e.toString)
new JmxMetricsCollector(selfAddress, defaultDecayFactor)
}
@ -206,10 +206,10 @@ class ClusterMetricsView(system: ExtendedActorSystem) extends Closeable {
override def preStart(): Unit = extension.subscribe(self)
override def postStop(): Unit = extension.unsubscribe(self)
def receive = {
case ClusterMetricsChanged(nodes)
case ClusterMetricsChanged(nodes) =>
currentMetricsSet = nodes
collectedMetricsList = nodes :: collectedMetricsList
case _
case _ =>
// Ignore.
}
}).withDispatcher(Dispatchers.DefaultDispatcherId).withDeploy(Deploy.local), name = "metrics-event-bus-listener")

View file

@ -37,7 +37,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
"WeightedRoutees" must {
"allocate weighted routees" in {
val weights = Map(a1 1, b1 3, c1 10)
val weights = Map(a1 -> 1, b1 -> 3, c1 -> 10)
val weighted = new WeightedRoutees(routees, a1, weights)
weighted(1) should ===(routeeA)
@ -53,7 +53,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
empty.total
}
val empty2 = new WeightedRoutees(Vector(routeeA), a1, Map(a1 0))
val empty2 = new WeightedRoutees(Vector(routeeA), a1, Map(a1 -> 0))
empty2.isEmpty should ===(true)
intercept[IllegalArgumentException] {
empty2.total
@ -73,7 +73,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
}
"allocate routees for undefined weight" in {
val weights = Map(a1 1, b1 7)
val weights = Map(a1 -> 1, b1 -> 7)
val weighted = new WeightedRoutees(routees, a1, weights)
weighted(1) should ===(routeeA)
@ -84,7 +84,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
}
"allocate weighted local routees" in {
val weights = Map(a1 2, b1 1, c1 10)
val weights = Map(a1 -> 2, b1 -> 1, c1 -> 10)
val routees2 = Vector(testActorRoutee, routeeB, routeeC)
val weighted = new WeightedRoutees(routees2, a1, weights)
@ -93,7 +93,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
}
"not allocate ref with weight zero" in {
val weights = Map(a1 0, b1 2, c1 10)
val weights = Map(a1 -> 0, b1 -> 2, c1 -> 10)
val weighted = new WeightedRoutees(routees, a1, weights)
1 to weighted.total foreach { weighted(_) should not be (routeeA) }

View file

@ -24,7 +24,7 @@ class MessageSerializerSpec extends AkkaSpec(
val blob = serializer.toBinary(obj)
val ref = serializer.fromBinary(blob, serializer.manifest(obj))
obj match {
case _
case _ =>
ref should ===(obj)
}