akalash commented on a change in pull request #15375:
URL: https://github.com/apache/flink/pull/15375#discussion_r604750548



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -535,41 +545,82 @@ protected void beforeInvoke() throws Exception {
         // -------- Invoke --------
         LOG.debug("Invoking {}", getName());
 
+        SuspendableMailboxLoop suspendableLoop = 
mailboxProcessor.getSuspendableLoop();
+
         // we need to make sure that any triggers scheduled in open() cannot be
         // executed before all operators are opened
-        actionExecutor.runThrowing(
-                () -> {
-                    SequentialChannelStateReader reader =
-                            getEnvironment()
-                                    .getTaskStateManager()
-                                    .getSequentialChannelStateReader();
-                    reader.readOutputData(
-                            getEnvironment().getAllWriters(),
-                            !configuration.isGraphContainingLoops());
-
-                    operatorChain.initializeStateAndOpenOperators(
-                            createStreamTaskStateInitializer());
-
-                    channelIOExecutor.execute(
-                            () -> {
-                                try {
-                                    
reader.readInputData(getEnvironment().getAllInputGates());
-                                } catch (Exception e) {
-                                    asyncExceptionHandler.handleAsyncException(
-                                            "Unable to read channel state", e);
-                                }
-                            });
+        CompletableFuture<Void> allGatesRecoveredFuture =
+                actionExecutor.call(
+                        () -> {
+                            SequentialChannelStateReader reader =
+                                    getEnvironment()
+                                            .getTaskStateManager()
+                                            .getSequentialChannelStateReader();
+                            reader.readOutputData(
+                                    getEnvironment().getAllWriters(),
+                                    !configuration.isGraphContainingLoops());
+
+                            operatorChain.initializeStateAndOpenOperators(
+                                    createStreamTaskStateInitializer());
+
+                            IndexedInputGate[] inputGates = 
getEnvironment().getAllInputGates();
+                            channelIOExecutor.execute(
+                                    () -> {
+                                        try {
+                                            reader.readInputData(inputGates);
+                                        } catch (Exception e) {
+                                            
asyncExceptionHandler.handleAsyncException(
+                                                    "Unable to read channel 
state", e);
+                                        }
+                                    });
+
+                            List<CompletableFuture<?>> recoveredFutures =
+                                    new ArrayList<>(inputGates.length);
+                            for (InputGate inputGate : inputGates) {
+                                
recoveredFutures.add(inputGate.getStateConsumedFuture());
+
+                                inputGate
+                                        .getStateConsumedFuture()
+                                        .thenRun(
+                                                () ->
+                                                        
mainMailboxExecutor.execute(
+                                                                
inputGate::requestPartitions,
+                                                                "Input gate 
request partitions"));
+                            }
 
-                    for (InputGate inputGate : 
getEnvironment().getAllInputGates()) {
-                        inputGate
-                                .getStateConsumedFuture()
-                                .thenRun(
-                                        () ->
-                                                mainMailboxExecutor.execute(
-                                                        
inputGate::requestPartitions,
-                                                        "Input gate request 
partitions"));
-                    }
-                });
+                            return gatesRecoveredFuture(suspendableLoop, 
recoveredFutures);
+                        });
+
+        // Run mailbox until all gates will be recovered.
+        do {
+            suspendableLoop.run();
+        } while (isMailboxLoopRunning() && !allGatesRecoveredFuture.isDone());
+
+        if (canceled) {
+            throw new CancelTaskException();
+        }
+
+        if (!allGatesRecoveredFuture.isDone()) {
+            throw new Exception("Mailbox loop interrupted before recovery was 
finished.");
+        }
+
+        restored = true;
+    }
+
+    @VisibleForTesting
+    CompletableFuture<Void> gatesRecoveredFuture(
+            SuspendableMailboxLoop suspendableLoop, List<CompletableFuture<?>> 
recoveredFutures) {
+        return CompletableFuture.allOf(recoveredFutures.toArray(new 
CompletableFuture[0]))
+                .thenRun(suspendableLoop::suspend);
+    }

Review comment:
       It is using in the test but anyway I am going to rewrite tests in order 
to avoiding the mockito and then I will be able to inline this 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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to