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


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/asyncprocessing/AsyncStateWindowJoinOperator.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.join.window.asyncprocessing;
+
+import org.apache.flink.api.common.functions.DefaultOpenContext;
+import org.apache.flink.api.common.state.v2.ListState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.state.v2.StateIterator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.runtime.state.v2.ListStateDescriptor;
+import org.apache.flink.runtime.state.v2.internal.InternalListState;
+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.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.operators.AsyncStateTableStreamOperator;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import 
org.apache.flink.table.runtime.operators.join.JoinConditionWithNullFilters;
+import org.apache.flink.table.runtime.operators.join.window.WindowJoinOperator;
+import 
org.apache.flink.table.runtime.operators.join.window.utils.WindowJoinHelper;
+import 
org.apache.flink.table.runtime.operators.window.tvf.asyncprocessing.state.WindowListAsyncState;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService;
+import 
org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowTimerServiceImpl;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/** A {@link WindowJoinOperator} implemented by async state api. */
+public class AsyncStateWindowJoinOperator extends 
AsyncStateTableStreamOperator<RowData>
+        implements TwoInputStreamOperator<RowData, RowData, RowData>,
+                Triggerable<RowData, Long>,
+                KeyContext {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final String LEFT_RECORDS_STATE_NAME = "left-records";
+    private static final String RIGHT_RECORDS_STATE_NAME = "right-records";
+
+    private final RowDataSerializer leftSerializer;
+    private final RowDataSerializer rightSerializer;
+    private final GeneratedJoinCondition generatedJoinCondition;
+
+    private final int leftWindowEndIndex;
+    private final int rightWindowEndIndex;
+
+    private final boolean[] filterNullKeys;
+    private final ZoneId shiftTimeZone;
+
+    private final FlinkJoinType joinType;
+
+    private transient WindowTimerService<Long> windowTimerService;
+
+    // ------------------------------------------------------------------------
+    private transient JoinConditionWithNullFilters joinCondition;
+
+    /** This is used for emitting elements with a given timestamp. */
+    private transient TimestampedCollector<RowData> collector;
+
+    private transient WindowListAsyncState<Long> leftWindowState;
+    private transient WindowListAsyncState<Long> rightWindowState;
+
+    private transient WindowJoinHelper helper;
+
+    public AsyncStateWindowJoinOperator(
+            TypeSerializer<RowData> leftSerializer,
+            TypeSerializer<RowData> rightSerializer,
+            GeneratedJoinCondition generatedJoinCondition,
+            int leftWindowEndIndex,
+            int rightWindowEndIndex,
+            boolean[] filterNullKeys,
+            ZoneId shiftTimeZone,
+            FlinkJoinType joinType) {
+        this.leftSerializer = (RowDataSerializer) leftSerializer;
+        this.rightSerializer = (RowDataSerializer) rightSerializer;
+        this.generatedJoinCondition = generatedJoinCondition;
+        this.leftWindowEndIndex = leftWindowEndIndex;
+        this.rightWindowEndIndex = rightWindowEndIndex;
+        this.filterNullKeys = filterNullKeys;
+        this.shiftTimeZone = shiftTimeZone;
+        this.joinType = joinType;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        this.collector = new TimestampedCollector<>(output);
+        collector.eraseTimestamp();
+
+        final LongSerializer windowSerializer = LongSerializer.INSTANCE;
+
+        InternalTimerService<Long> internalTimerService =
+                getInternalTimerService("window-timers", windowSerializer, 
this);
+        this.windowTimerService =
+                new SlicingWindowTimerServiceImpl(internalTimerService, 
shiftTimeZone);
+
+        // init join condition
+        JoinCondition condition =
+                
generatedJoinCondition.newInstance(getRuntimeContext().getUserCodeClassLoader());
+        this.joinCondition = new JoinConditionWithNullFilters(condition, 
filterNullKeys, this);
+        this.joinCondition.setRuntimeContext(getRuntimeContext());
+        this.joinCondition.open(DefaultOpenContext.INSTANCE);
+
+        // init state
+        ListStateDescriptor<RowData> leftRecordStateDesc =
+                new ListStateDescriptor<>(LEFT_RECORDS_STATE_NAME, 
leftSerializer);
+        ListState<RowData> leftListState =
+                getOrCreateKeyedState(Long.MIN_VALUE, windowSerializer, 
leftRecordStateDesc);
+        this.leftWindowState =
+                new WindowListAsyncState<>(
+                        (InternalListState<RowData, Long, RowData>) 
leftListState);
+
+        ListStateDescriptor<RowData> rightRecordStateDesc =
+                new ListStateDescriptor<>(RIGHT_RECORDS_STATE_NAME, 
rightSerializer);
+        ListState<RowData> rightListState =
+                getOrCreateKeyedState(Long.MIN_VALUE, windowSerializer, 
rightRecordStateDesc);
+        this.rightWindowState =
+                new WindowListAsyncState<>(
+                        (InternalListState<RowData, Long, RowData>) 
rightListState);
+
+        this.helper = new AsyncStateWindowJoinHelper();
+    }
+
+    @Override
+    public void close() throws Exception {
+        super.close();
+        collector = null;
+        if (joinCondition != null) {
+            joinCondition.close();
+        }
+    }
+
+    @Override
+    public void processElement1(StreamRecord<RowData> element) throws 
Exception {
+        helper.processElement(element.getValue(), true);
+    }
+
+    @Override
+    public void processElement2(StreamRecord<RowData> element) throws 
Exception {
+        helper.processElement(element.getValue(), false);
+    }
+
+    @Override
+    public void onProcessingTime(InternalTimer<RowData, Long> timer) throws 
Exception {
+        // Window join only support event-time now
+        throw new UnsupportedOperationException(
+                "This is a bug and should not happen. Please file an issue.");
+    }
+
+    @Override
+    public void onEventTime(InternalTimer<RowData, Long> timer) throws 
Exception {
+        asyncProcessWithKey(timer.getKey(), () -> 
triggerJoin(timer.getNamespace()));
+    }
+
+    /**
+     * Currently, similar to the {@link WindowJoinOperator#onEventTime} that 
uses the sync state
+     * api, we directly load the list data from the state into memory to 
perform join operations.
+     *
+     * <p>Note: The order of data in the left and right side lists must be 
preserved to ensure the
+     * output data sequence is maintained.
+     */
+    private void triggerJoin(long window) {
+        StateFuture<StateIterator<RowData>> leftDataFuture = 
leftWindowState.asyncGet(window);
+        StateFuture<StateIterator<RowData>> rightDataFuture = 
rightWindowState.asyncGet(window);
+
+        // join left records and right records
+        AtomicReference<List<RowData>> leftDataRef = new AtomicReference<>(new 
ArrayList<>());
+        AtomicReference<List<RowData>> rightDataRef = new 
AtomicReference<>(new ArrayList<>());
+        leftDataFuture.thenCombine(
+                rightDataFuture,
+                (leftDataIterator, rightDataIterator) -> {
+                    StateFuture<Void> leftLoadToMemFuture =
+                            leftDataIterator.onNext(
+                                    data -> {
+                                        leftDataRef.get().add(data);
+                                    });
+                    StateFuture<Void> rightLoadToMemFuture =
+                            rightDataIterator.onNext(
+                                    data -> {
+                                        rightDataRef.get().add(data);
+                                    });
+                    return leftLoadToMemFuture.thenCombine(
+                            rightLoadToMemFuture,
+                            (VOID1, VOID2) -> {
+                                helper.joinAndClear(window, leftDataRef.get(), 
rightDataRef.get());
+                                return null;
+                            });
+                });
+    }
+
+    private class AsyncStateWindowJoinHelper extends WindowJoinHelper {
+
+        public AsyncStateWindowJoinHelper() {
+            super(
+                    AsyncStateWindowJoinOperator.this.leftSerializer,
+                    AsyncStateWindowJoinOperator.this.rightSerializer,
+                    AsyncStateWindowJoinOperator.this.shiftTimeZone,
+                    AsyncStateWindowJoinOperator.this.windowTimerService,
+                    AsyncStateWindowJoinOperator.this.joinCondition,
+                    AsyncStateWindowJoinOperator.this.leftWindowEndIndex,
+                    AsyncStateWindowJoinOperator.this.rightWindowEndIndex,
+                    AsyncStateWindowJoinOperator.this.collector,
+                    AsyncStateWindowJoinOperator.this.joinType,
+                    AsyncStateWindowJoinOperator.this.metrics);
+        }
+
+        @Override
+        public void accToState(long windowEnd, RowData rowData, boolean 
isLeft) throws Exception {
+            // no need to wait these async requests to end
+            if (isLeft) {
+                leftWindowState.asyncAdd(windowEnd, rowData);
+            } else {
+                rightWindowState.asyncAdd(windowEnd, rowData);
+            }
+        }
+
+        @Override
+        public void clearState(long windowEnd, boolean isLeft) throws 
Exception {

Review Comment:
   Ditto



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/asyncprocessing/AsyncStateWindowJoinOperator.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.join.window.asyncprocessing;
+
+import org.apache.flink.api.common.functions.DefaultOpenContext;
+import org.apache.flink.api.common.state.v2.ListState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.state.v2.StateIterator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.runtime.state.v2.ListStateDescriptor;
+import org.apache.flink.runtime.state.v2.internal.InternalListState;
+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.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.operators.AsyncStateTableStreamOperator;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import 
org.apache.flink.table.runtime.operators.join.JoinConditionWithNullFilters;
+import org.apache.flink.table.runtime.operators.join.window.WindowJoinOperator;
+import 
org.apache.flink.table.runtime.operators.join.window.utils.WindowJoinHelper;
+import 
org.apache.flink.table.runtime.operators.window.tvf.asyncprocessing.state.WindowListAsyncState;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService;
+import 
org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowTimerServiceImpl;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/** A {@link WindowJoinOperator} implemented by async state api. */
+public class AsyncStateWindowJoinOperator extends 
AsyncStateTableStreamOperator<RowData>
+        implements TwoInputStreamOperator<RowData, RowData, RowData>,
+                Triggerable<RowData, Long>,
+                KeyContext {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final String LEFT_RECORDS_STATE_NAME = "left-records";
+    private static final String RIGHT_RECORDS_STATE_NAME = "right-records";
+
+    private final RowDataSerializer leftSerializer;
+    private final RowDataSerializer rightSerializer;
+    private final GeneratedJoinCondition generatedJoinCondition;
+
+    private final int leftWindowEndIndex;
+    private final int rightWindowEndIndex;
+
+    private final boolean[] filterNullKeys;
+    private final ZoneId shiftTimeZone;
+
+    private final FlinkJoinType joinType;
+
+    private transient WindowTimerService<Long> windowTimerService;
+
+    // ------------------------------------------------------------------------
+    private transient JoinConditionWithNullFilters joinCondition;
+
+    /** This is used for emitting elements with a given timestamp. */
+    private transient TimestampedCollector<RowData> collector;
+
+    private transient WindowListAsyncState<Long> leftWindowState;
+    private transient WindowListAsyncState<Long> rightWindowState;
+
+    private transient WindowJoinHelper helper;
+
+    public AsyncStateWindowJoinOperator(
+            TypeSerializer<RowData> leftSerializer,
+            TypeSerializer<RowData> rightSerializer,
+            GeneratedJoinCondition generatedJoinCondition,
+            int leftWindowEndIndex,
+            int rightWindowEndIndex,
+            boolean[] filterNullKeys,
+            ZoneId shiftTimeZone,
+            FlinkJoinType joinType) {
+        this.leftSerializer = (RowDataSerializer) leftSerializer;
+        this.rightSerializer = (RowDataSerializer) rightSerializer;
+        this.generatedJoinCondition = generatedJoinCondition;
+        this.leftWindowEndIndex = leftWindowEndIndex;
+        this.rightWindowEndIndex = rightWindowEndIndex;
+        this.filterNullKeys = filterNullKeys;
+        this.shiftTimeZone = shiftTimeZone;
+        this.joinType = joinType;
+    }
+
+    @Override
+    public void open() throws Exception {
+        super.open();
+
+        this.collector = new TimestampedCollector<>(output);
+        collector.eraseTimestamp();
+
+        final LongSerializer windowSerializer = LongSerializer.INSTANCE;
+
+        InternalTimerService<Long> internalTimerService =
+                getInternalTimerService("window-timers", windowSerializer, 
this);
+        this.windowTimerService =
+                new SlicingWindowTimerServiceImpl(internalTimerService, 
shiftTimeZone);
+
+        // init join condition
+        JoinCondition condition =
+                
generatedJoinCondition.newInstance(getRuntimeContext().getUserCodeClassLoader());
+        this.joinCondition = new JoinConditionWithNullFilters(condition, 
filterNullKeys, this);
+        this.joinCondition.setRuntimeContext(getRuntimeContext());
+        this.joinCondition.open(DefaultOpenContext.INSTANCE);
+
+        // init state
+        ListStateDescriptor<RowData> leftRecordStateDesc =
+                new ListStateDescriptor<>(LEFT_RECORDS_STATE_NAME, 
leftSerializer);
+        ListState<RowData> leftListState =
+                getOrCreateKeyedState(Long.MIN_VALUE, windowSerializer, 
leftRecordStateDesc);
+        this.leftWindowState =
+                new WindowListAsyncState<>(
+                        (InternalListState<RowData, Long, RowData>) 
leftListState);
+
+        ListStateDescriptor<RowData> rightRecordStateDesc =
+                new ListStateDescriptor<>(RIGHT_RECORDS_STATE_NAME, 
rightSerializer);
+        ListState<RowData> rightListState =
+                getOrCreateKeyedState(Long.MIN_VALUE, windowSerializer, 
rightRecordStateDesc);
+        this.rightWindowState =
+                new WindowListAsyncState<>(
+                        (InternalListState<RowData, Long, RowData>) 
rightListState);
+
+        this.helper = new AsyncStateWindowJoinHelper();
+    }
+
+    @Override
+    public void close() throws Exception {
+        super.close();
+        collector = null;
+        if (joinCondition != null) {
+            joinCondition.close();
+        }
+    }
+
+    @Override
+    public void processElement1(StreamRecord<RowData> element) throws 
Exception {
+        helper.processElement(element.getValue(), true);
+    }
+
+    @Override
+    public void processElement2(StreamRecord<RowData> element) throws 
Exception {
+        helper.processElement(element.getValue(), false);
+    }
+
+    @Override
+    public void onProcessingTime(InternalTimer<RowData, Long> timer) throws 
Exception {
+        // Window join only support event-time now
+        throw new UnsupportedOperationException(
+                "This is a bug and should not happen. Please file an issue.");
+    }
+
+    @Override
+    public void onEventTime(InternalTimer<RowData, Long> timer) throws 
Exception {
+        asyncProcessWithKey(timer.getKey(), () -> 
triggerJoin(timer.getNamespace()));
+    }
+
+    /**
+     * Currently, similar to the {@link WindowJoinOperator#onEventTime} that 
uses the sync state
+     * api, we directly load the list data from the state into memory to 
perform join operations.
+     *
+     * <p>Note: The order of data in the left and right side lists must be 
preserved to ensure the
+     * output data sequence is maintained.
+     */
+    private void triggerJoin(long window) {
+        StateFuture<StateIterator<RowData>> leftDataFuture = 
leftWindowState.asyncGet(window);
+        StateFuture<StateIterator<RowData>> rightDataFuture = 
rightWindowState.asyncGet(window);
+
+        // join left records and right records
+        AtomicReference<List<RowData>> leftDataRef = new AtomicReference<>(new 
ArrayList<>());
+        AtomicReference<List<RowData>> rightDataRef = new 
AtomicReference<>(new ArrayList<>());
+        leftDataFuture.thenCombine(
+                rightDataFuture,
+                (leftDataIterator, rightDataIterator) -> {
+                    StateFuture<Void> leftLoadToMemFuture =
+                            leftDataIterator.onNext(
+                                    data -> {
+                                        leftDataRef.get().add(data);
+                                    });
+                    StateFuture<Void> rightLoadToMemFuture =
+                            rightDataIterator.onNext(
+                                    data -> {
+                                        rightDataRef.get().add(data);
+                                    });
+                    return leftLoadToMemFuture.thenCombine(
+                            rightLoadToMemFuture,
+                            (VOID1, VOID2) -> {
+                                helper.joinAndClear(window, leftDataRef.get(), 
rightDataRef.get());
+                                return null;
+                            });
+                });
+    }
+
+    private class AsyncStateWindowJoinHelper extends WindowJoinHelper {
+
+        public AsyncStateWindowJoinHelper() {
+            super(
+                    AsyncStateWindowJoinOperator.this.leftSerializer,
+                    AsyncStateWindowJoinOperator.this.rightSerializer,
+                    AsyncStateWindowJoinOperator.this.shiftTimeZone,
+                    AsyncStateWindowJoinOperator.this.windowTimerService,
+                    AsyncStateWindowJoinOperator.this.joinCondition,
+                    AsyncStateWindowJoinOperator.this.leftWindowEndIndex,
+                    AsyncStateWindowJoinOperator.this.rightWindowEndIndex,
+                    AsyncStateWindowJoinOperator.this.collector,
+                    AsyncStateWindowJoinOperator.this.joinType,
+                    AsyncStateWindowJoinOperator.this.metrics);
+        }
+
+        @Override
+        public void accToState(long windowEnd, RowData rowData, boolean 
isLeft) throws Exception {

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



##########
flink-runtime/src/test/java/org/apache/flink/streaming/util/asyncprocessing/AsyncKeyedTwoInputStreamOperatorTestHarness.java:
##########
@@ -215,6 +217,28 @@ public void close() throws Exception {
         executor.shutdown();
     }
 
+    @Override
+    public int numKeyedStateEntries() {
+        AbstractAsyncStateStreamOperator<OUT> asyncOp =
+                (AbstractAsyncStateStreamOperator<OUT>) operator;
+        AsyncKeyedStateBackend<Object> asyncKeyedStateBackend = 
asyncOp.getAsyncKeyedStateBackend();
+        KeyedStateBackend<?> keyedStateBackend;
+        if (asyncKeyedStateBackend instanceof AsyncKeyedStateBackendAdaptor) {
+            keyedStateBackend =
+                    ((AsyncKeyedStateBackendAdaptor<?>) asyncKeyedStateBackend)
+                            .getKeyedStateBackend();
+
+        } else {
+            throw new UnsupportedOperationException();
+        }
+
+        if (keyedStateBackend instanceof HeapKeyedStateBackend) {
+            return ((HeapKeyedStateBackend) 
keyedStateBackend).numKeyValueStateEntries();
+        } else {
+            throw new UnsupportedOperationException();

Review Comment:
   Ditto



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/asyncprocessing/AsyncStateWindowJoinOperator.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.join.window.asyncprocessing;
+
+import org.apache.flink.api.common.functions.DefaultOpenContext;
+import org.apache.flink.api.common.state.v2.ListState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.state.v2.StateIterator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.runtime.state.v2.ListStateDescriptor;
+import org.apache.flink.runtime.state.v2.internal.InternalListState;
+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.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.operators.AsyncStateTableStreamOperator;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import 
org.apache.flink.table.runtime.operators.join.JoinConditionWithNullFilters;
+import org.apache.flink.table.runtime.operators.join.window.WindowJoinOperator;
+import 
org.apache.flink.table.runtime.operators.join.window.utils.WindowJoinHelper;
+import 
org.apache.flink.table.runtime.operators.window.tvf.asyncprocessing.state.WindowListAsyncState;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService;
+import 
org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowTimerServiceImpl;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/** A {@link WindowJoinOperator} implemented by async state api. */
+public class AsyncStateWindowJoinOperator extends 
AsyncStateTableStreamOperator<RowData>
+        implements TwoInputStreamOperator<RowData, RowData, RowData>,
+                Triggerable<RowData, Long>,
+                KeyContext {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final String LEFT_RECORDS_STATE_NAME = "left-records";
+    private static final String RIGHT_RECORDS_STATE_NAME = "right-records";
+
+    private final RowDataSerializer leftSerializer;
+    private final RowDataSerializer rightSerializer;
+    private final GeneratedJoinCondition generatedJoinCondition;
+
+    private final int leftWindowEndIndex;
+    private final int rightWindowEndIndex;
+
+    private final boolean[] filterNullKeys;
+    private final ZoneId shiftTimeZone;
+
+    private final FlinkJoinType joinType;
+
+    private transient WindowTimerService<Long> windowTimerService;
+
+    // ------------------------------------------------------------------------

Review Comment:
   Nit: remove unnecessary comment.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/WindowJoinOperator.java:
##########
@@ -63,27 +54,18 @@
  *
  * <p>Note: currently, {@link WindowJoinOperator} doesn't support DELETE or 
UPDATE_BEFORE input row.
  */
-public abstract class WindowJoinOperator extends TableStreamOperator<RowData>

Review Comment:
   Nit: Coud we highlight sync state api  in the comment? Btw, introduce an 
async state API version of this operator that implements the same functionality.



##########
flink-runtime/src/test/java/org/apache/flink/streaming/util/asyncprocessing/AsyncKeyedTwoInputStreamOperatorTestHarness.java:
##########
@@ -215,6 +217,28 @@ public void close() throws Exception {
         executor.shutdown();
     }
 
+    @Override
+    public int numKeyedStateEntries() {
+        AbstractAsyncStateStreamOperator<OUT> asyncOp =
+                (AbstractAsyncStateStreamOperator<OUT>) operator;
+        AsyncKeyedStateBackend<Object> asyncKeyedStateBackend = 
asyncOp.getAsyncKeyedStateBackend();
+        KeyedStateBackend<?> keyedStateBackend;
+        if (asyncKeyedStateBackend instanceof AsyncKeyedStateBackendAdaptor) {
+            keyedStateBackend =
+                    ((AsyncKeyedStateBackendAdaptor<?>) asyncKeyedStateBackend)
+                            .getKeyedStateBackend();
+
+        } else {
+            throw new UnsupportedOperationException();

Review Comment:
   Could we add a meaningful error message to the exception? This would help 
with debugging and give users more clarity when something goes wrong. 



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/asyncprocessing/AsyncStateWindowJoinOperator.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.join.window.asyncprocessing;
+
+import org.apache.flink.api.common.functions.DefaultOpenContext;
+import org.apache.flink.api.common.state.v2.ListState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.state.v2.StateIterator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.runtime.state.v2.ListStateDescriptor;
+import org.apache.flink.runtime.state.v2.internal.InternalListState;
+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.TimestampedCollector;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.operators.AsyncStateTableStreamOperator;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import 
org.apache.flink.table.runtime.operators.join.JoinConditionWithNullFilters;
+import org.apache.flink.table.runtime.operators.join.window.WindowJoinOperator;
+import 
org.apache.flink.table.runtime.operators.join.window.utils.WindowJoinHelper;
+import 
org.apache.flink.table.runtime.operators.window.tvf.asyncprocessing.state.WindowListAsyncState;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService;
+import 
org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowTimerServiceImpl;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+/** A {@link WindowJoinOperator} implemented by async state api. */

Review Comment:
   Nit: Could we add more detailed comments about this operator? While it is 
essentially an async state API version of the WindowJoinOperator, it would be 
helpful for newcomers to understand its purpose and functionality directly from 
this file. 



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/WindowJoinOperator.java:
##########
@@ -92,26 +74,20 @@ public abstract class WindowJoinOperator extends 
TableStreamOperator<RowData>
     private final boolean[] filterNullKeys;
     private final ZoneId shiftTimeZone;
 
+    private final FlinkJoinType joinType;
+
     private transient WindowTimerService<Long> windowTimerService;
 
     // ------------------------------------------------------------------------

Review Comment:
   Nit: remove unnecessary comment.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/utils/WindowJoinHelper.java:
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.join.window.utils;
+
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Meter;
+import org.apache.flink.metrics.MeterView;
+import org.apache.flink.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.RowDataUtil;
+import org.apache.flink.table.data.utils.JoinedRowData;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import 
org.apache.flink.table.runtime.operators.join.JoinConditionWithNullFilters;
+import org.apache.flink.table.runtime.operators.join.window.WindowJoinOperator;
+import 
org.apache.flink.table.runtime.operators.join.window.asyncprocessing.AsyncStateWindowJoinOperator;
+import 
org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.types.RowKind;
+
+import javax.annotation.Nullable;
+
+import java.time.ZoneId;
+import java.util.IdentityHashMap;
+
+import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired;
+
+/**
+ * A helper to do the window join operations for {@link WindowJoinOperator} 
and {@link
+ * AsyncStateWindowJoinOperator}.
+ */
+public abstract class WindowJoinHelper {
+
+    private static final String LEFT_LATE_ELEMENTS_DROPPED_METRIC_NAME =
+            "leftNumLateRecordsDropped";
+    private static final String LEFT_LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME =
+            "leftLateRecordsDroppedRate";
+    private static final String RIGHT_LATE_ELEMENTS_DROPPED_METRIC_NAME =
+            "rightNumLateRecordsDropped";
+    private static final String RIGHT_LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME =
+            "rightLateRecordsDroppedRate";
+    private static final String WATERMARK_LATENCY_METRIC_NAME = 
"watermarkLatency";
+
+    private final ZoneId shiftTimeZone;
+
+    private final WindowTimerService<Long> windowTimerService;
+
+    protected final RowDataSerializer leftSerializer;
+
+    protected final RowDataSerializer rightSerializer;
+
+    protected final JoinConditionWithNullFilters joinCondition;
+
+    private final int leftWindowEndIndex;
+    private final int rightWindowEndIndex;
+
+    /** This is used for emitting elements with a given timestamp. */
+    protected final TimestampedCollector<RowData> collector;
+
+    private final WindowJoinProcessor windowJoinProcessor;
+
+    // ------------------------------------------------------------------------
+    // Metrics
+    // ------------------------------------------------------------------------
+
+    private final Meter leftLateRecordsDroppedRate;
+    private final Meter rightLateRecordsDroppedRate;
+
+    public WindowJoinHelper(
+            RowDataSerializer leftSerializer,
+            RowDataSerializer rightSerializer,
+            ZoneId shiftTimeZone,
+            WindowTimerService<Long> windowTimerService,
+            JoinConditionWithNullFilters joinCondition,
+            int leftWindowEndIndex,
+            int rightWindowEndIndex,
+            TimestampedCollector<RowData> collector,
+            FlinkJoinType joinType,
+            OperatorMetricGroup metrics) {
+        this.leftSerializer = leftSerializer;
+        this.rightSerializer = rightSerializer;
+        this.shiftTimeZone = shiftTimeZone;
+        this.windowTimerService = windowTimerService;
+        this.joinCondition = joinCondition;
+        this.leftWindowEndIndex = leftWindowEndIndex;
+        this.rightWindowEndIndex = rightWindowEndIndex;
+        this.collector = collector;
+
+        this.windowJoinProcessor = getWindowJoinProcessor(joinType);
+
+        // register metrics
+        Counter leftNumLateRecordsDropped = 
metrics.counter(LEFT_LATE_ELEMENTS_DROPPED_METRIC_NAME);
+        this.leftLateRecordsDroppedRate =
+                metrics.meter(
+                        LEFT_LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME,
+                        new MeterView(leftNumLateRecordsDropped));
+        Counter rightNumLateRecordsDropped =
+                metrics.counter(RIGHT_LATE_ELEMENTS_DROPPED_METRIC_NAME);
+        this.rightLateRecordsDroppedRate =
+                metrics.meter(
+                        RIGHT_LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME,
+                        new MeterView(rightNumLateRecordsDropped));
+        metrics.gauge(
+                WATERMARK_LATENCY_METRIC_NAME,
+                () -> {
+                    long watermark = windowTimerService.currentWatermark();
+                    if (watermark < 0) {
+                        return 0L;
+                    } else {
+                        return windowTimerService.currentProcessingTime() - 
watermark;
+                    }
+                });
+    }
+
+    public void processElement(RowData inputRow, boolean isLeft) throws 
Exception {
+        Meter lateRecordsDroppedRate;
+        int windowEndIndex;
+        if (isLeft) {
+            windowEndIndex = leftWindowEndIndex;
+            lateRecordsDroppedRate = leftLateRecordsDroppedRate;
+        } else {
+            windowEndIndex = rightWindowEndIndex;
+            lateRecordsDroppedRate = rightLateRecordsDroppedRate;
+        }
+
+        long windowEnd = inputRow.getLong(windowEndIndex);
+        if (isWindowFired(windowEnd, windowTimerService.currentWatermark(), 
shiftTimeZone)) {
+            // element is late and should be dropped
+            lateRecordsDroppedRate.markEvent();
+            return;
+        }
+        if (RowDataUtil.isAccumulateMsg(inputRow)) {
+            accToState(windowEnd, inputRow, isLeft);
+        } else {
+            // Window join could not handle retraction input stream
+            throw new UnsupportedOperationException(
+                    "This is a bug and should not happen. Please file an 
issue.");
+        }
+        // always register time for every element
+        windowTimerService.registerEventTimeWindowTimer(windowEnd);
+    }
+
+    public void joinAndClear(
+            long windowEnd,
+            @Nullable Iterable<RowData> leftRecords,
+            @Nullable Iterable<RowData> rightRecords)
+            throws Exception {
+        windowJoinProcessor.doJoin(leftRecords, rightRecords);
+        // clear state
+        if (leftRecords != null) {
+            clearState(windowEnd, true);
+        }
+        if (rightRecords != null) {
+            clearState(windowEnd, false);
+        }
+    }
+
+    public abstract void accToState(long windowEnd, RowData rowData, boolean 
isLeft)
+            throws Exception;
+
+    public abstract void clearState(long windowEnd, boolean isLeft) throws 
Exception;
+
+    private WindowJoinProcessor getWindowJoinProcessor(FlinkJoinType joinType) 
{
+        switch (joinType) {
+            case INNER:
+                return new InnerWindowJoinProcessor();
+            case SEMI:
+                return new SemiAntiWindowJoinProcessor(false);
+            case ANTI:
+                return new SemiAntiWindowJoinProcessor(true);
+            case LEFT:
+                return new LeftOuterWindowJoinProcessor();
+            case RIGHT:
+                return new RightOuterWindowJoinProcessor();
+            case FULL:
+                return new FullOuterWindowJoinProcessor();
+            default:
+                throw new IllegalArgumentException("Invalid join type: " + 
joinType);
+        }
+    }
+
+    private interface WindowJoinProcessor {
+
+        void doJoin(
+                @Nullable Iterable<RowData> leftRecords, @Nullable 
Iterable<RowData> rightRecords);
+    }
+
+    private class SemiAntiWindowJoinProcessor implements WindowJoinProcessor {
+
+        private final boolean isAntiJoin;
+
+        public SemiAntiWindowJoinProcessor(boolean isAntiJoin) {
+            this.isAntiJoin = isAntiJoin;
+        }
+
+        @Override
+        public void doJoin(
+                @Nullable Iterable<RowData> leftRecords, @Nullable 
Iterable<RowData> rightRecords) {
+            if (leftRecords == null) {
+                return;
+            }
+            if (rightRecords == null) {
+                if (isAntiJoin) {
+                    for (RowData leftRecord : leftRecords) {
+                        collector.collect(leftRecord);
+                    }
+                }
+                return;
+            }
+            for (RowData leftRecord : leftRecords) {
+                boolean matches = false;
+                for (RowData rightRecord : rightRecords) {
+                    if (joinCondition.apply(leftRecord, rightRecord)) {
+                        matches = true;
+                        break;
+                    }
+                }
+                if (matches) {
+                    if (!isAntiJoin) {
+                        // emit left record if there are matched rows on the 
other side
+                        collector.collect(leftRecord);
+                    }
+                } else {
+                    if (isAntiJoin) {
+                        // emit left record if there is no matched row on the 
other side
+                        collector.collect(leftRecord);
+                    }
+                }
+            }
+        }
+    }
+
+    private class InnerWindowJoinProcessor implements WindowJoinProcessor {
+
+        private final JoinedRowData outRow = new JoinedRowData();
+
+        @Override
+        public void doJoin(
+                @Nullable Iterable<RowData> leftRecords, @Nullable 
Iterable<RowData> rightRecords) {
+            if (leftRecords == null || rightRecords == null) {
+                return;
+            }
+            for (RowData leftRecord : leftRecords) {
+                for (RowData rightRecord : rightRecords) {
+                    if (joinCondition.apply(leftRecord, rightRecord)) {
+                        outRow.setRowKind(RowKind.INSERT);
+                        outRow.replace(leftRecord, rightRecord);
+                        collector.collect(outRow);
+                    }
+                }
+            }
+        }
+    }
+
+    private abstract class AbstractOuterWindowJoinProcessor implements 
WindowJoinProcessor {
+
+        private final RowData leftNullRow = new 
GenericRowData(leftSerializer.getArity());
+        private final RowData rightNullRow = new 
GenericRowData(rightSerializer.getArity());
+        private final JoinedRowData outRow = new JoinedRowData();
+
+        protected void outputNullPadding(RowData row, boolean isLeft) {
+            if (isLeft) {
+                outRow.replace(row, rightNullRow);
+            } else {
+                outRow.replace(leftNullRow, row);
+            }
+            outRow.setRowKind(RowKind.INSERT);
+            collector.collect(outRow);
+        }
+
+        protected void outputNullPadding(Iterable<RowData> rows, boolean 
isLeft) {
+            for (RowData row : rows) {
+                outputNullPadding(row, isLeft);
+            }
+        }
+
+        protected void output(RowData inputRow, RowData otherRow, boolean 
inputIsLeft) {
+            if (inputIsLeft) {
+                outRow.replace(inputRow, otherRow);
+            } else {
+                outRow.replace(otherRow, inputRow);
+            }
+            outRow.setRowKind(RowKind.INSERT);
+            collector.collect(outRow);
+        }
+    }
+
+    private class LeftOuterWindowJoinProcessor extends 
AbstractOuterWindowJoinProcessor {
+
+        @Override
+        public void doJoin(

Review Comment:
   Nit: The logic in this method largely overlaps with the logic in 
RightOuterWindowJoinProcessor#doJoin.  Could we abstract the common 
functionality into a separate utility function? 



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