eemario commented on code in PR #27741:
URL: https://github.com/apache/flink/pull/27741#discussion_r2973219496


##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java:
##########
@@ -98,46 +123,93 @@ private void startServices() {
                             getClass().getSimpleName()),
                     e);
         }
+
+        try {
+            applicationStore.start();
+        } catch (Exception e) {
+            throw new FlinkRuntimeException(
+                    String.format(
+                            "Could not start %s when trying to start the %s.",
+                            applicationStore.getClass().getSimpleName(),
+                            getClass().getSimpleName()),
+                    e);
+        }
     }
 
     private void createDispatcherIfRunning(
             Collection<ExecutionPlan> executionPlans,
-            Collection<JobResult> recoveredDirtyJobResults) {
+            Collection<JobResult> recoveredDirtyJobResults,
+            Collection<AbstractApplication> recoveredApplications,
+            Collection<ApplicationResult> recoveredDirtyApplicationResults) {
         runIfStateIs(
-                State.RUNNING, () -> createDispatcher(executionPlans, 
recoveredDirtyJobResults));
+                State.RUNNING,
+                () ->
+                        createDispatcher(
+                                executionPlans,
+                                recoveredDirtyJobResults,
+                                recoveredApplications,
+                                recoveredDirtyApplicationResults));
     }
 
     private void createDispatcher(
             Collection<ExecutionPlan> executionPlans,
-            Collection<JobResult> recoveredDirtyJobResults) {
+            Collection<JobResult> recoveredDirtyJobResults,
+            Collection<AbstractApplication> recoveredApplications,
+            Collection<ApplicationResult> recoveredDirtyApplicationResults) {
 
         final DispatcherGatewayService dispatcherService =
                 dispatcherGatewayServiceFactory.create(
                         DispatcherId.fromUuid(getLeaderSessionId()),
                         executionPlans,
                         recoveredDirtyJobResults,
+                        recoveredApplications,
+                        recoveredDirtyApplicationResults,
                         executionPlanStore,
-                        jobResultStore);
+                        jobResultStore,
+                        applicationStore,
+                        applicationResultStore);
 
         completeDispatcherSetup(dispatcherService);
     }
 
-    private CompletableFuture<Void>
-            
createDispatcherBasedOnRecoveredExecutionPlansAndRecoveredDirtyJobResults() {
-        // TODO support application recovery which may require fetching user 
jar from blob server
-
-        final CompletableFuture<Collection<JobResult>> dirtyJobsFuture =
+    private CompletableFuture<Void> 
createDispatcherBasedOnRecoveredApplicationsAndJobs() {
+        final CompletableFuture<Collection<JobResult>> dirtyJobResultsFuture =
                 
CompletableFuture.supplyAsync(this::getDirtyJobResultsIfRunning, ioExecutor);
 
-        return dirtyJobsFuture
-                .thenApplyAsync(
-                        dirtyJobs ->
-                                this.recoverJobsIfRunning(
-                                        dirtyJobs.stream()
+        final CompletableFuture<Collection<ExecutionPlan>> recoveredJobsFuture 
=
+                dirtyJobResultsFuture.thenApplyAsync(
+                        dirtyJobResults ->
+                                recoverJobsIfRunning(
+                                        dirtyJobResults.stream()
                                                 .map(JobResult::getJobId)
                                                 .collect(Collectors.toSet())),
-                        ioExecutor)
-                .thenAcceptBoth(dirtyJobsFuture, 
this::createDispatcherIfRunning)
+                        ioExecutor);
+
+        final CompletableFuture<Collection<ApplicationResult>> 
dirtyApplicationResultsFuture =
+                CompletableFuture.supplyAsync(
+                        this::getDirtyApplicationResultsIfRunning, ioExecutor);
+
+        final CompletableFuture<Collection<AbstractApplication>> 
recoveredApplicationsFuture =
+                dirtyApplicationResultsFuture.thenCombineAsync(
+                        recoveredJobsFuture,
+                        (dirtyApplicationResults, recoveredJobs) -> {
+                            return recoverApplicationsIfRunning(
+                                    dirtyApplicationResults.stream()
+                                            
.map(ApplicationResult::getApplicationId)
+                                            .collect(Collectors.toSet()),
+                                    recoveredJobsFuture.join(),

Review Comment:
   Updated: use `allOf` instead.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java:
##########
@@ -887,6 +1062,83 @@ private void 
writeToArchivedApplicationStore(ArchivedApplication archivedApplica
         }
     }
 
+    private CompletableFuture<?> 
registerGloballyTerminatedApplicationInApplicationResultStore(
+            ArchivedApplication application) {
+        final ApplicationID applicationId = application.getApplicationId();
+
+        return applicationResultStore
+                .hasCleanApplicationResultEntryAsync(applicationId)
+                .thenCompose(
+                        hasCleanResult -> {
+                            if (hasCleanResult) {
+                                log.warn(
+                                        "Application {} is already marked as 
clean but clean up was triggered again.",
+                                        applicationId);
+                                return FutureUtils.completedVoidFuture();
+                            }
+
+                            return applicationResultStore
+                                    
.hasDirtyApplicationResultEntryAsync(applicationId)
+                                    .thenCompose(
+                                            hasDirtyResult -> {
+                                                if (hasDirtyResult) {
+                                                    return 
FutureUtils.completedVoidFuture();
+                                                }
+
+                                                return applicationResultStore
+                                                        
.createDirtyResultAsync(
+                                                                new 
ApplicationResultEntry(
+                                                                        
ApplicationResult
+                                                                               
 .createFrom(
+                                                                               
         application)));
+                                            });
+                        })
+                .handleAsync(
+                        (ignored, error) -> {
+                            if (error != null) {
+                                fatalErrorHandler.onFatalError(
+                                        new FlinkException(
+                                                String.format(
+                                                        "The application %s 
couldn't be marked as pre-cleanup finished in ApplicationResultStore.",
+                                                        applicationId),
+                                                error));

Review Comment:
   Updated.



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