dawidwys commented on a change in pull request #15055:
URL: https://github.com/apache/flink/pull/15055#discussion_r659725140
##########
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 don't think it works that way. The mailbox might be empty at the time
you call `tryYield` (honestly I think in most cases it will) and you will
immediately go for closing the task. If I understand correctly, at this point
the mailbox will mostly process checkpoint triggers from time to time.
Therefore we cannot close the executor immediately, however busy looping
waiting for the `triggerCheckpoint` mail is also not the best option as they
might be triggered occasionally.
--
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]