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:
parent
28e5cb5dd6
commit
e30b082259
|
@ -12,6 +12,9 @@ muzzle {
|
|||
}
|
||||
|
||||
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"))
|
||||
|
||||
library("org.apache.kafka:kafka-clients:0.11.0.0")
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,6 +20,9 @@ public class KafkaClientsInstrumentationModule extends InstrumentationModule {
|
|||
|
||||
@Override
|
||||
public List<TypeInstrumentation> typeInstrumentations() {
|
||||
return asList(new KafkaConsumerInstrumentation(), new KafkaProducerInstrumentation());
|
||||
return asList(
|
||||
new KafkaProducerInstrumentation(),
|
||||
new KafkaConsumerInstrumentation(),
|
||||
new ConsumerRecordsInstrumentation());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,86 +5,76 @@
|
|||
|
||||
package io.opentelemetry.javaagent.instrumentation.kafkaclients;
|
||||
|
||||
import static io.opentelemetry.javaagent.instrumentation.kafkaclients.KafkaSingletons.consumerInstrumenter;
|
||||
import static net.bytebuddy.matcher.ElementMatchers.isMethod;
|
||||
import static io.opentelemetry.javaagent.instrumentation.api.Java8BytecodeBridge.currentContext;
|
||||
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.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.context.Context;
|
||||
import io.opentelemetry.javaagent.extension.instrumentation.TypeInstrumentation;
|
||||
import io.opentelemetry.javaagent.extension.instrumentation.TypeTransformer;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import io.opentelemetry.javaagent.instrumentation.api.ContextStore;
|
||||
import io.opentelemetry.javaagent.instrumentation.api.InstrumentationContext;
|
||||
import java.time.Duration;
|
||||
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 KafkaConsumerInstrumentation implements TypeInstrumentation {
|
||||
|
||||
@Override
|
||||
public ElementMatcher<TypeDescription> typeMatcher() {
|
||||
return named("org.apache.kafka.clients.consumer.ConsumerRecords");
|
||||
return named("org.apache.kafka.clients.consumer.KafkaConsumer");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void transform(TypeTransformer transformer) {
|
||||
transformer.applyAdviceToMethod(
|
||||
isMethod()
|
||||
named("poll")
|
||||
.and(isPublic())
|
||||
.and(named("records"))
|
||||
.and(takesArgument(0, String.class))
|
||||
.and(returns(Iterable.class)),
|
||||
KafkaConsumerInstrumentation.class.getName() + "$IterableAdvice");
|
||||
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");
|
||||
.and(takesArguments(1))
|
||||
.and(takesArgument(0, long.class).or(takesArgument(0, Duration.class)))
|
||||
.and(returns(named("org.apache.kafka.clients.consumer.ConsumerRecords"))),
|
||||
this.getClass().getName() + "$PollAdvice");
|
||||
}
|
||||
|
||||
@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) {
|
||||
iterable = new TracingIterable(iterable);
|
||||
}
|
||||
public static class PollAdvice {
|
||||
@Advice.OnMethodEnter(suppress = Throwable.class)
|
||||
public static Timer onEnter() {
|
||||
return Timer.start();
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public static class ListAdvice {
|
||||
@Advice.OnMethodExit(suppress = Throwable.class, onThrowable = Throwable.class)
|
||||
public static void onExit(
|
||||
@Advice.Enter Timer timer,
|
||||
@Advice.Return ConsumerRecords<?, ?> records,
|
||||
@Advice.Thrown Throwable error) {
|
||||
|
||||
@Advice.OnMethodExit(suppress = Throwable.class)
|
||||
public static void wrap(@Advice.Return(readOnly = false) List<ConsumerRecord<?, ?>> iterable) {
|
||||
if (iterable != null) {
|
||||
iterable = new TracingList(iterable);
|
||||
// don't create spans when no records were received
|
||||
if (records == null || records.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public static class IteratorAdvice {
|
||||
Context parentContext = currentContext();
|
||||
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)
|
||||
public static void wrap(
|
||||
@Advice.Return(readOnly = false) Iterator<ConsumerRecord<?, ?>> iterator) {
|
||||
if (iterator != null) {
|
||||
iterator = new TracingIterator(iterator, consumerInstrumenter());
|
||||
// we're storing the context of the receive span so that process spans can use it as parent
|
||||
// context even though the span has ended
|
||||
// this is the suggested behavior according to the spec batch receive scenario:
|
||||
// https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/trace/semantic_conventions/messaging.md#batch-receiving
|
||||
ContextStore<ConsumerRecords, SpanContext> consumerRecordsSpan =
|
||||
InstrumentationContext.get(ConsumerRecords.class, SpanContext.class);
|
||||
consumerRecordsSpan.put(records, spanFromContext(context).getSpanContext());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -3,7 +3,7 @@
|
|||
* 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.MessagingAttributesExtractor;
|
||||
|
@ -13,22 +13,23 @@ import java.util.stream.Collectors;
|
|||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.checkerframework.checker.nullness.qual.Nullable;
|
||||
|
||||
public final class BatchConsumerAttributesExtractor
|
||||
extends MessagingAttributesExtractor<BatchRecords<?, ?>, Void> {
|
||||
public final class KafkaReceiveAttributesExtractor
|
||||
extends MessagingAttributesExtractor<ReceivedRecords, Void> {
|
||||
|
||||
@Override
|
||||
protected String system(BatchRecords<?, ?> batchRecords) {
|
||||
protected String system(ReceivedRecords receivedRecords) {
|
||||
return "kafka";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String destinationKind(BatchRecords<?, ?> batchRecords) {
|
||||
protected String destinationKind(ReceivedRecords receivedRecords) {
|
||||
return SemanticAttributes.MessagingDestinationKindValues.TOPIC;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected @Nullable String destination(BatchRecords<?, ?> batchRecords) {
|
||||
protected @Nullable String destination(ReceivedRecords receivedRecords) {
|
||||
Set<String> topics =
|
||||
batchRecords.records().partitions().stream()
|
||||
receivedRecords.records().partitions().stream()
|
||||
.map(TopicPartition::topic)
|
||||
.collect(Collectors.toSet());
|
||||
// only return topic when there's exactly one in the batch
|
||||
|
@ -36,47 +37,47 @@ public final class BatchConsumerAttributesExtractor
|
|||
}
|
||||
|
||||
@Override
|
||||
protected boolean temporaryDestination(BatchRecords<?, ?> batchRecords) {
|
||||
protected boolean temporaryDestination(ReceivedRecords receivedRecords) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected @Nullable String protocol(BatchRecords<?, ?> batchRecords) {
|
||||
protected @Nullable String protocol(ReceivedRecords receivedRecords) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected @Nullable String protocolVersion(BatchRecords<?, ?> batchRecords) {
|
||||
protected @Nullable String protocolVersion(ReceivedRecords receivedRecords) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected @Nullable String url(BatchRecords<?, ?> batchRecords) {
|
||||
protected @Nullable String url(ReceivedRecords receivedRecords) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected @Nullable String conversationId(BatchRecords<?, ?> batchRecords) {
|
||||
protected @Nullable String conversationId(ReceivedRecords receivedRecords) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected @Nullable Long messagePayloadSize(BatchRecords<?, ?> batchRecords) {
|
||||
protected @Nullable Long messagePayloadSize(ReceivedRecords receivedRecords) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected @Nullable Long messagePayloadCompressedSize(BatchRecords<?, ?> batchRecords) {
|
||||
protected @Nullable Long messagePayloadCompressedSize(ReceivedRecords receivedRecords) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected MessageOperation operation(BatchRecords<?, ?> batchRecords) {
|
||||
return MessageOperation.PROCESS;
|
||||
protected MessageOperation operation(ReceivedRecords receivedRecords) {
|
||||
return MessageOperation.RECEIVE;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected @Nullable String messageId(BatchRecords<?, ?> batchRecords, @Nullable Void unused) {
|
||||
protected @Nullable String messageId(ReceivedRecords receivedRecords, @Nullable Void unused) {
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -24,8 +24,10 @@ public final class KafkaSingletons {
|
|||
|
||||
private static final Instrumenter<ProducerRecord<?, ?>, Void> PRODUCER_INSTRUMENTER =
|
||||
buildProducerInstrumenter();
|
||||
private static final Instrumenter<ConsumerRecord<?, ?>, Void> CONSUMER_INSTRUMENTER =
|
||||
buildConsumerInstrumenter();
|
||||
private static final Instrumenter<ReceivedRecords, Void> CONSUMER_RECEIVE_INSTRUMENTER =
|
||||
buildConsumerReceiveInstrumenter();
|
||||
private static final Instrumenter<ConsumerRecord<?, ?>, Void> CONSUMER_PROCESS_INSTRUMENTER =
|
||||
buildConsumerProcessInstrumenter();
|
||||
|
||||
private static Instrumenter<ProducerRecord<?, ?>, Void> buildProducerInstrumenter() {
|
||||
KafkaProducerAttributesExtractor attributesExtractor = new KafkaProducerAttributesExtractor();
|
||||
|
@ -39,7 +41,19 @@ public final class KafkaSingletons {
|
|||
.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 =
|
||||
new KafkaConsumerAttributesExtractor(MessageOperation.PROCESS);
|
||||
SpanNameExtractor<ConsumerRecord<?, ?>> spanNameExtractor =
|
||||
|
@ -62,8 +76,12 @@ public final class KafkaSingletons {
|
|||
return PRODUCER_INSTRUMENTER;
|
||||
}
|
||||
|
||||
public static Instrumenter<ConsumerRecord<?, ?>, Void> consumerInstrumenter() {
|
||||
return CONSUMER_INSTRUMENTER;
|
||||
public static Instrumenter<ReceivedRecords, Void> consumerReceiveInstrumenter() {
|
||||
return CONSUMER_RECEIVE_INSTRUMENTER;
|
||||
}
|
||||
|
||||
public static Instrumenter<ConsumerRecord<?, ?>, Void> consumerProcessInstrumenter() {
|
||||
return CONSUMER_PROCESS_INSTRUMENTER;
|
||||
}
|
||||
|
||||
private KafkaSingletons() {}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -5,8 +5,6 @@
|
|||
|
||||
package io.opentelemetry.javaagent.instrumentation.kafkaclients;
|
||||
|
||||
import static io.opentelemetry.javaagent.instrumentation.kafkaclients.KafkaSingletons.consumerInstrumenter;
|
||||
|
||||
import java.util.Iterator;
|
||||
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
|
||||
// ConsumerRecords is performed in the same thread that called poll()
|
||||
if (firstIterator) {
|
||||
it = new TracingIterator<>(delegate.iterator(), consumerInstrumenter());
|
||||
it = new TracingIterator<>(delegate.iterator());
|
||||
firstIterator = false;
|
||||
} else {
|
||||
it = delegate.iterator();
|
||||
|
|
|
@ -5,9 +5,10 @@
|
|||
|
||||
package io.opentelemetry.javaagent.instrumentation.kafkaclients;
|
||||
|
||||
import static io.opentelemetry.javaagent.instrumentation.kafkaclients.KafkaSingletons.consumerProcessInstrumenter;
|
||||
|
||||
import io.opentelemetry.context.Context;
|
||||
import io.opentelemetry.context.Scope;
|
||||
import io.opentelemetry.instrumentation.api.instrumenter.Instrumenter;
|
||||
import io.opentelemetry.javaagent.instrumentation.kafka.KafkaConsumerIteratorWrapper;
|
||||
import java.util.Iterator;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||
|
@ -17,23 +18,22 @@ public class TracingIterator<K, V>
|
|||
implements Iterator<ConsumerRecord<K, V>>, KafkaConsumerIteratorWrapper<K, V> {
|
||||
|
||||
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;
|
||||
|
||||
/**
|
||||
/*
|
||||
* Note: this may potentially create problems if this iterator is used from different threads. But
|
||||
* at the moment we cannot do much about this.
|
||||
*/
|
||||
@Nullable private ConsumerRecord<?, ?> currentRequest;
|
||||
|
||||
@Nullable private Context currentContext;
|
||||
@Nullable private Scope currentScope;
|
||||
|
||||
public TracingIterator(
|
||||
Iterator<ConsumerRecord<K, V>> delegateIterator,
|
||||
Instrumenter<ConsumerRecord<?, ?>, Void> instrumenter) {
|
||||
public TracingIterator(Iterator<ConsumerRecord<K, V>> delegateIterator) {
|
||||
this.delegateIterator = delegateIterator;
|
||||
this.instrumenter = instrumenter;
|
||||
parentContext = Context.current();
|
||||
}
|
||||
|
||||
|
@ -49,9 +49,9 @@ public class TracingIterator<K, V>
|
|||
closeScopeAndEndSpan();
|
||||
|
||||
ConsumerRecord<K, V> next = delegateIterator.next();
|
||||
if (next != null && instrumenter.shouldStart(parentContext, next)) {
|
||||
if (next != null && consumerProcessInstrumenter().shouldStart(parentContext, next)) {
|
||||
currentRequest = next;
|
||||
currentContext = instrumenter.start(parentContext, currentRequest);
|
||||
currentContext = consumerProcessInstrumenter().start(parentContext, currentRequest);
|
||||
currentScope = currentContext.makeCurrent();
|
||||
}
|
||||
return next;
|
||||
|
@ -60,7 +60,7 @@ public class TracingIterator<K, V>
|
|||
private void closeScopeAndEndSpan() {
|
||||
if (currentScope != null) {
|
||||
currentScope.close();
|
||||
instrumenter.end(currentContext, currentRequest, null, null);
|
||||
consumerProcessInstrumenter().end(currentContext, currentRequest, null, null);
|
||||
currentScope = null;
|
||||
currentRequest = null;
|
||||
currentContext = null;
|
||||
|
|
|
@ -54,7 +54,7 @@ class KafkaClientPropagationDisabledTest extends KafkaClientBaseTest {
|
|||
// check that the message was received
|
||||
records.poll(5, TimeUnit.SECONDS) != null
|
||||
|
||||
assertTraces(2) {
|
||||
assertTraces(3) {
|
||||
trace(0, 1) {
|
||||
span(0) {
|
||||
name SHARED_TOPIC + " send"
|
||||
|
@ -68,6 +68,19 @@ class KafkaClientPropagationDisabledTest 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"
|
||||
}
|
||||
}
|
||||
}
|
||||
trace(2, 1) {
|
||||
span(0) {
|
||||
name SHARED_TOPIC + " process"
|
||||
kind CONSUMER
|
||||
|
@ -84,7 +97,6 @@ class KafkaClientPropagationDisabledTest extends KafkaClientBaseTest {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
cleanup:
|
||||
|
|
|
@ -4,9 +4,9 @@
|
|||
*/
|
||||
|
||||
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 io.opentelemetry.api.trace.SpanKind
|
||||
import io.opentelemetry.semconv.trace.attributes.SemanticAttributes
|
||||
import java.util.concurrent.LinkedBlockingQueue
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
@ -80,11 +80,13 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
received.value() == greeting
|
||||
received.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(2) {
|
||||
traces.sort(orderByRootSpanKind(INTERNAL, CONSUMER))
|
||||
|
||||
trace(0, 4) {
|
||||
span(0) {
|
||||
name "parent"
|
||||
kind SpanKind.INTERNAL
|
||||
kind INTERNAL
|
||||
hasNoParent()
|
||||
}
|
||||
span(1) {
|
||||
|
@ -114,10 +116,23 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
}
|
||||
span(3) {
|
||||
name "producer callback"
|
||||
kind SpanKind.INTERNAL
|
||||
kind INTERNAL
|
||||
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:
|
||||
|
@ -176,11 +191,13 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
received.value() == greeting
|
||||
received.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(2) {
|
||||
traces.sort(orderByRootSpanKind(INTERNAL, CONSUMER))
|
||||
|
||||
trace(0, 4) {
|
||||
span(0) {
|
||||
name "parent"
|
||||
kind SpanKind.INTERNAL
|
||||
kind INTERNAL
|
||||
hasNoParent()
|
||||
}
|
||||
span(1) {
|
||||
|
@ -210,10 +227,23 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
}
|
||||
span(3) {
|
||||
name "producer callback"
|
||||
kind SpanKind.INTERNAL
|
||||
kind INTERNAL
|
||||
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:
|
||||
|
@ -265,7 +295,9 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
received.value() == null
|
||||
received.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(2) {
|
||||
traces.sort(orderByRootSpanKind(PRODUCER, CONSUMER))
|
||||
|
||||
trace(0, 2) {
|
||||
// PRODUCER 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:
|
||||
|
@ -339,7 +384,9 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
first.value() == greeting
|
||||
first.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(2) {
|
||||
traces.sort(orderByRootSpanKind(PRODUCER, CONSUMER))
|
||||
|
||||
trace(0, 2) {
|
||||
span(0) {
|
||||
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:
|
||||
|
|
|
@ -54,7 +54,7 @@ class KafkaClientPropagationDisabledTest extends KafkaClientBaseTest {
|
|||
// check that the message was received
|
||||
records.poll(5, TimeUnit.SECONDS) != null
|
||||
|
||||
assertTraces(2) {
|
||||
assertTraces(3) {
|
||||
trace(0, 1) {
|
||||
span(0) {
|
||||
name SHARED_TOPIC + " send"
|
||||
|
@ -68,6 +68,19 @@ class KafkaClientPropagationDisabledTest 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"
|
||||
}
|
||||
}
|
||||
}
|
||||
trace(2, 1) {
|
||||
span(0) {
|
||||
name SHARED_TOPIC + " process"
|
||||
kind CONSUMER
|
||||
|
|
|
@ -4,9 +4,9 @@
|
|||
*/
|
||||
|
||||
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 io.opentelemetry.api.trace.SpanKind
|
||||
import io.opentelemetry.semconv.trace.attributes.SemanticAttributes
|
||||
import java.util.concurrent.LinkedBlockingQueue
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
@ -80,11 +80,13 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
received.value() == greeting
|
||||
received.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(2) {
|
||||
traces.sort(orderByRootSpanKind(INTERNAL, CONSUMER))
|
||||
|
||||
trace(0, 4) {
|
||||
span(0) {
|
||||
name "parent"
|
||||
kind SpanKind.INTERNAL
|
||||
kind INTERNAL
|
||||
hasNoParent()
|
||||
}
|
||||
span(1) {
|
||||
|
@ -114,10 +116,23 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
}
|
||||
span(3) {
|
||||
name "producer callback"
|
||||
kind SpanKind.INTERNAL
|
||||
kind INTERNAL
|
||||
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:
|
||||
|
@ -176,11 +191,13 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
received.value() == greeting
|
||||
received.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(2) {
|
||||
traces.sort(orderByRootSpanKind(INTERNAL, CONSUMER))
|
||||
|
||||
trace(0, 4) {
|
||||
span(0) {
|
||||
name "parent"
|
||||
kind SpanKind.INTERNAL
|
||||
kind INTERNAL
|
||||
hasNoParent()
|
||||
}
|
||||
span(1) {
|
||||
|
@ -210,10 +227,23 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
}
|
||||
span(3) {
|
||||
name "producer callback"
|
||||
kind SpanKind.INTERNAL
|
||||
kind INTERNAL
|
||||
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:
|
||||
|
@ -265,7 +295,9 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
received.value() == null
|
||||
received.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(2) {
|
||||
traces.sort(orderByRootSpanKind(PRODUCER, CONSUMER))
|
||||
|
||||
trace(0, 2) {
|
||||
// PRODUCER 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:
|
||||
|
@ -339,7 +384,9 @@ class KafkaClientPropagationEnabledTest extends KafkaClientBaseTest {
|
|||
first.value() == greeting
|
||||
first.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(2) {
|
||||
traces.sort(orderByRootSpanKind(PRODUCER, CONSUMER))
|
||||
|
||||
trace(0, 2) {
|
||||
span(0) {
|
||||
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:
|
||||
|
|
|
@ -140,7 +140,12 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
|
|||
received.value() == greeting.toLowerCase()
|
||||
received.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(3) {
|
||||
traces.sort(orderByRootSpanName(
|
||||
STREAM_PENDING + " send",
|
||||
STREAM_PENDING + " receive",
|
||||
STREAM_PROCESSED + " receive"))
|
||||
|
||||
trace(0, 5) {
|
||||
// PRODUCER 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()
|
||||
|
@ -233,7 +264,8 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
|
|||
}
|
||||
})
|
||||
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.spanId == streamSendSpan.spanId
|
||||
|
||||
|
|
|
@ -140,7 +140,12 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
|
|||
received.value() == greeting.toLowerCase()
|
||||
received.key() == null
|
||||
|
||||
assertTraces(1) {
|
||||
assertTraces(3) {
|
||||
traces.sort(orderByRootSpanName(
|
||||
STREAM_PENDING + " send",
|
||||
STREAM_PENDING + " receive",
|
||||
STREAM_PROCESSED + " receive"))
|
||||
|
||||
trace(0, 5) {
|
||||
// PRODUCER 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()
|
||||
|
@ -233,7 +264,8 @@ class KafkaStreamsTest extends AgentInstrumentationSpecification {
|
|||
}
|
||||
})
|
||||
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.spanId == streamSendSpan.spanId
|
||||
|
||||
|
|
|
@ -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() {}
|
||||
}
|
|
@ -6,19 +6,11 @@
|
|||
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.receiveInstrumenter;
|
||||
|
||||
import io.opentelemetry.api.trace.Span;
|
||||
import io.opentelemetry.api.trace.SpanContext;
|
||||
import io.opentelemetry.context.Context;
|
||||
import io.opentelemetry.context.Scope;
|
||||
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.ConsumerRecord;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||
import org.checkerframework.checker.nullness.qual.Nullable;
|
||||
import org.springframework.kafka.listener.BatchInterceptor;
|
||||
|
@ -35,48 +27,18 @@ public final class InstrumentedBatchInterceptor<K, V> implements BatchIntercepto
|
|||
}
|
||||
|
||||
@Override
|
||||
public ConsumerRecords<K, V> intercept(
|
||||
ConsumerRecords<K, V> consumerRecords, Consumer<K, V> consumer) {
|
||||
public ConsumerRecords<K, V> intercept(ConsumerRecords<K, V> records, Consumer<K, V> consumer) {
|
||||
|
||||
// TODO: use the receive spanContext that's linked to records
|
||||
Context parentContext = Context.current();
|
||||
|
||||
// create spans for all records received in a batch
|
||||
List<SpanContext> receiveSpanContexts = traceReceivingRecords(parentContext, consumerRecords);
|
||||
|
||||
// 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);
|
||||
if (processInstrumenter().shouldStart(parentContext, records)) {
|
||||
Context context = processInstrumenter().start(parentContext, records);
|
||||
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);
|
||||
}
|
||||
|
||||
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;
|
||||
return decorated == null ? records : decorated.intercept(records, consumer);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -7,63 +7,39 @@ package io.opentelemetry.javaagent.instrumentation.spring.kafka;
|
|||
|
||||
import io.opentelemetry.api.GlobalOpenTelemetry;
|
||||
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.SpanNameExtractor;
|
||||
import io.opentelemetry.instrumentation.api.instrumenter.messaging.MessageOperation;
|
||||
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.ConsumerRecords;
|
||||
|
||||
public final class SpringKafkaSingletons {
|
||||
|
||||
private static final String INSTRUMENTATION_NAME = "io.opentelemetry.spring-kafka-2.7";
|
||||
|
||||
private static final Instrumenter<ConsumerRecord<?, ?>, Void> RECEIVE_INSTRUMENTER =
|
||||
buildReceiveInstrumenter();
|
||||
private static final Instrumenter<BatchRecords<?, ?>, Void> PROCESS_INSTRUMENTER =
|
||||
private static final Instrumenter<ConsumerRecords<?, ?>, Void> PROCESS_INSTRUMENTER =
|
||||
buildProcessInstrumenter();
|
||||
|
||||
private static Instrumenter<ConsumerRecord<?, ?>, Void> buildReceiveInstrumenter() {
|
||||
KafkaConsumerAttributesExtractor consumerAttributesExtractor =
|
||||
new KafkaConsumerAttributesExtractor(MessageOperation.RECEIVE);
|
||||
SpanNameExtractor<ConsumerRecord<?, ?>> 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 =
|
||||
private static Instrumenter<ConsumerRecords<?, ?>, Void> buildProcessInstrumenter() {
|
||||
KafkaBatchProcessAttributesExtractor attributesExtractor =
|
||||
new KafkaBatchProcessAttributesExtractor();
|
||||
SpanNameExtractor<ConsumerRecords<?, ?>> spanNameExtractor =
|
||||
MessagingSpanNameExtractor.create(attributesExtractor);
|
||||
|
||||
return Instrumenter.<BatchRecords<?, ?>, Void>newBuilder(
|
||||
return Instrumenter.<ConsumerRecords<?, ?>, Void>newBuilder(
|
||||
GlobalOpenTelemetry.get(), INSTRUMENTATION_NAME, spanNameExtractor)
|
||||
.addAttributesExtractor(attributesExtractor)
|
||||
.addSpanLinksExtractor(BatchRecords.spanLinksExtractor())
|
||||
.addSpanLinksExtractor(
|
||||
new KafkaBatchProcessSpanLinksExtractor(GlobalOpenTelemetry.getPropagators()))
|
||||
.setErrorCauseExtractor(new KafkaBatchErrorCauseExtractor())
|
||||
.newInstrumenter(SpanKindExtractor.alwaysConsumer());
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
|
|
|
@ -8,16 +8,17 @@ package io.opentelemetry.javaagent.instrumentation.spring.kafka;
|
|||
import com.google.auto.value.AutoValue;
|
||||
import io.opentelemetry.context.Context;
|
||||
import io.opentelemetry.context.Scope;
|
||||
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
||||
|
||||
@AutoValue
|
||||
public abstract class State<K, V> {
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
public abstract BatchRecords<K, V> request();
|
||||
public abstract ConsumerRecords<K, V> request();
|
||||
|
||||
public abstract Context context();
|
||||
|
||||
|
|
|
@ -69,10 +69,12 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
|
|||
}
|
||||
|
||||
then:
|
||||
assertTraces(2) {
|
||||
SpanData consumer1, consumer2
|
||||
assertTraces(3) {
|
||||
traces.sort(orderByRootSpanName("producer", "testTopic receive", "testTopic process"))
|
||||
|
||||
trace(0, 5) {
|
||||
SpanData producer1, producer2
|
||||
|
||||
trace(0, 3) {
|
||||
span(0) {
|
||||
name "producer"
|
||||
}
|
||||
|
@ -87,21 +89,6 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
|
|||
}
|
||||
}
|
||||
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"
|
||||
kind PRODUCER
|
||||
childOf span(0)
|
||||
|
@ -111,31 +98,29 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
|
|||
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
|
||||
}
|
||||
}
|
||||
span(4) {
|
||||
|
||||
producer1 = span(1)
|
||||
producer2 = span(2)
|
||||
}
|
||||
trace(1, 1) {
|
||||
span(0) {
|
||||
name "testTopic receive"
|
||||
kind CONSUMER
|
||||
childOf span(3)
|
||||
hasNoParent()
|
||||
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
|
||||
}
|
||||
}
|
||||
|
||||
consumer1 = span(2)
|
||||
consumer2 = span(4)
|
||||
}
|
||||
trace(1, 2) {
|
||||
trace(2, 2) {
|
||||
span(0) {
|
||||
name "testTopic process"
|
||||
kind CONSUMER
|
||||
hasLink consumer1
|
||||
hasLink consumer2
|
||||
hasLink producer1
|
||||
hasLink producer2
|
||||
attributes {
|
||||
"${SemanticAttributes.MESSAGING_SYSTEM.key}" "kafka"
|
||||
"${SemanticAttributes.MESSAGING_DESTINATION.key}" "testTopic"
|
||||
|
@ -163,10 +148,12 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
|
|||
}
|
||||
|
||||
then:
|
||||
assertTraces(2) {
|
||||
SpanData consumer
|
||||
assertTraces(3) {
|
||||
traces.sort(orderByRootSpanName("producer", "testTopic receive", "testTopic process"))
|
||||
|
||||
trace(0, 3) {
|
||||
SpanData producer
|
||||
|
||||
trace(0, 2) {
|
||||
span(0) {
|
||||
name "producer"
|
||||
}
|
||||
|
@ -180,29 +167,27 @@ class SpringKafkaInstrumentationTest extends AgentInstrumentationSpecification {
|
|||
"${SemanticAttributes.MESSAGING_DESTINATION_KIND.key}" "topic"
|
||||
}
|
||||
}
|
||||
span(2) {
|
||||
|
||||
producer = span(1)
|
||||
}
|
||||
trace(1, 1) {
|
||||
span(0) {
|
||||
name "testTopic receive"
|
||||
kind CONSUMER
|
||||
childOf span(1)
|
||||
hasNoParent()
|
||||
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
|
||||
}
|
||||
}
|
||||
|
||||
consumer = span(2)
|
||||
}
|
||||
trace(1, 2) {
|
||||
trace(2, 2) {
|
||||
span(0) {
|
||||
name "testTopic process"
|
||||
kind CONSUMER
|
||||
hasLink consumer
|
||||
hasLink producer
|
||||
status ERROR
|
||||
errorEvent IllegalArgumentException, "boom"
|
||||
attributes {
|
||||
|
|
Loading…
Reference in New Issue