Zakelly commented on code in PR #26328:
URL: https://github.com/apache/flink/pull/26328#discussion_r2020796756


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/asyncprocess/AsyncProcTimeIntervalJoin.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.interval.asyncprocess;
+
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import 
org.apache.flink.table.runtime.operators.join.interval.IntervalJoinFunction;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+
+/** The function to execute processing time interval stream inner-join. */
+public final class AsyncProcTimeIntervalJoin extends AsyncTimeIntervalJoin {
+
+    private static final long serialVersionUID = 9204647938032023101L;

Review Comment:
   We could give another random number, different from the one for 
`ProcTimeIntervalJoin`.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/asyncprocess/AsyncTimeIntervalJoin.java:
##########
@@ -0,0 +1,347 @@
+/*
+ * 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.interval.asyncprocess;
+
+import org.apache.flink.api.common.functions.DefaultOpenContext;
+import org.apache.flink.api.common.functions.OpenContext;
+import org.apache.flink.api.common.state.v2.MapState;
+import org.apache.flink.api.common.state.v2.MapStateDescriptor;
+import org.apache.flink.api.common.state.v2.ValueState;
+import org.apache.flink.api.common.state.v2.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.ListTypeInfo;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.streaming.api.functions.co.KeyedCoProcessFunction;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.OuterJoinPaddingUtil;
+import 
org.apache.flink.table.runtime.operators.join.interval.EmitAwareCollector;
+import 
org.apache.flink.table.runtime.operators.join.interval.IntervalJoinFunction;
+import 
org.apache.flink.table.runtime.operators.join.interval.TimeIntervalJoinBase;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.util.Collector;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+/** The TimeIntervalJoinBase with async state. */
+public abstract class AsyncTimeIntervalJoin extends TimeIntervalJoinBase {
+    // cache to store rows form the left stream
+    private transient MapState<Long, List<Tuple2<RowData, Boolean>>> leftCache;
+    // cache to store rows from the right stream
+    private transient MapState<Long, List<Tuple2<RowData, Boolean>>> 
rightCache;
+
+    // state to record the timer on the left stream. 0 means no timer set
+    private transient ValueState<Long> leftTimerState;
+    // state to record the timer on the right stream. 0 means no timer set
+    private transient ValueState<Long> rightTimerState;
+
+    protected AsyncTimeIntervalJoin(
+            FlinkJoinType joinType,
+            long leftLowerBound,
+            long leftUpperBound,
+            long allowedLateness,
+            long minCleanUpInterval,
+            InternalTypeInfo<RowData> leftType,
+            InternalTypeInfo<RowData> rightType,
+            IntervalJoinFunction joinFunc) {
+        super(
+                joinType,
+                leftLowerBound,
+                leftUpperBound,
+                allowedLateness,
+                minCleanUpInterval,
+                leftType,
+                rightType,
+                joinFunc);
+    }
+
+    @Override
+    public void open(OpenContext openContext) throws Exception {
+        joinFunction.setRuntimeContext(getRuntimeContext());
+        joinFunction.open(DefaultOpenContext.INSTANCE);
+        joinCollector = new EmitAwareCollector();
+
+        // Initialize the data caches.
+        ListTypeInfo<Tuple2<RowData, Boolean>> leftRowListTypeInfo =
+                new ListTypeInfo<>(new TupleTypeInfo<>(leftType, 
BasicTypeInfo.BOOLEAN_TYPE_INFO));
+        MapStateDescriptor<Long, List<Tuple2<RowData, Boolean>>> 
leftMapStateDescriptor =
+                new MapStateDescriptor<>(
+                        "IntervalJoinLeftCache", BasicTypeInfo.LONG_TYPE_INFO, 
leftRowListTypeInfo);
+        leftCache = getRuntimeContext().getMapState(leftMapStateDescriptor);
+
+        ListTypeInfo<Tuple2<RowData, Boolean>> rightRowListTypeInfo =
+                new ListTypeInfo<>(new TupleTypeInfo<>(rightType, 
BasicTypeInfo.BOOLEAN_TYPE_INFO));
+        MapStateDescriptor<Long, List<Tuple2<RowData, Boolean>>> 
rightMapStateDescriptor =
+                new MapStateDescriptor<>(
+                        "IntervalJoinRightCache",
+                        BasicTypeInfo.LONG_TYPE_INFO,
+                        rightRowListTypeInfo);
+        rightCache = getRuntimeContext().getMapState(rightMapStateDescriptor);
+
+        // Initialize the timer states.
+        ValueStateDescriptor<Long> leftValueStateDescriptor =
+                new ValueStateDescriptor<>("IntervalJoinLeftTimerState", 
Long.class);
+        leftTimerState = 
getRuntimeContext().getState(leftValueStateDescriptor);
+
+        ValueStateDescriptor<Long> rightValueStateDescriptor =
+                new ValueStateDescriptor<>("IntervalJoinRightTimerState", 
Long.class);
+        rightTimerState = 
getRuntimeContext().getState(rightValueStateDescriptor);
+
+        paddingUtil = new OuterJoinPaddingUtil(leftType.toRowSize(), 
rightType.toRowSize());
+    }
+
+    @Override
+    public void onTimer(long timestamp, OnTimerContext ctx, Collector<RowData> 
out)
+            throws Exception {
+        joinFunction.setJoinKey(ctx.getCurrentKey());
+        joinCollector.setInnerCollector(out);
+        updateOperatorTime(ctx);
+        // In the future, we should separate the left and right watermarks. 
Otherwise, the
+        // registered timer of the faster stream will be delayed, even if the 
watermarks have
+        // already been emitted by the source.
+        leftTimerState
+                .asyncValue()
+                .thenAccept(
+                        leftCleanUpTime -> {
+                            if (leftCleanUpTime != null && timestamp == 
leftCleanUpTime) {
+                                rightExpirationTime =
+                                        calExpirationTime(leftOperatorTime, 
rightRelativeSize);
+                                removeExpiredRows(
+                                        rightExpirationTime,
+                                        rightCache,
+                                        leftTimerState,
+                                        ctx,
+                                        false);
+                            }
+                        });
+
+        rightTimerState
+                .asyncValue()
+                .thenAccept(
+                        rightCleanUpTime -> {
+                            if (rightCleanUpTime != null && timestamp == 
rightCleanUpTime) {
+                                leftExpirationTime =
+                                        calExpirationTime(rightOperatorTime, 
leftRelativeSize);
+                                removeExpiredRows(
+                                        leftExpirationTime, leftCache, 
rightTimerState, ctx, true);
+                            }
+                        });
+    }
+
+    @Override
+    protected boolean iterateRightCache(
+            RowData leftRow, long rightQualifiedLowerBound, long 
rightQualifiedUpperBound)
+            throws Exception {
+        Set<Long> toRemove = new HashSet<>();
+        final AtomicBoolean emitted = new AtomicBoolean(false);
+        CompletableFuture<Boolean> emitResult = new CompletableFuture<>();
+
+        Iterator<Map.Entry<Long, List<Tuple2<RowData, Boolean>>>> 
rightIterator =
+                rightCache.iterator();
+        rightCache
+                .asyncEntries()
+                .thenCompose(
+                        iter ->
+                                iter.onNext(
+                                        entry -> {
+                                            boolean entryEmitted =
+                                                    tryMatchLeftRowsByTime(
+                                                            leftRow,
+                                                            
rightIterator.next(),
+                                                            
rightQualifiedLowerBound,
+                                                            
rightQualifiedUpperBound,
+                                                            toRemove);
+                                            if (entryEmitted) {
+                                                emitted.set(true);
+                                            }
+                                        }))
+                .thenAccept(
+                        V -> {
+                            emitResult.complete(emitted.get());
+                            for (Long key : toRemove) {
+                                rightCache.asyncRemove(key);
+                            }
+                        });
+        return emitResult.get();

Review Comment:
   We better not blocking `get()` here. It will block the task thread as well 
as those callbacks for async state requests.



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