cadonna commented on a change in pull request #9648:
URL: https://github.com/apache/kafka/pull/9648#discussion_r529280072



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java
##########
@@ -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) {

Review comment:
       Could you also add a unit test that verifies that the issue is gone?

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
##########
@@ -198,9 +200,16 @@ 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()));
+            final KStream<String, String> otherStream = 
builder.stream(Pattern.compile("not-a-match"));
+
+            pattern1Stream
+                .selectKey((k, v) -> k)
+                .groupByKey()
+                .aggregate(() -> "", (k, v, a) -> v)
+                .toStream().to(outputTopic, Produced.with(Serdes.String(), 
Serdes.String()));
+
             streams = new KafkaStreams(builder.build(), streamsConfiguration);

Review comment:
       I would add the following assert here to ensure we have more than one 
sub-topology:
   
   ```
               final Topology topology = builder.build();
               assertThat(topology.describe().subtopologies().size(), 
greaterThan(1));
               streams = new KafkaStreams(topology, streamsConfiguration);
   ```
   
   Just to make it clear that we want to test with multiple sub-topologies.

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
##########
@@ -198,9 +200,16 @@ 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()));
+            final KStream<String, String> otherStream = 
builder.stream(Pattern.compile("not-a-match"));
+
+            pattern1Stream
+                .selectKey((k, v) -> k)
+                .groupByKey()
+                .aggregate(() -> "", (k, v, a) -> v)
+                .toStream().to(outputTopic, Produced.with(Serdes.String(), 
Serdes.String()));

Review comment:
       I would even add `otherStream` like:
   ```suggestion
                   .toStream()
                   .merge(otherStream)
                   .to(outputTopic, Produced.with(Serdes.String(), 
Serdes.String()));
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to