Skip to content

Commit

Permalink
Add support for Kafka consumer and producer interceptors, move common…
Browse files Browse the repository at this point in the history
… Kafka code to library module.
  • Loading branch information
alesj committed Sep 15, 2021
1 parent d2794fa commit 49c2070
Show file tree
Hide file tree
Showing 36 changed files with 543 additions and 72 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -12,10 +12,7 @@ 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"))
implementation(project(":instrumentation:kafka-clients:kafka-clients-common:library"))

library("org.apache.kafka:kafka-clients:0.11.0.0")

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,9 @@
import static net.bytebuddy.matcher.ElementMatchers.takesArguments;

import io.opentelemetry.api.trace.SpanContext;
import io.opentelemetry.instrumentation.kafka.TracingIterable;
import io.opentelemetry.instrumentation.kafka.TracingIterator;
import io.opentelemetry.instrumentation.kafka.TracingList;
import io.opentelemetry.javaagent.extension.instrumentation.TypeInstrumentation;
import io.opentelemetry.javaagent.extension.instrumentation.TypeTransformer;
import io.opentelemetry.javaagent.instrumentation.api.ContextStore;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,9 +5,9 @@

package io.opentelemetry.javaagent.instrumentation.kafkaclients;

import static io.opentelemetry.instrumentation.kafka.KafkaSingletons.consumerReceiveInstrumenter;
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;
Expand All @@ -16,6 +16,8 @@

import io.opentelemetry.api.trace.SpanContext;
import io.opentelemetry.context.Context;
import io.opentelemetry.instrumentation.kafka.ReceivedRecords;
import io.opentelemetry.instrumentation.kafka.Timer;
import io.opentelemetry.javaagent.extension.instrumentation.TypeInstrumentation;
import io.opentelemetry.javaagent.extension.instrumentation.TypeTransformer;
import io.opentelemetry.javaagent.instrumentation.api.ContextStore;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,14 +5,16 @@

package io.opentelemetry.javaagent.instrumentation.kafkaclients;

import static io.opentelemetry.javaagent.instrumentation.kafkaclients.KafkaSingletons.producerInstrumenter;
import static io.opentelemetry.instrumentation.kafka.KafkaSingletons.producerInstrumenter;
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.takesArgument;

import io.opentelemetry.context.Context;
import io.opentelemetry.context.Scope;
import io.opentelemetry.instrumentation.kafka.KafkaPropagation;
import io.opentelemetry.instrumentation.kafka.ProducerCallback;
import io.opentelemetry.javaagent.extension.instrumentation.TypeInstrumentation;
import io.opentelemetry.javaagent.extension.instrumentation.TypeTransformer;
import io.opentelemetry.javaagent.instrumentation.api.Java8BytecodeBridge;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
plugins {
id("otel.library-instrumentation")
}

dependencies {
implementation(project(":instrumentation:kafka-clients:kafka-clients-common:library"))
compileOnly("org.apache.kafka:kafka-clients:0.11.0.0")

testLibrary("org.springframework.kafka:spring-kafka:2.4.0.RELEASE")
testLibrary("org.springframework.kafka:spring-kafka-test:2.4.0.RELEASE")
testLibrary("org.springframework:spring-core:5.2.9.RELEASE")
testImplementation("javax.xml.bind:jaxb-api:2.2.3")

latestDepTestLibrary("org.apache.kafka:kafka_2.13:+")
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/

package io.opentelemetry.instrumentation.kafkaclients;

import io.opentelemetry.api.trace.Span;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.producer.ProducerRecord;

public interface SpanDecorator {

/** Method called before record is sent by producer. */
<K, V> void onSend(ProducerRecord<K, V> record, Span span);

/** Method called when record is received in consumer. */
<K, V> void onResponse(ConsumerRecord<K, V> record, Span span);

/** Method called when an error occurs. */
<K, V> void onError(Exception exception, Span span);

/** Gives a SpanDecorator with the standard tags. */
SpanDecorator STANDARD_TAGS = new StandardSpanDecorator();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/

package io.opentelemetry.instrumentation.kafkaclients;

import io.opentelemetry.api.common.Attributes;
import io.opentelemetry.api.common.AttributesBuilder;
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.semconv.trace.attributes.SemanticAttributes;
import java.io.PrintWriter;
import java.io.StringWriter;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.producer.ProducerRecord;

class StandardSpanDecorator implements SpanDecorator {

static final String COMPONENT_NAME = "java-kafka";
static final String KAFKA_SERVICE = "kafka";

@Override
public <K, V> void onSend(ProducerRecord<K, V> record, Span span) {
setCommonTags(span);
span.setAttribute(SemanticAttributes.MESSAGING_DESTINATION, record.topic());
if (record.partition() != null) {
span.setAttribute("partition", record.partition());
}
}

@Override
public <K, V> void onResponse(ConsumerRecord<K, V> record, Span span) {
setCommonTags(span);
span.setAttribute(SemanticAttributes.MESSAGING_DESTINATION, record.topic());
span.setAttribute("partition", record.partition());
span.setAttribute("offset", record.offset());
}

@Override
public void onError(Exception exception, Span span) {
span.setAttribute(SemanticAttributes.EXCEPTION_ESCAPED, Boolean.TRUE);
span.setAllAttributes(errorLogs(exception));
}

private static Attributes errorLogs(Throwable throwable) {
AttributesBuilder errorLogs = Attributes.builder();
errorLogs.put("event", SemanticAttributes.EXCEPTION_TYPE.getKey());
errorLogs.put("error.kind", throwable.getClass().getName());
errorLogs.put("error.object", throwable.toString());
errorLogs.put("message", throwable.getMessage());

StringWriter sw = new StringWriter();
throwable.printStackTrace(new PrintWriter(sw));
errorLogs.put("stack", sw.toString());

return errorLogs.build();
}

private static void setCommonTags(Span span) {
span.setAttribute(SemanticAttributes.PEER_SERVICE, KAFKA_SERVICE);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/

package io.opentelemetry.instrumentation.kafkaclients;

import java.util.Map;
import org.apache.kafka.clients.consumer.ConsumerInterceptor;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;

public class TracingConsumerInterceptor<K, V> implements ConsumerInterceptor<K, V> {

@Override
public ConsumerRecords<K, V> onConsume(ConsumerRecords<K, V> records) {
for (ConsumerRecord<K, V> record : records) {
TracingKafkaUtils.buildAndFinishChildSpan(record);
}

return records;
}

@Override
public void onCommit(Map<TopicPartition, OffsetAndMetadata> offsets) {}

@Override
public void close() {}

@Override
public void configure(Map<String, ?> configs) {}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,128 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/

package io.opentelemetry.instrumentation.kafkaclients;

import static io.opentelemetry.instrumentation.kafka.KafkaSingletons.consumerProcessInstrumenter;
import static io.opentelemetry.instrumentation.kafka.KafkaSingletons.producerInstrumenter;

import io.opentelemetry.api.GlobalOpenTelemetry;
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Context;
import io.opentelemetry.context.Scope;
import io.opentelemetry.context.propagation.TextMapGetter;
import io.opentelemetry.context.propagation.TextMapPropagator;
import io.opentelemetry.context.propagation.TextMapSetter;
import io.opentelemetry.instrumentation.kafka.KafkaHeadersGetter;
import io.opentelemetry.instrumentation.kafka.KafkaHeadersSetter;
import java.util.Collection;
import java.util.Collections;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.header.Headers;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class TracingKafkaUtils {

private static final Logger logger = LoggerFactory.getLogger(TracingKafkaUtils.class);

private static final TextMapGetter<Headers> GETTER = new KafkaHeadersGetter();

private static final TextMapSetter<Headers> SETTER = new KafkaHeadersSetter();

private static TextMapPropagator propagator() {
return GlobalOpenTelemetry.getPropagators().getTextMapPropagator();
}

/**
* Extract Context from record headers.
*
* @param headers record headers
* @return context
*/
public static Context extractContext(Headers headers) {
return propagator().extract(Context.current(), headers, GETTER);
}

/**
* Inject current Context to record headers.
*
* @param headers record headers
*/
public static void inject(Headers headers) {
inject(Context.current(), headers);
}

/**
* Inject Context to record headers.
*
* @param context the context
* @param headers record headers
*/
public static void inject(Context context, Headers headers) {
propagator().inject(context, headers, SETTER);
}

/**
* Build and inject span into record. Return Runnable handle to end the current span.
*
* @param record the producer record to inject span info.
* @return runnable to close the current span
*/
public static <K, V> Runnable buildAndInjectSpan(ProducerRecord<K, V> record) {
return buildAndInjectSpan(record, Collections.singletonList(SpanDecorator.STANDARD_TAGS));
}

public static <K, V> Runnable buildAndInjectSpan(
ProducerRecord<K, V> record, Collection<SpanDecorator> spanDecorators) {

Context spanContext = extractContext(record.headers());

Context current = producerInstrumenter().start(spanContext, record);
try (Scope ignored = current.makeCurrent()) {

Span span = Span.fromContext(current);
for (SpanDecorator decorator : spanDecorators) {
decorator.onSend(record, span);
}

try {
inject(record.headers());
} catch (Throwable t) {
// it can happen if headers are read only (when record is sent second time)
logger.error("failed to inject span context. sending record second time?", t);
}
}

return () -> producerInstrumenter().end(current, record, null, null);
}

public static <K, V> void buildAndFinishChildSpan(ConsumerRecord<K, V> record) {
buildAndFinishChildSpan(record, Collections.singletonList(SpanDecorator.STANDARD_TAGS));
}

public static <K, V> void buildAndFinishChildSpan(
ConsumerRecord<K, V> record, Collection<SpanDecorator> spanDecorators) {

Context parentContext = extractContext(record.headers());
Context current = consumerProcessInstrumenter().start(parentContext, record);
try (Scope ignored = current.makeCurrent()) {

Span span = Span.fromContext(current);
for (SpanDecorator decorator : spanDecorators) {
decorator.onResponse(record, span);
}
} catch (RuntimeException e) {
consumerProcessInstrumenter().end(current, record, null, e);
throw e;
}
consumerProcessInstrumenter().end(current, record, null, null);

// Inject created span context into record headers for extraction by client to continue span
// chain
inject(current, record.headers()); // TODO -- OK?
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* Copyright The OpenTelemetry Authors
* SPDX-License-Identifier: Apache-2.0
*/

package io.opentelemetry.instrumentation.kafkaclients;

import java.util.Map;
import org.apache.kafka.clients.producer.ProducerInterceptor;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;

public class TracingProducerInterceptor<K, V> implements ProducerInterceptor<K, V> {
@Override
public ProducerRecord<K, V> onSend(ProducerRecord<K, V> producerRecord) {
TracingKafkaUtils.buildAndInjectSpan(producerRecord).run();
return producerRecord;
}

@Override
public void onAcknowledgement(RecordMetadata recordMetadata, Exception e) {}

@Override
public void close() {}

@Override
public void configure(Map<String, ?> map) {}
}
Loading

0 comments on commit 49c2070

Please sign in to comment.