From b6ea0a93e6be8e1f355f1bc993618d178d0c9372 Mon Sep 17 00:00:00 2001 From: Ivan Topolnak Date: Mon, 27 Jan 2014 19:09:40 -0300 Subject: first implementation of a universal metrics repository for any kind of metrics we might be interested in --- kamon-core/src/main/resources/reference.conf | 19 +++- .../ActorMessagePassingTracing.scala | 22 ++-- .../main/scala/kamon/metrics/ActorMetrics.scala | 75 +++++++++---- .../main/scala/kamon/metrics/ActorMetricsOps.scala | 20 ++-- .../metrics/AtomicSnapshotableHistogram.scala | 75 ------------- .../kamon/metrics/HighDynamicRangeRecorder.scala | 86 ++++++++++++++ .../scala/kamon/metrics/MetricsExtension.scala | 124 +++++++++++++++++++++ .../src/main/scala/kamon/metrics/package.scala | 31 ------ .../scala/kamon/metrics/ActorMetricsSpec.scala | 2 +- .../spray/can/server/ServerRequestTracing.scala | 2 +- 10 files changed, 303 insertions(+), 153 deletions(-) delete mode 100644 kamon-core/src/main/scala/kamon/metrics/AtomicSnapshotableHistogram.scala create mode 100644 kamon-core/src/main/scala/kamon/metrics/HighDynamicRangeRecorder.scala create mode 100644 kamon-core/src/main/scala/kamon/metrics/MetricsExtension.scala delete mode 100644 kamon-core/src/main/scala/kamon/metrics/package.scala diff --git a/kamon-core/src/main/resources/reference.conf b/kamon-core/src/main/resources/reference.conf index 11e7cbb4..f05c11f6 100644 --- a/kamon-core/src/main/resources/reference.conf +++ b/kamon-core/src/main/resources/reference.conf @@ -2,11 +2,24 @@ kamon { metrics { tick-interval = 1 second - actors { - tracked = [] + filters = [ + { + actor { + includes = [] + excludes = [ "system/*", "user/IO-*" ] + } + }, + { + dispatcher { + includes = [ "default-dispatcher" ] + excludes = [] + } + } + ] + - excluded = [ "system/*", "user/IO-*" ] + actors { hdr-settings { processing-time { highest-trackable-value = 3600000000000 diff --git a/kamon-core/src/main/scala/akka/instrumentation/ActorMessagePassingTracing.scala b/kamon-core/src/main/scala/akka/instrumentation/ActorMessagePassingTracing.scala index 6cede344..68d606ba 100644 --- a/kamon-core/src/main/scala/akka/instrumentation/ActorMessagePassingTracing.scala +++ b/kamon-core/src/main/scala/akka/instrumentation/ActorMessagePassingTracing.scala @@ -20,26 +20,24 @@ import org.aspectj.lang.ProceedingJoinPoint import akka.actor.{ Cell, Props, ActorSystem, ActorRef } import akka.dispatch.{ Envelope, MessageDispatcher } import kamon.trace.{ TraceContext, ContextAware, Trace } -import kamon.metrics.{ HdrActorMetricsRecorder, ActorMetrics } +import kamon.metrics.{ ActorMetrics, HdrActorMetricsRecorder, Metrics } import kamon.Kamon +import kamon.metrics.ActorMetrics.ActorMetricRecorder @Aspect("perthis(actorCellCreation(*, *, *, *, *))") class BehaviourInvokeTracing { - var path: Option[String] = None - var actorMetrics: Option[HdrActorMetricsRecorder] = None + var path: String = _ + var actorMetrics: Option[ActorMetricRecorder] = None @Pointcut("execution(akka.actor.ActorCell.new(..)) && args(system, ref, props, dispatcher, parent)") def actorCellCreation(system: ActorSystem, ref: ActorRef, props: Props, dispatcher: MessageDispatcher, parent: ActorRef): Unit = {} @After("actorCellCreation(system, ref, props, dispatcher, parent)") def afterCreation(system: ActorSystem, ref: ActorRef, props: Props, dispatcher: MessageDispatcher, parent: ActorRef): Unit = { - val metricsExtension = Kamon(ActorMetrics)(system) - val simplePathString = ref.path.elements.mkString("/") + val metricsExtension = Kamon(Metrics)(system) - if (metricsExtension.shouldTrackActor(simplePathString)) { - path = Some(ref.path.toString) - actorMetrics = Some(metricsExtension.registerActor(simplePathString)) - } + path = ref.path.elements.mkString("/") + actorMetrics = metricsExtension.register(path, ActorMetrics) } @Pointcut("(execution(* akka.actor.ActorCell.invoke(*)) || execution(* akka.routing.RoutedActorCell.sendMessage(*))) && args(envelope)") @@ -55,8 +53,8 @@ class BehaviourInvokeTracing { } actorMetrics.map { am ⇒ - am.recordProcessingTime(System.nanoTime() - timestampBeforeProcessing) - am.recordTimeInMailbox(timestampBeforeProcessing - contextAndTimestamp.timestamp) + am.processingTime.record(System.nanoTime() - timestampBeforeProcessing) + am.timeInMailbox.record(timestampBeforeProcessing - contextAndTimestamp.timestamp) } } @@ -65,7 +63,7 @@ class BehaviourInvokeTracing { @After("actorStop(cell)") def afterStop(cell: Cell): Unit = { - path.map(p ⇒ Kamon(ActorMetrics)(cell.system).unregisterActor(p)) + actorMetrics.map(p ⇒ Kamon(Metrics)(cell.system).unregister(path, ActorMetrics)) } } diff --git a/kamon-core/src/main/scala/kamon/metrics/ActorMetrics.scala b/kamon-core/src/main/scala/kamon/metrics/ActorMetrics.scala index 72e473e8..96d2cd48 100644 --- a/kamon-core/src/main/scala/kamon/metrics/ActorMetrics.scala +++ b/kamon-core/src/main/scala/kamon/metrics/ActorMetrics.scala @@ -1,31 +1,66 @@ -/* =================================================== +/* + * ========================================================================================= * Copyright © 2013 the kamon project * - * 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 + * 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 + * 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. - * ========================================================== */ + * 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.metrics -import akka.actor.{ Props, ExtendedActorSystem, ExtensionIdProvider, ExtensionId } -import akka.actor -import kamon.Kamon +import com.typesafe.config.Config +import org.HdrHistogram.HighDynamicRangeRecorder -object ActorMetrics extends ExtensionId[ActorMetricsExtension] with ExtensionIdProvider { - def lookup(): ExtensionId[_ <: actor.Extension] = ActorMetrics - def createExtension(system: ExtendedActorSystem): ActorMetricsExtension = new ActorMetricsExtension(system) +object ActorMetrics extends MetricGroupIdentity.Category with MetricGroupFactory { + val name: String = "actor" + type Group = ActorMetricRecorder -} + case object ProcessingTime extends MetricIdentity { val name = "ProcessingTime" } + case object MailboxSize extends MetricIdentity { val name = "MailboxSize" } + case object TimeInMailbox extends MetricIdentity { val name = "TimeInMailbox" } + + case class ActorMetricRecorder(processingTime: MetricRecorder, mailboxSize: MetricRecorder, timeInMailbox: MetricRecorder) + extends MetricGroupRecorder { + + def record(identity: MetricIdentity, value: Long): Unit = identity match { + case ProcessingTime ⇒ processingTime.record(value) + case MailboxSize ⇒ mailboxSize.record(value) + case TimeInMailbox ⇒ timeInMailbox.record(value) + } + + def collect: MetricGroupSnapshot = { + ActorMetricSnapshot(processingTime.collect(), mailboxSize.collect(), timeInMailbox.collect()) + } + } + + case class ActorMetricSnapshot(processingTime: MetricSnapshot, mailboxSize: MetricSnapshot, timeInMailbox: MetricSnapshot) + extends MetricGroupSnapshot { + + def metrics: Map[MetricIdentity, MetricSnapshot] = Map( + (ProcessingTime -> processingTime), + (MailboxSize -> mailboxSize), + (TimeInMailbox -> timeInMailbox)) + } + + def create(config: Config): ActorMetricRecorder = { + import HighDynamicRangeRecorder.Configuration + + val settings = config.getConfig("kamon.metrics.actors.hdr-settings") + val processingTimeHdrConfig = Configuration.fromConfig(settings.getConfig("processing-time")) + val mailboxSizeHdrConfig = Configuration.fromConfig(settings.getConfig("mailbox-size")) + val timeInMailboxHdrConfig = Configuration.fromConfig(settings.getConfig("time-in-mailbox")) -class ActorMetricsExtension(val system: ExtendedActorSystem) extends Kamon.Extension with ActorMetricsOps { - lazy val metricsDispatcher = system.actorOf(Props[ActorMetricsDispatcher], "kamon-actor-metrics") + ActorMetricRecorder( + HighDynamicRangeRecorder(processingTimeHdrConfig), + HighDynamicRangeRecorder(mailboxSizeHdrConfig), + HighDynamicRangeRecorder(timeInMailboxHdrConfig)) + } } diff --git a/kamon-core/src/main/scala/kamon/metrics/ActorMetricsOps.scala b/kamon-core/src/main/scala/kamon/metrics/ActorMetricsOps.scala index 4a6c98f4..0e3af5fd 100644 --- a/kamon-core/src/main/scala/kamon/metrics/ActorMetricsOps.scala +++ b/kamon-core/src/main/scala/kamon/metrics/ActorMetricsOps.scala @@ -16,7 +16,7 @@ package kamon.metrics -import org.HdrHistogram.{ AbstractHistogram, AtomicHistogram } +import org.HdrHistogram.{ HighDynamicRangeRecorder, AbstractHistogram, AtomicHistogram } import kamon.util.GlobPathFilter import scala.collection.concurrent.TrieMap import scala.collection.JavaConversions.iterableAsScalaIterable @@ -28,7 +28,7 @@ import java.util.concurrent.TimeUnit import kamon.metrics.ActorMetricsDispatcher.Subscribe trait ActorMetricsOps { - self: ActorMetricsExtension ⇒ + self: MetricsExtension ⇒ val config = system.settings.config.getConfig("kamon.metrics.actors") val actorMetrics = TrieMap[String, HdrActorMetricsRecorder]() @@ -38,9 +38,9 @@ trait ActorMetricsOps { val actorMetricsFactory: () ⇒ HdrActorMetricsRecorder = { val settings = config.getConfig("hdr-settings") - val processingTimeHdrConfig = HdrConfiguration.fromConfig(settings.getConfig("processing-time")) - val timeInMailboxHdrConfig = HdrConfiguration.fromConfig(settings.getConfig("time-in-mailbox")) - val mailboxSizeHdrConfig = HdrConfiguration.fromConfig(settings.getConfig("mailbox-size")) + val processingTimeHdrConfig = HighDynamicRangeRecorder.Configuration.fromConfig(settings.getConfig("processing-time")) + val timeInMailboxHdrConfig = HighDynamicRangeRecorder.Configuration.fromConfig(settings.getConfig("time-in-mailbox")) + val mailboxSizeHdrConfig = HighDynamicRangeRecorder.Configuration.fromConfig(settings.getConfig("mailbox-size")) () ⇒ new HdrActorMetricsRecorder(processingTimeHdrConfig, timeInMailboxHdrConfig, mailboxSizeHdrConfig) } @@ -53,8 +53,8 @@ trait ActorMetricsOps { def unregisterActor(path: String): Unit = actorMetrics.remove(path) } -class HdrActorMetricsRecorder(processingTimeHdrConfig: HdrConfiguration, timeInMailboxHdrConfig: HdrConfiguration, - mailboxSizeHdrConfig: HdrConfiguration) { +class HdrActorMetricsRecorder(processingTimeHdrConfig: HighDynamicRangeRecorder.Configuration, timeInMailboxHdrConfig: HighDynamicRangeRecorder.Configuration, + mailboxSizeHdrConfig: HighDynamicRangeRecorder.Configuration) { val processingTimeHistogram = new AtomicHistogram(processingTimeHdrConfig.highestTrackableValue, processingTimeHdrConfig.significantValueDigits) val timeInMailboxHistogram = new AtomicHistogram(timeInMailboxHdrConfig.highestTrackableValue, timeInMailboxHdrConfig.significantValueDigits) @@ -103,8 +103,8 @@ class ActorMetricsDispatcher extends Actor { } def flushMetrics(): Unit = { - val currentTick = System.currentTimeMillis() - val snapshots = Kamon(ActorMetrics)(context.system).actorMetrics.map { + /* val currentTick = System.currentTimeMillis() + val snapshots = Kamon(Metrics)(context.system).actorMetrics.map { case (path, metrics) ⇒ val snapshot = metrics.snapshot() metrics.reset() @@ -116,7 +116,7 @@ class ActorMetricsDispatcher extends Actor { dispatchMetricsTo(subscribedForever, snapshots, currentTick) subscribedForOne = Map.empty - lastTick = currentTick + lastTick = currentTick*/ } def dispatchMetricsTo(subscribers: Map[GlobPathFilter, List[ActorRef]], snapshots: Map[String, HdrActorMetricsSnapshot], diff --git a/kamon-core/src/main/scala/kamon/metrics/AtomicSnapshotableHistogram.scala b/kamon-core/src/main/scala/kamon/metrics/AtomicSnapshotableHistogram.scala deleted file mode 100644 index c9e47792..00000000 --- a/kamon-core/src/main/scala/kamon/metrics/AtomicSnapshotableHistogram.scala +++ /dev/null @@ -1,75 +0,0 @@ -/* - * ========================================================================================= - * Copyright © 2013 the kamon project - * - * 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 org.HdrHistogram - -import java.util.concurrent.atomic.AtomicLongFieldUpdater -import scala.annotation.tailrec -import org.HdrHistogram.AtomicSnapshotableHistogram.{ Value, Snapshot } - -/** - * This implementation aims to be used for real time data collection where data snapshots are taken often over time. - * The snapshotAndReset() operation extracts all the recorded values from the histogram and resets the counts, but still - * leave it in a consistent state even in the case of concurrent modification while the snapshot is being taken. - */ -class AtomicSnapshotableHistogram(highestTrackableValue: Long, numberOfSignificantValueDigits: Int) - extends AtomicHistogram(1L, highestTrackableValue, numberOfSignificantValueDigits) { - - import AtomicSnapshotableHistogram.totalCountUpdater - - def snapshotAndReset(): Snapshot = { - val entries = Vector.newBuilder[Value] - val countsLength = counts.length() - - @tailrec def iterate(index: Int, previousValue: Long, nrOfRecordings: Long, bucketLimit: Long, increment: Long): Long = { - if (index < countsLength) { - val currentValue = previousValue + increment - val countAtValue = counts.getAndSet(index, 0) - - if (countAtValue > 0) - entries += Value(currentValue, countAtValue) - - if (currentValue == bucketLimit) - iterate(index + 1, currentValue, nrOfRecordings + countAtValue, (bucketLimit << 1) + 1, increment << 1) - else - iterate(index + 1, currentValue, nrOfRecordings + countAtValue, bucketLimit, increment) - } else { - nrOfRecordings - } - } - - val nrOfRecordings = iterate(0, -1, 0, subBucketMask, 1) - - def tryUpdateTotalCount: Boolean = { - val previousTotalCount = getTotalCount - val newTotalCount = previousTotalCount - nrOfRecordings - - totalCountUpdater.compareAndSet(this, previousTotalCount, newTotalCount) - } - - while (!tryUpdateTotalCount) {} - - Snapshot(nrOfRecordings, entries.result()) - } - -} - -object AtomicSnapshotableHistogram { - val totalCountUpdater = AtomicLongFieldUpdater.newUpdater(classOf[AtomicHistogram], "totalCount") - - case class Snapshot(nrOfRecordings: Long, values: Vector[Value]) - case class Value(magnitude: Long, count: Long) -} diff --git a/kamon-core/src/main/scala/kamon/metrics/HighDynamicRangeRecorder.scala b/kamon-core/src/main/scala/kamon/metrics/HighDynamicRangeRecorder.scala new file mode 100644 index 00000000..e31d0e11 --- /dev/null +++ b/kamon-core/src/main/scala/kamon/metrics/HighDynamicRangeRecorder.scala @@ -0,0 +1,86 @@ +/* + * ========================================================================================= + * Copyright © 2013 the kamon project + * + * 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 org.HdrHistogram + +import java.util.concurrent.atomic.AtomicLongFieldUpdater +import scala.annotation.tailrec +import kamon.metrics.{ DefaultMetricSnapshot, MetricSnapshot, MetricRecorder } +import com.typesafe.config.Config +import org.HdrHistogram.HighDynamicRangeRecorder.Configuration + +/** + * This implementation aims to be used for real time data collection where data snapshots are taken often over time. + * The snapshotAndReset() operation extracts all the recorded values from the histogram and resets the counts, but still + * leave it in a consistent state even in the case of concurrent modification while the snapshot is being taken. + */ +class HighDynamicRangeRecorder(configuration: Configuration) + extends AtomicHistogram(1L, configuration.highestTrackableValue, configuration.significantValueDigits) with MetricRecorder { + + import HighDynamicRangeRecorder.totalCountUpdater + + def record(value: Long): Unit = recordValue(value) + + def collect(): MetricSnapshot = { + val entries = Vector.newBuilder[MetricSnapshot.Measurement] + val countsLength = counts.length() + + @tailrec def iterate(index: Int, previousValue: Long, nrOfRecordings: Long, bucketLimit: Long, increment: Long): Long = { + if (index < countsLength) { + val currentValue = previousValue + increment + val countAtValue = counts.getAndSet(index, 0) + + if (countAtValue > 0) + entries += MetricSnapshot.Measurement(currentValue, countAtValue) + + if (currentValue == bucketLimit) + iterate(index + 1, currentValue, nrOfRecordings + countAtValue, (bucketLimit << 1) + 1, increment << 1) + else + iterate(index + 1, currentValue, nrOfRecordings + countAtValue, bucketLimit, increment) + } else { + nrOfRecordings + } + } + + val nrOfRecordings = iterate(0, -1, 0, subBucketMask, 1) + + def tryUpdateTotalCount: Boolean = { + val previousTotalCount = getTotalCount + val newTotalCount = previousTotalCount - nrOfRecordings + + totalCountUpdater.compareAndSet(this, previousTotalCount, newTotalCount) + } + + while (!tryUpdateTotalCount) {} + + DefaultMetricSnapshot(nrOfRecordings, entries.result()) + } + +} + +object HighDynamicRangeRecorder { + val totalCountUpdater = AtomicLongFieldUpdater.newUpdater(classOf[AtomicHistogram], "totalCount") + + def apply(configuration: Configuration): HighDynamicRangeRecorder = new HighDynamicRangeRecorder(configuration) + + case class Configuration(highestTrackableValue: Long, significantValueDigits: Int) + + case object Configuration { + def fromConfig(config: Config): Configuration = { + Configuration(config.getLong("highest-trackable-value"), config.getInt("significant-value-digits")) + } + } +} diff --git a/kamon-core/src/main/scala/kamon/metrics/MetricsExtension.scala b/kamon-core/src/main/scala/kamon/metrics/MetricsExtension.scala new file mode 100644 index 00000000..4bac3519 --- /dev/null +++ b/kamon-core/src/main/scala/kamon/metrics/MetricsExtension.scala @@ -0,0 +1,124 @@ +/* + * ========================================================================================= + * Copyright © 2013 the kamon project + * + * 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.metrics + +import scala.collection.concurrent.TrieMap +import akka.actor.{ ExtensionIdProvider, ExtensionId, ExtendedActorSystem } +import com.typesafe.config.Config +import kamon.util.GlobPathFilter +import kamon.Kamon +import akka.actor +import kamon.metrics.Metrics.MetricGroupFilter + +case class MetricGroupIdentity(name: String, category: MetricGroupIdentity.Category) + +trait MetricIdentity { + def name: String +} + +trait MetricGroupRecorder { + def record(identity: MetricIdentity, value: Long) + def collect: MetricGroupSnapshot +} + +trait MetricGroupSnapshot { + def metrics: Map[MetricIdentity, MetricSnapshot] +} + +trait MetricRecorder { + def record(value: Long) + def collect(): MetricSnapshot +} + +trait MetricSnapshot { + def numberOfMeasurements: Long + def measurementLevels: Vector[MetricSnapshot.Measurement] +} + +object MetricSnapshot { + case class Measurement(value: Long, count: Long) +} + +case class DefaultMetricSnapshot(numberOfMeasurements: Long, measurementLevels: Vector[MetricSnapshot.Measurement]) extends MetricSnapshot + +object MetricGroupIdentity { + trait Category { + def name: String + } +} + +trait MetricGroupFactory { + type Group <: MetricGroupRecorder + def create(config: Config): Group +} + +class MetricsExtension(val system: ExtendedActorSystem) extends Kamon.Extension { + val config = system.settings.config + val storage = TrieMap[MetricGroupIdentity, MetricGroupRecorder]() + val filters = loadFilters(config) + + def register(name: String, category: MetricGroupIdentity.Category with MetricGroupFactory): Option[category.Group] = { + if (shouldTrack(name, category)) + Some(storage.getOrElseUpdate(MetricGroupIdentity(name, category), category.create(config)).asInstanceOf[category.Group]) + else + None + } + + def unregister(name: String, category: MetricGroupIdentity.Category with MetricGroupFactory): Unit = { + storage.remove(MetricGroupIdentity(name, category)) + } + + def collect: Map[MetricGroupIdentity, MetricGroupSnapshot] = { + (for ((identity, recorder) ← storage) yield (identity, recorder.collect)).toMap + } + + private def shouldTrack(name: String, category: MetricGroupIdentity.Category): Boolean = { + filters.get(category.name).map(filter ⇒ filter.accept(name)).getOrElse(false) + } + + def loadFilters(config: Config): Map[String, MetricGroupFilter] = { + import scala.collection.JavaConverters._ + + val filters = config.getObjectList("kamon.metrics.filters").asScala + + val allFilters = + for ( + filter ← filters; + entry ← filter.entrySet().asScala + ) yield { + val key = entry.getKey + val keyBasedConfig = entry.getValue.atKey(key) + + val includes = keyBasedConfig.getStringList(s"$key.includes").asScala.map(inc ⇒ new GlobPathFilter(inc)).toList + val excludes = keyBasedConfig.getStringList(s"$key.excludes").asScala.map(exc ⇒ new GlobPathFilter(exc)).toList + + (key, MetricGroupFilter(includes, excludes)) + } + + allFilters.toMap + } + +} + +object Metrics extends ExtensionId[MetricsExtension] with ExtensionIdProvider { + def lookup(): ExtensionId[_ <: actor.Extension] = Metrics + def createExtension(system: ExtendedActorSystem): MetricsExtension = new MetricsExtension(system) + + case class MetricGroupFilter(includes: List[GlobPathFilter], excludes: List[GlobPathFilter]) { + def accept(name: String): Boolean = includes.exists(_.accept(name)) && !excludes.exists(_.accept(name)) + } +} diff --git a/kamon-core/src/main/scala/kamon/metrics/package.scala b/kamon-core/src/main/scala/kamon/metrics/package.scala deleted file mode 100644 index d6359ead..00000000 --- a/kamon-core/src/main/scala/kamon/metrics/package.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * ========================================================================================= - * Copyright © 2013 the kamon project - * - * 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 - -import scala.concurrent.duration._ -import com.typesafe.config.Config - -package object metrics { - val OneHour = 1.hour.toNanos - - case class HdrConfiguration(highestTrackableValue: Long, significantValueDigits: Int) - case object HdrConfiguration { - def fromConfig(config: Config): HdrConfiguration = { - HdrConfiguration(config.getLong("highest-trackable-value"), config.getInt("significant-value-digits")) - } - } -} diff --git a/kamon-core/src/test/scala/kamon/metrics/ActorMetricsSpec.scala b/kamon-core/src/test/scala/kamon/metrics/ActorMetricsSpec.scala index ff08ca0f..127de4c4 100644 --- a/kamon-core/src/test/scala/kamon/metrics/ActorMetricsSpec.scala +++ b/kamon-core/src/test/scala/kamon/metrics/ActorMetricsSpec.scala @@ -32,7 +32,7 @@ class ActorMetricsSpec extends TestKitBase with WordSpecLike with Matchers { implicit def self = testActor - lazy val metricsExtension = Kamon(ActorMetrics).metricsDispatcher + lazy val metricsExtension = Actor.noSender "the Kamon actor metrics" should { "track configured actors" in { diff --git a/kamon-spray/src/main/scala/spray/can/server/ServerRequestTracing.scala b/kamon-spray/src/main/scala/spray/can/server/ServerRequestTracing.scala index a7d48ac8..f0e52d12 100644 --- a/kamon-spray/src/main/scala/spray/can/server/ServerRequestTracing.scala +++ b/kamon-spray/src/main/scala/spray/can/server/ServerRequestTracing.scala @@ -39,7 +39,7 @@ class ServerRequestTracing { val system: ActorSystem = openRequest.asInstanceOf[OpenRequest].context.actorContext.system val config = system.settings.config.getConfig("kamon.spray") - val token = if(config.getBoolean("include-trace-token-header")) { + val token = if (config.getBoolean("include-trace-token-header")) { val traceTokenHeader = config.getString("trace-token-header-name") request.headers.find(_.name == traceTokenHeader).map(_.value) } else None -- cgit v1.2.3