Skip to content

Commit

Permalink
OpenTelemetry: add support for recording exceptions in spans (#695)
Browse files Browse the repository at this point in the history
* OpenTelemetry: add support for recording exceptions in spans

* OpenTelemetry: add support for recording exceptions in spans

* OpenTelemetry: add support for recording exceptions in spans

* OpenTelemetry: add support for recording exceptions in spans

* OpenTelemetry: add support for recording exceptions in spans

* OpenTelemetry: add support for recording exceptions in spans
  • Loading branch information
IvanFinochenko authored May 25, 2023
1 parent da67608 commit 510fd01
Show file tree
Hide file tree
Showing 4 changed files with 77 additions and 34 deletions.
Original file line number Diff line number Diff line change
@@ -1,14 +1,14 @@
package zio.telemetry.opentelemetry.example.http

import io.opentelemetry.api.trace.{ SpanKind, StatusCode }
import io.opentelemetry.api.trace.{SpanKind, StatusCode}
import zhttp.http._
import zio._
import zio.json.EncoderOps
import zio.telemetry.opentelemetry.baggage.Baggage
import zio.telemetry.opentelemetry.baggage.propagation.BaggagePropagator
import zio.telemetry.opentelemetry.context.OutgoingContextCarrier
import zio.telemetry.opentelemetry.tracing.propagation.TraceContextPropagator
import zio.telemetry.opentelemetry.tracing.{ ErrorMapper, Tracing }
import zio.telemetry.opentelemetry.tracing.{ErrorMapper, Tracing}

case class ProxyHttpApp(client: Client, tracing: Tracing, baggage: Baggage) {

Expand All @@ -19,7 +19,7 @@ case class ProxyHttpApp(client: Client, tracing: Tracing, baggage: Baggage) {

val routes: HttpApp[Any, Throwable] =
Http.collectZIO { case Method.GET -> _ / "statuses" =>
statuses @@ root("/statuses", SpanKind.SERVER, toErrorStatus = errorMapper)
statuses @@ root("/statuses", SpanKind.SERVER, errorMapper = errorMapper)
}

def statuses: Task[Response] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@ package zio.telemetry.opentelemetry.tracing

import io.opentelemetry.api.trace.StatusCode

case class ErrorMapper[E](body: PartialFunction[E, StatusCode])
case class ErrorMapper[E](body: PartialFunction[E, StatusCode], toThrowable: Option[E => Throwable] = None)

object ErrorMapper {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,8 +50,10 @@ trait Tracing { self =>
* name of the child span
* @param spanKind
* kind of the child span
* @param toErrorStatus
* @param errorMapper
* error mapper
* @param links
* spanContexts of the linked Spans.
* @param zio
* body of the child span
* @param trace
Expand All @@ -68,7 +70,7 @@ trait Tracing { self =>
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E] = ErrorMapper.default[E],
errorMapper: ErrorMapper[E] = ErrorMapper.default[E],
links: Seq[SpanContext] = Seq.empty
)(zio: => ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A]

Expand Down Expand Up @@ -110,8 +112,10 @@ trait Tracing { self =>
* name of the new root span
* @param spanKind
* name of the new root span
* @param toErrorStatus
* @param errorMapper
* error mapper
* @param links
* spanContexts of the linked Spans.
* @param zio
* body of the new root span
* @param trace
Expand All @@ -124,7 +128,7 @@ trait Tracing { self =>
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E] = ErrorMapper.default[E],
errorMapper: ErrorMapper[E] = ErrorMapper.default[E],
links: Seq[SpanContext] = Seq.empty
)(zio: => ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A]

Expand All @@ -137,8 +141,10 @@ trait Tracing { self =>
* name of the child span
* @param spanKind
* kind of the child span
* @param toErrorStatus
* @param errorMapper
* error mapper
* @param links
* spanContexts of the linked Spans.
* @param zio
* body of the child span
* @param trace
Expand All @@ -151,7 +157,7 @@ trait Tracing { self =>
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E] = ErrorMapper.default[E],
errorMapper: ErrorMapper[E] = ErrorMapper.default[E],
links: Seq[SpanContext] = Seq.empty
)(zio: => ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A]

Expand Down Expand Up @@ -266,8 +272,10 @@ trait Tracing { self =>
* name of the child span
* @param spanKind
* kind of the child span
* @param toErrorStatus
* @param errorMapper
* error mapper
* @param links
* spanContexts of the linked Spans.
* @param zio
* body of the child span
* @param trace
Expand All @@ -281,7 +289,7 @@ trait Tracing { self =>
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E] = ErrorMapper.default[E],
errorMapper: ErrorMapper[E] = ErrorMapper.default[E],
links: Seq[SpanContext] = Seq.empty
)(zio: => ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A]

Expand Down Expand Up @@ -428,49 +436,49 @@ trait Tracing { self =>
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E1] = ErrorMapper.default[E1],
errorMapper: ErrorMapper[E1] = ErrorMapper.default[E1],
links: Seq[SpanContext] = Seq.empty
): ZIOAspect[Nothing, Any, E1, E1, Nothing, Any] =
new ZIOAspect[Nothing, Any, E1, E1, Nothing, Any] {
override def apply[R, E >: E1 <: E1, A](zio: ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A] =
self.extractSpan(propagator, carrier, spanName, spanKind, attributes, toErrorStatus, links)(zio)
self.extractSpan(propagator, carrier, spanName, spanKind, attributes, errorMapper, links)(zio)
}

def root[E1](
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E1] = ErrorMapper.default[E1],
errorMapper: ErrorMapper[E1] = ErrorMapper.default[E1],
links: Seq[SpanContext] = Seq.empty
): ZIOAspect[Nothing, Any, E1, E1, Nothing, Any] =
new ZIOAspect[Nothing, Any, E1, E1, Nothing, Any] {
override def apply[R, E >: E1 <: E1, A](zio: ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A] =
self.root(spanName, spanKind, attributes, toErrorStatus, links)(zio)
self.root(spanName, spanKind, attributes, errorMapper, links)(zio)
}

def span[E1](
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E1] = ErrorMapper.default[E1],
errorMapper: ErrorMapper[E1] = ErrorMapper.default[E1],
links: Seq[SpanContext] = Seq.empty
): ZIOAspect[Nothing, Any, E1, E1, Nothing, Any] =
new ZIOAspect[Nothing, Any, E1, E1, Nothing, Any] {
override def apply[R, E >: E1 <: E1, A](zio: ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A] =
self.span(spanName, spanKind, attributes, toErrorStatus, links)(zio)
self.span(spanName, spanKind, attributes, errorMapper, links)(zio)
}

def inSpan[E1](
span: Span,
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E1] = ErrorMapper.default[E1],
errorMapper: ErrorMapper[E1] = ErrorMapper.default[E1],
links: Seq[SpanContext] = Seq.empty
): ZIOAspect[Nothing, Any, E1, E1, Nothing, Any] =
new ZIOAspect[Nothing, Any, E1, E1, Nothing, Any] {
override def apply[R, E >: E1 <: E1, A](zio: ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A] =
self.inSpan(span, spanName, spanKind, attributes, toErrorStatus, links)(zio)
self.inSpan(span, spanName, spanKind, attributes, errorMapper, links)(zio)
}

}
Expand Down Expand Up @@ -507,7 +515,7 @@ object Tracing {
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E] = ErrorMapper.default[E],
errorMapper: ErrorMapper[E] = ErrorMapper.default[E],
links: Seq[SpanContext] = Seq.empty
)(zio: => ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A] =
extractContext(propagator, carrier).flatMap { context =>
Expand All @@ -516,7 +524,7 @@ object Tracing {
} { case (endSpan, _) =>
endSpan
} { case (_, ctx) =>
finalizeSpanUsingEffect(zio, ctx, toErrorStatus)
finalizeSpanUsingEffect(zio, ctx, errorMapper)
}
}

Expand All @@ -540,22 +548,22 @@ object Tracing {
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E] = ErrorMapper.default[E],
errorMapper: ErrorMapper[E] = ErrorMapper.default[E],
links: Seq[SpanContext] = Seq.empty
)(zio: => ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A] =
ZIO.acquireReleaseWith {
createRoot(spanName, spanKind, attributes, links)
} { case (endSpan, _) =>
endSpan
} { case (_, ctx) =>
finalizeSpanUsingEffect(zio, ctx, toErrorStatus)
finalizeSpanUsingEffect(zio, ctx, errorMapper)
}

override def span[R, E, A](
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E] = ErrorMapper.default[E],
errorMapper: ErrorMapper[E] = ErrorMapper.default[E],
links: Seq[SpanContext] = Seq.empty
)(zio: => ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A] =
getCurrentContext.flatMap { old =>
Expand All @@ -564,7 +572,7 @@ object Tracing {
} { case (endSpan, _) =>
endSpan
} { case (_, ctx) =>
finalizeSpanUsingEffect(zio, ctx, toErrorStatus)
finalizeSpanUsingEffect(zio, ctx, errorMapper)
}
}

Expand Down Expand Up @@ -628,15 +636,15 @@ object Tracing {
spanName: String,
spanKind: SpanKind = SpanKind.INTERNAL,
attributes: Attributes = Attributes.empty(),
toErrorStatus: ErrorMapper[E] = ErrorMapper.default[E],
errorMapper: ErrorMapper[E] = ErrorMapper.default[E],
links: Seq[SpanContext] = Seq.empty
)(zio: => ZIO[R, E, A])(implicit trace: Trace): ZIO[R, E, A] =
ZIO.acquireReleaseWith {
createChild(Context.root().`with`(span), spanName, spanKind, attributes, links)
} { case (endSpan, _) =>
endSpan
} { case (_, ctx) =>
finalizeSpanUsingEffect(zio, ctx, toErrorStatus)
finalizeSpanUsingEffect(zio, ctx, errorMapper)
}

override def addEvent(name: String)(implicit trace: Trace): UIO[Span] =
Expand Down Expand Up @@ -704,14 +712,19 @@ object Tracing {
private def setErrorStatus[E](
span: Span,
cause: Cause[E],
toErrorStatus: ErrorMapper[E]
errorMapper: ErrorMapper[E]
)(implicit trace: Trace): UIO[Span] = {
val errorStatus =
cause.failureOption
.flatMap(toErrorStatus.body.lift)
.flatMap(errorMapper.body.lift)
.getOrElse(StatusCode.UNSET)

ZIO.succeed(span.setStatus(errorStatus, cause.prettyPrint))
for {
_ <- ZIO.succeed(span.setStatus(errorStatus, cause.prettyPrint))
result <- errorMapper.toThrowable.fold(ZIO.succeed(span))(toThrowable =>
ZIO.succeed(span.recordException(cause.squashWith(toThrowable)))
)
} yield result
}

/**
Expand All @@ -721,11 +734,11 @@ object Tracing {
private def finalizeSpanUsingEffect[R, E, A](
zio: ZIO[R, E, A],
ctx: Context,
toErrorStatus: ErrorMapper[E]
errorMapper: ErrorMapper[E]
)(implicit trace: Trace): ZIO[R, E, A] =
ctxStorage
.locally(ctx)(zio)
.tapErrorCause(setErrorStatus(Span.fromContext(ctx), _, toErrorStatus))
.tapErrorCause(setErrorStatus(Span.fromContext(ctx), _, errorMapper))

private def currentNanos(implicit trace: Trace): UIO[Long] =
Clock.currentTime(TimeUnit.NANOSECONDS)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
package zio.telemetry.opentelemetry.tracing

import io.opentelemetry.api.common.{AttributeKey, Attributes}
import io.opentelemetry.api.trace.{Span, SpanId, Tracer}
import io.opentelemetry.api.trace.{Span, SpanId, StatusCode, Tracer}
import io.opentelemetry.context.Context
import io.opentelemetry.sdk.testing.exporter.InMemorySpanExporter
import io.opentelemetry.sdk.trace.SdkTracerProvider
Expand Down Expand Up @@ -111,6 +111,36 @@ object TracingTest extends ZIOSpecDefault {
)
}
},
test("setError") {
ZIO.serviceWithZIO[Tracing] { tracing =>
import tracing.aspects._

val assertStatusCodeError =
hasField[SpanData, StatusCode]("statusCode", _.getStatus.getStatusCode, equalTo(StatusCode.ERROR))
val assertRecordedExceptionAttributes = hasField[SpanData, List[(String, String)]](
"exceptionAttributes",
_.getEvents.asScala.toList
.flatMap(_.getAttributes.asMap().asScala.toList.map(x => x._1.getKey -> x._2.toString)),
hasSubset(List("exception.message" -> "some_error", "exception.type" -> "java.lang.RuntimeException"))
)
val errorMapper = ErrorMapper[Throwable]({ case _ => StatusCode.ERROR }, Some(identity))

val alwaysTrue = true
val failedEffect: ZIO[Any, Throwable, Unit] =
ZIO.fail(new RuntimeException("some_error")).when(alwaysTrue).unit

for {
_ <- (failedEffect @@ span[Throwable]("Child", errorMapper = errorMapper) @@ span[Throwable](
"Root",
errorMapper = errorMapper
)).ignore
spans <- getFinishedSpans
root = spans.find(_.getName == "Root")
child = spans.find(_.getName == "Child")
} yield assert(root)(isSome(assertStatusCodeError && assertRecordedExceptionAttributes)) &&
assert(child)(isSome(assertStatusCodeError && assertRecordedExceptionAttributes))
}
},
test("root") {
ZIO.serviceWithZIO[Tracing] { tracing =>
import tracing.aspects._
Expand Down

0 comments on commit 510fd01

Please sign in to comment.