fredia commented on code in PR #24657:
URL: https://github.com/apache/flink/pull/24657#discussion_r1566645709
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/multipleinput/input/FirstInputOfTwoInput.java:
##########
@@ -54,4 +57,17 @@ public void processLatencyMarker(LatencyMarker
latencyMarker) throws Exception {
public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws
Exception {
operator.processWatermarkStatus1(watermarkStatus);
}
+
+ @Internal
+ @Override
+ public final boolean isAsyncStateProcessingEnabled() {
+ return (operator instanceof AsyncStateProcessing)
Review Comment:
Can `isAsyncStateProcessingEnabled` be cached in the `Input`? otherwise, we
should check these conditions per record.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##########
@@ -249,4 +250,22 @@ private void seizeCapacity() {
setCurrentContext(storedContext);
inFlightRecordNum.incrementAndGet();
}
+
+ /**
+ * A helper to request a {@link StateRequestType#SYNC_POINT} and run a
callback if it finishes.
Review Comment:
"if it finishes" is a little bit ambiguous, how about "if the record is not
blocked"?
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.streaming.runtime.operators.asyncprocessing;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController;
+import org.apache.flink.runtime.asyncprocessing.RecordContext;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.function.ThrowingConsumer;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+/**
+ * This operator is an abstract class that give the {@link
AbstractStreamOperator} the ability to
+ * perform {@link AsyncStateProcessing}. The aim is to make any subclass of
{@link
+ * AbstractStreamOperator} could manipulate async state with only a change of
base class.
+ */
+@Internal
+@SuppressWarnings("rawtypes")
+public abstract class AbstractAsyncStateStreamOperator<OUT> extends
AbstractStreamOperator<OUT>
+ implements AsyncStateProcessingOperator {
+
+ private AsyncExecutionController asyncExecutionController;
+
+ private RecordContext lastProcessContext;
Review Comment:
How about renaming it to `currentProcessContext`? like `currentKey`?
##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.streaming.runtime.operators.asyncprocessing;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController;
+import org.apache.flink.runtime.asyncprocessing.RecordContext;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.Input;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.function.ThrowingConsumer;
+import org.apache.flink.util.function.ThrowingRunnable;
+
+/**
+ * This operator is an abstract class that give the {@link
AbstractStreamOperator} the ability to
+ * perform {@link AsyncStateProcessing}. The aim is to make any subclass of
{@link
+ * AbstractStreamOperator} could manipulate async state with only a change of
base class.
+ */
+@Internal
+@SuppressWarnings("rawtypes")
+public abstract class AbstractAsyncStateStreamOperator<OUT> extends
AbstractStreamOperator<OUT>
+ implements AsyncStateProcessingOperator {
+
+ private AsyncExecutionController asyncExecutionController;
+
+ private RecordContext lastProcessContext;
+
+ /** Initialize necessary state components for {@link
AbstractStreamOperator}. */
+ @Override
+ public void setup(
+ StreamTask<?, ?> containingTask,
+ StreamConfig config,
+ Output<StreamRecord<OUT>> output) {
+ super.setup(containingTask, config, output);
+ // TODO: properly read config and setup
+ final MailboxExecutor mailboxExecutor =
+ containingTask.getEnvironment().getMainMailboxExecutor();
+ this.asyncExecutionController = new
AsyncExecutionController(mailboxExecutor, null);
+ }
+
+ @Override
+ public boolean isAsyncStateProcessingEnabled() {
+ return true;
+ }
+
+ @Override
+ public ElementOrder getElementOrder() {
+ return ElementOrder.RECORD_ORDER;
Review Comment:
Should a setter be provided for ElementOrder for internal usage?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##########
@@ -249,4 +250,22 @@ private void seizeCapacity() {
setCurrentContext(storedContext);
inFlightRecordNum.incrementAndGet();
}
+
+ /**
+ * A helper to request a {@link StateRequestType#SYNC_POINT} and run a
callback if it finishes.
+ *
+ * @param callback the callback to run if it finishes.
+ */
+ public void syncPointRequestWithCallback(ThrowingRunnable<Exception>
callback) {
Review Comment:
How about adding a UT for `SYNC_POINT`?
--
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]