dawidwys commented on a change in pull request #15055:
URL: https://github.com/apache/flink/pull/15055#discussion_r659741252



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -695,6 +697,27 @@ protected void afterInvoke() throws Exception {
         // close all operators in a chain effect way
         operatorChain.closeOperators(actionExecutor);
 
+        // If checkpoints are enabled, waits for all the records get processed 
by the downstream
+        // tasks. During this process, this task could coordinate with its 
downstream tasks to
+        // continue perform checkpoints.
+        if (configuration.isCheckpointingEnabled()) {
+            LOG.debug("Waiting for all the records processed by the downstream 
tasks.");
+            CompletableFuture<Void> combineFuture =
+                    FutureUtils.waitForAll(
+                            Arrays.stream(getEnvironment().getAllWriters())
+                                    .map(
+                                            FunctionUtils.uncheckedFunction(
+                                                    ResultPartitionWriter
+                                                            
::getAllRecordsProcessedFuture))
+                                    .collect(Collectors.toList()));
+
+            MailboxExecutor mailboxExecutor =
+                    
mailboxProcessor.getMailboxExecutor(TaskMailbox.MIN_PRIORITY);
+            while (!combineFuture.isDone()) {

Review comment:
       Ah right, I forgot/missed the outer loop. Yes, that would be one option. 
I don't like the sleep here, but I probably a better solution than busy 
looping. Not sure about the value of the sleep/wait. Maybe we could correlate 
it with the checkpoint interval? Still would like to hear from @pnowojski what 
he thinks about it.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -695,6 +697,27 @@ protected void afterInvoke() throws Exception {
         // close all operators in a chain effect way
         operatorChain.closeOperators(actionExecutor);
 
+        // If checkpoints are enabled, waits for all the records get processed 
by the downstream
+        // tasks. During this process, this task could coordinate with its 
downstream tasks to
+        // continue perform checkpoints.
+        if (configuration.isCheckpointingEnabled()) {
+            LOG.debug("Waiting for all the records processed by the downstream 
tasks.");
+            CompletableFuture<Void> combineFuture =
+                    FutureUtils.waitForAll(
+                            Arrays.stream(getEnvironment().getAllWriters())
+                                    .map(
+                                            FunctionUtils.uncheckedFunction(
+                                                    ResultPartitionWriter
+                                                            
::getAllRecordsProcessedFuture))
+                                    .collect(Collectors.toList()));
+
+            MailboxExecutor mailboxExecutor =
+                    
mailboxProcessor.getMailboxExecutor(TaskMailbox.MIN_PRIORITY);
+            while (!combineFuture.isDone()) {

Review comment:
       I think that's actually a nice idea. We would need to change the 
`processInput` method slightly so that it suspends the `mailboxProcessor` 
instead of calling `allActionsComplete`. Then in the afterInvoke method we 
would call `allActionsComplete` once the `combineFuture` finishes.
   
   ```
       protected void processInput(MailboxDefaultAction.Controller controller) 
throws Exception {
           InputStatus status = inputProcessor.processInput();
           if (status == InputStatus.MORE_AVAILABLE && 
recordWriter.isAvailable()) {
               return;
           }
           if (status == InputStatus.END_OF_INPUT) {
               mailboxProcessor.suspend();
               return;
           }
           ....
       }
   
       protected void afterInvoke() throws Exception {
           LOG.debug("Finished task {}", getName());
           getCompletionFuture().exceptionally(unused -> null).join();
   
           final CompletableFuture<Void> timersFinishedFuture = new 
CompletableFuture<>();
   
           // close all operators in a chain effect way
           operatorChain.closeOperators(actionExecutor);
   
           // If checkpoints are enabled, waits for all the records get 
processed by the downstream
           // tasks. During this process, this task could coordinate with its 
downstream tasks to
           // continue perform checkpoints.
           final CompletableFuture<Void> taskCanFinish;
           if (configuration.isCheckpointingEnabled()) {
               LOG.debug("Waiting for all the records processed by the 
downstream tasks.");
   
               List<CompletableFuture<Void>> allRecordsProcessedFutures = new 
ArrayList<>();
               for (ResultPartitionWriter partitionWriter : 
getEnvironment().getAllWriters()) {
                   partitionWriter.notifyEndOfUserRecords();
                   
allRecordsProcessedFutures.add(partitionWriter.getAllRecordsProcessedFuture());
               }
               taskCanFinish = 
FutureUtils.waitForAll(allRecordsProcessedFutures);
           } else {
               taskCanFinish = CompletableFuture.completedFuture(null);
           }
           taskCanFinish.thenRun(mailboxProcessor::allActionsCompleted);
           mailboxProcessor.runMailboxLoop();
    
           ....
       }
   
   ```




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


Reply via email to