aboutsummaryrefslogtreecommitdiff
path: root/kamon-newrelic
diff options
context:
space:
mode:
authorSlava Schmidt <slava.schmidt.extern@zalando.de>2015-07-08 12:22:27 +0200
committerSlava Schmidt <slava.schmidt.extern@zalando.de>2015-07-08 12:22:27 +0200
commitf9187cf42a2e6e0815eca4b916729d11b9dda467 (patch)
tree3989360cb0c88540bdb32502254247d1a52b7d0a /kamon-newrelic
parentda471e354ad1757a989ff40c06bafc1f7e332d17 (diff)
downloadKamon-f9187cf42a2e6e0815eca4b916729d11b9dda467.tar.gz
Kamon-f9187cf42a2e6e0815eca4b916729d11b9dda467.tar.bz2
Kamon-f9187cf42a2e6e0815eca4b916729d11b9dda467.zip
+ newrelic: add possibility to send akka metrics to the newrelic
Added possibility to send akka metrics to the newrelic as custom metrics. Externalized categories of newrelic subscription into the configuration file. This allow to define which metrics categories should be send to newrelic independent upon which metrics are actually collected. Akka metrics exported to the newrelic as custom metrics and available in custom dashboards in some format similar to: Cusom/akka-actor/{ActorSystemName[user|system|...]\{ActorName}/{MetricName} for actors and Custom/akka-thread-pool-executor\Some-Service\akka.io.pinned-dispatcher/ProcessedTasks for thread pools. Same metrics for multiple actors can be displayed as a single chart by using * (star) for part of the actor name. For example Cusom/akka-actor\MyActor\user\*DatabaseWorker/ProcessingTime will show processing time for all database workers. Example of actor metrics displayed by newrelic: http://s4.postimg.org/sfn9vjzgt/Screen_Shot_2015_04_22_at_11_24_15.png Example of pool metrics displayed by newrelic: http://s4.postimg.org/gchy7zoel/Screen_Shot_2015_04_22_at_11_24_24.png
Diffstat (limited to 'kamon-newrelic')
-rw-r--r--kamon-newrelic/src/main/resources/reference.conf10
-rw-r--r--kamon-newrelic/src/main/scala/kamon/newrelic/Agent.scala67
-rw-r--r--kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetricExtractor.scala30
-rw-r--r--kamon-newrelic/src/main/scala/kamon/newrelic/MetricReporter.scala35
-rw-r--r--kamon-newrelic/src/test/scala/kamon/newrelic/CustomMetricExtractorSpec.scala45
-rw-r--r--kamon-newrelic/src/test/scala/kamon/newrelic/MetricsSubscriptionSpec.scala54
6 files changed, 210 insertions, 31 deletions
diff --git a/kamon-newrelic/src/main/resources/reference.conf b/kamon-newrelic/src/main/resources/reference.conf
index de160105..a1c1c4d3 100644
--- a/kamon-newrelic/src/main/resources/reference.conf
+++ b/kamon-newrelic/src/main/resources/reference.conf
@@ -23,6 +23,16 @@ kamon {
# delay between connection attempts to NewRelic collector
connect-retry-delay = 30 seconds
+
+ custom-metric-subscriptions {
+ counter = [ "**" ]
+ histogram = [ "**" ]
+ min-max-counter = [ "**" ]
+ gauge = [ "**" ]
+ akka-actor = [ "**" ]
+ akka-dispatcher = [ "**" ]
+ akka-router = [ "**" ]
+ }
}
modules {
diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/Agent.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/Agent.scala
index 75f73ea4..3e43da36 100644
--- a/kamon-newrelic/src/main/scala/kamon/newrelic/Agent.scala
+++ b/kamon-newrelic/src/main/scala/kamon/newrelic/Agent.scala
@@ -16,10 +16,13 @@
package kamon.newrelic
-import akka.actor.{ ActorLogging, Actor }
+import akka.actor.{ ActorRef, ActorLogging, Actor }
+import akka.event.LoggingAdapter
import akka.io.IO
import akka.util.Timeout
import com.typesafe.config.Config
+import kamon.Kamon
+import kamon.metric.{ SegmentMetrics, TraceMetrics, MetricsModule, TickMetricSnapshotBuffer }
import spray.can.Http
import spray.json._
import scala.concurrent.Future
@@ -30,16 +33,28 @@ import kamon.util.ConfigTools.Syntax
import Agent._
import JsonProtocol._
import akka.pattern.pipe
-
+import scala.concurrent.duration._
import scala.concurrent.duration.FiniteDuration
-class Agent extends Actor with SprayJsonSupport with ActorLogging {
+class Agent extends Actor with SprayJsonSupport with ActorLogging with MetricsSubscription {
import context.dispatcher
- val agentSettings = AgentSettings.fromConfig(context.system.settings.config)
+ private val config = context.system.settings.config
+
+ val agentSettings = AgentSettings.fromConfig(config)
// Start the reporters
- context.actorOf(MetricReporter.props(agentSettings), "metric-reporter")
+ private val reporter = context.actorOf(MetricReporter.props(agentSettings), "metric-reporter")
+
+ val metricsSubscriber = {
+ val tickInterval = Kamon.metrics.settings.tickInterval
+
+ // Metrics are always sent to New Relic in 60 seconds intervals.
+ if (tickInterval == 60.seconds) reporter
+ else context.actorOf(TickMetricSnapshotBuffer.props(1 minute, reporter), "metric-buffer")
+ }
+
+ subscribeToMetrics(config, metricsSubscriber, Kamon.metrics)
// Start the connection to the New Relic collector.
self ! Connect
@@ -143,4 +158,46 @@ object AgentSettings {
newRelicConfig.getFiniteDuration("connect-retry-delay"),
newRelicConfig.getFiniteDuration("apdexT").toMillis / 1E3D)
}
+}
+
+trait MetricsSubscription {
+ import kamon.util.ConfigTools.Syntax
+ import scala.collection.JavaConverters._
+ import MetricsSubscription._
+
+ def log: LoggingAdapter
+
+ def subscriptions(config: Config) = config getConfig "kamon.newrelic" getConfig "custom-metric-subscriptions"
+
+ def subscriptionKeys(config: Config) = subscriptions(config).firstLevelKeys filterNot isTraceOrSegmentEntityName
+
+ def subscribeToMetrics(config: Config, metricsSubscriber: ActorRef, extension: MetricsModule): Unit = {
+ subscribeToCustomMetrics(config, metricsSubscriber, extension)
+ subscribeToTransactionMetrics(metricsSubscriber, extension)
+ }
+
+ def subscribeToCustomMetrics(config: Config, metricsSubscriber: ActorRef, extension: MetricsModule): Unit =
+ subscriptionKeys(config) foreach { subscriptionCategory ⇒
+ subscriptions(config).getStringList(subscriptionCategory).asScala foreach { pattern ⇒
+ log.debug("Subscribing NewRelic reporting for custom metric '{}' : {}", subscriptionCategory, pattern)
+ extension.subscribe(subscriptionCategory, pattern, metricsSubscriber)
+ }
+ }
+
+ def subscribeToTransactionMetrics(metricsSubscriber: ActorRef, extension: MetricsModule): Unit =
+ traceAndSegmentMetrics foreach { subscriptionCategory ⇒
+ log.debug("Subscribing NewRelic reporting for transaction metric '{}' : {}", subscriptionCategory, defaultPattern)
+ extension.subscribe(subscriptionCategory, defaultPattern, metricsSubscriber)
+ }
+
+}
+
+object MetricsSubscription {
+
+ private val defaultPattern = "**"
+
+ private val traceAndSegmentMetrics = Seq(TraceMetrics.category, SegmentMetrics.category)
+
+ def isTraceOrSegmentEntityName(name: String): Boolean = traceAndSegmentMetrics contains name
+
} \ No newline at end of file
diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetricExtractor.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetricExtractor.scala
index 6919a967..012ae1dc 100644
--- a/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetricExtractor.scala
+++ b/kamon-newrelic/src/main/scala/kamon/newrelic/CustomMetricExtractor.scala
@@ -16,22 +16,34 @@
package kamon.newrelic
-import kamon.metric.{ EntitySnapshot, Entity }
+import kamon.metric.{ MetricKey, EntitySnapshot, Entity }
import kamon.metric.instrument.CollectionContext
object CustomMetricExtractor extends MetricExtractor {
def extract(settings: AgentSettings, collectionContext: CollectionContext, metrics: Map[Entity, EntitySnapshot]): Map[MetricID, MetricData] = {
- def onlySimpleMetrics(kv: (Entity, EntitySnapshot)): Boolean =
- kamon.metric.SingleInstrumentEntityRecorder.AllCategories.contains(kv._1.category)
+ val (simple, complex) = metrics filter customMetric partition simpleMetrics
+ simple.flatMap(toNewRelicMetric(simpleName)) ++ complex.flatMap(toNewRelicMetric(complexName))
+ }
+
+ def simpleName(entity: Entity, metricKey: MetricKey) = s"Custom/${entity.category}/${normalize(entity.name)}"
+
+ def complexName(entity: Entity, metricKey: MetricKey) = s"${simpleName(entity, metricKey)}/${metricKey.name}"
- def toNewRelicMetric(kv: (Entity, EntitySnapshot)): (MetricID, MetricData) = {
- val (entity, entitySnapshot) = kv
- val (metricKey, instrumentSnapshot) = entitySnapshot.metrics.head
+ def normalize(name: String) = name.replace('/', '#').replaceAll("""[\]\[\|\*]""", "_")
- Metric(instrumentSnapshot, metricKey.unitOfMeasurement, s"Custom/${entity.name}", None)
- }
+ def customMetric(kv: (Entity, EntitySnapshot)): Boolean =
+ !MetricsSubscription.isTraceOrSegmentEntityName(kv._1.category)
- metrics.filter(onlySimpleMetrics).map(toNewRelicMetric)
+ def simpleMetrics(kv: (Entity, EntitySnapshot)): Boolean =
+ kamon.metric.SingleInstrumentEntityRecorder.AllCategories.contains(kv._1.category)
+
+ def toNewRelicMetric(name: (Entity, MetricKey) ⇒ String)(kv: (Entity, EntitySnapshot)) = {
+ val (entity, entitySnapshot) = kv
+ for {
+ (metricKey, instrumentSnapshot) ← entitySnapshot.metrics
+ nameStr = name(entity, metricKey)
+ } yield Metric(instrumentSnapshot, metricKey.unitOfMeasurement, nameStr, None)
}
+
}
diff --git a/kamon-newrelic/src/main/scala/kamon/newrelic/MetricReporter.scala b/kamon-newrelic/src/main/scala/kamon/newrelic/MetricReporter.scala
index cf252445..806ddb6f 100644
--- a/kamon-newrelic/src/main/scala/kamon/newrelic/MetricReporter.scala
+++ b/kamon-newrelic/src/main/scala/kamon/newrelic/MetricReporter.scala
@@ -1,8 +1,24 @@
+/*
+ * =========================================================================================
+ * Copyright © 2013-2014 the kamon project <http://kamon.io/>
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file
+ * except in compliance with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
+ * either express or implied. See the License for the specific language governing permissions
+ * and limitations under the License.
+ * =========================================================================================
+ */
+
package kamon.newrelic
-import akka.actor.{ Props, ActorLogging, Actor }
-import akka.pattern.pipe
+import akka.actor._
import akka.io.IO
+import akka.pattern.pipe
import kamon.Kamon
import kamon.metric.SubscriptionsDispatcher.TickMetricSnapshot
import kamon.metric._
@@ -11,7 +27,6 @@ import kamon.newrelic.ApiMethodClient.{ AgentShutdownRequiredException, AgentRes
import kamon.newrelic.MetricReporter.{ PostFailed, PostSucceeded }
import spray.can.Http
import spray.httpx.SprayJsonSupport
-import scala.concurrent.duration._
import JsonProtocol._
class MetricReporter(settings: AgentSettings) extends Actor with ActorLogging with SprayJsonSupport {
@@ -19,15 +34,6 @@ class MetricReporter(settings: AgentSettings) extends Actor with ActorLogging wi
val metricsExtension = Kamon.metrics
val collectionContext = metricsExtension.buildDefaultCollectionContext
- val metricsSubscriber = {
- val tickInterval = Kamon.metrics.settings.tickInterval
-
- // Metrics are always sent to New Relic in 60 seconds intervals.
- if (tickInterval == 60.seconds) self
- else context.actorOf(TickMetricSnapshotBuffer.props(1 minute, self), "metric-buffer")
- }
-
- subscribeToMetrics()
def receive = awaitingConfiguration(None)
@@ -88,11 +94,6 @@ class MetricReporter(settings: AgentSettings) extends Actor with ActorLogging wi
TimeSliceMetrics(tick.from.toTimestamp, tick.to.toTimestamp, extractedMetrics)
}
- def subscribeToMetrics(): Unit = {
- ("trace" :: "trace-segment" :: "counter" :: "histogram" :: "min-max-counter" :: "gauge" :: Nil).foreach { category ⇒
- metricsExtension.subscribe(category, "**", metricsSubscriber, permanently = true)
- }
- }
}
object MetricReporter {
diff --git a/kamon-newrelic/src/test/scala/kamon/newrelic/CustomMetricExtractorSpec.scala b/kamon-newrelic/src/test/scala/kamon/newrelic/CustomMetricExtractorSpec.scala
new file mode 100644
index 00000000..5ab21655
--- /dev/null
+++ b/kamon-newrelic/src/test/scala/kamon/newrelic/CustomMetricExtractorSpec.scala
@@ -0,0 +1,45 @@
+/*
+ * =========================================================================================
+ * Copyright © 2013-2014 the kamon project <http://kamon.io/>
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file
+ * except in compliance with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
+ * either express or implied. See the License for the specific language governing permissions
+ * and limitations under the License.
+ * =========================================================================================
+ */
+
+package kamon.newrelic
+
+import org.scalatest.{ Matchers, WordSpecLike }
+
+/**
+ * @since 21.04.2015
+ */
+class CustomMetricExtractorSpec extends WordSpecLike with Matchers {
+
+ val cme = CustomMetricExtractor
+
+ "the CustomMetricExtractor" should {
+ "have a normalize method" which {
+ "is ok with an empty string" in {
+ cme.normalize("") should be("")
+ }
+ "is ok with normal '/'" in {
+ cme.normalize("akka/dispatcher/string") should be("akka#dispatcher#string")
+ }
+ "is ok with multiple '//'" in {
+ cme.normalize("akka///dispatcher//string") should be("akka###dispatcher##string")
+ }
+ "is ok with other special symbols" in {
+ cme.normalize("][|*akka*dispatcher|string[") should be("____akka_dispatcher_string_")
+ }
+ }
+ }
+}
+
diff --git a/kamon-newrelic/src/test/scala/kamon/newrelic/MetricsSubscriptionSpec.scala b/kamon-newrelic/src/test/scala/kamon/newrelic/MetricsSubscriptionSpec.scala
new file mode 100644
index 00000000..13e52b19
--- /dev/null
+++ b/kamon-newrelic/src/test/scala/kamon/newrelic/MetricsSubscriptionSpec.scala
@@ -0,0 +1,54 @@
+/*
+ * =========================================================================================
+ * Copyright © 2013-2014 the kamon project <http://kamon.io/>
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file
+ * except in compliance with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the
+ * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
+ * either express or implied. See the License for the specific language governing permissions
+ * and limitations under the License.
+ * =========================================================================================
+ */
+
+package kamon.newrelic
+
+import akka.event.NoLogging
+import com.typesafe.config.ConfigFactory
+import org.scalatest._
+
+import scala.collection.JavaConversions._
+
+/**
+ * @since 21.04.2015
+ */
+class MetricsSubscriptionSpec extends WordSpecLike with Matchers {
+
+ val instance = new MetricsSubscription {
+ override def log = NoLogging
+ }
+
+ val metrics = Seq("user-metrics", "trace", "akka-dispatcher", "akka-actor").zipWithIndex
+ val metricsStr = metrics map { m ⇒ m._1 + " = \"" + "*" * (m._2 + 1) + "\"" } mkString "\n"
+ val fullConfig = ConfigFactory.parseString(s"kamon.newrelic.custom-metric-subscriptions { $metricsStr }")
+
+ "the MetricsSubscription" should {
+
+ "read correct subscriptions from full configuration" in {
+ val cfg = instance.subscriptions(fullConfig)
+ cfg.entrySet().size should be(4)
+ cfg.entrySet().foreach { metric ⇒
+ val idx = metrics.indexWhere(_._1 == metric.getKey)
+ metric.getValue.unwrapped().toString should be("*" * (idx + 1))
+ }
+ }
+ "filter correct subscriptions" in {
+ val keys = instance.subscriptionKeys(fullConfig)
+ keys.size should be(3)
+ keys.contains("trace") shouldBe false
+ }
+ }
+}