florin-akermann commented on code in PR #15189:
URL: https://github.com/apache/kafka/pull/15189#discussion_r1518671144


##########
streams/src/test/java/org/apache/kafka/streams/integration/RelaxedNullKeyRequirementJoinTest.java:
##########
@@ -71,11 +71,21 @@ void afterEach() {
     @Test
     void testRelaxedLeftStreamStreamJoin() {
         leftStream
-            .leftJoin(rightStream, JOINER, WINDOW)
+            .leftJoin(rightStream, JOINER, WINDOW_60MS_GRACE_10MS)
             .to(OUT);
         initTopology();
-        left.pipeInput(null, "leftValue", 1);
-        assertEquals(Collections.singletonList(new KeyValue<>(null, 
"leftValue|null")), out.readKeyValuesToList());
+        left.pipeInput(null, "leftValue1", 1);
+        left.pipeInput(null, "leftValue2", 90);
+        left.pipeInput(null, "lateArrival-Dropped", 19);
+        left.pipeInput(null, "lateArrivalWithinGrace", 20);
+        assertEquals(
+            Arrays.asList(
+                new KeyValue<>(null, "leftValue1|null"),
+                new KeyValue<>(null, "leftValue2|null"),
+                new KeyValue<>(null, "lateArrivalWithinGrace|null")

Review Comment:
   I wonder how to best solve this.
   Multiple null-key records would colide in the 'outerJoinStore' as they 
potentially all could have the same key of  type 'TimestampedKeyAndJoinSide<K>'.
   
   E.g. for  a left stream
   
   | Key  | value | ts |
   |------|-------|----|
   | null | a     | 1  |
   | null | b     | 1  |
   | null | c     | 1  |
   
   We probably would only get to see 'c' even though we would like to see 
'a','b' and 'c'?
   
   From the top of my head I see two options to handle this.
   - Maintain an additional store just for null-key records where such records 
wouldn't collide.
   - Adjust the 'outerJoinStore' key type `TimestampedKeyAndJoinSide<K>'`( E.g. 
by adding offset and partition as optional fields. This way null-key records 
could be distinguished and for keyed records the old behavior can be kept)
   
   Personally I prefer the latter.



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to