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


##########
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:
   EDIT: actually you're right! :) The maybeForward should still be called in 
`emit final` case.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to