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/08/10 18:26:44 UTC

[GitHub] [kafka] lct45 opened a new pull request #9157: Update for KIP-450 to handle early records

lct45 opened a new pull request #9157:
URL: https://github.com/apache/kafka/pull/9157


   Handles records that fall between 0 and the timeDifference that would normally create negative windows. This puts a new record that falls into this range in a window from [0, timeDifference] and creates the record's right windows as later records fall into it. 
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483214805



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),

Review comment:
       Given, that we call `processEarly` only if `0 < timestamp < timeDifferenceMs`, we know that `timestamp - 2 * windows.timeDifferenceMs()` would always be negative? Thus, we can just pass in zero here?
   
   If this is correct, we might want to add a check at the beginning of this method:
   ```
   if (timestamp < 0 || timestamp >= timeDifferenceMs) {
     throw new IllegalArgumentException("...");
   }
   ```




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478490860



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -232,40 +239,54 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
 
-                    if (endTime == windows.timeDifferenceMs()) {
+                    if (startTime == 0) {
                         combinedWindow = next;
-                    } else if (endTime > timestamp && startTime <= timestamp) {
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {
                         rightWinAlreadyCreated = true;
                     }
                 }
             }
 
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null) {
+                rightWinAgg = combinedWindow.value;
+            }
+
             if (combinedWindow == null) {
                 final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
                 final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
 
             } else {
-                //create the right window for the most recent max timestamp in the combined window
-                final long rightWinStart = combinedWindow.value.timestamp() + 1;
-                if (!windowStartTimes.contains(rightWinStart) && combinedWindow.value.timestamp() < timestamp) {
-                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;
+                //only create the right window if new record falls within it and it does not already exist
+                if (!windowStartTimes.contains(maxRightWindowStart) && previousRightWindowPossible(maxRightWindowStart, timestamp)) {

Review comment:
       The comment here was unclear, that's my bad. It should read `only create the previous record's right window if the new record falls within it ...` . This part doesn't leverage `rightWinAgg`, since the agg in the previous record's right window will just be the current record's value.




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483828060



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -228,13 +345,8 @@ private void putAndForward(final Window window,
             if (windowEnd > closeTime) {
                 //get aggregate from existing window
                 final Agg oldAgg = getValueOrNull(valueAndTime);
-                final Agg newAgg;
-                // keep old aggregate if adding a right window, else add new record's value
-                if (windowStart == timestamp + 1) {
-                    newAgg = oldAgg;
-                } else {
-                    newAgg = aggregator.apply(key, value, oldAgg);
-                }
+                final Agg newAgg = aggregator.apply(key, value, oldAgg);

Review comment:
       This was just from the semi-related cleanup of splitting `putAndForward` into a separate method for `createRightWindow`, which was done after the first PR was merged (hence the cleanup occurs in this PR). I think?




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r476781126



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -148,7 +153,7 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean rightWinAlreadyCreated = false;
 
             // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
             try (
                     final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(

Review comment:
       Are you sure it's actually returning something? Have you tested it with a rocksdb store or just with the in-memory store? I think the in-memory store would handle this fine since it never serializes the key/timestamps, but if you have a rocksdb store (or a caching layer) then the range query works by looking up any data between the serialized bounds. Unfortunately a negative long is lexicographically greater than a positive long when serialized to bytes. The "negative" is encoded as a leading 1 -- which means the lower bound ends up being "larger" than the upper bound.
   
   I would assume that would result in no data being returned, but I'm not actually 100% sure what would happen




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r476783905



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -211,6 +217,67 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),

Review comment:
       See above -- even if it seems to be working I think we should restrict the range anyway. We know that there's no windows earlier than 0 so why extend the query beyond this?




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483307081



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {

Review comment:
       I think the case you're referring to above is saying that for the out-of-order case, the previous record's right window should already exist -- this line is dealing with the right window of the current record. Maybe that's a signal that we need to clarify the comment/code above (you are referring to [this comment](https://github.com/apache/kafka/pull/9157/files#r483205168), right?)




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483715845



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -228,13 +345,8 @@ private void putAndForward(final Window window,
             if (windowEnd > closeTime) {
                 //get aggregate from existing window
                 final Agg oldAgg = getValueOrNull(valueAndTime);
-                final Agg newAgg;
-                // keep old aggregate if adding a right window, else add new record's value
-                if (windowStart == timestamp + 1) {
-                    newAgg = oldAgg;
-                } else {
-                    newAgg = aggregator.apply(key, value, oldAgg);
-                }
+                final Agg newAgg = aggregator.apply(key, value, oldAgg);

Review comment:
       I think it might've gotten pulled over when updating from the original PR




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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483729402



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -161,29 +157,31 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     windowStartTimes.add(next.key.window().start());

Review comment:
       Well, `currentWindows` sound like the window of the current record, while this variable point to other windows, too.




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



[GitHub] [kafka] ableegoldman commented on pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#issuecomment-689114712


   One unrelated test failure: `kafka.api.PlaintextAdminIntegrationTest.testAlterReplicaLogDirs`


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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483695703



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;

Review comment:
       If the previous record falls within the current record's left window, then it contains values other than just the current record. There is a scenario where we find a previous record just outside of the current record, which would indicate that the current record's left window only contains the current record. Using the timestamps seemed like a simple litmus test to see if we found anything that falls within the window.




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478435832



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
       I see what you're saying about checking to see if the left window (in this case [0,timeDifference]) was already a left window and I can substitute for that check. The overall point was to catch instances of a record coming in at say 10 (with a timeDifference of 10) and also having one at 5, so 5 needs a right window but the combined window will only be picked up by this `if()` since `endTime == timestamp`. Setting `latestLeftTypeWindow = next` here will generally _not_ be a left type window, but the combined window is a special case since there are records that may need a right window in there anyways. Because this tracks the _latest_ left type window, storing the combined window should never override any other formal left type window that needs a right window.
   
   That being said, I get that this is confusing. Do you think changing the check to `if (endTime == windows.TimeDifferenceMs() && !isLeftWindow(next)) would make it seem cleaner?




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483681956



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),

Review comment:
       Makes sense to me




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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483746166



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;

Review comment:
       Thanks. Should we add a comment?
   ```
   // checks if the previous record falls into the current records left window; if yes, the left window is not empty, otherwise it is empty
   ```
   
   Or something like this?




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483839060



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       Oh yeah you're right, I'd forgotten why we added that comment in the first place. What about something briefer like
   `We don't need to store previousRecordTimestamp if maxRecordTimestamp > timestamp because the previous record's right window (if there is a previous record) would have already been created by maxRecordTimestamp`




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r476531745



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -211,6 +217,67 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),

Review comment:
       Same as above, it seems like it still works as expected




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r482234612



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -311,13 +322,8 @@ private void putAndForward(final Window window,
             if (windowEnd > closeTime) {
                 //get aggregate from existing window
                 final Agg oldAgg = getValueOrNull(valueAndTime);

Review comment:
       True, I guess there's no reason the initializer can't return null. Nevermind then




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483307289



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {

Review comment:
       +1 to this




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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483740016



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {

Review comment:
       SGTM.




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r482078343



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -254,35 +257,43 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                 rightWinAgg = combinedWindow.value;
             }
 
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
             if (combinedWindow == null) {
                 final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
                 final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
 
             } else {
-                //create the right window for the previous record if the previous record exists and the window hasn't already been created
-                if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
-                    final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
-                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
-                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
-                }
                 //update the combined window with the new aggregate
                 putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
             //create right window for new record if needed
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
             }
         }
 
-        private void createRightWindow(final long timestamp,
-                                       final ValueAndTimestamp<Agg> rightWinAgg,
-                                       final K key,
-                                       final V value,
-                                       final long closeTime) {
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
             final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
-            putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), rightWinAgg.timestamp());

Review comment:
       Yeah good point, we check for null before calling it




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483308090



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -161,29 +180,31 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     windowStartTimes.add(next.key.window().start());
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
                     } else if (endTime > timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {

Review comment:
       Ah yeah, I do throw one in the updated version of this in the reverse iterator PR, but it hasn't gotten moved over here yet. I'll do that.




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478489438



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -232,40 +239,54 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
 
-                    if (endTime == windows.timeDifferenceMs()) {
+                    if (startTime == 0) {
                         combinedWindow = next;
-                    } else if (endTime > timestamp && startTime <= timestamp) {
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {
                         rightWinAlreadyCreated = true;
                     }
                 }
             }
 
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window

Review comment:
       There can be more than one record in the combined window, but if we don't find any right window agg then there are no records to the left of this record (in time) AND to the right of this record (could be one or the other). There could be multiple to the right though.
   
   It took me like 5 minutes to figure this out, but actually the call to `previousRightWindowPossible` is when we're creating the right window for the previous record, not for the current record. This call doesn't involved `rightWindowAgg` at all, since the check `rightWindowStart > currentRecordTimestamp` wouldn't help us figure out if `previous record > current record`.
   
   The check for this is actually a little further down (line 277) where we do `rightWindowIsNotEmpty`, and there it confirms that the values in rightWinAgg are > timestamp. 




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478768625



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,386 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window (window from [0,timeDifference] that contains any early records)
+                        // if it is found so that a right window can be created for the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousWindowEnd = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousWindowEnd == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousWindowEnd + 1;
+                if (rightWindowNecessaryAndPossible(windowStartTimes, rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create && max record falls within left window --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && recordFallsWithinCurrentRecordsLeftWindow(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;

Review comment:
       I know it's effectively the same thing, but it feels a bit harder to reason about a "hypothetical previous record's right window that may actually not be a previous record at all" than just "we do/do not have a previous record"




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483307081



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {

Review comment:
       I think the case you're referring to above is saying that for the out-of-order case, the previous record's right window should already exist -- this line is dealing with the right window of the current record. Maybe that's a signal that we need to clarify the comment/code above (you are referring to this, right?)




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478761540



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousRecord = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousRecord == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousRecord + 1;
+                if (!windowStartTimes.contains(rightWinStart) && previousRightWindowPossible(rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && previousRightWindowPossible(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;
+                //only create the right window if new record falls within it and it does not already exist
+                if (!windowStartTimes.contains(maxRightWindowStart) && previousRightWindowPossible(maxRightWindowStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(maxRightWindowStart, maxRightWindowStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        private boolean previousRightWindowPossible(
+            final long rightWindowStart,
+            final long currentRecordTimestamp) {

Review comment:
       I guess "Necessary" still seems kind of open-ended/vague. By that you mean, "is not already created", right? But maybe we should wait until we see the final form of this method in case there are any further changes, and then we can go back and try to fish out a more specific name




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r476781126



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -148,7 +153,7 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean rightWinAlreadyCreated = false;
 
             // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
             try (
                     final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(

Review comment:
       Are you sure it's actually returning something? Have you tested it with a rocksdb store or just with the in-memory store? I think the in-memory store would handle this fine since it never serializes the key/timestamps, but if you have a rocksdb store (or a caching layer) then the range query works by looking up any data between the serialized bounds. Unfortunately a negative long is lexicographically greater than a positive long when serialized to bytes. The "negative" is encoded as a leading 1 -- which means the lower bound ends up being "larger" than the upper bound. I assume that would result in no data being returned, but I'm not actually 100% sure what would happen




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478759996



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
        I guess you could just have a separate `setPreviousRecordTimestampIfNecessary(window, previousRecordTimestamp)` method that sets the `previousRecordTimestamp` to the window's max timestamp if it's larger. And then if it ends up that `previousRecordTimestamp == timestamp` then we can automatically skip all of the window creation 




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r481375834



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -164,11 +161,13 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
+                        // update to store the previous record

Review comment:
       I'm not sure we need to check that, I think that by the nature of going through windows forward, the next window we find will always have a max timestamp that's larger than the previous window. Right? Lemme do an example:
   Record comes in @30, previous record was at @23, timeDifference = 10. The last window we find with an endTime < timestamp will be 23's left window, where the max record value is 23. Any earlier windows with endTime < timestamp will have a max value less than 23, so we can override them safely




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483685325



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       Going back and re-reading this comments in the context of Matthias's later comment, it seems like maybe this one is unnecessary. Correct me if I'm wrong, but it feels like the statement `if (windowMaxRecordTimestamp < timestamp) {
                               previousRecordTimestamp = windowMaxRecordTimestamp;
                           }`
   is somewhat self explanatory. I also don't think I ever leverage the idea that if there is a record before the current record, then the previous record's right window has already been created. Below, I still check `previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)`. I think either the logic can be updated to leverage what the above comment indicates, or we can cut it out and keep the logic simple. WDYT?




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478758779



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
       Yeah sorry I didn't mean that we shouldn't have any conditionals here whatsoever, I just meant that we don't need the combined window check (or really anything other than what we need to accurately set `previousRecordTimestamp`)




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r485221052



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -146,96 +165,203 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean leftWinAlreadyCreated = false;
             boolean rightWinAlreadyCreated = false;
 
-            // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            Long previousRecordTimestamp = null;
+
             try (
                 final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
                     key,
                     key,
-                    timestamp - 2 * windows.timeDifferenceMs(),
+                    Math.max(0, inputRecordTimestamp - 2 * windows.timeDifferenceMs()),
                     // to catch the current record's right window, if it exists, without more calls to the store
-                    timestamp + 1)
+                    inputRecordTimestamp + 1)
             ) {
                 while (iterator.hasNext()) {
-                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next = iterator.next();
-                    windowStartTimes.add(next.key.window().start());
-                    final long startTime = next.key.window().start();
+                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> windowBeingProcessed = iterator.next();
+                    final long startTime = windowBeingProcessed.key.window().start();
+                    windowStartTimes.add(startTime);
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = windowBeingProcessed.value.timestamp();
 
-                    if (endTime < timestamp) {
-                        leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
-                    } else if (endTime == timestamp) {
+                    if (endTime < inputRecordTimestamp) {
+                        leftWinAgg = windowBeingProcessed.value;
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
+                    } else if (endTime == inputRecordTimestamp) {
                         leftWinAlreadyCreated = true;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else if (endTime > timestamp && startTime <= timestamp) {
-                        rightWinAgg = next.value;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                        if (windowMaxRecordTimestamp < inputRecordTimestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (endTime > inputRecordTimestamp && startTime <= inputRecordTimestamp) {
+                        rightWinAgg = windowBeingProcessed.value;
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (startTime == inputRecordTimestamp + 1) {
                         rightWinAlreadyCreated = true;
+                    } else {
+                        throw new IllegalStateException("Unexpected window found when processing sliding windows");

Review comment:
       nit: log an error and include the relevant info (eg `windowStart` and `inputRecordTimestamp` at least). Same for the IllegalStateException in `processEarly`




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r481460413



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +190,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);

Review comment:
       Sorry, I think my previous comment was a bit unclear. I meant that we should move this block up to where we set the `rightWinAgg` right after the loop, since this is what we actually use the `rightWinAgg` for.
   I think the "create right window for previous record" logic is fine wherever

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -254,35 +257,43 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                 rightWinAgg = combinedWindow.value;
             }
 
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
             if (combinedWindow == null) {
                 final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
                 final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
 
             } else {
-                //create the right window for the previous record if the previous record exists and the window hasn't already been created
-                if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
-                    final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
-                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
-                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
-                }
                 //update the combined window with the new aggregate
                 putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
             //create right window for new record if needed
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
             }
         }
 
-        private void createRightWindow(final long timestamp,
-                                       final ValueAndTimestamp<Agg> rightWinAgg,
-                                       final K key,
-                                       final V value,
-                                       final long closeTime) {
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
             final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
-            putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), rightWinAgg.timestamp());

Review comment:
       Do we need to use `getValueOrNull` here? It seems like `rightWinAgg` should never be null if we are using it to create a right window

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -311,13 +322,8 @@ private void putAndForward(final Window window,
             if (windowEnd > closeTime) {
                 //get aggregate from existing window
                 final Agg oldAgg = getValueOrNull(valueAndTime);

Review comment:
       Same here, can this ever be null? It doesn't seem like it, even if it's a new window we still pass in the initializer value so `valueAndTime` won't ever be null. ...right?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the previous record if the previous record exists and the window hasn't already been created
+                if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                    final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createRightWindow(final long timestamp,
+                                       final ValueAndTimestamp<Agg> rightWinAgg,
+                                       final K key,
+                                       final V value,
+                                       final long closeTime) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+            putAndForward(window, valueAndTime, key, value, closeTime, timestamp);

Review comment:
       Oh good point, we definitely need the key. But I think separating them turned out well 

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the previous record if the previous record exists and the window hasn't already been created
+                if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                    final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createRightWindow(final long timestamp,
+                                       final ValueAndTimestamp<Agg> rightWinAgg,
+                                       final K key,
+                                       final V value,
+                                       final long closeTime) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+            putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
+        }

Review comment:
       I think you mean  `processInOrder`, not  `processEarly` (this is correct in the code, it's just the latest comment that doesn't match up)




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



[GitHub] [kafka] ableegoldman commented on pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#issuecomment-689272398


   Looks like the three Java builds all passed, but the CI build failed due to
   ```
   [2020-09-09T02:35:12.324Z] + mkdir test-streams-archetype
   [2020-09-09T02:35:12.324Z] mkdir: cannot create directory 'test-streams-archetype': File exists
   [2020-09-09T02:35:12.325Z] + echo Could not create test directory for stream quickstart archetype
   [2020-09-09T02:35:12.325Z] Could not create test directory for stream quickstart archetype
   ```
   
   I guess there was some issue with cleaning up or running this twice...?


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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r485053442



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
+        }
+
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
+            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;

Review comment:
       I think the catch is that it's only _sometimes_ the aggregate that goes in the current record's right window. Sometimes we don't use the value in `rightWingAgg` because the right window is empty/doesn't exist, but if there _is_ a right window aggregate, it will be in `rightWinAgg`




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r485239998



##########
File path: streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -439,6 +442,185 @@ public void testJoin() {
         }
     }
 
+    @SuppressWarnings("unchecked")

Review comment:
       That seems weird to me. Guessing it's ultimately due to the `supplier.theCapturedProcessor().processed()` we loop over. But then wouldn't we get the warning a bit earlier? 🤷‍♀️ 




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



[GitHub] [kafka] mjsax merged pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax merged pull request #9157:
URL: https://github.com/apache/kafka/pull/9157


   


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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483077238



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {

Review comment:
       q: Should we rename `currentTimestamp` -> `timestamp` for consistency? (or maybe, rename `timestamp` to `inputRecordTimestamp` all over the place?




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r479411830



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
       Would the window's max timestamp always be larger than the previous window's by nature of moving forward in time? I think it would be somewhat more convenient but I feel like adding additional boolean checks makes the algorithm feel longer, especially if the issue will get handled in later checks




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483685325



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       Going back and re-reading this comments in the context of Matthias's later comment, it seems like maybe this one is unnecessary and could be combined with the later one. Correct me if I'm wrong, but it feels like the statement `if (windowMaxRecordTimestamp < timestamp) {
                               previousRecordTimestamp = windowMaxRecordTimestamp;
                           }`
   is somewhat self explanatory and only needs justification when we set the `rightWinAgg` later.




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r477784084



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
       We only need to check if the previous right window needs to be created if the current record's left window was _not_ previously a left-type window, ie the max_timestamp < timestamp. Otherwise, we'd have already created any windows since that implies we processed a record with this exact timestamp already. 
   So then why do we set `latestLeftTypeWindow = next` ? It seems like it's possible that this actually isn't a left-type window, in which case we shouldn't overwrite any existing value for `latestLeftTypeWindow`.
   On the other hand, if it actually _is_ a left type window, then that means we don't need to create the previous record's right window so we should just set it to null. But that seems to apply regardless of whether this is the early record combined window or not? 

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed

Review comment:
       This comment kind of comes out of nowhere since there's no concept of the "combined window" outside of `processEarly`. Maybe you can just add a quick mention of what it is and that it's for early records

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -174,12 +181,11 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     }
                 }
             }
-
             //create right window for previous record
             if (latestLeftTypeWindow != null) {
-                final long leftWindowEnd = latestLeftTypeWindow.key.window().end();
-                final long rightWinStart = leftWindowEnd == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : leftWindowEnd + 1;
-                if (!windowStartTimes.contains(rightWinStart)) {
+                final long previousRecord = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousRecord == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousRecord + 1;

Review comment:
       I'm having trouble wrapping my head around this line. Why would we create a right window at `latestLeftTypeWindow.maxTimestamp + 1` if the previous record was at `timeDifferenceMs`? Wouldn't we have created the right window for whatever is at `latestLeftTypeWindow.maxTimestamp + 1` when we processed the `previousRecord`?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -232,40 +239,54 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
 
-                    if (endTime == windows.timeDifferenceMs()) {
+                    if (startTime == 0) {
                         combinedWindow = next;
-                    } else if (endTime > timestamp && startTime <= timestamp) {
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {
                         rightWinAlreadyCreated = true;
                     }
                 }
             }
 
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null) {
+                rightWinAgg = combinedWindow.value;
+            }
+
             if (combinedWindow == null) {
                 final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
                 final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
 
             } else {
-                //create the right window for the most recent max timestamp in the combined window
-                final long rightWinStart = combinedWindow.value.timestamp() + 1;
-                if (!windowStartTimes.contains(rightWinStart) && combinedWindow.value.timestamp() < timestamp) {
-                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;
+                //only create the right window if new record falls within it and it does not already exist
+                if (!windowStartTimes.contains(maxRightWindowStart) && previousRightWindowPossible(maxRightWindowStart, timestamp)) {

Review comment:
       See above: we shouldn't rely on `previousRightWindow` here. Actually I don't think we need it at all? (assuming we move the check in it to the condition above where we use the combined window agg)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousRecord = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousRecord == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousRecord + 1;
+                if (!windowStartTimes.contains(rightWinStart) && previousRightWindowPossible(rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && previousRightWindowPossible(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;
+                //only create the right window if new record falls within it and it does not already exist
+                if (!windowStartTimes.contains(maxRightWindowStart) && previousRightWindowPossible(maxRightWindowStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(maxRightWindowStart, maxRightWindowStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        private boolean previousRightWindowPossible(
+            final long rightWindowStart,
+            final long currentRecordTimestamp) {

Review comment:
       Ok, I see that `rightWindowStart <= currentRecordTimestamp` isn't necessarily true when you call this from `processEarly` but I think you're kind of abusing this poor method 😜 . I would keep things simple here and make sure the parameters always mean exactly the same thing when you call this, ie `rightWindowStart` should _always_ mean that "the start time of the right window for the record which is previous to the current record" . If that means some duplicated boolean checks here and there, so be it.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -189,8 +195,8 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create --> new record's left window is not empty

Review comment:
       Now that I think about it, this isn't exactly true -- you could have the previous record at 10 and the current record at 21 (size is 10), then you'd have to create a right window for [11, 21] but the left window is also [11, 21] which is empty (except for the current record of course).So you'd want to check that `previousRecord >= currentRecord - timeDifference` which is actually technically what `previousRightWindowPossible(rightWindow, timestamp)` returns but it took me a long time to figure all that out because we're sort of passing in the wrong parameters here. In this call we pass in `latestLeftTypeWindow.value.timestamp` which is equivalent to `previousRecord.timestamp` which is not conceptually the same as the `rightWindowStart` . I know the logic works out the way you intended but it's pretty hard to untangle. You should update the comment and create a separate method for this case which you can name more accurately for this specific scenario

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {

Review comment:
       Not sure I understand the `|| endTime == windows.timeDifferenceMs()`, is that left over from the 1st PR? It seems important to enforce that the window `latestLeftTypeWindow` points to is actually a left window

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -232,40 +239,54 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
 
-                    if (endTime == windows.timeDifferenceMs()) {
+                    if (startTime == 0) {
                         combinedWindow = next;
-                    } else if (endTime > timestamp && startTime <= timestamp) {
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {
                         rightWinAlreadyCreated = true;
                     }
                 }
             }
 
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window

Review comment:
       `iwll` 🙂 

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousRecord = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousRecord == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousRecord + 1;
+                if (!windowStartTimes.contains(rightWinStart) && previousRightWindowPossible(rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && previousRightWindowPossible(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;
+                //only create the right window if new record falls within it and it does not already exist
+                if (!windowStartTimes.contains(maxRightWindowStart) && previousRightWindowPossible(maxRightWindowStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(maxRightWindowStart, maxRightWindowStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        private boolean previousRightWindowPossible(
+            final long rightWindowStart,
+            final long currentRecordTimestamp) {

Review comment:
       Parameter alignment is off. Also the naming here isn't very clear about what it's for/what it means. Don't have great suggestions but maybe `previousRightWindowMustBeCreated` or even just `previousRecordIsWithinMaxTimeDifferenceFromCurrentRecord` and then a quick comment saying that this means we will have to create a right window for the previous record. 
   
   Also shouldn't `rightWindowStart <= currentRecordTimestamp` always be true?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -232,40 +239,54 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
 
-                    if (endTime == windows.timeDifferenceMs()) {
+                    if (startTime == 0) {
                         combinedWindow = next;
-                    } else if (endTime > timestamp && startTime <= timestamp) {
+                    } else if (endTime >= timestamp && startTime <= timestamp) {

Review comment:
       Actually, do we even need the `endTime >= timestamp` part of the condition? We're really just iterating over the single dimension of the `startTime` from 0 to `timestamp + 1`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -232,40 +239,54 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
 
-                    if (endTime == windows.timeDifferenceMs()) {
+                    if (startTime == 0) {
                         combinedWindow = next;
-                    } else if (endTime > timestamp && startTime <= timestamp) {
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {
                         rightWinAlreadyCreated = true;
                     }
                 }
             }
 
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window

Review comment:
       Also, cool,  I think I understand the concept here but some of the details are a bit fuzzy. Basically if we don't find a right window agg that means we didn't find any windows (besides the combined window), which in turn means that there can only be a single record in the combined window (otherwise you'd get a right window for the earlier record). 
   So we need to use the combined window agg for the current record's right window.  But we should only do that if the one record is actually after the current record, right? I think you actually do implicitly check that is the case below but it's pretty subtle: basically in `previousRightWindowPossible` you would return false if `rightWindowStart > currentRecordTimestamp`. But we can check that right here and make it explicit, so that `rightWinAgg` only ever means the aggregate that we will actually put in the current record's right window. Then we can also clean up `previousRightWindowPossible`




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483302934



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -161,29 +180,31 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     windowStartTimes.add(next.key.window().start());
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
                     } else if (endTime > timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {

Review comment:
       It was my suggestion to explicitly check `if (startTime == timestamp + 1)` instead of just falling back to `else`, for code clarify and safety. But +1 to adding the `else throw IllegalStateException`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -161,29 +180,31 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     windowStartTimes.add(next.key.window().start());
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
                     } else if (endTime > timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {

Review comment:
       It was my suggestion to explicitly check `if (startTime == timestamp + 1)` instead of just falling back to `else`, for code clarify and safety, so blame me. But +1 to adding the `else throw IllegalStateException`




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478454716



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousRecord = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousRecord == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousRecord + 1;
+                if (!windowStartTimes.contains(rightWinStart) && previousRightWindowPossible(rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && previousRightWindowPossible(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;
+                //only create the right window if new record falls within it and it does not already exist
+                if (!windowStartTimes.contains(maxRightWindowStart) && previousRightWindowPossible(maxRightWindowStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(maxRightWindowStart, maxRightWindowStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        private boolean previousRightWindowPossible(
+            final long rightWindowStart,
+            final long currentRecordTimestamp) {

Review comment:
       Update: after parsing out the sketchy use of this function I went with 
   `// previous record's right window does not already exist and current record falls within previous record's right window
           private boolean rightWindowNecessaryAndPossible`
   I added the check for !windowStartTimesContains()` to clean up the original check (since both check that), which is why I added the "Necessary" to the function name. WDYT?




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478437596



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousRecord = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousRecord == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousRecord + 1;
+                if (!windowStartTimes.contains(rightWinStart) && previousRightWindowPossible(rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && previousRightWindowPossible(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;
+                //only create the right window if new record falls within it and it does not already exist
+                if (!windowStartTimes.contains(maxRightWindowStart) && previousRightWindowPossible(maxRightWindowStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(maxRightWindowStart, maxRightWindowStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        private boolean previousRightWindowPossible(
+            final long rightWindowStart,
+            final long currentRecordTimestamp) {

Review comment:
       Yeah I definitely overused this method, the boolean checks were all so similar but not quite the same that I just kinda forced it together. In terms of duplicated boolean checks, do you mean similar boolean methods would be preferable? Or keeping the boolean checks in the main algorithm




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r481512388



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -118,24 +118,20 @@ public void process(final K key, final V value) {
             }
 
             final long timestamp = context().timestamp();
-            //don't process records that don't fall within a full sliding window
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+

Review comment:
       By the way, I think we should also check if the record is so old that even the latest window it could possibly create/affect would be dropped, and then not process the record at all. (ie basically check if the current record's right window would be dropped) We can record on the lateRecordDropSensor and log the message using the current record's left window. 




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r485221931



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -146,96 +165,203 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean leftWinAlreadyCreated = false;
             boolean rightWinAlreadyCreated = false;
 
-            // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            Long previousRecordTimestamp = null;
+
             try (
                 final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
                     key,
                     key,
-                    timestamp - 2 * windows.timeDifferenceMs(),
+                    Math.max(0, inputRecordTimestamp - 2 * windows.timeDifferenceMs()),
                     // to catch the current record's right window, if it exists, without more calls to the store
-                    timestamp + 1)
+                    inputRecordTimestamp + 1)
             ) {
                 while (iterator.hasNext()) {
-                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next = iterator.next();
-                    windowStartTimes.add(next.key.window().start());
-                    final long startTime = next.key.window().start();
+                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> windowBeingProcessed = iterator.next();
+                    final long startTime = windowBeingProcessed.key.window().start();
+                    windowStartTimes.add(startTime);
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = windowBeingProcessed.value.timestamp();
 
-                    if (endTime < timestamp) {
-                        leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
-                    } else if (endTime == timestamp) {
+                    if (endTime < inputRecordTimestamp) {
+                        leftWinAgg = windowBeingProcessed.value;
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
+                    } else if (endTime == inputRecordTimestamp) {
                         leftWinAlreadyCreated = true;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else if (endTime > timestamp && startTime <= timestamp) {
-                        rightWinAgg = next.value;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                        if (windowMaxRecordTimestamp < inputRecordTimestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (endTime > inputRecordTimestamp && startTime <= inputRecordTimestamp) {
+                        rightWinAgg = windowBeingProcessed.value;
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (startTime == inputRecordTimestamp + 1) {
                         rightWinAlreadyCreated = true;
+                    } else {
+                        throw new IllegalStateException("Unexpected window found when processing sliding windows");
                     }
                 }
             }
 
             //create right window for previous record
-            if (latestLeftTypeWindow != null) {
-                final long rightWinStart = latestLeftTypeWindow.end() + 1;
-                if (!windowStartTimes.contains(rightWinStart)) {
-                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
-                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
-                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            if (previousRecordTimestamp != null) {
+                final long previousRightWinStart = previousRecordTimestamp + 1;
+                if (rightWindowNecessaryAndPossible(windowStartTimes, previousRightWinStart, inputRecordTimestamp)) {
+                    final TimeWindow window = new TimeWindow(previousRightWinStart, previousRightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp);
+                    updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp);
                 }
             }
 
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
-                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (leftWindowNotEmpty(previousRecordTimestamp, inputRecordTimestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), inputRecordTimestamp);
                 } else {
-                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp);
                 }
-                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
-                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                final TimeWindow window = new TimeWindow(inputRecordTimestamp - windows.timeDifferenceMs(), inputRecordTimestamp);
+                updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp);
             }
-            //create right window for new record
-            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
-                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, inputRecordTimestamp)) {
+                createCurrentRecordRightWindow(inputRecordTimestamp, rightWinAgg, key);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        /**
+         * Created to handle records where 0 < inputRecordTimestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, we will put them into the [0, timeDifferenceMs]
+         * window as a "workaround", and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long inputRecordTimestamp, final long closeTime) {
+            if (inputRecordTimestamp < 0 || inputRecordTimestamp >= windows.timeDifferenceMs()) {
+                throw new IllegalArgumentException("Early record for sliding windows must fall between 0 < inputRecordTimestamp < timeDifferenceMs");

Review comment:
       nit: log an error and include the `inputRecordTimestamp` 




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478445137



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {

Review comment:
       It is, until we have a combined window that holds records that don't have corresponding left windows. I treated the combined window as a defacto left window since it's taking the place of early record's left windows. EX: for `timeDifference=10`, a record at 4 would be stored in [0,10]. If a record comes in at 11, we need the window from [5,15], but we only create that if we find a `leftTypeWindow`, which doesn't exist for 4. For these records, [0,10] is the only window we have, and therefore the closest thing to a left type window.
   
   I didn't change variable names because I wanted the original algorithm to be understandable. It feels like adding early records has added a significant amount of complexity and I do wonder if it's worth it to have the addition confusion for testing flexibility - WDYT?




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478763071



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -189,8 +195,8 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create --> new record's left window is not empty

Review comment:
       Here too, is this kind of moot now that we can just track the `previousRecordTimestamp`? IIUC all we really want to do is make sure that the left window is not empty, which is actually a pretty simple calculation in terms of the `previousRecordTimestamp`




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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r481764391



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -161,29 +157,31 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     windowStartTimes.add(next.key.window().start());

Review comment:
       nit: `next` is not a great name; maybe `existingWindow` instead?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
+        }
+
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
+            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;

Review comment:
       Why do we need to check the timestamp?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;

Review comment:
       Why do we do this check based on timestamps?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       Not sure if I understand the last sentence: `This will always be true for early records, as they all fall within [0, timeDifferenceMs]`?
   
   Also not sure how this comment relates to setting `previousRecordTimestamp` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
+        }
+
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {

Review comment:
       Same question as above about `currentRecordTimestamp` (It seems best to me, to use the same variable name for the same think throughout all methods.)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {

Review comment:
       q: Should we rename `currentTimestamp` -> `timestamp` for consistency? (or maybe, rename `timestamp` to `inputRecordTimestamp` all other the place?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();

Review comment:
       nit: flip both lines:
   ```
   final long startTime = next.key.window().start();
   windowStartTimes.add(startTime);
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -228,13 +345,8 @@ private void putAndForward(final Window window,
             if (windowEnd > closeTime) {
                 //get aggregate from existing window
                 final Agg oldAgg = getValueOrNull(valueAndTime);
-                final Agg newAgg;
-                // keep old aggregate if adding a right window, else add new record's value
-                if (windowStart == timestamp + 1) {
-                    newAgg = oldAgg;
-                } else {
-                    newAgg = aggregator.apply(key, value, oldAgg);
-                }
+                final Agg newAgg = aggregator.apply(key, value, oldAgg);

Review comment:
       Not sure how this change relates to "early records"?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]

Review comment:
       nit: `Instead, they will fall within the [0, timeDifferenceMs]` -> `Instead, we will put them into the [0, timeDifferenceMs] window as a "workaround",

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);

Review comment:
       nit: should we rename `putAndForward` to `updateWindowAndForward`?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {

Review comment:
       Seem this can be a simple `else`? No need to verify the condition? (Maybe it's helpful to add a comment instead?)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),

Review comment:
       Given, that we call `processEarly` only if `0 < timestamp < timeDifferenceMs`, we now that `timestamp - 2 * windows.timeDifferenceMs()` would always be negative? Thus, we can just pass in zero here?
   
   If this is correct, we might want to add a check at the beginning of this method:
   ```
   if (timestamp < 0 || timestamp >= timeDifferenceMs) {
     throw new IllegalArgumentException("...");
   }
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -161,29 +180,31 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     windowStartTimes.add(next.key.window().start());
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
                     } else if (endTime > timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {

Review comment:
       Seems we don't need the `if` as this is the only valid case? Adding a comment might be helpful.
   
   If you want to keep the `if` as a sanity check, we should add a final:
   ```
   } else {
     throw new IllegalStateException(...):
   }
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {

Review comment:
       Not sure if I can follow. If `combinedWindow.value.timestamp() > timestamp` this seems to imply that the current record is out-of-order? Above, you state that for the out-of-order case, the right window would always exist already though? So why do we need this additional 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.

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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478768335



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,386 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window (window from [0,timeDifference] that contains any early records)
+                        // if it is found so that a right window can be created for the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousWindowEnd = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousWindowEnd == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousWindowEnd + 1;
+                if (rightWindowNecessaryAndPossible(windowStartTimes, rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create && max record falls within left window --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && recordFallsWithinCurrentRecordsLeftWindow(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;

Review comment:
       This is also kind of unclear (what is a max right window?), but I get that we can't call it `previousRecordRightWindow` since we don't know that it is a previous record or not at this point. I think yet again, just keeping track of the previous record's timestamp as we iterate through the windows, will be the most clear; if `previousRecordTimestamp` is still null by this point, we know right away that we don't have to create a previous right window. And then we can actually drop the `rightWindowNecessaryAndPossible` check altogether, since we know the current record has to be in range of the right window of the previous record (since we're in `processEarly`). The one exception is if the previous record and current record are on the same timestamp, so we can actually skip the previous right window creation if `previousRecordTimestamp ` is `null` OR equal to `timestamp`




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478762667



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -174,12 +181,11 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     }
                 }
             }
-
             //create right window for previous record
             if (latestLeftTypeWindow != null) {
-                final long leftWindowEnd = latestLeftTypeWindow.key.window().end();
-                final long rightWinStart = leftWindowEnd == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : leftWindowEnd + 1;
-                if (!windowStartTimes.contains(rightWinStart)) {
+                final long previousRecord = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousRecord == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousRecord + 1;

Review comment:
       Ok before I continue to try and wrap my head around this, is this particular line going to be moot if we start tracking `previousRecordTimestamp` instead of `latestLeftTypeWindow`? The point of it was just to distinguish that special case, but now we can just say `rightWindowStart = previousRecordTimestamp + 1` -- does that sound right?




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478762151



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,386 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window (window from [0,timeDifference] that contains any early records)
+                        // if it is found so that a right window can be created for the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousWindowEnd = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousWindowEnd == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousWindowEnd + 1;
+                if (rightWindowNecessaryAndPossible(windowStartTimes, rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create && max record falls within left window --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && recordFallsWithinCurrentRecordsLeftWindow(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;
+                //only create the previous record's right window if the new record falls within it and it does not already exist
+                if (rightWindowNecessaryAndPossible(windowStartTimes, maxRightWindowStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(maxRightWindowStart, maxRightWindowStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        private boolean recordFallsWithinCurrentRecordsLeftWindow(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
+        }
+
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(
+                                                    final Set<Long> windowStartTimes,
+                                                    final long rightWindowStart,
+                                                    final long currentRecordTimestamp) {

Review comment:
       Alignment is still off, the first parameter should be on the same line as the method declaration. Also, if this method is only ever used in deciding whether to create the previous record's right window, then let's name the parameter `previousRecordRightWindowStart` or something. Of course, you could also just pass in the `previousRecordTimestamp` now that we have that, and then do the math in here. Whatever you think makes the most sense




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478769004



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,386 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window (window from [0,timeDifference] that contains any early records)
+                        // if it is found so that a right window can be created for the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousWindowEnd = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousWindowEnd == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousWindowEnd + 1;
+                if (rightWindowNecessaryAndPossible(windowStartTimes, rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create && max record falls within left window --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && recordFallsWithinCurrentRecordsLeftWindow(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;
+                //only create the previous record's right window if the new record falls within it and it does not already exist
+                if (rightWindowNecessaryAndPossible(windowStartTimes, maxRightWindowStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(maxRightWindowStart, maxRightWindowStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);

Review comment:
       Is this the same as in the non-early `process`? Maybe we can factor it out into  its own `createRightWindowIfNeeded`(or whatever)  method.




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483835732



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       > the statement `if (windowMaxRecordTimestamp < timestamp) { previousRecordTimestamp = windowMaxRecordTimestamp; }`is somewhat self explanatory
   
   I think that's fair. My concern was with the `windowMaxRecordTimestamp > timestamp` case -- in that situation, we don't know and can't know what the `previousRecordTimestamp` is, because all we save is the maxTimestamp of the combined window and therefore the information is lost. I just thought we should clarify that this is actually ok, because if `windowMaxRecordTimestamp > timestamp` then we must have already created the right window of the previous record. So I agree that the `!windowStartTimes.contains(previousRecordTimestamp + 1)` check would logically catch this, but I don't think we can remove either check: 
   
   If we remove the `if (windowMaxRecordTimestamp < timestamp) { previousRecordTimestamp = windowMaxRecordTimestamp; }` then we might set `previousRecordTimestamp` to a value that isn't actually the timestamp of a previous record, and we should make sure all variables are always accurate
   
   If we remove the `!windowStartTimes.contains(previousRecordTimestamp + 1)` then we could miss a case where the previous record's right window was already created, but by a record out side of the combined window
   
   So, I think the code itself is complete as is. But probably we can simplify the comments to make it more understandable -- it seems reasonable to remove the comment here entirely. Then maybe you could leave a brief comment down below where we create the previous record's right window saying "if the `previousRecordTimestamp` is null, either there is no previous record or it's right window has already been created"




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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r485129599



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -117,25 +117,44 @@ public void process(final K key, final V value) {
                 return;
             }
 
-            final long timestamp = context().timestamp();
-            //don't process records that don't fall within a full sliding window
-            if (timestamp < windows.timeDifferenceMs()) {
+            final long inputRecordTimestamp = context().timestamp();
+            observedStreamTime = Math.max(observedStreamTime, inputRecordTimestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (inputRecordTimestamp + 1 + windows.timeDifferenceMs() <= closeTime) {

Review comment:
       nit: `1` -> `1L`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -146,96 +165,203 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean leftWinAlreadyCreated = false;
             boolean rightWinAlreadyCreated = false;
 
-            // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            Long previousRecordTimestamp = null;
+
             try (
                 final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
                     key,
                     key,
-                    timestamp - 2 * windows.timeDifferenceMs(),
+                    Math.max(0, inputRecordTimestamp - 2 * windows.timeDifferenceMs()),
                     // to catch the current record's right window, if it exists, without more calls to the store
-                    timestamp + 1)
+                    inputRecordTimestamp + 1)
             ) {
                 while (iterator.hasNext()) {
-                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next = iterator.next();
-                    windowStartTimes.add(next.key.window().start());
-                    final long startTime = next.key.window().start();
+                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> windowBeingProcessed = iterator.next();
+                    final long startTime = windowBeingProcessed.key.window().start();
+                    windowStartTimes.add(startTime);
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = windowBeingProcessed.value.timestamp();
 
-                    if (endTime < timestamp) {
-                        leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
-                    } else if (endTime == timestamp) {
+                    if (endTime < inputRecordTimestamp) {
+                        leftWinAgg = windowBeingProcessed.value;
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
+                    } else if (endTime == inputRecordTimestamp) {
                         leftWinAlreadyCreated = true;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else if (endTime > timestamp && startTime <= timestamp) {
-                        rightWinAgg = next.value;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                        if (windowMaxRecordTimestamp < inputRecordTimestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (endTime > inputRecordTimestamp && startTime <= inputRecordTimestamp) {
+                        rightWinAgg = windowBeingProcessed.value;
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (startTime == inputRecordTimestamp + 1) {
                         rightWinAlreadyCreated = true;
+                    } else {
+                        throw new IllegalStateException("Unexpected window found when processing sliding windows");
                     }
                 }
             }
 
             //create right window for previous record
-            if (latestLeftTypeWindow != null) {
-                final long rightWinStart = latestLeftTypeWindow.end() + 1;
-                if (!windowStartTimes.contains(rightWinStart)) {
-                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
-                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
-                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            if (previousRecordTimestamp != null) {
+                final long previousRightWinStart = previousRecordTimestamp + 1;
+                if (rightWindowNecessaryAndPossible(windowStartTimes, previousRightWinStart, inputRecordTimestamp)) {
+                    final TimeWindow window = new TimeWindow(previousRightWinStart, previousRightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp);
+                    updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp);
                 }
             }
 
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
-                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (leftWindowNotEmpty(previousRecordTimestamp, inputRecordTimestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), inputRecordTimestamp);
                 } else {
-                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp);
                 }
-                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
-                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                final TimeWindow window = new TimeWindow(inputRecordTimestamp - windows.timeDifferenceMs(), inputRecordTimestamp);
+                updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp);
             }
-            //create right window for new record
-            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
-                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, inputRecordTimestamp)) {
+                createCurrentRecordRightWindow(inputRecordTimestamp, rightWinAgg, key);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        /**
+         * Created to handle records where 0 < inputRecordTimestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, we will put them into the [0, timeDifferenceMs]
+         * window as a "workaround", and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long inputRecordTimestamp, final long closeTime) {
+            if (inputRecordTimestamp < 0 || inputRecordTimestamp >= windows.timeDifferenceMs()) {
+                throw new IllegalArgumentException("Early record for sliding windows must fall between 0 < inputRecordTimestamp < timeDifferenceMs");

Review comment:
       nit: `fall between 0 < inputRecordTimestamp` -> `fall between 0 <= inputRecordTimestamp`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -146,96 +165,203 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean leftWinAlreadyCreated = false;
             boolean rightWinAlreadyCreated = false;
 
-            // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            Long previousRecordTimestamp = null;
+
             try (
                 final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
                     key,
                     key,
-                    timestamp - 2 * windows.timeDifferenceMs(),
+                    Math.max(0, inputRecordTimestamp - 2 * windows.timeDifferenceMs()),
                     // to catch the current record's right window, if it exists, without more calls to the store
-                    timestamp + 1)
+                    inputRecordTimestamp + 1)
             ) {
                 while (iterator.hasNext()) {
-                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next = iterator.next();
-                    windowStartTimes.add(next.key.window().start());
-                    final long startTime = next.key.window().start();
+                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> windowBeingProcessed = iterator.next();
+                    final long startTime = windowBeingProcessed.key.window().start();
+                    windowStartTimes.add(startTime);
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = windowBeingProcessed.value.timestamp();
 
-                    if (endTime < timestamp) {
-                        leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
-                    } else if (endTime == timestamp) {
+                    if (endTime < inputRecordTimestamp) {
+                        leftWinAgg = windowBeingProcessed.value;
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
+                    } else if (endTime == inputRecordTimestamp) {
                         leftWinAlreadyCreated = true;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else if (endTime > timestamp && startTime <= timestamp) {
-                        rightWinAgg = next.value;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                        if (windowMaxRecordTimestamp < inputRecordTimestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (endTime > inputRecordTimestamp && startTime <= inputRecordTimestamp) {
+                        rightWinAgg = windowBeingProcessed.value;
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (startTime == inputRecordTimestamp + 1) {
                         rightWinAlreadyCreated = true;
+                    } else {
+                        throw new IllegalStateException("Unexpected window found when processing sliding windows");
                     }
                 }
             }
 
             //create right window for previous record
-            if (latestLeftTypeWindow != null) {
-                final long rightWinStart = latestLeftTypeWindow.end() + 1;
-                if (!windowStartTimes.contains(rightWinStart)) {
-                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
-                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
-                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            if (previousRecordTimestamp != null) {
+                final long previousRightWinStart = previousRecordTimestamp + 1;
+                if (rightWindowNecessaryAndPossible(windowStartTimes, previousRightWinStart, inputRecordTimestamp)) {
+                    final TimeWindow window = new TimeWindow(previousRightWinStart, previousRightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp);
+                    updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp);
                 }
             }
 
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
-                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (leftWindowNotEmpty(previousRecordTimestamp, inputRecordTimestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), inputRecordTimestamp);
                 } else {
-                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp);
                 }
-                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
-                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                final TimeWindow window = new TimeWindow(inputRecordTimestamp - windows.timeDifferenceMs(), inputRecordTimestamp);
+                updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp);
             }
-            //create right window for new record
-            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
-                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, inputRecordTimestamp)) {
+                createCurrentRecordRightWindow(inputRecordTimestamp, rightWinAgg, key);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        /**
+         * Created to handle records where 0 < inputRecordTimestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, we will put them into the [0, timeDifferenceMs]
+         * window as a "workaround", and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long inputRecordTimestamp, final long closeTime) {
+            if (inputRecordTimestamp < 0 || inputRecordTimestamp >= windows.timeDifferenceMs()) {
+                throw new IllegalArgumentException("Early record for sliding windows must fall between 0 < inputRecordTimestamp < timeDifferenceMs");
+            }
+
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    0,
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    inputRecordTimestamp + 1)
+            ) {
+                while (iterator.hasNext()) {
+                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> windowBeingProcessed = iterator.next();
+                    final long startTime = windowBeingProcessed.key.window().start();
+                    windowStartTimes.add(startTime);
+                    final long windowMaxRecordTimestamp = windowBeingProcessed.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = windowBeingProcessed;
+                        //We don't need to store previousRecordTimestamp if maxRecordTimestamp > timestamp

Review comment:
       nit: `maxRecordTimestamp > timestamp` -> `maxRecordTimestamp >= timestamp`
   
   nit: missing space: `// We`

##########
File path: streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -439,6 +442,185 @@ public void testJoin() {
         }
     }
 
+    @SuppressWarnings("unchecked")

Review comment:
       Why do we need to suppress?




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r481391993



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the previous record if the previous record exists and the window hasn't already been created
+                if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                    final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createRightWindow(final long timestamp,
+                                       final ValueAndTimestamp<Agg> rightWinAgg,
+                                       final K key,
+                                       final V value,
+                                       final long closeTime) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+            putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
+        }

Review comment:
       Just ran it with updates and we _do_ need to check to make sure that the previous record is close enough for us to make a right window from the current record, so I've updated the `processEarly` `create right window for previous record` to call this method again




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478462394



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -232,40 +239,54 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
 
-                    if (endTime == windows.timeDifferenceMs()) {
+                    if (startTime == 0) {
                         combinedWindow = next;
-                    } else if (endTime > timestamp && startTime <= timestamp) {
+                    } else if (endTime >= timestamp && startTime <= timestamp) {

Review comment:
       Yeah I think you're right, I'll take it out




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478730765



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
       I think with this replacement then we might be able to get out of doing any kind of special handling for the combined window outside of `processEarly`




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r485249087



##########
File path: streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -439,6 +442,185 @@ public void testJoin() {
         }
     }
 
+    @SuppressWarnings("unchecked")

Review comment:
       It's probably some weird Java thing where it lazily types the generics and doesn't force the cast until you put it in the map. (I just made that up, but @vvcephei  would probably know)




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478730258



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
       > That being said, I get that this is confusing. Do you think changing the check to `if (endTime == windows.TimeDifferenceMs() && !isLeftWindow(next)) would make it seem cleaner?
    Haha no, I don't think saying `if (!isLeftWindow(next)): then next = latestLeftTypeWindow` would be less confusing. If we call a variable `leftTypeWindow` then it should _always_ be a left type window.
   
   That said, I now see what you meant here and it's the same problem as above, with the same fix of replacing `latestLeftTypeWindow` with `previousRecordTimestamp`. In that case I think we can just remove this check entirely (ie, don't explicitly check if it's the combined window), and all we need to do is make sure `previousRecordTimestamp` is set correctly




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478459405



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -189,8 +195,8 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create --> new record's left window is not empty

Review comment:
       It actually still works for that example. The range doesn't pick up the window from [0,10] as it goes from timestamp - 2*timeDifference, so for this example the range would start at 1. I initially did this to avoid these scenarios entirely. You are right though that the comment doesn't hold for the combined window scenarios. For example, we find the combined window and store it as "latestLeftTypeWindow" when we process a record at say, 14. If the max record in "latestLeftTypeWindow" is 2, then we don't want to create the right window for that, since it would be [3,13].
   
   I improved the comment to be `if there's a right window that the new record could create && max record falls within left window --> new record's left window is not empty` 




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483828334



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -161,29 +157,31 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     windowStartTimes.add(next.key.window().start());

Review comment:
       Fair enough. I was thinking of `current` in the context of the while loop, but given that we refer to the "current record" elsewhere, `currentWindow` might be ambiguous 




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r482079220



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the previous record if the previous record exists and the window hasn't already been created
+                if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                    final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createRightWindow(final long timestamp,
+                                       final ValueAndTimestamp<Agg> rightWinAgg,
+                                       final K key,
+                                       final V value,
+                                       final long closeTime) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+            putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
+        }

Review comment:
       Aha yes I very much mean `processInOrder`




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r482297669



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,127 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(rightWinAgg.value(), rightWinAgg.timestamp());

Review comment:
       Wait...what's going on here? Aren't we just creating a new `ValueAndTimestamp` that's identical to the `rightWinAgg`? We don't need to make a copy, I assume




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r476546410



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -210,6 +216,66 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),
+                            // to catch the current record's right window, if it exists, without more calls to the store
+                            timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime == windows.timeDifferenceMs()) {
+                        combinedWindow = next;
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the most recent max timestamp in the combined window
+                final long rightWinStart = combinedWindow.value.timestamp() + 1;

Review comment:
       I went with `maxRightWindowStart` since it's not always going to be the previous record, either in terms of process order or time order.




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



[GitHub] [kafka] ableegoldman commented on pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#issuecomment-689272607


   Not sure what's the deal with that but I ran it locally myself and everything passed. Can we merge this @mjsax ?


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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r482084851



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -311,13 +322,8 @@ private void putAndForward(final Window window,
             if (windowEnd > closeTime) {
                 //get aggregate from existing window
                 final Agg oldAgg = getValueOrNull(valueAndTime);

Review comment:
       I just ran the tests and it's _mostly_ true that it won't be null. We do have a test, and so it seems like a real life scenario (?), where the reduce initializer is null, so when creating a new window `valueAndTime` is null.




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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483742361



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {

Review comment:
       > I think the case you're referring to above is saying that for the out-of-order case, the previous record's right window should already exist -- this line is dealing with the right window of the current record. 
   
   Ah. I missed this.
   
   @lct45: the explanation makes sense. Thx!




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r485235880



##########
File path: streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -439,6 +442,185 @@ public void testJoin() {
         }
     }
 
+    @SuppressWarnings("unchecked")

Review comment:
       Without it, there are warnings with the transition from `K, V` to `Long, ValueAndTimestamp` when adding and updating the hash map that holds the results.




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478766196



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -232,40 +239,54 @@ private void processEarly(final K key, final V value, final long timestamp, fina
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
 
-                    if (endTime == windows.timeDifferenceMs()) {
+                    if (startTime == 0) {
                         combinedWindow = next;
-                    } else if (endTime > timestamp && startTime <= timestamp) {
+                    } else if (endTime >= timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {
                         rightWinAlreadyCreated = true;
                     }
                 }
             }
 
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window iwll go in the new record's right window

Review comment:
       Couldn't there still be a record to the left? Like we could have a record at 5 and at 50 and nothing else, then all we would have so far is the combined window and one at [40, 50],  but `rightWindowAgg` would be null. So that's why we need to check that the `combinedWindow.maxTimestamp > timestamp` (and if not then we should leave `rightWinAgg` as null). Looks like this is what you're doing, so not suggesting any changes, just trying to make sure I have this right.




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r477380346



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -148,7 +153,7 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean rightWinAlreadyCreated = false;
 
             // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
             try (
                     final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(

Review comment:
       Ah yeah the different types of stores is a good point, I haven't tested with all of them. I changed it to do 0 or the positive fetch start




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478744162



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
       I think we'd still need a check within this part of the `if()` to catch the previous timestamp, right? I think we could take out the `if (endTime == windows.TimeDifferenceMS())` and instead always do `previousRecord = next.value.timestamp`




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r485222996



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -146,96 +165,203 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean leftWinAlreadyCreated = false;
             boolean rightWinAlreadyCreated = false;
 
-            // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            Long previousRecordTimestamp = null;
+
             try (
                 final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
                     key,
                     key,
-                    timestamp - 2 * windows.timeDifferenceMs(),
+                    Math.max(0, inputRecordTimestamp - 2 * windows.timeDifferenceMs()),
                     // to catch the current record's right window, if it exists, without more calls to the store
-                    timestamp + 1)
+                    inputRecordTimestamp + 1)
             ) {
                 while (iterator.hasNext()) {
-                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next = iterator.next();
-                    windowStartTimes.add(next.key.window().start());
-                    final long startTime = next.key.window().start();
+                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> windowBeingProcessed = iterator.next();
+                    final long startTime = windowBeingProcessed.key.window().start();
+                    windowStartTimes.add(startTime);
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = windowBeingProcessed.value.timestamp();
 
-                    if (endTime < timestamp) {
-                        leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
-                    } else if (endTime == timestamp) {
+                    if (endTime < inputRecordTimestamp) {
+                        leftWinAgg = windowBeingProcessed.value;
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
+                    } else if (endTime == inputRecordTimestamp) {
                         leftWinAlreadyCreated = true;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else if (endTime > timestamp && startTime <= timestamp) {
-                        rightWinAgg = next.value;
-                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                        if (windowMaxRecordTimestamp < inputRecordTimestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (endTime > inputRecordTimestamp && startTime <= inputRecordTimestamp) {
+                        rightWinAgg = windowBeingProcessed.value;
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (startTime == inputRecordTimestamp + 1) {
                         rightWinAlreadyCreated = true;
+                    } else {
+                        throw new IllegalStateException("Unexpected window found when processing sliding windows");
                     }
                 }
             }
 
             //create right window for previous record
-            if (latestLeftTypeWindow != null) {
-                final long rightWinStart = latestLeftTypeWindow.end() + 1;
-                if (!windowStartTimes.contains(rightWinStart)) {
-                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
-                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
-                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            if (previousRecordTimestamp != null) {
+                final long previousRightWinStart = previousRecordTimestamp + 1;
+                if (rightWindowNecessaryAndPossible(windowStartTimes, previousRightWinStart, inputRecordTimestamp)) {
+                    final TimeWindow window = new TimeWindow(previousRightWinStart, previousRightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp);
+                    updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp);
                 }
             }
 
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
-                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (leftWindowNotEmpty(previousRecordTimestamp, inputRecordTimestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), inputRecordTimestamp);
                 } else {
-                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), inputRecordTimestamp);
                 }
-                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
-                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                final TimeWindow window = new TimeWindow(inputRecordTimestamp - windows.timeDifferenceMs(), inputRecordTimestamp);
+                updateWindowAndForward(window, valueAndTime, key, value, closeTime, inputRecordTimestamp);
             }
-            //create right window for new record
-            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
-                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, inputRecordTimestamp)) {
+                createCurrentRecordRightWindow(inputRecordTimestamp, rightWinAgg, key);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        /**
+         * Created to handle records where 0 < inputRecordTimestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, we will put them into the [0, timeDifferenceMs]
+         * window as a "workaround", and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long inputRecordTimestamp, final long closeTime) {
+            if (inputRecordTimestamp < 0 || inputRecordTimestamp >= windows.timeDifferenceMs()) {
+                throw new IllegalArgumentException("Early record for sliding windows must fall between 0 < inputRecordTimestamp < timeDifferenceMs");
+            }
+
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    0,
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    inputRecordTimestamp + 1)
+            ) {
+                while (iterator.hasNext()) {
+                    final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> windowBeingProcessed = iterator.next();
+                    final long startTime = windowBeingProcessed.key.window().start();
+                    windowStartTimes.add(startTime);
+                    final long windowMaxRecordTimestamp = windowBeingProcessed.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = windowBeingProcessed;
+                        //We don't need to store previousRecordTimestamp if maxRecordTimestamp > timestamp
+                        // because the previous record's right window (if there is a previous record)
+                        // would have already been created by maxRecordTimestamp
+                        if (windowMaxRecordTimestamp < inputRecordTimestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= inputRecordTimestamp) {
+                        rightWinAgg = windowBeingProcessed.value;
+                        updateWindowAndForward(windowBeingProcessed.key.window(), windowBeingProcessed.value, key, value, closeTime, inputRecordTimestamp);
+                    } else if (startTime == inputRecordTimestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    } else {
+                        throw new IllegalStateException("Unexpected window found when processing sliding windows");
+                    }
+                }
+            }
+
+            // If there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window. We can be sure that the combined
+            // window only holds records that fall into the current record's right window for two reasons:
+            // 1. If there were records earlier than the current record AND later than the current record, there would be a right window found
+            // when we looked for right window agg.
+            // 2. If there was only a record before the current record, we wouldn't need a right window for the current record and wouldn't update the
+            // rightWinAgg value here, as the combinedWindow.value.timestamp() < inputRecordTimestamp
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > inputRecordTimestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed

Review comment:
       nit: this code is pretty self-explanatory now, so the comment isn't really adding anything




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483306109



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {

Review comment:
       Maybe we can do an `else throw IllegalStateException` here as well. I guess a comment could achieve the same code clarify, but personally I think it's a good idea to have this sanity 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.

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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483302538



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -161,29 +157,31 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     windowStartTimes.add(next.key.window().start());

Review comment:
       `currentWindow` is probably more traditional but `existingWindow` sounds good too




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478743528



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {

Review comment:
       Yeah I think this makes a lot of sense. The one catch I found is that we check to see if `latestLeftTypeWindow != null` when creating the current record's left window, and if we update `previousRecord` each time we see a window, we could get a number that isn't actually within range for the current record's left window. EX: record comes in at 30, timeDifference = 10. we have a window [10,20] but it's a right window, and within it the only record is at 12. since it's the max record within that window, we would store it in `previousRecordTimestamp` , but it's right window is [13,23] which we don't want to create. So I think storing just the long is fine (mostly typing it out to work out my thought process), and the check `rightWindowNecessaryAndPossible` would weed out us creating the [13,23] window on accident. Does this all seem right?




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483718480



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {

Review comment:
       Updated the comment in an attempt to do a mini-proof:
   `If there wasn't a right window agg found and we need a right window for our new record, the current aggregate in the combined window will go in the new record's right window. We can be sure that the combined window only holds records that fall into the current record's right window for two reasons:
   1. If there were records earlier than the current record AND later than the current record, there would be a right window found when we looked for right window agg. 
   2. If there was only a record before the current record, we wouldn't need a right window for the current record and wouldn't update the rightWinAgg value here, as the combinedWindow.value.timestamp() < inputRecordTimestamp

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {

Review comment:
       Updated the comment in an attempt to do a mini-proof:
   `If there wasn't a right window agg found and we need a right window for our new record, the current aggregate in the combined window will go in the new record's right window. We can be sure that the combined window only holds records that fall into the current record's right window for two reasons:
   1. If there were records earlier than the current record AND later than the current record, there would be a right window found when we looked for right window agg. 
   2. If there was only a record before the current record, we wouldn't need a right window for the current record and wouldn't update the rightWinAgg value here, as the combinedWindow.value.timestamp() < inputRecordTimestamp`




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r484094512



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       Ah, good call, that makes sense to me




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483695703



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;

Review comment:
       If the previous record falls within the current record's left window, then it contains values other than just the current record. There is a scenario where we find a previous record just outside of the current record, which would indicate that the current record's left window only contains the current record. 
   
   EX: if we have a record at `25`, and previous records at `13` and `4` and with a time difference of 10, we find `4`'s right window at [5,15] in our scan. We log the `maxRecordTimestamp` of this window, which is `13`, as `25`'s previous record. We don't actually want to create `13`'s right window though, even if it's the previous record, because `25` falls outside of [14,24].
   
   Using the timestamps seemed like a simple litmus test to see if we found anything that falls within the window and was necessary after we found the previous record consistently by looking at the maxRecordTimestamp of a window.




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483309087



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       Yeah that's right.
   > But for an early record, if maxRecordTimestamp > timestamp, we know that the previous record's right window must have already been created
   
   I think this is key - we know this because for any early record, it will _always_ fall within the right window of the previous record (given there is one), since they both fall within the [0, timeDifferenceMs] window. It's hard to phrase clearly on the comment, I can add another line about the proof if that would be helful




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r480194412



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -0,0 +1,386 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.kstream.Aggregator;
+import org.apache.kafka.streams.kstream.Initializer;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.SlidingWindows;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.TimestampedWindowStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrLateRecordDropSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor;
+import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
+
+public class KStreamSlidingWindowAggregate<K, V, Agg> implements KStreamAggProcessorSupplier<K, Windowed<K>, V, Agg> {
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final String storeName;
+    private final SlidingWindows windows;
+    private final Initializer<Agg> initializer;
+    private final Aggregator<? super K, ? super V, Agg> aggregator;
+
+    private boolean sendOldValues = false;
+
+    public KStreamSlidingWindowAggregate(final SlidingWindows windows,
+                                         final String storeName,
+                                         final Initializer<Agg> initializer,
+                                         final Aggregator<? super K, ? super V, Agg> aggregator) {
+        this.windows = windows;
+        this.storeName = storeName;
+        this.initializer = initializer;
+        this.aggregator = aggregator;
+    }
+
+    @Override
+    public Processor<K, V> get() {
+        return new KStreamSlidingWindowAggregateProcessor();
+    }
+
+    public SlidingWindows windows() {
+        return windows;
+    }
+
+    @Override
+    public void enableSendingOldValues() {
+        sendOldValues = true;
+    }
+
+    private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor<K, V> {
+        private TimestampedWindowStore<K, Agg> windowStore;
+        private TimestampedTupleForwarder<Windowed<K>, Agg> tupleForwarder;
+        private StreamsMetricsImpl metrics;
+        private InternalProcessorContext internalProcessorContext;
+        private Sensor lateRecordDropSensor;
+        private Sensor droppedRecordsSensor;
+        private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(final ProcessorContext context) {
+            super.init(context);
+            internalProcessorContext = (InternalProcessorContext) context;
+            metrics = internalProcessorContext.metrics();
+            final String threadId = Thread.currentThread().getName();
+            lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor(
+                threadId,
+                context.taskId().toString(),
+                internalProcessorContext.currentNode().name(),
+                metrics
+            );
+            droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics);
+            windowStore = (TimestampedWindowStore<K, Agg>) context.getStateStore(storeName);
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+
+        @Override
+        public void process(final K key, final V value) {
+            if (key == null || value == null) {
+                log.warn(
+                    "Skipping record due to null key or value. value=[{}] topic=[{}] partition=[{}] offset=[{}]",
+                    value, context().topic(), context().partition(), context().offset()
+                );
+                droppedRecordsSensor.record();
+                return;
+            }
+
+            final long timestamp = context().timestamp();
+            processInOrder(key, value, timestamp);
+        }
+
+        public void processInOrder(final K key, final V value, final long timestamp) {
+
+            observedStreamTime = Math.max(observedStreamTime, timestamp);
+            final long closeTime = observedStreamTime - windows.gracePeriodMs();
+
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;
+            }
+
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            // aggregate that will go in the current record’s left/right window (if needed)
+            ValueAndTimestamp<Agg> leftWinAgg = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+
+            //if current record's left/right windows already exist
+            boolean leftWinAlreadyCreated = false;
+            boolean rightWinAlreadyCreated = false;
+
+            // keep the left type window closest to the record
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime < timestamp) {
+                        leftWinAgg = next.value;
+                        // store the combined window (window from [0,timeDifference] that contains any early records)
+                        // if it is found so that a right window can be created for the combined window's max record, as needed
+                        if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                    } else if (endTime == timestamp) {
+                        leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+            //create right window for previous record
+            if (latestLeftTypeWindow != null) {
+                final long previousWindowEnd = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousWindowEnd == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousWindowEnd + 1;
+                if (rightWindowNecessaryAndPossible(windowStartTimes, rightWinStart, timestamp)) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+            }
+
+            //create left window for new record
+            if (!leftWinAlreadyCreated) {
+                final ValueAndTimestamp<Agg> valueAndTime;
+                // if there's a right window that the new record could create && max record falls within left window --> new record's left window is not empty
+                if (latestLeftTypeWindow != null && recordFallsWithinCurrentRecordsLeftWindow(latestLeftTypeWindow.value.timestamp(), timestamp)) {
+                    valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
+                } else {
+                    valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                }
+                final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+            //create right window for new record
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the combined window's max record before the current record was added
+                final long maxRightWindowStart = combinedWindow.value.timestamp() + 1;

Review comment:
       Yeah I see what you mean, I think it makes sense to utilize previous record for both `processInOrder` and `processEarly`




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r485248486



##########
File path: streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -439,6 +442,185 @@ public void testJoin() {
         }
     }
 
+    @SuppressWarnings("unchecked")

Review comment:
       Yeah I do find it odd that there are no warnings earlier, it's something about the Map that's triggering them




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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483739783



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       @ableegoldman WDYT?




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478442739



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -174,12 +181,11 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     }
                 }
             }
-
             //create right window for previous record
             if (latestLeftTypeWindow != null) {
-                final long leftWindowEnd = latestLeftTypeWindow.key.window().end();
-                final long rightWinStart = leftWindowEnd == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : leftWindowEnd + 1;
-                if (!windowStartTimes.contains(rightWinStart)) {
+                final long previousRecord = latestLeftTypeWindow.key.window().end();
+                final long rightWinStart = previousRecord == windows.timeDifferenceMs() ? latestLeftTypeWindow.value.timestamp() + 1 : previousRecord + 1;

Review comment:
       If previous record is at `timeDifferenceMs` then the "left type window" we found was the window from [0,timeDifference]. Since we know it's the left window from `windowEnd == timeDifference`, we know that either we'll create a right window for the max record in the combined window, or we'll create a right window for the record at the end point of the window (which would still be the max record). 
   
   I see that `previousRecord` is misleading, I renamed it to `previousWindowEnd`. `previousRecord` here was really just the `windowEnd`, since there might not be a _record_ at the end of the combined window, which is the case this is covering. This case is specifically for records that come outside of the combined window, but still need to create a right window for something _in_ the combined window, in which case the right window won't already exist.




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r476531518



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -148,7 +153,7 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean rightWinAlreadyCreated = false;
 
             // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
             try (
                     final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(

Review comment:
       Fetch having negative bounds doesn't throw any errors or cause any issues, is there a different reason to make sure the bounds aren't negative? Since we don't store windows with a negative start time it shouldn't return anything we don't expect




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483715336



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
+        }
+
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
+            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;

Review comment:
       In our scan for `rightWinAgg` , there's a possibility that we find a `rightWinAgg` that doesn't actually fall within the current record's right window. For example, if we have a record come in at `18`, previous records at `16` and `10`, we would find the window [11,21] (`10`'s right window) when scanning for our `rightWinAgg` in `processInOrder`. [11,21] contains only `16`, which is smaller than `18`, our current record, indicating that the rightWinAgg is incorrect and the right window is in fact empty.




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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483736252



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -161,29 +180,31 @@ public void processInOrder(final K key, final V value, final long timestamp) {
                     windowStartTimes.add(next.key.window().start());
                     final long startTime = next.key.window().start();
                     final long endTime = startTime + windows.timeDifferenceMs();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
+                        previousRecordTimestamp = windowMaxRecordTimestamp;
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
                     } else if (endTime > timestamp && startTime <= timestamp) {
                         rightWinAgg = next.value;
                         putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
-                    } else {
+                    } else if (startTime == timestamp + 1) {

Review comment:
       The suggest is not bad, but required to add the `else-throw` to make sense. Otherwise, an programming error could slip undetected.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]

Review comment:
       nit: `Instead, they will fall within the [0, timeDifferenceMs]` -> `Instead, we will put them into the [0, timeDifferenceMs] window as a "workaround",`




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r481338030



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -164,11 +161,13 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
-                        if (isLeftWindow(next)) {
-                            latestLeftTypeWindow = next.key.window();
-                        }
+                        // update to store the previous record

Review comment:
       This comment doesn't really add anything, it just describes what the code says. Also, don't we need to check that   `windowMaxTimestamp > previousRecordTimestamp` before updating `previousRecordTimestamp` (where `windowMaxTimestamp = next.value.timestamp` -- it would be nice to assign this to a variable with an explicit name to make it clear what `next.value.timestamp` actually means).
   Same goes for the below, I guess you could just put the check in a `maybeUpdatePreviousRecordTimestamp()` method and call it from both places

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -146,13 +142,14 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean leftWinAlreadyCreated = false;
             boolean rightWinAlreadyCreated = false;
 
-            // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            // Store the previous record
+            Long previousRecord = null;

Review comment:
       Use `previousRecordTimestamp` like in `processEarly`.  You can probably remove the comment then

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records

Review comment:
       ```suggestion
               // A window from [0, timeDifferenceMs] that holds all early records
   ```
   Also I'd suggest putting the `combinedWindow` declaration (and comment) above `rightWinAgg` to avoid ambiguity in what the comment refers to 

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later

Review comment:
       update, or create? (or both?)

##########
File path: streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
##########
@@ -239,6 +239,7 @@ private void doCountSlidingWindows(final  MockProcessorSupplier<Windowed<String>
             inputTopic.pipeInput("2", "B", 1000L);
             inputTopic.pipeInput("3", "C", 600L);
         }
+

Review comment:
       nit: remove this added line

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]

Review comment:
       ```suggestion
            * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }

Review comment:
       I think we should always assign the `next.value.timestamp` value to a variable with an explicit name, eg `windowMaxRecordTimestamp`, because it's pretty non-obvious what it means and easy to forget

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);

Review comment:
       I think we should add a comment somewhere clarifying some things about how we set the `previousRecordTimestamp` in `processEarly`:
   Basically, we only need to check and maybe set it when we're on the combined window, because if it's still null when we're past the combined window then we know there was a record greater than the current record in the combined window already, and in that case we must have already created the right window for the actual previous record. Hopefully you can find a better & more concise way to explain that 😄 

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create

Review comment:
       ```suggestion
            * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -135,7 +127,11 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             observedStreamTime = Math.max(observedStreamTime, timestamp);
             final long closeTime = observedStreamTime - windows.gracePeriodMs();
 
-            //store start times of windows we find
+            if (timestamp < windows.timeDifferenceMs()) {
+                processEarly(key, value, timestamp, closeTime);
+                return;

Review comment:
       Seems like we should move this above into the top-level `process` instead of first calling `processInOrder` and then calling `processEarly`.  For one thing, since we actually do need to iterate the full range for the early records, we can just call `processEarly` without having to decide between `processInOrder` and `processReverse`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);

Review comment:
       This doesn't look right..why would we need to pass in the `key` and `value` to `createRightWindow` ? The distinguishing feature of the current record's right window is that it doesn't  include the current record at all. I see that `createRightWindow` ultimately calls `putAndForward` which takes a key and value, but that just seems misleading. I think we should either pass in `null` to `putAndForward` for things we don't need, or better yet (imo) don't use `putAndForward` for the right window creation and just have a clean separation between creation of the right window and everything else

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {

Review comment:
       Just leaving a note to the next reviewer, and making sure I understand this myself: 
   If we didn't find a right window agg that either means 
   1) there were no other windows with startTime <= timestamp, which means there are no records earlier than the current one, and no records later than timeDifference (but within range of the current record). We take that to mean that whatever is in the combined window is an aggregate of records that are all to the right of the current record. Note that there could be a record at the same timestamp as the current record, but we will check for that before actually creating the right window below (by checking `!rightWinAlreadyCreated`)
   2) There is just a single record in the combined window that is earlier than the current record. We check for that with the `combinedWindow.value.timestamp() > timestamp` condition and in that case create no right window
   
   By the way, can we move the creation of the right window to just after this block, to keep all the relevant logic together?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the previous record if the previous record exists and the window hasn't already been created
+                if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                    final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createRightWindow(final long timestamp,
+                                       final ValueAndTimestamp<Agg> rightWinAgg,
+                                       final K key,
+                                       final V value,
+                                       final long closeTime) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+            putAndForward(window, valueAndTime, key, value, closeTime, timestamp);

Review comment:
       It's pretty weird to have to pass in `key` and `value` when they're not used to create the right window. I notice they're used in the log message for dropped late windows actually, but it seems odd that we should ever end up dropping the right window of the current record. If the record itself is that old, we should just drop it before even processing it, right? 
   Assuming you do that, then it feels a lot more reasonable to not call `putAndForward` from `createRightWindow` at all, and just do the actual putting and forwarding for the right window case inline. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the previous record if the previous record exists and the window hasn't already been created
+                if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                    final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createRightWindow(final long timestamp,
+                                       final ValueAndTimestamp<Agg> rightWinAgg,
+                                       final K key,
+                                       final V value,
+                                       final long closeTime) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+            putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
+        }

Review comment:
       This isn't used anymore, right?




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r481378201



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +191,113 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecord != null && leftWindowNotEmpty(previousRecord, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
+            }
+        }
+
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            //window from [0,timeDifference] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (next.value.timestamp() < timestamp) {
+                            previousRecordTimestamp = next.value.timestamp();
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the previous record if the previous record exists and the window hasn't already been created
+                if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                    final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createRightWindow(timestamp, rightWinAgg, key, value, closeTime);
             }
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createRightWindow(final long timestamp,
+                                       final ValueAndTimestamp<Agg> rightWinAgg,
+                                       final K key,
+                                       final V value,
+                                       final long closeTime) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+            putAndForward(window, valueAndTime, key, value, closeTime, timestamp);

Review comment:
       Key is still used when the window is put into the windowStore, but value isn't needed. I agree that it's nice to separate them to clean up the `if()` checks as well




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483827530



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
+        }
+
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
+            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;

Review comment:
       I thought it was called `rightWinAgg` because it's the aggregate that goes in the current record's right window. Of course we had to find this aggregate from some other existing window, eg from the "last window left of the current record". 




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478759996



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
        I guess you could just have a separate `setPreviousRecordTimestampIfNecessary(window, previousRecordTimestamp)` method that sets the `previousRecordTimestamp` to the window's max timestamp if it's larger. And then if it ends up that `previousRecordTimestamp == timestamp` then we can automatically skip all of the window creation below

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
        I guess you could just have a separate `setPreviousRecordTimestampIfNecessary(window, previousRecordTimestamp)` method that sets the `previousRecordTimestamp` to the window's max timestamp if it's larger. And then if it ends up that `previousRecordTimestamp == timestamp` then we can automatically skip all of the window creation below, which is nice




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



[GitHub] [kafka] mjsax commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483749204



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
+        }
+
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
+            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;

Review comment:
       Thanks. As above, maybe add a short comment? Can it be that we overload the term "right window"? For this case, `rightWinAgg` seems to be the aggregate of last window "left of the current record" (and it might contribute to the agg of the right window of the current record). However, it "is" not the "right window of the current record" itself?




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483305501



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       I think it means, for a generic out-of-order record, it's _possible_ that the previous record's right window will have already been created (by whatever record (s) are later than the current one). But for an early record, if `maxRecordTimestamp > timestamp`, then we _know_ that the previous record's right window must have already been created (by whatever record(s) are within the combined window but later than the current record). 
   This is relevant to setting `previousRecordTimestamp` because if `maxRecordTimestamp >= timestamp`, the previous record's right window has already been created. And if that's the case, we don't have to create it ourselves and thus we don't care about the `previousRecordTimestamp`
   Does that sound right Leah?




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478758482



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {

Review comment:
       Yeah, you're saying that we just always keep track of the `previousRecordTimestamp`, but before we go ahead and create a left window for the current window we just actually verify that the previous record is within range? That makes sense to me, actually if anything I feel like it will make `rightWindowNecessaryAndPossible` even more clear to put it in terms of `previousRecordTimestamp`. What I'm realizing from this is that it's easier to understand these boolean checks in terms of the actual record locations, in general. Maybe it's just my mental model, I still picture a rectangle sliding over boxes on a timeline 😄 




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483752603



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
             }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            windowStore.put(
+                key,
+                rightWinAgg,
+                window.start());
+            tupleForwarder.maybeForward(
+                new Windowed<>(key, window),
+                rightWinAgg.value(),
+                null,
+                rightWinAgg.timestamp());
+        }
+
+        private boolean leftWindowNotEmpty(final long previousTimestamp, final long currentTimestamp) {
+            return currentTimestamp - windows.timeDifferenceMs() <= previousTimestamp;
+        }
+
+        // previous record's right window does not already exist and current record falls within previous record's right window
+        private boolean rightWindowNecessaryAndPossible(final Set<Long> windowStartTimes,
+                                                        final long previousRightWindowStart,
+                                                        final long currentRecordTimestamp) {
+            return !windowStartTimes.contains(previousRightWindowStart) && previousRightWindowStart + windows.timeDifferenceMs() >= currentRecordTimestamp;
         }
 
-        private boolean isLeftWindow(final KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> window) {
-            return window.key.window().end() == window.value.timestamp();
+        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
+            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;

Review comment:
       Just added a comment similar to above.
   And yeah, we definitely overload the term "right window". The `rightWinAgg` holds what will be in the current record's right window, if anything, but is not _really_ the right window. A name change could be possible, something like `previousWindow` but I wonder if that would make using that aggregate to create the new record's right window more confusing.




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r483309087



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,125 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].

Review comment:
       Yeah that's right.
   > But for an early record, if maxRecordTimestamp > timestamp, we know that the previous record's right window must have already been created
   I think this is key - we know this because for any early record, it will _always_ fall within the right window of the previous record (given there is one), since they both fall within the [0, timeDifferenceMs] window. It's hard to phrase clearly on the comment, I can add another line about the proof if that would be helful




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r478730258



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -160,11 +160,18 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
                     if (endTime < timestamp) {
                         leftWinAgg = next.value;
+                        // store the combined window if it is found so that a right window can be created for
+                        // the combined window's max record, as needed
                         if (isLeftWindow(next) || endTime == windows.timeDifferenceMs()) {
                             latestLeftTypeWindow = next;
                         }
                     } else if (endTime == timestamp) {
                         leftWinAlreadyCreated = true;
+                        // if current record's left window is the combined window, need to check later if there is a
+                        // record that needs a right window within the combined window
+                        if (endTime == windows.timeDifferenceMs()) {
+                            latestLeftTypeWindow = next;
+                        }

Review comment:
       > That being said, I get that this is confusing. Do you think changing the check to `if (endTime == windows.TimeDifferenceMs() && !isLeftWindow(next))` would make it seem cleaner?
   
    Haha no, I don't think saying `if (!isLeftWindow(next)): then next = latestLeftTypeWindow` would be less confusing. If we call a variable `leftTypeWindow` then it should _always_ be a left type window.
   
   That said, I now see what you meant here and it's the same problem as above, with the same fix of replacing `latestLeftTypeWindow` with `previousRecordTimestamp`. In that case I think we can just remove this check entirely (ie, don't explicitly check if it's the combined window), and all we need to do is make sure `previousRecordTimestamp` is set correctly




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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r475985073



##########
File path: streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
##########
@@ -328,6 +328,68 @@ public void testAggregateLargeInput() {
         );
     }
 
+    @Test
+    public void testEarlyRecords() {

Review comment:
       Can we add maybe one or two more tests? I think at the least we should have one test that processes _only_ early records, and one test that covers input(s) with the same timestamp.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -148,7 +153,7 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             boolean rightWinAlreadyCreated = false;
 
             // keep the left type window closest to the record
-            Window latestLeftTypeWindow = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> latestLeftTypeWindow = null;
             try (
                     final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(

Review comment:
       We need to make sure the `fetch` bounds don't go into the negative. We only call `processEarly` if the record's timestamp is within the timeDifferenceMs, but here we search starting at timestamp - 2*timeDifferenceMs
   

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -210,6 +216,66 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),
+                            // to catch the current record's right window, if it exists, without more calls to the store
+                            timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime == windows.timeDifferenceMs()) {
+                        combinedWindow = next;
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {

Review comment:
       It took me a second to get this -- can we explicitly check `if startTime == timestamp + 1` instead of falling back to `else` and implicitly relying on the fetch bounds? You can just get rid of the `else` altogether or throw an IllegalStateException if none of the specific conditions are met and the else is reached, whatever makes sense to you

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -210,6 +216,66 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),
+                            // to catch the current record's right window, if it exists, without more calls to the store
+                            timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime == windows.timeDifferenceMs()) {
+                        combinedWindow = next;
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the most recent max timestamp in the combined window

Review comment:
       > most recent max timestamp
   
   Huh? I think I know what you're trying to say here but it seems like two different phrases got a bit mixed up here

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -210,6 +216,66 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),
+                            // to catch the current record's right window, if it exists, without more calls to the store
+                            timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime == windows.timeDifferenceMs()) {

Review comment:
       nit: can we make this `if startTime == 0` ? That seems slightly easier to understand, and then all the conditionals can be in terms of startTime which is a bit more intuitive since that's what we're iterating over. Context switching between startTime and endTime kind of makes me lose my train of thought

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -183,7 +188,8 @@ public void processInOrder(final K key, final V value, final long timestamp) {
 
             //create right window for previous record
             if (latestLeftTypeWindow != null) {
-                final long rightWinStart = latestLeftTypeWindow.end() + 1;
+                final long leftWindowEnd = latestLeftTypeWindow.key.window().end();

Review comment:
       This is really just the timestamp of the previous record, right? Can we call it something that reflects that

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -211,6 +217,67 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),

Review comment:
       I think this fetch might break if you go into the negatives, should just fetch starting from 0

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -210,6 +216,66 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;

Review comment:
       This is just a window from [0, timeDifferenceMs] that stores the aggregation of all the "early" records, right? I can't really think of a more descriptive name so can we just leave a comment explaining what it's for

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -210,6 +216,66 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),
+                            // to catch the current record's right window, if it exists, without more calls to the store
+                            timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime == windows.timeDifferenceMs()) {
+                        combinedWindow = next;
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the most recent max timestamp in the combined window
+                final long rightWinStart = combinedWindow.value.timestamp() + 1;
+                if (!windowStartTimes.contains(rightWinStart) && combinedWindow.value.timestamp() < timestamp) {
+                    final TimeWindow window = new TimeWindow(rightWinStart, rightWinStart + windows.timeDifferenceMs());
+                    final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                    putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+                }
+                //update the combined window with the new aggregated

Review comment:
       nit: `aggregated` -> `aggregate`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -210,6 +216,66 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),
+                            // to catch the current record's right window, if it exists, without more calls to the store
+                            timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime == windows.timeDifferenceMs()) {
+                        combinedWindow = next;
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the most recent max timestamp in the combined window
+                final long rightWinStart = combinedWindow.value.timestamp() + 1;

Review comment:
       This name keeps throwing me off...right window of what? Is it like `previousRecordRightWindow`?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -210,6 +216,66 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             }
         }
 
+        /**
+         * Created to handle records that have a timestamp > 0 but < timeDifference. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifference]
+         * window, and we will update their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            boolean rightWinAlreadyCreated = false;
+            final HashSet<Long> windowStartTimes = new HashSet<Long>();
+
+            try (
+                    final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                            key,
+                            key,
+                            timestamp - 2 * windows.timeDifferenceMs(),
+                            // to catch the current record's right window, if it exists, without more calls to the store
+                            timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long endTime = startTime + windows.timeDifferenceMs();
+
+                    if (endTime == windows.timeDifferenceMs()) {
+                        combinedWindow = next;
+                    } else if (endTime > timestamp && startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //create the right window for the most recent max timestamp in the combined window
+                final long rightWinStart = combinedWindow.value.timestamp() + 1;
+                if (!windowStartTimes.contains(rightWinStart) && combinedWindow.value.timestamp() < timestamp) {

Review comment:
       It's not immediately obvious why this is correct/captures all possible cases so we should leave a comment, or better yet factor out this condition into a descriptively named method (or best yet, do both)
   I was concerned about out-of-order records, since in that case the previous record would obviously not be the one with the maximum timestamp in the combined window. But I realized that we actually never need to create a previous record's right window for out-of-order early records, since there's no way for a full timeDifferenceMs to fit between the previous record and whatever the max record in the combined window is. So, all we need to do is make sure we only try to create the previous record's right window if the current window is the maximum record within the combined window, ie the `combinedWindow.value.timestamp() < timestamp`
   But that's a very long and pretty ineloquent comment to leave in the code. Hopefully you can come up with a more concise explanation 




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



[GitHub] [kafka] lct45 commented on a change in pull request #9157: Update for KIP-450 to handle early records

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9157:
URL: https://github.com/apache/kafka/pull/9157#discussion_r482324434



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
##########
@@ -192,29 +213,127 @@ public void processInOrder(final K key, final V value, final long timestamp) {
             //create left window for new record
             if (!leftWinAlreadyCreated) {
                 final ValueAndTimestamp<Agg> valueAndTime;
-                //there's a right window that the new record could create --> new record's left window is not empty
-                if (latestLeftTypeWindow != null) {
+                // if there's a right window that the new record could create && previous record falls within left window -> new record's left window is not empty
+                if (previousRecordTimestamp != null && leftWindowNotEmpty(previousRecordTimestamp, timestamp)) {
                     valueAndTime = ValueAndTimestamp.make(leftWinAgg.value(), timestamp);
                 } else {
                     valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 }
                 final TimeWindow window = new TimeWindow(timestamp - windows.timeDifferenceMs(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
-            //create right window for new record
             if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
-                final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
-                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(getValueOrNull(rightWinAgg), Math.max(rightWinAgg.timestamp(), timestamp));
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+        }
+
+        /**
+         * Created to handle records where 0 < timestamp < timeDifferenceMs. These records would create
+         * windows with negative start times, which is not supported. Instead, they will fall within the [0, timeDifferenceMs]
+         * window, and we will update or create their right windows as new records come in later
+         */
+        private void processEarly(final K key, final V value, final long timestamp, final long closeTime) {
+            // A window from [0, timeDifferenceMs] that holds all early records
+            KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> combinedWindow = null;
+            ValueAndTimestamp<Agg> rightWinAgg = null;
+            boolean rightWinAlreadyCreated = false;
+            final Set<Long> windowStartTimes = new HashSet<>();
+
+            Long previousRecordTimestamp = null;
+
+            try (
+                final KeyValueIterator<Windowed<K>, ValueAndTimestamp<Agg>> iterator = windowStore.fetch(
+                    key,
+                    key,
+                    Math.max(0, timestamp - 2 * windows.timeDifferenceMs()),
+                    // to catch the current record's right window, if it exists, without more calls to the store
+                    timestamp + 1)
+            ) {
+                KeyValue<Windowed<K>, ValueAndTimestamp<Agg>> next;
+                while (iterator.hasNext()) {
+                    next = iterator.next();
+                    windowStartTimes.add(next.key.window().start());
+                    final long startTime = next.key.window().start();
+                    final long windowMaxRecordTimestamp = next.value.timestamp();
+
+                    if (startTime == 0) {
+                        combinedWindow = next;
+                        if (windowMaxRecordTimestamp < timestamp) {
+                            // If maxRecordTimestamp > timestamp, the current record is out-of-order, meaning that the
+                            // previous record's right window would have been created already by other records. This
+                            // will always be true for early records, as they all fall within [0, timeDifferenceMs].
+                            previousRecordTimestamp = windowMaxRecordTimestamp;
+                        }
+
+                    } else if (startTime <= timestamp) {
+                        rightWinAgg = next.value;
+                        putAndForward(next.key.window(), next.value, key, value, closeTime, timestamp);
+                    } else if (startTime == timestamp + 1) {
+                        rightWinAlreadyCreated = true;
+                    }
+                }
+            }
+
+            // if there wasn't a right window agg found and we need a right window for our new record,
+            // the current aggregate in the combined window will go in the new record's right window
+            if (rightWinAgg == null && combinedWindow != null && combinedWindow.value.timestamp() > timestamp) {
+                rightWinAgg = combinedWindow.value;
+            }
+
+            //create right window for new record if needed
+            if (!rightWinAlreadyCreated && rightWindowIsNotEmpty(rightWinAgg, timestamp)) {
+                createCurrentRecordRightWindow(timestamp, rightWinAgg, key);
+            }
+
+            //create the right window for the previous record if the previous record exists and the window hasn't already been created
+            if (previousRecordTimestamp != null && !windowStartTimes.contains(previousRecordTimestamp + 1)) {
+                final TimeWindow window = new TimeWindow(previousRecordTimestamp + 1, previousRecordTimestamp + 1 + windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
                 putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
             }
+
+            if (combinedWindow == null) {
+                final TimeWindow window = new TimeWindow(0, windows.timeDifferenceMs());
+                final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(initializer.apply(), timestamp);
+                putAndForward(window, valueAndTime, key, value, closeTime, timestamp);
+
+            } else {
+                //update the combined window with the new aggregate
+                putAndForward(combinedWindow.key.window(), combinedWindow.value, key, value, closeTime, timestamp);
+            }
+
         }
 
-        private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<Agg> rightWinAgg, final long timestamp) {
-            return rightWinAgg != null && rightWinAgg.timestamp() > timestamp;
+        private void createCurrentRecordRightWindow(final long timestamp,
+                                                    final ValueAndTimestamp<Agg> rightWinAgg,
+                                                    final K key) {
+            final TimeWindow window = new TimeWindow(timestamp + 1, timestamp + 1 + windows.timeDifferenceMs());
+            final ValueAndTimestamp<Agg> valueAndTime = ValueAndTimestamp.make(rightWinAgg.value(), rightWinAgg.timestamp());

Review comment:
       Yup, good catch. Must've been leftover from a cleanup




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