Skip to content

Commit 834fefd

Browse files
committed
[FLINK-38543][checkpoint] Fix Mailbox loop interrupted before recovery finished
1 parent 95eecfd commit 834fefd

1 file changed

Lines changed: 8 additions & 2 deletions

File tree

flink-runtime/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -913,8 +913,14 @@ private CompletableFuture<Void> restoreStateAndGates(
913913
"Input gate request partitions"));
914914
}
915915

916-
return CompletableFuture.allOf(recoveredFutures.toArray(new CompletableFuture[0]))
917-
.thenRun(mailboxProcessor::suspend);
916+
// Return allOf result instead of thenRun result.
917+
// thenRun returns a NEW future that completes after the callback finishes.
918+
// Since suspend() runs on the async thread and just sends a poison mail,
919+
// the mailbox loop can exit before suspend() returns, causing isDone() to be false.
920+
CompletableFuture<Void> allRecoveredFuture =
921+
CompletableFuture.allOf(recoveredFutures.toArray(new CompletableFuture[0]));
922+
allRecoveredFuture.thenRun(mailboxProcessor::suspend);
923+
return allRecoveredFuture;
918924
}
919925

920926
private void ensureNotCanceled() {

0 commit comments

Comments
 (0)