XComp commented on a change in pull request #18987: URL: https://github.com/apache/flink/pull/18987#discussion_r822632235
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleaner.java ########## @@ -166,21 +188,49 @@ private DefaultResourceCleaner( mainThreadExecutor.assertRunningInMainThread(); CompletableFuture<Void> cleanupFuture = FutureUtils.completedVoidFuture(); - for (T cleanup : prioritizedCleanup) { - cleanupFuture = cleanupFuture.thenCompose(ignoredValue -> withRetry(jobId, cleanup)); + for (Map.Entry<String, T> cleanup : prioritizedCleanup.entrySet()) { + cleanupFuture = + cleanupFuture.thenCompose( + ignoredValue -> withRetry(jobId, cleanup.getKey(), cleanup.getValue())); } return cleanupFuture.thenCompose( ignoredValue -> FutureUtils.completeAll( - regularCleanup.stream() - .map(cleanup -> withRetry(jobId, cleanup)) + regularCleanup.entrySet().stream() + .map( + cleanupEntry -> + withRetry( + jobId, + cleanupEntry.getKey(), + cleanupEntry.getValue())) .collect(Collectors.toList()))); } - private CompletableFuture<Void> withRetry(JobID jobId, T cleanup) { + private CompletableFuture<Void> withRetry(JobID jobId, String label, T cleanup) { return FutureUtils.retryWithDelay( - () -> cleanupFn.cleanupAsync(cleanup, jobId, cleanupExecutor), + () -> + cleanupFn + .cleanupAsync(cleanup, jobId, cleanupExecutor) + .whenComplete( Review comment: `whenComplete` only provides a callback for after completion. I did a quick test to verify it: ``` @Test public void foo() throws ExecutionException, InterruptedException { CompletableFuture<String> f = new CompletableFuture<>(); CompletableFuture<String> result = f.whenComplete( (success, failure) -> { if (failure != null) { System.out.println(failure.getClass().getSimpleName()); } else { System.out.println("No Failure"); } }); f.completeExceptionally(new RuntimeException()); result.get(); } ``` The code above failed with printing the class name and failing in the end: > java.util.concurrent.ExecutionException: java.lang.RuntimeException at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908) [...] at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:221) at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54) Caused by: java.lang.RuntimeException at org.apache.flink.runtime.dispatcher.cleanup.DefaultResourceCleanerTest.foo(DefaultResourceCleanerTest.java:72) ... 67 more -- 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