-
Notifications
You must be signed in to change notification settings - Fork 15.2k
KAFKA-10758: ProcessorTopology should only consider its own nodes when updating regex source topics #9648
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-10758: ProcessorTopology should only consider its own nodes when updating regex source topics #9648
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 |
|---|---|---|
|
|
@@ -149,24 +149,30 @@ public boolean hasPersistentGlobalStore() { | |
| return false; | ||
| } | ||
|
|
||
| public void updateSourceTopics(final Map<String, List<String>> sourceTopicsByName) { | ||
| if (!sourceTopicsByName.keySet().equals(sourceNodesByName.keySet())) { | ||
| log.error("Set of source nodes do not match: \n" + | ||
| "sourceNodesByName = {}\n" + | ||
| "sourceTopicsByName = {}", | ||
| sourceNodesByName.keySet(), sourceTopicsByName.keySet()); | ||
| throw new IllegalStateException("Tried to update source topics but source nodes did not match"); | ||
| } | ||
| public void updateSourceTopics(final Map<String, List<String>> allSourceTopicsByNodeName) { | ||
|
ableegoldman marked this conversation as resolved.
|
||
| sourceNodesByTopic.clear(); | ||
| for (final Map.Entry<String, List<String>> sourceEntry : sourceTopicsByName.entrySet()) { | ||
| final String nodeName = sourceEntry.getKey(); | ||
| for (final String topic : sourceEntry.getValue()) { | ||
| for (final Map.Entry<String, SourceNode<?, ?, ?, ?>> sourceNodeEntry : sourceNodesByName.entrySet()) { | ||
|
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. In addition to removing the fault check, I slightly refactored this loop so that we only loop over the source nodes in this particular subtopology. Previously we would have added entries for all source nodes across the entire topology to our |
||
| final String sourceNodeName = sourceNodeEntry.getKey(); | ||
| final SourceNode<?, ?, ?, ?> sourceNode = sourceNodeEntry.getValue(); | ||
|
|
||
| final List<String> updatedSourceTopics = allSourceTopicsByNodeName.get(sourceNodeName); | ||
| if (updatedSourceTopics == null) { | ||
| log.error("Unable to find source node {} in updated topics map {}", | ||
| sourceNodeName, allSourceTopicsByNodeName); | ||
| throw new IllegalStateException("Node " + sourceNodeName + " not found in full topology"); | ||
| } | ||
|
|
||
| log.trace("Updating source node {} with new topics {}", sourceNodeName, updatedSourceTopics); | ||
| for (final String topic : updatedSourceTopics) { | ||
| if (sourceNodesByTopic.containsKey(topic)) { | ||
| log.error("Tried to subscribe topic {} to two nodes when updating topics from {}", | ||
| topic, allSourceTopicsByNodeName); | ||
| throw new IllegalStateException("Topic " + topic + " was already registered to source node " | ||
| + sourceNodesByTopic.get(topic).name()); | ||
| + sourceNodesByTopic.get(topic).name()); | ||
| } | ||
| sourceNodesByTopic.put(topic, sourceNodesByName.get(nodeName)); | ||
| sourceNodesByTopic.put(topic, sourceNode); | ||
| } | ||
|
|
||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change | ||||||||
|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -31,6 +31,7 @@ | |||||||||
| import org.apache.kafka.streams.KeyValue; | ||||||||||
| import org.apache.kafka.streams.StreamsBuilder; | ||||||||||
| import org.apache.kafka.streams.StreamsConfig; | ||||||||||
| import org.apache.kafka.streams.Topology; | ||||||||||
| import org.apache.kafka.streams.TopologyWrapper; | ||||||||||
| import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; | ||||||||||
| import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; | ||||||||||
|
|
@@ -55,6 +56,7 @@ | |||||||||
| import org.junit.rules.TestName; | ||||||||||
|
|
||||||||||
| import java.io.IOException; | ||||||||||
| import java.time.Duration; | ||||||||||
| import java.util.ArrayList; | ||||||||||
| import java.util.Arrays; | ||||||||||
| import java.util.Collection; | ||||||||||
|
|
@@ -67,9 +69,11 @@ | |||||||||
| import java.util.concurrent.atomic.AtomicInteger; | ||||||||||
| import java.util.regex.Pattern; | ||||||||||
|
|
||||||||||
| import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning; | ||||||||||
| import static org.hamcrest.CoreMatchers.equalTo; | ||||||||||
| import static org.hamcrest.CoreMatchers.is; | ||||||||||
| import static org.hamcrest.MatcherAssert.assertThat; | ||||||||||
| import static org.hamcrest.Matchers.greaterThan; | ||||||||||
|
|
||||||||||
| /** | ||||||||||
| * End-to-end integration test based on using regex and named topics for creating sources, using | ||||||||||
|
|
@@ -189,7 +193,7 @@ public void subscribe(final Pattern topics, final ConsumerRebalanceListener list | |||||||||
| } | ||||||||||
|
|
||||||||||
| @Test | ||||||||||
| public void testRegexRecordsAreProcessedAfterReassignment() throws Exception { | ||||||||||
| public void testRegexRecordsAreProcessedAfterNewTopicCreatedWithMultipleSubtopologies() throws Exception { | ||||||||||
| final String topic1 = "TEST-TOPIC-1"; | ||||||||||
| final String topic2 = "TEST-TOPIC-2"; | ||||||||||
|
|
||||||||||
|
|
@@ -198,9 +202,19 @@ public void testRegexRecordsAreProcessedAfterReassignment() throws Exception { | |||||||||
|
|
||||||||||
| final StreamsBuilder builder = new StreamsBuilder(); | ||||||||||
| final KStream<String, String> pattern1Stream = builder.stream(Pattern.compile("TEST-TOPIC-\\d")); | ||||||||||
| pattern1Stream.to(outputTopic, Produced.with(Serdes.String(), Serdes.String())); | ||||||||||
| streams = new KafkaStreams(builder.build(), streamsConfiguration); | ||||||||||
| streams.start(); | ||||||||||
| final KStream<String, String> otherStream = builder.stream(Pattern.compile("not-a-match")); | ||||||||||
|
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. We didn't catch the bug in this test for two reasons: it has only one subtopology, and it didn't wait for Streams to get to RUNNING before it created the new topic. So we weren't even covering the "update source topics" code path since all topics existed by the first assignment |
||||||||||
|
|
||||||||||
| pattern1Stream | ||||||||||
|
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. Technically it's sufficient to just add the second KStream above for a multi-subtopology application, but I felt the test coverage could only stand to benefit with (slightly) more complicated examples |
||||||||||
| .selectKey((k, v) -> k) | ||||||||||
| .groupByKey() | ||||||||||
| .aggregate(() -> "", (k, v, a) -> v) | ||||||||||
| .toStream().to(outputTopic, Produced.with(Serdes.String(), Serdes.String())); | ||||||||||
|
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. I would even add
Suggested change
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. But if we merge then won't that merge the subtopologies as well? (We would still have two subtopologies due to the upstream key-changing operation/repartition, but I wanted the test to cover different "kinds" of subtopologies like this)
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. You would have a source node in one sub-topology an a source node in the other sub-topology. I thought that was the pattern in the bug report, but I now realized that the bug report uses a pattern similar to the one you specified. Wouldn't it make sense to test both?
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. There are two ways to have more than one source node: either reading from a different input topic/pattern, or via a repartition. I agree that we should test both of these cases, but I'd prefer to do so in a single integration test rather than in two separate integration tests, to avoid making the test suite even longer |
||||||||||
|
|
||||||||||
| final Topology topology = builder.build(); | ||||||||||
| assertThat(topology.describe().subtopologies().size(), greaterThan(1)); | ||||||||||
| streams = new KafkaStreams(topology, streamsConfiguration); | ||||||||||
|
|
||||||||||
| startApplicationAndWaitUntilRunning(Collections.singletonList(streams), Duration.ofSeconds(30)); | ||||||||||
|
|
||||||||||
| CLUSTER.createTopic(topic2); | ||||||||||
|
|
||||||||||
|
|
||||||||||
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.
Since the root cause of this bug was basically just confusion over what exactly this set contains, a renaming feels in order