abbccdda commented on a change in pull request #8550:
URL: https://github.com/apache/kafka/pull/8550#discussion_r415209254



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
##########
@@ -312,6 +312,7 @@ public void buildAndOptimizeTopology(final Properties 
props) {
                 graphNodePriorityQueue.offer(graphNode);
             }
         }
+        internalTopologyBuilder.validateCoPartition();

Review comment:
       nit: validateCopartition

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -633,6 +634,45 @@ public final void copartitionSources(final 
Collection<String> sourceNodes) {
         copartitionSourceGroups.add(Collections.unmodifiableSet(new 
HashSet<>(sourceNodes)));
     }
 
+    public void validateCoPartition() {
+        final List<Set<String>> copartitionGroups =
+                copartitionSourceGroups
+                        .stream()
+                        .map(sourceGroup -> sourceGroup
+                                .stream()
+                                .flatMap(node -> 
nodeToSourceTopics.get(node).stream())
+                                .collect(Collectors.toSet())
+                        ).collect(Collectors.toList());
+        for (final Set<String> coPartition : copartitionGroups) {
+            final Map<String, InternalTopicProperties> coPartitionProperties = 
new HashMap<>();

Review comment:
       Let's try to be consistent to use `copartition` instead of `coPartition`

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -633,6 +634,45 @@ public final void copartitionSources(final 
Collection<String> sourceNodes) {
         copartitionSourceGroups.add(Collections.unmodifiableSet(new 
HashSet<>(sourceNodes)));
     }
 
+    public void validateCoPartition() {
+        final List<Set<String>> copartitionGroups =
+                copartitionSourceGroups
+                        .stream()
+                        .map(sourceGroup -> sourceGroup
+                                .stream()
+                                .flatMap(node -> 
nodeToSourceTopics.get(node).stream())
+                                .collect(Collectors.toSet())
+                        ).collect(Collectors.toList());
+        for (final Set<String> coPartition : copartitionGroups) {
+            final Map<String, InternalTopicProperties> coPartitionProperties = 
new HashMap<>();

Review comment:
       Since we are only going to verify number of partitions, I think we could 
just set value as integer

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java
##########
@@ -312,6 +312,7 @@ public void buildAndOptimizeTopology(final Properties 
props) {
                 graphNodePriorityQueue.offer(graphNode);
             }
         }
+        internalTopologyBuilder.validateCoPartition();

Review comment:
       One question, since we do verification in topology builder, is there any 
validation code in later stage that could be removed?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -633,6 +634,45 @@ public final void copartitionSources(final 
Collection<String> sourceNodes) {
         copartitionSourceGroups.add(Collections.unmodifiableSet(new 
HashSet<>(sourceNodes)));
     }
 
+    public void validateCoPartition() {
+        final List<Set<String>> copartitionGroups =
+                copartitionSourceGroups
+                        .stream()
+                        .map(sourceGroup -> sourceGroup
+                                .stream()
+                                .flatMap(node -> 
nodeToSourceTopics.get(node).stream())
+                                .collect(Collectors.toSet())
+                        ).collect(Collectors.toList());
+        for (final Set<String> coPartition : copartitionGroups) {
+            final Map<String, InternalTopicProperties> coPartitionProperties = 
new HashMap<>();
+            internalTopicNamesWithProperties.forEach((topic, prop) -> {
+                if (coPartition.contains(topic) && 
prop.getNumberOfPartitions().isPresent()) {
+                    coPartitionProperties.put(topic, prop);
+                }
+            });
+            if (coPartition.size() == coPartitionProperties.size()) {

Review comment:
       Could you clarify why we need this equality check?




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to