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


##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AsyncLookupJoinITCase.scala:
##########
@@ -133,17 +158,39 @@ class AsyncLookupJoinITCase(
     val dataId = TestValuesTableFactory.registerData(data)
     tEnv.executeSql(s"""
                        |CREATE TABLE $tableName (
-                       |  `id` BIGINT,
+                       |  `id` BIGINT PRIMARY KEY NOT ENFORCED,
                        |  `len` INT,
                        |  `content` STRING,
                        |  `proctime` AS PROCTIME()
                        |) WITH (
                        |  'connector' = 'values',
-                       |  'data-id' = '$dataId'
+                       |  'data-id' = '$dataId',
+                       |  'changelog-mode' = 'I,UA,UB,D'

Review Comment:
   Good catch. This is invalid.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java:
##########
@@ -522,7 +523,11 @@ protected StreamOperatorFactory<RowData> 
createAsyncLookupJoin(
                             asyncLookupOptions.asyncBufferCapacity);
         }
         if (asyncLookupOptions.keyOrdered) {
-            throw new UnsupportedOperationException("No proper operator is 
supported currently.");
+            return new TableKeyedAsyncWaitOperatorFactory<>(

Review Comment:
   In fact, we cannot treat the actual runtime judgment result AsyncOutputMode 
= UNORDERED as a precondition for keyOrdered, but instead should use the 
configuration AsyncOutputMode = allow_unordered as the precondition.
   
   Consider this case:
   `CREATE VIEW v1 AS SELECT a, b FROM AppendOnlySrc GROUP BY c;
   SELECT * FROM v1 LEFT JOIN lookupTable FOR SYSTEM_TIME AS OF {equal 
condition};
   `
   In reality, the AsyncOutputMode for the lookup join is ORDERED, because the 
aggregation generates a non-append-only stream. However, in practice, this does 
not affect the enabling of keyOrdered.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/TableKeyedAsyncWaitOperator.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.functions.async.CollectionSupplier;
+import org.apache.flink.streaming.api.functions.async.ResultFuture;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import 
org.apache.flink.streaming.api.operators.async.queue.StreamRecordQueueEntry;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import 
org.apache.flink.table.runtime.operators.join.lookup.keyordered.AecRecord;
+import 
org.apache.flink.table.runtime.operators.join.lookup.keyordered.TableAsyncExecutionController;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+/**
+ * This operator enables key-ordered processing of records in a table.
+ *
+ * <p>If the upsertKey is inferred by planner, this key will be used to 
determine the order of
+ * records during processing. If not, the entire record will serve as the 
fallback key for ordering
+ * purposes.
+ *
+ * @param <IN> Input type for the operator.
+ * @param <OUT> Output type for the operator.
+ * @param <KEY> Key type for the operator.
+ */
+public class TableKeyedAsyncWaitOperator<IN, OUT, KEY>
+        extends AbstractUdfStreamOperator<OUT, AsyncFunction<IN, OUT>>
+        implements OneInputStreamOperator<IN, OUT>, BoundedOneInput {
+
+    protected static final Logger LOG = 
LoggerFactory.getLogger(TableKeyedAsyncWaitOperator.class);
+
+    private static final long serialVersionUID = 1L;
+
+    private static final String STATE_NAME = 
"_keyed_async_wait_operator_state_";
+
+    /** Selector to get upsert key from input record. */
+    private final KeySelector<IN, KEY> keySelector;
+
+    /** Timeout for the async collectors. */
+    private final long timeout;
+
+    /** Max number of inflight invocation. */
+    private final transient int capacity;
+
+    /** {@link TypeSerializer} for inputs while making snapshots. */
+    protected transient StreamElementSerializer<IN> inStreamElementSerializer;
+
+    protected transient TimestampedCollector<OUT> timestampedCollector;
+
+    protected transient boolean needDeepCopy;
+
+    /** Structure to control the process order of input records. */
+    private transient TableAsyncExecutionController<IN, OUT, KEY> 
asyncExecutionController;
+
+    /** Recovered input stream elements backed by keyed state. */
+    private transient ListState<Tuple2<StreamElement, StreamElement>> 
recoveredStreamElements;
+
+    /** Mailbox executor used to yield while waiting for buffers to empty. */
+    private final transient MailboxExecutor mailboxExecutor;
+
+    /** Number of inputs which is invoked for lookup but do not output until 
now. */
+    private final transient AtomicInteger totalInflightNum;
+
+    public TableKeyedAsyncWaitOperator(
+            AsyncFunction<IN, OUT> asyncFunction,
+            KeySelector<IN, KEY> keySelector,
+            long timeout,
+            int capacity,
+            ProcessingTimeService processingTimeService,
+            MailboxExecutor mailboxExecutor) {
+        super(asyncFunction);
+        this.timeout = timeout;
+        this.keySelector = keySelector;
+        Preconditions.checkArgument(
+                capacity > 0,
+                "The maxInflight of concurrent async operation should be 
greater than 0.");
+        this.capacity = capacity;
+        this.totalInflightNum = new AtomicInteger(0);
+        this.mailboxExecutor = mailboxExecutor;
+        this.processingTimeService = 
Preconditions.checkNotNull(processingTimeService);
+    }
+
+    @Override
+    public void setup(
+            StreamTask<?, ?> containingTask,
+            StreamConfig config,
+            Output<StreamRecord<OUT>> output) {
+        super.setup(containingTask, config, output);
+
+        this.inStreamElementSerializer =
+                new StreamElementSerializer<>(
+                        getOperatorConfig().getTypeSerializerIn(0, 
getUserCodeClassloader()));
+
+        this.timestampedCollector = new TimestampedCollector<>(super.output);
+        this.asyncExecutionController =
+                new TableAsyncExecutionController<>(
+                        this::invoke,
+                        this::emitWatermark,
+                        entry -> {
+                            entry.emitResult(timestampedCollector);
+                            totalInflightNum.decrementAndGet();
+                        },
+                        // the drive side is always left side
+                        entry -> 0,
+                        (record, inputIndex) -> 
keySelector.getKey(record.getValue()));
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws 
Exception {
+        super.initializeState(context);
+        TypeSerializer[] elementSerializers =
+                new TypeSerializer[] {inStreamElementSerializer, 
inStreamElementSerializer};
+        TypeInformation<Tuple2<StreamElement, StreamElement>> typeInfo =
+                Types.TUPLE(
+                        TypeInformation.of(StreamElement.class),
+                        TypeInformation.of(StreamElement.class));
+        Class<Tuple2<StreamElement, StreamElement>> type = 
typeInfo.getTypeClass();
+        TupleSerializer<Tuple2<StreamElement, StreamElement>> stateSerializer =
+                new TupleSerializer<>(type, elementSerializers);
+        recoveredStreamElements =
+                context.getKeyedStateStore()
+                        .getListState(new ListStateDescriptor<>(STATE_NAME, 
stateSerializer));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void open() throws Exception {
+        super.open();
+        this.needDeepCopy = getExecutionConfig().isObjectReuseEnabled() && 
!config.isChainStart();
+        List<KEY> keys =
+                (List<KEY>)
+                        getKeyedStateBackend()
+                                .getKeys(STATE_NAME, VoidNamespace.INSTANCE)
+                                .collect(Collectors.toList());
+        for (KEY key : keys) {

Review Comment:
   It does not works because this restoration includes the logic of trigger 
which means the asyncfunction would be invoked. The  asyncfunction need to be 
opened first. And OPERATOR::initializeState happens before OPERATOR::open.



##########
flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/lookup/TableKeyedAsyncWaitOperatorTest.java:
##########
@@ -0,0 +1,541 @@
+/*
+ * 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.lookup;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.testutils.MockEnvironment;
+import org.apache.flink.runtime.state.TestTaskStateManager;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarness;
+import 
org.apache.flink.streaming.runtime.tasks.StreamTaskMailboxTestHarnessBuilder;
+import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.TestHarnessUtil;
+import org.apache.flink.table.runtime.operators.TableKeyedAsyncWaitOperator;
+import 
org.apache.flink.table.runtime.operators.TableKeyedAsyncWaitOperatorFactory;
+import org.apache.flink.table.runtime.operators.join.lookup.keyordered.Epoch;
+import 
org.apache.flink.table.runtime.operators.join.lookup.keyordered.TableAsyncExecutionController;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperatorTest.IgnoreTimeoutLazyAsyncFunction;
+import static 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperatorTest.LazyAsyncFunction;
+import static 
org.apache.flink.streaming.api.operators.async.AsyncWaitOperatorTest.MyAsyncFunction;
+import static 
org.apache.flink.table.runtime.util.AsyncKeyOrderedTestUtils.assertKeyOrdered;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** Harness tests for {@link TableKeyedAsyncWaitOperator}. */
+public class TableKeyedAsyncWaitOperatorTest {
+
+    private static final long TIMEOUT = 1000L;
+
+    private static final MyAsyncFunction myAsyncFunction = new 
MyAsyncFunction();
+
+    private static final KeySelector<Integer, Integer> keySelector = value -> 
value;
+
+    private static LazyAsyncFunction lazyAsyncFunction;
+
+    @BeforeEach
+    void beforeEach() throws Exception {
+        lazyAsyncFunction = new LazyAsyncFunction();
+    }
+
+    @Test
+    void testMultiKeysWithEventTime() throws Exception {
+        try (final KeyedOneInputStreamOperatorTestHarness<Integer, Integer, 
Integer> testHarness =
+                createKeyedTestHarness(lazyAsyncFunction, keySelector, 
TIMEOUT, 10)) {
+            testHarness.open();
+            lazyAsyncFunction.countDown();
+
+            TableKeyedAsyncWaitOperator<Integer, Integer, Integer> operator =
+                    (TableKeyedAsyncWaitOperator<Integer, Integer, Integer>)
+                            testHarness.getOperator();
+            TableAsyncExecutionController<Integer, Integer, Integer> aec =
+                    operator.getAsyncExecutionController();
+
+            final long initialTime = 0L;
+            final ConcurrentLinkedQueue<Object> expectedOutput = new 
ConcurrentLinkedQueue<>();
+
+            testHarness.processElement(new StreamRecord<>(0, initialTime + 1));
+            testHarness.processElement(new StreamRecord<>(1, initialTime + 2));
+
+            testHarness.processWatermark(new Watermark(initialTime + 2));
+            Epoch<Integer> epoch = new Epoch<>(new Watermark(initialTime + 2));
+            testHarness.processElement(new StreamRecord<>(0, initialTime + 3));
+            epoch.incrementCount();
+            testHarness.processElement(new StreamRecord<>(4, initialTime + 4));
+            epoch.incrementCount();
+            testHarness.processElement(new StreamRecord<>(3, initialTime + 5));
+            epoch.incrementCount();
+            assertThat(aec.getActiveEpoch()).isEqualTo(epoch);
+
+            testHarness.processWatermark(new Watermark(initialTime + 5));
+            epoch = new Epoch<>(new Watermark(initialTime + 5));
+            testHarness.processElement(new StreamRecord<>(2, initialTime + 6));
+            epoch.incrementCount();
+            testHarness.processElement(new StreamRecord<>(1, initialTime + 7));
+            epoch.incrementCount();
+            testHarness.processElement(new StreamRecord<>(0, initialTime + 8));
+            epoch.incrementCount();
+            assertThat(aec.getActiveEpoch()).isEqualTo(epoch);
+
+            testHarness.processWatermark(new Watermark(initialTime + 8));
+
+            testHarness.endInput();
+            epoch = new Epoch<>(new Watermark(initialTime + 8));
+            assertThat(aec.getActiveEpoch()).isEqualTo(epoch);
+
+            expectedOutput.add(new StreamRecord<>(0, initialTime + 1));
+            expectedOutput.add(new StreamRecord<>(1, initialTime + 2));
+            expectedOutput.add(new Watermark(initialTime + 2));
+            expectedOutput.add(new StreamRecord<>(0, initialTime + 3));
+            expectedOutput.add(new StreamRecord<>(4, initialTime + 4));
+            expectedOutput.add(new StreamRecord<>(3, initialTime + 5));
+            expectedOutput.add(new Watermark(initialTime + 5));
+            expectedOutput.add(new StreamRecord<>(2, initialTime + 6));
+            expectedOutput.add(new StreamRecord<>(1, initialTime + 7));
+            expectedOutput.add(new StreamRecord<>(0, initialTime + 8));
+            expectedOutput.add(new Watermark(initialTime + 8));
+
+            Queue<Object> expected =
+                    new LinkedList<>(
+                            Arrays.asList(
+                                    new StreamRecord<>(0, initialTime + 1),
+                                    new StreamRecord<>(0, initialTime + 3),
+                                    new StreamRecord<>(0, initialTime + 8)));
+            assertKeyOrdered(testHarness.getOutput(), expected);
+
+            expected =
+                    new LinkedList<>(
+                            Arrays.asList(
+                                    new StreamRecord<>(1, initialTime + 2),
+                                    new StreamRecord<>(1, initialTime + 7)));
+            assertKeyOrdered(testHarness.getOutput(), expected);
+
+            List<Integer> index = Stream.of(2, 6, 
10).collect(Collectors.toList());
+            TestHarnessUtil.assertOutputAtIndexEquals(

Review Comment:
   Actually, the index is the location of watermark in the sequence.



##########
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java:
##########
@@ -616,9 +620,11 @@ void testStateSnapshotAndRestore() throws Exception {
 
         testHarness.setupOutputForSingletonOperatorChain();
 
+        LazyAsyncFunction lazyAsyncFunction = new LazyAsyncFunction();
+
         AsyncWaitOperatorFactory<Integer, Integer> factory =
                 new AsyncWaitOperatorFactory<>(
-                        new LazyAsyncFunction(), TIMEOUT, 4, 
AsyncDataStream.OutputMode.ORDERED);
+                        lazyAsyncFunction, TIMEOUT, 4, 
AsyncDataStream.OutputMode.ORDERED);

Review Comment:
   countDown() method is changed to non-static. We need an instance here to 
call the method



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