Skip to content

Commit

Permalink
Add BaggageManager utility
Browse files Browse the repository at this point in the history
Add `BaggageManager` utility for accessing and modifying `Baggage`
in `Local` context.
  • Loading branch information
NthPortal committed Feb 3, 2025
1 parent 8af8ebb commit f1946fa
Show file tree
Hide file tree
Showing 17 changed files with 506 additions and 6 deletions.
2 changes: 1 addition & 1 deletion build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -632,7 +632,7 @@ lazy val `sdk-contrib-aws-xray` =
lazy val `oteljava-common` = project
.in(file("oteljava/common"))
.enablePlugins(BuildInfoPlugin)
.dependsOn(`core-common`.jvm)
.dependsOn(`core-common`.jvm % "compile->compile;test->test")
.settings(munitDependencies)
.settings(
name := "otel4s-oteljava-common",
Expand Down
4 changes: 4 additions & 0 deletions core/all/src/main/scala/org/typelevel/otel4s/Otel4s.scala
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@

package org.typelevel.otel4s

import org.typelevel.otel4s.baggage.BaggageManager
import org.typelevel.otel4s.context.propagation.ContextPropagators
import org.typelevel.otel4s.metrics.MeterProvider
import org.typelevel.otel4s.trace.TracerProvider
Expand All @@ -32,4 +33,7 @@ trait Otel4s[F[_]] {
/** An entry point of the tracing API.
*/
def tracerProvider: TracerProvider[F]

/** A utility for accessing and modifying [[baggage.Baggage `Baggage`]]. */
def baggageManager: BaggageManager[F]
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,20 @@ sealed trait Baggage {
*/
def updated(key: String, value: String, metadata: Option[String]): Baggage

/** Adds or updates the entry that has the given `key` if it is present.
*
* @param key
* the key for the entry
*
* @param value
* the value for the entry to associate with the key
*
* @param metadata
* the optional metadata to associate with the key
*/
final def updated(key: String, value: String, metadata: String): Baggage =
updated(key, value, Some(metadata))

/** Adds or updates the entry that has the given `key` if it is present.
*
* @param key
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* Copyright 2022 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.baggage

import cats.mtl.Local

/** A utility for accessing and modifying [[`Baggage`]]. */
trait BaggageManager[F[_]] extends Local[F, Baggage] {

/** @return the current `Baggage` */
final def current: F[Baggage] = ask[Baggage]

/** @return
* the [[Baggage.Entry entry]] to which the specified key is mapped, or `None` if the current `Baggage` contains no
* mapping for the key
*/
def get(key: String): F[Option[Baggage.Entry]] =
applicative.map(ask[Baggage])(_.get(key))

/** @return
* the value (without [[Baggage.Metadata metadata]]) to which the specified key is mapped, or `None` if the current
* `Baggage` contains no mapping for the key
*/
def getValue(key: String): F[Option[String]] =
applicative.map(ask[Baggage])(_.get(key).map(_.value))
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
/*
* Copyright 2022 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.baggage

import cats.effect.IO
import munit.CatsEffectSuite

abstract class BaggageManagerSuite extends CatsEffectSuite {
protected def baggageManager: IO[BaggageManager[IO]]

protected final def testManager(name: String)(f: BaggageManager[IO] => IO[Unit]): Unit =
test(name)(baggageManager.flatMap(f))

testManager(".ask consistent with .scope") { m =>
val b1 = Baggage.empty
.updated("key", "value")
.updated("foo", "bar", "baz")
m.scope(m.ask[Baggage].map(assertEquals(_, b1)))(b1)
}

testManager(".ask consistent with .local") { m =>
m.local {
for (baggage <- m.ask[Baggage])
yield {
assertEquals(baggage.get("key"), Some(Baggage.Entry("value", None)))
assertEquals(
baggage.get("foo"),
Some(Baggage.Entry("bar", Some(Baggage.Metadata("baz"))))
)
}
}(_.updated("key", "value").updated("foo", "bar", "baz"))
}

testManager(".current is equivalent to .ask") { m =>
val check = m.scope {
for {
a <- m.ask[Baggage]
b <- m.current
} yield assertEquals(a, b)
}(_)
check(Baggage.empty)
check(
Baggage.empty
.updated("key", "value")
.updated("foo", "bar", "baz")
)
}

testManager(".get consistent with .ask") { m =>
val check = m.scope {
for {
baggage <- m.ask[Baggage]
v1 <- m.get("key")
v2 <- m.get("foo")
} yield {
assertEquals(v1, baggage.get("key"))
assertEquals(v2, baggage.get("foo"))
}
}(_)
check(Baggage.empty)
check(
Baggage.empty
.updated("key", "value")
.updated("foo", "bar", "baz")
)
}

testManager(".getValue consistent with .ask") { m =>
val check = m.scope {
for {
baggage <- m.ask[Baggage]
v1 <- m.getValue("key")
v2 <- m.getValue("foo")
} yield {
assertEquals(v1, baggage.get("key").map(_.value))
assertEquals(v2, baggage.get("foo").map(_.value))
}
}(_)
check(Baggage.empty)
check(
Baggage.empty
.updated("key", "value")
.updated("foo", "bar", "baz")
)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,11 @@ import io.opentelemetry.sdk.autoconfigure.{AutoConfiguredOpenTelemetrySdk => Aut
import io.opentelemetry.sdk.autoconfigure.{AutoConfiguredOpenTelemetrySdkBuilder => AutoConfigOtelSdkBuilder}
import io.opentelemetry.sdk.common.CompletableResultCode
import org.typelevel.otel4s.Otel4s
import org.typelevel.otel4s.baggage.BaggageManager
import org.typelevel.otel4s.context.LocalProvider
import org.typelevel.otel4s.context.propagation.ContextPropagators
import org.typelevel.otel4s.metrics.MeterProvider
import org.typelevel.otel4s.oteljava.baggage.BaggageManagerImpl
import org.typelevel.otel4s.oteljava.context.Context
import org.typelevel.otel4s.oteljava.context.LocalContext
import org.typelevel.otel4s.oteljava.context.LocalContextProvider
Expand All @@ -47,6 +49,8 @@ final class OtelJava[F[_]] private (
extends Otel4s[F] {
type Ctx = Context

val baggageManager: BaggageManager[F] = BaggageManagerImpl.fromLocal

override def toString: String = s"OtelJava{$underlying}"
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ import scala.jdk.CollectionConverters._
* import org.typelevel.otel4s.oteljava.AttributeConverters._
*
* val attributes: Attributes =
* io.opentelemetry.api.common.Attributes.builder()
* JAttributes.builder()
* .put("key", "value")
* .build()
* .toScala
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/*
* Copyright 2022 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.oteljava.baggage

import io.opentelemetry.api.baggage.{Baggage => JBaggage}
import io.opentelemetry.api.baggage.BaggageEntry
import io.opentelemetry.api.baggage.BaggageEntryMetadata
import org.typelevel.otel4s.baggage.Baggage

/** This object provides extension methods that convert between Scala and Java `Baggage`s, Java `BaggageEntry`s, and
* Scala `Baggage.Entry`s using `toScala` and `toJava` extension methods.
*
* {{{
* import io.opentelemetry.api.baggage.{Baggage => JBaggage}
* import org.typelevel.baggage.Baggage
* import org.typelevel.otel4s.oteljava.baggage.BaggageConverters._
*
* val baggage: Baggage =
* JBaggage.builder()
* .put("key", "value")
* .build()
* .toScala
* }}}
*
* The conversions do not return wrappers.
*/
object BaggageConverters {

implicit final class BaggageHasToJava(private val baggage: Baggage) extends AnyVal {

/** Converts a Scala `Baggage` to a Java `Baggage`. */
def toJava: JBaggage = Explicit.toJava(baggage)
}

implicit final class BaggageEntryHasToScala(private val entry: BaggageEntry) extends AnyVal {

/** Converts a Java `BaggageEntry` to a Scala `Baggage.Entry`. */
def toScala: Baggage.Entry = Explicit.toScala(entry)
}

implicit final class BaggageHasToScala(private val baggage: JBaggage) extends AnyVal {

/** Converts a Java `Baggage` to a Scala `Baggage`. */
def toScala: Baggage = Explicit.toScala(baggage)
}

private[this] object Explicit {
def toJava(metadata: Option[Baggage.Metadata]): BaggageEntryMetadata =
metadata.fold(BaggageEntryMetadata.empty()) { m =>
BaggageEntryMetadata.create(m.value)
}

def toJava(baggage: Baggage): JBaggage = {
val builder = JBaggage.builder()
baggage.asMap.foreach { case (key, entry) =>
builder.put(key, entry.value, toJava(entry.metadata))
}
builder.build()
}

def toScala(metadata: BaggageEntryMetadata): Option[Baggage.Metadata] = {
val value = metadata.getValue
Option.unless(value.isEmpty)(Baggage.Metadata(value))
}

def toScala(entry: BaggageEntry): Baggage.Entry =
Baggage.Entry(entry.getValue, toScala(entry.getMetadata))

def toScala(baggage: JBaggage): Baggage = {
var res = Baggage.empty
baggage.forEach { (key, entry) =>
val metadata = entry.getMetadata.getValue
res = res.updated(key, entry.getValue, Option.unless(metadata.isEmpty)(metadata))
}
res
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* Copyright 2022 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.oteljava.baggage

import cats.Applicative
import io.opentelemetry.api.baggage.{Baggage => JBaggage}
import org.typelevel.otel4s.baggage.Baggage
import org.typelevel.otel4s.baggage.BaggageManager
import org.typelevel.otel4s.oteljava.baggage.BaggageConverters._
import org.typelevel.otel4s.oteljava.context.Context
import org.typelevel.otel4s.oteljava.context.LocalContext

private final class BaggageManagerImpl[F[_]] private (implicit localContext: LocalContext[F])
extends BaggageManager[F] {
def applicative: Applicative[F] = localContext.applicative
def ask[E2 >: Baggage]: F[E2] =
applicative.map(localContext.ask[Context]) { ctx =>
Option(JBaggage.fromContextOrNull(ctx.underlying))
.fold(Baggage.empty)(_.toScala)
}
def local[A](fa: F[A])(f: Baggage => Baggage): F[A] =
localContext.local(fa) { ctx =>
val jCtx = ctx.underlying
val jBaggage = JBaggage.fromContext(jCtx)
val updated = f(jBaggage.toScala).toJava
Context.wrap(jCtx.`with`(updated))
}
override def get(key: String): F[Option[Baggage.Entry]] =
applicative.map(localContext.ask[Context]) { ctx =>
Option(JBaggage.fromContext(ctx.underlying).getEntry(key))
.map(_.toScala)
}
override def getValue(key: String): F[Option[String]] =
applicative.map(localContext.ask[Context]) { ctx =>
Option(JBaggage.fromContext(ctx.underlying).getEntryValue(key))
}
}

private[oteljava] object BaggageManagerImpl {
def fromLocal[F[_]: LocalContext]: BaggageManager[F] =
new BaggageManagerImpl[F]
}
Loading

0 comments on commit f1946fa

Please sign in to comment.