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


##########
flink-runtime/src/main/java/org/apache/flink/runtime/asyncprocessing/AsyncExecutionController.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.state.InternalStateFuture;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The Async Execution Controller (AEC) receives processing requests from 
operators, and put them
+ * into execution according to some strategies.
+ *
+ * <p>It is responsible for:
+ * <li>Preserving the sequence of elements bearing the same key by delaying 
subsequent requests
+ *     until the processing of preceding ones is finalized.
+ * <li>Tracking the in-flight data(records) and blocking the input if too much 
data in flight
+ *     (back-pressure). It invokes {@link MailboxExecutor#yield()} to pause 
current operations,
+ *     allowing for the execution of callbacks (mails in Mailbox).
+ *
+ * @param <R> the type of the record
+ * @param <K> the type of the key
+ */
+public class AsyncExecutionController<R, K> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(AsyncExecutionController.class);
+
+    public static final int DEFAULT_MAX_IN_FLIGHT_RECORD_NUM = 6000;
+
+    /** The max allow number of in-flight records. */
+    private final int maxInFlightRecordNum;
+
+    /** The key accounting unit which is used to detect the key conflict. */
+    final KeyAccountingUnit<R, K> keyAccountingUnit;
+
+    /**
+     * A factory to build {@link 
org.apache.flink.core.state.InternalStateFuture}, this will auto
+     * wire the created future with mailbox executor. Also conducting the 
context switch.
+     */
+    private final StateFutureFactory<R, K> stateFutureFactory;
+
+    /** The state executor where the {@link StateRequest} is actually 
executed. */
+    final StateExecutor stateExecutor;
+
+    /** The corresponding context that currently runs in task thread. */
+    RecordContext<R, K> currentContext;
+
+    public AsyncExecutionController(MailboxExecutor mailboxExecutor, 
StateExecutor stateExecutor) {
+        this(mailboxExecutor, stateExecutor, DEFAULT_MAX_IN_FLIGHT_RECORD_NUM);
+    }
+
+    public AsyncExecutionController(
+            MailboxExecutor mailboxExecutor, StateExecutor stateExecutor, int 
maxInFlightRecords) {
+        this.keyAccountingUnit = new KeyAccountingUnit<>();
+        this.stateFutureFactory = new StateFutureFactory<>(this, 
mailboxExecutor);
+        this.stateExecutor = stateExecutor;
+        this.maxInFlightRecordNum = maxInFlightRecords;
+        LOG.info("Create AsyncExecutionController: maxInFlightRecordsNum {}", 
maxInFlightRecords);
+    }
+
+    /**
+     * Build a new context based on record and key. Also wired with internal 
{@link
+     * KeyAccountingUnit}.
+     *
+     * @param record the given record.
+     * @param key the given key.
+     * @return the built record context.
+     */
+    public RecordContext<R, K> buildContext(R record, K key) {

Review Comment:
   Well, building/initializing is a special use case other than simple setting. 
We can change the behavior of this method to something like `buildAndSet`. But 
I slightly tend to make interfaces implement more specialized(single) functions 
rather than multiple.



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