jectpro7 commented on code in PR #24740:
URL: https://github.com/apache/flink/pull/24740#discussion_r1584465123


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/asyncprocessing/AbstractAsyncStateStreamOperator.java:
##########
@@ -74,15 +76,20 @@ public void setup(
         final int asyncBufferSize = 
environment.getExecutionConfig().getAsyncStateBufferSize();
         final long asyncBufferTimeout =
                 environment.getExecutionConfig().getAsyncStateBufferTimeout();
-        // TODO: initial state executor and set state executor for aec
+
+        AsyncKeyedStateBackend asyncKeyedStateBackend =
+                Preconditions.checkNotNull(
+                        stateHandler.getAsyncKeyedStateBackend(),
+                        "Current State Backend doesn't support async access");
         this.asyncExecutionController =
                 new AsyncExecutionController(
                         mailboxExecutor,
-                        null,
+                        asyncKeyedStateBackend.createStateExecutor(),
                         maxParallelism,
                         asyncBufferSize,
                         asyncBufferTimeout,
                         inFlightRecordsLimit);
+        asyncKeyedStateBackend.setup(asyncExecutionController);

Review Comment:
   the dependency looks like not very clear, can we make it like this?
   ```
   new AsyncExecutionController(
                           mailboxExecutor,
                           asyncKeyedStateBackend,
                           maxParallelism,
                           asyncBufferSize,
                           asyncBufferTimeout,
                           inFlightRecordsLimit);
   ```
   The AEC is responsible for creating the `stateExecutor` and set it up inside 
the construction 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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to