From 3a8c0fa25f12230b27e943d1fffe07f814c650fe Mon Sep 17 00:00:00 2001 From: Ivan Topolnjak Date: Mon, 14 Aug 2017 17:30:16 +0200 Subject: implement Span propagation on top of Kamon.Context --- .../src/main/scala/kamon/context/Codec.scala | 132 +++++++++++++++++++++ 1 file changed, 132 insertions(+) create mode 100644 kamon-core/src/main/scala/kamon/context/Codec.scala (limited to 'kamon-core/src/main/scala/kamon/context/Codec.scala') diff --git a/kamon-core/src/main/scala/kamon/context/Codec.scala b/kamon-core/src/main/scala/kamon/context/Codec.scala new file mode 100644 index 00000000..957c3e26 --- /dev/null +++ b/kamon-core/src/main/scala/kamon/context/Codec.scala @@ -0,0 +1,132 @@ +package kamon +package context + +import com.typesafe.config.Config +import kamon.util.DynamicAccess +import org.slf4j.LoggerFactory +import scala.collection.mutable + +class Codec(initialConfig: Config) { + private val log = LoggerFactory.getLogger(classOf[Codec]) + + @volatile private var httpHeaders: Codec.ForContext[TextMap] = new Codec.HttpHeaders(Map.empty) + //val Binary: Codec.ForContext[ByteBuffer] = _ + reconfigure(initialConfig) + + + def HttpHeaders: Codec.ForContext[TextMap] = + httpHeaders + + def reconfigure(config: Config): Unit = { + httpHeaders = new Codec.HttpHeaders(readEntryCodecs("kamon.context.encoding.http-headers", config)) + + + // Kamon.contextCodec.httpHeaderExport(current) + // Kamon.exportContext(HTTP, context) + // Kamon.importContext(HTTP, textMap) + // Kamon.currentContext() + // Kamon.storeContext(context) + + } + + private def readEntryCodecs[T](rootKey: String, config: Config): Map[String, Codec.ForEntry[T]] = { + val rootConfig = config.getConfig(rootKey) + val dynamic = new DynamicAccess(getClass.getClassLoader) + val entries = Map.newBuilder[String, Codec.ForEntry[T]] + + rootConfig.topLevelKeys.foreach(key => { + try { + val fqcn = rootConfig.getString(key) + entries += ((key, dynamic.createInstanceFor[Codec.ForEntry[T]](fqcn, Nil).get)) + } catch { + case e: Throwable => + log.error(s"Failed to initialize codec for key [$key]", e) + } + }) + + entries.result() + } +} + +object Codec { + + trait ForContext[T] { + def encode(context: Context): T + def decode(carrier: T): Context + } + + trait ForEntry[T] { + def encode(context: Context): T + def decode(carrier: T, context: Context): Context + } + + final class HttpHeaders(entryCodecs: Map[String, Codec.ForEntry[TextMap]]) extends Codec.ForContext[TextMap] { + private val log = LoggerFactory.getLogger(classOf[HttpHeaders]) + + override def encode(context: Context): TextMap = { + val encoded = TextMap.Default() + + context.entries.foreach { + case (key, _) if key.broadcast => + entryCodecs.get(key.name) match { + case Some(codec) => + try { + codec.encode(context).values.foreach(pair => encoded.put(pair._1, pair._2)) + } catch { + case e: Throwable => log.error(s"Failed to encode key [${key.name}]", e) + } + + case None => + log.error("Context key [{}] should be encoded in HttpHeaders but no codec was found for it.", key.name) + } + } + + encoded + } + + override def decode(carrier: TextMap): Context = { + var context: Context = Context.Empty + + try { + context = entryCodecs.foldLeft(context)((ctx, codecEntry) => { + val (_, codec) = codecEntry + codec.decode(carrier, ctx) + }) + + } catch { + case e: Throwable => + log.error("Failed to decode context from HttpHeaders", e) + } + + context + } + } + +} + + +trait TextMap { + def get(key: String): Option[String] + + def put(key: String, value: String): Unit + + def values: Iterator[(String, String)] +} + +object TextMap { + + class Default extends TextMap { + private val storage = mutable.Map.empty[String, String] + + override def get(key: String): Option[String] = storage.get(key) + + override def put(key: String, value: String): Unit = storage.put(key, value) + + override def values: Iterator[(String, String)] = storage.toIterator + } + + object Default { + def apply(): Default = new Default() + } + +} \ No newline at end of file -- cgit v1.2.3 From 407d74e89acdd4dff5a887c1ae5793d63e9e7a2e Mon Sep 17 00:00:00 2001 From: Ivan Topolnjak Date: Mon, 14 Aug 2017 23:08:14 +0200 Subject: expose the identity provider through the Tracer --- kamon-core/src/main/resources/reference.conf | 2 +- kamon-core/src/main/scala/kamon/Kamon.scala | 4 + .../src/main/scala/kamon/context/Codec.scala | 30 ++-- .../main/scala/kamon/trace/IdentityProvider.scala | 10 +- .../src/main/scala/kamon/trace/SpanCodec.scala | 99 +++++++++++ .../src/main/scala/kamon/trace/SpanContext.scala | 48 +---- .../main/scala/kamon/trace/SpanContextCodec.scala | 97 ----------- kamon-core/src/main/scala/kamon/trace/Tracer.scala | 14 +- .../src/test/scala/kamon/LogInterceptor.scala | 30 ---- .../scala/kamon/context/ContextCodecSpec.scala | 2 +- .../test/scala/kamon/testkit/SpanBuilding.scala | 10 +- .../test/scala/kamon/trace/B3SpanCodecSpec.scala | 192 ++++++++++++++++++++ .../kamon/trace/DefaultIdentityGeneratorSpec.scala | 4 +- .../DoubleLengthTraceIdentityGeneratorSpec.scala | 4 +- .../trace/ExtendedB3SpanContextCodecSpec.scala | 193 --------------------- 15 files changed, 341 insertions(+), 398 deletions(-) create mode 100644 kamon-core/src/main/scala/kamon/trace/SpanCodec.scala delete mode 100644 kamon-core/src/main/scala/kamon/trace/SpanContextCodec.scala delete mode 100644 kamon-core/src/test/scala/kamon/LogInterceptor.scala create mode 100644 kamon-core/src/test/scala/kamon/trace/B3SpanCodecSpec.scala delete mode 100644 kamon-core/src/test/scala/kamon/trace/ExtendedB3SpanContextCodecSpec.scala (limited to 'kamon-core/src/main/scala/kamon/context/Codec.scala') diff --git a/kamon-core/src/main/resources/reference.conf b/kamon-core/src/main/resources/reference.conf index 665cfc08..ad180f1c 100644 --- a/kamon-core/src/main/resources/reference.conf +++ b/kamon-core/src/main/resources/reference.conf @@ -144,7 +144,7 @@ kamon { encoding { http-headers { - span = "kamon.trace.propagation.B3" + span = "kamon.trace.SpanCodec$B3" } binary { diff --git a/kamon-core/src/main/scala/kamon/Kamon.scala b/kamon-core/src/main/scala/kamon/Kamon.scala index 5c33b3b5..b1490e32 100644 --- a/kamon-core/src/main/scala/kamon/Kamon.scala +++ b/kamon-core/src/main/scala/kamon/Kamon.scala @@ -94,6 +94,10 @@ object Kamon extends MetricLookup with ReporterRegistry with Tracer { override def buildSpan(operationName: String): Tracer.SpanBuilder = _tracer.buildSpan(operationName) + + override def identityProvider: IdentityProvider = + _tracer.identityProvider + def currentContext(): Context = _contextStorage.current() diff --git a/kamon-core/src/main/scala/kamon/context/Codec.scala b/kamon-core/src/main/scala/kamon/context/Codec.scala index 957c3e26..50b7e93d 100644 --- a/kamon-core/src/main/scala/kamon/context/Codec.scala +++ b/kamon-core/src/main/scala/kamon/context/Codec.scala @@ -2,11 +2,13 @@ package kamon package context import com.typesafe.config.Config +import kamon.trace.IdentityProvider import kamon.util.DynamicAccess import org.slf4j.LoggerFactory + import scala.collection.mutable -class Codec(initialConfig: Config) { +class Codec(identityProvider: IdentityProvider, initialConfig: Config) { private val log = LoggerFactory.getLogger(classOf[Codec]) @volatile private var httpHeaders: Codec.ForContext[TextMap] = new Codec.HttpHeaders(Map.empty) @@ -19,14 +21,6 @@ class Codec(initialConfig: Config) { def reconfigure(config: Config): Unit = { httpHeaders = new Codec.HttpHeaders(readEntryCodecs("kamon.context.encoding.http-headers", config)) - - - // Kamon.contextCodec.httpHeaderExport(current) - // Kamon.exportContext(HTTP, context) - // Kamon.importContext(HTTP, textMap) - // Kamon.currentContext() - // Kamon.storeContext(context) - } private def readEntryCodecs[T](rootKey: String, config: Config): Map[String, Codec.ForEntry[T]] = { @@ -101,11 +95,11 @@ object Codec { context } } - } trait TextMap { + def get(key: String): Option[String] def put(key: String, value: String): Unit @@ -116,17 +110,21 @@ trait TextMap { object TextMap { class Default extends TextMap { - private val storage = mutable.Map.empty[String, String] + private val storage = + mutable.Map.empty[String, String] - override def get(key: String): Option[String] = storage.get(key) + override def get(key: String): Option[String] = + storage.get(key) - override def put(key: String, value: String): Unit = storage.put(key, value) + override def put(key: String, value: String): Unit = + storage.put(key, value) - override def values: Iterator[(String, String)] = storage.toIterator + override def values: Iterator[(String, String)] = + storage.toIterator } object Default { - def apply(): Default = new Default() + def apply(): Default = + new Default() } - } \ No newline at end of file diff --git a/kamon-core/src/main/scala/kamon/trace/IdentityProvider.scala b/kamon-core/src/main/scala/kamon/trace/IdentityProvider.scala index 3f44629e..937200f5 100644 --- a/kamon-core/src/main/scala/kamon/trace/IdentityProvider.scala +++ b/kamon-core/src/main/scala/kamon/trace/IdentityProvider.scala @@ -8,8 +8,8 @@ import kamon.util.HexCodec import scala.util.Try trait IdentityProvider { - def traceIdentifierGenerator(): IdentityProvider.Generator - def spanIdentifierGenerator(): IdentityProvider.Generator + def traceIdGenerator(): IdentityProvider.Generator + def spanIdGenerator(): IdentityProvider.Generator } object IdentityProvider { @@ -57,8 +57,8 @@ object IdentityProvider { } getOrElse(IdentityProvider.NoIdentifier) } - override def traceIdentifierGenerator(): Generator = longGenerator - override def spanIdentifierGenerator(): Generator = longGenerator + override def traceIdGenerator(): Generator = longGenerator + override def spanIdGenerator(): Generator = longGenerator } object Default { @@ -97,7 +97,7 @@ object IdentityProvider { } getOrElse(IdentityProvider.NoIdentifier) } - override def traceIdentifierGenerator(): Generator = doubleLongGenerator + override def traceIdGenerator(): Generator = doubleLongGenerator } object DoubleSizeTraceID { diff --git a/kamon-core/src/main/scala/kamon/trace/SpanCodec.scala b/kamon-core/src/main/scala/kamon/trace/SpanCodec.scala new file mode 100644 index 00000000..e04ceb03 --- /dev/null +++ b/kamon-core/src/main/scala/kamon/trace/SpanCodec.scala @@ -0,0 +1,99 @@ +/* ========================================================================================= + * Copyright © 2013-2017 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.trace + +import java.net.{URLDecoder, URLEncoder} + +import kamon.Kamon +import kamon.context.{Codec, Context, TextMap} +import kamon.trace.SpanContext.SamplingDecision + + +object SpanCodec { + + class B3 extends Codec.ForEntry[TextMap] { + import B3.Headers + + override def encode(context: Context): TextMap = { + val span = context.get(Span.ContextKey) + val carrier = TextMap.Default() + + if(span.nonEmpty()) { + val spanContext = span.context + carrier.put(Headers.TraceIdentifier, urlEncode(spanContext.traceID.string)) + carrier.put(Headers.SpanIdentifier, urlEncode(spanContext.spanID.string)) + carrier.put(Headers.ParentSpanIdentifier, urlEncode(spanContext.parentID.string)) + + encodeSamplingDecision(spanContext.samplingDecision).foreach { samplingDecision => + carrier.put(Headers.Sampled, samplingDecision) + } + } + + carrier + } + + override def decode(carrier: TextMap, context: Context): Context = { + val identityProvider = Kamon.tracer.identityProvider + val traceID = carrier.get(Headers.TraceIdentifier) + .map(id => identityProvider.traceIdGenerator().from(urlDecode(id))) + .getOrElse(IdentityProvider.NoIdentifier) + + val spanID = carrier.get(Headers.SpanIdentifier) + .map(id => identityProvider.spanIdGenerator().from(urlDecode(id))) + .getOrElse(IdentityProvider.NoIdentifier) + + if(traceID != IdentityProvider.NoIdentifier && spanID != IdentityProvider.NoIdentifier) { + val parentID = carrier.get(Headers.ParentSpanIdentifier) + .map(id => identityProvider.spanIdGenerator().from(urlDecode(id))) + .getOrElse(IdentityProvider.NoIdentifier) + + val flags = carrier.get(Headers.Flags) + + val samplingDecision = flags.orElse(carrier.get(Headers.Sampled)) match { + case Some(sampled) if sampled == "1" => SamplingDecision.Sample + case Some(sampled) if sampled == "0" => SamplingDecision.DoNotSample + case _ => SamplingDecision.Unknown + } + + context.withKey(Span.ContextKey, Span.Remote(SpanContext(traceID, spanID, parentID, samplingDecision))) + + } else context + } + + private def encodeSamplingDecision(samplingDecision: SamplingDecision): Option[String] = samplingDecision match { + case SamplingDecision.Sample => Some("1") + case SamplingDecision.DoNotSample => Some("0") + case SamplingDecision.Unknown => None + } + + private def urlEncode(s: String): String = URLEncoder.encode(s, "UTF-8") + private def urlDecode(s: String): String = URLDecoder.decode(s, "UTF-8") + } + + object B3 { + + def apply(): B3 = + new B3() + + object Headers { + val TraceIdentifier = "X-B3-TraceId" + val ParentSpanIdentifier = "X-B3-ParentSpanId" + val SpanIdentifier = "X-B3-SpanId" + val Sampled = "X-B3-Sampled" + val Flags = "X-B3-Flags" + } + } +} \ No newline at end of file diff --git a/kamon-core/src/main/scala/kamon/trace/SpanContext.scala b/kamon-core/src/main/scala/kamon/trace/SpanContext.scala index e8b239ba..4d013881 100644 --- a/kamon-core/src/main/scala/kamon/trace/SpanContext.scala +++ b/kamon-core/src/main/scala/kamon/trace/SpanContext.scala @@ -34,64 +34,32 @@ case class SpanContext(traceID: Identifier, spanID: Identifier, parentID: Identi object SpanContext { val EmptySpanContext = SpanContext( - traceID = IdentityProvider.NoIdentifier, - spanID = IdentityProvider.NoIdentifier, - parentID = IdentityProvider.NoIdentifier, + traceID = IdentityProvider.NoIdentifier, + spanID = IdentityProvider.NoIdentifier, + parentID = IdentityProvider.NoIdentifier, samplingDecision = SamplingDecision.DoNotSample ) sealed trait SamplingDecision + object SamplingDecision { /** - * The Trace is sampled, all child Spans should be sampled as well. + * The Trace is sampled, all child Spans should be sampled as well. */ case object Sample extends SamplingDecision /** - * The Trace is not sampled, none of the child Spans should be sampled. + * The Trace is not sampled, none of the child Spans should be sampled. */ case object DoNotSample extends SamplingDecision /** - * The sampling decision has not been taken yet, the Tracer is free to decide when creating a Span. + * The sampling decision has not been taken yet, the Tracer is free to decide when creating a Span. */ case object Unknown extends SamplingDecision - } - - /** - * - */ - sealed trait Baggage { - def add(key: String, value:String): Unit - def get(key: String): Option[String] - def getAll(): Map[String, String] } - object Baggage { - def apply(): Baggage = new DefaultBaggage() - - case object EmptyBaggage extends Baggage { - override def add(key: String, value: String): Unit = {} - override def get(key: String): Option[String] = None - override def getAll: Map[String, String] = Map.empty - } - - - final class DefaultBaggage extends Baggage { - private var baggage: Map[String, String] = Map.empty - - def add(key: String, value: String): Unit = synchronized { - baggage = baggage + (key -> value) - } - - def get(key: String): Option[String] = - baggage.get(key) - - def getAll: Map[String, String] = - baggage - } - } -} +} \ No newline at end of file diff --git a/kamon-core/src/main/scala/kamon/trace/SpanContextCodec.scala b/kamon-core/src/main/scala/kamon/trace/SpanContextCodec.scala deleted file mode 100644 index 1db55694..00000000 --- a/kamon-core/src/main/scala/kamon/trace/SpanContextCodec.scala +++ /dev/null @@ -1,97 +0,0 @@ -/* ========================================================================================= - * Copyright © 2013-2017 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.trace - -import java.net.{URLDecoder, URLEncoder} - -import kamon.context.{Codec, Context, TextMap} -import kamon.trace.SpanContext.SamplingDecision - - -object SpanContextCodec { - - class ExtendedB3(identityProvider: IdentityProvider) extends Codec.ForEntry[TextMap] { - import ExtendedB3.Headers - - override def encode(context: Context): TextMap = { - val span = context.get(Span.ContextKey) - val carrier = TextMap.Default() - - if(span.nonEmpty()) { - val spanContext = span.context - carrier.put(Headers.TraceIdentifier, urlEncode(spanContext.traceID.string)) - carrier.put(Headers.SpanIdentifier, urlEncode(spanContext.spanID.string)) - carrier.put(Headers.ParentSpanIdentifier, urlEncode(spanContext.parentID.string)) - - encodeSamplingDecision(spanContext.samplingDecision).foreach { samplingDecision => - carrier.put(Headers.Sampled, samplingDecision) - } - } - - carrier - } - - override def decode(carrier: TextMap, context: Context): Context = { - val traceID = carrier.get(Headers.TraceIdentifier) - .map(id => identityProvider.traceIdentifierGenerator().from(urlDecode(id))) - .getOrElse(IdentityProvider.NoIdentifier) - - val spanID = carrier.get(Headers.SpanIdentifier) - .map(id => identityProvider.spanIdentifierGenerator().from(urlDecode(id))) - .getOrElse(IdentityProvider.NoIdentifier) - - if(traceID != IdentityProvider.NoIdentifier && spanID != IdentityProvider.NoIdentifier) { - val parentID = carrier.get(Headers.ParentSpanIdentifier) - .map(id => identityProvider.spanIdentifierGenerator().from(urlDecode(id))) - .getOrElse(IdentityProvider.NoIdentifier) - - val flags = carrier.get(Headers.Flags) - - val samplingDecision = flags.orElse(carrier.get(Headers.Sampled)) match { - case Some(sampled) if sampled == "1" => SamplingDecision.Sample - case Some(sampled) if sampled == "0" => SamplingDecision.DoNotSample - case _ => SamplingDecision.Unknown - } - - context.withKey(Span.ContextKey, Span.Remote(SpanContext(traceID, spanID, parentID, samplingDecision))) - - } else context - } - - private def encodeSamplingDecision(samplingDecision: SamplingDecision): Option[String] = samplingDecision match { - case SamplingDecision.Sample => Some("1") - case SamplingDecision.DoNotSample => Some("0") - case SamplingDecision.Unknown => None - } - - private def urlEncode(s: String): String = URLEncoder.encode(s, "UTF-8") - private def urlDecode(s: String): String = URLDecoder.decode(s, "UTF-8") - } - - object ExtendedB3 { - - def apply(identityProvider: IdentityProvider): ExtendedB3 = - new ExtendedB3(identityProvider) - - object Headers { - val TraceIdentifier = "X-B3-TraceId" - val ParentSpanIdentifier = "X-B3-ParentSpanId" - val SpanIdentifier = "X-B3-SpanId" - val Sampled = "X-B3-Sampled" - val Flags = "X-B3-Flags" - } - } -} \ No newline at end of file diff --git a/kamon-core/src/main/scala/kamon/trace/Tracer.scala b/kamon-core/src/main/scala/kamon/trace/Tracer.scala index 65307b95..7d8830ca 100644 --- a/kamon-core/src/main/scala/kamon/trace/Tracer.scala +++ b/kamon-core/src/main/scala/kamon/trace/Tracer.scala @@ -29,6 +29,7 @@ import scala.util.Try trait Tracer { def buildSpan(operationName: String): SpanBuilder + def identityProvider: IdentityProvider } object Tracer { @@ -39,13 +40,16 @@ object Tracer { private[Tracer] val tracerMetrics = new TracerMetrics(metrics) @volatile private[Tracer] var joinRemoteParentsWithSameSpanID: Boolean = true @volatile private[Tracer] var configuredSampler: Sampler = Sampler.Never - @volatile private[Tracer] var identityProvider: IdentityProvider = IdentityProvider.Default() + @volatile private[Tracer] var _identityProvider: IdentityProvider = IdentityProvider.Default() reconfigure(initialConfig) override def buildSpan(operationName: String): SpanBuilder = new SpanBuilder(operationName, this, reporterRegistry) + override def identityProvider: IdentityProvider = + this._identityProvider + def sampler: Sampler = configuredSampler @@ -69,7 +73,7 @@ object Tracer { configuredSampler = newSampler joinRemoteParentsWithSameSpanID = newJoinRemoteParentsWithSameSpanID - identityProvider = newIdentityProvider + _identityProvider = newIdentityProvider }.failed.foreach { ex => logger.error("Unable to reconfigure Kamon Tracer", ex) @@ -150,12 +154,12 @@ object Tracer { if(parent.isRemote() && tracer.joinRemoteParentsWithSameSpanID) parent.context().copy(samplingDecision = samplingDecision) else - parent.context().createChild(tracer.identityProvider.spanIdentifierGenerator().generate(), samplingDecision) + parent.context().createChild(tracer._identityProvider.spanIdGenerator().generate(), samplingDecision) private def newSpanContext(samplingDecision: SamplingDecision): SpanContext = SpanContext( - traceID = tracer.identityProvider.traceIdentifierGenerator().generate(), - spanID = tracer.identityProvider.spanIdentifierGenerator().generate(), + traceID = tracer._identityProvider.traceIdGenerator().generate(), + spanID = tracer._identityProvider.spanIdGenerator().generate(), parentID = IdentityProvider.NoIdentifier, samplingDecision = samplingDecision ) diff --git a/kamon-core/src/test/scala/kamon/LogInterceptor.scala b/kamon-core/src/test/scala/kamon/LogInterceptor.scala deleted file mode 100644 index 76480a2f..00000000 --- a/kamon-core/src/test/scala/kamon/LogInterceptor.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* ========================================================================================= - * Copyright © 2013-2017 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 uk.org.lidalia.slf4jext.Level -//import uk.org.lidalia.slf4jtest.{LoggingEvent, TestLogger} -// -//trait LogInterceptor { -// -// def interceptLog[T](level: Level)(code: => T)(implicit tl: TestLogger): Seq[LoggingEvent] = { -// import scala.collection.JavaConverters._ -// tl.clear() -// val run = code -// tl.getLoggingEvents().asScala.filter(_.getLevel == level) -// } -//} diff --git a/kamon-core/src/test/scala/kamon/context/ContextCodecSpec.scala b/kamon-core/src/test/scala/kamon/context/ContextCodecSpec.scala index 242c3345..11be85a7 100644 --- a/kamon-core/src/test/scala/kamon/context/ContextCodecSpec.scala +++ b/kamon-core/src/test/scala/kamon/context/ContextCodecSpec.scala @@ -14,5 +14,5 @@ class ContextCodecSpec extends WordSpec with Matchers { } } - val ContextCodec = new Codec(Kamon.config()) + val ContextCodec = new Codec(Kamon.identityProvider, Kamon.config()) } diff --git a/kamon-core/src/test/scala/kamon/testkit/SpanBuilding.scala b/kamon-core/src/test/scala/kamon/testkit/SpanBuilding.scala index 29678b34..7a216ecc 100644 --- a/kamon-core/src/test/scala/kamon/testkit/SpanBuilding.scala +++ b/kamon-core/src/test/scala/kamon/testkit/SpanBuilding.scala @@ -1,18 +1,16 @@ package kamon.testkit import kamon.trace.SpanContext.SamplingDecision -import kamon.trace.{IdentityProvider, SpanContext, SpanContextCodec} +import kamon.trace.{IdentityProvider, SpanContext} trait SpanBuilding { private val identityProvider = IdentityProvider.Default() - private val extendedB3Codec = SpanContextCodec.ExtendedB3(identityProvider) def createSpanContext(samplingDecision: SamplingDecision = SamplingDecision.Sample): SpanContext = SpanContext( - traceID = identityProvider.traceIdentifierGenerator().generate(), - spanID = identityProvider.spanIdentifierGenerator().generate(), - parentID = identityProvider.spanIdentifierGenerator().generate(), + traceID = identityProvider.traceIdGenerator().generate(), + spanID = identityProvider.spanIdGenerator().generate(), + parentID = identityProvider.spanIdGenerator().generate(), samplingDecision = samplingDecision ) - } diff --git a/kamon-core/src/test/scala/kamon/trace/B3SpanCodecSpec.scala b/kamon-core/src/test/scala/kamon/trace/B3SpanCodecSpec.scala new file mode 100644 index 00000000..e6fa283e --- /dev/null +++ b/kamon-core/src/test/scala/kamon/trace/B3SpanCodecSpec.scala @@ -0,0 +1,192 @@ +/* + * ========================================================================================= + * Copyright © 2013-2017 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.trace + +import kamon.context.{Context, TextMap} +import kamon.testkit.SpanBuilding +import kamon.trace.IdentityProvider.Identifier +import kamon.trace.SpanContext.SamplingDecision +import org.scalatest.{Matchers, OptionValues, WordSpecLike} + + +class B3SpanCodecSpec extends WordSpecLike with Matchers with OptionValues with SpanBuilding { + val extendedB3Codec = SpanCodec.B3() + + "The ExtendedB3 SpanContextCodec" should { + "return a TextMap containing the SpanContext data" in { + val context = testContext() + + val textMap = extendedB3Codec.encode(context) + textMap.get("X-B3-TraceId").value shouldBe "1234" + textMap.get("X-B3-ParentSpanId").value shouldBe "2222" + textMap.get("X-B3-SpanId").value shouldBe "4321" + textMap.get("X-B3-Sampled").value shouldBe "1" + } + + + "not inject anything if there is no Span in the Context" in { + val textMap = extendedB3Codec.encode(Context.Empty) + textMap.values shouldBe empty + } + + "extract a RemoteSpan from a TextMap when all fields are set" in { + val textMap = TextMap.Default() + textMap.put("X-B3-TraceId", "1234") + textMap.put("X-B3-ParentSpanId", "2222") + textMap.put("X-B3-SpanId", "4321") + textMap.put("X-B3-Sampled", "1") + textMap.put("X-B3-Extra-Baggage", "some=baggage;more=baggage") + + val spanContext = extendedB3Codec.decode(textMap, Context.Empty).get(Span.ContextKey).context() + spanContext.traceID.string shouldBe "1234" + spanContext.spanID.string shouldBe "4321" + spanContext.parentID.string shouldBe "2222" + spanContext.samplingDecision shouldBe SamplingDecision.Sample + } + + "decode the sampling decision based on the X-B3-Sampled header" in { + val sampledTextMap = TextMap.Default() + sampledTextMap.put("X-B3-TraceId", "1234") + sampledTextMap.put("X-B3-SpanId", "4321") + sampledTextMap.put("X-B3-Sampled", "1") + + val notSampledTextMap = TextMap.Default() + notSampledTextMap.put("X-B3-TraceId", "1234") + notSampledTextMap.put("X-B3-SpanId", "4321") + notSampledTextMap.put("X-B3-Sampled", "0") + + val noSamplingTextMap = TextMap.Default() + noSamplingTextMap.put("X-B3-TraceId", "1234") + noSamplingTextMap.put("X-B3-SpanId", "4321") + + extendedB3Codec.decode(sampledTextMap, Context.Empty) + .get(Span.ContextKey).context().samplingDecision shouldBe SamplingDecision.Sample + + extendedB3Codec.decode(notSampledTextMap, Context.Empty) + .get(Span.ContextKey).context().samplingDecision shouldBe SamplingDecision.DoNotSample + + extendedB3Codec.decode(noSamplingTextMap, Context.Empty) + .get(Span.ContextKey).context().samplingDecision shouldBe SamplingDecision.Unknown + } + + "not include the X-B3-Sampled header if the sampling decision is unknown" in { + val context = testContext() + val sampledSpanContext = context.get(Span.ContextKey).context() + val notSampledSpanContext = Context.Empty.withKey(Span.ContextKey, + Span.Remote(sampledSpanContext.copy(samplingDecision = SamplingDecision.DoNotSample))) + val unknownSamplingSpanContext = Context.Empty.withKey(Span.ContextKey, + Span.Remote(sampledSpanContext.copy(samplingDecision = SamplingDecision.Unknown))) + + extendedB3Codec.encode(context).get("X-B3-Sampled").value shouldBe("1") + extendedB3Codec.encode(notSampledSpanContext).get("X-B3-Sampled").value shouldBe("0") + extendedB3Codec.encode(unknownSamplingSpanContext).get("X-B3-Sampled") shouldBe empty + } + + "use the Debug flag to override the sampling decision, if provided." in { + val textMap = TextMap.Default() + textMap.put("X-B3-TraceId", "1234") + textMap.put("X-B3-SpanId", "4321") + textMap.put("X-B3-Sampled", "0") + textMap.put("X-B3-Flags", "1") + + val spanContext = extendedB3Codec.decode(textMap, Context.Empty).get(Span.ContextKey).context() + spanContext.samplingDecision shouldBe SamplingDecision.Sample + } + + "use the Debug flag as sampling decision when Sampled is not provided" in { + val textMap = TextMap.Default() + textMap.put("X-B3-TraceId", "1234") + textMap.put("X-B3-SpanId", "4321") + textMap.put("X-B3-Flags", "1") + + val spanContext = extendedB3Codec.decode(textMap, Context.Empty).get(Span.ContextKey).context() + spanContext.samplingDecision shouldBe SamplingDecision.Sample + } + + "extract a minimal SpanContext from a TextMap containing only the Trace ID and Span ID" in { + val textMap = TextMap.Default() + textMap.put("X-B3-TraceId", "1234") + textMap.put("X-B3-SpanId", "4321") + + val spanContext = extendedB3Codec.decode(textMap, Context.Empty).get(Span.ContextKey).context() + spanContext.traceID.string shouldBe "1234" + spanContext.spanID.string shouldBe "4321" + spanContext.parentID shouldBe IdentityProvider.NoIdentifier + spanContext.samplingDecision shouldBe SamplingDecision.Unknown + } + + "do not extract a SpanContext if Trace ID and Span ID are not provided" in { + val onlyTraceID = TextMap.Default() + onlyTraceID.put("X-B3-TraceId", "1234") + onlyTraceID.put("X-B3-Sampled", "0") + onlyTraceID.put("X-B3-Flags", "1") + + val onlySpanID = TextMap.Default() + onlySpanID.put("X-B3-SpanId", "4321") + onlySpanID.put("X-B3-Sampled", "0") + onlySpanID.put("X-B3-Flags", "1") + + val noIds = TextMap.Default() + noIds.put("X-B3-Sampled", "0") + noIds.put("X-B3-Flags", "1") + + extendedB3Codec.decode(onlyTraceID, Context.Empty).get(Span.ContextKey) shouldBe Span.Empty + extendedB3Codec.decode(onlySpanID, Context.Empty).get(Span.ContextKey) shouldBe Span.Empty + extendedB3Codec.decode(noIds, Context.Empty).get(Span.ContextKey) shouldBe Span.Empty + } + + "round trip a Span from TextMap -> Context -> TextMap" in { + val textMap = TextMap.Default() + textMap.put("X-B3-TraceId", "1234") + textMap.put("X-B3-ParentSpanId", "2222") + textMap.put("X-B3-SpanId", "4321") + textMap.put("X-B3-Sampled", "1") + + val context = extendedB3Codec.decode(textMap, Context.Empty) + val injectTextMap = extendedB3Codec.encode(context) + + textMap.values.toSeq should contain theSameElementsAs(injectTextMap.values.toSeq) + } + + /* + // TODO: Should we be supporting this use case? maybe even have the concept of Debug requests ourselves? + "internally carry the X-B3-Flags value so that it can be injected in outgoing requests" in { + val textMap = TextMap.Default() + textMap.put("X-B3-TraceId", "1234") + textMap.put("X-B3-ParentSpanId", "2222") + textMap.put("X-B3-SpanId", "4321") + textMap.put("X-B3-Sampled", "1") + textMap.put("X-B3-Flags", "1") + + val spanContext = extendedB3Codec.extract(textMap).value + val injectTextMap = extendedB3Codec.inject(spanContext) + + injectTextMap.get("X-B3-Flags").value shouldBe("1") + }*/ + } + + def testContext(): Context = { + val spanContext = createSpanContext().copy( + traceID = Identifier("1234", Array[Byte](1, 2, 3, 4)), + spanID = Identifier("4321", Array[Byte](4, 3, 2, 1)), + parentID = Identifier("2222", Array[Byte](2, 2, 2, 2)) + ) + + Context.create().withKey(Span.ContextKey, Span.Remote(spanContext)) + } + +} \ No newline at end of file diff --git a/kamon-core/src/test/scala/kamon/trace/DefaultIdentityGeneratorSpec.scala b/kamon-core/src/test/scala/kamon/trace/DefaultIdentityGeneratorSpec.scala index 8977e3cd..8f9af7b0 100644 --- a/kamon-core/src/test/scala/kamon/trace/DefaultIdentityGeneratorSpec.scala +++ b/kamon-core/src/test/scala/kamon/trace/DefaultIdentityGeneratorSpec.scala @@ -6,8 +6,8 @@ import org.scalactic.TimesOnInt._ class DefaultIdentityGeneratorSpec extends WordSpecLike with Matchers with OptionValues { val idProvider = IdentityProvider.Default() - val traceGenerator = idProvider.traceIdentifierGenerator() - val spanGenerator = idProvider.spanIdentifierGenerator() + val traceGenerator = idProvider.traceIdGenerator() + val spanGenerator = idProvider.spanIdGenerator() validateGenerator("TraceID Generator", traceGenerator) validateGenerator("SpanID Generator", spanGenerator) diff --git a/kamon-core/src/test/scala/kamon/trace/DoubleLengthTraceIdentityGeneratorSpec.scala b/kamon-core/src/test/scala/kamon/trace/DoubleLengthTraceIdentityGeneratorSpec.scala index 54e590ad..b22f17e1 100644 --- a/kamon-core/src/test/scala/kamon/trace/DoubleLengthTraceIdentityGeneratorSpec.scala +++ b/kamon-core/src/test/scala/kamon/trace/DoubleLengthTraceIdentityGeneratorSpec.scala @@ -6,8 +6,8 @@ import org.scalatest.{Matchers, OptionValues, WordSpecLike} class DoubleLengthTraceIdentityGeneratorSpec extends WordSpecLike with Matchers with OptionValues { val idProvider = IdentityProvider.DoubleSizeTraceID() - val traceGenerator = idProvider.traceIdentifierGenerator() - val spanGenerator = idProvider.spanIdentifierGenerator() + val traceGenerator = idProvider.traceIdGenerator() + val spanGenerator = idProvider.spanIdGenerator() "The DoubleSizeTraceID identity provider" when { "generating trace identifiers" should { diff --git a/kamon-core/src/test/scala/kamon/trace/ExtendedB3SpanContextCodecSpec.scala b/kamon-core/src/test/scala/kamon/trace/ExtendedB3SpanContextCodecSpec.scala deleted file mode 100644 index cc886bd9..00000000 --- a/kamon-core/src/test/scala/kamon/trace/ExtendedB3SpanContextCodecSpec.scala +++ /dev/null @@ -1,193 +0,0 @@ -/* - * ========================================================================================= - * Copyright © 2013-2017 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.trace - -import kamon.context.{Context, TextMap} -import kamon.testkit.SpanBuilding -import kamon.trace.IdentityProvider.Identifier -import kamon.trace.SpanContext.SamplingDecision -import org.scalatest.{Matchers, OptionValues, WordSpecLike} - - -class ExtendedB3SpanContextCodecSpec extends WordSpecLike with Matchers with OptionValues with SpanBuilding { - val identityProvider = IdentityProvider.Default() - val extendedB3Codec = SpanContextCodec.ExtendedB3(identityProvider) - - "The ExtendedB3 SpanContextCodec" should { - "return a TextMap containing the SpanContext data" in { - val context = testContext() - - val textMap = extendedB3Codec.encode(context) - textMap.get("X-B3-TraceId").value shouldBe "1234" - textMap.get("X-B3-ParentSpanId").value shouldBe "2222" - textMap.get("X-B3-SpanId").value shouldBe "4321" - textMap.get("X-B3-Sampled").value shouldBe "1" - } - - - "not inject anything if there is no Span in the Context" in { - val textMap = extendedB3Codec.encode(Context.Empty) - textMap.values shouldBe empty - } - - "extract a RemoteSpan from a TextMap when all fields are set" in { - val textMap = TextMap.Default() - textMap.put("X-B3-TraceId", "1234") - textMap.put("X-B3-ParentSpanId", "2222") - textMap.put("X-B3-SpanId", "4321") - textMap.put("X-B3-Sampled", "1") - textMap.put("X-B3-Extra-Baggage", "some=baggage;more=baggage") - - val spanContext = extendedB3Codec.decode(textMap, Context.Empty).get(Span.ContextKey).context() - spanContext.traceID.string shouldBe "1234" - spanContext.spanID.string shouldBe "4321" - spanContext.parentID.string shouldBe "2222" - spanContext.samplingDecision shouldBe SamplingDecision.Sample - } - - "decode the sampling decision based on the X-B3-Sampled header" in { - val sampledTextMap = TextMap.Default() - sampledTextMap.put("X-B3-TraceId", "1234") - sampledTextMap.put("X-B3-SpanId", "4321") - sampledTextMap.put("X-B3-Sampled", "1") - - val notSampledTextMap = TextMap.Default() - notSampledTextMap.put("X-B3-TraceId", "1234") - notSampledTextMap.put("X-B3-SpanId", "4321") - notSampledTextMap.put("X-B3-Sampled", "0") - - val noSamplingTextMap = TextMap.Default() - noSamplingTextMap.put("X-B3-TraceId", "1234") - noSamplingTextMap.put("X-B3-SpanId", "4321") - - extendedB3Codec.decode(sampledTextMap, Context.Empty) - .get(Span.ContextKey).context().samplingDecision shouldBe SamplingDecision.Sample - - extendedB3Codec.decode(notSampledTextMap, Context.Empty) - .get(Span.ContextKey).context().samplingDecision shouldBe SamplingDecision.DoNotSample - - extendedB3Codec.decode(noSamplingTextMap, Context.Empty) - .get(Span.ContextKey).context().samplingDecision shouldBe SamplingDecision.Unknown - } - - "not include the X-B3-Sampled header if the sampling decision is unknown" in { - val context = testContext() - val sampledSpanContext = context.get(Span.ContextKey).context() - val notSampledSpanContext = Context.Empty.withKey(Span.ContextKey, - Span.Remote(sampledSpanContext.copy(samplingDecision = SamplingDecision.DoNotSample))) - val unknownSamplingSpanContext = Context.Empty.withKey(Span.ContextKey, - Span.Remote(sampledSpanContext.copy(samplingDecision = SamplingDecision.Unknown))) - - extendedB3Codec.encode(context).get("X-B3-Sampled").value shouldBe("1") - extendedB3Codec.encode(notSampledSpanContext).get("X-B3-Sampled").value shouldBe("0") - extendedB3Codec.encode(unknownSamplingSpanContext).get("X-B3-Sampled") shouldBe empty - } - - "use the Debug flag to override the sampling decision, if provided." in { - val textMap = TextMap.Default() - textMap.put("X-B3-TraceId", "1234") - textMap.put("X-B3-SpanId", "4321") - textMap.put("X-B3-Sampled", "0") - textMap.put("X-B3-Flags", "1") - - val spanContext = extendedB3Codec.decode(textMap, Context.Empty).get(Span.ContextKey).context() - spanContext.samplingDecision shouldBe SamplingDecision.Sample - } - - "use the Debug flag as sampling decision when Sampled is not provided" in { - val textMap = TextMap.Default() - textMap.put("X-B3-TraceId", "1234") - textMap.put("X-B3-SpanId", "4321") - textMap.put("X-B3-Flags", "1") - - val spanContext = extendedB3Codec.decode(textMap, Context.Empty).get(Span.ContextKey).context() - spanContext.samplingDecision shouldBe SamplingDecision.Sample - } - - "extract a minimal SpanContext from a TextMap containing only the Trace ID and Span ID" in { - val textMap = TextMap.Default() - textMap.put("X-B3-TraceId", "1234") - textMap.put("X-B3-SpanId", "4321") - - val spanContext = extendedB3Codec.decode(textMap, Context.Empty).get(Span.ContextKey).context() - spanContext.traceID.string shouldBe "1234" - spanContext.spanID.string shouldBe "4321" - spanContext.parentID shouldBe IdentityProvider.NoIdentifier - spanContext.samplingDecision shouldBe SamplingDecision.Unknown - } - - "do not extract a SpanContext if Trace ID and Span ID are not provided" in { - val onlyTraceID = TextMap.Default() - onlyTraceID.put("X-B3-TraceId", "1234") - onlyTraceID.put("X-B3-Sampled", "0") - onlyTraceID.put("X-B3-Flags", "1") - - val onlySpanID = TextMap.Default() - onlySpanID.put("X-B3-SpanId", "4321") - onlySpanID.put("X-B3-Sampled", "0") - onlySpanID.put("X-B3-Flags", "1") - - val noIds = TextMap.Default() - noIds.put("X-B3-Sampled", "0") - noIds.put("X-B3-Flags", "1") - - extendedB3Codec.decode(onlyTraceID, Context.Empty).get(Span.ContextKey) shouldBe Span.Empty - extendedB3Codec.decode(onlySpanID, Context.Empty).get(Span.ContextKey) shouldBe Span.Empty - extendedB3Codec.decode(noIds, Context.Empty).get(Span.ContextKey) shouldBe Span.Empty - } - - "round trip a Span from TextMap -> Context -> TextMap" in { - val textMap = TextMap.Default() - textMap.put("X-B3-TraceId", "1234") - textMap.put("X-B3-ParentSpanId", "2222") - textMap.put("X-B3-SpanId", "4321") - textMap.put("X-B3-Sampled", "1") - - val context = extendedB3Codec.decode(textMap, Context.Empty) - val injectTextMap = extendedB3Codec.encode(context) - - textMap.values.toSeq should contain theSameElementsAs(injectTextMap.values.toSeq) - } - - /* - // TODO: Should we be supporting this use case? maybe even have the concept of Debug requests ourselves? - "internally carry the X-B3-Flags value so that it can be injected in outgoing requests" in { - val textMap = TextMap.Default() - textMap.put("X-B3-TraceId", "1234") - textMap.put("X-B3-ParentSpanId", "2222") - textMap.put("X-B3-SpanId", "4321") - textMap.put("X-B3-Sampled", "1") - textMap.put("X-B3-Flags", "1") - - val spanContext = extendedB3Codec.extract(textMap).value - val injectTextMap = extendedB3Codec.inject(spanContext) - - injectTextMap.get("X-B3-Flags").value shouldBe("1") - }*/ - } - - def testContext(): Context = { - val spanContext = createSpanContext().copy( - traceID = Identifier("1234", Array[Byte](1, 2, 3, 4)), - spanID = Identifier("4321", Array[Byte](4, 3, 2, 1)), - parentID = Identifier("2222", Array[Byte](2, 2, 2, 2)) - ) - - Context.create().withKey(Span.ContextKey, Span.Remote(spanContext)) - } - -} \ No newline at end of file -- cgit v1.2.3