VictorvandenHoven commented on code in PR #15510: URL: https://github.com/apache/kafka/pull/15510#discussion_r1527973416
########## streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java: ########## @@ -223,57 +224,51 @@ private void emitNonJoinedOuterRecords( try (final KeyValueIterator<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>> it = store.all()) { TimestampedKeyAndJoinSide<K> prevKey = null; + boolean outerJoinLeftWindowOpen = false; + boolean outerJoinRightWindowOpen = false; while (it.hasNext()) { - boolean outerJoinLeftBreak = false; - boolean outerJoinRightBreak = false; final KeyValue<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>> next = it.next(); final TimestampedKeyAndJoinSide<K> timestampedKeyAndJoinSide = next.key; - final LeftOrRightValue<V1, V2> value = next.value; - final K key = timestampedKeyAndJoinSide.getKey(); final long timestamp = timestampedKeyAndJoinSide.getTimestamp(); sharedTimeTracker.minTime = timestamp; - // Skip next records if window has not closed + // Skip next records if window has not closed yet + // We rely on the <timestamp><left/right-boolean><key> ordering of KeyValueIterator final long outerJoinLookBackTimeMs = getOuterJoinLookBackTimeMs(timestampedKeyAndJoinSide); if (sharedTimeTracker.minTime + outerJoinLookBackTimeMs + joinGraceMs >= sharedTimeTracker.streamTime) { if (timestampedKeyAndJoinSide.isLeftSide()) { - outerJoinLeftBreak = true; // there are no more candidates to emit on left-outerJoin-side - } else { - outerJoinRightBreak = true; // there are no more candidates to emit on right-outerJoin-side - } - if (outerJoinLeftBreak && outerJoinRightBreak) { - break; // there are no more candidates to emit on left-outerJoin-side and - // right-outerJoin-side + outerJoinLeftWindowOpen = true; // there are no more candidates to emit on left-outerJoin-side } else { - continue; // there are possibly candidates left on the other outerJoin-side + outerJoinRightWindowOpen = true; // there are no more candidates to emit on right-outerJoin-side } } - final VOut nullJoinedValue; - if (isLeftSide) { - nullJoinedValue = joiner.apply(key, - value.getLeftValue(), - value.getRightValue()); - } else { - nullJoinedValue = joiner.apply(key, - (V1) value.getRightValue(), - (V2) value.getLeftValue()); + if (outerJoinLeftWindowOpen && outerJoinRightWindowOpen) { + // if windows are open for both joinSides we can break since there are no more candidates to emit + break; + } else if (windowOpenForJoinSide(outerJoinLeftWindowOpen, outerJoinRightWindowOpen, timestampedKeyAndJoinSide)) { + // else if window is open only for this joinSide we continue with the next outer record + continue; } - context().forward( - record.withKey(key).withValue(nullJoinedValue).withTimestamp(timestamp) - ); - - if (prevKey != null && !prevKey.equals(timestampedKeyAndJoinSide)) { - // blind-delete the previous key from the outer window store now it is emitted; - // we do this because this delete would remove the whole list of values of the same key, - // and hence if we delete eagerly and then fail, we would miss emitting join results of the later - // values in the list. - // we do not use delete() calls since it would incur extra get() - store.put(prevKey, null); + final K key = timestampedKeyAndJoinSide.getKey(); + final LeftOrRightValue<V1, V2> leftOrRightValue = next.value; + final VOut nullJoinedValue = getNullJoinedValue(key, leftOrRightValue); + if (nullJoinedValue != null) { Review Comment: removed the null 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. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org