Skip to content

Commit

Permalink
Replace SpanBuilder#wrapResource API
Browse files Browse the repository at this point in the history
Replace `SpanBuilder#wrapResource` with
`SpanOps#resource:Resource[F,F~>F]`, which no longer automatically
provides 'acquire', 'use' and 'release' spans.

This API change allows further simplification of the API in future
commits.
  • Loading branch information
NthPortal committed Jul 7, 2023
1 parent bc52ad9 commit ec21382
Show file tree
Hide file tree
Showing 11 changed files with 90 additions and 173 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,8 @@
package org.typelevel.otel4s
package trace

import cats.effect.kernel.Resource
import cats.effect.Resource
import cats.~>

private[otel4s] trait TracerMacro[F[_]] {
self: Tracer[F] =>
Expand Down Expand Up @@ -101,10 +102,11 @@ private[otel4s] trait TracerMacro[F[_]] {
* @param attributes
* the set of attributes to associate with the span
*/
def resourceSpan[A](name: String, attributes: Attribute[_]*)(
resource: Resource[F, A]
): SpanOps.Aux[F, Span.Res[F, A]] =
macro TracerMacro.resourceSpan[F, A]
def spanResource(
name: String,
attributes: Attribute[_]*
): Resource[F, F ~> F] =
macro TracerMacro.spanResource
}

object TracerMacro {
Expand All @@ -128,14 +130,13 @@ object TracerMacro {
q"(if ($meta.isEnabled) ${c.prefix}.spanBuilder($name).root.addAttributes(..$attributes) else $meta.noopSpanBuilder).build"
}

def resourceSpan[F[_], A](c: blackbox.Context)(
def spanResource(c: blackbox.Context)(
name: c.Expr[String],
attributes: c.Expr[Attribute[_]]*
)(resource: c.Expr[Resource[F, A]]): c.universe.Tree = {
): c.universe.Tree = {
import c.universe._
val meta = q"${c.prefix}.meta"

q"if ($meta.isEnabled) ${c.prefix}.spanBuilder($name).addAttributes(..$attributes).wrapResource($resource).build else $meta.noopResSpan($resource).build"
q"if ($meta.isEnabled) ${c.prefix}.spanBuilder($name).addAttributes(..$attributes).build.resource else $meta.noopResSpan"
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.typelevel.otel4s
package trace

import cats.~>
import cats.effect.kernel.Resource

import scala.quoted.*
Expand Down Expand Up @@ -106,11 +107,11 @@ private[otel4s] trait TracerMacro[F[_]] {
* @param attributes
* the set of attributes to associate with the span
*/
inline def resourceSpan[A](
inline def spanResource(
inline name: String,
inline attributes: Attribute[_]*
)(inline resource: Resource[F, A]): SpanOps.Aux[F, Span.Res[F, A]] =
${ TracerMacro.resourceSpan('self, 'name, 'attributes, 'resource) }
): Resource[F, F ~> F] =
${ TracerMacro.spanResource('self, 'name, 'attributes) }

}

Expand Down Expand Up @@ -138,20 +139,19 @@ object TracerMacro {
else $tracer.meta.noopSpanBuilder.build
}

def resourceSpan[F[_], A](
def spanResource[F[_]](
tracer: Expr[Tracer[F]],
name: Expr[String],
attributes: Expr[Seq[Attribute[_]]],
resource: Expr[Resource[F, A]]
)(using Quotes, Type[F], Type[A]) =
attributes: Expr[Seq[Attribute[_]]]
)(using Quotes, Type[F]) =
'{
if ($tracer.meta.isEnabled)
$tracer
.spanBuilder($name)
.addAttributes($attributes*)
.wrapResource($resource)
.build
else $tracer.meta.noopResSpan($resource).build
.resource
else $tracer.meta.noopResSpan
}

}
29 changes: 0 additions & 29 deletions core/trace/src/main/scala/org/typelevel/otel4s/trace/Span.scala
Original file line number Diff line number Diff line change
Expand Up @@ -157,33 +157,4 @@ object Span {
new Span[F] {
def backend: Backend[F] = back
}

/** The allocation and release stages of a supplied resource are traced by
* separate spans. Carries a value of a wrapped resource.
*
* The structure of the inner spans:
* {{{
* > span-name
* > acquire
* > use
* > release
* }}}
*/
trait Res[F[_], A] extends Span[F] {
def value: A
}

object Res {
def unapply[F[_], A](span: Span.Res[F, A]): Option[A] =
Some(span.value)

private[otel4s] def fromBackend[F[_], A](
a: A,
back: Backend[F]
): Res[F, A] =
new Res[F, A] {
def value: A = a
def backend: Backend[F] = back
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,10 @@ package org.typelevel.otel4s
package trace

import cats.Applicative
import cats.effect.kernel.MonadCancelThrow
import cats.effect.kernel.Resource
import cats.arrow.FunctionK
import cats.effect.MonadCancelThrow
import cats.effect.Resource
import cats.~>

import scala.concurrent.duration.FiniteDuration

Expand Down Expand Up @@ -110,41 +112,6 @@ trait SpanBuilder[F[_]] {
*/
def withParent(parent: SpanContext): Builder

/** Wraps the given resource to trace it upon the start.
*
* The span is started upon resource allocation and ended upon finalization.
* The allocation and release stages of the `resource` are traced by separate
* spans. Carries a value of the given `resource`.
*
* The structure of the inner spans:
* {{{
* > span-name
* > acquire
* > use
* > release
* }}}
*
* The finalization strategy is determined by [[SpanFinalizer.Strategy]]. By
* default, the abnormal termination (error, cancelation) is recorded.
*
* @see
* default finalization strategy [[SpanFinalizer.Strategy.reportAbnormal]]
* @example
* {{{
* val tracer: Tracer[F] = ???
* val resource: Resource[F, String] = Resource.eval(Sync[F].delay("string"))
* val ok: F[Unit] =
* tracer.spanBuilder("wrapped-resource").wrapResource(resource).build.use { case span @ Span.Res(value) =>
* span.setStatus(Status.Ok, s"all good. resource value: $${value}")
* }
* }}}
* @param resource
* the resource to trace
*/
def wrapResource[A](
resource: Resource[F, A]
)(implicit ev: Result =:= Span[F]): SpanBuilder.Aux[F, Span.Res[F, A]]

def build: SpanOps.Aux[F, Result]
}

Expand All @@ -168,14 +135,6 @@ object SpanBuilder {

private val span: Span[F] = Span.fromBackend(back)

def wrapResource[A](
resource: Resource[F, A]
)(implicit ev: Result =:= Span[F]): SpanBuilder.Aux[F, Span.Res[F, A]] =
make(
back,
resource.map(r => Span.Res.fromBackend(r, back))
)

def addAttribute[A](attribute: Attribute[A]): Builder = this

def addAttributes(attributes: Attribute[_]*): Builder = this
Expand Down Expand Up @@ -207,6 +166,9 @@ object SpanBuilder {

def surround[A](fa: F[A]): F[A] =
fa

def resource: Resource[F, F ~> F] =
Resource.pure(FunctionK.id)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,9 @@

package org.typelevel.otel4s.trace

import cats.effect.Resource
import cats.~>

trait SpanOps[F[_]] {
type Result <: Span[F]

Expand Down Expand Up @@ -104,6 +107,8 @@ trait SpanOps[F[_]] {
* See [[use]] for more details regarding lifecycle strategy
*/
def surround[A](fa: F[A]): F[A]

def resource: Resource[F, F ~> F]
}

object SpanOps {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,9 @@ package org.typelevel.otel4s
package trace

import cats.Applicative
import cats.effect.kernel.MonadCancelThrow
import cats.effect.kernel.Resource
import cats.effect.MonadCancelThrow
import cats.effect.Resource
import cats.~>
import org.typelevel.otel4s.meta.InstrumentMeta

@annotation.implicitNotFound("""
Expand Down Expand Up @@ -180,10 +181,8 @@ object Tracer {

trait Meta[F[_]] extends InstrumentMeta[F] {
def noopSpanBuilder: SpanBuilder.Aux[F, Span[F]]
final def noopResSpan[A](
resource: Resource[F, A]
): SpanBuilder.Aux[F, Span.Res[F, A]] =
noopSpanBuilder.wrapResource(resource)
final def noopResSpan: Resource[F, F ~> F] =
noopSpanBuilder.build.resource
}

object Meta {
Expand Down
3 changes: 2 additions & 1 deletion examples/src/main/scala/TraceExample.scala
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,8 @@ object TraceExample extends IOApp.Simple {
val resource: Resource[IO, Unit] =
Resource.make(IO.sleep(50.millis))(_ => IO.sleep(100.millis))
tracer
.resourceSpan("Start up")(resource)
.spanResource("Start up")
.flatMap(resource.mapK(_))
.surround(
userIdAlg
.getAllUsersForInstitution(
Expand Down
3 changes: 2 additions & 1 deletion examples/src/main/scala/TracingExample.scala
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,8 @@ object TracingExample extends IOApp.Simple {
val resource: Resource[IO, Unit] =
Resource.make(IO.sleep(50.millis))(_ => IO.sleep(100.millis))
tracer
.resourceSpan("resource")(resource)
.spanResource("resource")
.flatMap(resource.mapK(_))
.surround(
Work[IO].request(
Map(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,11 +78,6 @@ private[java] final case class SpanBuilderImpl[F[_]: Sync, Res <: Span[F]](
def withFinalizationStrategy(strategy: SpanFinalizer.Strategy): Builder =
copy(finalizationStrategy = strategy)

def wrapResource[A](
resource: Resource[F, A]
)(implicit ev: Result =:= Span[F]): SpanBuilder.Aux[F, Span.Res[F, A]] =
copy(runner = SpanRunner.resource(scope, resource, jTracer))

def build: SpanOps.Aux[F, Result] = new SpanOps[F] {
type Result = Res

Expand All @@ -98,6 +93,8 @@ private[java] final case class SpanBuilderImpl[F[_]: Sync, Res <: Span[F]](
def surround[A](fa: F[A]): F[A] =
use(_ => fa)

def resource: Resource[F, F ~> F] = start.map(_._2)

private def start: Resource[F, (Result, F ~> F)] =
Resource.eval(runnerContext).flatMap(ctx => runner.start(ctx))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import cats.syntax.foldable._
import cats.syntax.functor._
import cats.~>
import io.opentelemetry.api.trace.{SpanBuilder => JSpanBuilder}
import io.opentelemetry.api.trace.{Tracer => JTracer}
import io.opentelemetry.context.{Context => JContext}
import org.typelevel.otel4s.trace.Span
import org.typelevel.otel4s.trace.SpanFinalizer
Expand Down Expand Up @@ -61,57 +60,6 @@ private[java] object SpanRunner {
}
}

def resource[F[_]: Sync, A](
scope: TraceScope[F],
resource: Resource[F, A],
jTracer: JTracer
): SpanRunner[F, Span.Res[F, A]] =
new SpanRunner[F, Span.Res[F, A]] {
def start(
ctx: Option[RunnerContext]
): Resource[F, (Span.Res[F, A], F ~> F)] =
ctx match {
case Some(RunnerContext(builder, parent, hasStartTimestamp, fin)) =>
def child(
name: String,
parent: JContext
): Resource[F, (SpanBackendImpl[F], F ~> F)] =
startManaged(
builder = jTracer.spanBuilder(name).setParent(parent),
hasStartTimestamp = false,
finalizationStrategy = fin,
scope = scope
)

for {
rootBackend <- startManaged(
builder = builder,
hasStartTimestamp = hasStartTimestamp,
finalizationStrategy = fin,
scope = scope
)

rootCtx <- Resource.pure(parent.`with`(rootBackend._1.jSpan))

pair <- Resource.make(
child("acquire", rootCtx).use(b => b._2(resource.allocated))
) { case (_, release) =>
child("release", rootCtx).use(b => b._2(release))
}
(value, _) = pair

pair2 <- child("use", rootCtx)
(useSpanBackend, nt) = pair2
} yield (Span.Res.fromBackend(value, useSpanBackend), nt)

case None =>
resource.map(a =>
(Span.Res.fromBackend(a, Span.Backend.noop), FunctionK.id)
)
}

}

def startUnmanaged[F[_]: Sync](context: Option[RunnerContext]): F[Span[F]] =
context match {
case Some(RunnerContext(builder, _, ts, _)) =>
Expand Down
Loading

0 comments on commit ec21382

Please sign in to comment.