mjsax commented on code in PR #21094:
URL: https://github.com/apache/kafka/pull/21094#discussion_r2836807781


##########
docs/streams/architecture.md:
##########
@@ -54,6 +54,22 @@ The following diagram shows two tasks each assigned with one 
partition of the in
 
 ![](/43/images/streams-architecture-tasks.jpg)   
 
+### Increasing Parallelism with Multiple Input Topics 
{#streams_architecture_subtopology}

Review Comment:
   What's `{#streams_architecture_subtopology}` ?



##########
docs/streams/architecture.md:
##########
@@ -54,6 +54,22 @@ The following diagram shows two tasks each assigned with one 
partition of the in
 
 ![](/43/images/streams-architecture-tasks.jpg)   
 
+### Increasing Parallelism with Multiple Input Topics 
{#streams_architecture_subtopology}
+
+When your application reads from multiple input topics, the way you structure 
your topology has a significant impact on parallelism. If you use 
`builder.stream(Arrays.asList("topic-a", "topic-b", "topic-c"))` or a regex 
pattern to subscribe to multiple topics, Kafka Streams creates a single 
sub-topology for all these topics. In this case, the maximum number of tasks is 
determined by the **maximum** number of partitions across all the input topics, 
not the sum. This means that data from all topics with the same partition 
number will be funneled through a single task.
+
+For example, if topic-a has 5 partitions, topic-b has 3 partitions, and 
topic-c has 4 partitions, subscribing to all three topics in a single 
`stream()` call will result in only 5 tasks. All records from partition-0 of 
topic-a, topic-b, and topic-c will be processed by the same task.
+
+To achieve independent parallelism per topic, you can structure your topology 
to create separate sub-topologies by processing each topic individually:
+
+    List<String> topics = Arrays.asList("topic-a", "topic-b", "topic-c");
+
+    for (String topic : topics) {
+        KStream<String, String> input = builder.stream(topic);
+        input.filter(...).map(...).to("output-topic");
+    }
+
+With this approach, each input topic gets its own sub-topology and its own 
independent set of tasks. The total number of tasks becomes the **sum** of 
partitions across all topics (5 + 3 + 4 = 12 in the above example), allowing 
for greater parallelism. Each topic's data is processed independently, which 
can significantly improve throughput when you have multiple input topics with 
substantial data volumes.

Review Comment:
   ```suggestion
   With this approach, each input topic gets its own sub-topology and its own 
independent set of tasks. The total number of tasks becomes the **sum** of 
partitions across all topics (5 + 3 + 4 = 12 in the above example), allowing 
for higher parallelism. Each topic's data is processed independently, which can 
significantly improve throughput when you have multiple input topics with 
substantial data volumes.
   ```



##########
docs/streams/architecture.md:
##########
@@ -54,6 +54,22 @@ The following diagram shows two tasks each assigned with one 
partition of the in
 
 ![](/43/images/streams-architecture-tasks.jpg)   
 
+### Increasing Parallelism with Multiple Input Topics 
{#streams_architecture_subtopology}
+
+When your application reads from multiple input topics, the way you structure 
your topology has a significant impact on parallelism. If you use 
`builder.stream(Arrays.asList("topic-a", "topic-b", "topic-c"))` or a regex 
pattern to subscribe to multiple topics, Kafka Streams creates a single 
sub-topology for all these topics. In this case, the maximum number of tasks is 
determined by the **maximum** number of partitions across all the input topics, 
not the sum. This means that data from all topics with the same partition 
number will be funneled through a single task.

Review Comment:
   ```suggestion
   When your application reads from multiple input topics, the way you 
structure your topology has a significant impact on parallelism. If you use 
`builder.stream(Arrays.asList("topic-a", "topic-b", "topic-c"))` or a regex 
pattern to subscribe to multiple topics, Kafka Streams creates a single 
sub-topology for all these topics. In this case, the maximum number of tasks is 
determined by the **maximum** number of partitions across all these input 
topics, not the sum. This means that data from all topics with the same 
partition number will be funneled through a single task.
   ```



-- 
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.

To unsubscribe, e-mail: [email protected]

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

Reply via email to