aboutsummaryrefslogtreecommitdiff
path: root/kamon-core/src
diff options
context:
space:
mode:
Diffstat (limited to 'kamon-core/src')
-rw-r--r--kamon-core/src/main/resources/reference.conf13
-rw-r--r--kamon-core/src/main/scala/kamon/metric/EntityRecorder.scala3
-rw-r--r--kamon-core/src/main/scala/kamon/trace/MetricsOnlyContext.scala4
-rw-r--r--kamon-core/src/main/scala/kamon/trace/Sampler.scala32
-rw-r--r--kamon-core/src/main/scala/kamon/trace/TraceLocal.scala31
-rw-r--r--kamon-core/src/main/scala/kamon/trace/TraceSettings.scala (renamed from kamon-core/src/main/scala/kamon/trace/TracerExtensionSettings.scala)7
-rw-r--r--kamon-core/src/main/scala/kamon/trace/TracerModule.scala17
-rw-r--r--kamon-core/src/test/scala/kamon/testkit/BaseKamonSpec.scala2
-rw-r--r--kamon-core/src/test/scala/kamon/trace/SamplerSpec.scala76
-rw-r--r--kamon-core/src/test/scala/kamon/trace/TraceLocalSpec.scala17
10 files changed, 160 insertions, 42 deletions
diff --git a/kamon-core/src/main/resources/reference.conf b/kamon-core/src/main/resources/reference.conf
index c78d56cd..b3adbe83 100644
--- a/kamon-core/src/main/resources/reference.conf
+++ b/kamon-core/src/main/resources/reference.conf
@@ -87,15 +87,15 @@ kamon {
trace {
- # Level of detail used when recording trace information. The posible values are:
+ # Level of detail used when recording trace information. The possible values are:
# - metrics-only: metrics for all included traces and all segments are recorded, but no Trace messages will be sent
- # to the subscriptors of trace data.
+ # to the subscribers of trace data.
# - simple-trace: metrics for all included traces and all segments are recorded and additionally a Trace message
# containing the trace and segments details and metadata.
level-of-detail = metrics-only
- # Sampling strategy to apply when the tracing level is set to `simple-trace`. The options are: all, random, ordered
- # and threshold. The details of each sampler are bellow.
+ # Sampling strategy to apply when the tracing level is set to `simple-trace`. The options are: all, random, ordered,
+ # threshold and clock. The details of each sampler are below.
sampling = random
# Use a ThreadLocalRandom to generate numbers between 1 and 100, if the random number is less or equal to .chance
@@ -120,6 +120,11 @@ kamon {
minimum-elapsed-time = 1 second
}
+ # Use a FiniteDuration to only record a trace each .pause nanoseconds.
+ clock-sampler {
+ pause = 1 second
+ }
+
incubator {
# Minimum time to stay in the trace incubator before checking if the trace should not be incubated anymore. No
# checks are made at least until this period has passed.
diff --git a/kamon-core/src/main/scala/kamon/metric/EntityRecorder.scala b/kamon-core/src/main/scala/kamon/metric/EntityRecorder.scala
index 65dafa9a..15e20d1a 100644
--- a/kamon-core/src/main/scala/kamon/metric/EntityRecorder.scala
+++ b/kamon-core/src/main/scala/kamon/metric/EntityRecorder.scala
@@ -204,6 +204,9 @@ abstract class GenericEntityRecorder(instrumentFactory: InstrumentFactory) exten
protected def counter(name: String): Counter =
register(CounterKey(name, UnitOfMeasurement.Unknown), instrumentFactory.createCounter())
+ protected def counter(name: String, unitOfMeasurement: UnitOfMeasurement): Counter =
+ register(CounterKey(name, unitOfMeasurement), instrumentFactory.createCounter())
+
protected def counter(key: CounterKey): Counter =
register(key, instrumentFactory.createCounter())
diff --git a/kamon-core/src/main/scala/kamon/trace/MetricsOnlyContext.scala b/kamon-core/src/main/scala/kamon/trace/MetricsOnlyContext.scala
index 17be661b..869bcc12 100644
--- a/kamon-core/src/main/scala/kamon/trace/MetricsOnlyContext.scala
+++ b/kamon-core/src/main/scala/kamon/trace/MetricsOnlyContext.scala
@@ -39,7 +39,7 @@ private[kamon] class MetricsOnlyContext(traceName: String, val token: String, iz
def rename(newName: String): Unit =
if (isOpen)
_name = newName
- else if (log.isWarningEnabled)
+ else
log.warning("Can't rename trace from [{}] to [{}] because the trace is already closed.", name, newName)
def name: String = _name
@@ -101,7 +101,7 @@ private[kamon] class MetricsOnlyContext(traceName: String, val token: String, iz
def rename(newName: String): Unit =
if (isOpen)
_segmentName = newName
- else if (log.isWarningEnabled)
+ else
log.warning("Can't rename segment from [{}] to [{}] because the segment is already closed.", name, newName)
def finish: Unit = {
diff --git a/kamon-core/src/main/scala/kamon/trace/Sampler.scala b/kamon-core/src/main/scala/kamon/trace/Sampler.scala
index ac780d5e..8480ecff 100644
--- a/kamon-core/src/main/scala/kamon/trace/Sampler.scala
+++ b/kamon-core/src/main/scala/kamon/trace/Sampler.scala
@@ -18,8 +18,7 @@ package kamon.trace
import java.net.InetAddress
import java.util.concurrent.atomic.AtomicLong
-
-import kamon.util.{ NanoInterval, Sequencer }
+import kamon.util.{ NanoTimestamp, NanoInterval, Sequencer }
import scala.concurrent.forkjoin.ThreadLocalRandom
import scala.util.Try
@@ -50,8 +49,8 @@ class RandomSampler(chance: Int) extends Sampler {
class OrderedSampler(interval: Int) extends Sampler {
import OrderedSampler._
- require(interval > 0, "kamon.trace.ordered-sampler.interval cannot be <= 0")
- assume(interval isPowerOfTwo, "kamon.trace.ordered-sampler.interval must be power of two")
+ require(interval > 0, "kamon.trace.ordered-sampler.sample-interval cannot be <= 0")
+ assume(interval isPowerOfTwo, "kamon.trace.ordered-sampler.sample-interval must be power of two")
private val sequencer = Sequencer()
@@ -69,11 +68,27 @@ object OrderedSampler {
}
}
-class ThresholdSampler(thresholdInNanoseconds: Long) extends Sampler {
- require(thresholdInNanoseconds > 0, "kamon.trace.threshold-sampler.minimum-elapsed-time cannot be <= 0")
+class ThresholdSampler(thresholdInNanoseconds: NanoInterval) extends Sampler {
+ require(thresholdInNanoseconds.nanos > 0, "kamon.trace.threshold-sampler.minimum-elapsed-time cannot be <= 0")
def shouldTrace: Boolean = true
- def shouldReport(traceElapsedTime: NanoInterval): Boolean = traceElapsedTime.nanos >= thresholdInNanoseconds
+ def shouldReport(traceElapsedTime: NanoInterval): Boolean = traceElapsedTime >= thresholdInNanoseconds
+}
+
+class ClockSampler(pauseInNanoseconds: NanoInterval) extends Sampler {
+ require(pauseInNanoseconds.nanos > 0, "kamon.trace.clock-sampler.pause cannot be <= 0")
+
+ private val timer: AtomicLong = new AtomicLong(0L)
+
+ def shouldTrace: Boolean = {
+ val now = NanoTimestamp.now.nanos
+ val lastTimer = timer.get()
+ if ((lastTimer + pauseInNanoseconds.nanos) < now)
+ timer.compareAndSet(lastTimer, now)
+ else
+ false
+ }
+ def shouldReport(traceElapsedTime: NanoInterval): Boolean = true
}
class DefaultTokenGenerator extends Function0[String] {
@@ -83,5 +98,4 @@ class DefaultTokenGenerator extends Function0[String] {
def apply(): String = {
_hostnamePrefix + "-" + String.valueOf(_tokenCounter.incrementAndGet())
}
-}
-
+} \ No newline at end of file
diff --git a/kamon-core/src/main/scala/kamon/trace/TraceLocal.scala b/kamon-core/src/main/scala/kamon/trace/TraceLocal.scala
index 0a0a120a..460e4b22 100644
--- a/kamon-core/src/main/scala/kamon/trace/TraceLocal.scala
+++ b/kamon-core/src/main/scala/kamon/trace/TraceLocal.scala
@@ -17,17 +17,14 @@
package kamon.trace
import kamon.trace.TraceLocal.TraceLocalKey
-
+import kamon.util.Supplier
import scala.collection.concurrent.TrieMap
object TraceLocal {
- trait TraceLocalKey {
- type ValueType
- }
+ trait TraceLocalKey[T]
- trait AvailableToMdc extends TraceLocalKey {
- override type ValueType = String
+ trait AvailableToMdc extends TraceLocalKey[String] {
def mdcKey: String
}
@@ -38,26 +35,30 @@ object TraceLocal {
def apply(mdcKey: String): AvailableToMdc = fromKey(mdcKey)
}
- case class HttpContext(agent: String, uri: String, xforwarded: String)
-
- object HttpContextKey extends TraceLocal.TraceLocalKey { type ValueType = HttpContext }
-
- def store(key: TraceLocalKey)(value: key.ValueType): Unit = Tracer.currentContext match {
+ def store[T](key: TraceLocalKey[T])(value: Any): Unit = Tracer.currentContext match {
case ctx: MetricsOnlyContext ⇒ ctx.traceLocalStorage.store(key)(value)
case EmptyTraceContext ⇒ // Can't store in the empty context.
}
- def retrieve(key: TraceLocalKey): Option[key.ValueType] = Tracer.currentContext match {
+ def retrieve[T](key: TraceLocalKey[T]): Option[T] = Tracer.currentContext match {
case ctx: MetricsOnlyContext ⇒ ctx.traceLocalStorage.retrieve(key)
case EmptyTraceContext ⇒ None // Can't retrieve anything from the empty context.
}
+ // Java variant
+ @throws(classOf[NoSuchElementException])
+ def get[T](key: TraceLocalKey[T]): T = retrieve(key).get
+
+ def getOrElse[T](key: TraceLocalKey[T], code: Supplier[T]): T = retrieve(key).getOrElse(code.get)
+
def storeForMdc(key: String, value: String): Unit = store(AvailableToMdc.fromKey(key))(value)
+
+ def newTraceLocalKey[T]: TraceLocalKey[T] = new TraceLocalKey[T] {}
}
class TraceLocalStorage {
- val underlyingStorage = TrieMap[TraceLocal.TraceLocalKey, Any]()
+ val underlyingStorage = TrieMap[TraceLocalKey[_], Any]()
- def store(key: TraceLocalKey)(value: key.ValueType): Unit = underlyingStorage.put(key, value)
- def retrieve(key: TraceLocalKey): Option[key.ValueType] = underlyingStorage.get(key).map(_.asInstanceOf[key.ValueType])
+ def store[T](key: TraceLocalKey[T])(value: Any): Unit = underlyingStorage.put(key, value)
+ def retrieve[T](key: TraceLocalKey[T]): Option[T] = underlyingStorage.get(key).map(_.asInstanceOf[T])
}
diff --git a/kamon-core/src/main/scala/kamon/trace/TracerExtensionSettings.scala b/kamon-core/src/main/scala/kamon/trace/TraceSettings.scala
index 80f59466..06677314 100644
--- a/kamon-core/src/main/scala/kamon/trace/TracerExtensionSettings.scala
+++ b/kamon-core/src/main/scala/kamon/trace/TraceSettings.scala
@@ -16,9 +16,9 @@
package kamon.trace
-import java.util.concurrent.TimeUnit
import kamon.util.ConfigTools.Syntax
import com.typesafe.config.Config
+import kamon.util.NanoInterval
case class TraceSettings(levelOfDetail: LevelOfDetail, sampler: Sampler, tokenGeneratorFQN: String)
@@ -37,8 +37,9 @@ object TraceSettings {
else tracerConfig.getString("sampling") match {
case "all" ⇒ SampleAll
case "random" ⇒ new RandomSampler(tracerConfig.getInt("random-sampler.chance"))
- case "ordered" ⇒ new OrderedSampler(tracerConfig.getInt("ordered-sampler.interval"))
- case "threshold" ⇒ new ThresholdSampler(tracerConfig.getFiniteDuration("threshold-sampler.minimum-elapsed-time").toNanos)
+ case "ordered" ⇒ new OrderedSampler(tracerConfig.getInt("ordered-sampler.sample-interval"))
+ case "threshold" ⇒ new ThresholdSampler(new NanoInterval(tracerConfig.getFiniteDuration("threshold-sampler.minimum-elapsed-time").toNanos))
+ case "clock" ⇒ new ClockSampler(new NanoInterval(tracerConfig.getFiniteDuration("clock-sampler.pause").toNanos))
}
val tokenGeneratorFQN = tracerConfig.getString("token-generator")
diff --git a/kamon-core/src/main/scala/kamon/trace/TracerModule.scala b/kamon-core/src/main/scala/kamon/trace/TracerModule.scala
index 88464a30..06286cae 100644
--- a/kamon-core/src/main/scala/kamon/trace/TracerModule.scala
+++ b/kamon-core/src/main/scala/kamon/trace/TracerModule.scala
@@ -17,6 +17,7 @@
package kamon.trace
import akka.actor._
+import akka.event.{ LoggingAdapter, Logging }
import com.typesafe.config.Config
import kamon.Kamon
import kamon.metric.MetricsModule
@@ -112,17 +113,17 @@ private[kamon] class TracerModuleImpl(metricsExtension: MetricsModule, config: C
isOpen: Boolean = true, isLocal: Boolean = true): TraceContext = {
def newMetricsOnlyContext(token: String): TraceContext =
- new MetricsOnlyContext(traceName, token, isOpen, _settings.levelOfDetail, startTimestamp, null)
+ new MetricsOnlyContext(traceName, token, isOpen, _settings.levelOfDetail, startTimestamp, _logger)
val traceToken = token.getOrElse(newToken)
- if (_settings.levelOfDetail == LevelOfDetail.MetricsOnly || !isLocal)
- newMetricsOnlyContext(traceToken)
- else {
- if (!_settings.sampler.shouldTrace)
+ _settings.levelOfDetail match {
+ case LevelOfDetail.MetricsOnly ⇒
newMetricsOnlyContext(traceToken)
- else
- new TracingContext(traceName, traceToken, true, _settings.levelOfDetail, isLocal, startTimestamp, null, dispatchTracingContext)
+ case _ if !isLocal || !_settings.sampler.shouldTrace ⇒
+ newMetricsOnlyContext(traceToken)
+ case _ ⇒
+ new TracingContext(traceName, traceToken, true, _settings.levelOfDetail, isLocal, startTimestamp, _logger, dispatchTracingContext)
}
}
@@ -143,6 +144,7 @@ private[kamon] class TracerModuleImpl(metricsExtension: MetricsModule, config: C
* Tracer Extension initialization.
*/
private var _system: ActorSystem = null
+ private var _logger: LoggingAdapter = null
private lazy val _start = {
val subscriptions = _system.actorOf(Props[TraceSubscriptions], "trace-subscriptions")
_subscriptions.point(subscriptions)
@@ -151,6 +153,7 @@ private[kamon] class TracerModuleImpl(metricsExtension: MetricsModule, config: C
def start(system: ActorSystem): Unit = synchronized {
_system = system
+ _logger = Logging(_system, "TracerModule")
_start
_system = null
}
diff --git a/kamon-core/src/test/scala/kamon/testkit/BaseKamonSpec.scala b/kamon-core/src/test/scala/kamon/testkit/BaseKamonSpec.scala
index 61cc4e6b..cb909ad9 100644
--- a/kamon-core/src/test/scala/kamon/testkit/BaseKamonSpec.scala
+++ b/kamon-core/src/test/scala/kamon/testkit/BaseKamonSpec.scala
@@ -28,7 +28,7 @@ import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpecLike }
abstract class BaseKamonSpec(actorSystemName: String) extends TestKitBase with WordSpecLike with Matchers with ImplicitSender with BeforeAndAfterAll {
lazy val collectionContext = Kamon.metrics.buildDefaultCollectionContext
implicit lazy val system: ActorSystem = {
- //Kamon.start(config.withFallback(ConfigFactory.load()))
+ Kamon.start(config.withFallback(ConfigFactory.load()))
ActorSystem(actorSystemName, config)
}
diff --git a/kamon-core/src/test/scala/kamon/trace/SamplerSpec.scala b/kamon-core/src/test/scala/kamon/trace/SamplerSpec.scala
new file mode 100644
index 00000000..88cdf116
--- /dev/null
+++ b/kamon-core/src/test/scala/kamon/trace/SamplerSpec.scala
@@ -0,0 +1,76 @@
+package kamon.trace
+
+import kamon.testkit.BaseKamonSpec
+import kamon.util.NanoInterval
+
+class SamplerSpec extends BaseKamonSpec("sampler-spec") {
+
+ "the Sampler" should {
+ "work as intended" when {
+ "using all mode" in {
+ val sampler = SampleAll
+
+ sampler.shouldTrace should be(true)
+
+ sampler.shouldReport(NanoInterval.default) should be(true)
+ }
+
+ "using random mode" in {
+ val sampler = new RandomSampler(100)
+
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(true)
+
+ sampler.shouldReport(NanoInterval.default) should be(true)
+ }
+
+ "using ordered mode" in {
+ var sampler = new OrderedSampler(1)
+
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(true)
+
+ sampler = new OrderedSampler(2)
+
+ sampler.shouldTrace should be(false)
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(false)
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(false)
+ sampler.shouldTrace should be(true)
+
+ sampler.shouldReport(NanoInterval.default) should be(true)
+ }
+
+ "using threshold mode" in {
+ val sampler = new ThresholdSampler(new NanoInterval(10000000L))
+
+ sampler.shouldTrace should be(true)
+
+ sampler.shouldReport(new NanoInterval(5000000L)) should be(false)
+ sampler.shouldReport(new NanoInterval(10000000L)) should be(true)
+ sampler.shouldReport(new NanoInterval(15000000L)) should be(true)
+ sampler.shouldReport(new NanoInterval(0L)) should be(false)
+ }
+
+ "using clock mode" in {
+ val sampler = new ClockSampler(new NanoInterval(10000000L))
+
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(false)
+ Thread.sleep(1L)
+ sampler.shouldTrace should be(false)
+ Thread.sleep(10L)
+ sampler.shouldTrace should be(true)
+ sampler.shouldTrace should be(false)
+
+ sampler.shouldReport(NanoInterval.default) should be(true)
+ }
+ }
+ }
+
+}
diff --git a/kamon-core/src/test/scala/kamon/trace/TraceLocalSpec.scala b/kamon-core/src/test/scala/kamon/trace/TraceLocalSpec.scala
index b58b247f..41d5bc83 100644
--- a/kamon-core/src/test/scala/kamon/trace/TraceLocalSpec.scala
+++ b/kamon-core/src/test/scala/kamon/trace/TraceLocalSpec.scala
@@ -19,6 +19,7 @@ package kamon.trace
import kamon.testkit.BaseKamonSpec
import kamon.trace.TraceLocal.AvailableToMdc
import kamon.trace.logging.MdcKeysSupport
+import kamon.util.Supplier
import org.scalatest.concurrent.PatienceConfiguration
import org.scalatest.OptionValues
import org.slf4j.MDC
@@ -26,7 +27,7 @@ import org.slf4j.MDC
class TraceLocalSpec extends BaseKamonSpec("trace-local-spec") with PatienceConfiguration with OptionValues with MdcKeysSupport {
val SampleTraceLocalKeyAvailableToMDC = AvailableToMdc("someKey")
- object SampleTraceLocalKey extends TraceLocal.TraceLocalKey { type ValueType = String }
+ object SampleTraceLocalKey extends TraceLocal.TraceLocalKey[String]
"the TraceLocal storage" should {
"allow storing and retrieving values" in {
@@ -44,6 +45,20 @@ class TraceLocalSpec extends BaseKamonSpec("trace-local-spec") with PatienceConf
}
}
+ "throws an exception when trying to get a non existent key" in {
+ Tracer.withContext(newContext("non-existent-key")) {
+ intercept[NoSuchElementException] {
+ TraceLocal.get(SampleTraceLocalKey)
+ }
+ }
+ }
+
+ "return the given value when retrieving a non existent key" in {
+ Tracer.withContext(newContext("non-existent-key")) {
+ TraceLocal.getOrElse(SampleTraceLocalKey, new Supplier[String] { def get = "optionalValue" }) should equal("optionalValue")
+ }
+ }
+
"return None when retrieving a key without a current TraceContext" in {
TraceLocal.retrieve(SampleTraceLocalKey) should equal(None)
}