guozhangwang commented on a change in pull request #10917:
URL: https://github.com/apache/kafka/pull/10917#discussion_r659908813



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
##########
@@ -186,12 +190,17 @@ public void process(final K key, final V1 value) {
 
         @SuppressWarnings("unchecked")
         private void emitNonJoinedOuterRecords(final 
WindowStore<KeyAndJoinSide<K>, LeftOrRightValue> store) {
+            if (minTime.minTime >= maxObservedStreamTime.get() - joinAfterMs - 
joinBeforeMs - joinGraceMs) {
+                return;
+            }
+
             try (final KeyValueIterator<Windowed<KeyAndJoinSide<K>>, 
LeftOrRightValue> it = store.all()) {
                 while (it.hasNext()) {
                     final KeyValue<Windowed<KeyAndJoinSide<K>>, 
LeftOrRightValue> record = it.next();
 
                     final Windowed<KeyAndJoinSide<K>> windowedKey = record.key;
                     final LeftOrRightValue value = record.value;
+                    minTime.minTime = windowedKey.window().start();

Review comment:
       Instead of trying to update minTime on each record, could we just set it 
once in line 207 below, plus setting it to MAX if we've exhausted all records 
(as @spena indicated above)?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java
##########
@@ -69,6 +69,10 @@ public long get() {
         }
     }
 
+    static class MinTime {

Review comment:
       Could we merge this and MaxObservedStreamTime into a single class to be 
shared among operator nodes?




-- 
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