tillrohrmann commented on a change in pull request #13217: URL: https://github.com/apache/flink/pull/13217#discussion_r476207888
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java ########## @@ -0,0 +1,256 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.webmonitor.JobDetails; +import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.util.AutoCloseableAsync; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.util.concurrent.CompletableFuture; + +/** + * Abstraction used by the {@link Dispatcher} to manage jobs. + */ +public final class DispatcherJob implements AutoCloseableAsync { + + private final Logger log = LoggerFactory.getLogger(DispatcherJob.class); + + private final CompletableFuture<JobManagerRunner> jobManagerRunnerFuture; + private final CompletableFuture<ArchivedExecutionGraph> jobResultFuture; + private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>(); + + private final long initializationTimestamp; + private final JobID jobId; + private final String jobName; + + private final Object lock = new Object(); + + // internal field to track job status during initialization. Is not updated anymore after + // job is initialized, cancelled or failed. + @GuardedBy("lock") + private DispatcherJobStatus jobStatus = DispatcherJobStatus.INITIALIZING; + + private enum DispatcherJobStatus { + // We are waiting for the JobManagerRunner to be initialized + INITIALIZING(JobStatus.INITIALIZING), + // JobManagerRunner is initialized + JOB_MANAGER_RUNNER_INITIALIZED(null), + // waiting for cancellation. We stay in this status until the job result future completed. + CANCELLING(JobStatus.CANCELLING); + + @Nullable + private final JobStatus jobStatus; + + DispatcherJobStatus(JobStatus jobStatus) { + this.jobStatus = jobStatus; + } + + public JobStatus asJobStatus() { + if (jobStatus == null) { + throw new IllegalStateException("This state is not defined as a 'JobStatus'"); + } + return jobStatus; + } + } + + static DispatcherJob createFor( + CompletableFuture<JobManagerRunner> jobManagerRunnerFuture, + JobID jobId, + String jobName, + Dispatcher.ExecutionType executionType) { + return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, executionType); + } + + private DispatcherJob( + CompletableFuture<JobManagerRunner> jobManagerRunnerFuture, + JobID jobId, + String jobName, + Dispatcher.ExecutionType executionType) { + this.jobManagerRunnerFuture = jobManagerRunnerFuture; + this.jobId = jobId; + this.jobName = jobName; + this.initializationTimestamp = System.currentTimeMillis(); + this.jobResultFuture = new CompletableFuture<>(); + + FutureUtils.assertNoException(this.jobManagerRunnerFuture.handle((jobManagerRunner, throwable) -> { + // JM has been initialized, or the initialization failed + synchronized (lock) { + if (throwable == null) { + // Forward result future + FutureUtils.forward(jobManagerRunner.getResultFuture(), jobResultFuture); + + if (jobStatus != DispatcherJobStatus.CANCELLING) { + jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED; + } + } else { // failure during initialization + if (executionType == Dispatcher.ExecutionType.RECOVERY) { + jobResultFuture.completeExceptionally(throwable); + } else { + jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob( + jobId, + jobName, + JobStatus.FAILED, + throwable, + initializationTimestamp)); + } + } + } + return null; + })); + } + + public CompletableFuture<ArchivedExecutionGraph> getResultFuture() { + return jobResultFuture; + } + + public CompletableFuture<JobDetails> requestJobDetails(Time timeout) { + synchronized (lock) { + if (isInitialized()) { + return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails( + timeout)); + } else { + return getInitializingJobDetails(timeout); + } + } + } + + public CompletableFuture<Acknowledge> cancel(Time timeout) { + synchronized (lock) { + if (isInitialized()) { + return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.cancel(timeout)); + } else { + log.info("Cancellation during initialization requested for job {}. Job will be cancelled once JobManager has been initialized.", jobId); + + // cancel job + jobManagerRunnerFuture + .thenCompose(jobManagerRunner -> getJobMasterGateway()) + .thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT)) + .whenComplete((ignored, cancelThrowable) -> { + if (cancelThrowable != null) { + log.warn("Cancellation of job {} failed", jobId, cancelThrowable); + } + }); + jobStatus = DispatcherJobStatus.CANCELLING; + jobResultFuture.whenComplete(((archivedExecutionGraph, throwable) -> { + if (archivedExecutionGraph != null) { + jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED; + } + })); + return jobResultFuture.thenApply(ignored -> Acknowledge.get()); + } + } + } + + public CompletableFuture<JobStatus> requestJobStatus(Time timeout) { + return requestJob(timeout).thenApply(ArchivedExecutionGraph::getState); + } + + public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) { + synchronized (lock) { + if (isInitialized()) { + if (jobResultFuture.isDone()) { // job is not running anymore + return jobResultFuture; + } + return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob( + timeout)); + } else { + Preconditions.checkState(jobStatus == DispatcherJobStatus.INITIALIZING || jobStatus == DispatcherJobStatus.CANCELLING); + return CompletableFuture.supplyAsync(() -> { + synchronized (lock) { + return ArchivedExecutionGraph.createFromInitializingJob( + jobId, + jobName, + jobStatus.asJobStatus(), + null, + initializationTimestamp); + } + }); Review comment: Why are you using `supplyAsync` here? `createFromInitializingJob` should be non-blocking. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org