You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/05/02 00:24:11 UTC

[GitHub] [kafka] mjsax commented on a change in pull request #8550: KAFKA-9850 Move KStream#repartition operator validation during Topolo…

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



##########
File path: streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamRepartitionTest.java
##########
@@ -104,6 +114,54 @@ public void shouldInvokePartitionerWhenSet() {
         verify(streamPartitionerMock);
     }
 
+    @Test
+    public void shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationsDoNotMatchWhenJoining() {
+        final String topicB = "topic-b";
+        final String outputTopic = "topic-output";
+        final String topicBRepartitionedName = "topic-b-scale-up";
+        final String inputTopicRepartitionedName = "input-topic-scale-up";
+        final int topicBNumberOfPartitions = 2;
+        final int inputTopicNumberOfPartitions = 4;
+        final StreamsBuilder builder = new StreamsBuilder();
+
+        final Repartitioned<Integer, String> inputTopicRepartitioned = Repartitioned
+                .<Integer, String>as(inputTopicRepartitionedName)
+                .withNumberOfPartitions(inputTopicNumberOfPartitions);
+
+        final Repartitioned<Integer, String> topicBRepartitioned = Repartitioned
+                .<Integer, String>as(topicBRepartitionedName)
+                .withNumberOfPartitions(topicBNumberOfPartitions);
+
+        final KStream<Integer, String> topicBStream = builder
+                .stream(topicB, Consumed.with(Serdes.Integer(), Serdes.String()))
+                .repartition(topicBRepartitioned);
+
+        builder.stream(inputTopic, Consumed.with(Serdes.Integer(), Serdes.String()))
+                .repartition(inputTopicRepartitioned)
+                .join(topicBStream, (value1, value2) -> value2, JoinWindows.of(Duration.ofSeconds(10)))
+                .to(outputTopic);
+
+        final Map<String, Integer> repartitionTopicsWithNumOfPartitions = Utils.mkMap(
+                Utils.mkEntry(toRepartitionTopicName(topicBRepartitionedName), topicBNumberOfPartitions),
+                Utils.mkEntry(toRepartitionTopicName(inputTopicRepartitionedName), inputTopicNumberOfPartitions)
+        );
+
+        try {
+            builder.build(props);
+            Assert.fail();
+        } catch (final TopologyException t) {

Review comment:
       It's better to use:
   ```
   final TopologyException expected = assertThrows(
     TopologyException.class,
     () -> builder.build(props)
   );
   // put assertions here
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -633,6 +634,43 @@ public final void copartitionSources(final Collection<String> sourceNodes) {
         copartitionSourceGroups.add(Collections.unmodifiableSet(new HashSet<>(sourceNodes)));
     }
 
+    public void validateCopartition() {
+        final List<Set<String>> copartitionGroups =

Review comment:
       If I understand this code correct, `copartitionGroups` take the list of co-partitioned nodes (ie, processor names -> `copartitionSourceGroups`) and replaces each processor name (each processor should be a source-processor?) with the corresponding source topic name?
   
   If yes, a comment might be helpful. Also, name rename `copartitionGroups` -> `allCopartitionedSourceTopics` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -633,6 +634,43 @@ 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())

Review comment:
       rename `node` -> `sourceNodeName` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -633,6 +634,43 @@ 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, Integer> topicPartNum = new HashMap<>();
+            copartition.forEach(topic -> {
+                final InternalTopicProperties prop = internalTopicNamesWithProperties.get(topic);
+                if (prop != null && prop.getNumberOfPartitions().isPresent()) {
+                    topicPartNum.put(topic, prop.getNumberOfPartitions().get());
+                }
+            });
+            internalTopicNamesWithProperties.forEach((topic, prop) -> {
+                if (copartition.contains(topic) && prop.getNumberOfPartitions().isPresent()) {
+                    topicPartNum.put(topic, prop.getNumberOfPartitions().get());
+                }
+            });
+            if (copartition.equals(topicPartNum.keySet())) {
+                final Collection<Integer> partNums = topicPartNum.values();
+                final Integer first = partNums.iterator().next();

Review comment:
       How can we be sure that `partNums` is not empty? If empty, `next()` would throw.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -633,6 +634,43 @@ 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, Integer> topicPartNum = new HashMap<>();

Review comment:
       What is `topicPartNum`? Please avoid abbreviation; make the code hard to read?
   
   Should this be `numberOfPartitionsPerTopic` ?

##########
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:
       I don't think so. Because the later verification that covers the case if the user specified all partitions, is the same code that verify co-partitioning depending on the input topic partition numbers. And we still need that check anyway.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -633,6 +634,43 @@ 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, Integer> topicPartNum = new HashMap<>();
+            copartition.forEach(topic -> {
+                final InternalTopicProperties prop = internalTopicNamesWithProperties.get(topic);
+                if (prop != null && prop.getNumberOfPartitions().isPresent()) {
+                    topicPartNum.put(topic, prop.getNumberOfPartitions().get());
+                }
+            });
+            internalTopicNamesWithProperties.forEach((topic, prop) -> {
+                if (copartition.contains(topic) && prop.getNumberOfPartitions().isPresent()) {
+                    topicPartNum.put(topic, prop.getNumberOfPartitions().get());
+                }
+            });
+            if (copartition.equals(topicPartNum.keySet())) {
+                final Collection<Integer> partNums = topicPartNum.values();
+                final Integer first = partNums.iterator().next();
+                for (final Integer partNum : partNums) {
+                    if (partNum.equals(first)) {

Review comment:
       Should this be `!partNum.equals(first)` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java
##########
@@ -633,6 +634,43 @@ 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, Integer> topicPartNum = new HashMap<>();
+            copartition.forEach(topic -> {
+                final InternalTopicProperties prop = internalTopicNamesWithProperties.get(topic);
+                if (prop != null && prop.getNumberOfPartitions().isPresent()) {
+                    topicPartNum.put(topic, prop.getNumberOfPartitions().get());
+                }
+            });
+            internalTopicNamesWithProperties.forEach((topic, prop) -> {
+                if (copartition.contains(topic) && prop.getNumberOfPartitions().isPresent()) {
+                    topicPartNum.put(topic, prop.getNumberOfPartitions().get());

Review comment:
       Why do we add all the partition numbers for all internal topics here? Seems redundant to the step from above?




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