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 2022/05/10 22:43:05 UTC

[GitHub] [kafka] mjsax commented on a diff in pull request #12135: KAFKA-13785: [7/N][Emit final] emit final for sliding window

mjsax commented on code in PR #12135:
URL: https://github.com/apache/kafka/pull/12135#discussion_r869501138


##########
streams/src/main/java/org/apache/kafka/streams/kstream/EmitStrategy.java:
##########
@@ -19,15 +19,52 @@
 import org.apache.kafka.streams.kstream.internals.UnlimitedWindow;
 import org.apache.kafka.streams.kstream.internals.emitstrategy.WindowCloseStrategy;
 import org.apache.kafka.streams.kstream.internals.emitstrategy.WindowUpdateStrategy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * This interface controls the strategy that can be used to control how we emit results in a processor.
  */
 public interface EmitStrategy {
 
+    Logger log = LoggerFactory.getLogger(EmitStrategy.class);
+
     enum StrategyType {
-        ON_WINDOW_CLOSE,
-        ON_WINDOW_UPDATE
+        ON_WINDOW_UPDATE(0, new WindowUpdateStrategy()),
+        ON_WINDOW_CLOSE(1, new WindowCloseStrategy());
+
+        private final short code;
+        private final EmitStrategy strategy;
+
+        private short code() {
+            return this.code;
+        }
+
+        private EmitStrategy strategy() {
+            return this.strategy;
+        }
+
+        StrategyType(final int code, final EmitStrategy strategy) {
+            this.code = (short) code;
+            this.strategy = strategy;
+        }
+
+        private final static Map<Short, EmitStrategy> TYPE_TO_STRATEGY = new HashMap<>();
+
+        static {
+            for (final StrategyType type : StrategyType.values()) {
+                if (TYPE_TO_STRATEGY.put(type.code(), type.strategy()) != null)
+                    throw new IllegalStateException("Code " + type.code() + " for type " +

Review Comment:
   Never seen anything like this before -- is it best practice to have a guard like this?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKStreamTimeWindowAggregateProcessor.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static org.apache.kafka.streams.StreamsConfig.InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emitFinalLatencySensor;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emittedRecordsSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.kstream.EmitStrategy;
+import org.apache.kafka.streams.kstream.EmitStrategy.StrategyType;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTracker;
+import org.apache.kafka.streams.processor.api.ContextualProcessor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+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;
+
+public abstract class AbstractKStreamTimeWindowAggregateProcessor<KIn, VIn, VAgg> extends ContextualProcessor<KIn, VIn, Windowed<KIn>, Change<VAgg>> {
+
+    private final Time time = Time.SYSTEM;
+    private final String storeName;
+    private final EmitStrategy emitStrategy;
+    private final boolean sendOldValues;
+    protected final TimeTracker timeTracker = new TimeTracker();
+
+    private TimestampedTupleForwarder<Windowed<KIn>, VAgg> tupleForwarder;
+    protected TimestampedWindowStore<KIn, VAgg> windowStore;
+    protected Sensor droppedRecordsSensor;
+    protected Sensor emittedRecordsSensor;
+    protected Sensor emitFinalLatencySensor;
+    protected long lastEmitWindowCloseTime = ConsumerRecord.NO_TIMESTAMP;
+    protected InternalProcessorContext<Windowed<KIn>, Change<VAgg>> internalProcessorContext;
+
+    protected AbstractKStreamTimeWindowAggregateProcessor(final String storeName,
+                                                          final EmitStrategy emitStrategy,
+                                                          final boolean sendOldValues) {
+        this.storeName = storeName;
+        this.emitStrategy = emitStrategy;
+        this.sendOldValues = sendOldValues;
+    }
+
+    @Override
+    public void init(final ProcessorContext<Windowed<KIn>, Change<VAgg>> context) {
+        super.init(context);
+        internalProcessorContext = (InternalProcessorContext<Windowed<KIn>, Change<VAgg>>) context;
+        final StreamsMetricsImpl metrics = internalProcessorContext.metrics();
+        final String threadId = Thread.currentThread().getName();
+        droppedRecordsSensor = droppedRecordsSensor(threadId, context.taskId().toString(), metrics);
+        emittedRecordsSensor = emittedRecordsSensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        emitFinalLatencySensor = emitFinalLatencySensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        windowStore = context.getStateStore(storeName);
+
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            // Restore last emit close time for ON_WINDOW_CLOSE strategy
+            final Long lastEmitTime = internalProcessorContext.processorMetadataForKey(storeName);
+            if (lastEmitTime != null) {
+                lastEmitWindowCloseTime = lastEmitTime;
+            }
+            final long emitInterval = StreamsConfig.InternalConfig.getLong(
+                context.appConfigs(),
+                EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION,
+                1000L
+            );
+            timeTracker.setEmitInterval(emitInterval);
+        } else {
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+    }
+
+    protected void maybeForwardUpdate(final Record<KIn, VIn> record,
+                                      final Window window,
+                                      final VAgg oldAgg,
+                                      final VAgg newAgg,
+                                      final long newTimestamp) {
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            return;
+        } else if (tupleForwarder == null) {
+            throw new IllegalStateException("Emit strategy type is " + emitStrategy.type() + " but flush listener is not initialized.");
+        }
+
+        tupleForwarder.maybeForward(
+            record.withKey(new Windowed<>(record.key(), window))
+                .withValue(new Change<>(newAgg, sendOldValues ? oldAgg : null))
+                .withTimestamp(newTimestamp));
+    }
+
+    protected boolean shouldEmitFinal(final long closeTime) {
+        if (emitStrategy.type() != StrategyType.ON_WINDOW_CLOSE) {
+            return false;
+        }
+
+        final long now = internalProcessorContext.currentSystemTimeMs();
+        // Throttle emit frequency
+        if (now < timeTracker.nextTimeToEmit) {
+            return false;
+        }
+
+        // Schedule next emit time based on now to avoid the case that if system time jumps a lot,
+        // this can be triggered every time
+        timeTracker.nextTimeToEmit = now;
+        timeTracker.advanceNextTimeToEmit();
+
+        // Only EMIT if the window close time does progress
+        return lastEmitWindowCloseTime == ConsumerRecord.NO_TIMESTAMP || lastEmitWindowCloseTime < closeTime;
+    }
+
+    protected void fetchAndEmit(final Record<KIn, VIn> record,
+                                final long closeTime,
+                                final long emitRangeLowerBoundInclusive,
+                                final long emitRangeUpperBoundInclusive) {
+        final long startMs = time.milliseconds();
+
+        final KeyValueIterator<Windowed<KIn>, ValueAndTimestamp<VAgg>> windowToEmit = windowStore
+            .fetchAll(emitRangeLowerBoundInclusive, emitRangeUpperBoundInclusive);
+
+        int emittedCount = 0;
+        while (windowToEmit.hasNext()) {
+            emittedCount++;
+            final KeyValue<Windowed<KIn>, ValueAndTimestamp<VAgg>> kv = windowToEmit.next();
+            tupleForwarder.maybeForward(
+                record.withKey(kv.key)
+                    .withValue(new Change<>(kv.value.value(), null))
+                    .withTimestamp(kv.value.timestamp())
+                    .withHeaders(record.headers()));
+        }
+        emittedRecordsSensor.record(emittedCount);
+        emitFinalLatencySensor.record(time.milliseconds() - startMs);
+
+        lastEmitWindowCloseTime = closeTime;
+        internalProcessorContext.addProcessorMetadataKeyValue(storeName, closeTime);
+    }
+
+    abstract protected void maybeForwardFinalResult(final Record<KIn, VIn> record, final long windowCloseTime);
+
+    protected void maybeMeasureEmitFinalLatency(final Record<KIn, VIn> record, final long windowCloseTime) {

Review Comment:
   Nit: Should we make this method `final` ?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/EmitStrategy.java:
##########
@@ -19,15 +19,52 @@
 import org.apache.kafka.streams.kstream.internals.UnlimitedWindow;
 import org.apache.kafka.streams.kstream.internals.emitstrategy.WindowCloseStrategy;
 import org.apache.kafka.streams.kstream.internals.emitstrategy.WindowUpdateStrategy;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * This interface controls the strategy that can be used to control how we emit results in a processor.
  */
 public interface EmitStrategy {
 
+    Logger log = LoggerFactory.getLogger(EmitStrategy.class);
+
     enum StrategyType {
-        ON_WINDOW_CLOSE,
-        ON_WINDOW_UPDATE
+        ON_WINDOW_UPDATE(0, new WindowUpdateStrategy()),
+        ON_WINDOW_CLOSE(1, new WindowCloseStrategy());
+
+        private final short code;
+        private final EmitStrategy strategy;
+
+        private short code() {
+            return this.code;
+        }
+
+        private EmitStrategy strategy() {
+            return this.strategy;
+        }
+
+        StrategyType(final int code, final EmitStrategy strategy) {

Review Comment:
   Why are you using `int` here? Seems cleaner to use `short` ?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKStreamTimeWindowAggregateProcessor.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static org.apache.kafka.streams.StreamsConfig.InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emitFinalLatencySensor;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emittedRecordsSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.kstream.EmitStrategy;
+import org.apache.kafka.streams.kstream.EmitStrategy.StrategyType;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTracker;
+import org.apache.kafka.streams.processor.api.ContextualProcessor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+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;
+
+public abstract class AbstractKStreamTimeWindowAggregateProcessor<KIn, VIn, VAgg> extends ContextualProcessor<KIn, VIn, Windowed<KIn>, Change<VAgg>> {
+
+    private final Time time = Time.SYSTEM;
+    private final String storeName;
+    private final EmitStrategy emitStrategy;
+    private final boolean sendOldValues;
+    protected final TimeTracker timeTracker = new TimeTracker();
+
+    private TimestampedTupleForwarder<Windowed<KIn>, VAgg> tupleForwarder;
+    protected TimestampedWindowStore<KIn, VAgg> windowStore;
+    protected Sensor droppedRecordsSensor;
+    protected Sensor emittedRecordsSensor;
+    protected Sensor emitFinalLatencySensor;
+    protected long lastEmitWindowCloseTime = ConsumerRecord.NO_TIMESTAMP;
+    protected InternalProcessorContext<Windowed<KIn>, Change<VAgg>> internalProcessorContext;
+
+    protected AbstractKStreamTimeWindowAggregateProcessor(final String storeName,
+                                                          final EmitStrategy emitStrategy,
+                                                          final boolean sendOldValues) {
+        this.storeName = storeName;
+        this.emitStrategy = emitStrategy;
+        this.sendOldValues = sendOldValues;
+    }
+
+    @Override
+    public void init(final ProcessorContext<Windowed<KIn>, Change<VAgg>> context) {
+        super.init(context);
+        internalProcessorContext = (InternalProcessorContext<Windowed<KIn>, Change<VAgg>>) context;
+        final StreamsMetricsImpl metrics = internalProcessorContext.metrics();
+        final String threadId = Thread.currentThread().getName();
+        droppedRecordsSensor = droppedRecordsSensor(threadId, context.taskId().toString(), metrics);
+        emittedRecordsSensor = emittedRecordsSensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        emitFinalLatencySensor = emitFinalLatencySensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        windowStore = context.getStateStore(storeName);
+
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            // Restore last emit close time for ON_WINDOW_CLOSE strategy
+            final Long lastEmitTime = internalProcessorContext.processorMetadataForKey(storeName);
+            if (lastEmitTime != null) {
+                lastEmitWindowCloseTime = lastEmitTime;
+            }
+            final long emitInterval = StreamsConfig.InternalConfig.getLong(
+                context.appConfigs(),
+                EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION,
+                1000L
+            );
+            timeTracker.setEmitInterval(emitInterval);
+        } else {
+            tupleForwarder = new TimestampedTupleForwarder<>(

Review Comment:
   Why do we init the `tupleForwarder` only for "emit on change" -- don't we also need the forwarder for "emit final" (and just _use_ it differently)?
   
   In the original code, we setup `TimestampedTupleForwarder` w/ or w/o the `TimestampedCacheFlushListener` for both cases.



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKStreamTimeWindowAggregateProcessor.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static org.apache.kafka.streams.StreamsConfig.InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emitFinalLatencySensor;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emittedRecordsSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.kstream.EmitStrategy;
+import org.apache.kafka.streams.kstream.EmitStrategy.StrategyType;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTracker;
+import org.apache.kafka.streams.processor.api.ContextualProcessor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+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;
+
+public abstract class AbstractKStreamTimeWindowAggregateProcessor<KIn, VIn, VAgg> extends ContextualProcessor<KIn, VIn, Windowed<KIn>, Change<VAgg>> {
+
+    private final Time time = Time.SYSTEM;
+    private final String storeName;
+    private final EmitStrategy emitStrategy;
+    private final boolean sendOldValues;
+    protected final TimeTracker timeTracker = new TimeTracker();
+
+    private TimestampedTupleForwarder<Windowed<KIn>, VAgg> tupleForwarder;
+    protected TimestampedWindowStore<KIn, VAgg> windowStore;
+    protected Sensor droppedRecordsSensor;
+    protected Sensor emittedRecordsSensor;
+    protected Sensor emitFinalLatencySensor;
+    protected long lastEmitWindowCloseTime = ConsumerRecord.NO_TIMESTAMP;
+    protected InternalProcessorContext<Windowed<KIn>, Change<VAgg>> internalProcessorContext;
+
+    protected AbstractKStreamTimeWindowAggregateProcessor(final String storeName,
+                                                          final EmitStrategy emitStrategy,
+                                                          final boolean sendOldValues) {
+        this.storeName = storeName;
+        this.emitStrategy = emitStrategy;
+        this.sendOldValues = sendOldValues;
+    }
+
+    @Override
+    public void init(final ProcessorContext<Windowed<KIn>, Change<VAgg>> context) {
+        super.init(context);
+        internalProcessorContext = (InternalProcessorContext<Windowed<KIn>, Change<VAgg>>) context;
+        final StreamsMetricsImpl metrics = internalProcessorContext.metrics();
+        final String threadId = Thread.currentThread().getName();
+        droppedRecordsSensor = droppedRecordsSensor(threadId, context.taskId().toString(), metrics);
+        emittedRecordsSensor = emittedRecordsSensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        emitFinalLatencySensor = emitFinalLatencySensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        windowStore = context.getStateStore(storeName);
+
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            // Restore last emit close time for ON_WINDOW_CLOSE strategy
+            final Long lastEmitTime = internalProcessorContext.processorMetadataForKey(storeName);
+            if (lastEmitTime != null) {
+                lastEmitWindowCloseTime = lastEmitTime;
+            }
+            final long emitInterval = StreamsConfig.InternalConfig.getLong(
+                context.appConfigs(),
+                EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION,
+                1000L
+            );
+            timeTracker.setEmitInterval(emitInterval);
+        } else {
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+    }
+
+    protected void maybeForwardUpdate(final Record<KIn, VIn> record,
+                                      final Window window,
+                                      final VAgg oldAgg,
+                                      final VAgg newAgg,
+                                      final long newTimestamp) {
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            return;
+        } else if (tupleForwarder == null) {
+            throw new IllegalStateException("Emit strategy type is " + emitStrategy.type() + " but flush listener is not initialized.");
+        }
+
+        tupleForwarder.maybeForward(
+            record.withKey(new Windowed<>(record.key(), window))
+                .withValue(new Change<>(newAgg, sendOldValues ? oldAgg : null))
+                .withTimestamp(newTimestamp));
+    }
+
+    protected boolean shouldEmitFinal(final long closeTime) {
+        if (emitStrategy.type() != StrategyType.ON_WINDOW_CLOSE) {
+            return false;
+        }
+
+        final long now = internalProcessorContext.currentSystemTimeMs();
+        // Throttle emit frequency
+        if (now < timeTracker.nextTimeToEmit) {
+            return false;
+        }
+
+        // Schedule next emit time based on now to avoid the case that if system time jumps a lot,
+        // this can be triggered every time
+        timeTracker.nextTimeToEmit = now;
+        timeTracker.advanceNextTimeToEmit();
+
+        // Only EMIT if the window close time does progress
+        return lastEmitWindowCloseTime == ConsumerRecord.NO_TIMESTAMP || lastEmitWindowCloseTime < closeTime;
+    }
+
+    protected void fetchAndEmit(final Record<KIn, VIn> record,
+                                final long closeTime,
+                                final long emitRangeLowerBoundInclusive,
+                                final long emitRangeUpperBoundInclusive) {
+        final long startMs = time.milliseconds();
+
+        final KeyValueIterator<Windowed<KIn>, ValueAndTimestamp<VAgg>> windowToEmit = windowStore
+            .fetchAll(emitRangeLowerBoundInclusive, emitRangeUpperBoundInclusive);
+
+        int emittedCount = 0;
+        while (windowToEmit.hasNext()) {
+            emittedCount++;
+            final KeyValue<Windowed<KIn>, ValueAndTimestamp<VAgg>> kv = windowToEmit.next();
+            tupleForwarder.maybeForward(
+                record.withKey(kv.key)
+                    .withValue(new Change<>(kv.value.value(), null))
+                    .withTimestamp(kv.value.timestamp())
+                    .withHeaders(record.headers()));
+        }
+        emittedRecordsSensor.record(emittedCount);
+        emitFinalLatencySensor.record(time.milliseconds() - startMs);

Review Comment:
   What's the definition for this sensor? Wondering about the semantics given the current implemenation?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKStreamTimeWindowAggregateProcessor.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static org.apache.kafka.streams.StreamsConfig.InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emitFinalLatencySensor;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emittedRecordsSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.kstream.EmitStrategy;
+import org.apache.kafka.streams.kstream.EmitStrategy.StrategyType;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTracker;
+import org.apache.kafka.streams.processor.api.ContextualProcessor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+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;
+
+public abstract class AbstractKStreamTimeWindowAggregateProcessor<KIn, VIn, VAgg> extends ContextualProcessor<KIn, VIn, Windowed<KIn>, Change<VAgg>> {
+
+    private final Time time = Time.SYSTEM;
+    private final String storeName;
+    private final EmitStrategy emitStrategy;
+    private final boolean sendOldValues;
+    protected final TimeTracker timeTracker = new TimeTracker();
+
+    private TimestampedTupleForwarder<Windowed<KIn>, VAgg> tupleForwarder;
+    protected TimestampedWindowStore<KIn, VAgg> windowStore;
+    protected Sensor droppedRecordsSensor;
+    protected Sensor emittedRecordsSensor;
+    protected Sensor emitFinalLatencySensor;
+    protected long lastEmitWindowCloseTime = ConsumerRecord.NO_TIMESTAMP;
+    protected InternalProcessorContext<Windowed<KIn>, Change<VAgg>> internalProcessorContext;
+
+    protected AbstractKStreamTimeWindowAggregateProcessor(final String storeName,
+                                                          final EmitStrategy emitStrategy,
+                                                          final boolean sendOldValues) {
+        this.storeName = storeName;
+        this.emitStrategy = emitStrategy;
+        this.sendOldValues = sendOldValues;
+    }
+
+    @Override
+    public void init(final ProcessorContext<Windowed<KIn>, Change<VAgg>> context) {
+        super.init(context);
+        internalProcessorContext = (InternalProcessorContext<Windowed<KIn>, Change<VAgg>>) context;
+        final StreamsMetricsImpl metrics = internalProcessorContext.metrics();
+        final String threadId = Thread.currentThread().getName();
+        droppedRecordsSensor = droppedRecordsSensor(threadId, context.taskId().toString(), metrics);
+        emittedRecordsSensor = emittedRecordsSensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        emitFinalLatencySensor = emitFinalLatencySensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        windowStore = context.getStateStore(storeName);
+
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            // Restore last emit close time for ON_WINDOW_CLOSE strategy
+            final Long lastEmitTime = internalProcessorContext.processorMetadataForKey(storeName);
+            if (lastEmitTime != null) {
+                lastEmitWindowCloseTime = lastEmitTime;
+            }
+            final long emitInterval = StreamsConfig.InternalConfig.getLong(
+                context.appConfigs(),
+                EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION,
+                1000L
+            );
+            timeTracker.setEmitInterval(emitInterval);
+        } else {
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+    }
+
+    protected void maybeForwardUpdate(final Record<KIn, VIn> record,
+                                      final Window window,
+                                      final VAgg oldAgg,
+                                      final VAgg newAgg,
+                                      final long newTimestamp) {
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            return;
+        } else if (tupleForwarder == null) {
+            throw new IllegalStateException("Emit strategy type is " + emitStrategy.type() + " but flush listener is not initialized.");
+        }
+
+        tupleForwarder.maybeForward(
+            record.withKey(new Windowed<>(record.key(), window))
+                .withValue(new Change<>(newAgg, sendOldValues ? oldAgg : null))
+                .withTimestamp(newTimestamp));
+    }
+
+    protected boolean shouldEmitFinal(final long closeTime) {
+        if (emitStrategy.type() != StrategyType.ON_WINDOW_CLOSE) {
+            return false;
+        }
+
+        final long now = internalProcessorContext.currentSystemTimeMs();
+        // Throttle emit frequency
+        if (now < timeTracker.nextTimeToEmit) {
+            return false;
+        }
+
+        // Schedule next emit time based on now to avoid the case that if system time jumps a lot,
+        // this can be triggered every time
+        timeTracker.nextTimeToEmit = now;
+        timeTracker.advanceNextTimeToEmit();
+
+        // Only EMIT if the window close time does progress
+        return lastEmitWindowCloseTime == ConsumerRecord.NO_TIMESTAMP || lastEmitWindowCloseTime < closeTime;
+    }
+
+    protected void fetchAndEmit(final Record<KIn, VIn> record,
+                                final long closeTime,
+                                final long emitRangeLowerBoundInclusive,
+                                final long emitRangeUpperBoundInclusive) {
+        final long startMs = time.milliseconds();
+
+        final KeyValueIterator<Windowed<KIn>, ValueAndTimestamp<VAgg>> windowToEmit = windowStore
+            .fetchAll(emitRangeLowerBoundInclusive, emitRangeUpperBoundInclusive);
+
+        int emittedCount = 0;
+        while (windowToEmit.hasNext()) {
+            emittedCount++;
+            final KeyValue<Windowed<KIn>, ValueAndTimestamp<VAgg>> kv = windowToEmit.next();
+            tupleForwarder.maybeForward(

Review Comment:
   Seems we use `tupleForwarded` for the "emit final" case here, but it seems it was not initialized (cf my commend above on `init()` method)?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKStreamTimeWindowAggregateProcessor.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static org.apache.kafka.streams.StreamsConfig.InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emitFinalLatencySensor;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emittedRecordsSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.kstream.EmitStrategy;
+import org.apache.kafka.streams.kstream.EmitStrategy.StrategyType;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTracker;
+import org.apache.kafka.streams.processor.api.ContextualProcessor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+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;
+
+public abstract class AbstractKStreamTimeWindowAggregateProcessor<KIn, VIn, VAgg> extends ContextualProcessor<KIn, VIn, Windowed<KIn>, Change<VAgg>> {
+
+    private final Time time = Time.SYSTEM;
+    private final String storeName;
+    private final EmitStrategy emitStrategy;
+    private final boolean sendOldValues;
+    protected final TimeTracker timeTracker = new TimeTracker();
+
+    private TimestampedTupleForwarder<Windowed<KIn>, VAgg> tupleForwarder;
+    protected TimestampedWindowStore<KIn, VAgg> windowStore;
+    protected Sensor droppedRecordsSensor;
+    protected Sensor emittedRecordsSensor;
+    protected Sensor emitFinalLatencySensor;
+    protected long lastEmitWindowCloseTime = ConsumerRecord.NO_TIMESTAMP;
+    protected InternalProcessorContext<Windowed<KIn>, Change<VAgg>> internalProcessorContext;
+
+    protected AbstractKStreamTimeWindowAggregateProcessor(final String storeName,
+                                                          final EmitStrategy emitStrategy,
+                                                          final boolean sendOldValues) {
+        this.storeName = storeName;
+        this.emitStrategy = emitStrategy;
+        this.sendOldValues = sendOldValues;
+    }
+
+    @Override
+    public void init(final ProcessorContext<Windowed<KIn>, Change<VAgg>> context) {
+        super.init(context);
+        internalProcessorContext = (InternalProcessorContext<Windowed<KIn>, Change<VAgg>>) context;
+        final StreamsMetricsImpl metrics = internalProcessorContext.metrics();
+        final String threadId = Thread.currentThread().getName();
+        droppedRecordsSensor = droppedRecordsSensor(threadId, context.taskId().toString(), metrics);
+        emittedRecordsSensor = emittedRecordsSensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        emitFinalLatencySensor = emitFinalLatencySensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        windowStore = context.getStateStore(storeName);
+
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            // Restore last emit close time for ON_WINDOW_CLOSE strategy
+            final Long lastEmitTime = internalProcessorContext.processorMetadataForKey(storeName);
+            if (lastEmitTime != null) {
+                lastEmitWindowCloseTime = lastEmitTime;
+            }
+            final long emitInterval = StreamsConfig.InternalConfig.getLong(
+                context.appConfigs(),
+                EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION,
+                1000L
+            );
+            timeTracker.setEmitInterval(emitInterval);
+        } else {
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+    }
+
+    protected void maybeForwardUpdate(final Record<KIn, VIn> record,
+                                      final Window window,
+                                      final VAgg oldAgg,
+                                      final VAgg newAgg,
+                                      final long newTimestamp) {
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            return;
+        } else if (tupleForwarder == null) {
+            throw new IllegalStateException("Emit strategy type is " + emitStrategy.type() + " but flush listener is not initialized.");
+        }
+
+        tupleForwarder.maybeForward(
+            record.withKey(new Windowed<>(record.key(), window))
+                .withValue(new Change<>(newAgg, sendOldValues ? oldAgg : null))
+                .withTimestamp(newTimestamp));
+    }
+
+    protected boolean shouldEmitFinal(final long closeTime) {

Review Comment:
   It seems this method is called as a first step inside the implementation of `maybeForwardFinalResult` -- thus, I am wondering if we should not make it private and call inside `maybeMeasureEmitFinalLatency` ?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java:
##########
@@ -248,80 +182,43 @@ public void process(final Record<KIn, VIn> record) {
                 }
             }
 
-            tryEmitFinalResult(record, windowCloseTime);
+            maybeMeasureEmitFinalLatency(record, windowCloseTime);
         }
 
-        private void tryEmitFinalResult(final Record<KIn, VIn> record, final long windowCloseTime) {
-            if (emitStrategy.type() != StrategyType.ON_WINDOW_CLOSE) {
-                return;
-            }
-
-            final long now = internalProcessorContext.currentSystemTimeMs();
-            // Throttle emit frequency as an optimization, the tradeoff is that we need to remember the
-            // window close time when we emitted last time so that we can restart from there in the next emit
-            if (now < timeTracker.nextTimeToEmit) {
-                return;
-            }
-
-            // Schedule next emit time based on now to avoid the case that if system time jumps a lot,
-            // this can be triggered every time
-            timeTracker.nextTimeToEmit = now;
-            timeTracker.advanceNextTimeToEmit();
-
-            // Window close time has not progressed, there will be no windows to close hence no records to emit
-            if (lastEmitWindowCloseTime != ConsumerRecord.NO_TIMESTAMP && lastEmitWindowCloseTime >= windowCloseTime) {
+        @Override
+        protected void maybeForwardFinalResult(final Record<KIn, VIn> record, final long windowCloseTime) {
+            if (!shouldEmitFinal(windowCloseTime)) {
                 return;
             }
 
             final long emitRangeUpperBoundInclusive = windowCloseTime - windows.size();
-            // No window has ever closed and hence no need to emit any records
             if (emitRangeUpperBoundInclusive < 0) {
+                // If emitRangeUpperBoundInclusive is 0, it means first window closes since windowEndTime
+                // is exclusive
                 return;
             }
 
-
-            // Set emitRangeLowerBoundInclusive to -1L if lastEmitWindowCloseTime was not set so that
-            // we would fetch from 0L for the first time; otherwise set it to lastEmitWindowCloseTime - windows.size().
-            //
-            // Note if we get here, it means emitRangeUpperBoundInclusive > 0, which means windowCloseTime > windows.size(),
-            // Because we always set lastEmitWindowCloseTime to windowCloseTime before, it means
-            // lastEmitWindowCloseTime - windows.size() should always > 0
-            // As a result, emitRangeLowerBoundInclusive is always >= 0
+            // Because we only get here when emitRangeUpperBoundInclusive > 0 which means closeTime > windows.size()
+            // Since we set lastEmitCloseTime to closeTime before storing to processor metadata
+            // lastEmitCloseTime - windows.size() is always > 0
+            // Set emitRangeLowerBoundInclusive to -1L if not set so that when we fetchAll, we fetch from 0L
             final long emitRangeLowerBoundInclusive = lastEmitWindowCloseTime == ConsumerRecord.NO_TIMESTAMP ?
                 -1L : lastEmitWindowCloseTime - windows.size();

Review Comment:
   The code up to here is basically identical, except the use of `window.size()` vs `window. timeDifferenceMs()` -- would it be worth to unify?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractKStreamTimeWindowAggregateProcessor.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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 static org.apache.kafka.streams.StreamsConfig.InternalConfig.EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emitFinalLatencySensor;
+import static org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics.emittedRecordsSensor;
+import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency;
+import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.kstream.EmitStrategy;
+import org.apache.kafka.streams.kstream.EmitStrategy.StrategyType;
+import org.apache.kafka.streams.kstream.Window;
+import org.apache.kafka.streams.kstream.Windowed;
+import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTracker;
+import org.apache.kafka.streams.processor.api.ContextualProcessor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+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;
+
+public abstract class AbstractKStreamTimeWindowAggregateProcessor<KIn, VIn, VAgg> extends ContextualProcessor<KIn, VIn, Windowed<KIn>, Change<VAgg>> {
+
+    private final Time time = Time.SYSTEM;
+    private final String storeName;
+    private final EmitStrategy emitStrategy;
+    private final boolean sendOldValues;
+    protected final TimeTracker timeTracker = new TimeTracker();
+
+    private TimestampedTupleForwarder<Windowed<KIn>, VAgg> tupleForwarder;
+    protected TimestampedWindowStore<KIn, VAgg> windowStore;
+    protected Sensor droppedRecordsSensor;
+    protected Sensor emittedRecordsSensor;
+    protected Sensor emitFinalLatencySensor;
+    protected long lastEmitWindowCloseTime = ConsumerRecord.NO_TIMESTAMP;
+    protected InternalProcessorContext<Windowed<KIn>, Change<VAgg>> internalProcessorContext;
+
+    protected AbstractKStreamTimeWindowAggregateProcessor(final String storeName,
+                                                          final EmitStrategy emitStrategy,
+                                                          final boolean sendOldValues) {
+        this.storeName = storeName;
+        this.emitStrategy = emitStrategy;
+        this.sendOldValues = sendOldValues;
+    }
+
+    @Override
+    public void init(final ProcessorContext<Windowed<KIn>, Change<VAgg>> context) {
+        super.init(context);
+        internalProcessorContext = (InternalProcessorContext<Windowed<KIn>, Change<VAgg>>) context;
+        final StreamsMetricsImpl metrics = internalProcessorContext.metrics();
+        final String threadId = Thread.currentThread().getName();
+        droppedRecordsSensor = droppedRecordsSensor(threadId, context.taskId().toString(), metrics);
+        emittedRecordsSensor = emittedRecordsSensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        emitFinalLatencySensor = emitFinalLatencySensor(threadId, context.taskId().toString(),
+            internalProcessorContext.currentNode().name(), metrics);
+        windowStore = context.getStateStore(storeName);
+
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            // Restore last emit close time for ON_WINDOW_CLOSE strategy
+            final Long lastEmitTime = internalProcessorContext.processorMetadataForKey(storeName);
+            if (lastEmitTime != null) {
+                lastEmitWindowCloseTime = lastEmitTime;
+            }
+            final long emitInterval = StreamsConfig.InternalConfig.getLong(
+                context.appConfigs(),
+                EMIT_INTERVAL_MS_KSTREAMS_WINDOWED_AGGREGATION,
+                1000L
+            );
+            timeTracker.setEmitInterval(emitInterval);
+        } else {
+            tupleForwarder = new TimestampedTupleForwarder<>(
+                windowStore,
+                context,
+                new TimestampedCacheFlushListener<>(context),
+                sendOldValues);
+        }
+    }
+
+    protected void maybeForwardUpdate(final Record<KIn, VIn> record,
+                                      final Window window,
+                                      final VAgg oldAgg,
+                                      final VAgg newAgg,
+                                      final long newTimestamp) {
+        if (emitStrategy.type() == StrategyType.ON_WINDOW_CLOSE) {
+            return;
+        } else if (tupleForwarder == null) {
+            throw new IllegalStateException("Emit strategy type is " + emitStrategy.type() + " but flush listener is not initialized.");
+        }
+
+        tupleForwarder.maybeForward(
+            record.withKey(new Windowed<>(record.key(), window))
+                .withValue(new Change<>(newAgg, sendOldValues ? oldAgg : null))
+                .withTimestamp(newTimestamp));
+    }
+
+    protected boolean shouldEmitFinal(final long closeTime) {
+        if (emitStrategy.type() != StrategyType.ON_WINDOW_CLOSE) {
+            return false;
+        }
+
+        final long now = internalProcessorContext.currentSystemTimeMs();
+        // Throttle emit frequency
+        if (now < timeTracker.nextTimeToEmit) {
+            return false;
+        }
+
+        // Schedule next emit time based on now to avoid the case that if system time jumps a lot,
+        // this can be triggered every time
+        timeTracker.nextTimeToEmit = now;
+        timeTracker.advanceNextTimeToEmit();
+
+        // Only EMIT if the window close time does progress
+        return lastEmitWindowCloseTime == ConsumerRecord.NO_TIMESTAMP || lastEmitWindowCloseTime < closeTime;
+    }
+
+    protected void fetchAndEmit(final Record<KIn, VIn> record,
+                                final long closeTime,
+                                final long emitRangeLowerBoundInclusive,
+                                final long emitRangeUpperBoundInclusive) {
+        final long startMs = time.milliseconds();
+
+        final KeyValueIterator<Windowed<KIn>, ValueAndTimestamp<VAgg>> windowToEmit = windowStore
+            .fetchAll(emitRangeLowerBoundInclusive, emitRangeUpperBoundInclusive);
+
+        int emittedCount = 0;
+        while (windowToEmit.hasNext()) {
+            emittedCount++;
+            final KeyValue<Windowed<KIn>, ValueAndTimestamp<VAgg>> kv = windowToEmit.next();
+            tupleForwarder.maybeForward(
+                record.withKey(kv.key)
+                    .withValue(new Change<>(kv.value.value(), null))
+                    .withTimestamp(kv.value.timestamp())
+                    .withHeaders(record.headers()));
+        }
+        emittedRecordsSensor.record(emittedCount);
+        emitFinalLatencySensor.record(time.milliseconds() - startMs);
+
+        lastEmitWindowCloseTime = closeTime;
+        internalProcessorContext.addProcessorMetadataKeyValue(storeName, closeTime);
+    }
+
+    abstract protected void maybeForwardFinalResult(final Record<KIn, VIn> record, final long windowCloseTime);

Review Comment:
   I though `fetchAndEmit` would take care of "emit final" -- what is this method about?



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java:
##########
@@ -467,13 +459,35 @@ private boolean rightWindowIsNotEmpty(final ValueAndTimestamp<VAgg> rightWinAgg,
             return rightWinAgg != null && rightWinAgg.timestamp() > inputRecordTimestamp;
         }
 
+        @Override
+        protected void maybeForwardFinalResult(final Record<KIn, VIn> record, final long windowCloseTime) {
+            if (!shouldEmitFinal(windowCloseTime)) {
+                return;
+            }
+
+            final long emitRangeUpperBoundExclusive = windowCloseTime - windows.timeDifferenceMs();
+
+            if (emitRangeUpperBoundExclusive <= 0) {
+                // Sliding window's start and end timestamps are inclusive, so
+                // the window is not closed if emitRangeUpperBoundExclusive is 0,
+                // and we shouldn't emit
+                return;
+            }
+
+            final long emitRangeLowerBoundInclusive = lastEmitWindowCloseTime == ConsumerRecord.NO_TIMESTAMP ?
+                0L : lastEmitWindowCloseTime - windows.timeDifferenceMs();

Review Comment:
   Should this be:
   ```
   final long emitRangeLowerBoundInclusive = Math.max(0L, lastEmitWindowCloseTime - windows.timeDifferenceMs());
   ```
   
   Otherwise, if `lastEmitWindowCloseTime < windows.timeDifferenceMs()` the result could still be negative?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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