fredia commented on code in PR #26682:
URL: https://github.com/apache/flink/pull/26682#discussion_r2152085211


##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/SimpleAsyncExecutionController.java:
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.runtime.asyncprocessing;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.core.asyncprocessing.AsyncFutureImpl;
+import org.apache.flink.core.asyncprocessing.InternalAsyncFuture;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.asyncprocessing.declare.DeclarationManager;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+import org.apache.flink.util.function.CheckedSupplier;
+
+import javax.annotation.Nullable;
+
+import java.util.LinkedList;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/** The {@link SimpleAsyncExecutionController} is responsible for handling 
simple runnable tasks. */
+public class SimpleAsyncExecutionController<K>
+        extends AsyncExecutionController<K, 
SimpleAsyncExecutionController.RunnableTask<K, ?>> {
+
+    public SimpleAsyncExecutionController(
+            MailboxExecutor mailboxExecutor,
+            AsyncFutureImpl.AsyncFrameworkExceptionHandler exceptionHandler,
+            ExecutorService asyncThreadPool,
+            DeclarationManager declarationManager,
+            EpochManager.ParallelMode epochParallelMode,
+            int maxParallelism,
+            int batchSize,
+            long bufferTimeout,
+            int maxInFlightRecords,
+            @Nullable SwitchContextListener<K> switchContextListener,
+            @Nullable MetricGroup metricGroup) {
+        super(
+                mailboxExecutor,
+                exceptionHandler,
+                new TaskExecutor<>(asyncThreadPool),
+                declarationManager,
+                epochParallelMode,
+                maxParallelism,
+                batchSize,
+                bufferTimeout,
+                maxInFlightRecords,
+                switchContextListener,
+                metricGroup);
+    }
+
+    /**
+     * Submit a {@link AsyncRequest} to this AsyncExecutionController and 
trigger it if needed.
+     *
+     * @param checkedSupplier the runnable to execute.
+     * @param allowOverdraft whether to allow overdraft.
+     * @return the state future.
+     */
+    public <R> InternalAsyncFuture<R> handleRequest(
+            CheckedSupplier<R> checkedSupplier, boolean allowOverdraft) {
+        InternalAsyncFuture<R> asyncFuture = 
asyncFutureFactory.create(currentContext);
+        RunnableTask<K, R> request =
+                new RunnableTask<>(currentContext, false, asyncFuture, 
checkedSupplier);
+
+        handleRequest(request, allowOverdraft);
+        return asyncFuture;
+    }
+
+    public static class RunnableTask<K, RET> extends AsyncRequest<K> {
+
+        final CheckedSupplier<RET> runnable;
+
+        public RunnableTask(
+                RecordContext<K> context,
+                boolean sync,
+                InternalAsyncFuture<RET> asyncFuture,
+                CheckedSupplier<RET> runnable) {
+            super(context, sync, asyncFuture);
+            this.runnable = runnable;
+        }
+
+        private void run() throws Exception {
+            asyncFuture.complete(runnable.get());
+        }
+    }
+
+    static class TaskExecutor<K> implements AsyncExecutor<RunnableTask<K, ?>> {
+
+        private final ExecutorService taskExecutorService;
+        private final boolean managedExecutor;
+
+        public TaskExecutor(ExecutorService taskExecutorService) {
+            this.taskExecutorService = taskExecutorService;
+            this.managedExecutor = false;
+        }
+
+        public TaskExecutor(int poolSize) {

Review Comment:
   nit: unused



##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/operators/AbstractAsyncStateStreamOperatorV2.java:
##########
@@ -64,20 +66,20 @@
 
 /**
  * This operator is an abstract class that give the {@link 
AbstractStreamOperatorV2} the ability to
- * perform {@link AsyncStateProcessing}. The aim is to make any subclass of 
{@link
+ * perform {@link AsyncKeyOrderedProcessing}. The aim is to make any subclass 
of {@link
  * AbstractStreamOperatorV2} could manipulate async state with only a change 
of base class.
  */
 @Internal
 @SuppressWarnings("rawtypes")
 public abstract class AbstractAsyncStateStreamOperatorV2<OUT> extends 
AbstractStreamOperatorV2<OUT>

Review Comment:
   Do we need `AbstractAsyncKeyOrderedStreamOperatorV2`?



##########
flink-core/src/main/java/org/apache/flink/core/state/StateFutureUtils.java:
##########
@@ -36,12 +39,12 @@
 public class StateFutureUtils {

Review Comment:
   How about renaming it to `AsynFutureUtils`?



##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/operators/AbstractAsyncRunnableStreamOperator.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.runtime.asyncprocessing.operators;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.core.asyncprocessing.AsyncFuture;
+import org.apache.flink.runtime.asyncprocessing.AsyncExecutionController;
+import org.apache.flink.runtime.asyncprocessing.EpochManager;
+import org.apache.flink.runtime.asyncprocessing.SimpleAsyncExecutionController;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import 
org.apache.flink.streaming.runtime.operators.asyncprocessing.AsyncKeyOrderedProcessing;
+import 
org.apache.flink.streaming.runtime.operators.asyncprocessing.AsyncKeyOrderedProcessingOperator;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.function.CheckedSupplier;
+
+import java.util.concurrent.ExecutorService;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This operator is an abstract class that give the {@link 
AbstractStreamOperator} the ability to
+ * perform {@link AsyncKeyOrderedProcessing}. Note that this operator is not 
under keyed context,
+ * but providing key selectors for the async processing.
+ */
+@Internal
+@SuppressWarnings("rawtypes")
+public abstract class AbstractAsyncRunnableStreamOperator<OUT>
+        extends AbstractAsyncKeyOrderedStreamOperator<OUT>
+        implements AsyncKeyOrderedProcessingOperator {
+
+    final KeySelector<?, ?> keySelector1;
+    final KeySelector<?, ?> keySelector2;
+    final ExecutorService asyncThreadPool;
+    final int asyncBufferSize;
+    final long asyncBufferTimeout;
+    final int inFlightRecordsLimit;
+
+    public AbstractAsyncRunnableStreamOperator(
+            KeySelector<?, ?> keySelector1,
+            KeySelector<?, ?> keySelector2,
+            ExecutorService asyncThreadPool,
+            int asyncBufferSize,
+            long asyncBufferTimeout,
+            int inFlightRecordsLimit) {
+        this.keySelector1 = keySelector1;
+        this.keySelector2 = keySelector2;
+        this.asyncThreadPool = asyncThreadPool;
+        this.asyncBufferSize = asyncBufferSize;
+        this.asyncBufferTimeout = asyncBufferTimeout;
+        this.inFlightRecordsLimit = inFlightRecordsLimit;
+    }
+
+    @Override
+    protected KeySelector getKeySelectorForAsyncKeyedContext(int index) {
+        switch (index) {
+            case 1:
+                return keySelector1;
+            case 2:
+                return keySelector2;
+            default:
+                throw new ArrayIndexOutOfBoundsException(
+                        "Try to get key selector for index " + index);
+        }
+    }
+
+    @Override
+    protected AsyncExecutionController createAsyncExecutionController() {
+        if (isAsyncKeyOrderedProcessingEnabled()) {
+            final StreamTask<?, ?> containingTask = 
checkNotNull(getContainingTask());
+            final MailboxExecutor mailboxExecutor =
+                    containingTask
+                            .getMailboxExecutorFactory()
+                            
.createExecutor(getOperatorConfig().getChainIndex());
+            final int maxParallelism = 
environment.getTaskInfo().getMaxNumberOfParallelSubtasks();
+            return new SimpleAsyncExecutionController(
+                    mailboxExecutor,
+                    this::handleAsyncException,
+                    asyncThreadPool,
+                    getDeclarationManager(),
+                    getEpochParallelMode(),
+                    maxParallelism,
+                    asyncBufferSize,
+                    asyncBufferTimeout,
+                    inFlightRecordsLimit,
+                    null,
+                    getMetricGroup().addGroup("asyncProcessing"));
+        }
+        return null;
+    }
+
+    /**
+     * Define the parallel mode of the epoch manager. The default is {@link
+     * EpochManager.ParallelMode#SERIAL_BETWEEN_EPOCH}. Subclasses can 
override this method to
+     * change the parallel mode.
+     */
+    protected EpochManager.ParallelMode getEpochParallelMode() {
+        return EpochManager.ParallelMode.SERIAL_BETWEEN_EPOCH;
+    }
+
+    @SuppressWarnings("unchecked")
+    protected <RET> AsyncFuture<RET> asyncProcess(CheckedSupplier<RET> 
runnable) {
+        return ((SimpleAsyncExecutionController) asyncExecutionController)

Review Comment:
   check `isAsyncKeyOrderedProcessingEnabled` here?



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