xishuaidelin commented on code in PR #25861:
URL: https://github.com/apache/flink/pull/25861#discussion_r1912430348


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/AsyncStateTableStreamOperator.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.flink.table.runtime.operators;
+
+import org.apache.flink.core.memory.ManagedMemoryUseCase;
+import 
org.apache.flink.runtime.asyncprocessing.operators.AbstractAsyncStateStreamOperator;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Table operator to invoke close always. This is a base class for both batch 
and stream operators
+ * without key.
+ *
+ * <p>This class is nearly identical with {@link TableStreamOperator}, but 
extending from {@link
+ * AbstractAsyncStateStreamOperator} to integrate with asynchronous state 
access.
+ */
+public abstract class AsyncStateTableStreamOperator<OUT>
+        extends AbstractAsyncStateStreamOperator<OUT> {
+
+    /** We listen to this ourselves because we don't have an {@link 
InternalTimerService}. */
+    protected long currentWatermark = Long.MIN_VALUE;
+
+    protected transient ContextImpl ctx;
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+        this.ctx = new ContextImpl(getProcessingTimeService());
+    }
+
+    @Override
+    public boolean useSplittableTimers() {
+        return true;
+    }
+
+    /** Compute memory size from memory faction. */
+    public long computeMemorySize() {
+        final Environment environment = getContainingTask().getEnvironment();
+        return environment
+                .getMemoryManager()
+                .computeMemorySize(
+                        getOperatorConfig()
+                                .getManagedMemoryFractionOperatorUseCaseOfSlot(
+                                        ManagedMemoryUseCase.OPERATOR,
+                                        environment.getJobConfiguration(),
+                                        
environment.getTaskManagerInfo().getConfiguration(),
+                                        
environment.getUserCodeClassLoader().asClassLoader()));
+    }
+
+    @Override
+    public Watermark preProcessWatermark(Watermark mark) throws Exception {
+        currentWatermark = mark.getTimestamp();
+        return super.preProcessWatermark(mark);
+    }
+
+    /** Information available in an invocation of processElement. */
+    protected class ContextImpl implements TimerService {
+
+        protected final ProcessingTimeService timerService;
+
+        public StreamRecord<?> element;
+
+        ContextImpl(ProcessingTimeService timerService) {
+            this.timerService = checkNotNull(timerService);
+        }
+
+        public Long timestamp() {
+            checkState(element != null);
+
+            if (element.hasTimestamp()) {
+                return element.getTimestamp();
+            } else {
+                return null;
+            }
+        }
+
+        @Override
+        public long currentProcessingTime() {
+            return timerService.getCurrentProcessingTime();
+        }
+
+        @Override
+        public long currentWatermark() {
+            return currentWatermark;
+        }
+
+        @Override
+        public void registerProcessingTimeTimer(long time) {
+            throw new UnsupportedOperationException(
+                    "Setting timers is only supported on a keyed streams.");

Review Comment:
   Nit: Setting timers is only supported on keyed streams.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/asyncprocessing/tvf/common/AsyncStateWindowAggOperator.java:
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.flink.table.runtime.operators.window.asyncprocessing.tvf.common;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.v2.ListState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.MeterView;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.state.v2.ListStateDescriptor;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.KeyContext;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.AsyncStateTableStreamOperator;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.tvf.state.AsyncStateKeyContext;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowProcessor;
+import org.apache.flink.table.runtime.util.AsyncStateUtils;
+
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A processor that processes elements for windows with async state. */
+public class AsyncStateWindowAggOperator<K, W> extends 
AsyncStateTableStreamOperator<RowData>

Review Comment:
   Nit: add @Internal and final



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceSharedWindowAggProcessor.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.flink.table.runtime.operators.aggregate.asyncwindow.processors;
+
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.state.StateFutureUtils;
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import 
org.apache.flink.table.runtime.operators.aggregate.asyncwindow.buffers.AsyncStateWindowBuffer;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.AsyncMergeCallback;
+import 
org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceSharedAssigner;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * A window aggregate processor implementation which works for {@link 
SliceSharedAssigner} with
+ * async state api, e.g. hopping windows and cumulative windows.
+ */
+public final class AsyncStateSliceSharedWindowAggProcessor
+        extends AbstractAsyncStateSliceWindowAggProcessor
+        implements AsyncMergeCallback<Long, Iterable<Long>> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SliceSharedAssigner sliceSharedAssigner;
+
+    public AsyncStateSliceSharedWindowAggProcessor(
+            GeneratedNamespaceAggsHandleFunction<Long> genAggsHandler,
+            AsyncStateWindowBuffer.Factory bufferFactory,
+            SliceSharedAssigner sliceAssigner,
+            TypeSerializer<RowData> accSerializer,
+            int indexOfCountStar,
+            ZoneId shiftTimeZone) {
+        super(
+                genAggsHandler,
+                bufferFactory,
+                sliceAssigner,
+                accSerializer,
+                indexOfCountStar,
+                shiftTimeZone);
+        this.sliceSharedAssigner = sliceAssigner;
+    }
+
+    @Override
+    public StateFuture<Void> fireWindow(long timerTimestamp, Long windowEnd) 
throws Exception {
+        return sliceSharedAssigner
+                .asyncMergeSlices(windowEnd, this)
+                .thenAccept(
+                        accAndAggResult -> {
+                            if (!emptyChecker.apply(accAndAggResult.f0)) {
+                                // if the triggered window is an empty window, 
we shouldn't emit it
+                                collect(
+                                        
ctx.getAsyncKeyContext().getCurrentKey(),
+                                        accAndAggResult.f1);
+                            }
+
+                            // we should register next window timer here,
+                            // because slices are shared, maybe no elements 
arrived for the next
+                            // slices
+                            Optional<Long> nextWindowEndOptional =
+                                    sliceSharedAssigner.nextTriggerWindow(
+                                            windowEnd, accAndAggResult.f0, 
emptyChecker);
+                            if (nextWindowEndOptional.isPresent()) {
+                                long nextWindowEnd = 
nextWindowEndOptional.get();
+                                if (sliceSharedAssigner.isEventTime()) {
+                                    
windowTimerService.registerEventTimeWindowTimer(nextWindowEnd);
+                                } else {
+                                    
windowTimerService.registerProcessingTimeWindowTimer(
+                                            nextWindowEnd);
+                                }
+                            }
+                        });
+    }
+
+    @Override
+    public StateFuture<Tuple2<RowData, RowData>> asyncMerge(
+            @Nullable Long mergeResult, Iterable<Long> toBeMerged, Long 
resultNamespace)
+            throws Exception {
+        // get base accumulator
+        final StateFuture<RowData> accOfMergeResultFuture;
+        if (mergeResult == null) {
+            // null means the merged is not on state, create a new acc
+            accOfMergeResultFuture =
+                    
StateFutureUtils.completedFuture(aggregator.createAccumulators());
+        } else {
+            accOfMergeResultFuture =
+                    windowState
+                            .asyncValue(mergeResult)
+                            .thenApply(
+                                    stateAcc -> {
+                                        if (stateAcc == null) {
+                                            return 
aggregator.createAccumulators();
+                                        } else {
+                                            return stateAcc;
+                                        }
+                                    });
+        }
+
+        StateFuture<Collection<Tuple2<Long, RowData>>> 
allAccOfSlicesToBeMergedFuture =
+                collectAccOfSlicesToBeMerged(toBeMerged);
+
+        return accOfMergeResultFuture
+                .thenCombine(
+                        allAccOfSlicesToBeMergedFuture,
+                        (accOfMergeResult, allAccOfSlicesToBeMerged) -> {
+                            // set base accumulator
+                            aggregator.setAccumulators(mergeResult, 
accOfMergeResult);
+
+                            // merge slice accumulators
+                            for (Tuple2<Long, RowData> sliceAndAcc : 
allAccOfSlicesToBeMerged) {
+                                RowData sliceAcc = sliceAndAcc.f1;
+                                if (sliceAcc != null) {
+                                    aggregator.merge(sliceAndAcc.f0, sliceAcc);
+                                }
+                            }
+
+                            return Tuple2.of(
+                                    aggregator.getAccumulators(),
+                                    aggregator.getValue(resultNamespace));
+                        })
+                .thenCompose(
+                        accAndAggResult -> {
+                            // set merged acc into state if the merged acc is 
on state
+                            if (mergeResult != null) {
+                                return windowState
+                                        .asyncUpdate(mergeResult, 
accAndAggResult.f0)
+                                        .thenApply(VOID -> accAndAggResult);
+                            } else {
+                                return 
StateFutureUtils.completedFuture(accAndAggResult);
+                            }
+                        });
+    }
+
+    private StateFuture<Collection<Tuple2<Long, RowData>>> 
collectAccOfSlicesToBeMerged(
+            Iterable<Long> slicesToBeMerged) throws Exception {
+        List<StateFuture<Tuple2<Long, RowData>>> futures = new ArrayList<>();
+        for (Long slice : slicesToBeMerged) {
+            futures.add(windowState.asyncValue(slice).thenApply(acc -> 
Tuple2.of(slice, acc)));
+        }
+        return StateFutureUtils.combineAll(futures);
+    }
+
+    @Override
+    protected StateFuture<Long> sliceStateMergeTarget(long sliceToMerge) 
throws Exception {
+        SliceMergeTargetHelper mergeHelper = new SliceMergeTargetHelper();
+        return sliceSharedAssigner
+                .asyncMergeSlices(sliceToMerge, mergeHelper)
+                .thenApply(
+                        VOID -> {
+                            // the mergeTarget might be null, which means the 
merging happens in
+                            // memory instead of
+                            // on state, so the slice state to merge into is 
itself.
+                            if (mergeHelper.getMergeTarget() != null) {
+                                return mergeHelper.getMergeTarget();
+                            } else {
+                                return sliceToMerge;
+                            }
+                        });
+    }
+
+    // 
------------------------------------------------------------------------------------------
+
+    private static final class SliceMergeTargetHelper
+            implements AsyncMergeCallback<Long, Iterable<Long>>, Serializable {
+
+        private static final long serialVersionUID = 1L;
+        private static final StateFuture<Tuple2<RowData, RowData>> 
REUSABLE_FUTURE_RESULT =
+                StateFutureUtils.completedFuture(null);
+        private Long mergeTarget = null;
+
+        @Override
+        public StateFuture<Tuple2<RowData, RowData>> asyncMerge(
+                @Nullable Long mergeResult, Iterable<Long> toBeMerged, Long 
resultNamespace)
+                throws Exception {

Review Comment:
   Nit: the exception is never thrown in the method.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceSharedWindowAggProcessor.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.flink.table.runtime.operators.aggregate.asyncwindow.processors;
+
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.state.StateFutureUtils;
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import 
org.apache.flink.table.runtime.operators.aggregate.asyncwindow.buffers.AsyncStateWindowBuffer;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.AsyncMergeCallback;
+import 
org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceSharedAssigner;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * A window aggregate processor implementation which works for {@link 
SliceSharedAssigner} with
+ * async state api, e.g. hopping windows and cumulative windows.
+ */
+public final class AsyncStateSliceSharedWindowAggProcessor
+        extends AbstractAsyncStateSliceWindowAggProcessor
+        implements AsyncMergeCallback<Long, Iterable<Long>> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SliceSharedAssigner sliceSharedAssigner;
+
+    public AsyncStateSliceSharedWindowAggProcessor(
+            GeneratedNamespaceAggsHandleFunction<Long> genAggsHandler,
+            AsyncStateWindowBuffer.Factory bufferFactory,
+            SliceSharedAssigner sliceAssigner,
+            TypeSerializer<RowData> accSerializer,
+            int indexOfCountStar,
+            ZoneId shiftTimeZone) {
+        super(
+                genAggsHandler,
+                bufferFactory,
+                sliceAssigner,
+                accSerializer,
+                indexOfCountStar,
+                shiftTimeZone);
+        this.sliceSharedAssigner = sliceAssigner;
+    }
+
+    @Override
+    public StateFuture<Void> fireWindow(long timerTimestamp, Long windowEnd) 
throws Exception {
+        return sliceSharedAssigner
+                .asyncMergeSlices(windowEnd, this)
+                .thenAccept(
+                        accAndAggResult -> {
+                            if (!emptyChecker.apply(accAndAggResult.f0)) {
+                                // if the triggered window is an empty window, 
we shouldn't emit it
+                                collect(
+                                        
ctx.getAsyncKeyContext().getCurrentKey(),
+                                        accAndAggResult.f1);
+                            }
+
+                            // we should register next window timer here,
+                            // because slices are shared, maybe no elements 
arrived for the next
+                            // slices
+                            Optional<Long> nextWindowEndOptional =
+                                    sliceSharedAssigner.nextTriggerWindow(
+                                            windowEnd, accAndAggResult.f0, 
emptyChecker);
+                            if (nextWindowEndOptional.isPresent()) {
+                                long nextWindowEnd = 
nextWindowEndOptional.get();
+                                if (sliceSharedAssigner.isEventTime()) {
+                                    
windowTimerService.registerEventTimeWindowTimer(nextWindowEnd);
+                                } else {
+                                    
windowTimerService.registerProcessingTimeWindowTimer(
+                                            nextWindowEnd);
+                                }
+                            }
+                        });
+    }
+
+    @Override
+    public StateFuture<Tuple2<RowData, RowData>> asyncMerge(
+            @Nullable Long mergeResult, Iterable<Long> toBeMerged, Long 
resultNamespace)
+            throws Exception {
+        // get base accumulator
+        final StateFuture<RowData> accOfMergeResultFuture;
+        if (mergeResult == null) {
+            // null means the merged is not on state, create a new acc
+            accOfMergeResultFuture =
+                    
StateFutureUtils.completedFuture(aggregator.createAccumulators());
+        } else {
+            accOfMergeResultFuture =
+                    windowState
+                            .asyncValue(mergeResult)
+                            .thenApply(
+                                    stateAcc -> {
+                                        if (stateAcc == null) {
+                                            return 
aggregator.createAccumulators();
+                                        } else {
+                                            return stateAcc;
+                                        }
+                                    });
+        }
+
+        StateFuture<Collection<Tuple2<Long, RowData>>> 
allAccOfSlicesToBeMergedFuture =
+                collectAccOfSlicesToBeMerged(toBeMerged);
+
+        return accOfMergeResultFuture
+                .thenCombine(
+                        allAccOfSlicesToBeMergedFuture,
+                        (accOfMergeResult, allAccOfSlicesToBeMerged) -> {
+                            // set base accumulator
+                            aggregator.setAccumulators(mergeResult, 
accOfMergeResult);
+
+                            // merge slice accumulators
+                            for (Tuple2<Long, RowData> sliceAndAcc : 
allAccOfSlicesToBeMerged) {
+                                RowData sliceAcc = sliceAndAcc.f1;
+                                if (sliceAcc != null) {
+                                    aggregator.merge(sliceAndAcc.f0, sliceAcc);
+                                }
+                            }
+
+                            return Tuple2.of(
+                                    aggregator.getAccumulators(),
+                                    aggregator.getValue(resultNamespace));
+                        })
+                .thenCompose(
+                        accAndAggResult -> {
+                            // set merged acc into state if the merged acc is 
on state
+                            if (mergeResult != null) {
+                                return windowState
+                                        .asyncUpdate(mergeResult, 
accAndAggResult.f0)
+                                        .thenApply(VOID -> accAndAggResult);
+                            } else {
+                                return 
StateFutureUtils.completedFuture(accAndAggResult);
+                            }
+                        });
+    }
+
+    private StateFuture<Collection<Tuple2<Long, RowData>>> 
collectAccOfSlicesToBeMerged(
+            Iterable<Long> slicesToBeMerged) throws Exception {
+        List<StateFuture<Tuple2<Long, RowData>>> futures = new ArrayList<>();
+        for (Long slice : slicesToBeMerged) {
+            futures.add(windowState.asyncValue(slice).thenApply(acc -> 
Tuple2.of(slice, acc)));
+        }
+        return StateFutureUtils.combineAll(futures);
+    }
+
+    @Override
+    protected StateFuture<Long> sliceStateMergeTarget(long sliceToMerge) 
throws Exception {
+        SliceMergeTargetHelper mergeHelper = new SliceMergeTargetHelper();
+        return sliceSharedAssigner
+                .asyncMergeSlices(sliceToMerge, mergeHelper)
+                .thenApply(
+                        VOID -> {
+                            // the mergeTarget might be null, which means the 
merging happens in
+                            // memory instead of
+                            // on state, so the slice state to merge into is 
itself.
+                            if (mergeHelper.getMergeTarget() != null) {
+                                return mergeHelper.getMergeTarget();
+                            } else {
+                                return sliceToMerge;
+                            }
+                        });
+    }
+
+    // 
------------------------------------------------------------------------------------------
+
+    private static final class SliceMergeTargetHelper

Review Comment:
   Nit: Could we rename this to SliceMergeTargetCallback to better reflect its 
purpose?



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/buffers/AsyncStateRecordsWindowBuffer.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.flink.table.runtime.operators.aggregate.asyncwindow.buffers;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
+import org.apache.flink.table.runtime.generated.RecordEqualiser;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.tvf.combines.AsyncStateRecordsCombiner;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.tvf.state.AsyncStateKeyContext;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.tvf.state.WindowAsyncState;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService;
+import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer;
+import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer;
+import org.apache.flink.table.runtime.typeutils.WindowKeySerializer;
+import org.apache.flink.table.runtime.util.KeyValueIterator;
+import org.apache.flink.table.runtime.util.WindowKey;
+import org.apache.flink.table.runtime.util.collections.binary.BytesMap;
+import 
org.apache.flink.table.runtime.util.collections.binary.WindowBytesMultiMap;
+
+import javax.annotation.Nullable;
+
+import java.io.EOFException;
+import java.time.ZoneId;
+import java.util.Iterator;
+
+import static 
org.apache.flink.table.runtime.util.AsyncStateUtils.REUSABLE_VOID_STATE_FUTURE;
+import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired;
+
+/**
+ * An implementation of {@link AsyncStateWindowBuffer} that buffers input 
elements in a {@link
+ * WindowBytesMultiMap} and combines buffered elements into async state when 
flushing.
+ */
+public class AsyncStateRecordsWindowBuffer implements AsyncStateWindowBuffer {

Review Comment:
   Nit: add final



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/asyncprocessing/tvf/common/AsyncStateWindowAggOperator.java:
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.flink.table.runtime.operators.window.asyncprocessing.tvf.common;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.v2.ListState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.MeterView;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.state.v2.ListStateDescriptor;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.KeyContext;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.AsyncStateTableStreamOperator;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.tvf.state.AsyncStateKeyContext;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowProcessor;
+import org.apache.flink.table.runtime.util.AsyncStateUtils;
+
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A processor that processes elements for windows with async state. */

Review Comment:
   Nit: Could we add more detailed comments here? While we don't need to copy 
the entire Java docs from WindowAggOperator, a brief explanation along with a 
reference to WindowAggOperator would help viewers quickly locate the relevant 
information.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/asyncprocessing/tvf/common/AsyncStateWindowProcessor.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.flink.table.runtime.operators.window.asyncprocessing.tvf.common;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.state.StateFutureUtils;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.tvf.state.AsyncStateKeyContext;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+
+/** A processor that processes elements for windows with async state. */
+@Internal
+public interface AsyncStateWindowProcessor<W> extends Serializable {
+
+    /** Initialization method for the function. It is called before the actual 
working methods. */
+    void open(Context<W> context) throws Exception;
+
+    /**
+     * Initializes the watermark which restores from state. The method is 
called after open method
+     * and before the actual working methods.
+     *
+     * @param watermark the initial watermark
+     */
+    void initializeWatermark(long watermark);
+
+    /**
+     * Process an element with associated key from the input stream. Returns 
true if this element is
+     * dropped because of late arrival.
+     *
+     * @param key the key associated with the element
+     * @param element The element to process.
+     */
+    StateFuture<Boolean> processElement(RowData key, RowData element) throws 
Exception;
+
+    /**
+     * Advances the progress time, the progress time is watermark if working 
in event-time mode, or
+     * current processing time if working in processing-time mode.
+     *
+     * <p>This will potentially flush buffered data into states, because the 
watermark advancement
+     * may be in a very small step, but we don't need to flush buffered data 
for every watermark
+     * advancement.
+     *
+     * <p>Note: There may be multiple different keys within the buffer. When 
flushing them to the
+     * async state, only the async state request for the current key of the 
operator will be
+     * returned as a {@link StateFuture}. Requests for async states for other 
keys will not be
+     * waited on.
+     *
+     * @param currentKey the current key of the operator used to return the 
result of accessing
+     *     async state associated with the same key. If it is null, it means 
that the returns of
+     *     asynchronous state requests for all keys will not be awaited.
+     * @param progress the current progress time
+     * @return the future of the flush operation about current key if the 
current key is not null,
+     *     else a {@link StateFutureUtils#completedVoidFuture()} will be 
returned.
+     */
+    StateFuture<Void> advanceProgress(@Nullable RowData currentKey, long 
progress) throws Exception;
+
+    /** Performs a preparation before checkpoint. This usually flushes 
buffered data into state. */
+    StateFuture<Void> prepareCheckpoint() throws Exception;
+
+    /**
+     * Emit results of the given window.
+     *
+     * <p>Note: the key context has been set.
+     *
+     * @param timerTimestamp the fired timestamp
+     * @param window the window to emit
+     */
+    StateFuture<Void> fireWindow(long timerTimestamp, W window) throws 
Exception;
+
+    /**
+     * Clear state and resources associated with the given window namespace.
+     *
+     * <p>Note: the key context has been set.
+     *
+     * @param timerTimestamp the fired timestamp
+     * @param window the window to clear
+     */
+    StateFuture<Void> clearWindow(long timerTimestamp, W window) throws 
Exception;
+
+    /**
+     * The tear-down method of the function. It is called after the last call 
to the main working
+     * methods.
+     */
+    void close() throws Exception;
+
+    /** Returns the serializer of the window type. */
+    TypeSerializer<W> createWindowSerializer();
+
+    // 
------------------------------------------------------------------------------------------
+
+    /** Information available in an invocation of methods of {@link 
AsyncStateWindowProcessor}. */
+    interface Context<W> {

Review Comment:
   Nit: Could we avoid introducing a new Context here? It is almost identical 
to WindowProcessor#Context, except for one method. Perhaps we can abstract the 
common logic to reduce code redundancy.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AbstractAsyncStateSliceWindowAggProcessor.java:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.flink.table.runtime.operators.aggregate.asyncwindow.processors;
+
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.core.state.StateFutureUtils;
+import org.apache.flink.table.data.RowData;
+import 
org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import 
org.apache.flink.table.runtime.operators.aggregate.asyncwindow.buffers.AsyncStateWindowBuffer;
+import org.apache.flink.table.runtime.operators.window.MergeCallback;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.tvf.common.AsyncStateWindowProcessor;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.tvf.slicing.AsyncStateSlicingWindowProcessor;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService;
+import 
org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigner;
+import 
org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceSharedAssigner;
+import 
org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowTimerServiceImpl;
+
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.List;
+
+import static 
org.apache.flink.table.runtime.util.AsyncStateUtils.REUSABLE_TRUE_STATE_FUTURE;
+import static 
org.apache.flink.table.runtime.util.AsyncStateUtils.REUSABLE_VOID_STATE_FUTURE;
+import static 
org.apache.flink.table.runtime.util.TimeWindowUtil.getNextTriggerWatermark;
+import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired;
+
+/** A base implementation of {@link AsyncStateSlicingWindowProcessor} for 
window aggregate. */
+public abstract class AbstractAsyncStateSliceWindowAggProcessor
+        extends AbstractAsyncStateWindowAggProcessor<Long>
+        implements AsyncStateSlicingWindowProcessor<Long> {
+
+    protected final AsyncStateWindowBuffer.Factory windowBufferFactory;
+    protected final SliceAssigner sliceAssigner;
+    protected final long windowInterval;
+
+    // 
----------------------------------------------------------------------------------------
+
+    /** The next progress to trigger windows. */
+    private transient long nextTriggerProgress;
+
+    protected transient AsyncStateWindowBuffer windowBuffer;
+
+    public AbstractAsyncStateSliceWindowAggProcessor(
+            GeneratedNamespaceAggsHandleFunction<Long> genAggsHandler,
+            AsyncStateWindowBuffer.Factory bufferFactory,
+            SliceAssigner sliceAssigner,
+            TypeSerializer<RowData> accSerializer,
+            int indexOfCountStar,
+            ZoneId shiftTimeZone) {
+        super(
+                genAggsHandler,
+                sliceAssigner,
+                accSerializer,
+                sliceAssigner.isEventTime(),
+                indexOfCountStar,
+                shiftTimeZone,
+                Long.MIN_VALUE);
+        this.windowBufferFactory = bufferFactory;
+        this.sliceAssigner = sliceAssigner;
+        this.windowInterval = sliceAssigner.getSliceEndInterval();
+    }
+
+    @Override
+    public void open(AsyncStateWindowProcessor.Context<Long> context) throws 
Exception {
+        super.open(context);
+        this.windowBuffer =
+                windowBufferFactory.create(
+                        ctx.getOperatorOwner(),
+                        ctx.getMemoryManager(),
+                        ctx.getMemorySize(),
+                        ctx.getRuntimeContext(),
+                        windowTimerService,
+                        ctx.getAsyncKeyContext(),
+                        windowState,
+                        isEventTime,
+                        shiftTimeZone);
+
+        this.nextTriggerProgress = Long.MIN_VALUE;
+    }
+
+    @Override
+    protected WindowTimerService<Long> getWindowTimerService() {
+        return new SlicingWindowTimerServiceImpl(ctx.getTimerService(), 
shiftTimeZone);
+    }
+
+    @Override
+    public StateFuture<Boolean> processElement(RowData key, RowData element) 
throws Exception {

Review Comment:
   This method closely resembles 
AbstractSliceWindowAggProcessor#processElement. In fact, we have multiple 
methods with similar logic. Could we consider abstracting these common patterns 
into a shared function to reduce code redundancy and improve maintainability?



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/asyncprocessing/tvf/common/AsyncStateWindowAggOperator.java:
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.flink.table.runtime.operators.window.asyncprocessing.tvf.common;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.v2.ListState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.MeterView;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.state.OperatorStateBackend;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.state.v2.ListStateDescriptor;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.KeyContext;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.AsyncStateTableStreamOperator;
+import 
org.apache.flink.table.runtime.operators.window.asyncprocessing.tvf.state.AsyncStateKeyContext;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowProcessor;
+import org.apache.flink.table.runtime.util.AsyncStateUtils;
+
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A processor that processes elements for windows with async state. */
+public class AsyncStateWindowAggOperator<K, W> extends 
AsyncStateTableStreamOperator<RowData>
+        implements OneInputStreamOperator<RowData, RowData>, Triggerable<K, 
W>, KeyContext {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final String LATE_ELEMENTS_DROPPED_METRIC_NAME = 
"numLateRecordsDropped";
+    private static final String LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME = 
"lateRecordsDroppedRate";
+    private static final String WATERMARK_LATENCY_METRIC_NAME = 
"watermarkLatency";
+
+    /** The concrete window operator implementation. */
+    private final AsyncStateWindowProcessor<W> windowProcessor;
+
+    private final boolean isEventTime;
+
+    // ------------------------------------------------------------------------
+
+    /** This is used for emitting elements with a given timestamp. */
+    private transient TimestampedCollector<RowData> collector;
+
+    /** The service to register timers. */
+    private transient InternalTimerService<W> internalTimerService;
+
+    /** The tracked processing time triggered last time. */
+    private transient long lastTriggeredProcessingTime;
+
+    /** The operator state to store watermark. */
+    private transient ListState<Long> watermarkState;
+
+    // ------------------------------------------------------------------------
+    // Metrics
+    // ------------------------------------------------------------------------
+
+    private transient Counter numLateRecordsDropped;
+    private transient Meter lateRecordsDroppedRate;
+
+    private transient Gauge<Long> watermarkLatency;
+
+    public AsyncStateWindowAggOperator(
+            AsyncStateWindowProcessor<W> windowProcessor, boolean isEventTime) 
{
+        this.windowProcessor = windowProcessor;
+        this.isEventTime = isEventTime;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        lastTriggeredProcessingTime = Long.MIN_VALUE;
+        collector = new TimestampedCollector<>(output);
+        collector.eraseTimestamp();
+
+        internalTimerService =
+                getInternalTimerService(
+                        "window-timers", 
windowProcessor.createWindowSerializer(), this);
+
+        windowProcessor.open(
+                new WindowProcessorContext<>(
+                        getContainingTask(),
+                        
getContainingTask().getEnvironment().getMemoryManager(),
+                        computeMemorySize(),
+                        internalTimerService,
+                        new AsyncStateKeyContext(this, 
getAsyncKeyedStateBackend()),
+                        collector,
+                        getRuntimeContext()));
+        // initialize watermark
+        windowProcessor.initializeWatermark(currentWatermark);
+
+        // metrics
+        this.numLateRecordsDropped = 
metrics.counter(LATE_ELEMENTS_DROPPED_METRIC_NAME);
+        this.lateRecordsDroppedRate =
+                metrics.meter(
+                        LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME,
+                        new MeterView(numLateRecordsDropped));
+        this.watermarkLatency =
+                metrics.gauge(
+                        WATERMARK_LATENCY_METRIC_NAME,
+                        () -> {
+                            long watermark = 
internalTimerService.currentWatermark();
+                            if (watermark < 0) {
+                                return 0L;
+                            } else {
+                                return 
internalTimerService.currentProcessingTime() - watermark;
+                            }
+                        });
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws 
Exception {
+        super.initializeState(context);
+        ListStateDescriptor<Long> watermarkStateDesc =
+                new ListStateDescriptor<>("watermark", 
LongSerializer.INSTANCE);
+        this.watermarkState =
+                ((OperatorStateBackend) context.getOperatorStateStore())
+                        .getUnionListState(watermarkStateDesc);
+        if (context.isRestored()) {
+            AtomicLong minWatermark = new AtomicLong(Long.MAX_VALUE);
+            watermarkState
+                    .asyncGet()
+                    .thenCompose(
+                            its ->
+                                    its.onNext(
+                                            watermark -> {
+                                                minWatermark.set(
+                                                        Math.min(watermark, 
minWatermark.get()));
+                                            }))
+                    .thenAccept(
+                            VOID -> {
+                                if (minWatermark.get() != Long.MAX_VALUE) {
+                                    this.currentWatermark = minWatermark.get();
+                                }
+                            });
+        }
+    }
+
+    @Override
+    public void snapshotState(StateSnapshotContext context) throws Exception {
+        super.snapshotState(context);
+        
this.watermarkState.asyncUpdate(Collections.singletonList(currentWatermark));
+    }
+
+    @Override
+    public void close() throws Exception {
+        super.close();
+        collector = null;
+        windowProcessor.close();
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception 
{
+        RowData inputRow = element.getValue();
+        RowData currentKey = (RowData) getCurrentKey();
+        windowProcessor
+                .processElement(currentKey, inputRow)
+                .thenAccept(
+                        isElementDropped -> {
+                            if (isElementDropped) {
+                                // markEvent will increase 
numLateRecordsDropped
+                                lateRecordsDroppedRate.markEvent();
+                            }
+                        });
+    }
+
+    @Override
+    public Watermark preProcessWatermark(Watermark mark) throws Exception {
+        if (mark.getTimestamp() > currentWatermark) {
+            // If this is a proctime window, progress should not be advanced 
by watermark, or it'll
+            // disturb timer-based processing
+            if (isEventTime) {
+                windowProcessor.advanceProgress(null, mark.getTimestamp());
+            }
+            return super.preProcessWatermark(mark);
+        } else {
+            return super.preProcessWatermark(new Watermark(currentWatermark));
+        }
+    }
+
+    @Override
+    public void onEventTime(InternalTimer<K, W> timer) throws Exception {
+        onTimer(timer);
+    }
+
+    @Override
+    public void onProcessingTime(InternalTimer<K, W> timer) throws Exception {
+        StateFuture<Void> advanceFuture = 
AsyncStateUtils.REUSABLE_VOID_STATE_FUTURE;
+        if (timer.getTimestamp() > lastTriggeredProcessingTime) {
+            // similar to the watermark advance,
+            // we need to notify WindowProcessor first to flush buffer into 
state
+            lastTriggeredProcessingTime = timer.getTimestamp();
+            advanceFuture =
+                    windowProcessor.advanceProgress(
+                            (RowData) getCurrentKey(), timer.getTimestamp());
+            // timers registered in advanceProgress() should always be smaller 
than current timer
+            // so, it should be safe to trigger current timer straightforwards.
+        }
+        advanceFuture.thenAccept(VOID -> onTimer(timer));
+    }
+
+    private void onTimer(InternalTimer<K, W> timer) throws Exception {
+        W window = timer.getNamespace();
+        windowProcessor
+                .fireWindow(timer.getTimestamp(), window)
+                .thenAccept(VOID -> 
windowProcessor.clearWindow(timer.getTimestamp(), window));
+        // we don't need to clear window timers,
+        // because there should only be one timer for each window now, which 
is current timer.
+    }
+
+    @Override
+    public void prepareSnapshotPreBarrier(long checkpointId) throws Exception {
+        windowProcessor.prepareCheckpoint();
+    }
+
+    /** Context implementation for {@link WindowProcessor.Context}. */
+    private static final class WindowProcessorContext<W>

Review Comment:
   Ditto



-- 
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: issues-unsubscr...@flink.apache.org

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

Reply via email to