Enable fatal warnings for cluster metrics and slf4j (#26678)
This commit is contained in:
parent
4bb60bbcc8
commit
f798481de5
12 changed files with 67 additions and 48 deletions
|
|
@ -39,7 +39,7 @@ trait SigarProvider {
|
|||
SigarProvider.close(sigar)
|
||||
true
|
||||
} catch {
|
||||
case e: Throwable => false
|
||||
case _: Throwable => false
|
||||
}
|
||||
|
||||
/** Create sigar and verify it works. */
|
||||
|
|
|
|||
|
|
@ -26,6 +26,7 @@ import akka.dispatch.Dispatchers
|
|||
/**
|
||||
* Protobuf serializer for [[akka.cluster.metrics.ClusterMetricsMessage]] types.
|
||||
*/
|
||||
@ccompatUsedUntil213
|
||||
class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer {
|
||||
|
||||
private final val BufferSize = 4 * 1024
|
||||
|
|
|
|||
|
|
@ -147,7 +147,7 @@ abstract class ClusterMetricsDisabledSpec
|
|||
//clusterView.clusterMetrics.size should ===(0)
|
||||
metricsView.clusterMetrics.size should ===(0)
|
||||
ClusterMetricsExtension(system).subscribe(testActor)
|
||||
expectNoMsg
|
||||
expectNoMessage
|
||||
// TODO ensure same contract
|
||||
//clusterView.clusterMetrics.size should ===(0)
|
||||
metricsView.clusterMetrics.size should ===(0)
|
||||
|
|
|
|||
|
|
@ -6,6 +6,7 @@ package akka.cluster.metrics
|
|||
|
||||
import language.postfixOps
|
||||
import java.lang.management.ManagementFactory
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import com.typesafe.config.Config
|
||||
|
|
@ -22,6 +23,7 @@ import akka.routing.ActorRefRoutee
|
|||
import akka.routing.Routees
|
||||
import akka.cluster.routing.ClusterRouterPool
|
||||
import akka.cluster.routing.ClusterRouterPoolSettings
|
||||
import akka.util.unused
|
||||
|
||||
object AdaptiveLoadBalancingRouterConfig extends MultiNodeConfig {
|
||||
|
||||
|
|
@ -105,7 +107,7 @@ object AdaptiveLoadBalancingRouterConfig extends MultiNodeConfig {
|
|||
|
||||
}
|
||||
|
||||
class TestCustomMetricsSelector(config: Config) extends MetricsSelector {
|
||||
class TestCustomMetricsSelector(@unused config: Config) extends MetricsSelector {
|
||||
override def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] = Map.empty
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -11,7 +11,10 @@ import akka.testkit._
|
|||
import akka.cluster.metrics.StandardMetrics._
|
||||
import akka.cluster.Cluster
|
||||
|
||||
class MetricsExtensionSpec extends AkkaSpec(MetricsConfig.clusterSigarMock) with ImplicitSender with RedirectLogging {
|
||||
class ClusterMetricsExtensionSpec
|
||||
extends AkkaSpec(MetricsConfig.clusterSigarMock)
|
||||
with ImplicitSender
|
||||
with RedirectLogging {
|
||||
|
||||
val cluster = Cluster(system)
|
||||
|
||||
|
|
@ -49,7 +52,7 @@ class MetricsExtensionSpec extends AkkaSpec(MetricsConfig.clusterSigarMock) with
|
|||
extension.supervisor ! CollectionStopMessage
|
||||
awaitSample()
|
||||
metricsNodeCount should ===(nodeCount)
|
||||
metricsHistorySize should be >= (sampleCount)
|
||||
metricsHistorySize should be >= sampleCount
|
||||
}
|
||||
|
||||
"verify sigar mock data matches expected ewma data" in {
|
||||
|
|
@ -95,12 +98,12 @@ class MetricsExtensionSpec extends AkkaSpec(MetricsConfig.clusterSigarMock) with
|
|||
extension.supervisor ! CollectionStartMessage
|
||||
awaitSample()
|
||||
val size3 = metricsHistorySize
|
||||
size3 should be > (size2)
|
||||
size3 should be > size2
|
||||
|
||||
extension.supervisor ! CollectionStopMessage
|
||||
awaitSample()
|
||||
val size4 = metricsHistorySize
|
||||
size4 should be >= (size3)
|
||||
size4 should be >= size3
|
||||
|
||||
awaitSample()
|
||||
val size5 = metricsHistorySize
|
||||
|
|
@ -108,7 +111,7 @@ class MetricsExtensionSpec extends AkkaSpec(MetricsConfig.clusterSigarMock) with
|
|||
|
||||
}
|
||||
|
||||
(1 to 3).foreach { step =>
|
||||
(1 to 3).foreach { _ =>
|
||||
cycle()
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -8,6 +8,9 @@ import scala.concurrent.duration._
|
|||
import akka.testkit.{ AkkaSpec, LongRunningTest }
|
||||
import java.util.concurrent.ThreadLocalRandom
|
||||
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent
|
||||
class EWMASpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory {
|
||||
|
||||
val collector = createMetricsCollector
|
||||
|
|
|
|||
|
|
@ -7,12 +7,16 @@ package akka.cluster.metrics
|
|||
import org.scalatest.WordSpec
|
||||
import org.scalatest.Matchers
|
||||
import akka.cluster.metrics.StandardMetrics._
|
||||
|
||||
import scala.util.Failure
|
||||
import akka.actor.Address
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.testkit.ImplicitSender
|
||||
import java.lang.System.{ currentTimeMillis => newTimestamp }
|
||||
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent
|
||||
class MetricNumericConverterSpec extends WordSpec with Matchers with MetricNumericConverter {
|
||||
|
||||
"MetricNumericConverter" must {
|
||||
|
|
@ -51,6 +55,7 @@ class MetricNumericConverterSpec extends WordSpec with Matchers with MetricNumer
|
|||
}
|
||||
}
|
||||
|
||||
@silent
|
||||
class NodeMetricsSpec extends WordSpec with Matchers {
|
||||
|
||||
val node1 = Address("akka.tcp", "sys", "a", 2554)
|
||||
|
|
@ -149,8 +154,8 @@ class MetricsGossipSpec
|
|||
val m1 = NodeMetrics(Address("akka.tcp", "sys", "a", 2554), newTimestamp, collector.sample.metrics)
|
||||
val m2 = NodeMetrics(Address("akka.tcp", "sys", "a", 2555), newTimestamp, collector.sample.metrics)
|
||||
|
||||
m1.metrics.size should be > (3)
|
||||
m2.metrics.size should be > (3)
|
||||
m1.metrics.size should be > 3
|
||||
m2.metrics.size should be > 3
|
||||
|
||||
val g1 = MetricsGossip.empty :+ m1
|
||||
g1.nodes.size should ===(1)
|
||||
|
|
@ -168,7 +173,6 @@ class MetricsGossipSpec
|
|||
|
||||
val g1 = MetricsGossip.empty :+ m1 :+ m2
|
||||
g1.nodes.size should ===(2)
|
||||
val beforeMergeNodes = g1.nodes
|
||||
|
||||
val m2Updated = m2.copy(metrics = newSample(m2.metrics), timestamp = m2.timestamp + 1000)
|
||||
val g2 = g1 :+ m2Updated // merge peers
|
||||
|
|
@ -195,7 +199,7 @@ class MetricsGossipSpec
|
|||
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.nodes.foreach(_.metrics.size should be > 3)
|
||||
mergedGossip.nodeMetricsFor(m2.address).map(_.timestamp) should ===(Some(m2Updated.timestamp))
|
||||
}
|
||||
|
||||
|
|
@ -233,6 +237,7 @@ class MetricsGossipSpec
|
|||
}
|
||||
}
|
||||
|
||||
@silent
|
||||
class MetricValuesSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with MetricsCollectorFactory {
|
||||
import akka.cluster.metrics.StandardMetrics._
|
||||
|
||||
|
|
@ -256,37 +261,37 @@ class MetricValuesSpec extends AkkaSpec(MetricsConfig.defaultEnabled) with Metri
|
|||
"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)
|
||||
stream1 should be >= stream2
|
||||
}
|
||||
|
||||
"extract expected MetricValue types for load balancing" in {
|
||||
nodes.foreach { node =>
|
||||
node match {
|
||||
case HeapMemory(address, _, used, committed, _) =>
|
||||
used should be > (0L)
|
||||
committed should be >= (used)
|
||||
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)
|
||||
StandardMetrics.extractHeapMemory(node) should not be null
|
||||
}
|
||||
|
||||
node match {
|
||||
case Cpu(address, _, systemLoadAverageOption, cpuCombinedOption, cpuStolenOption, processors) =>
|
||||
processors should be > (0)
|
||||
case Cpu(_, _, systemLoadAverageOption, cpuCombinedOption, cpuStolenOption, processors) =>
|
||||
processors should be > 0
|
||||
if (systemLoadAverageOption.isDefined)
|
||||
systemLoadAverageOption.get should be >= (0.0)
|
||||
systemLoadAverageOption.get should be >= 0.0
|
||||
if (cpuCombinedOption.isDefined) {
|
||||
cpuCombinedOption.get should be <= (1.0)
|
||||
cpuCombinedOption.get should be >= (0.0)
|
||||
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)
|
||||
cpuStolenOption.get should be <= 1.0
|
||||
cpuStolenOption.get should be >= 0.0
|
||||
}
|
||||
// extract is the java api
|
||||
StandardMetrics.extractCpu(node) should not be (null)
|
||||
StandardMetrics.extractCpu(node) should not be null
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,13 +5,13 @@
|
|||
package akka.cluster.metrics
|
||||
|
||||
import scala.language.postfixOps
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.{ Try }
|
||||
|
||||
import scala.util.Try
|
||||
import akka.testkit._
|
||||
import akka.cluster.metrics.StandardMetrics._
|
||||
import com.github.ghik.silencer.silent
|
||||
|
||||
@silent
|
||||
class MetricsCollectorSpec
|
||||
extends AkkaSpec(MetricsConfig.defaultEnabled)
|
||||
with ImplicitSender
|
||||
|
|
@ -22,10 +22,10 @@ class MetricsCollectorSpec
|
|||
"Metric must" must {
|
||||
|
||||
"merge 2 metrics that are tracking the same metric" in {
|
||||
for (i <- 1 to 20) {
|
||||
for (_ <- 1 to 20) {
|
||||
val sample1 = collector.sample.metrics
|
||||
val sample2 = collector.sample.metrics
|
||||
val merged12 = sample2.flatMap(latest =>
|
||||
sample2.flatMap(latest =>
|
||||
sample1.collect {
|
||||
case peer if latest.sameAs(peer) =>
|
||||
val m = peer :+ latest
|
||||
|
|
@ -36,7 +36,7 @@ class MetricsCollectorSpec
|
|||
|
||||
val sample3 = collector.sample.metrics
|
||||
val sample4 = collector.sample.metrics
|
||||
val merged34 = sample4.flatMap(latest =>
|
||||
sample4.flatMap(latest =>
|
||||
sample3.collect {
|
||||
case peer if latest.sameAs(peer) =>
|
||||
val m = peer :+ latest
|
||||
|
|
@ -60,20 +60,22 @@ class MetricsCollectorSpec
|
|||
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 (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)
|
||||
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)
|
||||
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)
|
||||
b.doubleValue should be <= 1.0
|
||||
b.doubleValue should be >= 0.0
|
||||
case unexpected =>
|
||||
fail(s"Unexpected metric type $unexpected")
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -92,7 +94,7 @@ class MetricsCollectorSpec
|
|||
"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)
|
||||
sample.metrics.size should be >= 3
|
||||
Thread.sleep(100)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ import akka.actor.Deploy
|
|||
import akka.dispatch.UnboundedMessageQueueSemantics
|
||||
import akka.actor.PoisonPill
|
||||
import akka.actor.ActorLogging
|
||||
import org.scalatest.mock.MockitoSugar
|
||||
import org.scalatestplus.mockito.MockitoSugar
|
||||
import akka.actor.ActorSystem
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.remote.RARP
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import org.slf4j.{ MDC, Marker, MarkerFactory, Logger => SLFLogger, LoggerFactor
|
|||
import akka.event.Logging._
|
||||
import akka.actor._
|
||||
import akka.event.{ LogMarker, _ }
|
||||
import akka.util.Helpers
|
||||
import akka.util.{ unused, Helpers }
|
||||
import akka.dispatch.RequiresMessageQueue
|
||||
|
||||
/**
|
||||
|
|
@ -146,7 +146,8 @@ class Slf4jLogger extends Actor with SLF4JLogging with RequiresMessageQueue[Logg
|
|||
* backend configuration (e.g. logback.xml) to filter log events before publishing
|
||||
* the log events to the `eventStream`.
|
||||
*/
|
||||
class Slf4jLoggingFilter(settings: ActorSystem.Settings, eventStream: EventStream) extends LoggingFilterWithMarker {
|
||||
class Slf4jLoggingFilter(@unused settings: ActorSystem.Settings, eventStream: EventStream)
|
||||
extends LoggingFilterWithMarker {
|
||||
def isErrorEnabled(logClass: Class[_], logSource: String) =
|
||||
(eventStream.logLevel >= ErrorLevel) && Logger(logClass, logSource).isErrorEnabled
|
||||
def isWarningEnabled(logClass: Class[_], logSource: String) =
|
||||
|
|
|
|||
|
|
@ -107,7 +107,7 @@ class Slf4jLoggingFilterSpec extends AkkaSpec(Slf4jLoggingFilterSpec.config) wit
|
|||
val debugLevelProducer = system.actorOf(Props[WarningLevelProducer], name = "warningLevelProducer")
|
||||
debugLevelProducer ! "test2"
|
||||
probe.expectMsgType[Warning].message should be("test2")
|
||||
probe.expectNoMsg(500.millis)
|
||||
probe.expectNoMessage(500.millis)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -28,7 +28,9 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport {
|
|||
"akka-persistence",
|
||||
"akka-cluster-tools",
|
||||
"akka-cluster-sharding",
|
||||
"akka-stream")
|
||||
"akka-stream",
|
||||
"akka-cluster-metrics",
|
||||
"akka-slf4j")
|
||||
|
||||
val strictProjects = Set("akka-discovery", "akka-protobuf", "akka-coordination")
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue