Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Kafka Streams helper operations #833

Merged
merged 19 commits into from
Dec 11, 2018
Merged
Show file tree
Hide file tree
Changes from 11 commits
Commits
Show all changes
19 commits
Select commit Hold shift + click to select a range
30e01aa
feat: map, mapValues, foreach
jeqo Nov 25, 2018
c61c656
feat: peek and mark
jeqo Nov 25, 2018
5909158
feat: testing kstream methods
jeqo Nov 25, 2018
2a4a41d
feat: add annotation to test
jeqo Nov 25, 2018
cfb7915
refactor: peek and mark from processor to transformer
jeqo Nov 26, 2018
371e97b
docs: add javadocs to new methods
jeqo Nov 26, 2018
c37a359
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
adriancole Nov 27, 2018
fad5ba1
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
adriancole Nov 27, 2018
493df93
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
adriancole Nov 27, 2018
1f8b894
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
adriancole Nov 27, 2018
9d9ff72
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
adriancole Nov 27, 2018
a745f12
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
ImFlog Nov 27, 2018
8303c40
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
ImFlog Nov 27, 2018
1da53fb
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
ImFlog Nov 27, 2018
3c1f04b
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
ImFlog Nov 27, 2018
933d101
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
ImFlog Nov 27, 2018
85a4c4a
Update instrumentation/kafka-streams/src/main/java/brave/kafka/stream…
ImFlog Nov 27, 2018
9daac4a
docs: describe mark and rename name -> spanName
jeqo Nov 27, 2018
960ae7a
fix: typo
jeqo Nov 29, 2018
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
package brave.kafka.streams;

import org.apache.kafka.streams.kstream.Transformer;
import org.apache.kafka.streams.processor.ProcessorContext;

public abstract class AbstractTracingTransformer<K, V, R> implements
Transformer<K, V, R> {

@Override public void init(ProcessorContext context) {
}

@Override public void close() {
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
package brave.kafka.streams;

import org.apache.kafka.streams.kstream.ValueTransformer;
import org.apache.kafka.streams.processor.ProcessorContext;

public abstract class AbstractTracingValueTransformer<V, VR> implements
ValueTransformer<V, VR> {

@Override public void init(ProcessorContext context) {
}

@Override public void close() {
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
package brave.kafka.streams;

import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
import org.apache.kafka.streams.processor.ProcessorContext;

public abstract class AbstractTracingValueTransformerWithKey<K, V, VR> implements
ValueTransformerWithKey<K, V, VR> {

@Override public void init(ProcessorContext context) {
}

@Override public void close() {
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -7,16 +7,24 @@
import brave.propagation.TraceContext;
import brave.propagation.TraceContextOrSamplingFlags;
import java.util.Properties;
import java.util.function.BiConsumer;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.streams.KafkaClientSupplier;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.kstream.ForeachAction;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.KeyValueMapper;
import org.apache.kafka.streams.kstream.Transformer;
import org.apache.kafka.streams.kstream.TransformerSupplier;
import org.apache.kafka.streams.kstream.ValueMapper;
import org.apache.kafka.streams.kstream.ValueMapperWithKey;
import org.apache.kafka.streams.kstream.ValueTransformer;
import org.apache.kafka.streams.kstream.ValueTransformerSupplier;
import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
import org.apache.kafka.streams.processor.AbstractProcessor;
import org.apache.kafka.streams.processor.Processor;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.ProcessorSupplier;
Expand All @@ -38,8 +46,8 @@ public static KafkaStreamsTracing create(Tracing tracing) {

/**
* Creates a {@link KafkaStreams} instance with a tracing-enabled {@link KafkaClientSupplier}. All
* Topology Sources and Sinks (including internal Topics) will create Spans on records
* processed (i.e. send or consumed).
* Topology Sources and Sinks (including internal Topics) will create Spans on records processed
* (i.e. send or consumed).
*
* Use this instead of {@link KafkaStreams} constructor.
*
Expand Down Expand Up @@ -121,6 +129,132 @@ public <K, V, VR> ValueTransformerWithKeySupplier<K, V, VR> valueTransformerWith
return new TracingValueTransformerWithKeySupplier<>(this, name, valueTransformerWithKey);
}

/**
* Create a foreach processor, similar to {@link KStream#foreach(ForeachAction)}, where its action
* will be recorded in a new span with the indicated name.
*
* <p>Simple example using Kafka Streams DSL:
* <pre>{@code
* StreamsBuilder builder = new StreamsBuilder();
* builder.stream(inputTopic)
* .process(kafkaStraemsTracing.foreach("myForeach", (k, v) -> ...);
jeqo marked this conversation as resolved.
Show resolved Hide resolved
* }</pre>
*/
public <K, V> ProcessorSupplier<K, V> foreach(String name, ForeachAction<K, V> action) {
jeqo marked this conversation as resolved.
Show resolved Hide resolved
return new TracingProcessorSupplier<>(this, name, new AbstractProcessor<K, V>() {
@Override public void process(K key, V value) {
action.apply(key, value);
}
});
}

/**
* Create a peek transformer, similar to {@link KStream#peek(ForeachAction)}, where its action
* will be recorded in a new span with the indicated name.
*
* <p>Simple example using Kafka Streams DSL:
* <pre>{@code
* StreamsBuilder builder = new StreamsBuilder();
* builder.stream(inputTopic)
* .transform(kafkaStraemsTracing.peek("myPeek", (k, v) -> ...)
jeqo marked this conversation as resolved.
Show resolved Hide resolved
* .to(outputTopic);
* }</pre>
*/
public <K, V> TransformerSupplier<K, V, KeyValue<K, V>> peek(String name, ForeachAction<K, V> action) {
return new TracingTransformerSupplier<>(this, name, new AbstractTracingTransformer<K, V, KeyValue<K, V>>() {
@Override public KeyValue<K, V> transform(K key, V value) {
action.apply(key, value);
return KeyValue.pair(key, value);
}
});
}

/**
* Create a mark transformer, similar to {@link KStream#peek(ForeachAction)}, but no action is executed.
* Instead only a span is created to mark the beginning or end of a task.
jeqo marked this conversation as resolved.
Show resolved Hide resolved
*
* <p>Simple example using Kafka Streams DSL:
* <pre>{@code
* StreamsBuilder builder = new StreamsBuilder();
* builder.stream(inputTopic)
* .transform(kafkaStraemsTracing.mark("beginning")
jeqo marked this conversation as resolved.
Show resolved Hide resolved
* .map(mapper)
* .transform(kafkaStreamsTracing.mark("end")
* .to(outputTopic);
* }</pre>
*/
public <K, V> TransformerSupplier<K, V, KeyValue<K, V>> mark(String name) {
return new TracingTransformerSupplier<>(this, name, new AbstractTracingTransformer<K, V, KeyValue<K, V>>() {
@Override public KeyValue<K, V> transform(K key, V value) {
return KeyValue.pair(key, value);
}
});
}

/**
* Create a map transformer, similar to {@link KStream#map(KeyValueMapper)}, where its mapper action
* will be recorded in a new span with the indicated name.
*
* <p>Simple example using Kafka Streams DSL:
* <pre>{@code
* StreamsBuilder builder = new StreamsBuilder();
* builder.stream(inputTopic)
* .transform(kafkaStraemsTracing.map("myMap", (k, v) -> ...)
jeqo marked this conversation as resolved.
Show resolved Hide resolved
* .to(outputTopic);
* }</pre>
*/
public <K, V, KR, VR> TransformerSupplier<K, V, KeyValue<KR, VR>> map(String name,
KeyValueMapper<K, V, KeyValue<KR, VR>> mapper) {
return new TracingTransformerSupplier<>(this, name,
new AbstractTracingTransformer<K, V, KeyValue<KR, VR>>() {
@Override public KeyValue<KR, VR> transform(K key, V value) {
return mapper.apply(key, value);
}
});
}

/**
* Create a peek transformer, similar to {@link KStream#mapValues(ValueMapperWithKey)}, where its mapper action
* will be recorded in a new span with the indicated name.
*
* <p>Simple example using Kafka Streams DSL:
* <pre>{@code
* StreamsBuilder builder = new StreamsBuilder();
* builder.stream(inputTopic)
* .transform(kafkaStraemsTracing.mapValues("myMapValues", (k, v) -> ...)
jeqo marked this conversation as resolved.
Show resolved Hide resolved
* .to(outputTopic);
* }</pre>
*/
public <K, V, VR> ValueTransformerWithKeySupplier<K, V, VR> mapValues(String name,
ValueMapperWithKey<K, V, VR> mapper) {
return new TracingValueTransformerWithKeySupplier<>(this, name,
new AbstractTracingValueTransformerWithKey<K, V, VR>() {
@Override public VR transform(K readOnlyKey, V value) {
return mapper.apply(readOnlyKey, value);
}
});
}

/**
* Create a peek transformer, similar to {@link KStream#mapValues(ValueMapper)}, where its mapper action
* will be recorded in a new span with the indicated name.
*
* <p>Simple example using Kafka Streams DSL:
* <pre>{@code
* StreamsBuilder builder = new StreamsBuilder();
* builder.stream(inputTopic)
* .transform(kafkaStraemsTracing.mapValues("myMapValues", v -> ...)
jeqo marked this conversation as resolved.
Show resolved Hide resolved
* .to(outputTopic);
* }</pre>
*/
public <V, VR> ValueTransformerSupplier<V, VR> mapValues(String name, ValueMapper<V, VR> mapper) {
return new TracingValueTransformerSupplier<>(this, name, new AbstractTracingValueTransformer<V, VR>() {
@Override public VR transform(V value) {
return mapper.apply(value);
}
});
}

static void addTags(ProcessorContext processorContext, SpanCustomizer result) {
result.tag(KafkaStreamsTags.KAFKA_STREAMS_APPLICATION_ID_TAG, processorContext.applicationId());
result.tag(KafkaStreamsTags.KAFKA_STREAMS_TASK_ID_TAG, processorContext.taskId().toString());
Expand Down
Loading