Start a CONSUMER span for Kafka poll(); and refactor spring-kafka... (#4041)

* Start a separate CONSUMER receive span for each non-empty KafkaConsumer#poll() call

* One batch receive + one batch process span in spring-kafka

* Add CONSUMER receive spans to kafka-streams too

* codenarc

* code review comments
This commit is contained in:
Mateusz Rzeszutek 2021-09-08 22:12:58 +02:00 committed by GitHub
parent 28e5cb5dd6
commit e30b082259
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 669 additions and 268 deletions

View File

@ -12,6 +12,9 @@ muzzle {
} }
dependencies { dependencies {
compileOnly("com.google.auto.value:auto-value-annotations")
annotationProcessor("com.google.auto.value:auto-value")
implementation(project(":instrumentation:kafka-clients:kafka-clients-common:javaagent")) implementation(project(":instrumentation:kafka-clients:kafka-clients-common:javaagent"))
library("org.apache.kafka:kafka-clients:0.11.0.0") library("org.apache.kafka:kafka-clients:0.11.0.0")

View File

@ -0,0 +1,96 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/
package io.opentelemetry.javaagent.instrumentation.kafkaclients;
import static net.bytebuddy.matcher.ElementMatchers.isMethod;
import static net.bytebuddy.matcher.ElementMatchers.isPublic;
import static net.bytebuddy.matcher.ElementMatchers.named;
import static net.bytebuddy.matcher.ElementMatchers.returns;
import static net.bytebuddy.matcher.ElementMatchers.takesArgument;
import static net.bytebuddy.matcher.ElementMatchers.takesArguments;
import io.opentelemetry.api.trace.SpanContext;
import io.opentelemetry.javaagent.extension.instrumentation.TypeInstrumentation;
import io.opentelemetry.javaagent.extension.instrumentation.TypeTransformer;
import io.opentelemetry.javaagent.instrumentation.api.ContextStore;
import io.opentelemetry.javaagent.instrumentation.api.InstrumentationContext;
import java.util.Iterator;
import java.util.List;
import net.bytebuddy.asm.Advice;
import net.bytebuddy.description.type.TypeDescription;
import net.bytebuddy.matcher.ElementMatcher;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
public class ConsumerRecordsInstrumentation implements TypeInstrumentation {
@Override
public ElementMatcher<TypeDescription> typeMatcher() {
return named("org.apache.kafka.clients.consumer.ConsumerRecords");
}
@Override
public void transform(TypeTransformer transformer) {
transformer.applyAdviceToMethod(
isMethod()
.and(isPublic())
.and(named("records"))
.and(takesArgument(0, String.class))
.and(returns(Iterable.class)),
ConsumerRecordsInstrumentation.class.getName() + "$IterableAdvice");
transformer.applyAdviceToMethod(
isMethod()
.and(isPublic())
.and(named("records"))
.and(takesArgument(0, named("org.apache.kafka.common.TopicPartition")))
.and(returns(List.class)),
ConsumerRecordsInstrumentation.class.getName() + "$ListAdvice");
transformer.applyAdviceToMethod(
isMethod()
.and(isPublic())
.and(named("iterator"))
.and(takesArguments(0))
.and(returns(Iterator.class)),
ConsumerRecordsInstrumentation.class.getName() + "$IteratorAdvice");
}
@SuppressWarnings("unused")
public static class IterableAdvice {
@Advice.OnMethodExit(suppress = Throwable.class)
public static void wrap(
@Advice.Return(readOnly = false) Iterable<ConsumerRecord<?, ?>> iterable) {
if (iterable != null) {
ContextStore<ConsumerRecords, SpanContext> consumerRecordsSpan =
InstrumentationContext.get(ConsumerRecords.class, SpanContext.class);
iterable = new TracingIterable(iterable);
}
}
}
@SuppressWarnings("unused")
public static class ListAdvice {
@Advice.OnMethodExit(suppress = Throwable.class)
public static void wrap(@Advice.Return(readOnly = false) List<ConsumerRecord<?, ?>> iterable) {
if (iterable != null) {
iterable = new TracingList(iterable);
}
}
}
@SuppressWarnings("unused")
public static class IteratorAdvice {
@Advice.OnMethodExit(suppress = Throwable.class)
public static void wrap(
@Advice.Return(readOnly = false) Iterator<ConsumerRecord<?, ?>> iterator) {
if (iterator != null) {
iterator = new TracingIterator(iterator);
}
}
}
}

View File

@ -20,6 +20,9 @@ public class KafkaClientsInstrumentationModule extends InstrumentationModule {
@Override @Override
public List<TypeInstrumentation> typeInstrumentations() { public List<TypeInstrumentation> typeInstrumentations() {
return asList(new KafkaConsumerInstrumentation(), new KafkaProducerInstrumentation()); return asList(
new KafkaProducerInstrumentation(),
new KafkaConsumerInstrumentation(),
new ConsumerRecordsInstrumentation());
} }
} }

View File

@ -5,86 +5,76 @@
package io.opentelemetry.javaagent.instrumentation.kafkaclients; package io.opentelemetry.javaagent.instrumentation.kafkaclients;
import static io.opentelemetry.javaagent.instrumentation.kafkaclients.KafkaSingletons.consumerInstrumenter; import static io.opentelemetry.javaagent.instrumentation.api.Java8BytecodeBridge.currentContext;
import static net.bytebuddy.matcher.ElementMatchers.isMethod; import static io.opentelemetry.javaagent.instrumentation.api.Java8BytecodeBridge.spanFromContext;
import static io.opentelemetry.javaagent.instrumentation.kafkaclients.KafkaSingletons.consumerReceiveInstrumenter;
import static net.bytebuddy.matcher.ElementMatchers.isPublic; import static net.bytebuddy.matcher.ElementMatchers.isPublic;
import static net.bytebuddy.matcher.ElementMatchers.named; import static net.bytebuddy.matcher.ElementMatchers.named;
import static net.bytebuddy.matcher.ElementMatchers.returns; import static net.bytebuddy.matcher.ElementMatchers.returns;
import static net.bytebuddy.matcher.ElementMatchers.takesArgument; import static net.bytebuddy.matcher.ElementMatchers.takesArgument;
import static net.bytebuddy.matcher.ElementMatchers.takesArguments; import static net.bytebuddy.matcher.ElementMatchers.takesArguments;
import io.opentelemetry.api.trace.SpanContext;
import io.opentelemetry.context.Context;
import io.opentelemetry.javaagent.extension.instrumentation.TypeInstrumentation; import io.opentelemetry.javaagent.extension.instrumentation.TypeInstrumentation;
import io.opentelemetry.javaagent.extension.instrumentation.TypeTransformer; import io.opentelemetry.javaagent.extension.instrumentation.TypeTransformer;
import java.util.Iterator; import io.opentelemetry.javaagent.instrumentation.api.ContextStore;
import java.util.List; import io.opentelemetry.javaagent.instrumentation.api.InstrumentationContext;
import java.time.Duration;
import net.bytebuddy.asm.Advice; import net.bytebuddy.asm.Advice;
import net.bytebuddy.description.type.TypeDescription; import net.bytebuddy.description.type.TypeDescription;
import net.bytebuddy.matcher.ElementMatcher; import net.bytebuddy.matcher.ElementMatcher;
import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords;
public class KafkaConsumerInstrumentation implements TypeInstrumentation { public class KafkaConsumerInstrumentation implements TypeInstrumentation {
@Override @Override
public ElementMatcher<TypeDescription> typeMatcher() { public ElementMatcher<TypeDescription> typeMatcher() {
return named("org.apache.kafka.clients.consumer.ConsumerRecords"); return named("org.apache.kafka.clients.consumer.KafkaConsumer");
} }
@Override @Override
public void transform(TypeTransformer transformer) { public void transform(TypeTransformer transformer) {
transformer.applyAdviceToMethod( transformer.applyAdviceToMethod(
isMethod() named("poll")
.and(isPublic()) .and(isPublic())
.and(named("records")) .and(takesArguments(1))
.and(takesArgument(0, String.class)) .and(takesArgument(0, long.class).or(takesArgument(0, Duration.class)))
.and(returns(Iterable.class)), .and(returns(named("org.apache.kafka.clients.consumer.ConsumerRecords"))),
KafkaConsumerInstrumentation.class.getName() + "$IterableAdvice"); this.getClass().getName() + "$PollAdvice");
transformer.applyAdviceToMethod(
isMethod()
.and(isPublic())
.and(named("records"))
.and(takesArgument(0, named("org.apache.kafka.common.TopicPartition")))
.and(returns(List.class)),
KafkaConsumerInstrumentation.class.getName() + "$ListAdvice");
transformer.applyAdviceToMethod(
isMethod()
.and(isPublic())
.and(named("iterator"))
.and(takesArguments(0))
.and(returns(Iterator.class)),
KafkaConsumerInstrumentation.class.getName() + "$IteratorAdvice");
} }
@SuppressWarnings("unused") @SuppressWarnings("unused")
public static class IterableAdvice { public static class PollAdvice {
@Advice.OnMethodEnter(suppress = Throwable.class)
@Advice.OnMethodExit(suppress = Throwable.class) public static Timer onEnter() {
public static void wrap( return Timer.start();
@Advice.Return(readOnly = false) Iterable<ConsumerRecord<?, ?>> iterable) {
if (iterable != null) {
iterable = new TracingIterable(iterable);
}
} }
}
@SuppressWarnings("unused") @Advice.OnMethodExit(suppress = Throwable.class, onThrowable = Throwable.class)
public static class ListAdvice { public static void onExit(
@Advice.Enter Timer timer,
@Advice.Return ConsumerRecords<?, ?> records,
@Advice.Thrown Throwable error) {
@Advice.OnMethodExit(suppress = Throwable.class) // don't create spans when no records were received
public static void wrap(@Advice.Return(readOnly = false) List<ConsumerRecord<?, ?>> iterable) { if (records == null || records.isEmpty()) {
if (iterable != null) { return;
iterable = new TracingList(iterable);
} }
}
}
@SuppressWarnings("unused") Context parentContext = currentContext();
public static class IteratorAdvice { ReceivedRecords receivedRecords = ReceivedRecords.create(records, timer);
if (consumerReceiveInstrumenter().shouldStart(parentContext, receivedRecords)) {
Context context = consumerReceiveInstrumenter().start(parentContext, receivedRecords);
consumerReceiveInstrumenter().end(context, receivedRecords, null, error);
@Advice.OnMethodExit(suppress = Throwable.class) // we're storing the context of the receive span so that process spans can use it as parent
public static void wrap( // context even though the span has ended
@Advice.Return(readOnly = false) Iterator<ConsumerRecord<?, ?>> iterator) { // this is the suggested behavior according to the spec batch receive scenario:
if (iterator != null) { // https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/trace/semantic_conventions/messaging.md#batch-receiving
iterator = new TracingIterator(iterator, consumerInstrumenter()); ContextStore<ConsumerRecords, SpanContext> consumerRecordsSpan =
InstrumentationContext.get(ConsumerRecords.class, SpanContext.class);
consumerRecordsSpan.put(records, spanFromContext(context).getSpanContext());
} }
} }
} }

View File

@ -3,7 +3,7 @@
* SPDX-License-Identifier: Apache-2.0 * SPDX-License-Identifier: Apache-2.0
*/ */
package io.opentelemetry.javaagent.instrumentation.spring.kafka; package io.opentelemetry.javaagent.instrumentation.kafkaclients;
import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessageOperation; import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessageOperation;
import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessagingAttributesExtractor; import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessagingAttributesExtractor;
@ -13,22 +13,23 @@ import java.util.stream.Collectors;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.checker.nullness.qual.Nullable;
public final class BatchConsumerAttributesExtractor public final class KafkaReceiveAttributesExtractor
extends MessagingAttributesExtractor<BatchRecords<?, ?>, Void> { extends MessagingAttributesExtractor<ReceivedRecords, Void> {
@Override @Override
protected String system(BatchRecords<?, ?> batchRecords) { protected String system(ReceivedRecords receivedRecords) {
return "kafka"; return "kafka";
} }
@Override @Override
protected String destinationKind(BatchRecords<?, ?> batchRecords) { protected String destinationKind(ReceivedRecords receivedRecords) {
return SemanticAttributes.MessagingDestinationKindValues.TOPIC; return SemanticAttributes.MessagingDestinationKindValues.TOPIC;
} }
@Override @Override
protected @Nullable String destination(BatchRecords<?, ?> batchRecords) { protected @Nullable String destination(ReceivedRecords receivedRecords) {
Set<String> topics = Set<String> topics =
batchRecords.records().partitions().stream() receivedRecords.records().partitions().stream()
.map(TopicPartition::topic) .map(TopicPartition::topic)
.collect(Collectors.toSet()); .collect(Collectors.toSet());
// only return topic when there's exactly one in the batch // only return topic when there's exactly one in the batch
@ -36,47 +37,47 @@ public final class BatchConsumerAttributesExtractor
} }
@Override @Override
protected boolean temporaryDestination(BatchRecords<?, ?> batchRecords) { protected boolean temporaryDestination(ReceivedRecords receivedRecords) {
return false; return false;
} }
@Override @Override
protected @Nullable String protocol(BatchRecords<?, ?> batchRecords) { protected @Nullable String protocol(ReceivedRecords receivedRecords) {
return null; return null;
} }
@Override @Override
protected @Nullable String protocolVersion(BatchRecords<?, ?> batchRecords) { protected @Nullable String protocolVersion(ReceivedRecords receivedRecords) {
return null; return null;
} }
@Override @Override
protected @Nullable String url(BatchRecords<?, ?> batchRecords) { protected @Nullable String url(ReceivedRecords receivedRecords) {
return null; return null;
} }
@Override @Override
protected @Nullable String conversationId(BatchRecords<?, ?> batchRecords) { protected @Nullable String conversationId(ReceivedRecords receivedRecords) {
return null; return null;
} }
@Override @Override
protected @Nullable Long messagePayloadSize(BatchRecords<?, ?> batchRecords) { protected @Nullable Long messagePayloadSize(ReceivedRecords receivedRecords) {
return null; return null;
} }
@Override @Override
protected @Nullable Long messagePayloadCompressedSize(BatchRecords<?, ?> batchRecords) { protected @Nullable Long messagePayloadCompressedSize(ReceivedRecords receivedRecords) {
return null; return null;
} }
@Override @Override
protected MessageOperation operation(BatchRecords<?, ?> batchRecords) { protected MessageOperation operation(ReceivedRecords receivedRecords) {
return MessageOperation.PROCESS; return MessageOperation.RECEIVE;
} }
@Override @Override
protected @Nullable String messageId(BatchRecords<?, ?> batchRecords, @Nullable Void unused) { protected @Nullable String messageId(ReceivedRecords receivedRecords, @Nullable Void unused) {
return null; return null;
} }
} }

View File

@ -24,8 +24,10 @@ public final class KafkaSingletons {
private static final Instrumenter<ProducerRecord<?, ?>, Void> PRODUCER_INSTRUMENTER = private static final Instrumenter<ProducerRecord<?, ?>, Void> PRODUCER_INSTRUMENTER =
buildProducerInstrumenter(); buildProducerInstrumenter();
private static final Instrumenter<ConsumerRecord<?, ?>, Void> CONSUMER_INSTRUMENTER = private static final Instrumenter<ReceivedRecords, Void> CONSUMER_RECEIVE_INSTRUMENTER =
buildConsumerInstrumenter(); buildConsumerReceiveInstrumenter();
private static final Instrumenter<ConsumerRecord<?, ?>, Void> CONSUMER_PROCESS_INSTRUMENTER =
buildConsumerProcessInstrumenter();
private static Instrumenter<ProducerRecord<?, ?>, Void> buildProducerInstrumenter() { private static Instrumenter<ProducerRecord<?, ?>, Void> buildProducerInstrumenter() {
KafkaProducerAttributesExtractor attributesExtractor = new KafkaProducerAttributesExtractor(); KafkaProducerAttributesExtractor attributesExtractor = new KafkaProducerAttributesExtractor();
@ -39,7 +41,19 @@ public final class KafkaSingletons {
.newInstrumenter(SpanKindExtractor.alwaysProducer()); .newInstrumenter(SpanKindExtractor.alwaysProducer());
} }
private static Instrumenter<ConsumerRecord<?, ?>, Void> buildConsumerInstrumenter() { private static Instrumenter<ReceivedRecords, Void> buildConsumerReceiveInstrumenter() {
KafkaReceiveAttributesExtractor attributesExtractor = new KafkaReceiveAttributesExtractor();
SpanNameExtractor<ReceivedRecords> spanNameExtractor =
MessagingSpanNameExtractor.create(attributesExtractor);
return Instrumenter.<ReceivedRecords, Void>newBuilder(
GlobalOpenTelemetry.get(), INSTRUMENTATION_NAME, spanNameExtractor)
.addAttributesExtractor(attributesExtractor)
.setTimeExtractors(ReceivedRecords::startTime, (request, response) -> request.now())
.newInstrumenter(SpanKindExtractor.alwaysConsumer());
}
private static Instrumenter<ConsumerRecord<?, ?>, Void> buildConsumerProcessInstrumenter() {
KafkaConsumerAttributesExtractor attributesExtractor = KafkaConsumerAttributesExtractor attributesExtractor =
new KafkaConsumerAttributesExtractor(MessageOperation.PROCESS); new KafkaConsumerAttributesExtractor(MessageOperation.PROCESS);
SpanNameExtractor<ConsumerRecord<?, ?>> spanNameExtractor = SpanNameExtractor<ConsumerRecord<?, ?>> spanNameExtractor =
@ -62,8 +76,12 @@ public final class KafkaSingletons {
return PRODUCER_INSTRUMENTER; return PRODUCER_INSTRUMENTER;
} }
public static Instrumenter<ConsumerRecord<?, ?>, Void> consumerInstrumenter() { public static Instrumenter<ReceivedRecords, Void> consumerReceiveInstrumenter() {
return CONSUMER_INSTRUMENTER; return CONSUMER_RECEIVE_INSTRUMENTER;
}
public static Instrumenter<ConsumerRecord<?, ?>, Void> consumerProcessInstrumenter() {
return CONSUMER_PROCESS_INSTRUMENTER;
} }
private KafkaSingletons() {} private KafkaSingletons() {}

View File

@ -0,0 +1,30 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/
package io.opentelemetry.javaagent.instrumentation.kafkaclients;
import com.google.auto.value.AutoValue;
import java.time.Instant;
import org.apache.kafka.clients.consumer.ConsumerRecords;
@AutoValue
public abstract class ReceivedRecords {
public static ReceivedRecords create(ConsumerRecords<?, ?> records, Timer timer) {
return new AutoValue_ReceivedRecords(records, timer);
}
public abstract ConsumerRecords<?, ?> records();
abstract Timer timer();
public Instant startTime() {
return timer().startTime();
}
public Instant now() {
return timer().now();
}
}

View File

@ -0,0 +1,32 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/
package io.opentelemetry.javaagent.instrumentation.kafkaclients;
import java.time.Instant;
public final class Timer {
public static Timer start() {
return new Timer(Instant.now(), System.nanoTime());
}
private final Instant startTime;
private final long startNanoTime;
private Timer(Instant startTime, long startNanoTime) {
this.startTime = startTime;
this.startNanoTime = startNanoTime;
}
public Instant startTime() {
return startTime;
}
public Instant now() {
long durationNanos = System.nanoTime() - startNanoTime;
return startTime().plusNanos(durationNanos);
}
}

View File

@ -5,8 +5,6 @@
package io.opentelemetry.javaagent.instrumentation.kafkaclients; package io.opentelemetry.javaagent.instrumentation.kafkaclients;
import static io.opentelemetry.javaagent.instrumentation.kafkaclients.KafkaSingletons.consumerInstrumenter;
import java.util.Iterator; import java.util.Iterator;
import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecord;
@ -25,7 +23,7 @@ public class TracingIterable<K, V> implements Iterable<ConsumerRecord<K, V>> {
// However, this is not thread-safe, but usually the first (hopefully only) traversal of // However, this is not thread-safe, but usually the first (hopefully only) traversal of
// ConsumerRecords is performed in the same thread that called poll() // ConsumerRecords is performed in the same thread that called poll()
if (firstIterator) { if (firstIterator) {
it = new TracingIterator<>(delegate.iterator(), consumerInstrumenter()); it = new TracingIterator<>(delegate.iterator());
firstIterator = false; firstIterator = false;
} else { } else {
it = delegate.iterator(); it = delegate.iterator();

View File

@ -5,9 +5,10 @@
package io.opentelemetry.javaagent.instrumentation.kafkaclients; package io.opentelemetry.javaagent.instrumentation.kafkaclients;
import static io.opentelemetry.javaagent.instrumentation.kafkaclients.KafkaSingletons.consumerProcessInstrumenter;
import io.opentelemetry.context.Context; import io.opentelemetry.context.Context;
import io.opentelemetry.context.Scope; import io.opentelemetry.context.Scope;
import io.opentelemetry.instrumentation.api.instrumenter.Instrumenter;
import io.opentelemetry.javaagent.instrumentation.kafka.KafkaConsumerIteratorWrapper; import io.opentelemetry.javaagent.instrumentation.kafka.KafkaConsumerIteratorWrapper;
import java.util.Iterator; import java.util.Iterator;
import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecord;
@ -17,23 +18,22 @@ public class TracingIterator<K, V>
implements Iterator<ConsumerRecord<K, V>>, KafkaConsumerIteratorWrapper<K, V> { implements Iterator<ConsumerRecord<K, V>>, KafkaConsumerIteratorWrapper<K, V> {
private final Iterator<ConsumerRecord<K, V>> delegateIterator; private final Iterator<ConsumerRecord<K, V>> delegateIterator;
private final Instrumenter<ConsumerRecord<?, ?>, Void> instrumenter; // TODO: use the context extracted from ConsumerRecords (receive context) as the parent span
// for that to work properly we'd have to modify the consumer span suppression strategy to
// differentiate between receive and process consumer spans - right now if we were to pass the
// receive context to this instrumentation it'd be suppressed
private final Context parentContext; private final Context parentContext;
/** /*
* Note: this may potentially create problems if this iterator is used from different threads. But * Note: this may potentially create problems if this iterator is used from different threads. But
* at the moment we cannot do much about this. * at the moment we cannot do much about this.
*/ */
@Nullable private ConsumerRecord<?, ?> currentRequest; @Nullable private ConsumerRecord<?, ?> currentRequest;
@Nullable private Context currentContext; @Nullable private Context currentContext;
@Nullable private Scope currentScope; @Nullable private Scope currentScope;
public TracingIterator( public TracingIterator(Iterator<ConsumerRecord<K, V>> delegateIterator) {
Iterator<ConsumerRecord<K, V>> delegateIterator,
Instrumenter<ConsumerRecord<?, ?>, Void> instrumenter) {
this.delegateIterator = delegateIterator; this.delegateIterator = delegateIterator;
this.instrumenter = instrumenter;
parentContext = Context.current(); parentContext = Context.current();
} }
@ -49,9 +49,9 @@ public class TracingIterator<K, V>
closeScopeAndEndSpan(); closeScopeAndEndSpan();
ConsumerRecord<K, V> next = delegateIterator.next(); ConsumerRecord<K, V> next = delegateIterator.next();
if (next != null && instrumenter.shouldStart(parentContext, next)) { if (next != null && consumerProcessInstrumenter().shouldStart(parentContext, next)) {
currentRequest = next; currentRequest = next;
currentContext = instrumenter.start(parentContext, currentRequest); currentContext = consumerProcessInstrumenter().start(parentContext, currentRequest);
currentScope = currentContext.makeCurrent(); currentScope = currentContext.makeCurrent();
} }
return next; return next;
@ -60,7 +60,7 @@ public class TracingIterator<K, V>
private void closeScopeAndEndSpan() { private void closeScopeAndEndSpan() {
if (currentScope != null) { if (currentScope != null) {
currentScope.close(); currentScope.close();
instrumenter.end(currentContext, currentRequest, null, null); consumerProcessInstrumenter().end(currentContext, currentRequest, null, null);
currentScope = null; currentScope = null;
currentRequest = null; currentRequest = null;
currentContext = null; currentContext = null;

View File

@ -54,7 +54,7 @@ class KafkaClientPropagationDisabledTest extends KafkaClientBaseTest {
// check that the message was received // check that the message was received
records.poll(5, TimeUnit.SECONDS) != null records.poll(5, TimeUnit.SECONDS) != null
assertTraces(2) { assertTraces(3) {
trace(0, 1) { trace(0, 1) {
span(0) { span(0) {
name SHARED_TOPIC + " send" name SHARED_TOPIC + " send"
@ -68,6 +68,19 @@ class KafkaClientPropagationDisabledTest extends KafkaClientBaseTest {
} }
} }
trace(1, 1) { trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
trace(2, 1) {
span(0) { span(0) {
name SHARED_TOPIC + " process" name SHARED_TOPIC + " process"
kind CONSUMER kind CONSUMER
@ -84,7 +97,6 @@ class KafkaClientPropagationDisabledTest extends KafkaClientBaseTest {
} }
} }
} }
} }
cleanup: cleanup:

View File

@ -4,9 +4,9 @@
*/ */
import static io.opentelemetry.api.trace.SpanKind.CONSUMER import static io.opentelemetry.api.trace.SpanKind.CONSUMER
import static io.opentelemetry.api.trace.SpanKind.INTERNAL
import static io.opentelemetry.api.trace.SpanKind.PRODUCER import static io.opentelemetry.api.trace.SpanKind.PRODUCER
import io.opentelemetry.api.trace.SpanKind
import io.opentelemetry.semconv.trace.attributes.SemanticAttributes import io.opentelemetry.semconv.trace.attributes.SemanticAttributes
import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.LinkedBlockingQueue
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
@ -80,11 +80,13 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
received.value() == greeting received.value() == greeting
received.key() == null received.key() == null
assertTraces(1) { assertTraces(2) {
traces.sort(orderByRootSpanKind(INTERNAL, CONSUMER))
trace(0, 4) { trace(0, 4) {
span(0) { span(0) {
name "parent" name "parent"
kind SpanKind.INTERNAL kind INTERNAL
hasNoParent() hasNoParent()
} }
span(1) { span(1) {
@ -114,10 +116,23 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
} }
span(3) { span(3) {
name "producer callback" name "producer callback"
kind SpanKind.INTERNAL kind INTERNAL
childOf span(0) childOf span(0)
} }
} }
trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
cleanup: cleanup:
@ -176,11 +191,13 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
received.value() == greeting received.value() == greeting
received.key() == null received.key() == null
assertTraces(1) { assertTraces(2) {
traces.sort(orderByRootSpanKind(INTERNAL, CONSUMER))
trace(0, 4) { trace(0, 4) {
span(0) { span(0) {
name "parent" name "parent"
kind SpanKind.INTERNAL kind INTERNAL
hasNoParent() hasNoParent()
} }
span(1) { span(1) {
@ -210,10 +227,23 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
} }
span(3) { span(3) {
name "producer callback" name "producer callback"
kind SpanKind.INTERNAL kind INTERNAL
childOf span(0) childOf span(0)
} }
} }
trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
cleanup: cleanup:
@ -265,7 +295,9 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
received.value() == null received.value() == null
received.key() == null received.key() == null
assertTraces(1) { assertTraces(2) {
traces.sort(orderByRootSpanKind(PRODUCER, CONSUMER))
trace(0, 2) { trace(0, 2) {
// PRODUCER span 0 // PRODUCER span 0
span(0) { span(0) {
@ -297,6 +329,19 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
} }
} }
} }
trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
cleanup: cleanup:
@ -339,7 +384,9 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
first.value() == greeting first.value() == greeting
first.key() == null first.key() == null
assertTraces(1) { assertTraces(2) {
traces.sort(orderByRootSpanKind(PRODUCER, CONSUMER))
trace(0, 2) { trace(0, 2) {
span(0) { span(0) {
name SHARED_TOPIC + " send" name SHARED_TOPIC + " send"
@ -368,6 +415,19 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
} }
} }
} }
trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
cleanup: cleanup:

View File

@ -54,7 +54,7 @@ class KafkaClientPropagationDisabledTest extends KafkaClientBaseTest {
// check that the message was received // check that the message was received
records.poll(5, TimeUnit.SECONDS) != null records.poll(5, TimeUnit.SECONDS) != null
assertTraces(2) { assertTraces(3) {
trace(0, 1) { trace(0, 1) {
span(0) { span(0) {
name SHARED_TOPIC + " send" name SHARED_TOPIC + " send"
@ -68,6 +68,19 @@ class KafkaClientPropagationDisabledTest extends KafkaClientBaseTest {
} }
} }
trace(1, 1) { trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
trace(2, 1) {
span(0) { span(0) {
name SHARED_TOPIC + " process" name SHARED_TOPIC + " process"
kind CONSUMER kind CONSUMER

View File

@ -4,9 +4,9 @@
*/ */
import static io.opentelemetry.api.trace.SpanKind.CONSUMER import static io.opentelemetry.api.trace.SpanKind.CONSUMER
import static io.opentelemetry.api.trace.SpanKind.INTERNAL
import static io.opentelemetry.api.trace.SpanKind.PRODUCER import static io.opentelemetry.api.trace.SpanKind.PRODUCER
import io.opentelemetry.api.trace.SpanKind
import io.opentelemetry.semconv.trace.attributes.SemanticAttributes import io.opentelemetry.semconv.trace.attributes.SemanticAttributes
import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.LinkedBlockingQueue
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
@ -80,11 +80,13 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
received.value() == greeting received.value() == greeting
received.key() == null received.key() == null
assertTraces(1) { assertTraces(2) {
traces.sort(orderByRootSpanKind(INTERNAL, CONSUMER))
trace(0, 4) { trace(0, 4) {
span(0) { span(0) {
name "parent" name "parent"
kind SpanKind.INTERNAL kind INTERNAL
hasNoParent() hasNoParent()
} }
span(1) { span(1) {
@ -114,10 +116,23 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
} }
span(3) { span(3) {
name "producer callback" name "producer callback"
kind SpanKind.INTERNAL kind INTERNAL
childOf span(0) childOf span(0)
} }
} }
trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
cleanup: cleanup:
@ -176,11 +191,13 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
received.value() == greeting received.value() == greeting
received.key() == null received.key() == null
assertTraces(1) { assertTraces(2) {
traces.sort(orderByRootSpanKind(INTERNAL, CONSUMER))
trace(0, 4) { trace(0, 4) {
span(0) { span(0) {
name "parent" name "parent"
kind SpanKind.INTERNAL kind INTERNAL
hasNoParent() hasNoParent()
} }
span(1) { span(1) {
@ -210,10 +227,23 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
} }
span(3) { span(3) {
name "producer callback" name "producer callback"
kind SpanKind.INTERNAL kind INTERNAL
childOf span(0) childOf span(0)
} }
} }
trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
cleanup: cleanup:
@ -265,7 +295,9 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
received.value() == null received.value() == null
received.key() == null received.key() == null
assertTraces(1) { assertTraces(2) {
traces.sort(orderByRootSpanKind(PRODUCER, CONSUMER))
trace(0, 2) { trace(0, 2) {
// PRODUCER span 0 // PRODUCER span 0
span(0) { span(0) {
@ -297,6 +329,19 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
} }
} }
} }
trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
cleanup: cleanup:
@ -339,7 +384,9 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
first.value() == greeting first.value() == greeting
first.key() == null first.key() == null
assertTraces(1) { assertTraces(2) {
traces.sort(orderByRootSpanKind(PRODUCER, CONSUMER))
trace(0, 2) { trace(0, 2) {
span(0) { span(0) {
name SHARED_TOPIC + " send" name SHARED_TOPIC + " send"
@ -368,6 +415,19 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
} }
} }
} }
trace(1, 1) {
span(0) {
name SHARED_TOPIC + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" SHARED_TOPIC
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
cleanup: cleanup:

View File

@ -140,7 +140,12 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
received.value() == greeting.toLowerCase() received.value() == greeting.toLowerCase()
received.key() == null received.key() == null
assertTraces(1) { assertTraces(3) {
traces.sort(orderByRootSpanName(
STREAM_PENDING + " send",
STREAM_PENDING + " receive",
STREAM_PROCESSED + " receive"))
trace(0, 5) { trace(0, 5) {
// PRODUCER span 0 // PRODUCER span 0
span(0) { span(0) {
@ -213,6 +218,32 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
} }
} }
} }
trace(1, 1) {
span(0) {
name STREAM_PENDING + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" STREAM_PENDING
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
trace(2, 1) {
span(0) {
name STREAM_PROCESSED + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" STREAM_PROCESSED
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
def headers = received.headers() def headers = received.headers()
@ -233,7 +264,8 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
} }
}) })
def spanContext = Span.fromContext(context).getSpanContext() def spanContext = Span.fromContext(context).getSpanContext()
def streamSendSpan = traces[0][3] def streamTrace = traces.find { it.size() == 5 }
def streamSendSpan = streamTrace[3]
spanContext.traceId == streamSendSpan.traceId spanContext.traceId == streamSendSpan.traceId
spanContext.spanId == streamSendSpan.spanId spanContext.spanId == streamSendSpan.spanId

View File

@ -140,7 +140,12 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
received.value() == greeting.toLowerCase() received.value() == greeting.toLowerCase()
received.key() == null received.key() == null
assertTraces(1) { assertTraces(3) {
traces.sort(orderByRootSpanName(
STREAM_PENDING + " send",
STREAM_PENDING + " receive",
STREAM_PROCESSED + " receive"))
trace(0, 5) { trace(0, 5) {
// PRODUCER span 0 // PRODUCER span 0
span(0) { span(0) {
@ -213,6 +218,32 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
} }
} }
} }
trace(1, 1) {
span(0) {
name STREAM_PENDING + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" STREAM_PENDING
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
trace(2, 1) {
span(0) {
name STREAM_PROCESSED + " receive"
kind CONSUMER
hasNoParent()
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" STREAM_PROCESSED
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
}
}
}
} }
def headers = received.headers() def headers = received.headers()
@ -233,7 +264,8 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
} }
}) })
def spanContext = Span.fromContext(context).getSpanContext() def spanContext = Span.fromContext(context).getSpanContext()
def streamSendSpan = traces[0][3] def streamTrace = traces.find { it.size() == 5 }
def streamSendSpan = streamTrace[3]
spanContext.traceId == streamSendSpan.traceId spanContext.traceId == streamSendSpan.traceId
spanContext.spanId == streamSendSpan.spanId spanContext.spanId == streamSendSpan.spanId

View File

@ -1,33 +0,0 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/
package io.opentelemetry.javaagent.instrumentation.spring.kafka;
import com.google.auto.value.AutoValue;
import io.opentelemetry.api.trace.SpanContext;
import io.opentelemetry.instrumentation.api.instrumenter.SpanLinksExtractor;
import java.util.List;
import org.apache.kafka.clients.consumer.ConsumerRecords;
@AutoValue
public abstract class BatchRecords<K, V> {
public static <K, V> BatchRecords<K, V> create(
ConsumerRecords<K, V> consumerRecords, List<SpanContext> linkedReceiveSpans) {
return new AutoValue_BatchRecords<>(consumerRecords, linkedReceiveSpans);
}
public abstract ConsumerRecords<K, V> records();
public abstract List<SpanContext> linkedReceiveSpans();
public static SpanLinksExtractor<BatchRecords<?, ?>> spanLinksExtractor() {
return (spanLinks, parentContext, batchRecords) -> {
batchRecords.linkedReceiveSpans().forEach(spanLinks::addLink);
};
}
BatchRecords() {}
}

View File

@ -6,19 +6,11 @@
package io.opentelemetry.javaagent.instrumentation.spring.kafka; package io.opentelemetry.javaagent.instrumentation.spring.kafka;
import static io.opentelemetry.javaagent.instrumentation.spring.kafka.SpringKafkaSingletons.processInstrumenter; import static io.opentelemetry.javaagent.instrumentation.spring.kafka.SpringKafkaSingletons.processInstrumenter;
import static io.opentelemetry.javaagent.instrumentation.spring.kafka.SpringKafkaSingletons.receiveInstrumenter;
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.api.trace.SpanContext;
import io.opentelemetry.context.Context; import io.opentelemetry.context.Context;
import io.opentelemetry.context.Scope; import io.opentelemetry.context.Scope;
import io.opentelemetry.javaagent.instrumentation.api.ContextStore; import io.opentelemetry.javaagent.instrumentation.api.ContextStore;
import io.opentelemetry.javaagent.instrumentation.kafka.KafkaConsumerIteratorWrapper;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.checker.nullness.qual.Nullable;
import org.springframework.kafka.listener.BatchInterceptor; import org.springframework.kafka.listener.BatchInterceptor;
@ -35,48 +27,18 @@ public final class InstrumentedBatchInterceptor<K, V> implements BatchIntercepto
} }
@Override @Override
public ConsumerRecords<K, V> intercept( public ConsumerRecords<K, V> intercept(ConsumerRecords<K, V> records, Consumer<K, V> consumer) {
ConsumerRecords<K, V> consumerRecords, Consumer<K, V> consumer) {
// TODO: use the receive spanContext that's linked to records
Context parentContext = Context.current(); Context parentContext = Context.current();
// create spans for all records received in a batch if (processInstrumenter().shouldStart(parentContext, records)) {
List<SpanContext> receiveSpanContexts = traceReceivingRecords(parentContext, consumerRecords); Context context = processInstrumenter().start(parentContext, records);
// then start a span for processing that links all those receive spans
BatchRecords<K, V> batchRecords = BatchRecords.create(consumerRecords, receiveSpanContexts);
if (processInstrumenter().shouldStart(parentContext, batchRecords)) {
Context context = processInstrumenter().start(parentContext, batchRecords);
Scope scope = context.makeCurrent(); Scope scope = context.makeCurrent();
contextStore.put(consumerRecords, State.create(batchRecords, context, scope)); contextStore.put(records, State.create(records, context, scope));
} }
return decorated == null ? consumerRecords : decorated.intercept(consumerRecords, consumer); return decorated == null ? records : decorated.intercept(records, consumer);
}
private List<SpanContext> traceReceivingRecords(
Context parentContext, ConsumerRecords<K, V> records) {
List<SpanContext> receiveSpanContexts = new ArrayList<>();
Iterator<ConsumerRecord<K, V>> it = records.iterator();
// this will forcefully suppress the kafka-clients CONSUMER instrumentation even though there's
// no current CONSUMER span
// this instrumentation will create CONSUMER receive spans for each record instead of
// kafka-clients
if (it instanceof KafkaConsumerIteratorWrapper) {
it = ((KafkaConsumerIteratorWrapper<K, V>) it).unwrap();
}
while (it.hasNext()) {
ConsumerRecord<K, V> record = it.next();
if (receiveInstrumenter().shouldStart(parentContext, record)) {
Context context = receiveInstrumenter().start(parentContext, record);
receiveSpanContexts.add(Span.fromContext(context).getSpanContext());
receiveInstrumenter().end(context, record, null, null);
}
}
return receiveSpanContexts;
} }
@Override @Override

View File

@ -0,0 +1,81 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/
package io.opentelemetry.javaagent.instrumentation.spring.kafka;
import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessageOperation;
import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessagingAttributesExtractor;
import io.opentelemetry.semconv.trace.attributes.SemanticAttributes;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.common.TopicPartition;
import org.checkerframework.checker.nullness.qual.Nullable;
public final class KafkaBatchProcessAttributesExtractor
extends MessagingAttributesExtractor<ConsumerRecords<?, ?>, Void> {
@Override
protected String system(ConsumerRecords<?, ?> records) {
return "kafka";
}
@Override
protected String destinationKind(ConsumerRecords<?, ?> records) {
return SemanticAttributes.MessagingDestinationKindValues.TOPIC;
}
@Override
protected @Nullable String destination(ConsumerRecords<?, ?> records) {
Set<String> topics =
records.partitions().stream().map(TopicPartition::topic).collect(Collectors.toSet());
// only return topic when there's exactly one in the batch
return topics.size() == 1 ? topics.iterator().next() : null;
}
@Override
protected boolean temporaryDestination(ConsumerRecords<?, ?> records) {
return false;
}
@Override
protected @Nullable String protocol(ConsumerRecords<?, ?> records) {
return null;
}
@Override
protected @Nullable String protocolVersion(ConsumerRecords<?, ?> records) {
return null;
}
@Override
protected @Nullable String url(ConsumerRecords<?, ?> records) {
return null;
}
@Override
protected @Nullable String conversationId(ConsumerRecords<?, ?> records) {
return null;
}
@Override
protected @Nullable Long messagePayloadSize(ConsumerRecords<?, ?> records) {
return null;
}
@Override
protected @Nullable Long messagePayloadCompressedSize(ConsumerRecords<?, ?> records) {
return null;
}
@Override
protected MessageOperation operation(ConsumerRecords<?, ?> records) {
return MessageOperation.PROCESS;
}
@Override
protected @Nullable String messageId(ConsumerRecords<?, ?> records, @Nullable Void unused) {
return null;
}
}

View File

@ -0,0 +1,49 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/
package io.opentelemetry.javaagent.instrumentation.spring.kafka;
import io.opentelemetry.context.Context;
import io.opentelemetry.context.propagation.ContextPropagators;
import io.opentelemetry.instrumentation.api.instrumenter.SpanLinksBuilder;
import io.opentelemetry.instrumentation.api.instrumenter.SpanLinksExtractor;
import io.opentelemetry.javaagent.instrumentation.kafka.KafkaConsumerIteratorWrapper;
import io.opentelemetry.javaagent.instrumentation.kafka.KafkaHeadersGetter;
import java.util.Iterator;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
public class KafkaBatchProcessSpanLinksExtractor
implements SpanLinksExtractor<ConsumerRecords<?, ?>> {
private final SpanLinksExtractor<ConsumerRecord<?, ?>> singleRecordLinkExtractor;
public KafkaBatchProcessSpanLinksExtractor(ContextPropagators contextPropagators) {
this.singleRecordLinkExtractor =
SpanLinksExtractor.fromUpstreamRequest(contextPropagators, new KafkaHeadersGetter());
}
@Override
public void extract(
SpanLinksBuilder spanLinks, Context parentContext, ConsumerRecords<?, ?> records) {
Iterator<? extends ConsumerRecord<?, ?>> it = records.iterator();
// this will forcefully suppress the kafka-clients CONSUMER instrumentation even though there's
// no current CONSUMER span
// this instrumentation will create CONSUMER receive spans for each record instead of
// kafka-clients
if (it instanceof KafkaConsumerIteratorWrapper) {
it = ((KafkaConsumerIteratorWrapper<?, ?>) it).unwrap();
}
while (it.hasNext()) {
ConsumerRecord<?, ?> record = it.next();
// explicitly passing root to avoid situation where context propagation is turned off and the
// parent (CONSUMER receive) span is linked
singleRecordLinkExtractor.extract(spanLinks, Context.root(), record);
}
}
}

View File

@ -7,63 +7,39 @@ package io.opentelemetry.javaagent.instrumentation.spring.kafka;
import io.opentelemetry.api.GlobalOpenTelemetry; import io.opentelemetry.api.GlobalOpenTelemetry;
import io.opentelemetry.instrumentation.api.instrumenter.Instrumenter; import io.opentelemetry.instrumentation.api.instrumenter.Instrumenter;
import io.opentelemetry.instrumentation.api.instrumenter.InstrumenterBuilder;
import io.opentelemetry.instrumentation.api.instrumenter.SpanKindExtractor; import io.opentelemetry.instrumentation.api.instrumenter.SpanKindExtractor;
import io.opentelemetry.instrumentation.api.instrumenter.SpanNameExtractor; import io.opentelemetry.instrumentation.api.instrumenter.SpanNameExtractor;
import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessageOperation;
import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessagingSpanNameExtractor; import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessagingSpanNameExtractor;
import io.opentelemetry.javaagent.instrumentation.kafka.KafkaConsumerAdditionalAttributesExtractor;
import io.opentelemetry.javaagent.instrumentation.kafka.KafkaConsumerAttributesExtractor;
import io.opentelemetry.javaagent.instrumentation.kafka.KafkaConsumerExperimentalAttributesExtractor;
import io.opentelemetry.javaagent.instrumentation.kafka.KafkaHeadersGetter;
import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
public final class SpringKafkaSingletons { public final class SpringKafkaSingletons {
private static final String INSTRUMENTATION_NAME = "io.opentelemetry.spring-kafka-2.7"; private static final String INSTRUMENTATION_NAME = "io.opentelemetry.spring-kafka-2.7";
private static final Instrumenter<ConsumerRecord<?, ?>, Void> RECEIVE_INSTRUMENTER = private static final Instrumenter<ConsumerRecords<?, ?>, Void> PROCESS_INSTRUMENTER =
buildReceiveInstrumenter();
private static final Instrumenter<BatchRecords<?, ?>, Void> PROCESS_INSTRUMENTER =
buildProcessInstrumenter(); buildProcessInstrumenter();
private static Instrumenter<ConsumerRecord<?, ?>, Void> buildReceiveInstrumenter() { private static Instrumenter<ConsumerRecords<?, ?>, Void> buildProcessInstrumenter() {
KafkaConsumerAttributesExtractor consumerAttributesExtractor = KafkaBatchProcessAttributesExtractor attributesExtractor =
new KafkaConsumerAttributesExtractor(MessageOperation.RECEIVE); new KafkaBatchProcessAttributesExtractor();
SpanNameExtractor<ConsumerRecord<?, ?>> spanNameExtractor = SpanNameExtractor<ConsumerRecords<?, ?>> spanNameExtractor =
MessagingSpanNameExtractor.create(consumerAttributesExtractor);
InstrumenterBuilder<ConsumerRecord<?, ?>, Void> builder =
Instrumenter.<ConsumerRecord<?, ?>, Void>newBuilder(
GlobalOpenTelemetry.get(), INSTRUMENTATION_NAME, spanNameExtractor)
.addAttributesExtractor(consumerAttributesExtractor)
.addAttributesExtractor(new KafkaConsumerAdditionalAttributesExtractor());
if (KafkaConsumerExperimentalAttributesExtractor.isEnabled()) {
builder.addAttributesExtractor(new KafkaConsumerExperimentalAttributesExtractor());
}
return builder.newConsumerInstrumenter(new KafkaHeadersGetter());
}
private static Instrumenter<BatchRecords<?, ?>, Void> buildProcessInstrumenter() {
BatchConsumerAttributesExtractor attributesExtractor = new BatchConsumerAttributesExtractor();
SpanNameExtractor<BatchRecords<?, ?>> spanNameExtractor =
MessagingSpanNameExtractor.create(attributesExtractor); MessagingSpanNameExtractor.create(attributesExtractor);
return Instrumenter.<BatchRecords<?, ?>, Void>newBuilder( return Instrumenter.<ConsumerRecords<?, ?>, Void>newBuilder(
GlobalOpenTelemetry.get(), INSTRUMENTATION_NAME, spanNameExtractor) GlobalOpenTelemetry.get(), INSTRUMENTATION_NAME, spanNameExtractor)
.addAttributesExtractor(attributesExtractor) .addAttributesExtractor(attributesExtractor)
.addSpanLinksExtractor(BatchRecords.spanLinksExtractor()) .addSpanLinksExtractor(
new KafkaBatchProcessSpanLinksExtractor(GlobalOpenTelemetry.getPropagators()))
.setErrorCauseExtractor(new KafkaBatchErrorCauseExtractor()) .setErrorCauseExtractor(new KafkaBatchErrorCauseExtractor())
.newInstrumenter(SpanKindExtractor.alwaysConsumer()); .newInstrumenter(SpanKindExtractor.alwaysConsumer());
} }
public static Instrumenter<ConsumerRecord<?, ?>, Void> receiveInstrumenter() { public static Instrumenter<ConsumerRecord<?, ?>, Void> receiveInstrumenter() {
return RECEIVE_INSTRUMENTER; return null;
} }
public static Instrumenter<BatchRecords<?, ?>, Void> processInstrumenter() { public static Instrumenter<ConsumerRecords<?, ?>, Void> processInstrumenter() {
return PROCESS_INSTRUMENTER; return PROCESS_INSTRUMENTER;
} }

View File

@ -8,16 +8,17 @@ package io.opentelemetry.javaagent.instrumentation.spring.kafka;
import com.google.auto.value.AutoValue; import com.google.auto.value.AutoValue;
import io.opentelemetry.context.Context; import io.opentelemetry.context.Context;
import io.opentelemetry.context.Scope; import io.opentelemetry.context.Scope;
import org.apache.kafka.clients.consumer.ConsumerRecords;
@AutoValue @AutoValue
public abstract class State<K, V> { public abstract class State<K, V> {
public static <K, V> State<K, V> create( public static <K, V> State<K, V> create(
BatchRecords<K, V> request, Context context, Scope scope) { ConsumerRecords<K, V> request, Context context, Scope scope) {
return new AutoValue_State<>(request, context, scope); return new AutoValue_State<>(request, context, scope);
} }
public abstract BatchRecords<K, V> request(); public abstract ConsumerRecords<K, V> request();
public abstract Context context(); public abstract Context context();

View File

@ -69,10 +69,12 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
} }
then: then:
assertTraces(2) { assertTraces(3) {
SpanData consumer1, consumer2 traces.sort(orderByRootSpanName("producer", "testTopic receive", "testTopic process"))
trace(0, 5) { SpanData producer1, producer2
trace(0, 3) {
span(0) { span(0) {
name "producer" name "producer"
} }
@ -87,21 +89,6 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
} }
} }
span(2) { span(2) {
name "testTopic receive"
kind CONSUMER
childOf span(1)
attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" "testTopic"
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
"${SemanticAttributes.MESSAGING_MESSAGE_PAYLOAD_SIZE_BYTES.key}" Long
"${SemanticAttributes.MESSAGING_KAFKA_PARTITION.key}" 0
"kafka.offset" Long
"kafka.record.queue_time_ms" Long
}
}
span(3) {
name "testTopic send" name "testTopic send"
kind PRODUCER kind PRODUCER
childOf span(0) childOf span(0)
@ -111,31 +98,29 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic" "${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
} }
} }
span(4) {
producer1 = span(1)
producer2 = span(2)
}
trace(1, 1) {
span(0) {
name "testTopic receive" name "testTopic receive"
kind CONSUMER kind CONSUMER
childOf span(3) hasNoParent()
attributes { attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka" "${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" "testTopic" "${SemanticAttributes.MESSAGING_DESTINATION.key}" "testTopic"
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic" "${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive" "${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
"${SemanticAttributes.MESSAGING_MESSAGE_PAYLOAD_SIZE_BYTES.key}" Long
"${SemanticAttributes.MESSAGING_KAFKA_PARTITION.key}" 0
"kafka.offset" Long
"kafka.record.queue_time_ms" Long
} }
} }
consumer1 = span(2)
consumer2 = span(4)
} }
trace(1, 2) { trace(2, 2) {
span(0) { span(0) {
name "testTopic process" name "testTopic process"
kind CONSUMER kind CONSUMER
hasLink consumer1 hasLink producer1
hasLink consumer2 hasLink producer2
attributes { attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka" "${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" "testTopic" "${SemanticAttributes.MESSAGING_DESTINATION.key}" "testTopic"
@ -163,10 +148,12 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
} }
then: then:
assertTraces(2) { assertTraces(3) {
SpanData consumer traces.sort(orderByRootSpanName("producer", "testTopic receive", "testTopic process"))
trace(0, 3) { SpanData producer
trace(0, 2) {
span(0) { span(0) {
name "producer" name "producer"
} }
@ -180,29 +167,27 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic" "${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
} }
} }
span(2) {
producer = span(1)
}
trace(1, 1) {
span(0) {
name "testTopic receive" name "testTopic receive"
kind CONSUMER kind CONSUMER
childOf span(1) hasNoParent()
attributes { attributes {
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka" "${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
"${SemanticAttributes.MESSAGING_DESTINATION.key}" "testTopic" "${SemanticAttributes.MESSAGING_DESTINATION.key}" "testTopic"
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic" "${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
"${SemanticAttributes.MESSAGING_OPERATION.key}" "receive" "${SemanticAttributes.MESSAGING_OPERATION.key}" "receive"
"${SemanticAttributes.MESSAGING_MESSAGE_PAYLOAD_SIZE_BYTES.key}" Long
"${SemanticAttributes.MESSAGING_KAFKA_PARTITION.key}" 0
"kafka.offset" Long
"kafka.record.queue_time_ms" Long
} }
} }
consumer = span(2)
} }
trace(1, 2) { trace(2, 2) {
span(0) { span(0) {
name "testTopic process" name "testTopic process"
kind CONSUMER kind CONSUMER
hasLink consumer hasLink producer
status ERROR status ERROR
errorEvent IllegalArgumentException, "boom" errorEvent IllegalArgumentException, "boom"
attributes { attributes {