-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-8410: Migrating stateful operators to new Processor API #10507
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
Changes from all commits
f7aaa63
bed7c96
9657601
9983c9c
a619816
19c8a8f
990ba66
fdaf408
ae995d2
a94eb30
ea67f68
c7be1c2
0c3cbc9
8baf291
662a10c
75a8f60
8bd1018
31f51b0
c41ecc7
d1f29ad
c5fb7eb
fa52447
7e83419
b25fe00
71fa2fd
32f05a1
7c18abd
34336fe
b557fe0
dda022c
1aaadf2
2c9285e
846f756
5057f88
0f9f492
5850705
46ca917
ec89277
d168526
442f351
786c339
486657a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,7 +18,7 @@ | |
|
|
||
|
|
||
| import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
| import org.apache.kafka.streams.processor.ProcessorContext; | ||
| import org.apache.kafka.streams.processor.api.ProcessorContext; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -32,7 +32,7 @@ public class LogAndContinueExceptionHandler implements DeserializationExceptionH | |
| private static final Logger log = LoggerFactory.getLogger(LogAndContinueExceptionHandler.class); | ||
|
|
||
| @Override | ||
| public DeserializationHandlerResponse handle(final ProcessorContext context, | ||
| public DeserializationHandlerResponse handle(final ProcessorContext<?, ?> context, | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we need to deprecate also this method and add a new one? Technically, it is a class of the public API that can be extended. |
||
| final ConsumerRecord<byte[], byte[]> record, | ||
| final Exception exception) { | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,7 +17,7 @@ | |
| package org.apache.kafka.streams.errors; | ||
|
|
||
| import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
| import org.apache.kafka.streams.processor.ProcessorContext; | ||
| import org.apache.kafka.streams.processor.api.ProcessorContext; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -32,7 +32,7 @@ public class LogAndFailExceptionHandler implements DeserializationExceptionHandl | |
| private static final Logger log = LoggerFactory.getLogger(LogAndFailExceptionHandler.class); | ||
|
|
||
| @Override | ||
| public DeserializationHandlerResponse handle(final ProcessorContext context, | ||
| public DeserializationHandlerResponse handle(final ProcessorContext<?, ?> context, | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we need to deprecate also this method and add a new one? Technically, it is a class of the public API that can be extended. |
||
| final ConsumerRecord<byte[], byte[]> record, | ||
| final Exception exception) { | ||
|
|
||
|
|
||
Large diffs are not rendered by default.
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,9 +23,9 @@ | |
| * record-pair of a {@link KStream}-{@link KStream}, {@link KStream}-{@link KTable}, or {@link KTable}-{@link KTable} | ||
| * join. | ||
| * | ||
| * @param <V1> first value type | ||
| * @param <V2> second value type | ||
| * @param <VR> joined value type | ||
|
Comment on lines
-26
to
-28
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Also here: it doesn't seem strictly necessary to rename the generic parameters as part of this PR. Specifically, funny story: these params used to be called |
||
| * @param <V> first value type | ||
| * @param <V1> second value type | ||
| * @param <VOut> joined value type | ||
| * @see KStream#join(KStream, ValueJoiner, JoinWindows) | ||
| * @see KStream#join(KStream, ValueJoiner, JoinWindows, StreamJoined) | ||
| * @see KStream#leftJoin(KStream, ValueJoiner, JoinWindows) | ||
|
|
@@ -40,7 +40,7 @@ | |
| * @see KTable#leftJoin(KTable, ValueJoiner) | ||
| * @see KTable#outerJoin(KTable, ValueJoiner) | ||
| */ | ||
| public interface ValueJoiner<V1, V2, VR> { | ||
| public interface ValueJoiner<V, V1, VOut> { | ||
|
|
||
| /** | ||
| * Return a joined value consisting of {@code value1} and {@code value2}. | ||
|
|
@@ -49,5 +49,5 @@ public interface ValueJoiner<V1, V2, VR> { | |
| * @param value2 the second value for joining | ||
| * @return the joined value | ||
| */ | ||
| VR apply(final V1 value1, final V2 value2); | ||
| VOut apply(final V value1, final V1 value2); | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,6 +16,10 @@ | |
| */ | ||
| package org.apache.kafka.streams.kstream.internals; | ||
|
|
||
| import java.util.Collection; | ||
| import java.util.HashSet; | ||
| import java.util.Objects; | ||
| import java.util.Set; | ||
|
Comment on lines
+19
to
+22
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In KAFKA-10787 we agreed on an import order Note, PR #10428 introduces check and a formatter for this.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @cadonna The sooner you merge the PR, I can start to apply the formatter to the streams module sooner. 😃 |
||
| import org.apache.kafka.common.serialization.Serde; | ||
| import org.apache.kafka.streams.internals.ApiUtils; | ||
| import org.apache.kafka.streams.kstream.ValueJoiner; | ||
|
|
@@ -31,11 +35,6 @@ | |
| import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; | ||
| import org.apache.kafka.streams.state.StoreBuilder; | ||
|
|
||
| import java.util.Collection; | ||
| import java.util.HashSet; | ||
| import java.util.Objects; | ||
| import java.util.Set; | ||
|
|
||
| /* | ||
| * Any classes (KTable, KStream, etc) extending this class should follow the serde specification precedence ordering as: | ||
| * | ||
|
|
@@ -144,7 +143,7 @@ public Set<StoreBuilder<?>> stores() { | |
| }; | ||
| } | ||
|
|
||
| static <K, V1, V2, VR> ValueJoinerWithKey<K, V1, V2, VR> toValueJoinerWithKey(final ValueJoiner<V1, V2, VR> valueJoiner) { | ||
| static <K, V, V1, VOut> ValueJoinerWithKey<K, V, V1, VOut> toValueJoinerWithKey(final ValueJoiner<V, V1, VOut> valueJoiner) { | ||
| Objects.requireNonNull(valueJoiner, "joiner can't be null"); | ||
| return (readOnlyKey, value1, value2) -> valueJoiner.apply(value1, value2); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,7 +38,7 @@ | |
| import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters; | ||
| import org.apache.kafka.streams.kstream.internals.graph.StatefulProcessorNode; | ||
| import org.apache.kafka.streams.kstream.internals.graph.GraphNode; | ||
| import org.apache.kafka.streams.processor.ProcessorSupplier; | ||
| import org.apache.kafka.streams.processor.api.ProcessorSupplier; | ||
| import org.apache.kafka.streams.state.StoreBuilder; | ||
|
|
||
| class CogroupedStreamAggregateBuilder<K, VOut> { | ||
|
|
@@ -48,21 +48,20 @@ class CogroupedStreamAggregateBuilder<K, VOut> { | |
| CogroupedStreamAggregateBuilder(final InternalStreamsBuilder builder) { | ||
| this.builder = builder; | ||
| } | ||
| @SuppressWarnings("unchecked") | ||
| <KR> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? super K, ? super Object, VOut>> groupPatterns, | ||
| final Initializer<VOut> initializer, | ||
| final NamedInternal named, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final Serde<KR> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName) { | ||
| <KOut> KTable<KOut, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? super K, ? super Object, VOut>> groupPatterns, | ||
| final Initializer<VOut> initializer, | ||
| final NamedInternal named, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final Serde<KOut> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName) { | ||
| processRepartitions(groupPatterns, storeBuilder); | ||
| final Collection<GraphNode> processors = new ArrayList<>(); | ||
| final Collection<KStreamAggProcessorSupplier> parentProcessors = new ArrayList<>(); | ||
| final Collection<KStreamAggregateProcessorSupplier> parentProcessors = new ArrayList<>(); | ||
| boolean stateCreated = false; | ||
| int counter = 0; | ||
| for (final Entry<KGroupedStreamImpl<K, ?>, Aggregator<? super K, Object, VOut>> kGroupedStream : groupPatterns.entrySet()) { | ||
| final KStreamAggProcessorSupplier<K, K, ?, ?> parentProcessor = | ||
| final KStreamAggregateProcessorSupplier<K, K, ?, ?> parentProcessor = | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shouldn't this be |
||
| new KStreamAggregate<>(storeBuilder.name(), initializer, kGroupedStream.getValue()); | ||
| parentProcessors.add(parentProcessor); | ||
| final StatefulProcessorNode<K, ?> statefulProcessorNode = getStatefulProcessorNode( | ||
|
|
@@ -80,24 +79,23 @@ <KR> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? sup | |
| return createTable(processors, parentProcessors, named, keySerde, valueSerde, queryableName, storeBuilder.name()); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| <KR, W extends Window> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? super K, ? super Object, VOut>> groupPatterns, | ||
| final Initializer<VOut> initializer, | ||
| final NamedInternal named, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final Serde<KR> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName, | ||
| final Windows<W> windows) { | ||
| <KOut, W extends Window> KTable<KOut, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? super K, ? super Object, VOut>> groupPatterns, | ||
| final Initializer<VOut> initializer, | ||
| final NamedInternal named, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final Serde<KOut> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName, | ||
| final Windows<W> windows) { | ||
| processRepartitions(groupPatterns, storeBuilder); | ||
|
|
||
| final Collection<GraphNode> processors = new ArrayList<>(); | ||
| final Collection<KStreamAggProcessorSupplier> parentProcessors = new ArrayList<>(); | ||
| final Collection<KStreamAggregateProcessorSupplier> parentProcessors = new ArrayList<>(); | ||
| boolean stateCreated = false; | ||
| int counter = 0; | ||
| for (final Entry<KGroupedStreamImpl<K, ?>, Aggregator<? super K, Object, VOut>> kGroupedStream : groupPatterns.entrySet()) { | ||
| final KStreamAggProcessorSupplier<K, K, ?, ?> parentProcessor = | ||
| (KStreamAggProcessorSupplier<K, K, ?, ?>) new KStreamWindowAggregate<K, K, VOut, W>( | ||
| final KStreamWindowAggregate<K, K, VOut, W> parentProcessor = | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shouldn't this be |
||
| new KStreamWindowAggregate<>( | ||
| windows, | ||
| storeBuilder.name(), | ||
| initializer, | ||
|
|
@@ -118,24 +116,23 @@ <KR, W extends Window> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?> | |
| return createTable(processors, parentProcessors, named, keySerde, valueSerde, queryableName, storeBuilder.name()); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| <KR> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? super K, ? super Object, VOut>> groupPatterns, | ||
| final Initializer<VOut> initializer, | ||
| final NamedInternal named, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final Serde<KR> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName, | ||
| final SessionWindows sessionWindows, | ||
| final Merger<? super K, VOut> sessionMerger) { | ||
| <KOut> KTable<KOut, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? super K, ? super Object, VOut>> groupPatterns, | ||
| final Initializer<VOut> initializer, | ||
| final NamedInternal named, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final Serde<KOut> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName, | ||
| final SessionWindows sessionWindows, | ||
| final Merger<? super K, VOut> sessionMerger) { | ||
| processRepartitions(groupPatterns, storeBuilder); | ||
| final Collection<GraphNode> processors = new ArrayList<>(); | ||
| final Collection<KStreamAggProcessorSupplier> parentProcessors = new ArrayList<>(); | ||
| final Collection<KStreamAggregateProcessorSupplier> parentProcessors = new ArrayList<>(); | ||
| boolean stateCreated = false; | ||
| int counter = 0; | ||
| for (final Entry<KGroupedStreamImpl<K, ?>, Aggregator<? super K, Object, VOut>> kGroupedStream : groupPatterns.entrySet()) { | ||
| final KStreamAggProcessorSupplier<K, K, ?, ?> parentProcessor = | ||
| (KStreamAggProcessorSupplier<K, K, ?, ?>) new KStreamSessionWindowAggregate<K, K, VOut>( | ||
| final KStreamSessionWindowAggregate<K, K, VOut> parentProcessor = | ||
| new KStreamSessionWindowAggregate<>( | ||
| sessionWindows, | ||
| storeBuilder.name(), | ||
| initializer, | ||
|
|
@@ -158,22 +155,22 @@ <KR> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? sup | |
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| <KR> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? super K, ? super Object, VOut>> groupPatterns, | ||
| final Initializer<VOut> initializer, | ||
| final NamedInternal named, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final Serde<KR> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName, | ||
| final SlidingWindows slidingWindows) { | ||
| <KOut> KTable<KOut, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? super K, ? super Object, VOut>> groupPatterns, | ||
| final Initializer<VOut> initializer, | ||
| final NamedInternal named, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final Serde<KOut> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName, | ||
| final SlidingWindows slidingWindows) { | ||
| processRepartitions(groupPatterns, storeBuilder); | ||
| final Collection<KStreamAggProcessorSupplier> parentProcessors = new ArrayList<>(); | ||
| final Collection<KStreamAggregateProcessorSupplier> parentProcessors = new ArrayList<>(); | ||
| final Collection<GraphNode> processors = new ArrayList<>(); | ||
| boolean stateCreated = false; | ||
| int counter = 0; | ||
| for (final Entry<KGroupedStreamImpl<K, ?>, Aggregator<? super K, Object, VOut>> kGroupedStream : groupPatterns.entrySet()) { | ||
| final KStreamAggProcessorSupplier<K, K, ?, ?> parentProcessor = | ||
| (KStreamAggProcessorSupplier<K, K, ?, ?>) new KStreamSlidingWindowAggregate<K, K, VOut>( | ||
| final KStreamSlidingWindowAggregate<K, K, VOut> parentProcessor = | ||
| new KStreamSlidingWindowAggregate<>( | ||
| slidingWindows, | ||
| storeBuilder.name(), | ||
| initializer, | ||
|
|
@@ -225,26 +222,26 @@ private void processRepartitions(final Map<KGroupedStreamImpl<K, ?>, Aggregator< | |
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| <KR, VIn> KTable<KR, VOut> createTable(final Collection<GraphNode> processors, | ||
| final Collection<KStreamAggProcessorSupplier> parentProcessors, | ||
| final NamedInternal named, | ||
| final Serde<KR> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName, | ||
| final String storeName) { | ||
| <KOut, VIn> KTable<KOut, VOut> createTable(final Collection<GraphNode> processors, | ||
| final Collection<KStreamAggregateProcessorSupplier> parentProcessors, | ||
| final NamedInternal named, | ||
| final Serde<KOut> keySerde, | ||
| final Serde<VOut> valueSerde, | ||
| final String queryableName, | ||
| final String storeName) { | ||
|
|
||
| final String mergeProcessorName = named.suffixWithOrElseGet( | ||
| "-cogroup-merge", | ||
| builder, | ||
| CogroupedKStreamImpl.MERGE_NAME); | ||
| final KTableProcessorSupplier<K, VOut, VOut> passThrough = new KTablePassThrough<>(parentProcessors, storeName); | ||
| final KTableChangeProcessorSupplier<K, VOut, VOut, K, VOut> passThrough = new KTablePassThrough<>(parentProcessors, storeName); | ||
| final ProcessorParameters<K, VOut, ?, ?> processorParameters = new ProcessorParameters(passThrough, mergeProcessorName); | ||
| final ProcessorGraphNode<K, VOut> mergeNode = | ||
| new ProcessorGraphNode<>(mergeProcessorName, processorParameters); | ||
|
|
||
| builder.addGraphNode(processors, mergeNode); | ||
|
|
||
| return new KTableImpl<KR, VIn, VOut>( | ||
| return new KTableImpl<KOut, VIn, VOut>( | ||
| mergeProcessorName, | ||
| keySerde, | ||
| valueSerde, | ||
|
|
@@ -258,7 +255,7 @@ <KR, VIn> KTable<KR, VOut> createTable(final Collection<GraphNode> processors, | |
| private StatefulProcessorNode<K, ?> getStatefulProcessorNode(final String processorName, | ||
| final boolean stateCreated, | ||
| final StoreBuilder<?> storeBuilder, | ||
| final ProcessorSupplier<K, ?> kStreamAggregate) { | ||
| final ProcessorSupplier<K, ?, ?, ?> kStreamAggregate) { | ||
| final StatefulProcessorNode<K, ?> statefulProcessorNode; | ||
| if (!stateCreated) { | ||
| statefulProcessorNode = | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, man. I overlooked this in the KIP, and we can't just change this in-place, as it will break any subclasses.
What we need to do is deprecate this method and introduce a new one with a default implementation that calls back here. We can update the KIP with this change, since it's a simple oversight and follows established patterns for migrating interfaces.