-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-18026: KIP-1112, migrate foreign-key joins to use ProcesserSupplier#stores #18194
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
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 |
|---|---|---|
|
|
@@ -38,6 +38,7 @@ | |
| import org.apache.kafka.streams.kstream.SessionWindows; | ||
| import org.apache.kafka.streams.kstream.SlidingWindows; | ||
| import org.apache.kafka.streams.kstream.StreamJoined; | ||
| import org.apache.kafka.streams.kstream.TableJoined; | ||
| import org.apache.kafka.streams.kstream.TimeWindows; | ||
| import org.apache.kafka.streams.processor.StateStore; | ||
| import org.apache.kafka.streams.processor.api.Processor; | ||
|
|
@@ -2153,6 +2154,84 @@ public void shouldWrapProcessorsForTableTableOuterJoin() { | |
| assertThat(counter.numConnectedStateStores(), CoreMatchers.is(3)); | ||
| } | ||
|
|
||
| @Test | ||
| public void shouldWrapProcessorsForForeignKeyInnerJoin() { | ||
| final Map<Object, Object> props = dummyStreamsConfigMap(); | ||
| props.put(PROCESSOR_WRAPPER_CLASS_CONFIG, RecordingProcessorWrapper.class); | ||
|
|
||
| final WrapperRecorder counter = new WrapperRecorder(); | ||
| props.put(PROCESSOR_WRAPPER_COUNTER_CONFIG, counter); | ||
|
|
||
| final StreamsBuilder builder = new StreamsBuilder(new TopologyConfig(new StreamsConfig(props))); | ||
|
|
||
| final KTable<String, String> left = builder.table("input1", Consumed.as("input1")); | ||
| final KTable<String, String> right = builder.table("input2", Consumed.as("input2")); | ||
|
|
||
| left.join(right, | ||
| value -> value, | ||
| (v1, v2) -> v1 + v2, | ||
| TableJoined.as("join"), | ||
| Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("materialized-store").withValueSerde(Serdes.String())) | ||
| .toStream(Named.as("toStream")) | ||
| .to("output", Produced.as("sink")); | ||
|
|
||
| builder.build(); | ||
| assertThat(counter.numWrappedProcessors(), CoreMatchers.is(9)); | ||
| assertThat(counter.wrappedProcessorNames().toString(), counter.wrappedProcessorNames(), Matchers.containsInAnyOrder( | ||
| "input1", | ||
| "input2", | ||
| "join-foreign-join-subscription", | ||
| "join-subscription-join-foreign", | ||
| "join-subscription-registration-processor", | ||
| "join-subscription-receive", | ||
| "join-result", | ||
| "join-subscription-response-resolver", | ||
| "toStream" | ||
| )); | ||
|
|
||
| assertThat(counter.numUniqueStateStores(), CoreMatchers.is(4)); // table1, table2, subscription store, and join materialized | ||
| assertThat(counter.numConnectedStateStores(), CoreMatchers.is(5)); | ||
| } | ||
|
|
||
| @Test | ||
| public void shouldWrapProcessorsForForeignKeyLeftJoin() { | ||
| final Map<Object, Object> props = dummyStreamsConfigMap(); | ||
| props.put(PROCESSOR_WRAPPER_CLASS_CONFIG, RecordingProcessorWrapper.class); | ||
|
|
||
| final WrapperRecorder counter = new WrapperRecorder(); | ||
| props.put(PROCESSOR_WRAPPER_COUNTER_CONFIG, counter); | ||
|
|
||
| final StreamsBuilder builder = new StreamsBuilder(new TopologyConfig(new StreamsConfig(props))); | ||
|
|
||
| final KTable<String, String> left = builder.table("input1", Consumed.as("input1")); | ||
| final KTable<String, String> right = builder.table("input2", Consumed.as("input2")); | ||
|
|
||
| left.leftJoin(right, | ||
| value -> value, | ||
| (v1, v2) -> v1 + v2, | ||
| TableJoined.as("l-join"), | ||
| Materialized.<String, String, KeyValueStore<Bytes, byte[]>>as("materialized-store").withValueSerde(Serdes.String())) | ||
| .toStream(Named.as("toStream")) // 6 | ||
|
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. What the comment
Member
Author
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. copy-paste error 🙃 thanks And yeah, it should have 5 stores
Member
Author
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. ill address this in my next PR so i doin't have to run the tests again before merging |
||
| .to("output", Produced.as("sink")); | ||
|
|
||
| builder.build(); | ||
| assertThat(counter.numWrappedProcessors(), CoreMatchers.is(9)); | ||
| assertThat(counter.wrappedProcessorNames().toString(), counter.wrappedProcessorNames(), Matchers.containsInAnyOrder( | ||
| "input1", | ||
| "input2", | ||
| "l-join-foreign-join-subscription", | ||
| "l-join-subscription-join-foreign", | ||
| "l-join-subscription-registration-processor", | ||
| "l-join-subscription-receive", | ||
| "l-join-result", | ||
| "l-join-subscription-response-resolver", | ||
| "toStream" | ||
| )); | ||
|
|
||
| assertThat(counter.numUniqueStateStores(), CoreMatchers.is(4)); // table1, table2, subscription store, and join materialized | ||
| assertThat(counter.numConnectedStateStores(), CoreMatchers.is(5)); | ||
| } | ||
|
|
||
| @Test | ||
| public void shouldAllowStreamsFromSameTopic() { | ||
| builder.stream("topic"); | ||
|
|
||
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.
Just curious is the final goal to eventually remove
StatefulProcessorNodeand only useProcessorGraphNode?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.
good question -- basically yes and no. Almost everything has/will be converted from StatefulProcesorNode to ProcessorGraphNode, but there are a few exceptions where we still need the StatefulProcessorNode to connect a processor and state store. For example any processors that access upstream state via a value getter, or a custom ProcessorSupplier passed into the
.process/.processValuesoperator that doesn't implement#storesand manually adds stores by callingStreamsBuilder#addStateStoreinstead.I actually have the PR for this ready but it was done on top of this so I was waiting for this one to be merged before calling for review. But it's ready now if you want to take a look (though note that it still can't be merged until Almog's TableSuppressNode PR is merged). See #18195