aboutsummaryrefslogtreecommitdiff
path: root/kamon-core/src/main/scala/kamon
diff options
context:
space:
mode:
Diffstat (limited to 'kamon-core/src/main/scala/kamon')
-rw-r--r--kamon-core/src/main/scala/kamon/ClassLoading.scala6
-rw-r--r--kamon-core/src/main/scala/kamon/Kamon.scala7
-rw-r--r--kamon-core/src/main/scala/kamon/Metrics.scala2
-rw-r--r--kamon-core/src/main/scala/kamon/ModuleLoading.scala5
-rw-r--r--kamon-core/src/main/scala/kamon/Status.scala40
-rw-r--r--kamon-core/src/main/scala/kamon/metric/Metric.scala3
-rw-r--r--kamon-core/src/main/scala/kamon/metric/MetricRegistry.scala13
-rw-r--r--kamon-core/src/main/scala/kamon/module/Module.scala416
-rw-r--r--kamon-core/src/main/scala/kamon/module/ModuleRegistry.scala477
-rw-r--r--kamon-core/src/main/scala/kamon/module/ReportingModule.scala21
-rw-r--r--kamon-core/src/main/scala/kamon/status/Status.scala146
11 files changed, 739 insertions, 397 deletions
diff --git a/kamon-core/src/main/scala/kamon/ClassLoading.scala b/kamon-core/src/main/scala/kamon/ClassLoading.scala
index 5b097af1..54d1922c 100644
--- a/kamon-core/src/main/scala/kamon/ClassLoading.scala
+++ b/kamon-core/src/main/scala/kamon/ClassLoading.scala
@@ -23,4 +23,10 @@ trait ClassLoading {
def createInstance[T: ClassTag](fqcn: String, args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
_dynamicAccess.createInstanceFor(fqcn, args)
+
+ def createInstance[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
+ _dynamicAccess.createInstanceFor(clazz, args)
+
+ def resolveClass[T: ClassTag](fqcn: String): Try[Class[_ <: T]] =
+ _dynamicAccess.getClassFor(fqcn)
}
diff --git a/kamon-core/src/main/scala/kamon/Kamon.scala b/kamon-core/src/main/scala/kamon/Kamon.scala
index ab95d773..8b2b64e6 100644
--- a/kamon-core/src/main/scala/kamon/Kamon.scala
+++ b/kamon-core/src/main/scala/kamon/Kamon.scala
@@ -22,7 +22,8 @@ object Kamon extends ClassLoading
with Tracing
with ModuleLoading
with ContextPropagation
- with ContextStorage {
+ with ContextStorage
+ with Status {
@volatile private var _environment = Environment.fromConfig(config())
@@ -31,6 +32,6 @@ object Kamon extends ClassLoading
_environment
onReconfigure(newConfig => {
- _environment = Environment.fromConfig(config)
+ _environment = Environment.fromConfig(newConfig)
})
-}
+} \ No newline at end of file
diff --git a/kamon-core/src/main/scala/kamon/Metrics.scala b/kamon-core/src/main/scala/kamon/Metrics.scala
index a2b74c20..5863b73e 100644
--- a/kamon-core/src/main/scala/kamon/Metrics.scala
+++ b/kamon-core/src/main/scala/kamon/Metrics.scala
@@ -6,7 +6,7 @@ import kamon.metric._
trait Metrics extends MetricLookup { self: Configuration with Utilities =>
- private val _metricsRegistry = new MetricRegistry(self.config(), self.scheduler())
+ protected val _metricsRegistry = new MetricRegistry(self.config(), self.scheduler())
override def histogram(name: String, unit: MeasurementUnit, dynamicRange: Option[DynamicRange]): HistogramMetric =
_metricsRegistry.histogram(name, unit, dynamicRange)
diff --git a/kamon-core/src/main/scala/kamon/ModuleLoading.scala b/kamon-core/src/main/scala/kamon/ModuleLoading.scala
index 8fe035d6..9501f277 100644
--- a/kamon-core/src/main/scala/kamon/ModuleLoading.scala
+++ b/kamon-core/src/main/scala/kamon/ModuleLoading.scala
@@ -3,6 +3,7 @@ package kamon
import com.typesafe.config.Config
import kamon.metric.{MetricsSnapshot, PeriodSnapshot}
import kamon.module.Module
+import kamon.module.ModuleRegistry
import kamon.util.Registration
import kamon.module.{MetricReporter => NewMetricReporter}
import kamon.module.{SpanReporter => NewSpanReporter}
@@ -30,13 +31,15 @@ trait SpanReporter extends kamon.module.SpanReporter { }
* kamon.modules {
* module-name {
* enabled = true
+ * description = "A module description"
+ * kind = "combined | metric | span | plain"
* class = "com.example.MyModule"
* }
* }
*
*/
trait ModuleLoading { self: ClassLoading with Configuration with Utilities with Metrics with Tracing =>
- private val _moduleRegistry = new Module.Registry(self, self, clock(), self.metricRegistry(), self.tracer())
+ protected val _moduleRegistry = new ModuleRegistry(self, self, clock(), self.metricRegistry(), self.tracer())
self.onReconfigure(newConfig => self._moduleRegistry.reconfigure(newConfig))
diff --git a/kamon-core/src/main/scala/kamon/Status.scala b/kamon-core/src/main/scala/kamon/Status.scala
new file mode 100644
index 00000000..a5cfe868
--- /dev/null
+++ b/kamon-core/src/main/scala/kamon/Status.scala
@@ -0,0 +1,40 @@
+/* =========================================================================================
+ * Copyright © 2013-2019 the kamon project <https://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
+
+/**
+ * Exposes access to the Kamon's current status. The status information contains details about the internal state of
+ * several Kamon components and is exposed for the sole purpose of troubleshooting and debugging issues that might be
+ * related to Kamon.
+ *
+ * The Status APIs might change between minor versions.
+ */
+trait Status { self: ModuleLoading with Metrics with Configuration =>
+ private val _status = new kamon.status.Status(self._moduleRegistry, self._metricsRegistry, self)
+
+ /**
+ * Returns an accessor to Kamon's current status. The current status information is split into four main sections:
+ * - Settings: which include the Kamon version, environment and configuration being used.
+ * - Module Registry: Lists all modules that have been detected on the classpath and their current state.
+ * - Metric Registry: Lists all metrics currently registered in Kamon and all instruments belonging to them.
+ * - Instrumentation: Lists all instrumentation modules that have been detected and their current state.
+ *
+ * All information exposed by the Status API represents an immutable snapshot of the state at the moment the status
+ * was requested.
+ */
+ def status(): kamon.status.Status =
+ _status
+}
diff --git a/kamon-core/src/main/scala/kamon/metric/Metric.scala b/kamon-core/src/main/scala/kamon/metric/Metric.scala
index a66d1229..f5ce7b45 100644
--- a/kamon-core/src/main/scala/kamon/metric/Metric.scala
+++ b/kamon-core/src/main/scala/kamon/metric/Metric.scala
@@ -85,6 +85,9 @@ private[kamon] abstract sealed class BaseMetric[T, S](val instrumentType: Instru
private[kamon] def snapshot(): Seq[S] =
instruments.values.map(createSnapshot).toSeq
+ private[kamon] def incarnations(): Seq[Map[String, String]] =
+ instruments.keys.toSeq
+
protected def createInstrument(tags: Tags): T
protected def createSnapshot(instrument: T): S
diff --git a/kamon-core/src/main/scala/kamon/metric/MetricRegistry.scala b/kamon-core/src/main/scala/kamon/metric/MetricRegistry.scala
index cc111d95..05bff637 100644
--- a/kamon-core/src/main/scala/kamon/metric/MetricRegistry.scala
+++ b/kamon-core/src/main/scala/kamon/metric/MetricRegistry.scala
@@ -26,6 +26,7 @@ import scala.collection.concurrent.TrieMap
import java.time.Duration
import java.util.concurrent.ScheduledExecutorService
+import kamon.status.Status
import org.slf4j.LoggerFactory
@@ -88,6 +89,18 @@ class MetricRegistry(initialConfig: Config, scheduler: ScheduledExecutorService)
metric.asInstanceOf[T]
}
+
+ private[kamon] def status(): Status.MetricRegistry = {
+ var registeredMetrics = Seq.empty[Status.Metric]
+ metrics.foreach {
+ case (metricName, metric) =>
+ metric.incarnations().foreach(incarnation => {
+ registeredMetrics = registeredMetrics :+ Status.Metric(metricName, incarnation, metric.unit, metric.instrumentType)
+ })
+ }
+
+ Status.MetricRegistry(registeredMetrics)
+ }
}
trait MetricsSnapshotGenerator {
diff --git a/kamon-core/src/main/scala/kamon/module/Module.scala b/kamon-core/src/main/scala/kamon/module/Module.scala
index 41649629..592e02aa 100644
--- a/kamon-core/src/main/scala/kamon/module/Module.scala
+++ b/kamon-core/src/main/scala/kamon/module/Module.scala
@@ -1,20 +1,9 @@
package kamon
package module
-import java.time.{Duration, Instant}
-import java.util.concurrent.{CountDownLatch, Executors, ScheduledFuture, TimeUnit}
-import java.util.concurrent.atomic.AtomicReference
-
import com.typesafe.config.Config
-import kamon.metric.{MetricsSnapshotGenerator, PeriodSnapshot}
-import kamon.trace.Tracer.SpanBuffer
-import kamon.util.Clock
-import org.slf4j.LoggerFactory
-
-import scala.collection.JavaConverters.collectionAsScalaIterableConverter
-import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService, Future, Promise}
-import scala.util.Try
-import scala.util.control.NonFatal
+import kamon.metric.PeriodSnapshot
+import kamon.trace.Span
/**
* Modules provide additional capabilities to Kamon, like collecting JVM metrics or exporting the metrics and trace
@@ -46,8 +35,39 @@ trait Module {
}
+/**
+ * Modules implementing this trait will get registered for periodically receiving metric period snapshots. The
+ * frequency of the period snapshots is controlled by the kamon.metric.tick-interval setting.
+ */
+trait MetricReporter extends Module {
+ def reportPeriodSnapshot(snapshot: PeriodSnapshot): Unit
+}
+
+/**
+ * Modules implementing this trait will get registered for periodically receiving span batches. The frequency of the
+ * span batches is controlled by the kamon.trace.tick-interval setting.
+ */
+trait SpanReporter extends Module {
+ def reportSpans(spans: Seq[Span.FinishedSpan]): Unit
+}
+
+/**
+ * Modules implementing this trait will get registered for periodically receiving metric period snapshots and span
+ * batches.
+ */
+trait CombinedReporter extends MetricReporter with SpanReporter
+
+
object Module {
+ sealed trait Kind
+ object Kind {
+ case object Combined extends Kind
+ case object Metric extends Kind
+ case object Span extends Kind
+ case object Plain extends Kind
+ }
+
/**
* Represents a module's registration on the module registry. A module can be stopped at any time by cancelling its
* registration.
@@ -61,366 +81,20 @@ object Module {
}
/**
- * Controls the lifecycle of all available modules.
+ * Configuration of a given module present in the classpath.
+ *
+ * @param name Module's name
+ * @param description Module's description.
+ * @param clazz The class implementing the configured module.
+ * @param kind Module kind.
+ * @param enabled Whether the module is enabled or not. Enabled modules in the classpath will be automatically
+ * started in any call to Kamon.loadModules().
*/
- class Registry(classLoading: ClassLoading, configuration: Configuration, clock: Clock, snapshotGenerator: MetricsSnapshotGenerator, spanBuffer: SpanBuffer) {
- private val _logger = LoggerFactory.getLogger(classOf[Registry])
- private val _metricsTickerExecutor = Executors.newScheduledThreadPool(1, threadFactory("kamon-metrics-ticker", daemon = true))
- private val _spansTickerExecutor = Executors.newScheduledThreadPool(1, threadFactory("kamon-spans-ticker", daemon = true))
-
- private val _metricsTickerSchedule = new AtomicReference[ScheduledFuture[_]]()
- private val _spansTickerSchedule = new AtomicReference[ScheduledFuture[_]]()
-
- private var _registrySettings = readRegistryConfiguration(configuration.config())
- private var _registeredModules: Map[String, Entry[Module]] = Map.empty
- private var _metricReporterModules: Map[String, Entry[MetricReporter]] = Map.empty
- private var _spanReporterModules: Map[String, Entry[SpanReporter]] = Map.empty
-
- // Start ticking as soon as the registry is created.
- scheduleMetricsTicker()
- scheduleSpansTicker()
-
-
- /**
- * Registers a module that has already been instantiated by the user. The start callback will be executed as part
- * of the registration process. If a module with the specified name already exists the registration will fail. If
- * the registered module is a MetricReporter and/or SpanReporter it will also be configured to receive the metrics
- * and spans data upon every tick.
- *
- * @param name Desired module name.
- * @param module Module instance.
- * @return A registration that can be used to stop the module at any time.
- */
- def register(name: String, module: Module): Registration = synchronized {
- if(_registeredModules.get(name).isEmpty) {
- val moduleEntry = createEntry(name, true, module)
- startModule(moduleEntry)
- registration(moduleEntry)
-
- } else {
- _logger.warn(s"Cannot register module [$name], a module with that name already exists.")
- noopRegistration(name)
- }
- }
-
- /**
- * Reads all available modules from the config and either starts, stops or reconfigures them in order to match the
- * configured modules state.
- */
- def load(config: Config): Unit = synchronized {
- val configuredModules = readModuleSettings(config)
- val automaticallyRegisteredModules = _registeredModules.filterNot { case (_, module) => module.programmaticallyAdded }
-
- // Start, reconfigure and stop modules that are still present but disabled.
- configuredModules.foreach { moduleSettings =>
- automaticallyRegisteredModules.get(moduleSettings.name).fold {
- // The module does not exist in the registry, the only possible action is starting it, if enabled.
- if(moduleSettings.enabled) {
- createModule(moduleSettings).foreach(entry => startModule(entry))
- }
-
- } { existentModuleSettings =>
- // When a module already exists it can either need to be stopped, or to be reconfigured.
- if(moduleSettings.enabled) {
- reconfigureModule(existentModuleSettings, config)
- } else {
- stopModule(existentModuleSettings)
- }
- }
- }
-
- // Remove all modules that no longer exist in the configuration.
- val missingModules = automaticallyRegisteredModules.filterKeys(moduleName => configuredModules.find(_.name == moduleName).isEmpty)
- missingModules.foreach {
- case (_, entry) => stopModule(entry)
- }
- }
-
- /**
- * Schedules the reconfigure hook on all registered modules and applies the latest configuration settings to the
- * registry.
- */
- def reconfigure(newConfig: Config): Unit = synchronized {
- _registrySettings = readRegistryConfiguration(configuration.config())
- _registeredModules.values.foreach(entry => reconfigureModule(entry, newConfig))
- scheduleMetricsTicker()
- scheduleSpansTicker()
- }
-
- /**
- * Stops all registered modules. As part of the stop process, all modules get a last chance to report metrics and
- * spans available until the call to stop.
- */
- def stop(): Future[Unit] = synchronized {
- implicit val cleanupExecutor = ExecutionContext.Implicits.global
- scheduleMetricsTicker(once = true)
- scheduleSpansTicker(once = true)
-
- val stopSignals = _registeredModules.values.map(stopModule)
- val latch = new CountDownLatch(stopSignals.size)
- stopSignals.foreach(f => f.onComplete(_ => latch.countDown()))
-
- // There is a global 30 seconds limit to shutdown after which all executors will shut down.
- val stopCompletionFuture = Future(latch.await(30, TimeUnit.SECONDS))
- stopCompletionFuture.onComplete(_ => {
- _metricsTickerExecutor.shutdown()
- _spansTickerExecutor.shutdown()
- })
-
- stopCompletionFuture.map(_ => ())
- }
-
-
- /**
- * (Re)Schedules the metrics ticker that periodically takes snapshots from the metric registry and sends them to
- * all available metric reporting modules. If a ticker was already scheduled then that scheduled job will be
- * cancelled and scheduled again.
- */
- private def scheduleMetricsTicker(once: Boolean = false): Unit = {
- val currentMetricsTicker = _metricsTickerSchedule.get()
- if(currentMetricsTicker != null)
- currentMetricsTicker.cancel(false)
-
- _metricsTickerSchedule.set {
- val interval = _registrySettings.metricTickInterval.toMillis
- val initialDelay = if(_registrySettings.optimisticMetricTickAlignment) {
- val now = clock.instant()
- val nextTick = Clock.nextTick(now, _registrySettings.metricTickInterval)
- Duration.between(now, nextTick).toMillis
- } else _registrySettings.metricTickInterval.toMillis
-
- val ticker = new Runnable {
- var lastInstant = Instant.now(clock)
-
- override def run(): Unit = try {
- val currentInstant = Instant.now(clock)
- val periodSnapshot = PeriodSnapshot(
- from = lastInstant,
- to = currentInstant,
- metrics = snapshotGenerator.snapshot())
-
- metricReporterModules().foreach { entry =>
- Future {
- Try(entry.module.reportPeriodSnapshot(periodSnapshot)).failed.foreach { error =>
- _logger.error(s"Reporter [${entry.name}] failed to process a metrics tick.", error)
- }
- }(entry.executionContext)
- }
-
- lastInstant = currentInstant
- } catch {
- case NonFatal(t) => _logger.error("Failed to run a metrics tick", t)
- }
- }
-
- if(once)
- _metricsTickerExecutor.schedule(ticker, 0L, TimeUnit.MILLISECONDS)
- else
- _metricsTickerExecutor.scheduleAtFixedRate(ticker, initialDelay, interval, TimeUnit.MILLISECONDS)
- }
- }
-
- /**
- * (Re)Schedules the spans ticker that periodically takes the spans accumulated by the tracer and flushes them to
- * all available span reporting modules. If a ticker was already scheduled then that scheduled job will be
- * cancelled and scheduled again.
- */
- private def scheduleSpansTicker(once: Boolean = false): Unit = {
- val currentSpansTicker = _spansTickerSchedule.get()
- if(currentSpansTicker != null)
- currentSpansTicker.cancel(false)
-
- _spansTickerSchedule.set {
- val interval = _registrySettings.traceTickInterval.toMillis
-
- val ticker = new Runnable {
- override def run(): Unit = try {
- val spanBatch = spanBuffer.flush()
-
- spanReporterModules().foreach { entry =>
- Future {
- Try(entry.module.reportSpans(spanBatch)).failed.foreach { error =>
- _logger.error(s"Reporter [${entry.name}] failed to process a spans tick.", error)
- }
- }(entry.executionContext)
- }
-
- } catch {
- case NonFatal(t) => _logger.error("Failed to run a spans tick", t)
- }
- }
-
- if(once)
- _spansTickerExecutor.schedule(ticker, 0L, TimeUnit.MILLISECONDS)
- else
- _spansTickerExecutor.scheduleAtFixedRate(ticker, interval, interval, TimeUnit.MILLISECONDS)
- }
- }
-
- private def metricReporterModules(): Iterable[Entry[MetricReporter]] = synchronized {
- _metricReporterModules.values
- }
-
- private def spanReporterModules(): Iterable[Entry[SpanReporter]] = synchronized {
- _spanReporterModules.values
- }
-
- private def readModuleSettings(config: Config): Seq[Settings] = {
- val moduleConfigs = config.getConfig("kamon.modules").configurations
- val moduleSettings = moduleConfigs.map {
- case (moduleName, moduleConfig) =>
- val moduleSettings = Try {
- Settings(
- moduleName,
- moduleConfig.getString("class"),
- moduleConfig.getBoolean("enabled")
- )
- }
-
- moduleSettings.failed.foreach { t =>
- _logger.warn(s"Failed to read configuration for module [$moduleName]", t)
-
- if(moduleConfig.hasPath("requires-aspectj") || moduleConfig.hasPath("auto-start") || moduleConfig.hasPath("extension-class")) {
- _logger.warn(s"Module [$moduleName] contains legacy configuration settings, please ensure that no legacy configuration")
- }
- }
-
- moduleSettings
-
- } filter(_.isSuccess) map(_.get) toSeq
-
- // Legacy modules from <1.2.0
- val legacyModules = config.getStringList("kamon.reporters").asScala map { moduleClass =>
- Settings(moduleClass, moduleClass, true)
- } toSeq
-
- val (repeatedLegacyModules, uniqueLegacyModules) = legacyModules.partition(lm => moduleSettings.find(_.fqcn == lm.fqcn).nonEmpty)
- repeatedLegacyModules.foreach(m =>
- _logger.warn(s"Module [${m.name}] is configured twice, please remove it from the deprecated kamon.reporters setting."))
-
- uniqueLegacyModules.foreach(m =>
- _logger.warn(s"Module [${m.name}] is configured in the deprecated kamon.reporters setting, please consider moving it to kamon.modules."))
-
- moduleSettings ++ uniqueLegacyModules
- }
-
- /**
- * Creates a module from the provided settings.
- */
- private def createModule(settings: Settings): Option[Entry[Module]] = {
- val moduleInstance = classLoading.createInstance[Module](settings.fqcn, Nil)
- val moduleEntry = moduleInstance.map(instance => createEntry(settings.name, false, instance))
-
- moduleEntry.failed.foreach(t => _logger.warn(s"Failed to create instance of module [${settings.name}]", t))
- moduleEntry.toOption
- }
-
- private def createEntry(name: String, programmaticallyAdded: Boolean, instance: Module): Entry[Module] = {
- val executor = Executors.newSingleThreadExecutor(threadFactory(name))
- Entry(name, ExecutionContext.fromExecutorService(executor), programmaticallyAdded, instance)
- }
-
-
- /**
- * Registers a module and schedules execution of its start procedure.
- */
- private def startModule(entry: Entry[Module]): Unit = {
- registerModule(entry)
-
- // Schedule the start hook on the module
- entry.executionContext.execute(new Runnable {
- override def run(): Unit =
- Try(entry.module.start())
- .failed.foreach(t => _logger.warn(s"Failure occurred while starting module [${entry.name}]", t))
- })
- }
-
- private def registerModule(entry: Entry[Module]): Unit = {
- _registeredModules = _registeredModules + (entry.name -> entry)
- if(entry.module.isInstanceOf[MetricReporter])
- _metricReporterModules = _metricReporterModules + (entry.name -> entry.asInstanceOf[Entry[MetricReporter]])
- if(entry.module.isInstanceOf[SpanReporter])
- _spanReporterModules = _spanReporterModules + (entry.name -> entry.asInstanceOf[Entry[SpanReporter]])
-
- }
-
- /**
- * Removes the module from the registry and schedules a call to the stop lifecycle hook on the module's execution
- * context. The returned future completes when the module finishes its stop procedure.
- */
- private def stopModule(entry: Entry[Module]): Future[Unit] = synchronized {
- val cleanupExecutor = ExecutionContext.Implicits.global
-
- // Remove the module from all registries
- _registeredModules = _registeredModules - entry.name
- if(entry.module.isInstanceOf[MetricReporter])
- _metricReporterModules = _metricReporterModules - entry.name
- if(entry.module.isInstanceOf[SpanReporter])
- _spanReporterModules = _spanReporterModules - entry.name
-
-
- // Schedule a call to stop on the module
- val stopPromise = Promise[Unit]()
- entry.executionContext.execute(new Runnable {
- override def run(): Unit =
- stopPromise.complete {
- val stopResult = Try(entry.module.stop())
- stopResult.failed.foreach(t => _logger.warn(s"Failure occurred while stopping module [${entry.name}]", t))
- stopResult
- }
-
- })
-
- stopPromise.future.onComplete(_ => entry.executionContext.shutdown())(cleanupExecutor)
- stopPromise.future
- }
-
- /**
- * Schedules a call to reconfigure on the module's execution context.
- */
- private def reconfigureModule(entry: Entry[Module], config: Config): Unit = {
- entry.executionContext.execute(new Runnable {
- override def run(): Unit =
- Try(entry.module.reconfigure(config))
- .failed.foreach(t => _logger.warn(s"Failure occurred while reconfiguring module [${entry.name}]", t))
- })
- }
-
- private def noopRegistration(moduleName: String): Registration = new Registration {
- override def cancel(): Unit =
- _logger.warn(s"Cannot cancel registration on module [$moduleName] because the module was not added properly")
- }
-
- private def registration(entry: Entry[Module]): Registration = new Registration {
- override def cancel(): Unit = stopModule(entry)
- }
- }
-
- private def readRegistryConfiguration(config: Config): RegistrySettings =
- RegistrySettings(
- metricTickInterval = config.getDuration("kamon.metric.tick-interval"),
- optimisticMetricTickAlignment = config.getBoolean("kamon.metric.optimistic-tick-alignment"),
- traceTickInterval = config.getDuration("kamon.trace.tick-interval"),
- traceReporterQueueSize = config.getInt("kamon.trace.reporter-queue-size")
- )
-
- private case class RegistrySettings(
- metricTickInterval: Duration,
- optimisticMetricTickAlignment: Boolean,
- traceTickInterval: Duration,
- traceReporterQueueSize: Int
- )
-
- private case class Settings(
+ case class Settings(
name: String,
- fqcn: String,
+ description: String,
+ clazz: Class[_ <: Module],
+ kind: Module.Kind,
enabled: Boolean
)
-
- private case class Entry[T <: Module](
- name: String,
- executionContext: ExecutionContextExecutorService,
- programmaticallyAdded: Boolean,
- module: T
- )
} \ No newline at end of file
diff --git a/kamon-core/src/main/scala/kamon/module/ModuleRegistry.scala b/kamon-core/src/main/scala/kamon/module/ModuleRegistry.scala
new file mode 100644
index 00000000..d45cd80f
--- /dev/null
+++ b/kamon-core/src/main/scala/kamon/module/ModuleRegistry.scala
@@ -0,0 +1,477 @@
+package kamon
+package module
+
+import java.time.{Duration, Instant}
+import java.util.concurrent.{CountDownLatch, Executors, ScheduledFuture, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+import com.typesafe.config.Config
+import kamon.metric.{MetricsSnapshotGenerator, PeriodSnapshot}
+import kamon.module.Module.Registration
+import kamon.status.Status
+import kamon.trace.Tracer.SpanBuffer
+import kamon.util.Clock
+import org.slf4j.LoggerFactory
+
+import scala.collection.JavaConverters.asScalaBufferConverter
+import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService, Future, Promise}
+import scala.util.Try
+import scala.util.control.NonFatal
+
+
+
+/**
+ * Controls the lifecycle of all available modules.
+ */
+class ModuleRegistry(classLoading: ClassLoading, configuration: Configuration, clock: Clock, snapshotGenerator: MetricsSnapshotGenerator, spanBuffer: SpanBuffer) {
+ private val _logger = LoggerFactory.getLogger(classOf[ModuleRegistry])
+ private val _metricsTickerExecutor = Executors.newScheduledThreadPool(1, threadFactory("kamon-metrics-ticker", daemon = true))
+ private val _spansTickerExecutor = Executors.newScheduledThreadPool(1, threadFactory("kamon-spans-ticker", daemon = true))
+
+ private val _metricsTickerSchedule = new AtomicReference[ScheduledFuture[_]]()
+ private val _spansTickerSchedule = new AtomicReference[ScheduledFuture[_]]()
+
+ private var _registrySettings = readRegistrySettings(configuration.config())
+ private var _registeredModules: Map[String, Entry[Module]] = Map.empty
+ private var _metricReporterModules: Map[String, Entry[MetricReporter]] = Map.empty
+ private var _spanReporterModules: Map[String, Entry[SpanReporter]] = Map.empty
+
+ // Start ticking as soon as the registry is created.
+ scheduleMetricsTicker()
+ scheduleSpansTicker()
+
+
+ /**
+ * Registers a module that has already been instantiated by the user. The start callback will be executed as part
+ * of the registration process. If a module with the specified name already exists the registration will fail. If
+ * the registered module is a MetricReporter and/or SpanReporter it will also be configured to receive the metrics
+ * and spans data upon every tick.
+ *
+ * @param name Desired module name.
+ * @param module Module instance.
+ * @return A registration that can be used to stop the module at any time.
+ */
+ def register(name: String, module: Module): Registration = synchronized {
+ if(_registeredModules.get(name).isEmpty) {
+ val inferredSettings = Module.Settings(
+ name,
+ module.getClass.getName,
+ module.getClass,
+ inferModuleKind(module.getClass),
+ true
+ )
+
+ val moduleEntry = createEntry(inferredSettings, true, module)
+ startModule(moduleEntry)
+ registration(moduleEntry)
+
+ } else {
+ _logger.warn(s"Cannot register module [$name], a module with that name already exists.")
+ noopRegistration(name)
+ }
+ }
+
+ /**
+ * Reads all available modules from the config and either starts, stops or reconfigures them in order to match the
+ * configured modules state.
+ */
+ def load(config: Config): Unit = synchronized {
+ val configuredModules = readModuleSettings(config, true)
+ val automaticallyRegisteredModules = _registeredModules.filterNot { case (_, module) => module.programmaticallyAdded }
+
+ // Start, reconfigure and stop modules that are still present but disabled.
+ configuredModules.foreach { moduleSettings =>
+ automaticallyRegisteredModules.get(moduleSettings.name).fold {
+ // The module does not exist in the registry, the only possible action is starting it, if enabled.
+ if(moduleSettings.enabled) {
+ createModule(moduleSettings).foreach(entry => startModule(entry))
+ }
+
+ } { existentModuleSettings =>
+ // When a module already exists it can either need to be stopped, or to be reconfigured.
+ if(moduleSettings.enabled) {
+ reconfigureModule(existentModuleSettings, config)
+ } else {
+ stopModule(existentModuleSettings)
+ }
+ }
+ }
+
+ // Remove all modules that no longer exist in the configuration.
+ val missingModules = automaticallyRegisteredModules.filterKeys(moduleName => configuredModules.find(_.name == moduleName).isEmpty)
+ missingModules.foreach {
+ case (_, entry) => stopModule(entry)
+ }
+ }
+
+ /**
+ * Schedules the reconfigure hook on all registered modules and applies the latest configuration settings to the
+ * registry.
+ */
+ def reconfigure(newConfig: Config): Unit = synchronized {
+ _registrySettings = readRegistrySettings(configuration.config())
+ _registeredModules.values.foreach(entry => reconfigureModule(entry, newConfig))
+ scheduleMetricsTicker()
+ scheduleSpansTicker()
+ }
+
+ /**
+ * Stops all registered modules. As part of the stop process, all modules get a last chance to report metrics and
+ * spans available until the call to stop.
+ */
+ def stop(): Future[Unit] = synchronized {
+ implicit val cleanupExecutor = ExecutionContext.Implicits.global
+ scheduleMetricsTicker(once = true)
+ scheduleSpansTicker(once = true)
+
+ val stopSignals = _registeredModules.values.map(stopModule)
+ val latch = new CountDownLatch(stopSignals.size)
+ stopSignals.foreach(f => f.onComplete(_ => latch.countDown()))
+
+ // There is a global 30 seconds limit to shutdown after which all executors will shut down.
+ val stopCompletionFuture = Future(latch.await(30, TimeUnit.SECONDS))
+ stopCompletionFuture.onComplete(_ => {
+ _metricsTickerExecutor.shutdown()
+ _spansTickerExecutor.shutdown()
+ })
+
+ stopCompletionFuture.map(_ => ())
+ }
+
+
+ /**
+ * (Re)Schedules the metrics ticker that periodically takes snapshots from the metric registry and sends them to
+ * all available metric reporting modules. If a ticker was already scheduled then that scheduled job will be
+ * cancelled and scheduled again.
+ */
+ private def scheduleMetricsTicker(once: Boolean = false): Unit = {
+ val currentMetricsTicker = _metricsTickerSchedule.get()
+ if(currentMetricsTicker != null)
+ currentMetricsTicker.cancel(false)
+
+ _metricsTickerSchedule.set {
+ val interval = _registrySettings.metricTickInterval.toMillis
+ val initialDelay = if(_registrySettings.optimisticMetricTickAlignment) {
+ val now = clock.instant()
+ val nextTick = Clock.nextTick(now, _registrySettings.metricTickInterval)
+ Duration.between(now, nextTick).toMillis
+ } else _registrySettings.metricTickInterval.toMillis
+
+ val ticker = new Runnable {
+ var lastInstant = Instant.now(clock)
+
+ override def run(): Unit = try {
+ val currentInstant = Instant.now(clock)
+ val periodSnapshot = PeriodSnapshot(
+ from = lastInstant,
+ to = currentInstant,
+ metrics = snapshotGenerator.snapshot())
+
+ metricReporterModules().foreach { entry =>
+ Future {
+ Try(entry.module.reportPeriodSnapshot(periodSnapshot)).failed.foreach { error =>
+ _logger.error(s"Reporter [${entry.name}] failed to process a metrics tick.", error)
+ }
+ }(entry.executionContext)
+ }
+
+ lastInstant = currentInstant
+ } catch {
+ case NonFatal(t) => _logger.error("Failed to run a metrics tick", t)
+ }
+ }
+
+ if(once)
+ _metricsTickerExecutor.schedule(ticker, 0L, TimeUnit.MILLISECONDS)
+ else
+ _metricsTickerExecutor.scheduleAtFixedRate(ticker, initialDelay, interval, TimeUnit.MILLISECONDS)
+ }
+ }
+
+ /**
+ * (Re)Schedules the spans ticker that periodically takes the spans accumulated by the tracer and flushes them to
+ * all available span reporting modules. If a ticker was already scheduled then that scheduled job will be
+ * cancelled and scheduled again.
+ */
+ private def scheduleSpansTicker(once: Boolean = false): Unit = {
+ val currentSpansTicker = _spansTickerSchedule.get()
+ if(currentSpansTicker != null)
+ currentSpansTicker.cancel(false)
+
+ _spansTickerSchedule.set {
+ val interval = _registrySettings.traceTickInterval.toMillis
+
+ val ticker = new Runnable {
+ override def run(): Unit = try {
+ val spanBatch = spanBuffer.flush()
+
+ spanReporterModules().foreach { entry =>
+ Future {
+ Try(entry.module.reportSpans(spanBatch)).failed.foreach { error =>
+ _logger.error(s"Reporter [${entry.name}] failed to process a spans tick.", error)
+ }
+ }(entry.executionContext)
+ }
+
+ } catch {
+ case NonFatal(t) => _logger.error("Failed to run a spans tick", t)
+ }
+ }
+
+ if(once)
+ _spansTickerExecutor.schedule(ticker, 0L, TimeUnit.MILLISECONDS)
+ else
+ _spansTickerExecutor.scheduleAtFixedRate(ticker, interval, interval, TimeUnit.MILLISECONDS)
+ }
+ }
+
+ private def metricReporterModules(): Iterable[Entry[MetricReporter]] = synchronized {
+ _metricReporterModules.values
+ }
+
+ private def spanReporterModules(): Iterable[Entry[SpanReporter]] = synchronized {
+ _spanReporterModules.values
+ }
+
+ private def readModuleSettings(config: Config, emitConfigurationWarnings: Boolean): Seq[Module.Settings] = {
+ val moduleConfigs = config.getConfig("kamon.modules").configurations
+ val moduleSettings = moduleConfigs.map {
+ case (moduleName, moduleConfig) =>
+ val moduleSettings = Try {
+ Module.Settings(
+ moduleName,
+ moduleConfig.getString("description"),
+ classLoading.resolveClass[Module](moduleConfig.getString("class")).get,
+ parseModuleKind(moduleConfig.getString("kind")),
+ moduleConfig.getBoolean("enabled")
+ )
+ }.map(ms => {
+ val inferredModuleKind = inferModuleKind(ms.clazz)
+ assert(inferredModuleKind == ms.kind,
+ s"Module [${ms.name}] is configured as [${ms.kind}] but the actual type does not comply to the expected interface.")
+ ms
+ })
+
+
+ if(emitConfigurationWarnings) {
+ moduleSettings.failed.foreach { t =>
+ _logger.warn(s"Failed to read configuration for module [$moduleName]", t)
+
+ val hasLegacySettings =
+ moduleConfig.hasPath("requires-aspectj") ||
+ moduleConfig.hasPath("auto-start") ||
+ moduleConfig.hasPath("extension-class")
+
+ if (hasLegacySettings) {
+ _logger.warn(s"Module [$moduleName] contains legacy configuration settings, please ensure that no legacy configuration")
+ }
+ }
+ }
+
+ moduleSettings
+
+ } filter(_.isSuccess) map(_.get) toSeq
+
+
+ // Load all modules that might have been configured using the legacy "kamon.reporters" setting from <1.2.0
+ // versions. This little hack should be removed by the time we release 2.0.
+ //
+ if(config.hasPath("kamon.reporters")) {
+ val legacyModuleSettings = config.getStringList("kamon.reporters").asScala
+ .map(moduleClass => {
+ val moduleSettings = Try {
+ val moduleClazz = classLoading.resolveClass[Module](moduleClass).get
+ val inferredModuleKind = inferModuleKind(moduleClazz)
+ val name = moduleClazz.getName()
+ val description = "Module detected from the legacy kamon.reporters configuration."
+
+ Module.Settings(name, description, moduleClazz, inferredModuleKind, true)
+ }
+
+ if(emitConfigurationWarnings) {
+ moduleSettings.failed.foreach(t => _logger.error(s"Failed to load legacy reporter module [${moduleClass}]", t))
+ }
+
+ moduleSettings
+ })
+ .filter(_.isSuccess)
+ .map(_.get)
+
+
+ val (repeatedLegacyModules, uniqueLegacyModules) = legacyModuleSettings
+ .partition(lm => moduleSettings.find(_.clazz.getName == lm.clazz.getName).nonEmpty)
+
+ if(emitConfigurationWarnings) {
+ repeatedLegacyModules.foreach(m =>
+ _logger.warn(s"Module [${m.name}] is configured twice, please remove it from the deprecated kamon.reporters setting."))
+
+ uniqueLegacyModules.foreach(m =>
+ _logger.warn(s"Module [${m.name}] is configured in the deprecated kamon.reporters setting, please consider moving it to kamon.modules."))
+ }
+
+ moduleSettings ++ uniqueLegacyModules
+
+ } else moduleSettings
+ }
+
+ /**
+ * Creates a module from the provided settings.
+ */
+ private def createModule(settings: Module.Settings): Option[Entry[Module]] = {
+ val moduleInstance = classLoading.createInstance[Module](settings.clazz, Nil)
+ val moduleEntry = moduleInstance.map(instance => createEntry(settings, false, instance))
+
+ moduleEntry.failed.foreach(t => _logger.warn(s"Failed to create instance of module [${settings.name}]", t))
+ moduleEntry.toOption
+ }
+
+ private def createEntry(settings: Module.Settings, programmaticallyAdded: Boolean, instance: Module): Entry[Module] = {
+ val executor = Executors.newSingleThreadExecutor(threadFactory(settings.name))
+ Entry(settings.name, ExecutionContext.fromExecutorService(executor), programmaticallyAdded, settings, instance)
+ }
+
+ private def inferModuleKind(clazz: Class[_ <: Module]): Module.Kind = {
+ if(classOf[CombinedReporter].isAssignableFrom(clazz))
+ Module.Kind.Combined
+ else if(classOf[MetricReporter].isAssignableFrom(clazz))
+ Module.Kind.Metric
+ else if(classOf[SpanReporter].isAssignableFrom(clazz))
+ Module.Kind.Span
+ else
+ Module.Kind.Plain
+ }
+
+
+ /**
+ * Returns the current status of this module registry.
+ */
+ private[kamon] def status(): Status.ModuleRegistry = {
+ val automaticallyAddedModules = readModuleSettings(configuration.config(), false).map(moduleSettings => {
+ val isActive = _registeredModules.get(moduleSettings.name).nonEmpty
+
+ Status.Module(
+ moduleSettings.name,
+ moduleSettings.description,
+ moduleSettings.clazz.getCanonicalName,
+ moduleSettings.kind,
+ programmaticallyRegistered = false,
+ moduleSettings.enabled,
+ isActive)
+ })
+
+ val programmaticallyAddedModules = _registeredModules
+ .filter { case (_, entry) => entry.programmaticallyAdded }
+ .map { case (name, entry) => Status.Module(name, entry.settings.description, entry.settings.clazz.getCanonicalName,
+ entry.settings.kind, true, true, true) }
+
+ val allModules = automaticallyAddedModules ++ programmaticallyAddedModules
+ Status.ModuleRegistry(allModules)
+ }
+
+
+ /**
+ * Registers a module and schedules execution of its start procedure.
+ */
+ private def startModule(entry: Entry[Module]): Unit = {
+ registerModule(entry)
+
+ // Schedule the start hook on the module
+ entry.executionContext.execute(new Runnable {
+ override def run(): Unit =
+ Try(entry.module.start())
+ .failed.foreach(t => _logger.warn(s"Failure occurred while starting module [${entry.name}]", t))
+ })
+ }
+
+ private def registerModule(entry: Entry[Module]): Unit = {
+ _registeredModules = _registeredModules + (entry.name -> entry)
+ if(entry.module.isInstanceOf[MetricReporter])
+ _metricReporterModules = _metricReporterModules + (entry.name -> entry.asInstanceOf[Entry[MetricReporter]])
+ if(entry.module.isInstanceOf[SpanReporter])
+ _spanReporterModules = _spanReporterModules + (entry.name -> entry.asInstanceOf[Entry[SpanReporter]])
+
+ }
+
+ /**
+ * Removes the module from the registry and schedules a call to the stop lifecycle hook on the module's execution
+ * context. The returned future completes when the module finishes its stop procedure.
+ */
+ private def stopModule(entry: Entry[Module]): Future[Unit] = synchronized {
+ val cleanupExecutor = ExecutionContext.Implicits.global
+
+ // Remove the module from all registries
+ _registeredModules = _registeredModules - entry.name
+ if(entry.module.isInstanceOf[MetricReporter])
+ _metricReporterModules = _metricReporterModules - entry.name
+ if(entry.module.isInstanceOf[SpanReporter])
+ _spanReporterModules = _spanReporterModules - entry.name
+
+
+ // Schedule a call to stop on the module
+ val stopPromise = Promise[Unit]()
+ entry.executionContext.execute(new Runnable {
+ override def run(): Unit =
+ stopPromise.complete {
+ val stopResult = Try(entry.module.stop())
+ stopResult.failed.foreach(t => _logger.warn(s"Failure occurred while stopping module [${entry.name}]", t))
+ stopResult
+ }
+
+ })
+
+ stopPromise.future.onComplete(_ => entry.executionContext.shutdown())(cleanupExecutor)
+ stopPromise.future
+ }
+
+ /**
+ * Schedules a call to reconfigure on the module's execution context.
+ */
+ private def reconfigureModule(entry: Entry[Module], config: Config): Unit = {
+ entry.executionContext.execute(new Runnable {
+ override def run(): Unit =
+ Try(entry.module.reconfigure(config))
+ .failed.foreach(t => _logger.warn(s"Failure occurred while reconfiguring module [${entry.name}]", t))
+ })
+ }
+
+ private def noopRegistration(moduleName: String): Registration = new Registration {
+ override def cancel(): Unit =
+ _logger.warn(s"Cannot cancel registration on module [$moduleName] because the module was not added properly")
+ }
+
+ private def registration(entry: Entry[Module]): Registration = new Registration {
+ override def cancel(): Unit = stopModule(entry)
+ }
+
+ private def parseModuleKind(kind: String): Module.Kind = kind.toLowerCase match {
+ case "combined" => Module.Kind.Combined
+ case "metric" => Module.Kind.Metric
+ case "span" => Module.Kind.Span
+ case "plain" => Module.Kind.Plain
+ }
+
+ private def readRegistrySettings(config: Config): Settings =
+ Settings(
+ metricTickInterval = config.getDuration("kamon.metric.tick-interval"),
+ optimisticMetricTickAlignment = config.getBoolean("kamon.metric.optimistic-tick-alignment"),
+ traceTickInterval = config.getDuration("kamon.trace.tick-interval"),
+ traceReporterQueueSize = config.getInt("kamon.trace.reporter-queue-size")
+ )
+
+ private case class Settings(
+ metricTickInterval: Duration,
+ optimisticMetricTickAlignment: Boolean,
+ traceTickInterval: Duration,
+ traceReporterQueueSize: Int
+ )
+
+
+ private case class Entry[T <: Module](
+ name: String,
+ executionContext: ExecutionContextExecutorService,
+ programmaticallyAdded: Boolean,
+ settings: Module.Settings,
+ module: T
+ )
+}
+
diff --git a/kamon-core/src/main/scala/kamon/module/ReportingModule.scala b/kamon-core/src/main/scala/kamon/module/ReportingModule.scala
deleted file mode 100644
index 0e88fc23..00000000
--- a/kamon-core/src/main/scala/kamon/module/ReportingModule.scala
+++ /dev/null
@@ -1,21 +0,0 @@
-package kamon
-package module
-
-import kamon.trace.Span
-import kamon.metric.PeriodSnapshot
-
-/**
- * Modules implementing this trait will get registered for periodically receiving metric period snapshots. The
- * frequency of the period snapshots is controlled by the kamon.metric.tick-interval setting.
- */
-trait MetricReporter extends Module {
- def reportPeriodSnapshot(snapshot: PeriodSnapshot): Unit
-}
-
-/**
- * Modules implementing this trait will get registered for periodically receiving span batches. The frequency of the
- * span batches is controlled by the kamon.trace.tick-interval setting.
- */
-trait SpanReporter extends Module {
- def reportSpans(spans: Seq[Span.FinishedSpan]): Unit
-}
diff --git a/kamon-core/src/main/scala/kamon/status/Status.scala b/kamon-core/src/main/scala/kamon/status/Status.scala
new file mode 100644
index 00000000..5c6d7cb0
--- /dev/null
+++ b/kamon-core/src/main/scala/kamon/status/Status.scala
@@ -0,0 +1,146 @@
+package kamon.status
+
+import com.typesafe.config.Config
+import kamon.metric.InstrumentFactory.InstrumentType
+import kamon.metric.{MeasurementUnit, MetricRegistry}
+import kamon.{Configuration, Environment, Kamon}
+import kamon.module.ModuleRegistry
+import kamon.module.Module.{Kind => ModuleKind}
+import java.util.{Collections, List => JavaList, Map => JavaMap}
+
+/**
+ * Exposes Kamon components' status information. This is meant to be used for informational and debugging purposes and
+ * by no means should replace the use of reporters to extract information from Kamon.
+ */
+class Status(_moduleRegistry: ModuleRegistry, _metricRegistry: MetricRegistry, configuration: Configuration) {
+
+ /**
+ * Settings currently used by Kamon.
+ */
+ def settings(): Status.Settings =
+ Status.Settings(BuildInfo.version, Kamon.environment, configuration.config())
+
+ /**
+ * Status of the module registry. Describes what modules have been detected and registered, either from the classpath
+ * or programmatically and their current status.
+ */
+ def moduleRegistry(): Status.ModuleRegistry =
+ _moduleRegistry.status()
+
+ /**
+ * Status of the metric registry. Describes all metrics currently tracked by Kamon.
+ */
+ def metricRegistry(): Status.MetricRegistry =
+ _metricRegistry.status()
+
+
+ /**
+ * Status of instrumentation modules that have been detected and/or loaded into the current JVM. Read the
+ * [[Status.Instrumentation]] companion object's docs for more information.
+ */
+ def instrumentation(): Status.Instrumentation = {
+ import Status.Instrumentation._
+
+ Status.Instrumentation(
+ isActive(),
+ modules(),
+ errors()
+ )
+ }
+}
+
+
+
+object Status {
+
+ case class Settings(
+ version: String,
+ environment: Environment,
+ config: Config
+ )
+
+
+ case class ModuleRegistry(
+ modules: Seq[Module]
+ )
+
+ case class Module(
+ name: String,
+ description: String,
+ clazz: String,
+ kind: ModuleKind,
+ programmaticallyRegistered: Boolean,
+ enabled: Boolean,
+ started: Boolean
+ )
+
+ case class MetricRegistry(
+ metrics: Seq[Metric]
+ )
+
+ case class Metric(
+ name: String,
+ tags: Map[String, String],
+ unit: MeasurementUnit,
+ instrumentType: InstrumentType
+ )
+
+
+ /**
+ * Status of the instrumentation modules. This data is completely untyped and not expected to be used anywhere
+ * outside Kamon.
+ */
+ private[kamon] case class Instrumentation(
+ active: Boolean,
+ modules: JavaMap[String, String],
+ errors: JavaMap[String, JavaList[Throwable]]
+ )
+
+
+ /**
+ * This object works as a bridge between Kamon and Kanela to gather information about instrumentation modules. When
+ * instrumentation is enabled, it should replace the implementation of the members of this object and return proper
+ * information.
+ *
+ * This data is only exposed directly to the status page API because it lacks any sort of type safety. We might
+ * change this in the future and provide proper types for all instrumentation modules' info.
+ */
+ private[kamon] object Instrumentation {
+
+ /**
+ * Whether instrumentation is active or not. When Kanela is present it will replace this method to return true.
+ */
+ def isActive(): java.lang.Boolean =
+ false
+
+ /**
+ * List all instrumentation modules known and their current status. The result map contains the module name as keys
+ * and a JSON representation of the module status as values. The expected structure in the JSON representations is
+ * as follows:
+ *
+ * {
+ * 'description': 'A explicative module description',
+ * 'isEnabled': true | false,
+ * 'isActive': true | false
+ * }
+ *
+ * The "isEnabled" flag tells whether the module is able to instrument classes or not. By default, all modules are
+ * able to instrument classes but some modules might be shipped in a disabled state or forced to be disabled via
+ * configuration.
+ *
+ * The "isActive" flag tells whether the modules has already applied instrumentation to any of its target classes.
+ *
+ */
+ def modules(): JavaMap[String, String] =
+ Collections.emptyMap()
+
+
+ /**
+ * List all errors that might have happened during the instrumentation initialization. The resulting map contains
+ * a list of modules and any exceptions thrown by them during initialization. If not exceptions are thrown the map
+ * will always be empty.
+ */
+ def errors(): JavaMap[String, JavaList[Throwable]] =
+ Collections.emptyMap()
+ }
+}