Skip to content

Commit

Permalink
Merge pull request #403 from iRevive/sdk-trace/sdk-span-builder
Browse files Browse the repository at this point in the history
sdk-trace: add `SdkSpanBuilder`
  • Loading branch information
iRevive authored Dec 27, 2023
2 parents 1411b0a + 6c3879b commit 04fd83a
Show file tree
Hide file tree
Showing 8 changed files with 468 additions and 8 deletions.
24 changes: 22 additions & 2 deletions core/trace/src/main/scala/org/typelevel/otel4s/trace/Span.scala
Original file line number Diff line number Diff line change
Expand Up @@ -159,12 +159,32 @@ object Span {
}

object Backend {

/** Returns a non-recording backend that holds the provided [[SpanContext]]
* but all operations have no effect. The span will not be exported and all
* tracing operations are no-op, but it can be used to propagate a valid
* [[SpanContext]] downstream.
*
* @param context
* the context to propagate
*/
private[otel4s] def propagating[F[_]: Applicative](
context: SpanContext
): Backend[F] =
make(InstrumentMeta.enabled, context)

def noop[F[_]: Applicative]: Backend[F] =
make(InstrumentMeta.disabled, SpanContext.invalid)

private def make[F[_]: Applicative](
m: InstrumentMeta[F],
ctx: SpanContext
): Backend[F] =
new Backend[F] {
private val unit = Applicative[F].unit

val meta: InstrumentMeta[F] = InstrumentMeta.disabled
val context: SpanContext = SpanContext.invalid
val meta: InstrumentMeta[F] = m
val context: SpanContext = ctx

def updateName(name: String): F[Unit] = unit

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,239 @@
/*
* Copyright 2023 Typelevel
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.typelevel.otel4s
package sdk
package trace

import cats.arrow.FunctionK
import cats.effect.Resource
import cats.effect.Temporal
import cats.effect.std.Console
import cats.syntax.flatMap._
import cats.syntax.foldable._
import cats.syntax.functor._
import cats.syntax.semigroup._
import cats.~>
import org.typelevel.otel4s.sdk.common.InstrumentationScope
import org.typelevel.otel4s.sdk.trace.data.LinkData
import org.typelevel.otel4s.sdk.trace.samplers.SamplingResult
import org.typelevel.otel4s.trace.Span
import org.typelevel.otel4s.trace.SpanBuilder
import org.typelevel.otel4s.trace.SpanContext
import org.typelevel.otel4s.trace.SpanFinalizer
import org.typelevel.otel4s.trace.SpanKind
import org.typelevel.otel4s.trace.SpanOps
import org.typelevel.otel4s.trace.TraceFlags
import org.typelevel.otel4s.trace.TraceState
import scodec.bits.ByteVector

import scala.concurrent.duration.FiniteDuration

private[trace] final case class SdkSpanBuilder[F[_]: Temporal: Console](
name: String,
scopeInfo: InstrumentationScope,
tracerSharedState: TracerSharedState[F],
scope: SdkTraceScope[F],
parent: SdkSpanBuilder.Parent = SdkSpanBuilder.Parent.Propagate,
finalizationStrategy: SpanFinalizer.Strategy =
SpanFinalizer.Strategy.reportAbnormal,
kind: Option[SpanKind] = None,
links: Vector[LinkData] = Vector.empty,
attributes: Vector[Attribute[_]] = Vector.empty,
startTimestamp: Option[FiniteDuration] = None
) extends SpanBuilder[F] {
import SdkSpanBuilder._

def withSpanKind(spanKind: SpanKind): SpanBuilder[F] =
copy(kind = Some(spanKind))

def addAttribute[A](attribute: Attribute[A]): SpanBuilder[F] =
copy(attributes = attributes :+ attribute)

def addAttributes(attributes: Attribute[_]*): SpanBuilder[F] =
copy(attributes = this.attributes ++ attributes)

def addLink(
spanContext: SpanContext,
attributes: Attribute[_]*
): SpanBuilder[F] =
copy(links =
links :+ LinkData(spanContext, Attributes.fromSpecific(attributes))
)

def root: SpanBuilder[F] =
copy(parent = Parent.Root)

def withParent(parent: SpanContext): SpanBuilder[F] =
copy(parent = Parent.Explicit(parent))

def withStartTimestamp(timestamp: FiniteDuration): SpanBuilder[F] =
copy(startTimestamp = Some(timestamp))

def withFinalizationStrategy(
strategy: SpanFinalizer.Strategy
): SpanBuilder[F] =
copy(finalizationStrategy = strategy)

def build: SpanOps[F] = new SpanOps[F] {
def startUnmanaged: F[Span[F]] =
start.map(backend => Span.fromBackend(backend))

def resource: Resource[F, SpanOps.Res[F]] =
startAsRes

def use[A](f: Span[F] => F[A]): F[A] =
resource.use(res => res.trace(f(res.span)))

def use_ : F[Unit] =
use(_ => Temporal[F].unit)
}

private def startAsRes: Resource[F, SpanOps.Res[F]] =
Resource.eval(scope.current).flatMap {
case spanContext if spanContext.forall(_.isValid) =>
startManaged.map { case (back, nt) =>
SpanOps.Res(Span.fromBackend(back), nt)
}

case _ =>
Resource.pure(
SpanOps.Res(Span.fromBackend(Span.Backend.noop), FunctionK.id)
)
}

private def startManaged: Resource[F, (Span.Backend[F], F ~> F)] = {
def acquire: F[Span.Backend[F]] =
start

def release(backend: Span.Backend[F], ec: Resource.ExitCase): F[Unit] =
for {
_ <- finalizationStrategy
.lift(ec)
.foldMapM(SpanFinalizer.run(backend, _))
_ <- backend.end
} yield ()

for {
backend <- Resource.makeCase(acquire) { case (b, ec) => release(b, ec) }
nt <- Resource.eval(scope.childScope(backend.context))
} yield (backend, nt)
}

private def start: F[Span.Backend[F]] = {
val idGenerator = tracerSharedState.idGenerator
val spanKind = kind.getOrElse(SpanKind.Internal)
val attrs = Attributes.fromSpecific(attributes)

def genTraceId(parent: Option[SpanContext]): F[ByteVector] =
parent
.filter(_.isValid)
.fold(idGenerator.generateTraceId)(ctx => Temporal[F].pure(ctx.traceId))

def sample(
parent: Option[SpanContext],
traceId: ByteVector
): SamplingResult =
tracerSharedState.sampler.shouldSample(
parentContext = parent,
traceId = traceId,
name = name,
spanKind = spanKind,
attributes = attrs,
parentLinks = links
)

for {
parentSpanContext <- chooseParentSpanContext
spanId <- idGenerator.generateSpanId
traceId <- genTraceId(parentSpanContext)

backend <- {
val samplingResult = sample(parentSpanContext, traceId)
val samplingDecision = samplingResult.decision

val traceFlags =
if (samplingDecision.isSampled) TraceFlags.Sampled
else TraceFlags.Default

val traceState =
parentSpanContext.fold(TraceState.empty) { ctx =>
samplingResult.traceStateUpdater.update(ctx.traceState)
}

val spanContext =
createSpanContext(traceId, spanId, traceFlags, traceState)

if (!samplingDecision.isRecording) {
Temporal[F].pure(Span.Backend.propagating(spanContext))
} else {
SdkSpanBackend
.start[F](
context = spanContext,
name = name,
scopeInfo = scopeInfo,
resource = tracerSharedState.resource,
kind = spanKind,
parentContext = parentSpanContext,
processor = tracerSharedState.spanProcessor,
attributes = attrs |+| samplingResult.attributes,
links = links,
userStartTimestamp = startTimestamp
)
.widen
}
}
} yield backend
}

private def chooseParentSpanContext: F[Option[SpanContext]] =
parent match {
case Parent.Root => Temporal[F].pure(None)
case Parent.Propagate => scope.current
case Parent.Explicit(parent) => Temporal[F].pure(Some(parent))
}

private def createSpanContext(
traceId: ByteVector,
spanId: ByteVector,
flags: TraceFlags,
state: TraceState
): SpanContext =
if (tracerSharedState.idGenerator.canSkipIdValidation) {
SpanContext.createInternal(
traceId,
spanId,
flags,
state,
remote = false,
isValid = true
)
} else {
SpanContext(traceId, spanId, flags, state, remote = false)
}

}

private[trace] object SdkSpanBuilder {

sealed trait Parent
object Parent {
case object Propagate extends Parent
case object Root extends Parent
final case class Explicit(parent: SpanContext) extends Parent
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* Copyright 2023 Typelevel
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.typelevel.otel4s.sdk
package trace

import org.typelevel.otel4s.sdk.trace.processor.SpanProcessor
import org.typelevel.otel4s.sdk.trace.samplers.Sampler

private final case class TracerSharedState[F[_]](
idGenerator: IdGenerator[F],
resource: Resource,
sampler: Sampler,
spanProcessor: SpanProcessor[F]
)
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ private[samplers] final class ParentBasedSampler private (
name: String,
spanKind: SpanKind,
attributes: Attributes,
parentLinks: List[LinkData]
parentLinks: Vector[LinkData]
): SamplingResult = {
val sampler = parentContext.filter(_.isValid) match {
case Some(ctx) if ctx.isRemote =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ trait Sampler {
name: String,
spanKind: SpanKind,
attributes: Attributes,
parentLinks: List[LinkData]
parentLinks: Vector[LinkData]
): SamplingResult

/** The description of the [[Sampler]]. This may be displayed on debug pages
Expand Down Expand Up @@ -142,7 +142,7 @@ object Sampler {
name: String,
spanKind: SpanKind,
attributes: Attributes,
parentLinks: List[LinkData]
parentLinks: Vector[LinkData]
): SamplingResult =
result
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ private[samplers] final class TraceIdRatioBasedSampler private (
name: String,
spanKind: SpanKind,
attributes: Attributes,
parentLinks: List[LinkData]
parentLinks: Vector[LinkData]
): SamplingResult =
if (math.abs(traceIdRandomPart(traceId)) < idUpperBound)
SamplingResult.RecordAndSample
Expand Down
Loading

0 comments on commit 04fd83a

Please sign in to comment.