[ 
https://issues.apache.org/jira/browse/KAFKA-7671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16702630#comment-16702630
 ] 

ASF GitHub Bot commented on KAFKA-7671:
---------------------------------------

guozhangwang closed pull request #5959: KAFKA-7671: Stream-Global Table join 
should not reset repartition flag
URL: https://github.com/apache/kafka/pull/5959
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
index 3b691513004..ed5625e3a17 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
@@ -753,7 +753,7 @@ public void process(final ProcessorSupplier<? super K, ? 
super V> processorSuppl
         builder.addGraphNode(this.streamsGraphNode, streamTableJoinNode);
 
         // do not have serde for joined result
-        return new KStreamImpl<>(name, keySerde, null, sourceNodes, false, 
streamTableJoinNode, builder);
+        return new KStreamImpl<>(name, keySerde, null, sourceNodes, 
repartitionRequired, streamTableJoinNode, builder);
     }
 
     @SuppressWarnings("unchecked")
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
index 772836f9961..e0c38c656c2 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
@@ -61,15 +61,18 @@
 import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import static java.time.Duration.ofMillis;
 import static java.util.Arrays.asList;
 import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.notNullValue;
 import static org.hamcrest.core.IsInstanceOf.instanceOf;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 @SuppressWarnings("unchecked")
@@ -437,6 +440,27 @@ public void 
shouldUseRecordMetadataTimestampExtractorWhenInternalRepartitioningT
             }
         }
     }
+
+    @Test
+    public void shouldPropagateRepartitionFlagAfterGlobalKTableJoin() {
+        final StreamsBuilder builder = new StreamsBuilder();
+        final GlobalKTable<String, String> globalKTable = 
builder.globalTable("globalTopic");
+        final KeyValueMapper<String, String, String> kvMappper = (k, v) -> k + 
v;
+        final ValueJoiner<String, String, String> valueJoiner = (v1, v2) -> v1 
+ v2;
+        builder.<String, String>stream("topic").selectKey((k, v) -> v)
+            .join(globalKTable, kvMappper, valueJoiner)
+            .groupByKey()
+            .count();
+
+        final Pattern repartitionTopicPattern = Pattern.compile("Sink: 
.*-repartition");
+        final String topology = builder.build().describe().toString();
+        final Matcher matcher = repartitionTopicPattern.matcher(topology);
+        assertTrue(matcher.find());
+        final String match = matcher.group();
+        assertThat(match, notNullValue());
+        assertTrue(match.endsWith("repartition"));
+
+    }
     
     @Test
     public void testToWithNullValueSerdeDoesntNPE() {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> A KStream/GlobalKTable join shouldn't reset the repartition flag
> ----------------------------------------------------------------
>
>                 Key: KAFKA-7671
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7671
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>    Affects Versions: 2.0.1
>            Reporter: Andy Bryant
>            Assignee: Bill Bejeck
>            Priority: Major
>             Fix For: 2.2.0, 2.1.1
>
>
> Currently a KStream/GlobalKTable join resets the repartition required flag to 
> false.
> I have a topology where I map a stream, join with a GlobalKTable, then 
> groupByKey then aggregate.
> The aggregate wasn't behaving correctly because it didn't force a repartition 
> as I expected. The KStream/GlobalKTable join had reset the flag and hence I 
> was getting the same keys in different partitions.
> Since a KStream/GlobalKTable join does not itself force a repartition, it 
> should simply propagate the flag down to the resultant KStream the same way 
> most other operators work.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to