You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/08/21 12:53:27 UTC

[GitHub] [flink] rmetzger opened a new pull request #13217: [FLINK-16866] Make jobsubmission non-blocking

rmetzger opened a new pull request #13217:
URL: https://github.com/apache/flink/pull/13217


   ## What is the purpose of the change
   
   This is changing the semantics of the job submission: Instead of completing the `Dispatcher.submitJob()` future after all the initialization happened (which can potentially involve calling external systems etc.), the `.submitJob()` call now returns as soon as the job has been accepted by the Dispatcher.
   The benefit of this change is that the users will see the root cause of a submission timeout, instead of an akka.ask.timeout.
   
   ## Brief change log
   
   - Introduce a `DispatcherJob` abstraction that manages the job in a new `INITIALIZING` state
   - Change web frontend to cope with initializing jobs
   - change clients to submit & poll
   
   
   ## Verifying this change
   
   This PR introduces various new tests for verification.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no 
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: yes 🔥 
     - The S3 file system connector: no 
   
   ## Documentation
   
   This change is transparent to the user and doesn't need a documentation update.
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475667454



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}

Review comment:
       Well, you are right, the result will still be wrong. We need to delegate to the resultFuture when it's complete.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475437759



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -802,23 +769,28 @@ private void jobMasterFailed(JobID jobId, Throwable cause) {
 		onFatalError(new FlinkException(String.format("JobMaster for job %s failed.", jobId), cause));
 	}
 
-	private CompletableFuture<JobMasterGateway> getJobMasterGatewayFuture(JobID jobId) {
-		final CompletableFuture<JobManagerRunner> jobManagerRunnerFuture = jobManagerRunnerFutures.get(jobId);
+	/**
+	 * Ensures that the JobMasterGateway is available.
+	 */
+	private CompletableFuture<JobMasterGateway> getJobMasterGateway(JobID jobId) throws
+		FlinkJobNotFoundException,
+		UnavailableDispatcherOperationException {

Review comment:
       If this method wouldn't throw exceptions but only return exceptionally futures, then `performOperationOnJobMasterGateway` could be simplified.

##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,41 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+		SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+		SupplierWithException<JobResult, Exception> jobResultSupplier
+		) throws CompletionException {

Review comment:
       Why are we throwing a `CompletionException`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {

Review comment:
       Maybe
   
   ```suggestion
   	private enum ExecutionType {
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -682,15 +650,14 @@ private void registerJobManagerRunnerTerminationFuture(JobID jobId, CompletableF
 	}
 
 	private CompletableFuture<Void> removeJob(JobID jobId, boolean cleanupHA) {
-		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture = jobManagerRunnerFutures.remove(jobId);
+		DispatcherJob job = runningJobs.remove(jobId);
 
 		final CompletableFuture<Void> jobManagerRunnerTerminationFuture;

Review comment:
       Please update field names to reflect new abstraction. I think it should be the `jobTerminationFuture` or `dispatcherJobTerminationFuture` now.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */

Review comment:
       It is not mainly during initialization. The `DispatcherJob` is the basic abstraction of a Flink job seen from the `Dispatcher`. It has its own lifecycle and offers a set of operations.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();

Review comment:
       I would suggest to not mix final and non-final fields.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
##########
@@ -351,4 +352,52 @@ public static ArchivedExecutionGraph createFrom(ExecutionGraph executionGraph) {
 			executionGraph.getCheckpointStatsSnapshot(),
 			executionGraph.getStateBackendName().orElse(null));
 	}
+
+	/**
+	 * Create an ArchivedExecutionGraph from an initializing job.
+	 */
+	public static ArchivedExecutionGraph createFromInitializingJob(
+		JobID jobId,
+		String jobName,
+		@Nullable Throwable throwable,
+		JobStatus finalJobStatus,
+		long initializationTimestamp) {
+
+		long failureTime = System.currentTimeMillis();
+		Map<JobVertexID, ArchivedExecutionJobVertex> archivedTasks = Collections.emptyMap();
+		List<ArchivedExecutionJobVertex> archivedVerticesInCreationOrder = Collections.emptyList();
+		final Map<String, SerializedValue<OptionalFailure<Object>>> serializedUserAccumulators = Collections.emptyMap();
+		StringifiedAccumulatorResult[] archivedUserAccumulators = new StringifiedAccumulatorResult[]{};
+
+		final long[] timestamps = new long[JobStatus.values().length];
+		timestamps[JobStatus.CREATED.ordinal()] = initializationTimestamp;
+		timestamps[JobStatus.INITIALIZING.ordinal()] = initializationTimestamp;
+
+		String jsonPlan = "{}";
+
+		ErrorInfo failureInfo = null;
+		if (throwable != null) {
+			failureInfo = new ErrorInfo(throwable, failureTime);
+			timestamps[JobStatus.FAILED.ordinal()] = failureTime;

Review comment:
       I would suggest to assert that the `finalJobStatus` is `FAILED`.

##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,41 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+		SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+		SupplierWithException<JobResult, Exception> jobResultSupplier
+		) throws CompletionException {
+		LOG.debug("Wait until job initialization is finished");
+		WaitStrategy waitStrategy = new ExponentialWaitStrategy(50, 2000);
+		try {
+			JobStatus status = jobStatusSupplier.get();
+			long attempt = 0;
+			while (status == JobStatus.INITIALIZING) {
+				Thread.sleep(waitStrategy.sleepTime(attempt++));
+				status = jobStatusSupplier.get();
+			}
+			if (status == JobStatus.FAILED) {
+				// note: we can not distinguish between initialization failures and failures once
+				// execution has started. Execution errors are potentially reported here.
+				JobResult result = jobResultSupplier.get();
+				Optional<SerializedThrowable> throwable = result.getSerializedThrowable();
+				if (throwable.isPresent()) {
+					throw new CompletionException(throwable.get().deserializeError(Thread.currentThread().getContextClassLoader()));

Review comment:
       Is the context class loader strictly the user code class loader?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -350,71 +341,42 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
 		}, getRpcService().getExecutor());
 	}
 
-	private CompletableFuture<Void> persistAndRunJob(JobGraph jobGraph) throws Exception {
+	private Void persistAndRunJob(JobGraph jobGraph) throws Exception {
 		jobGraphWriter.putJobGraph(jobGraph);
-
-		final CompletableFuture<Void> runJobFuture = runJob(jobGraph);
-
-		return runJobFuture.whenComplete(BiConsumerWithException.unchecked((Object ignored, Throwable throwable) -> {
-			if (throwable != null) {
-				jobGraphWriter.removeJobGraph(jobGraph.getJobID());
-			}
-		}));
+		runJob(jobGraph, false);
+		return null;

Review comment:
       please remove

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -350,71 +341,42 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
 		}, getRpcService().getExecutor());
 	}
 
-	private CompletableFuture<Void> persistAndRunJob(JobGraph jobGraph) throws Exception {
+	private Void persistAndRunJob(JobGraph jobGraph) throws Exception {

Review comment:
       ```suggestion
   	private void persistAndRunJob(JobGraph jobGraph) throws Exception {
   ```

##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,41 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+		SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+		SupplierWithException<JobResult, Exception> jobResultSupplier
+		) throws CompletionException {

Review comment:
       Nit: this is a bit uncommon formatting in the Flink code base.
   
   ```suggestion
   	public static void waitUntilJobInitializationFinished(
   		    SupplierWithException<JobStatus, Exception> jobStatusSupplier,
   		    SupplierWithException<JobResult, Exception> jobResultSupplier) 
   		throws CompletionException {
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;

Review comment:
       I'd suggest to use a different type here because we are effectively limiting ourselves in the implementation to `INITIALIZING`, `CANCELLING` and `RUNNING` if I am not mistaken.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -350,71 +341,42 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
 		}, getRpcService().getExecutor());
 	}
 
-	private CompletableFuture<Void> persistAndRunJob(JobGraph jobGraph) throws Exception {
+	private Void persistAndRunJob(JobGraph jobGraph) throws Exception {
 		jobGraphWriter.putJobGraph(jobGraph);
-
-		final CompletableFuture<Void> runJobFuture = runJob(jobGraph);
-
-		return runJobFuture.whenComplete(BiConsumerWithException.unchecked((Object ignored, Throwable throwable) -> {
-			if (throwable != null) {
-				jobGraphWriter.removeJobGraph(jobGraph.getJobID());
-			}
-		}));
+		runJob(jobGraph, false);
+		return null;
 	}
 
-	private CompletableFuture<Void> runJob(JobGraph jobGraph) {
-		Preconditions.checkState(!jobManagerRunnerFutures.containsKey(jobGraph.getJobID()));
-
-		final CompletableFuture<JobManagerRunner> jobManagerRunnerFuture = createJobManagerRunner(jobGraph);
+	private void runJob(JobGraph jobGraph, boolean isRecovery) {

Review comment:
       It is easier for maintenance purposes to introduce an enum instead of boolean parameters or to introduce a new method name (e.g. `runRecoveredJob` and `runSubmittedJob`) to give more meaning and context.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -732,7 +699,7 @@ private void cleanUpJobData(JobID jobId, boolean cleanupHA) {
 	private void terminateJobManagerRunners() {

Review comment:
       Please update method names with reference to the `JobManagerRunner` throughout the `Dispatcher` class.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -802,23 +769,28 @@ private void jobMasterFailed(JobID jobId, Throwable cause) {
 		onFatalError(new FlinkException(String.format("JobMaster for job %s failed.", jobId), cause));
 	}
 
-	private CompletableFuture<JobMasterGateway> getJobMasterGatewayFuture(JobID jobId) {
-		final CompletableFuture<JobManagerRunner> jobManagerRunnerFuture = jobManagerRunnerFutures.get(jobId);
+	/**
+	 * Ensures that the JobMasterGateway is available.
+	 */
+	private CompletableFuture<JobMasterGateway> getJobMasterGateway(JobID jobId) throws
+		FlinkJobNotFoundException,
+		UnavailableDispatcherOperationException {
+		DispatcherJob job = runningJobs.get(jobId);
+		if (job == null) {
+			throw new FlinkJobNotFoundException(jobId);
+		}
+		if (!job.isRunning()) {
+			throw new UnavailableDispatcherOperationException("Unable to get JobMasterGateway for initializing job. "
+				+ "The requested operation is not available while the JobManager is still initializing.");

Review comment:
       ```suggestion
   				+ "The requested operation is not available while the JobManager is initializing.");
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -427,21 +389,39 @@ private JobManagerRunner startJobManagerRunner(JobManagerRunner jobManagerRunner
 							}
 						}
 					} else {
-						log.debug("There is a newer JobManagerRunner for the job {}.", jobId);
+						log.debug("Job {} is not registered anymore at dispatcher", jobId);
 					}
-
 					return null;
 				}, getMainThreadExecutor()));
+	}
 
-		jobManagerRunner.start();
-
-		return jobManagerRunner;
+	CompletableFuture<JobManagerRunner> createJobManagerRunner(JobGraph jobGraph) {
+		final RpcService rpcService = getRpcService();
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					JobManagerRunner runner = jobManagerRunnerFactory.createJobManagerRunner(
+						jobGraph,
+						configuration,
+						rpcService,
+						highAvailabilityServices,
+						heartbeatServices,
+						jobManagerSharedServices,
+						new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup),
+						fatalErrorHandler);
+					runner.start();
+					return runner;
+				} catch (Exception e) {
+					throw new CompletionException(new JobExecutionException(jobGraph.getJobID(), "Could not instantiate JobManager.", e));
+				}
+			},
+			rpcService.getExecutor()); // do not use main thread executor. Otherwise, Dispatcher is blocked on JobManager creation

Review comment:
       Let's file a follow up issue to replace `rpcService.getExecutor` with a proper `ioExecutor`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY

Review comment:
       ```suggestion
   		SUBMISSION, RECOVERY
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;

Review comment:
       `jobStatus` is accessed outside of `lock` in the method `requestJob`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});

Review comment:
       What happens if I call `dispatcherJob.requestJob` after the job has been cancelled in state `INITIALIZING`? I would like to obtain the final `CANCELED` `ArchivedExecutionGraph` ideally.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));

Review comment:
       factor out into `createInitializingJobDetails`

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}
+		}
+	}
+
+	public boolean isRunning() {
+		synchronized (lock) {
+			return jobStatus == JobStatus.RUNNING;
+		}
+	}
+
+	public CompletableFuture<JobMasterGateway> getJobMasterGateway() {
+		Preconditions.checkState(
+			isRunning(),
+			"JobMaster Gateway is not available during initialization");
+		try {
+			return jobManagerRunnerFuture.thenCompose(JobManagerRunner::getJobMasterGateway);
+		} catch (Throwable e) {
+			throw new IllegalStateException("JobMaster gateway is not available", e);
+		}

Review comment:
       Why is this catch block necessary?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}
+		}
+	}
+
+	public boolean isRunning() {
+		synchronized (lock) {
+			return jobStatus == JobStatus.RUNNING;
+		}
+	}
+
+	public CompletableFuture<JobMasterGateway> getJobMasterGateway() {
+		Preconditions.checkState(
+			isRunning(),
+			"JobMaster Gateway is not available during initialization");
+		try {
+			return jobManagerRunnerFuture.thenCompose(JobManagerRunner::getJobMasterGateway);
+		} catch (Throwable e) {
+			throw new IllegalStateException("JobMaster gateway is not available", e);
+		}
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {

Review comment:
       What is the status of a `DispatcherJob` after `closeAsync` is being called? More concretely, what happens if `jobStatus` is `INITIALIZING`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}
+		}
+	}
+
+	public boolean isRunning() {
+		synchronized (lock) {
+			return jobStatus == JobStatus.RUNNING;
+		}
+	}
+
+	public CompletableFuture<JobMasterGateway> getJobMasterGateway() {
+		Preconditions.checkState(
+			isRunning(),
+			"JobMaster Gateway is not available during initialization");
+		try {
+			return jobManagerRunnerFuture.thenCompose(JobManagerRunner::getJobMasterGateway);
+		} catch (Throwable e) {
+			throw new IllegalStateException("JobMaster gateway is not available", e);
+		}
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		jobManagerRunnerFuture.handle((runner, throwable) -> {

Review comment:
       `FutureUtils.assertNoException` missing

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}

Review comment:
       I think this will return the wrong `ArchivedExecutionGraph` after the job has reached a globally terminal state.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));

Review comment:
       What `requestJobStatus` return in this case?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
##########
@@ -351,4 +352,52 @@ public static ArchivedExecutionGraph createFrom(ExecutionGraph executionGraph) {
 			executionGraph.getCheckpointStatsSnapshot(),
 			executionGraph.getStateBackendName().orElse(null));
 	}
+
+	/**
+	 * Create an ArchivedExecutionGraph from an initializing job.
+	 */
+	public static ArchivedExecutionGraph createFromInitializingJob(
+		JobID jobId,
+		String jobName,
+		@Nullable Throwable throwable,
+		JobStatus finalJobStatus,

Review comment:
       Maybe it is just me, but I think the oder of the arguments are a bit arbitrary. I would expect that the `JobStatus` comes before a potential failure cause.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});

Review comment:
       Why is it beneficial to wait for the `JobManagerRunner` to complete instead of enqueuing this operation when `cancel` is being called?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();

Review comment:
       It seems a bit inconsistent to set the `jobStatus` to the terminal state only in the case where we cancel the job during the `initialization` phase.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
##########
@@ -351,4 +352,52 @@ public static ArchivedExecutionGraph createFrom(ExecutionGraph executionGraph) {
 			executionGraph.getCheckpointStatsSnapshot(),
 			executionGraph.getStateBackendName().orElse(null));
 	}
+
+	/**
+	 * Create an ArchivedExecutionGraph from an initializing job.
+	 */

Review comment:
       Please add proper JavaDocs here

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}
+		}
+	}
+
+	public boolean isRunning() {
+		synchronized (lock) {
+			return jobStatus == JobStatus.RUNNING;
+		}
+	}
+
+	public CompletableFuture<JobMasterGateway> getJobMasterGateway() {

Review comment:
       Please add a description of the contract of this method (e.g. that it must be `running` otherwise it fails with an `IllegalStateException`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
##########
@@ -351,4 +352,52 @@ public static ArchivedExecutionGraph createFrom(ExecutionGraph executionGraph) {
 			executionGraph.getCheckpointStatsSnapshot(),
 			executionGraph.getStateBackendName().orElse(null));
 	}
+
+	/**
+	 * Create an ArchivedExecutionGraph from an initializing job.
+	 */
+	public static ArchivedExecutionGraph createFromInitializingJob(
+		JobID jobId,
+		String jobName,
+		@Nullable Throwable throwable,
+		JobStatus finalJobStatus,
+		long initializationTimestamp) {
+
+		long failureTime = System.currentTimeMillis();
+		Map<JobVertexID, ArchivedExecutionJobVertex> archivedTasks = Collections.emptyMap();
+		List<ArchivedExecutionJobVertex> archivedVerticesInCreationOrder = Collections.emptyList();
+		final Map<String, SerializedValue<OptionalFailure<Object>>> serializedUserAccumulators = Collections.emptyMap();
+		StringifiedAccumulatorResult[] archivedUserAccumulators = new StringifiedAccumulatorResult[]{};
+
+		final long[] timestamps = new long[JobStatus.values().length];
+		timestamps[JobStatus.CREATED.ordinal()] = initializationTimestamp;

Review comment:
       Why is this correct?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}
+		}
+	}
+
+	public boolean isRunning() {
+		synchronized (lock) {
+			return jobStatus == JobStatus.RUNNING;
+		}
+	}
+
+	public CompletableFuture<JobMasterGateway> getJobMasterGateway() {
+		Preconditions.checkState(
+			isRunning(),
+			"JobMaster Gateway is not available during initialization");
+		try {
+			return jobManagerRunnerFuture.thenCompose(JobManagerRunner::getJobMasterGateway);
+		} catch (Throwable e) {
+			throw new IllegalStateException("JobMaster gateway is not available", e);
+		}
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		jobManagerRunnerFuture.handle((runner, throwable) -> {
+			if (throwable == null) {
+				// init was successful: close jobManager runner.
+				CompletableFuture<Void> jobManagerRunnerClose = jobManagerRunnerFuture.thenCompose(
+					AutoCloseableAsync::closeAsync);
+				FutureUtils.forward(jobManagerRunnerClose, terminationFuture);
+			} else {
+				// initialization has failed. Termination complete.

Review comment:
       Why are we not forwarding the failure to the `terminationFuture`?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r478416484



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static junit.framework.Assert.fail;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Test for the {@link DispatcherJob} class.
+ */
+public class DispatcherJobTest extends TestLogger {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	@Test
+	public void testStatusWhenInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+	}
+
+	@Test
+	public void testStatusWhenRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish initialization
+		testContext.setRunning();
+
+		assertJobStatus(dispatcherJob, JobStatus.RUNNING);
+
+		// result future not done
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+	}
+
+	@Test
+	public void testStatusWhenJobFinished() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish job
+		testContext.setRunning();
+		testContext.finishJob();
+
+		assertJobStatus(dispatcherJob, JobStatus.FINISHED);
+
+		// assert result future done
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+
+		Assert.assertThat(aeg.getState(), is(JobStatus.FINISHED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(
+			TIMEOUT);
+
+		Assert.assertThat(cancelFuture.isDone(), is(false));
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+
+		testContext.setRunning();
+		testContext.finishCancellation();
+
+		// assert that cancel future completes
+		cancelFuture.get();
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		// assert that the result future completes
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		testContext.setRunning();
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(TIMEOUT);
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+		testContext.finishCancellation();
+
+		cancelFuture.get();
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileFailed() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		CommonTestUtils.assertThrows("Artificial failure", ExecutionException.class, () -> dispatcherJob.cancel(TIMEOUT).get());
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+	}
+
+	@Test
+	public void testErrorWhileInitializing() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.dispatcherJob;
+
+		// now fail
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+		Assert.assertThat(aeg.getFailureInfo().getException().deserializeError(ClassLoader.getSystemClassLoader()), is(exception));
+
+		Assert.assertTrue(dispatcherJob.closeAsync().isDone() && dispatcherJob.closeAsync().isCompletedExceptionally());
+	}
+
+	@Test
+	public void testCloseWhileInitializingSuccessfully() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		// set job running, so that we can cancel it
+		testContext.setRunning();
+
+		// assert future completes now
+		closeFuture.get();
+
+		// ensure the result future is complete (how it completes is up to the JobManager)
+		CompletableFuture<ArchivedExecutionGraph> resultFuture = dispatcherJob.getResultFuture();
+		CommonTestUtils.assertThrows("has not been finished", ExecutionException.class,
+			resultFuture::get);
+	}
+
+	@Test
+	public void testCloseWhileInitializingErroneously() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		testContext.failInitialization(new RuntimeException("fail"));
+
+		// assert future completes now
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			closeFuture::get);
+
+		// ensure the result future is complete
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.FAILED));
+	}
+
+	@Test
+	public void testCloseWhileInitializingErroneouslyForRecovery() {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.RECOVERY);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+
+		testContext.failInitialization(new RuntimeException("fail"));
+
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			closeFuture::get);
+		// ensure the result future is completing exceptionally when using RECOVERY execution
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			() -> dispatcherJob.getResultFuture().get());
+	}
+
+	@Test
+	public void testCloseWhileRunning() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// complete JobManager runner future to indicate to the DispatcherJob that the Runner has been initialized
+		testContext.setRunning();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+
+		closeFuture.get();
+
+		// result future should complete exceptionally.
+		CompletableFuture<ArchivedExecutionGraph> resultFuture = dispatcherJob.getResultFuture();
+		CommonTestUtils.assertThrows("has not been finished", ExecutionException.class,
+			resultFuture::get);
+	}
+
+	@Test(expected = IllegalStateException.class)
+	public void testUnavailableJobMasterGateway() {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		dispatcherJob.getJobMasterGateway();
+	}
+
+	private TestContext createTestContext(Dispatcher.ExecutionType type) {
+		final JobVertex testVertex = new JobVertex("testVertex");
+		testVertex.setInvokableClass(NoOpInvokable.class);
+
+		JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex);
+		CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture = new CompletableFuture<>();
+		DispatcherJob dispatcherJob = DispatcherJob.createFor(jobManagerRunnerCompletableFuture,
+			jobGraph.getJobID(), jobGraph.getName(), type);
+
+		return new TestContext(
+			jobManagerRunnerCompletableFuture,
+			dispatcherJob,
+			jobGraph);
+	}
+
+	private static class TestContext {
+		private final CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture;
+		private final DispatcherJob dispatcherJob;
+		private final JobGraph jobGraph;
+		private final TestingJobMasterGateway mockRunningJobMasterGateway;
+		private final CompletableFuture<Acknowledge> cancellationFuture;
+
+		private JobStatus internalJobStatus = JobStatus.INITIALIZING;
+
+		public TestContext(
+			CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture,
+			DispatcherJob dispatcherJob,
+			JobGraph jobGraph) {
+			this.jobManagerRunnerCompletableFuture = jobManagerRunnerCompletableFuture;
+			this.dispatcherJob = dispatcherJob;
+			this.jobGraph = jobGraph;
+
+			this.cancellationFuture = new CompletableFuture<>();
+			this.mockRunningJobMasterGateway = new TestingJobMasterGatewayBuilder()
+				.setRequestJobSupplier(() -> CompletableFuture.completedFuture(ArchivedExecutionGraph.createFromInitializingJob(getJobID(), "test", internalJobStatus, null, 1337)))
+				.setRequestJobDetailsSupplier(() -> {
+					JobDetails jobDetails = new JobDetails(getJobID(), "", 0, 0, 0, internalJobStatus, 0,
+						new int[]{0, 0, 0, 0, 0, 0, 0, 0, 0}, 0);
+					return CompletableFuture.completedFuture(jobDetails);
+				})
+				// once JobManagerRunner is initialized, complete result future with CANCELLED AEG and ack cancellation.
+				.setCancelFunction(() -> {
+					internalJobStatus = JobStatus.CANCELLING;
+					return cancellationFuture;
+				})
+				.build();
+		}
+
+		public JobID getJobID() {
+			return jobGraph.getJobID();
+		}
+
+		public void failInitialization(Throwable ex) {
+			jobManagerRunnerCompletableFuture.completeExceptionally(ex);
+		}
+
+		public DispatcherJob getDispatcherJob() {
+			return dispatcherJob;
+		}
+
+		public void setRunning() {
+			internalJobStatus = JobStatus.RUNNING;
+			TestingJobManagerRunner jobManagerRunner =
+				new TestingJobManagerRunner(getJobID(), false);
+			jobManagerRunner.getJobMasterGateway().complete(mockRunningJobMasterGateway);
+			jobManagerRunnerCompletableFuture.complete(jobManagerRunner);
+		}
+
+		public void finishJob() {
+			try {
+				internalJobStatus = JobStatus.FINISHED;
+				jobManagerRunnerCompletableFuture.get().getResultFuture()
+					.complete(ArchivedExecutionGraph.createFromInitializingJob(getJobID(), "test", JobStatus.FINISHED, null, 1337));

Review comment:
       Same hack as in `setRunning`. I'd suggest to pass in a `jobResultFuture` to the `JobManagerRunner` which is created to complete `jobManagerRunnerCompletableFuture`.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 9127b879bec8dc1c05176c1acb89680c0e7baf04 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475636835



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}
+		}
+	}
+
+	public boolean isRunning() {
+		synchronized (lock) {
+			return jobStatus == JobStatus.RUNNING;
+		}
+	}
+
+	public CompletableFuture<JobMasterGateway> getJobMasterGateway() {
+		Preconditions.checkState(
+			isRunning(),
+			"JobMaster Gateway is not available during initialization");
+		try {
+			return jobManagerRunnerFuture.thenCompose(JobManagerRunner::getJobMasterGateway);
+		} catch (Throwable e) {
+			throw new IllegalStateException("JobMaster gateway is not available", e);
+		}
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {

Review comment:
       Unless we add a new `JobStatus.SUSPENDING` or `JobStatus.TERMINATING`, I don't see a good way of reporting to the user that we are waiting for the initialization to be finished so that we can close the JobManagerRunner.
   
   Afaik this PR is not changing the semantics of a job stuck INITIALIZING. Previously, a job would report `CREATED` or `RUNNING` until it reaches the final `SUSPENDED` state, if `Dispatcher.onStop` get's called.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8036016c752bce433dc65d1c08695377c917836f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6162",
       "triggerID" : "8036016c752bce433dc65d1c08695377c917836f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d4d6690d8ad282f71e5f3d890e28421aef17766",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6223",
       "triggerID" : "4d4d6690d8ad282f71e5f3d890e28421aef17766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 8036016c752bce433dc65d1c08695377c917836f Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6162) 
   * 4d4d6690d8ad282f71e5f3d890e28421aef17766 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6223) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-686343191


   @rmetzger when closing the corresponding ticket, please add a release note stating the difference in submission semantics. E.g. when submitting a job one cannot immediately trigger job operation (triggering of savepoint, etc.). This is maybe one follow up issue we might wanna think through and adjust if it should become a usability problem.
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475600543



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}
+		}
+	}
+
+	public boolean isRunning() {
+		synchronized (lock) {
+			return jobStatus == JobStatus.RUNNING;
+		}
+	}
+
+	public CompletableFuture<JobMasterGateway> getJobMasterGateway() {
+		Preconditions.checkState(
+			isRunning(),
+			"JobMaster Gateway is not available during initialization");
+		try {
+			return jobManagerRunnerFuture.thenCompose(JobManagerRunner::getJobMasterGateway);
+		} catch (Throwable e) {
+			throw new IllegalStateException("JobMaster gateway is not available", e);
+		}
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		jobManagerRunnerFuture.handle((runner, throwable) -> {
+			if (throwable == null) {
+				// init was successful: close jobManager runner.
+				CompletableFuture<Void> jobManagerRunnerClose = jobManagerRunnerFuture.thenCompose(
+					AutoCloseableAsync::closeAsync);
+				FutureUtils.forward(jobManagerRunnerClose, terminationFuture);
+			} else {
+				// initialization has failed. Termination complete.

Review comment:
       You are right. I will complete the terminationFuture exceptionally.
   I probably thought that the exception is already forwarded to the job result future, and handled accordingly.
   But it is more consistent with the other case (job is already running, need to close the job manager runner).)
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678283618


   Note: I'm still reviewing one last unstable test `FunctionITCase.testInvalidUseOfTableFunction()`


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r479921560



##########
File path: flink-clients/src/test/java/org/apache/flink/client/ClientUtilsTest.java
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.BlobServerOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.blob.BlobServer;
+import org.apache.flink.runtime.blob.VoidBlobStore;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.client.JobInitializationException;
+import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.DefaultDispatcherBootstrap;
+import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory;
+import org.apache.flink.runtime.dispatcher.Dispatcher;
+import org.apache.flink.runtime.dispatcher.DispatcherGateway;
+import org.apache.flink.runtime.dispatcher.DispatcherId;
+import org.apache.flink.runtime.dispatcher.DispatcherServices;
+import org.apache.flink.runtime.dispatcher.DispatcherTest;
+import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.NoOpJobGraphWriter;
+import org.apache.flink.runtime.dispatcher.TestingDispatcher;
+import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.JobGraphWriter;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcUtils;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Test for the ClientUtils.
+ */
+public class ClientUtilsTest extends TestLogger {
+
+	@ClassRule

Review comment:
       The reason why I'm using real components is that I want to ensure the right exception is thrown from the `Dispatcher`: If somebody stops throwing a `JobInitializationException` during initialization in the Dispatcher, the test for `waitUntilJobInitializationFinished` should fail.
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r479931866



##########
File path: flink-clients/src/test/java/org/apache/flink/client/ClientUtilsTest.java
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.BlobServerOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.blob.BlobServer;
+import org.apache.flink.runtime.blob.VoidBlobStore;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.client.JobInitializationException;
+import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.DefaultDispatcherBootstrap;
+import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory;
+import org.apache.flink.runtime.dispatcher.Dispatcher;
+import org.apache.flink.runtime.dispatcher.DispatcherGateway;
+import org.apache.flink.runtime.dispatcher.DispatcherId;
+import org.apache.flink.runtime.dispatcher.DispatcherServices;
+import org.apache.flink.runtime.dispatcher.DispatcherTest;
+import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.NoOpJobGraphWriter;
+import org.apache.flink.runtime.dispatcher.TestingDispatcher;
+import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.JobGraphWriter;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcUtils;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Test for the ClientUtils.
+ */
+public class ClientUtilsTest extends TestLogger {
+
+	@ClassRule

Review comment:
       This could also be tested in two steps: 1. test the `ClientUtils` in isolation. 2. test that the `Dispatcher` throws such an exception under the required conditions (w/o involving the `ClientUtils`).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * c52eaf11a6832049ad9a18c784d2661101d91b8c Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114) 
   * 31d810f7c4d93043d8444a78f921b263f1258646 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475542880



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -427,21 +389,39 @@ private JobManagerRunner startJobManagerRunner(JobManagerRunner jobManagerRunner
 							}
 						}
 					} else {
-						log.debug("There is a newer JobManagerRunner for the job {}.", jobId);
+						log.debug("Job {} is not registered anymore at dispatcher", jobId);
 					}
-
 					return null;
 				}, getMainThreadExecutor()));
+	}
 
-		jobManagerRunner.start();
-
-		return jobManagerRunner;
+	CompletableFuture<JobManagerRunner> createJobManagerRunner(JobGraph jobGraph) {
+		final RpcService rpcService = getRpcService();
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					JobManagerRunner runner = jobManagerRunnerFactory.createJobManagerRunner(
+						jobGraph,
+						configuration,
+						rpcService,
+						highAvailabilityServices,
+						heartbeatServices,
+						jobManagerSharedServices,
+						new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup),
+						fatalErrorHandler);
+					runner.start();
+					return runner;
+				} catch (Exception e) {
+					throw new CompletionException(new JobExecutionException(jobGraph.getJobID(), "Could not instantiate JobManager.", e));
+				}
+			},
+			rpcService.getExecutor()); // do not use main thread executor. Otherwise, Dispatcher is blocked on JobManager creation

Review comment:
       Done --> https://issues.apache.org/jira/browse/FLINK-19037




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * c741363eaee4cbff8edaa3cab54c3bfbdd668184 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872) 
   * 41769d3c4ebae61f502e16398207d477f59451f9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r478403026



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
##########
@@ -351,4 +352,53 @@ public static ArchivedExecutionGraph createFrom(ExecutionGraph executionGraph) {
 			executionGraph.getCheckpointStatsSnapshot(),
 			executionGraph.getStateBackendName().orElse(null));
 	}
+
+	/**
+	 * Create a sparse ArchivedExecutionGraph for a job while it is still initializing.
+	 * Most fields will be empty, only job status and error-related fields are set.
+	 */
+	public static ArchivedExecutionGraph createFromInitializingJob(
+		JobID jobId,
+		String jobName,
+		JobStatus jobStatus,
+		@Nullable Throwable throwable,
+		long initializationTimestamp) {
+
+		long failureTime = System.currentTimeMillis();

Review comment:
       Move this into the if branch.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static junit.framework.Assert.fail;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Test for the {@link DispatcherJob} class.
+ */
+public class DispatcherJobTest extends TestLogger {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	@Test
+	public void testStatusWhenInitializing() throws
+		Exception {

Review comment:
       nit: this is just a personal question of taste: I think I would leave `Exception` on the same line as `throws`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static junit.framework.Assert.fail;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Test for the {@link DispatcherJob} class.
+ */
+public class DispatcherJobTest extends TestLogger {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	@Test
+	public void testStatusWhenInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+	}
+
+	@Test
+	public void testStatusWhenRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish initialization
+		testContext.setRunning();
+
+		assertJobStatus(dispatcherJob, JobStatus.RUNNING);
+
+		// result future not done
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+	}
+
+	@Test
+	public void testStatusWhenJobFinished() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish job
+		testContext.setRunning();
+		testContext.finishJob();
+
+		assertJobStatus(dispatcherJob, JobStatus.FINISHED);
+
+		// assert result future done
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+
+		Assert.assertThat(aeg.getState(), is(JobStatus.FINISHED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(
+			TIMEOUT);
+
+		Assert.assertThat(cancelFuture.isDone(), is(false));
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+
+		testContext.setRunning();
+		testContext.finishCancellation();
+
+		// assert that cancel future completes
+		cancelFuture.get();
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		// assert that the result future completes
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		testContext.setRunning();
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(TIMEOUT);
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+		testContext.finishCancellation();
+
+		cancelFuture.get();
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileFailed() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		CommonTestUtils.assertThrows("Artificial failure", ExecutionException.class, () -> dispatcherJob.cancel(TIMEOUT).get());
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+	}
+
+	@Test
+	public void testErrorWhileInitializing() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.dispatcherJob;
+
+		// now fail
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+		Assert.assertThat(aeg.getFailureInfo().getException().deserializeError(ClassLoader.getSystemClassLoader()), is(exception));
+
+		Assert.assertTrue(dispatcherJob.closeAsync().isDone() && dispatcherJob.closeAsync().isCompletedExceptionally());
+	}
+
+	@Test
+	public void testCloseWhileInitializingSuccessfully() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		// set job running, so that we can cancel it
+		testContext.setRunning();
+
+		// assert future completes now
+		closeFuture.get();
+
+		// ensure the result future is complete (how it completes is up to the JobManager)
+		CompletableFuture<ArchivedExecutionGraph> resultFuture = dispatcherJob.getResultFuture();
+		CommonTestUtils.assertThrows("has not been finished", ExecutionException.class,
+			resultFuture::get);
+	}
+
+	@Test
+	public void testCloseWhileInitializingErroneously() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		testContext.failInitialization(new RuntimeException("fail"));
+
+		// assert future completes now
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			closeFuture::get);
+
+		// ensure the result future is complete
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.FAILED));
+	}
+
+	@Test
+	public void testCloseWhileInitializingErroneouslyForRecovery() {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.RECOVERY);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+
+		testContext.failInitialization(new RuntimeException("fail"));
+
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			closeFuture::get);
+		// ensure the result future is completing exceptionally when using RECOVERY execution
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			() -> dispatcherJob.getResultFuture().get());
+	}
+
+	@Test
+	public void testCloseWhileRunning() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// complete JobManager runner future to indicate to the DispatcherJob that the Runner has been initialized
+		testContext.setRunning();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+
+		closeFuture.get();
+
+		// result future should complete exceptionally.
+		CompletableFuture<ArchivedExecutionGraph> resultFuture = dispatcherJob.getResultFuture();
+		CommonTestUtils.assertThrows("has not been finished", ExecutionException.class,
+			resultFuture::get);
+	}
+
+	@Test(expected = IllegalStateException.class)
+	public void testUnavailableJobMasterGateway() {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		dispatcherJob.getJobMasterGateway();
+	}
+
+	private TestContext createTestContext(Dispatcher.ExecutionType type) {
+		final JobVertex testVertex = new JobVertex("testVertex");
+		testVertex.setInvokableClass(NoOpInvokable.class);
+
+		JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex);
+		CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture = new CompletableFuture<>();
+		DispatcherJob dispatcherJob = DispatcherJob.createFor(jobManagerRunnerCompletableFuture,
+			jobGraph.getJobID(), jobGraph.getName(), type);
+
+		return new TestContext(
+			jobManagerRunnerCompletableFuture,
+			dispatcherJob,
+			jobGraph);
+	}
+
+	private static class TestContext {
+		private final CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture;
+		private final DispatcherJob dispatcherJob;
+		private final JobGraph jobGraph;
+		private final TestingJobMasterGateway mockRunningJobMasterGateway;
+		private final CompletableFuture<Acknowledge> cancellationFuture;
+
+		private JobStatus internalJobStatus = JobStatus.INITIALIZING;
+
+		public TestContext(
+			CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture,
+			DispatcherJob dispatcherJob,
+			JobGraph jobGraph) {
+			this.jobManagerRunnerCompletableFuture = jobManagerRunnerCompletableFuture;
+			this.dispatcherJob = dispatcherJob;
+			this.jobGraph = jobGraph;
+
+			this.cancellationFuture = new CompletableFuture<>();
+			this.mockRunningJobMasterGateway = new TestingJobMasterGatewayBuilder()
+				.setRequestJobSupplier(() -> CompletableFuture.completedFuture(ArchivedExecutionGraph.createFromInitializingJob(getJobID(), "test", internalJobStatus, null, 1337)))
+				.setRequestJobDetailsSupplier(() -> {
+					JobDetails jobDetails = new JobDetails(getJobID(), "", 0, 0, 0, internalJobStatus, 0,
+						new int[]{0, 0, 0, 0, 0, 0, 0, 0, 0}, 0);
+					return CompletableFuture.completedFuture(jobDetails);
+				})
+				// once JobManagerRunner is initialized, complete result future with CANCELLED AEG and ack cancellation.
+				.setCancelFunction(() -> {
+					internalJobStatus = JobStatus.CANCELLING;
+					return cancellationFuture;
+				})
+				.build();
+		}
+
+		public JobID getJobID() {
+			return jobGraph.getJobID();
+		}
+
+		public void failInitialization(Throwable ex) {
+			jobManagerRunnerCompletableFuture.completeExceptionally(ex);
+		}
+
+		public DispatcherJob getDispatcherJob() {
+			return dispatcherJob;
+		}
+
+		public void setRunning() {
+			internalJobStatus = JobStatus.RUNNING;
+			TestingJobManagerRunner jobManagerRunner =
+				new TestingJobManagerRunner(getJobID(), false);
+			jobManagerRunner.getJobMasterGateway().complete(mockRunningJobMasterGateway);
+			jobManagerRunnerCompletableFuture.complete(jobManagerRunner);
+		}
+
+		public void finishJob() {
+			try {
+				internalJobStatus = JobStatus.FINISHED;
+				jobManagerRunnerCompletableFuture.get().getResultFuture()

Review comment:
       `CompletableFuture.join()` throws an unchecked exception which might make this a bit easier here.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -544,10 +705,13 @@ public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception {
 			.build();
 
 		dispatcher.start();
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
 
 		final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
-		final CompletableFuture<Acknowledge> submissionFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT);
-		submissionFuture.get();
+		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
+
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get());

Review comment:
       Why do we have to wait for this here?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -329,6 +345,125 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception {
 		}
 	}
 
+	@Test(timeout = 5_000L)
+	public void testNonBlockingJobSubmission() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		// ensure INITIALIZING status from status
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT);
+		Assert.assertEquals(JobStatus.INITIALIZING, jobStatusFuture.get());
+
+		// ensure correct JobDetails
+		MultipleJobsDetails multiDetails = dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get();
+		Assert.assertEquals(1, multiDetails.getJobs().size());
+		Assert.assertEquals(blockingJobGraph.getJobID(), multiDetails.getJobs().iterator().next().getJobId());
+
+		// submission has succeeded, let the initialization finish.
+		blockingJobVertex.unblock();
+
+		// wait till job is running
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT).get());
+
+		dispatcherGateway.cancelJob(blockingJobGraph.getJobID(), TIMEOUT).get();
+	}
+
+	@Test(timeout = 5_000L)
+	public void testInvalidCallDuringInitialization() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		// ensure INITIALIZING status
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT);
+		Assert.assertEquals(JobStatus.INITIALIZING, jobStatusFuture.get());
+
+		// this call is supposed to fail
+		boolean exceptionSeen = false;

Review comment:
       by adding a `fail` after `dG.triggerSavepoint`, we could get rid of `exceptionSeen`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java
##########
@@ -31,11 +31,11 @@
 /**
  * {@link Dispatcher} implementation used for testing purposes.
  */
-class TestingDispatcher extends Dispatcher {
+public class TestingDispatcher extends Dispatcher {

Review comment:
       Please revert after updating the `ClientUtilsTest`.

##########
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,
+		// then we consider the JobManager to be initialized.
+		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 (jobStatus != DispatcherJobStatus.CANCELLING) {
+					jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+				}
+				if (throwable == null) {
+					// Forward result future
+					FutureUtils.forward(jobManagerRunner.getResultFuture(), jobResultFuture);
+				} 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) {
+		return requestJobStatus(timeout).thenApply(status -> {
+			int[] tasksPerState = new int[ExecutionState.values().length];
+			synchronized (lock) {
+				return new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					status,
+					0,
+					tasksPerState,
+					0);
+			}
+		});
+	}
+
+	/**
+	 * Cancel job.
+	 * A cancellation will be scheduled if the initialization is not completed.
+	 * The returned future will complete exceptionally if the JobManagerRunner initialization failed.
+	 */
+	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(jobMasterGateway -> jobMasterGateway.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);
+	}
+
+	/**
+	 * Returns a future completing to the ArchivedExecutionGraph of the job.
+	 */
+	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(this.jobStatus == DispatcherJobStatus.INITIALIZING || jobStatus == DispatcherJobStatus.CANCELLING);
+				return CompletableFuture.completedFuture(
+					ArchivedExecutionGraph.createFromInitializingJob(
+						jobId,
+						jobName,
+						jobStatus.asJobStatus(),
+						null,
+						initializationTimestamp));
+			}
+		}
+	}
+
+	/**
+	 * The job is initialized once the JobManager runner has been initialized.
+	 * It is also initialized if the runner initialization failed, or of it has been
+	 * canceled (and the cancellation is complete).
+	 */
+	public boolean isInitialized() {
+		synchronized (lock) {
+			return jobStatus == DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+		}
+	}
+
+	/**
+	 * Returns the {@link JobMasterGateway} from the JobManagerRunner.
+	 * This method will fail with an {@link IllegalStateException} if the job is initialized.
+	 * The returned future will complete exceptionally if the JobManagerRunner initialization failed.
+	 * @return the {@link JobMasterGateway}.
+	 */
+	public CompletableFuture<JobMasterGateway> getJobMasterGateway() {
+		Preconditions.checkState(
+			isInitialized(),
+			"JobMaster Gateway is not available during initialization");
+		return jobManagerRunnerFuture.thenCompose(JobManagerRunner::getJobMasterGateway);
+	}
+
+	@Override
+	public CompletableFuture<Void> closeAsync() {
+		FutureUtils.assertNoException(jobManagerRunnerFuture.handle((runner, throwable) -> {
+			if (throwable == null) {
+				// init was successful: close jobManager runner.
+				CompletableFuture<Void> jobManagerRunnerClose = jobManagerRunnerFuture.thenCompose(
+					AutoCloseableAsync::closeAsync);
+				FutureUtils.forward(jobManagerRunnerClose, terminationFuture);
+			} else {
+				// initialization has failed: forward failure.
+				terminationFuture.completeExceptionally(throwable);

Review comment:
       I might have given you bad advice here. Forwarding an initialization problem which is not an abnormal condition for the `Dispatcher` might lead to a failure reported during the shut down of the `Dispatcher`. I think it is fine to say that a failed job manager initialization is an acceptable condition and, hence, to simply complete the `terminationFuture` here.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -329,6 +345,125 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception {
 		}
 	}
 
+	@Test(timeout = 5_000L)
+	public void testNonBlockingJobSubmission() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		// ensure INITIALIZING status from status
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT);
+		Assert.assertEquals(JobStatus.INITIALIZING, jobStatusFuture.get());
+
+		// ensure correct JobDetails
+		MultipleJobsDetails multiDetails = dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get();
+		Assert.assertEquals(1, multiDetails.getJobs().size());
+		Assert.assertEquals(blockingJobGraph.getJobID(), multiDetails.getJobs().iterator().next().getJobId());
+
+		// submission has succeeded, let the initialization finish.
+		blockingJobVertex.unblock();
+
+		// wait till job is running
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT).get());
+
+		dispatcherGateway.cancelJob(blockingJobGraph.getJobID(), TIMEOUT).get();
+	}
+
+	@Test(timeout = 5_000L)
+	public void testInvalidCallDuringInitialization() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);

Review comment:
       Duplicated code. I'd suggest to create a method which can be reused.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -835,51 +796,50 @@ private void jobMasterFailed(JobID jobId, Throwable cause) {
 	}
 
 	@Nonnull
-	private <T> List<CompletableFuture<Optional<T>>> queryJobMastersForInformation(Function<JobMasterGateway, CompletableFuture<T>> queryFunction) {
-		final int numberJobsRunning = jobManagerRunnerFutures.size();
+	private <T> List<CompletableFuture<Optional<T>>> queryJobMastersForInformation(Function<DispatcherJob, CompletableFuture<T>> queryFunction) {
 
 		ArrayList<CompletableFuture<Optional<T>>> optionalJobInformation = new ArrayList<>(
-			numberJobsRunning);
+			runningJobs.size());
 
-		for (JobID jobId : jobManagerRunnerFutures.keySet()) {
-			final CompletableFuture<JobMasterGateway> jobMasterGatewayFuture = getJobMasterGatewayFuture(jobId);
-
-			final CompletableFuture<Optional<T>> optionalRequest = jobMasterGatewayFuture
-				.thenCompose(queryFunction::apply)
-				.handle((T value, Throwable throwable) -> Optional.ofNullable(value));
-
-			optionalJobInformation.add(optionalRequest);
+		for (DispatcherJob job : runningJobs.values()) {
+			final CompletableFuture<Optional<T>> queryResult = queryFunction.apply(job)
+					.handle((T value, Throwable t) -> Optional.ofNullable(value));
+			optionalJobInformation.add(queryResult);
 		}
 		return optionalJobInformation;
 	}
 
-	private CompletableFuture<Void> waitForTerminatingJobManager(JobID jobId, JobGraph jobGraph, FunctionWithException<JobGraph, CompletableFuture<Void>, ?> action) {
+	private CompletableFuture<Void> waitForTerminatingJobManager(JobID jobId, JobGraph jobGraph, ThrowingConsumer<JobGraph, ?> action) {
 		final CompletableFuture<Void> jobManagerTerminationFuture = getJobTerminationFuture(jobId)
 			.exceptionally((Throwable throwable) -> {
-				throw new CompletionException(
-					new DispatcherException(
-						String.format("Termination of previous JobManager for job %s failed. Cannot submit job under the same job id.", jobId),
-						throwable)); });
-
-		return jobManagerTerminationFuture.thenComposeAsync(
-			FunctionUtils.uncheckedFunction((ignored) -> {
-				jobManagerTerminationFutures.remove(jobId);
-				return action.apply(jobGraph);
+				if (!ExceptionUtils.findThrowable(throwable, JobInitializationException.class).isPresent()) {
+					throw new CompletionException(
+						new DispatcherException(
+							String.format("Termination of previous JobManager for job %s failed. Cannot submit job under the same job id.", jobId),
+							throwable));
+				}

Review comment:
       If we don't complete the `DispatcherJob.terminationFuture` with failures from the `JobManagerRunner`, then we don't have do to this special casing here.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static junit.framework.Assert.fail;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Test for the {@link DispatcherJob} class.
+ */
+public class DispatcherJobTest extends TestLogger {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	@Test
+	public void testStatusWhenInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+	}
+
+	@Test
+	public void testStatusWhenRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish initialization
+		testContext.setRunning();
+
+		assertJobStatus(dispatcherJob, JobStatus.RUNNING);
+
+		// result future not done
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+	}
+
+	@Test
+	public void testStatusWhenJobFinished() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish job
+		testContext.setRunning();
+		testContext.finishJob();
+
+		assertJobStatus(dispatcherJob, JobStatus.FINISHED);
+
+		// assert result future done
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+
+		Assert.assertThat(aeg.getState(), is(JobStatus.FINISHED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(
+			TIMEOUT);
+
+		Assert.assertThat(cancelFuture.isDone(), is(false));
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+
+		testContext.setRunning();
+		testContext.finishCancellation();
+
+		// assert that cancel future completes
+		cancelFuture.get();
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		// assert that the result future completes
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		testContext.setRunning();
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(TIMEOUT);
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+		testContext.finishCancellation();
+
+		cancelFuture.get();
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileFailed() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		CommonTestUtils.assertThrows("Artificial failure", ExecutionException.class, () -> dispatcherJob.cancel(TIMEOUT).get());
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+	}
+
+	@Test
+	public void testErrorWhileInitializing() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.dispatcherJob;
+
+		// now fail
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+		Assert.assertThat(aeg.getFailureInfo().getException().deserializeError(ClassLoader.getSystemClassLoader()), is(exception));
+
+		Assert.assertTrue(dispatcherJob.closeAsync().isDone() && dispatcherJob.closeAsync().isCompletedExceptionally());

Review comment:
       I am not sure whether this should be the case.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -329,6 +345,125 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception {
 		}
 	}
 
+	@Test(timeout = 5_000L)
+	public void testNonBlockingJobSubmission() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		// ensure INITIALIZING status from status
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT);
+		Assert.assertEquals(JobStatus.INITIALIZING, jobStatusFuture.get());
+
+		// ensure correct JobDetails
+		MultipleJobsDetails multiDetails = dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get();
+		Assert.assertEquals(1, multiDetails.getJobs().size());
+		Assert.assertEquals(blockingJobGraph.getJobID(), multiDetails.getJobs().iterator().next().getJobId());
+
+		// submission has succeeded, let the initialization finish.
+		blockingJobVertex.unblock();
+
+		// wait till job is running
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT).get());
+
+		dispatcherGateway.cancelJob(blockingJobGraph.getJobID(), TIMEOUT).get();

Review comment:
       What does this call test?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static junit.framework.Assert.fail;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Test for the {@link DispatcherJob} class.
+ */
+public class DispatcherJobTest extends TestLogger {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	@Test
+	public void testStatusWhenInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+	}
+
+	@Test
+	public void testStatusWhenRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish initialization
+		testContext.setRunning();
+
+		assertJobStatus(dispatcherJob, JobStatus.RUNNING);
+
+		// result future not done
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+	}
+
+	@Test
+	public void testStatusWhenJobFinished() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish job
+		testContext.setRunning();
+		testContext.finishJob();
+
+		assertJobStatus(dispatcherJob, JobStatus.FINISHED);
+
+		// assert result future done
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+
+		Assert.assertThat(aeg.getState(), is(JobStatus.FINISHED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(
+			TIMEOUT);
+
+		Assert.assertThat(cancelFuture.isDone(), is(false));
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+
+		testContext.setRunning();
+		testContext.finishCancellation();
+
+		// assert that cancel future completes
+		cancelFuture.get();
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		// assert that the result future completes
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		testContext.setRunning();
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(TIMEOUT);
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+		testContext.finishCancellation();
+
+		cancelFuture.get();
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileFailed() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		CommonTestUtils.assertThrows("Artificial failure", ExecutionException.class, () -> dispatcherJob.cancel(TIMEOUT).get());
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+	}
+
+	@Test
+	public void testErrorWhileInitializing() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.dispatcherJob;
+
+		// now fail
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+		Assert.assertThat(aeg.getFailureInfo().getException().deserializeError(ClassLoader.getSystemClassLoader()), is(exception));
+
+		Assert.assertTrue(dispatcherJob.closeAsync().isDone() && dispatcherJob.closeAsync().isCompletedExceptionally());
+	}
+
+	@Test
+	public void testCloseWhileInitializingSuccessfully() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		// set job running, so that we can cancel it
+		testContext.setRunning();
+
+		// assert future completes now
+		closeFuture.get();
+
+		// ensure the result future is complete (how it completes is up to the JobManager)
+		CompletableFuture<ArchivedExecutionGraph> resultFuture = dispatcherJob.getResultFuture();
+		CommonTestUtils.assertThrows("has not been finished", ExecutionException.class,
+			resultFuture::get);
+	}
+
+	@Test
+	public void testCloseWhileInitializingErroneously() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		testContext.failInitialization(new RuntimeException("fail"));
+
+		// assert future completes now
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			closeFuture::get);
+
+		// ensure the result future is complete
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.FAILED));
+	}
+
+	@Test
+	public void testCloseWhileInitializingErroneouslyForRecovery() {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.RECOVERY);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+
+		testContext.failInitialization(new RuntimeException("fail"));
+
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			closeFuture::get);
+		// ensure the result future is completing exceptionally when using RECOVERY execution
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			() -> dispatcherJob.getResultFuture().get());
+	}
+
+	@Test
+	public void testCloseWhileRunning() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// complete JobManager runner future to indicate to the DispatcherJob that the Runner has been initialized
+		testContext.setRunning();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+
+		closeFuture.get();
+
+		// result future should complete exceptionally.
+		CompletableFuture<ArchivedExecutionGraph> resultFuture = dispatcherJob.getResultFuture();
+		CommonTestUtils.assertThrows("has not been finished", ExecutionException.class,
+			resultFuture::get);
+	}
+
+	@Test(expected = IllegalStateException.class)
+	public void testUnavailableJobMasterGateway() {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		dispatcherJob.getJobMasterGateway();
+	}
+
+	private TestContext createTestContext(Dispatcher.ExecutionType type) {
+		final JobVertex testVertex = new JobVertex("testVertex");
+		testVertex.setInvokableClass(NoOpInvokable.class);
+
+		JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex);
+		CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture = new CompletableFuture<>();
+		DispatcherJob dispatcherJob = DispatcherJob.createFor(jobManagerRunnerCompletableFuture,
+			jobGraph.getJobID(), jobGraph.getName(), type);
+
+		return new TestContext(
+			jobManagerRunnerCompletableFuture,
+			dispatcherJob,
+			jobGraph);
+	}
+
+	private static class TestContext {
+		private final CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture;
+		private final DispatcherJob dispatcherJob;
+		private final JobGraph jobGraph;
+		private final TestingJobMasterGateway mockRunningJobMasterGateway;
+		private final CompletableFuture<Acknowledge> cancellationFuture;
+
+		private JobStatus internalJobStatus = JobStatus.INITIALIZING;
+
+		public TestContext(
+			CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture,
+			DispatcherJob dispatcherJob,
+			JobGraph jobGraph) {
+			this.jobManagerRunnerCompletableFuture = jobManagerRunnerCompletableFuture;
+			this.dispatcherJob = dispatcherJob;
+			this.jobGraph = jobGraph;
+
+			this.cancellationFuture = new CompletableFuture<>();
+			this.mockRunningJobMasterGateway = new TestingJobMasterGatewayBuilder()
+				.setRequestJobSupplier(() -> CompletableFuture.completedFuture(ArchivedExecutionGraph.createFromInitializingJob(getJobID(), "test", internalJobStatus, null, 1337)))
+				.setRequestJobDetailsSupplier(() -> {
+					JobDetails jobDetails = new JobDetails(getJobID(), "", 0, 0, 0, internalJobStatus, 0,
+						new int[]{0, 0, 0, 0, 0, 0, 0, 0, 0}, 0);
+					return CompletableFuture.completedFuture(jobDetails);
+				})
+				// once JobManagerRunner is initialized, complete result future with CANCELLED AEG and ack cancellation.
+				.setCancelFunction(() -> {
+					internalJobStatus = JobStatus.CANCELLING;
+					return cancellationFuture;
+				})
+				.build();
+		}
+
+		public JobID getJobID() {
+			return jobGraph.getJobID();
+		}
+
+		public void failInitialization(Throwable ex) {
+			jobManagerRunnerCompletableFuture.completeExceptionally(ex);
+		}
+
+		public DispatcherJob getDispatcherJob() {
+			return dispatcherJob;
+		}
+
+		public void setRunning() {
+			internalJobStatus = JobStatus.RUNNING;
+			TestingJobManagerRunner jobManagerRunner =
+				new TestingJobManagerRunner(getJobID(), false);
+			jobManagerRunner.getJobMasterGateway().complete(mockRunningJobMasterGateway);
+			jobManagerRunnerCompletableFuture.complete(jobManagerRunner);
+		}
+
+		public void finishJob() {
+			try {
+				internalJobStatus = JobStatus.FINISHED;
+				jobManagerRunnerCompletableFuture.get().getResultFuture()
+					.complete(ArchivedExecutionGraph.createFromInitializingJob(getJobID(), "test", JobStatus.FINISHED, null, 1337));

Review comment:
       Same hack as in `setRunning`. I'd suggest to pass in a `jobResultFuture to the `JobManagerRunner` which is created to complete `jobManagerRunnerCompletableFuture`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -419,19 +554,29 @@ public void testWaitingForJobMasterLeadership() throws Exception {
 
 		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
-		final CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT);
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT);
 
 		assertThat(jobStatusFuture.isDone(), is(false));

Review comment:
       Are you sure that this holds now? I thought it would return `JobStatus.INITIALIZING` at least.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static junit.framework.Assert.fail;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Test for the {@link DispatcherJob} class.
+ */
+public class DispatcherJobTest extends TestLogger {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	@Test
+	public void testStatusWhenInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+	}
+
+	@Test
+	public void testStatusWhenRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish initialization
+		testContext.setRunning();
+
+		assertJobStatus(dispatcherJob, JobStatus.RUNNING);
+
+		// result future not done
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+	}
+
+	@Test
+	public void testStatusWhenJobFinished() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish job
+		testContext.setRunning();
+		testContext.finishJob();
+
+		assertJobStatus(dispatcherJob, JobStatus.FINISHED);
+
+		// assert result future done
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+
+		Assert.assertThat(aeg.getState(), is(JobStatus.FINISHED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(
+			TIMEOUT);
+
+		Assert.assertThat(cancelFuture.isDone(), is(false));
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+
+		testContext.setRunning();
+		testContext.finishCancellation();
+
+		// assert that cancel future completes
+		cancelFuture.get();
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		// assert that the result future completes
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		testContext.setRunning();
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(TIMEOUT);
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+		testContext.finishCancellation();
+
+		cancelFuture.get();
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileFailed() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		CommonTestUtils.assertThrows("Artificial failure", ExecutionException.class, () -> dispatcherJob.cancel(TIMEOUT).get());
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+	}
+
+	@Test
+	public void testErrorWhileInitializing() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.dispatcherJob;
+
+		// now fail
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+		Assert.assertThat(aeg.getFailureInfo().getException().deserializeError(ClassLoader.getSystemClassLoader()), is(exception));
+
+		Assert.assertTrue(dispatcherJob.closeAsync().isDone() && dispatcherJob.closeAsync().isCompletedExceptionally());
+	}
+
+	@Test
+	public void testCloseWhileInitializingSuccessfully() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		// set job running, so that we can cancel it
+		testContext.setRunning();
+
+		// assert future completes now
+		closeFuture.get();
+
+		// ensure the result future is complete (how it completes is up to the JobManager)
+		CompletableFuture<ArchivedExecutionGraph> resultFuture = dispatcherJob.getResultFuture();
+		CommonTestUtils.assertThrows("has not been finished", ExecutionException.class,
+			resultFuture::get);
+	}
+
+	@Test
+	public void testCloseWhileInitializingErroneously() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		testContext.failInitialization(new RuntimeException("fail"));
+
+		// assert future completes now
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			closeFuture::get);
+
+		// ensure the result future is complete
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.FAILED));
+	}
+
+	@Test
+	public void testCloseWhileInitializingErroneouslyForRecovery() {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.RECOVERY);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+
+		testContext.failInitialization(new RuntimeException("fail"));
+
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			closeFuture::get);
+		// ensure the result future is completing exceptionally when using RECOVERY execution
+		CommonTestUtils.assertThrows("fail", ExecutionException.class,
+			() -> dispatcherJob.getResultFuture().get());
+	}
+
+	@Test
+	public void testCloseWhileRunning() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// complete JobManager runner future to indicate to the DispatcherJob that the Runner has been initialized
+		testContext.setRunning();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+
+		closeFuture.get();
+
+		// result future should complete exceptionally.
+		CompletableFuture<ArchivedExecutionGraph> resultFuture = dispatcherJob.getResultFuture();
+		CommonTestUtils.assertThrows("has not been finished", ExecutionException.class,
+			resultFuture::get);
+	}
+
+	@Test(expected = IllegalStateException.class)
+	public void testUnavailableJobMasterGateway() {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		dispatcherJob.getJobMasterGateway();
+	}
+
+	private TestContext createTestContext(Dispatcher.ExecutionType type) {
+		final JobVertex testVertex = new JobVertex("testVertex");
+		testVertex.setInvokableClass(NoOpInvokable.class);
+
+		JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex);
+		CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture = new CompletableFuture<>();
+		DispatcherJob dispatcherJob = DispatcherJob.createFor(jobManagerRunnerCompletableFuture,
+			jobGraph.getJobID(), jobGraph.getName(), type);
+
+		return new TestContext(
+			jobManagerRunnerCompletableFuture,
+			dispatcherJob,
+			jobGraph);
+	}
+
+	private static class TestContext {
+		private final CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture;
+		private final DispatcherJob dispatcherJob;
+		private final JobGraph jobGraph;
+		private final TestingJobMasterGateway mockRunningJobMasterGateway;
+		private final CompletableFuture<Acknowledge> cancellationFuture;
+
+		private JobStatus internalJobStatus = JobStatus.INITIALIZING;
+
+		public TestContext(
+			CompletableFuture<JobManagerRunner> jobManagerRunnerCompletableFuture,
+			DispatcherJob dispatcherJob,
+			JobGraph jobGraph) {
+			this.jobManagerRunnerCompletableFuture = jobManagerRunnerCompletableFuture;
+			this.dispatcherJob = dispatcherJob;
+			this.jobGraph = jobGraph;
+
+			this.cancellationFuture = new CompletableFuture<>();
+			this.mockRunningJobMasterGateway = new TestingJobMasterGatewayBuilder()
+				.setRequestJobSupplier(() -> CompletableFuture.completedFuture(ArchivedExecutionGraph.createFromInitializingJob(getJobID(), "test", internalJobStatus, null, 1337)))
+				.setRequestJobDetailsSupplier(() -> {
+					JobDetails jobDetails = new JobDetails(getJobID(), "", 0, 0, 0, internalJobStatus, 0,
+						new int[]{0, 0, 0, 0, 0, 0, 0, 0, 0}, 0);
+					return CompletableFuture.completedFuture(jobDetails);
+				})
+				// once JobManagerRunner is initialized, complete result future with CANCELLED AEG and ack cancellation.
+				.setCancelFunction(() -> {
+					internalJobStatus = JobStatus.CANCELLING;
+					return cancellationFuture;
+				})
+				.build();
+		}
+
+		public JobID getJobID() {
+			return jobGraph.getJobID();
+		}
+
+		public void failInitialization(Throwable ex) {
+			jobManagerRunnerCompletableFuture.completeExceptionally(ex);
+		}
+
+		public DispatcherJob getDispatcherJob() {
+			return dispatcherJob;
+		}
+
+		public void setRunning() {
+			internalJobStatus = JobStatus.RUNNING;
+			TestingJobManagerRunner jobManagerRunner =
+				new TestingJobManagerRunner(getJobID(), false);
+			jobManagerRunner.getJobMasterGateway().complete(mockRunningJobMasterGateway);

Review comment:
       This is a hack. I would suggest to add a `TestingJobManagerRunnerBuilder` where one can set the result of the `getJobMasterGateway()` function.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -329,6 +345,125 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception {
 		}
 	}
 
+	@Test(timeout = 5_000L)
+	public void testNonBlockingJobSubmission() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		// ensure INITIALIZING status from status
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT);
+		Assert.assertEquals(JobStatus.INITIALIZING, jobStatusFuture.get());
+
+		// ensure correct JobDetails
+		MultipleJobsDetails multiDetails = dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get();
+		Assert.assertEquals(1, multiDetails.getJobs().size());
+		Assert.assertEquals(blockingJobGraph.getJobID(), multiDetails.getJobs().iterator().next().getJobId());
+
+		// submission has succeeded, let the initialization finish.
+		blockingJobVertex.unblock();
+
+		// wait till job is running
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT).get());
+
+		dispatcherGateway.cancelJob(blockingJobGraph.getJobID(), TIMEOUT).get();
+	}
+
+	@Test(timeout = 5_000L)
+	public void testInvalidCallDuringInitialization() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		// ensure INITIALIZING status
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT);
+		Assert.assertEquals(JobStatus.INITIALIZING, jobStatusFuture.get());
+
+		// this call is supposed to fail
+		boolean exceptionSeen = false;
+		try {
+			dispatcherGateway.triggerSavepoint(blockingJobGraph.getJobID(), "file:///tmp/savepoint", false, TIMEOUT).get();
+		} catch (ExecutionException t) {
+			Assert.assertTrue(t.getCause() instanceof UnavailableDispatcherOperationException);
+			exceptionSeen = true;
+		}
+		Assert.assertTrue(exceptionSeen);
+
+		// submission has succeeded, let the initialization finish.
+		blockingJobVertex.unblock();
+	}
+
+	@Test(timeout = 5_000L)
+	public void testCancellationDuringInitialization() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);

Review comment:
       Duplicated code.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -285,12 +299,14 @@ public void tearDown() throws Exception {
 	@Test
 	public void testJobSubmission() throws Exception {
 		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
-
+		// grant leadership to job master so that we can call dispatcherGateway.requestJobStatus().
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
 		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
 
-		CompletableFuture<Acknowledge> acknowledgeFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT);
+		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
-		acknowledgeFuture.get();
+		CommonTestUtils.waitUntilCondition(() -> dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get() == JobStatus.RUNNING,
+			Deadline.fromNow(Duration.of(10, ChronoUnit.SECONDS)), 20L);

Review comment:
       Why don't we wait on `jobMasterLeaderElectionService.getStartFuture` instead of doing busy polling?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -603,6 +769,9 @@ public void testOnRemovedJobGraphDoesNotCleanUpHAFiles() throws Exception {
 			.build();
 		dispatcher.start();
 
+		// we need to elect a jobmaster leader to be able to cancel the job on the JobMaster.
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID()).get();

Review comment:
       Why is this needed? Why do we need to cancel the job here?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -561,10 +725,12 @@ public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception {
 	@Test
 	public void testJobSuspensionWhenDispatcherIsTerminated() throws Exception {
 		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
 
 		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
 
 		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get());

Review comment:
       Why do we have to wait on this here?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -509,28 +657,41 @@ public void testFailingJobManagerRunnerCleanup() throws Exception {
 				if (exception != null) {
 					throw exception;
 				}
-			}));
+			}, getMockedRunningJobMasterGateway()));
 
 		final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
 
-		CompletableFuture<Acknowledge> submissionFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT);
+		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
-		assertThat(submissionFuture.isDone(), is(false));
+		assertThat(dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(), is(JobStatus.INITIALIZING));
 
 		queue.offer(Optional.of(testException));
 
-		try {
-			submissionFuture.get();
-			fail("Should fail because we could not instantiate the JobManagerRunner.");
-		} catch (Exception e) {
-			assertThat(ExceptionUtils.findThrowable(e, t -> t.equals(testException)).isPresent(), is(true));
-		}
-
-		submissionFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT);
+		// wait till job is failed
+		JobStatus status;
+		do {
+			status = dispatcherGateway.requestJobStatus(
+				jobGraph.getJobID(),
+				TIMEOUT).get();
+			Assert.assertThat(
+				status,
+				either(Matchers.is(JobStatus.INITIALIZING)).or(Matchers.is(JobStatus.FAILED)));
+			Thread.sleep(20);
+		} while (status != JobStatus.FAILED);

Review comment:
       Waiting on `dispatcherGateway.requestJobResult` could save use the busy waiting loop.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/BlobsCleanupITCase.java
##########
@@ -181,17 +186,21 @@ private void testBlobServerCleanup(final TestCase testCase) throws Exception {
 		}
 
 		final CompletableFuture<JobSubmissionResult> submissionFuture = miniCluster.submitJob(jobGraph);
+		final JobSubmissionResult jobSubmissionResult = submissionFuture.get();
+
+		// wait until job is submitted
+		CommonTestUtils.waitUntilCondition(() -> miniCluster.getJobStatus(jobGraph.getJobID()).get() != JobStatus.INITIALIZING,
+			Deadline.fromNow(Duration.of(10, ChronoUnit.SECONDS)), 20L);

Review comment:
       Why do we have to wait on this?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -617,16 +786,23 @@ public void testOnRemovedJobGraphDoesNotCleanUpHAFiles() throws Exception {
 
 		@Nonnull
 		private final ThrowingRunnable<Exception> jobManagerRunnerCreationLatch;
+		@Nullable
+		private final JobMasterGateway jobMasterGateway;
 
-		BlockingJobManagerRunnerFactory(@Nonnull ThrowingRunnable<Exception> jobManagerRunnerCreationLatch) {
+		BlockingJobManagerRunnerFactory(@Nonnull ThrowingRunnable<Exception> jobManagerRunnerCreationLatch, @Nullable JobMasterGateway jobMasterGateway) {
 			this.jobManagerRunnerCreationLatch = jobManagerRunnerCreationLatch;
+			this.jobMasterGateway = jobMasterGateway;
 		}
 
 		@Override
 		public TestingJobManagerRunner createJobManagerRunner(JobGraph jobGraph, Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, JobManagerSharedServices jobManagerSharedServices, JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, FatalErrorHandler fatalErrorHandler) throws Exception {
 			jobManagerRunnerCreationLatch.run();
 
-			return super.createJobManagerRunner(jobGraph, configuration, rpcService, highAvailabilityServices, heartbeatServices, jobManagerSharedServices, jobManagerJobMetricGroupFactory, fatalErrorHandler);
+			TestingJobManagerRunner runner = super.createJobManagerRunner(jobGraph, configuration, rpcService, highAvailabilityServices, heartbeatServices, jobManagerSharedServices, jobManagerJobMetricGroupFactory, fatalErrorHandler);
+			if (jobMasterGateway != null) {
+				runner.getJobMasterGateway().complete(jobMasterGateway);
+			}

Review comment:
       Let's not use this hack here. We don't know whether `getJobMasterGateway` will always return the same future.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -690,4 +866,66 @@ public JobManagerRunner createJobManagerRunner(
 		}
 	}
 
+	private static class BlockingJobVertex extends JobVertex {
+		private final Object lock = new Object();
+		private boolean blocking = true;
+		public BlockingJobVertex(String name) {
+			super(name);
+		}
+
+		@Override
+		public void initializeOnMaster(ClassLoader loader) throws Exception {
+			super.initializeOnMaster(loader);
+
+			while (true) {
+				synchronized (lock) {
+					if (!blocking) {
+						return;
+					}
+					lock.wait(10);
+				}
+			}

Review comment:
       I'd suggest to use a `OneShotLatch` here.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -472,23 +618,25 @@ public void testBlockingJobManagerRunner() throws Exception {
 		dispatcher = createAndStartDispatcher(
 			heartbeatServices,
 			haServices,
-			new BlockingJobManagerRunnerFactory(jobManagerRunnerCreationLatch::await));
-
+			new BlockingJobManagerRunnerFactory(jobManagerRunnerCreationLatch::await, getMockedRunningJobMasterGateway()));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
 		final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
 
-		final CompletableFuture<Acknowledge> submissionFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT);
+		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
-		assertThat(submissionFuture.isDone(), is(false));
+		assertThat(dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(), is(JobStatus.INITIALIZING));
 
 		final CompletableFuture<Collection<String>> metricQueryServiceAddressesFuture = dispatcherGateway.requestMetricQueryServiceAddresses(Time.seconds(5L));
 
 		assertThat(metricQueryServiceAddressesFuture.get(), is(empty()));
 
-		assertThat(submissionFuture.isDone(), is(false));
+		assertThat(dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(), is(JobStatus.INITIALIZING));
 
 		jobManagerRunnerCreationLatch.trigger();
 
-		submissionFuture.get();
+		CommonTestUtils.waitUntilCondition(() -> dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get() == JobStatus.RUNNING,
+			Deadline.fromNow(Duration.of(10, ChronoUnit.SECONDS)), 5L);
+		assertThat(dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(), is(JobStatus.RUNNING));

Review comment:
       That's a bit doppelt gemoppelt.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -419,19 +554,29 @@ public void testWaitingForJobMasterLeadership() throws Exception {
 
 		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
 
-		final CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT);
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT);
 
 		assertThat(jobStatusFuture.isDone(), is(false));
 
-		try {
-			jobStatusFuture.get(10, TimeUnit.MILLISECONDS);
-			fail("Should not complete.");
-		} catch (TimeoutException ignored) {
-			// ignored
+		// Make sure that the jobstatus request is blocking after it has left the INITIALIZING status
+		boolean timeoutSeen = false;
+		while (!timeoutSeen) {
+			try {
+				jobStatusFuture = dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT);
+				JobStatus jobStatus = jobStatusFuture.get(10, TimeUnit.MILLISECONDS);
+				if (jobStatus != JobStatus.INITIALIZING) {
+					fail("Should not complete.");
+				} else {
+					Thread.sleep(10); // give more time to initialize
+				}
+			} catch (TimeoutException ignored) {
+				timeoutSeen = true;
+			}
 		}
-
+		// Job is initialized. Make the master leader
 		jobMasterLeaderElectionService.isLeader(UUID.randomUUID()).get();
 
+		// ensure that the future is now completing

Review comment:
       Maybe it would be better to test the completion of the leader future directly on the `JobManagerRunnerImpl`.
   
   I have to admit that I am not a huge fan of busy waiting loops. Whenever one sees one, one should think whether one can't express the test differently.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java
##########
@@ -166,21 +167,28 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception {
 				defaultDispatcherRunnerFactory)) {
 
 				// initial run
-				DispatcherGateway dispatcherGateway = grantLeadership(dispatcherLeaderElectionService);
+				final DispatcherGateway dispatcherGateway = grantLeadership(dispatcherLeaderElectionService);
 
 				LOG.info("Initial job submission {}.", jobGraph.getJobID());
 				dispatcherGateway.submitJob(jobGraph, TESTING_TIMEOUT).get();
 
+				// wait until job is running
+				CommonTestUtils.waitUntilCondition(() ->
+					dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TESTING_TIMEOUT).get() == JobStatus.RUNNING, Deadline.fromNow(VERIFICATION_TIMEOUT), 20L);

Review comment:
       Why is this required here?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -329,6 +345,125 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception {
 		}
 	}
 
+	@Test(timeout = 5_000L)
+	public void testNonBlockingJobSubmission() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		// ensure INITIALIZING status from status
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT);
+		Assert.assertEquals(JobStatus.INITIALIZING, jobStatusFuture.get());
+
+		// ensure correct JobDetails
+		MultipleJobsDetails multiDetails = dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get();
+		Assert.assertEquals(1, multiDetails.getJobs().size());
+		Assert.assertEquals(blockingJobGraph.getJobID(), multiDetails.getJobs().iterator().next().getJobId());
+
+		// submission has succeeded, let the initialization finish.
+		blockingJobVertex.unblock();
+
+		// wait till job is running
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT).get());
+
+		dispatcherGateway.cancelJob(blockingJobGraph.getJobID(), TIMEOUT).get();
+	}
+
+	@Test(timeout = 5_000L)
+	public void testInvalidCallDuringInitialization() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		// ensure INITIALIZING status
+		CompletableFuture<JobStatus> jobStatusFuture = dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT);
+		Assert.assertEquals(JobStatus.INITIALIZING, jobStatusFuture.get());
+
+		// this call is supposed to fail
+		boolean exceptionSeen = false;
+		try {
+			dispatcherGateway.triggerSavepoint(blockingJobGraph.getJobID(), "file:///tmp/savepoint", false, TIMEOUT).get();
+		} catch (ExecutionException t) {
+			Assert.assertTrue(t.getCause() instanceof UnavailableDispatcherOperationException);
+			exceptionSeen = true;
+		}
+		Assert.assertTrue(exceptionSeen);
+
+		// submission has succeeded, let the initialization finish.
+		blockingJobVertex.unblock();
+	}
+
+	@Test(timeout = 5_000L)
+	public void testCancellationDuringInitialization() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph of a job that blocks forever
+		final BlockingJobVertex blockingJobVertex = new BlockingJobVertex("testVertex");
+		blockingJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "blockingTestJob", blockingJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		Assert.assertThat(dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT).get(), is(JobStatus.INITIALIZING));
+
+		// submission has succeeded, now cancel the job
+		CompletableFuture<Acknowledge> cancellationFuture = dispatcherGateway.cancelJob(blockingJobGraph.getJobID(), TIMEOUT);
+		Assert.assertThat(dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT).get(), is(JobStatus.CANCELLING));
+		Assert.assertThat(cancellationFuture.isDone(), is(false));
+		// unblock
+		blockingJobVertex.unblock();
+		// wait until cancelled
+		cancellationFuture.get();
+		Assert.assertThat(dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT).get(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testErrorDuringInitialization() throws Exception {
+		dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch));
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		// create a job graph that fails during initialization
+		final FailingInitializationJobVertex failingInitializationJobVertex = new FailingInitializationJobVertex(
+			"testVertex");
+		failingInitializationJobVertex.setInvokableClass(NoOpInvokable.class);
+		JobGraph blockingJobGraph = new JobGraph(TEST_JOB_ID, "failingTestJob", failingInitializationJobVertex);
+
+		dispatcherGateway.submitJob(blockingJobGraph, TIMEOUT).get();
+
+		// wait till job has failed
+		JobStatus status;
+		do {
+			CompletableFuture<JobStatus> statusFuture = dispatcherGateway.requestJobStatus(blockingJobGraph.getJobID(), TIMEOUT);
+			status = statusFuture.get();
+			Thread.sleep(50);
+			Assert.assertThat(status, either(is(JobStatus.INITIALIZING)).or(is(JobStatus.FAILED)));
+		} while (status != JobStatus.FAILED);

Review comment:
       Can't we replace this with `dispatcherGateway.requestJobResult` and then waiting on the returned future?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java
##########
@@ -134,14 +135,13 @@ public void leaderChange_afterJobSubmission_recoversSubmittedJob() throws Except
 			final UUID firstLeaderSessionId = UUID.randomUUID();
 
 			final DispatcherGateway firstDispatcherGateway = electLeaderAndRetrieveGateway(firstLeaderSessionId);
-
 			firstDispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
+			CommonTestUtils.waitUntilJobManagerIsInitialized(() -> firstDispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get());

Review comment:
       Why is this needed?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java
##########
@@ -103,6 +104,8 @@ public void testReelectionOfDispatcher() throws Exception {
 
 		submissionFuture.get();
 
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> miniCluster.getJobStatus(jobId).get());

Review comment:
       Why is this required?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
##########
@@ -0,0 +1,355 @@
+/*
+ * 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.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static junit.framework.Assert.fail;
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Test for the {@link DispatcherJob} class.
+ */
+public class DispatcherJobTest extends TestLogger {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	@Test
+	public void testStatusWhenInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+	}
+
+	@Test
+	public void testStatusWhenRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish initialization
+		testContext.setRunning();
+
+		assertJobStatus(dispatcherJob, JobStatus.RUNNING);
+
+		// result future not done
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+	}
+
+	@Test
+	public void testStatusWhenJobFinished() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish job
+		testContext.setRunning();
+		testContext.finishJob();
+
+		assertJobStatus(dispatcherJob, JobStatus.FINISHED);
+
+		// assert result future done
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+
+		Assert.assertThat(aeg.getState(), is(JobStatus.FINISHED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileInitializing() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(
+			TIMEOUT);
+
+		Assert.assertThat(cancelFuture.isDone(), is(false));
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+
+		testContext.setRunning();
+		testContext.finishCancellation();
+
+		// assert that cancel future completes
+		cancelFuture.get();
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		// assert that the result future completes
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileRunning() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		testContext.setRunning();
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(TIMEOUT);
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+		testContext.finishCancellation();
+
+		cancelFuture.get();
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileFailed() throws
+		Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		CommonTestUtils.assertThrows("Artificial failure", ExecutionException.class, () -> dispatcherJob.cancel(TIMEOUT).get());
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+	}
+
+	@Test
+	public void testErrorWhileInitializing() throws Exception {
+		TestContext testContext = createTestContext(Dispatcher.ExecutionType.SUBMISSION);
+		DispatcherJob dispatcherJob = testContext.dispatcherJob;
+
+		// now fail
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get();
+		Assert.assertThat(aeg.getFailureInfo().getException().deserializeError(ClassLoader.getSystemClassLoader()), is(exception));
+
+		Assert.assertTrue(dispatcherJob.closeAsync().isDone() && dispatcherJob.closeAsync().isCompletedExceptionally());

Review comment:
       This condition is actually covered by `testCloseWhileInitializingErroneously`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java
##########
@@ -351,4 +352,53 @@ public static ArchivedExecutionGraph createFrom(ExecutionGraph executionGraph) {
 			executionGraph.getCheckpointStatsSnapshot(),
 			executionGraph.getStateBackendName().orElse(null));
 	}
+
+	/**
+	 * Create a sparse ArchivedExecutionGraph for a job while it is still initializing.
+	 * Most fields will be empty, only job status and error-related fields are set.
+	 */
+	public static ArchivedExecutionGraph createFromInitializingJob(
+		JobID jobId,
+		String jobName,
+		JobStatus jobStatus,
+		@Nullable Throwable throwable,
+		long initializationTimestamp) {
+
+		long failureTime = System.currentTimeMillis();

Review comment:
       It does not matter if this will cost us 5ms.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java
##########
@@ -135,6 +138,8 @@ public void testReelectionOfJobMaster() throws Exception {
 
 		submissionFuture.get();
 
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> miniCluster.getJobStatus(jobId).get());

Review comment:
       Same here?

##########
File path: flink-table/flink-table-planner-blink/src/test/resources/log4j2-test.properties
##########
@@ -18,7 +18,7 @@
 
 # Set root logger level to OFF to not flood build logs
 # set manually to INFO for debugging purposes
-rootLogger.level = OFF
+rootLogger.level = DEBUG

Review comment:
       Please revert

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java
##########
@@ -132,6 +132,7 @@ public void testJobRecoveryWithFailingTaskExecutor() throws Exception {
 	private CompletableFuture<JobResult> submitJobAndWaitUntilRunning(JobGraph jobGraph) throws Exception {
 		miniCluster.submitJob(jobGraph).get();
 
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> miniCluster.getJobStatus(jobGraph.getJobID()).get());

Review comment:
       Why is this needed here?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475643136



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});

Review comment:
       > What happens if I call dispatcherJob.requestJob after the job has been cancelled in state INITIALIZING? I would like to obtain the final CANCELED ArchivedExecutionGraph ideally.
   
   Since we are forwarding the the result future from the runner to the `DispatcherJob`'s result future, you should get a proper `ArchivedExecutionGraph` with the correct final state.
   
   (note: I have removed this entire `.whenComplete()` stage, as it is not necessary to track the FAILED or CANCELLED etc. states).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r479980792



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java
##########
@@ -135,6 +138,8 @@ public void testReelectionOfJobMaster() throws Exception {
 
 		submissionFuture.get();
 
+		CommonTestUtils.waitUntilJobManagerIsInitialized(() -> miniCluster.getJobStatus(jobId).get());

Review comment:
       The next statement in the test is to revoke the JobMasterLeadership.
   This revocation is only possible if we have leadership, which depends on the job being initialized.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 41769d3c4ebae61f502e16398207d477f59451f9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 08f8c0a7fbfecf7a6eba88653140439f3673c2cc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-684635708


   Thanks a lot for this extensive review. I believe I have addressed all comments. I'm looking forward to more feedback.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475554941



##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,41 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+		SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+		SupplierWithException<JobResult, Exception> jobResultSupplier
+		) throws CompletionException {

Review comment:
       I'd say let's go with c) Throw a properly defined exception which signals what went wrong and let the caller be responsible for wrapping it into the right format. The latter should not be the responsibility of this method which already does enough.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [flink] asfgit closed pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #13217:
URL: https://github.com/apache/flink/pull/13217


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 52a49f0b0840aa9220de72d64c50a6b33f6adf92 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475556774



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -350,71 +341,42 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
 		}, getRpcService().getExecutor());
 	}
 
-	private CompletableFuture<Void> persistAndRunJob(JobGraph jobGraph) throws Exception {
+	private Void persistAndRunJob(JobGraph jobGraph) throws Exception {

Review comment:
       Unfortunately not: https://stackoverflow.com/questions/28762715/cannot-convert-void-to-java-lang-void




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475522684



##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,41 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+		SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+		SupplierWithException<JobResult, Exception> jobResultSupplier
+		) throws CompletionException {

Review comment:
       For some reason, I assumed that `CompletionExceptions` are the proper way of signaling an error in futures. However, this assumption seems to be false, any RuntimeException will be wrapped into a CompletionException by the future.
   
   I see solutions 2 here
   a) declare the method throws any Throwable (then I have to use `FutureUtils.uncheckedFunction` when I use it within futures, to wrap errors in RuntimeExceptions
   b) wrap any Throwable in a `RuntimeException` and remove the declaration on the method.
   
   Unless you tell suggest otherwise, I'll go with b) for now.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475555137



##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,41 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+		SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+		SupplierWithException<JobResult, Exception> jobResultSupplier
+		) throws CompletionException {
+		LOG.debug("Wait until job initialization is finished");
+		WaitStrategy waitStrategy = new ExponentialWaitStrategy(50, 2000);
+		try {
+			JobStatus status = jobStatusSupplier.get();
+			long attempt = 0;
+			while (status == JobStatus.INITIALIZING) {
+				Thread.sleep(waitStrategy.sleepTime(attempt++));
+				status = jobStatusSupplier.get();
+			}
+			if (status == JobStatus.FAILED) {
+				// note: we can not distinguish between initialization failures and failures once
+				// execution has started. Execution errors are potentially reported here.
+				JobResult result = jobResultSupplier.get();
+				Optional<SerializedThrowable> throwable = result.getSerializedThrowable();
+				if (throwable.isPresent()) {
+					throw new CompletionException(throwable.get().deserializeError(Thread.currentThread().getContextClassLoader()));

Review comment:
       I think it would be the right approach, yes.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-685969473


   Thanks a lot for your feedback! I addressed all comments and rebased the code to the latest master to get rid of the conflicts.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r476284026



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});

Review comment:
       You are right. it is more elegant to schedule the cancel immediately.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r480067151



##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,48 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+				JobID jobID,
+				SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+				SupplierWithException<JobResult, Exception> jobResultSupplier,
+				ClassLoader userCodeClassloader)
+			throws JobInitializationException {
+		LOG.debug("Wait until job initialization is finished");
+		WaitStrategy waitStrategy = new ExponentialWaitStrategy(50, 2000);
+		try {
+			JobStatus status = jobStatusSupplier.get();
+			long attempt = 0;
+			while (status == JobStatus.INITIALIZING) {
+				Thread.sleep(waitStrategy.sleepTime(attempt++));
+				status = jobStatusSupplier.get();
+			}
+			if (status == JobStatus.FAILED) {
+				JobResult result = jobResultSupplier.get();
+				Optional<SerializedThrowable> throwable = result.getSerializedThrowable();
+				if (throwable.isPresent()) {
+					Throwable t = throwable.get().deserializeError(userCodeClassloader);
+					t = ExceptionUtils.stripCompletionException(t);
+					if (t instanceof JobInitializationException) {
+						throw t;
+					}
+				}
+			}
+		} catch (JobInitializationException initializationException) {
+			throw initializationException;
+		} catch (Throwable throwable) {
+			ExceptionUtils.checkInterrupted(throwable);
+			throw new JobInitializationException(jobID, "Error while waiting for job to be initialized", throwable);
+		}
+	}
+
+	public static <T> void waitUntilJobInitializationFinished(JobID id, ClusterClient<T> client, ClassLoader userCodeClassloader) throws
+		JobInitializationException {
+		waitUntilJobInitializationFinished(id, () -> client.getJobStatus(id).get(), () -> client.requestJobResult(id).get(), userCodeClassloader);

Review comment:
       As far as I understand, the per job clusters are submitting jobs through `AbstractJobClusterExecutor.execute()`. They never use the `waitUntilJobInitializationFinished` utility on that codepath.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475665594



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}

Review comment:
       I think it will return the right result, but the naming of methods / variables here is terrible: `jobStatus == JobStatus.RUNNING` as soon as the `jobManagerRunnerFuture` completed. The job is not necessarily in RUNNING state. It could be in `CREATED` or `FAILING` ... we would still do the correct thing in `DispatcherJob`, but it is not very obvious.
   
   I will improve the tracking of the job status inside the `DispatcherJob`, in particular the names (this seems to be the biggest flaw of my proposed changes).




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8036016c752bce433dc65d1c08695377c917836f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6162",
       "triggerID" : "8036016c752bce433dc65d1c08695377c917836f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d4d6690d8ad282f71e5f3d890e28421aef17766",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d4d6690d8ad282f71e5f3d890e28421aef17766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 8036016c752bce433dc65d1c08695377c917836f Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6162) 
   * 4d4d6690d8ad282f71e5f3d890e28421aef17766 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-680075151


   Thanks a lot for your extensive and detailed review. It was very helpful for me!
   
   I have addressed most of you your comments. I extended the `DispatcherJobTest`. 
   I'm in the process of testing if the refactoring to the client has introduced any new test failures, and I might clean up my new tests in `DispatcherJob` (I would thus recommend you to review them last).
   
   I might push a small change to the client code later today to better distinguish between initialization and runtime failures, so that we can solve the unstable `FunctionITCase.testInvalidUseOfTableFunction()` test.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r482792155



##########
File path: flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java
##########
@@ -80,19 +75,6 @@ private static JobGraph getWorkingJobGraph() {
 		return new JobGraph("Working testing job", jobVertex);
 	}
 
-	// --------------------------------------------------------------------------------------------
-
-	private final boolean detached;
-
-	public JobSubmissionFailsITCase(boolean detached) {
-		this.detached = detached;
-	}
-
-	@Parameterized.Parameters(name = "Detached mode = {0}")
-	public static Collection<Boolean[]> executionModes(){
-		return Arrays.asList(new Boolean[]{false},
-				new Boolean[]{true});
-	}

Review comment:
       Thanks for the explanation.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475527372



##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,41 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+		SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+		SupplierWithException<JobResult, Exception> jobResultSupplier
+		) throws CompletionException {
+		LOG.debug("Wait until job initialization is finished");
+		WaitStrategy waitStrategy = new ExponentialWaitStrategy(50, 2000);
+		try {
+			JobStatus status = jobStatusSupplier.get();
+			long attempt = 0;
+			while (status == JobStatus.INITIALIZING) {
+				Thread.sleep(waitStrategy.sleepTime(attempt++));
+				status = jobStatusSupplier.get();
+			}
+			if (status == JobStatus.FAILED) {
+				// note: we can not distinguish between initialization failures and failures once
+				// execution has started. Execution errors are potentially reported here.
+				JobResult result = jobResultSupplier.get();
+				Optional<SerializedThrowable> throwable = result.getSerializedThrowable();
+				if (throwable.isPresent()) {
+					throw new CompletionException(throwable.get().deserializeError(Thread.currentThread().getContextClassLoader()));

Review comment:
       My assumption is yes, but I agree this could be fragile.
   
   If we are passing the wrong classloader here, we'll throw the `SerializedThrowable`, which contains contains the exception message and stacktrace as strings as well (users will see the cause, but they can't check for instance types in the causes).
   
   Getting the usercode classloader into this method should be doable if you think it is the right approach. I just didn't want to be too invasive with this change :)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8036016c752bce433dc65d1c08695377c917836f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6162",
       "triggerID" : "8036016c752bce433dc65d1c08695377c917836f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d4d6690d8ad282f71e5f3d890e28421aef17766",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6223",
       "triggerID" : "4d4d6690d8ad282f71e5f3d890e28421aef17766",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 4d4d6690d8ad282f71e5f3d890e28421aef17766 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6223) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 41769d3c4ebae61f502e16398207d477f59451f9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875) 
   * 08f8c0a7fbfecf7a6eba88653140439f3673c2cc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475533221



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -350,71 +341,42 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
 		}, getRpcService().getExecutor());
 	}
 
-	private CompletableFuture<Void> persistAndRunJob(JobGraph jobGraph) throws Exception {
+	private Void persistAndRunJob(JobGraph jobGraph) throws Exception {

Review comment:
       I have this ugly `Void` and `return null` here so that I can call the method as a method reference. 
   My next push will contain an attempt to make this cleaner (by introducing a `VoidFunctionWithException` interface)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 805f3142619f652b96804ab3c65beca2ba50f5d4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860) 
   * c741363eaee4cbff8edaa3cab54c3bfbdd668184 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85c9bbac2d9ca07435fbd80d76208fa2a51e5d37 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678327440


   I assumed the tests to be stable, because my personal CI finished [pretty green](https://dev.azure.com/rmetzger/Flink/_build/results?buildId=8308&view=results). I will investigate the unstable tests.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 805f3142619f652b96804ab3c65beca2ba50f5d4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860) 
   * c741363eaee4cbff8edaa3cab54c3bfbdd668184 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872) 
   * 41769d3c4ebae61f502e16398207d477f59451f9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r482246102



##########
File path: flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java
##########
@@ -80,19 +75,6 @@ private static JobGraph getWorkingJobGraph() {
 		return new JobGraph("Working testing job", jobVertex);
 	}
 
-	// --------------------------------------------------------------------------------------------
-
-	private final boolean detached;
-
-	public JobSubmissionFailsITCase(boolean detached) {
-		this.detached = detached;
-	}
-
-	@Parameterized.Parameters(name = "Detached mode = {0}")
-	public static Collection<Boolean[]> executionModes(){
-		return Arrays.asList(new Boolean[]{false},
-				new Boolean[]{true});
-	}

Review comment:
       We can not distinguish between detached and attached job submission anymore. All submissions are detached.
   
   This is the old code:
   ```java
   			if (detached) {
   				client.submitJob(jobGraph).get();
   			} else {
   				submitJobAndWaitForResult(client, jobGraph, getClass().getClassLoader());
   			}
   // method for reference:
   	public static void submitJobAndWaitForResult(ClusterClient<?> client, JobGraph jobGraph, ClassLoader classLoader) throws Exception {
   		client.submitJob(jobGraph)
   			.thenCompose(client::requestJobResult)
   			.get()
   			.toJobExecutionResult(classLoader);
   	}
   ```
   ... and with the non-blocking job submission, we would basically do the same in both cases.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 52a49f0b0840aa9220de72d64c50a6b33f6adf92 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832) 
   * 805f3142619f652b96804ab3c65beca2ba50f5d4 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 9127b879bec8dc1c05176c1acb89680c0e7baf04 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807) 
   * 52a49f0b0840aa9220de72d64c50a6b33f6adf92 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 41769d3c4ebae61f502e16398207d477f59451f9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875) 
   * 08f8c0a7fbfecf7a6eba88653140439f3673c2cc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 08f8c0a7fbfecf7a6eba88653140439f3673c2cc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051) 
   * c52eaf11a6832049ad9a18c784d2661101d91b8c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114) 
   * 31d810f7c4d93043d8444a78f921b263f1258646 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8036016c752bce433dc65d1c08695377c917836f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6162",
       "triggerID" : "8036016c752bce433dc65d1c08695377c917836f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 31d810f7c4d93043d8444a78f921b263f1258646 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118) 
   * 8036016c752bce433dc65d1c08695377c917836f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6162) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85c9bbac2d9ca07435fbd80d76208fa2a51e5d37 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776) 
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 9127b879bec8dc1c05176c1acb89680c0e7baf04 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8036016c752bce433dc65d1c08695377c917836f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8036016c752bce433dc65d1c08695377c917836f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 31d810f7c4d93043d8444a78f921b263f1258646 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118) 
   * 8036016c752bce433dc65d1c08695377c917836f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85c9bbac2d9ca07435fbd80d76208fa2a51e5d37 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 08f8c0a7fbfecf7a6eba88653140439f3673c2cc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051) 
   * c52eaf11a6832049ad9a18c784d2661101d91b8c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114) 
   * 31d810f7c4d93043d8444a78f921b263f1258646 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475556728



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -427,21 +389,39 @@ private JobManagerRunner startJobManagerRunner(JobManagerRunner jobManagerRunner
 							}
 						}
 					} else {
-						log.debug("There is a newer JobManagerRunner for the job {}.", jobId);
+						log.debug("Job {} is not registered anymore at dispatcher", jobId);
 					}
-
 					return null;
 				}, getMainThreadExecutor()));
+	}
 
-		jobManagerRunner.start();
-
-		return jobManagerRunner;
+	CompletableFuture<JobManagerRunner> createJobManagerRunner(JobGraph jobGraph) {
+		final RpcService rpcService = getRpcService();
+		return CompletableFuture.supplyAsync(
+			() -> {
+				try {
+					JobManagerRunner runner = jobManagerRunnerFactory.createJobManagerRunner(
+						jobGraph,
+						configuration,
+						rpcService,
+						highAvailabilityServices,
+						heartbeatServices,
+						jobManagerSharedServices,
+						new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup),
+						fatalErrorHandler);
+					runner.start();
+					return runner;
+				} catch (Exception e) {
+					throw new CompletionException(new JobExecutionException(jobGraph.getJobID(), "Could not instantiate JobManager.", e));
+				}
+			},
+			rpcService.getExecutor()); // do not use main thread executor. Otherwise, Dispatcher is blocked on JobManager creation

Review comment:
       Great, thanks.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 805f3142619f652b96804ab3c65beca2ba50f5d4 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860) 
   * c741363eaee4cbff8edaa3cab54c3bfbdd668184 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872) 
   * 41769d3c4ebae61f502e16398207d477f59451f9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85c9bbac2d9ca07435fbd80d76208fa2a51e5d37 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776) 
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r477149500



##########
File path: flink-clients/src/test/java/org/apache/flink/client/ClientUtilsTest.java
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.BlobServerOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.blob.BlobServer;
+import org.apache.flink.runtime.blob.VoidBlobStore;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.client.JobInitializationException;
+import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.DefaultDispatcherBootstrap;
+import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory;
+import org.apache.flink.runtime.dispatcher.Dispatcher;
+import org.apache.flink.runtime.dispatcher.DispatcherGateway;
+import org.apache.flink.runtime.dispatcher.DispatcherId;
+import org.apache.flink.runtime.dispatcher.DispatcherServices;
+import org.apache.flink.runtime.dispatcher.DispatcherTest;
+import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.NoOpJobGraphWriter;
+import org.apache.flink.runtime.dispatcher.TestingDispatcher;
+import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.JobGraphWriter;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcUtils;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Test for the ClientUtils.
+ */
+public class ClientUtilsTest extends TestLogger {
+
+	@ClassRule

Review comment:
       This whole setup procedure seems way too involved. I guess a MiniCluster would do the job as well? Please +1 if you agree :) 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 08f8c0a7fbfecf7a6eba88653140439f3673c2cc Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051) 
   * c52eaf11a6832049ad9a18c784d2661101d91b8c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r481192301



##########
File path: flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutor.java
##########
@@ -122,6 +123,14 @@ public EmbeddedExecutor(
 				timeout);
 
 		return jobSubmissionFuture
+				.thenApplyAsync(FunctionUtils.uncheckedFunction(jobId -> {
+					org.apache.flink.client.ClientUtils.waitUntilJobInitializationFinished(
+						() -> dispatcherGateway.requestJobStatus(jobId, timeout).get(),
+						() -> dispatcherGateway.requestJobResult(jobId, timeout).get(),
+						userCodeClassloader
+					);

Review comment:
       nit: `);` could be on the previous line.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -516,59 +498,52 @@ private JobManagerRunner startJobManagerRunner(JobManagerRunner jobManagerRunner
 
 	@Override
 	public CompletableFuture<JobStatus> requestJobStatus(JobID jobId, Time timeout) {
-
-		final CompletableFuture<JobMasterGateway> jobMasterGatewayFuture = getJobMasterGatewayFuture(jobId);
-
-		final CompletableFuture<JobStatus> jobStatusFuture = jobMasterGatewayFuture.thenCompose(
-			(JobMasterGateway jobMasterGateway) -> jobMasterGateway.requestJobStatus(timeout));
-
-		return jobStatusFuture.exceptionally(
-			(Throwable throwable) -> {
-				final JobDetails jobDetails = archivedExecutionGraphStore.getAvailableJobDetails(jobId);
-
-				// check whether it is a completed job
-				if (jobDetails == null) {
-					throw new CompletionException(ExceptionUtils.stripCompletionException(throwable));
-				} else {
-					return jobDetails.getStatus();
-				}
-			});
+		Optional<DispatcherJob> maybeJob = getDispatcherJob(jobId);
+		if (maybeJob.isPresent()) {
+			return maybeJob.get().requestJobStatus(timeout);
+		} else {
+			// is it a completed job?
+			final JobDetails jobDetails = archivedExecutionGraphStore.getAvailableJobDetails(jobId);
+			if (jobDetails == null) {
+				return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId));
+			} else {
+				return CompletableFuture.completedFuture(jobDetails.getStatus());
+			}
+		}
 	}
 
 	@Override
 	public CompletableFuture<OperatorBackPressureStatsResponse> requestOperatorBackPressureStats(
 			final JobID jobId,
 			final JobVertexID jobVertexId) {
-		final CompletableFuture<JobMasterGateway> jobMasterGatewayFuture = getJobMasterGatewayFuture(jobId);
-
-		return jobMasterGatewayFuture.thenCompose((JobMasterGateway jobMasterGateway) -> jobMasterGateway.requestOperatorBackPressureStats(jobVertexId));
+		return performOperationOnJobMasterGateway(jobId, gateway -> gateway.requestOperatorBackPressureStats(jobVertexId));
 	}
 
 	@Override
 	public CompletableFuture<ArchivedExecutionGraph> requestJob(JobID jobId, Time timeout) {
-		final CompletableFuture<JobMasterGateway> jobMasterGatewayFuture = getJobMasterGatewayFuture(jobId);
-
-		final CompletableFuture<ArchivedExecutionGraph> archivedExecutionGraphFuture = jobMasterGatewayFuture.thenCompose(
-			(JobMasterGateway jobMasterGateway) -> jobMasterGateway.requestJob(timeout));
-
-		return archivedExecutionGraphFuture.exceptionally(
-			(Throwable throwable) -> {
-				final ArchivedExecutionGraph serializableExecutionGraph = archivedExecutionGraphStore.get(jobId);
-
-				// check whether it is a completed job
-				if (serializableExecutionGraph == null) {
-					throw new CompletionException(ExceptionUtils.stripCompletionException(throwable));
-				} else {
-					return serializableExecutionGraph;
-				}
-			});
+		Function<Throwable, ArchivedExecutionGraph> checkExecutionGraphStoreeOnException = throwable ->  {

Review comment:
       ```suggestion
   		Function<Throwable, ArchivedExecutionGraph> checkExecutionGraphStoreOnException = throwable ->  {
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.runtime.executiongraph.ArchivedExecutionGraph;
+
+import javax.annotation.Nullable;
+
+/**
+ * Container for returning the ArchivedExecutionGraph and a flag whether the initialization failed.
+ * For initialization failures, the throwable is also attached, to avoid deserializing it from the ArchivedExecutionGraph.
+ */
+public class DispatcherJobResult {
+	private final boolean initializationFailure;
+	private final ArchivedExecutionGraph archivedExecutionGraph;
+	@Nullable
+	private final Throwable throwable;
+
+	private DispatcherJobResult(ArchivedExecutionGraph archivedExecutionGraph, Throwable throwable, boolean initializationFailure) {
+		this.archivedExecutionGraph = archivedExecutionGraph;
+		this.initializationFailure = initializationFailure;
+		this.throwable = throwable;
+	}
+
+	public boolean isInitializationFailure() {
+		return initializationFailure;
+	}
+
+	public ArchivedExecutionGraph getArchivedExecutionGraph() {
+		return archivedExecutionGraph;
+	}
+
+	public Throwable getThrowable() {

Review comment:
       ```suggestion
   	public Throwable getInitializationFailure() {
   ```

##########
File path: flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java
##########
@@ -80,19 +75,6 @@ private static JobGraph getWorkingJobGraph() {
 		return new JobGraph("Working testing job", jobVertex);
 	}
 
-	// --------------------------------------------------------------------------------------------
-
-	private final boolean detached;
-
-	public JobSubmissionFailsITCase(boolean detached) {
-		this.detached = detached;
-	}
-
-	@Parameterized.Parameters(name = "Detached mode = {0}")
-	public static Collection<Boolean[]> executionModes(){
-		return Arrays.asList(new Boolean[]{false},
-				new Boolean[]{true});
-	}

Review comment:
       Why can we remove the detached test case here? Aren't we losing test coverage by removing this code?

##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +120,42 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 * @param jobResultSupplier supplier returning the job result. This will only be called if the job reaches the FAILED state.
+	 * @throws JobInitializationException If the initialization failed or RuntimeException if this method has an internal error.

Review comment:
       ```suggestion
   	 * @throws JobInitializationException If the initialization failed
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -516,59 +498,52 @@ private JobManagerRunner startJobManagerRunner(JobManagerRunner jobManagerRunner
 
 	@Override
 	public CompletableFuture<JobStatus> requestJobStatus(JobID jobId, Time timeout) {
-
-		final CompletableFuture<JobMasterGateway> jobMasterGatewayFuture = getJobMasterGatewayFuture(jobId);
-
-		final CompletableFuture<JobStatus> jobStatusFuture = jobMasterGatewayFuture.thenCompose(
-			(JobMasterGateway jobMasterGateway) -> jobMasterGateway.requestJobStatus(timeout));
-
-		return jobStatusFuture.exceptionally(
-			(Throwable throwable) -> {
-				final JobDetails jobDetails = archivedExecutionGraphStore.getAvailableJobDetails(jobId);
-
-				// check whether it is a completed job
-				if (jobDetails == null) {
-					throw new CompletionException(ExceptionUtils.stripCompletionException(throwable));
-				} else {
-					return jobDetails.getStatus();
-				}
-			});
+		Optional<DispatcherJob> maybeJob = getDispatcherJob(jobId);
+		if (maybeJob.isPresent()) {
+			return maybeJob.get().requestJobStatus(timeout);
+		} else {
+			// is it a completed job?
+			final JobDetails jobDetails = archivedExecutionGraphStore.getAvailableJobDetails(jobId);
+			if (jobDetails == null) {
+				return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId));
+			} else {
+				return CompletableFuture.completedFuture(jobDetails.getStatus());
+			}
+		}
 	}
 
 	@Override
 	public CompletableFuture<OperatorBackPressureStatsResponse> requestOperatorBackPressureStats(
 			final JobID jobId,
 			final JobVertexID jobVertexId) {
-		final CompletableFuture<JobMasterGateway> jobMasterGatewayFuture = getJobMasterGatewayFuture(jobId);
-
-		return jobMasterGatewayFuture.thenCompose((JobMasterGateway jobMasterGateway) -> jobMasterGateway.requestOperatorBackPressureStats(jobVertexId));
+		return performOperationOnJobMasterGateway(jobId, gateway -> gateway.requestOperatorBackPressureStats(jobVertexId));
 	}
 
 	@Override
 	public CompletableFuture<ArchivedExecutionGraph> requestJob(JobID jobId, Time timeout) {
-		final CompletableFuture<JobMasterGateway> jobMasterGatewayFuture = getJobMasterGatewayFuture(jobId);
-
-		final CompletableFuture<ArchivedExecutionGraph> archivedExecutionGraphFuture = jobMasterGatewayFuture.thenCompose(
-			(JobMasterGateway jobMasterGateway) -> jobMasterGateway.requestJob(timeout));
-
-		return archivedExecutionGraphFuture.exceptionally(
-			(Throwable throwable) -> {
-				final ArchivedExecutionGraph serializableExecutionGraph = archivedExecutionGraphStore.get(jobId);
-
-				// check whether it is a completed job
-				if (serializableExecutionGraph == null) {
-					throw new CompletionException(ExceptionUtils.stripCompletionException(throwable));
-				} else {
-					return serializableExecutionGraph;
-				}
-			});
+		Function<Throwable, ArchivedExecutionGraph> checkExecutionGraphStoreeOnException = throwable ->  {
+			// check whether it is a completed job
+			final ArchivedExecutionGraph archivedExecutionGraph = archivedExecutionGraphStore.get(jobId);
+			if (archivedExecutionGraph == null) {
+				throw new CompletionException(ExceptionUtils.stripCompletionException(throwable));
+			} else {
+				return archivedExecutionGraph;
+			}
+		};
+		Optional<DispatcherJob> maybeJob = getDispatcherJob(jobId);
+		if (maybeJob.isPresent()) {
+			DispatcherJob job = maybeJob.get();
+			return job.requestJob(timeout).exceptionally(checkExecutionGraphStoreeOnException);
+		} else {
+			return CompletableFuture.completedFuture(checkExecutionGraphStoreeOnException.apply(new FlinkJobNotFoundException(jobId)));
+		}

Review comment:
       I different way to express this logic could be:
   
   ```
   return maybeJob.map(job -> job.requestJob(timeout))
   			.orElse(FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)))
   			.exceptionally(checkExecutionGraphStoreOnException);
   ```

##########
File path: flink-clients/src/test/java/org/apache/flink/client/ClientUtilsTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.client.JobInitializationException;
+import org.apache.flink.runtime.jobmaster.JobResult;
+import org.apache.flink.util.SerializedThrowable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Iterator;
+
+/**
+ * Test for the ClientUtils.
+ */
+public class ClientUtilsTest extends TestLogger {
+
+	/**
+	 * Ensure that the waitUntilJobInitializationFinished() method throws JobInitializationException.
+	 */
+	@Test
+	public void testWaitUntilJobInitializationFinished_throwsInitializationException() {
+		JobID jobID = new JobID();
+
+		Iterator<JobStatus> statusSequenceIterator = Arrays.asList(
+			JobStatus.INITIALIZING,
+			JobStatus.INITIALIZING,
+			JobStatus.FAILED).iterator();
+
+		CommonTestUtils.assertThrows("Something is wrong", JobInitializationException.class, () -> {
+			ClientUtils.waitUntilJobInitializationFinished(
+				statusSequenceIterator::next, () -> {

Review comment:
       line break after `statusSequenceIterator::next,` could make the code more readable.

##########
File path: flink-clients/src/test/java/org/apache/flink/client/ClientUtilsTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.client.JobInitializationException;
+import org.apache.flink.runtime.jobmaster.JobResult;
+import org.apache.flink.util.SerializedThrowable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Iterator;
+
+/**
+ * Test for the ClientUtils.
+ */
+public class ClientUtilsTest extends TestLogger {
+
+	/**
+	 * Ensure that the waitUntilJobInitializationFinished() method throws JobInitializationException.
+	 */
+	@Test
+	public void testWaitUntilJobInitializationFinished_throwsInitializationException() {
+		JobID jobID = new JobID();
+
+		Iterator<JobStatus> statusSequenceIterator = Arrays.asList(
+			JobStatus.INITIALIZING,
+			JobStatus.INITIALIZING,
+			JobStatus.FAILED).iterator();
+
+		CommonTestUtils.assertThrows("Something is wrong", JobInitializationException.class, () -> {
+			ClientUtils.waitUntilJobInitializationFinished(
+				statusSequenceIterator::next, () -> {
+					SerializedThrowable throwable = new SerializedThrowable(new JobInitializationException(
+						jobID,
+						"Something is wrong",
+						new RuntimeException("Err")));
+					return new JobResult.Builder().jobId(jobID).serializedThrowable(throwable).netRuntime(1).build();

Review comment:
       I'd move this out of the lambda to make it a bit easier.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -687,7 +840,39 @@ public JobManagerRunner createJobManagerRunner(
 				jobManagerSharedServices,
 				jobManagerJobMetricGroupFactory,
 				fatalErrorHandler);
+
+			return jobManagerRunner;

Review comment:
       nit: this could be reverted.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.ExceptionUtils;
+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<DispatcherJobResult> 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,
+		// then we consider the JobManager to be initialized.
+		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) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName) {
+		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 (jobStatus != DispatcherJobStatus.CANCELLING) {
+					jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+				}
+
+				if (throwable == null) { // initialization succeeded
+					// Forward result future
+					jobManagerRunner.getResultFuture().whenComplete((archivedExecutionGraph, resultThrowable) -> {
+						if (archivedExecutionGraph != null) {
+							jobResultFuture.complete(DispatcherJobResult.forSuccess(archivedExecutionGraph));
+						} else {
+							jobResultFuture.completeExceptionally(ExceptionUtils.stripCompletionException(resultThrowable));
+						}
+					});

Review comment:
       One could express this block a bit more succinctly:
   
   ```
   FutureUtils.forward(
   	jobManagerRunner.getResultFuture().thenApply(DispatcherJobResult::forSuccess),
   	jobResultFuture);
   ```

##########
File path: flink-clients/src/main/java/org/apache/flink/client/program/PerJobMiniClusterFactory.java
##########
@@ -81,13 +82,20 @@ private PerJobMiniClusterFactory(
 	/**
 	 * Starts a {@link MiniCluster} and submits a job.
 	 */
-	public CompletableFuture<JobClient> submitJob(JobGraph jobGraph) throws Exception {
+	public CompletableFuture<JobClient> submitJob(JobGraph jobGraph, ClassLoader userCodeClassloader) throws Exception {
 		MiniClusterConfiguration miniClusterConfig = getMiniClusterConfig(jobGraph.getMaximumParallelism());
 		MiniCluster miniCluster = miniClusterFactory.apply(miniClusterConfig);
 		miniCluster.start();
 
 		return miniCluster
 			.submitJob(jobGraph)
+			.thenApplyAsync(FunctionUtils.uncheckedFunction(submissionResult -> {
+				org.apache.flink.client.ClientUtils.waitUntilJobInitializationFinished(
+					() -> miniCluster.getJobStatus(submissionResult.getJobID()).get(),
+					() -> miniCluster.requestJobResult(submissionResult.getJobID()).get(),
+					userCodeClassloader);
+				return submissionResult;

Review comment:
       Do we also have to do this for `MiniCluster.executeJobBlocking`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.runtime.executiongraph.ArchivedExecutionGraph;
+
+import javax.annotation.Nullable;
+
+/**
+ * Container for returning the ArchivedExecutionGraph and a flag whether the initialization failed.
+ * For initialization failures, the throwable is also attached, to avoid deserializing it from the ArchivedExecutionGraph.
+ */
+public class DispatcherJobResult {
+	private final boolean initializationFailure;
+	private final ArchivedExecutionGraph archivedExecutionGraph;
+	@Nullable
+	private final Throwable throwable;
+
+	private DispatcherJobResult(ArchivedExecutionGraph archivedExecutionGraph, Throwable throwable, boolean initializationFailure) {

Review comment:
       ```suggestion
   	private DispatcherJobResult(ArchivedExecutionGraph archivedExecutionGraph, @Nullable Throwable throwable, boolean initializationFailure) {
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.ExceptionUtils;
+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<DispatcherJobResult> 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,
+		// then we consider the JobManager to be initialized.
+		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) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName) {
+		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 (jobStatus != DispatcherJobStatus.CANCELLING) {
+					jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+				}
+
+				if (throwable == null) { // initialization succeeded
+					// Forward result future
+					jobManagerRunner.getResultFuture().whenComplete((archivedExecutionGraph, resultThrowable) -> {
+						if (archivedExecutionGraph != null) {
+							jobResultFuture.complete(DispatcherJobResult.forSuccess(archivedExecutionGraph));
+						} else {
+							jobResultFuture.completeExceptionally(ExceptionUtils.stripCompletionException(resultThrowable));
+						}
+					});
+				} else { // failure during initialization
+					final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(throwable);
+					ArchivedExecutionGraph archivedExecutionGraph = ArchivedExecutionGraph.createFromInitializingJob(
+						jobId,
+						jobName,
+						JobStatus.FAILED,
+						strippedThrowable,
+						initializationTimestamp);
+					jobResultFuture.complete(DispatcherJobResult.forInitializationFailure(archivedExecutionGraph, strippedThrowable));
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<DispatcherJobResult> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		return requestJobStatus(timeout).thenApply(status -> {
+			int[] tasksPerState = new int[ExecutionState.values().length];
+			synchronized (lock) {
+				return new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					status,
+					0,
+					tasksPerState,
+					0);
+			}
+		});
+	}
+
+	/**
+	 * Cancel job.
+	 * A cancellation will be scheduled if the initialization is not completed.
+	 * The returned future will complete exceptionally if the JobManagerRunner initialization failed.
+	 */
+	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
+				CompletableFuture<Acknowledge> cancelFuture = jobManagerRunnerFuture
+					.thenCompose(JobManagerRunner::getJobMasterGateway)
+					.thenCompose(jobMasterGateway -> jobMasterGateway.cancel(RpcUtils.INF_TIMEOUT));
+				cancelFuture.whenComplete((ignored, cancelThrowable) -> {
+					if (cancelThrowable != null) {
+						log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+					}
+				});
+				jobStatus = DispatcherJobStatus.CANCELLING;
+				jobResultFuture.whenComplete(((archivedExecutionGraph, throwable) -> {
+					synchronized (lock) {
+						if (archivedExecutionGraph != null) {
+							jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+						}
+					}
+				}));

Review comment:
       Why is this special casing needed? Why wouldn't it work if we unconditionally set `jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED` in the handle callback when creating the `DispatcherJob`?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
+import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.JobDetails;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static org.hamcrest.core.Is.is;
+
+/**
+ * Test for the {@link DispatcherJob} class.
+ */
+public class DispatcherJobTest extends TestLogger {
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	@Test
+	public void testStatusWhenInitializing() throws Exception {
+		TestContext testContext = createTestContext();
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+	}
+
+	@Test
+	public void testStatusWhenRunning() throws Exception {
+		TestContext testContext = createTestContext();
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish initialization
+		testContext.setRunning();
+
+		assertJobStatus(dispatcherJob, JobStatus.RUNNING);
+
+		// result future not done
+		Assert.assertThat(dispatcherJob.getResultFuture().isDone(), is(false));
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+	}
+
+	@Test
+	public void testStatusWhenJobFinished() throws Exception {
+		TestContext testContext = createTestContext();
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// finish job
+		testContext.setRunning();
+		testContext.finishJob();
+
+		assertJobStatus(dispatcherJob, JobStatus.FINISHED);
+
+		// assert result future done
+		DispatcherJobResult result = dispatcherJob.getResultFuture().get();
+
+		Assert.assertThat(result.getArchivedExecutionGraph().getState(), is(JobStatus.FINISHED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileInitializing() throws Exception {
+		TestContext testContext = createTestContext();
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+		assertJobStatus(dispatcherJob, JobStatus.INITIALIZING);
+
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(
+			TIMEOUT);
+
+		Assert.assertThat(cancelFuture.isDone(), is(false));
+		Assert.assertThat(dispatcherJob.isInitialized(), is(false));
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+
+		testContext.setRunning();
+		testContext.finishCancellation();
+
+		// assert that cancel future completes
+		cancelFuture.get();
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		// assert that the result future completes
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getArchivedExecutionGraph().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileRunning() throws Exception {
+		TestContext testContext = createTestContext();
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		testContext.setRunning();
+		CompletableFuture<Acknowledge> cancelFuture = dispatcherJob.cancel(TIMEOUT);
+
+		assertJobStatus(dispatcherJob, JobStatus.CANCELLING);
+		testContext.finishCancellation();
+
+		cancelFuture.get();
+		assertJobStatus(dispatcherJob, JobStatus.CANCELED);
+		Assert.assertThat(dispatcherJob.getResultFuture().get().getArchivedExecutionGraph().getState(), is(JobStatus.CANCELED));
+	}
+
+	@Test
+	public void testStatusWhenCancellingWhileFailed() throws Exception {
+		TestContext testContext = createTestContext();
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		CommonTestUtils.assertThrows("Artificial failure", ExecutionException.class, () -> dispatcherJob.cancel(TIMEOUT).get());
+
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+	}
+
+	@Test
+	public void testErrorWhileInitializing() throws Exception {
+		TestContext testContext = createTestContext();
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		// now fail
+		RuntimeException exception = new RuntimeException("Artificial failure in runner initialization");
+		testContext.failInitialization(exception);
+
+		Assert.assertThat(dispatcherJob.isInitialized(), is(true));
+		assertJobStatus(dispatcherJob, JobStatus.FAILED);
+
+		ArchivedExecutionGraph aeg = dispatcherJob.getResultFuture().get().getArchivedExecutionGraph();
+		Assert.assertThat(aeg.getFailureInfo().getException().deserializeError(ClassLoader.getSystemClassLoader()), is(exception));
+	}
+
+	@Test
+	public void testCloseWhileInitializingSuccessfully() throws Exception {
+		TestContext testContext = createTestContext();
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		// set job running, so that we can cancel it
+		testContext.setRunning();
+
+		// assert future completes now
+		closeFuture.get();
+
+		// ensure the result future is complete (how it completes is up to the JobManager)
+		CompletableFuture<DispatcherJobResult> resultFuture = dispatcherJob.getResultFuture();
+		CommonTestUtils.assertThrows("has not been finished", ExecutionException.class,
+			resultFuture::get);
+	}
+
+	@Test
+	public void testCloseWhileInitializingErroneously() throws Exception {
+		TestContext testContext = createTestContext();
+		DispatcherJob dispatcherJob = testContext.getDispatcherJob();
+
+		CompletableFuture<Void> closeFuture = dispatcherJob.closeAsync();
+		Assert.assertThat(closeFuture.isDone(), is(false));
+
+		testContext.failInitialization(new RuntimeException("fail"));
+
+		// assert future completes now
+		closeFuture.get();
+
+		// ensure the result future is complete
+		DispatcherJobResult result = dispatcherJob.getResultFuture().get();
+		Assert.assertThat(result.isInitializationFailure(), is(true));
+		Assert.assertThat(result.getArchivedExecutionGraph().getState(), is(JobStatus.FAILED));

Review comment:
       I would make this a separate test case which check what the resultFuture value is if there is an initialization error.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.ExceptionUtils;
+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<DispatcherJobResult> 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,
+		// then we consider the JobManager to be initialized.
+		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) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName) {
+		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 (jobStatus != DispatcherJobStatus.CANCELLING) {
+					jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+				}
+
+				if (throwable == null) { // initialization succeeded
+					// Forward result future
+					jobManagerRunner.getResultFuture().whenComplete((archivedExecutionGraph, resultThrowable) -> {
+						if (archivedExecutionGraph != null) {
+							jobResultFuture.complete(DispatcherJobResult.forSuccess(archivedExecutionGraph));
+						} else {
+							jobResultFuture.completeExceptionally(ExceptionUtils.stripCompletionException(resultThrowable));
+						}
+					});
+				} else { // failure during initialization
+					final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(throwable);
+					ArchivedExecutionGraph archivedExecutionGraph = ArchivedExecutionGraph.createFromInitializingJob(
+						jobId,
+						jobName,
+						JobStatus.FAILED,
+						strippedThrowable,
+						initializationTimestamp);
+					jobResultFuture.complete(DispatcherJobResult.forInitializationFailure(archivedExecutionGraph, strippedThrowable));
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<DispatcherJobResult> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		return requestJobStatus(timeout).thenApply(status -> {
+			int[] tasksPerState = new int[ExecutionState.values().length];
+			synchronized (lock) {
+				return new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					status,
+					0,
+					tasksPerState,
+					0);
+			}
+		});
+	}
+
+	/**
+	 * Cancel job.
+	 * A cancellation will be scheduled if the initialization is not completed.
+	 * The returned future will complete exceptionally if the JobManagerRunner initialization failed.
+	 */
+	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
+				CompletableFuture<Acknowledge> cancelFuture = jobManagerRunnerFuture
+					.thenCompose(JobManagerRunner::getJobMasterGateway)
+					.thenCompose(jobMasterGateway -> jobMasterGateway.cancel(RpcUtils.INF_TIMEOUT));

Review comment:
       Why are we using `RpcUtils.INF_TIMEOUT` here instead of `timeout`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.runtime.executiongraph.ArchivedExecutionGraph;
+
+import javax.annotation.Nullable;
+
+/**
+ * Container for returning the ArchivedExecutionGraph and a flag whether the initialization failed.
+ * For initialization failures, the throwable is also attached, to avoid deserializing it from the ArchivedExecutionGraph.
+ */
+public class DispatcherJobResult {
+	private final boolean initializationFailure;
+	private final ArchivedExecutionGraph archivedExecutionGraph;
+	@Nullable
+	private final Throwable throwable;
+
+	private DispatcherJobResult(ArchivedExecutionGraph archivedExecutionGraph, Throwable throwable, boolean initializationFailure) {
+		this.archivedExecutionGraph = archivedExecutionGraph;
+		this.initializationFailure = initializationFailure;
+		this.throwable = throwable;
+	}
+
+	public boolean isInitializationFailure() {
+		return initializationFailure;
+	}
+
+	public ArchivedExecutionGraph getArchivedExecutionGraph() {
+		return archivedExecutionGraph;
+	}
+
+	public Throwable getThrowable() {
+		return throwable;
+	}
+
+	public static DispatcherJobResult forInitializationFailure(ArchivedExecutionGraph archivedExecutionGraph, Throwable throwable) {
+		return new DispatcherJobResult(archivedExecutionGraph, throwable, true);
+	}
+
+	public static DispatcherJobResult forSuccess(ArchivedExecutionGraph archivedExecutionGraph) {
+		return new DispatcherJobResult(archivedExecutionGraph, null, false);
+	}

Review comment:
       Nice.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.runtime.executiongraph.ArchivedExecutionGraph;
+
+import javax.annotation.Nullable;
+
+/**
+ * Container for returning the ArchivedExecutionGraph and a flag whether the initialization failed.

Review comment:
       ```suggestion
    * Container for returning the {@link ArchivedExecutionGraph} and a flag whether the initialization has failed.
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.runtime.executiongraph.ArchivedExecutionGraph;
+
+import javax.annotation.Nullable;
+
+/**
+ * Container for returning the ArchivedExecutionGraph and a flag whether the initialization failed.
+ * For initialization failures, the throwable is also attached, to avoid deserializing it from the ArchivedExecutionGraph.
+ */
+public class DispatcherJobResult {
+	private final boolean initializationFailure;

Review comment:
       Couldn't we get rid of this field by saying `throwable` encodes (being set or not) whether it was an initialization failure or not?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.runtime.executiongraph.ArchivedExecutionGraph;
+
+import javax.annotation.Nullable;
+
+/**
+ * Container for returning the ArchivedExecutionGraph and a flag whether the initialization failed.
+ * For initialization failures, the throwable is also attached, to avoid deserializing it from the ArchivedExecutionGraph.
+ */
+public class DispatcherJobResult {
+	private final boolean initializationFailure;
+	private final ArchivedExecutionGraph archivedExecutionGraph;
+	@Nullable
+	private final Throwable throwable;
+
+	private DispatcherJobResult(ArchivedExecutionGraph archivedExecutionGraph, Throwable throwable, boolean initializationFailure) {
+		this.archivedExecutionGraph = archivedExecutionGraph;
+		this.initializationFailure = initializationFailure;
+		this.throwable = throwable;
+	}
+
+	public boolean isInitializationFailure() {
+		return initializationFailure;
+	}
+
+	public ArchivedExecutionGraph getArchivedExecutionGraph() {
+		return archivedExecutionGraph;
+	}
+
+	public Throwable getThrowable() {
+		return throwable;

Review comment:
       before returning `throwable` we should add a check statement which ensures that `throwable` is non null.

##########
File path: flink-clients/src/main/java/org/apache/flink/client/program/PerJobMiniClusterFactory.java
##########
@@ -81,13 +82,20 @@ private PerJobMiniClusterFactory(
 	/**
 	 * Starts a {@link MiniCluster} and submits a job.
 	 */
-	public CompletableFuture<JobClient> submitJob(JobGraph jobGraph) throws Exception {
+	public CompletableFuture<JobClient> submitJob(JobGraph jobGraph, ClassLoader userCodeClassloader) throws Exception {
 		MiniClusterConfiguration miniClusterConfig = getMiniClusterConfig(jobGraph.getMaximumParallelism());
 		MiniCluster miniCluster = miniClusterFactory.apply(miniClusterConfig);
 		miniCluster.start();
 
 		return miniCluster
 			.submitJob(jobGraph)
+			.thenApplyAsync(FunctionUtils.uncheckedFunction(submissionResult -> {
+				org.apache.flink.client.ClientUtils.waitUntilJobInitializationFinished(
+					() -> miniCluster.getJobStatus(submissionResult.getJobID()).get(),
+					() -> miniCluster.requestJobResult(submissionResult.getJobID()).get(),
+					userCodeClassloader);
+				return submissionResult;

Review comment:
       Given that no test fails, I guess it should be fine.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.runtime.executiongraph.ArchivedExecutionGraph;
+
+import javax.annotation.Nullable;
+
+/**
+ * Container for returning the ArchivedExecutionGraph and a flag whether the initialization failed.
+ * For initialization failures, the throwable is also attached, to avoid deserializing it from the ArchivedExecutionGraph.
+ */
+public class DispatcherJobResult {

Review comment:
       Could probably be package private.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.ExceptionUtils;
+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<DispatcherJobResult> 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,
+		// then we consider the JobManager to be initialized.
+		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) {

Review comment:
       Just a very minor comment: By indenting the argument list with two tabs, one can make it easier to distinguish from the method body.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475590387



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;

Review comment:
       This is a very good idea! Makes the code more concise.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85c9bbac2d9ca07435fbd80d76208fa2a51e5d37 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r482801219



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -107,18 +106,13 @@ private DispatcherJob(
 		FutureUtils.assertNoException(this.jobManagerRunnerFuture.handle((jobManagerRunner, throwable) -> {
 			// JM has been initialized, or the initialization failed
 			synchronized (lock) {
-				if (jobStatus != DispatcherJobStatus.CANCELLING) {
-					jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
-				}
-
+				jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
 				if (throwable == null) { // initialization succeeded
 					// Forward result future
 					jobManagerRunner.getResultFuture().whenComplete((archivedExecutionGraph, resultThrowable) -> {
-						if (archivedExecutionGraph != null) {
-							jobResultFuture.complete(DispatcherJobResult.forSuccess(archivedExecutionGraph));
-						} else {
-							jobResultFuture.completeExceptionally(ExceptionUtils.stripCompletionException(resultThrowable));
-						}
+						FutureUtils.forward(
+							jobManagerRunner.getResultFuture().thenApply(DispatcherJobResult::forSuccess),
+							jobResultFuture);
 					});

Review comment:
       This was again ill advice because by calling `thenApply` we potentially introduce a `CompletionException` which will make the `DispatcherResourceCleanupTest.testBlobServerCleanupWhenJobNotFinished` fail. Hence, I would suggest to revert this change. 
   
   On a side note: What I meant was actually to replace the `whenComplete` call with
   
   ```
   FutureUtils.forward(
   	  jobManagerRunner.getResultFuture().thenApply(DispatcherJobResult::forSuccess),
   	  jobResultFuture);
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-686343191


   When closing the corresponding ticket, please add a release note stating the difference in submission semantics. E.g. when submitting a job one cannot immediately trigger job operation (triggering of savepoint, etc.). This is maybe one follow up issue we might wanna think through and adjust if it should become a usability problem.
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 31d810f7c4d93043d8444a78f921b263f1258646 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5860",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5872",
       "triggerID" : "c741363eaee4cbff8edaa3cab54c3bfbdd668184",
       "triggerType" : "PUSH"
     }, {
       "hash" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5875",
       "triggerID" : "41769d3c4ebae61f502e16398207d477f59451f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6051",
       "triggerID" : "08f8c0a7fbfecf7a6eba88653140439f3673c2cc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6114",
       "triggerID" : "c52eaf11a6832049ad9a18c784d2661101d91b8c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6118",
       "triggerID" : "31d810f7c4d93043d8444a78f921b263f1258646",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8036016c752bce433dc65d1c08695377c917836f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6162",
       "triggerID" : "8036016c752bce433dc65d1c08695377c917836f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 8036016c752bce433dc65d1c08695377c917836f Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=6162) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678277886


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 85c9bbac2d9ca07435fbd80d76208fa2a51e5d37 (Fri Aug 21 13:00:46 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475555435



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -350,71 +341,42 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
 		}, getRpcService().getExecutor());
 	}
 
-	private CompletableFuture<Void> persistAndRunJob(JobGraph jobGraph) throws Exception {
+	private Void persistAndRunJob(JobGraph jobGraph) throws Exception {

Review comment:
       shouldn't this also work if the method has the signature: `void persistAndRunJob(...)`?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r478371328



##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,48 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+				JobID jobID,
+				SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+				SupplierWithException<JobResult, Exception> jobResultSupplier,
+				ClassLoader userCodeClassloader)
+			throws JobInitializationException {
+		LOG.debug("Wait until job initialization is finished");
+		WaitStrategy waitStrategy = new ExponentialWaitStrategy(50, 2000);
+		try {
+			JobStatus status = jobStatusSupplier.get();
+			long attempt = 0;
+			while (status == JobStatus.INITIALIZING) {
+				Thread.sleep(waitStrategy.sleepTime(attempt++));
+				status = jobStatusSupplier.get();
+			}
+			if (status == JobStatus.FAILED) {
+				JobResult result = jobResultSupplier.get();
+				Optional<SerializedThrowable> throwable = result.getSerializedThrowable();
+				if (throwable.isPresent()) {
+					Throwable t = throwable.get().deserializeError(userCodeClassloader);
+					t = ExceptionUtils.stripCompletionException(t);

Review comment:
       Where does the `CompletionException` come from? Ideally we strip it before we create the failure result of the job execution/submission because it is an implementation detail which should not leak out.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -332,7 +329,8 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
 	private CompletableFuture<Acknowledge> internalSubmitJob(JobGraph jobGraph) {
 		log.info("Submitting job {} ({}).", jobGraph.getJobID(), jobGraph.getName());
 
-		final CompletableFuture<Acknowledge> persistAndRunFuture = waitForTerminatingJobManager(jobGraph.getJobID(), jobGraph, this::persistAndRunJob)
+		final CompletableFuture<Acknowledge> persistAndRunFuture = waitForTerminatingJobManager(jobGraph.getJobID(), jobGraph,

Review comment:
       ```suggestion
   		final CompletableFuture<Acknowledge> persistAndRunFuture = waitForTerminatingJob(jobGraph.getJobID(), jobGraph,
   ```

##########
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,
+		// then we consider the JobManager to be initialized.
+		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 (jobStatus != DispatcherJobStatus.CANCELLING) {
+					jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+				}
+				if (throwable == null) {
+					// Forward result future
+					FutureUtils.forward(jobManagerRunner.getResultFuture(), jobResultFuture);
+				} 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) {
+		return requestJobStatus(timeout).thenApply(status -> {
+			int[] tasksPerState = new int[ExecutionState.values().length];
+			synchronized (lock) {
+				return new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					status,
+					0,
+					tasksPerState,
+					0);
+			}
+		});
+	}
+
+	/**
+	 * Cancel job.
+	 * A cancellation will be scheduled if the initialization is not completed.
+	 * The returned future will complete exceptionally if the JobManagerRunner initialization failed.
+	 */
+	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(jobMasterGateway -> jobMasterGateway.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;

Review comment:
       `lock` is missing

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/UnavailableDispatcherOperationException.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.util.FlinkException;
+
+/**
+ * Exception indicating that a Dispatcher operation is temporarily unavailable.
+ */
+public class UnavailableDispatcherOperationException extends FlinkException {

Review comment:
       Maybe it could extend `DispatcherException`.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -465,17 +441,24 @@ private JobManagerRunner startJobManagerRunner(JobManagerRunner jobManagerRunner
 
 	@Override
 	public CompletableFuture<Acknowledge> cancelJob(JobID jobId, Time timeout) {
-		final CompletableFuture<JobMasterGateway> jobMasterGatewayFuture = getJobMasterGatewayFuture(jobId);
+		Optional<DispatcherJob> maybeJob = getDispatcherJob(jobId);
+		if (maybeJob.isPresent()) {
+			return maybeJob.get().cancel(timeout);
+		} else {
+			log.debug("Dispatcher is unable to cancel job {}: not found", jobId);
+			return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId));
+		}
+	}
 
-		return jobMasterGatewayFuture.thenCompose((JobMasterGateway jobMasterGateway) -> jobMasterGateway.cancel(timeout));
+	private Optional<DispatcherJob> getDispatcherJob(JobID jobId) {
+		return Optional.ofNullable(runningJobs.get(jobId));
 	}
 
 	@Override
 	public CompletableFuture<ClusterOverview> requestClusterOverview(Time timeout) {
 		CompletableFuture<ResourceOverview> taskManagerOverviewFuture = runResourceManagerCommand(resourceManagerGateway -> resourceManagerGateway.requestResourceOverview(timeout));
 
-		final List<CompletableFuture<Optional<JobStatus>>> optionalJobInformation = queryJobMastersForInformation(
-			(JobMasterGateway jobMasterGateway) -> jobMasterGateway.requestJobStatus(timeout));
+		final List<CompletableFuture<Optional<JobStatus>>> optionalJobInformation = queryJobMastersForInformation(dj -> dj.requestJobStatus(timeout));

Review comment:
       nit: In the absence of type information I wouldn't mind spelling variable names out. So instead of `dj` one could name it `dispatcherJob`. This will give the reader of the code a bit more information about the type of the parameter.

##########
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,
+		// then we consider the JobManager to be initialized.
+		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) {

Review comment:
       Looking at the `ExecutionType`, I am wondering whether this isn't more a concept of the user of a `DispatcherJob`. Differently said, why don't we let the `Dispatcher` decide how to handle an initialization error? If we wanted to do it like this, then we would have to return a bit more information via the `jobResultFuture` (e.g. whether it failed during the initialization). One could create a `DispatcherJobResult` which contains the `ArchivedExecutionGraph` and whether it failed during initialization or not.

##########
File path: flink-clients/src/main/java/org/apache/flink/client/program/PerJobMiniClusterFactory.java
##########
@@ -88,6 +89,13 @@ private PerJobMiniClusterFactory(
 
 		return miniCluster
 			.submitJob(jobGraph)
+			.thenApplyAsync(FunctionUtils.uncheckedFunction(submissionResult -> {
+				org.apache.flink.client.ClientUtils.waitUntilJobInitializationFinished(submissionResult.getJobID(),
+					() -> miniCluster.getJobStatus(submissionResult.getJobID()).get(),
+					() -> miniCluster.requestJobResult(submissionResult.getJobID()).get(),
+					Thread.currentThread().getContextClassLoader());

Review comment:
       Are we sure that the context class loader is the right class loader here?

##########
File path: flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
##########
@@ -111,4 +122,48 @@ public static void executeProgram(
 			Thread.currentThread().setContextClassLoader(contextClassLoader);
 		}
 	}
+
+	/**
+	 * This method blocks until the job status is not INITIALIZING anymore.
+	 * If the job is FAILED, it throws an CompletionException with the failure cause.
+	 * @param jobStatusSupplier supplier returning the job status.
+	 */
+	public static void waitUntilJobInitializationFinished(
+				JobID jobID,
+				SupplierWithException<JobStatus, Exception> jobStatusSupplier,
+				SupplierWithException<JobResult, Exception> jobResultSupplier,
+				ClassLoader userCodeClassloader)
+			throws JobInitializationException {
+		LOG.debug("Wait until job initialization is finished");
+		WaitStrategy waitStrategy = new ExponentialWaitStrategy(50, 2000);
+		try {
+			JobStatus status = jobStatusSupplier.get();
+			long attempt = 0;
+			while (status == JobStatus.INITIALIZING) {
+				Thread.sleep(waitStrategy.sleepTime(attempt++));
+				status = jobStatusSupplier.get();
+			}
+			if (status == JobStatus.FAILED) {
+				JobResult result = jobResultSupplier.get();
+				Optional<SerializedThrowable> throwable = result.getSerializedThrowable();
+				if (throwable.isPresent()) {
+					Throwable t = throwable.get().deserializeError(userCodeClassloader);
+					t = ExceptionUtils.stripCompletionException(t);
+					if (t instanceof JobInitializationException) {
+						throw t;
+					}
+				}
+			}
+		} catch (JobInitializationException initializationException) {
+			throw initializationException;
+		} catch (Throwable throwable) {
+			ExceptionUtils.checkInterrupted(throwable);
+			throw new JobInitializationException(jobID, "Error while waiting for job to be initialized", throwable);
+		}
+	}
+
+	public static <T> void waitUntilJobInitializationFinished(JobID id, ClusterClient<T> client, ClassLoader userCodeClassloader) throws
+		JobInitializationException {
+		waitUntilJobInitializationFinished(id, () -> client.getJobStatus(id).get(), () -> client.requestJobResult(id).get(), userCodeClassloader);

Review comment:
       `requestJobResult` can lead to a shut down of an attached per job cluster because the attached per job cluster waits until the job result has been served in case an attached deployment.

##########
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,
+		// then we consider the JobManager to be initialized.
+		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 (jobStatus != DispatcherJobStatus.CANCELLING) {
+					jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+				}
+				if (throwable == null) {
+					// Forward result future
+					FutureUtils.forward(jobManagerRunner.getResultFuture(), jobResultFuture);
+				} 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) {
+		return requestJobStatus(timeout).thenApply(status -> {
+			int[] tasksPerState = new int[ExecutionState.values().length];
+			synchronized (lock) {
+				return new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					status,
+					0,
+					tasksPerState,
+					0);
+			}
+		});
+	}
+
+	/**
+	 * Cancel job.
+	 * A cancellation will be scheduled if the initialization is not completed.
+	 * The returned future will complete exceptionally if the JobManagerRunner initialization failed.
+	 */
+	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(jobMasterGateway -> jobMasterGateway.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);
+	}
+
+	/**
+	 * Returns a future completing to the ArchivedExecutionGraph of the job.
+	 */
+	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(this.jobStatus == DispatcherJobStatus.INITIALIZING || jobStatus == DispatcherJobStatus.CANCELLING);
+				return CompletableFuture.completedFuture(
+					ArchivedExecutionGraph.createFromInitializingJob(
+						jobId,
+						jobName,
+						jobStatus.asJobStatus(),
+						null,
+						initializationTimestamp));
+			}
+		}
+	}
+
+	/**
+	 * The job is initialized once the JobManager runner has been initialized.
+	 * It is also initialized if the runner initialization failed, or of it has been
+	 * canceled (and the cancellation is complete).
+	 */
+	public boolean isInitialized() {
+		synchronized (lock) {
+			return jobStatus == DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+		}
+	}
+
+	/**
+	 * Returns the {@link JobMasterGateway} from the JobManagerRunner.
+	 * This method will fail with an {@link IllegalStateException} if the job is initialized.
+	 * The returned future will complete exceptionally if the JobManagerRunner initialization failed.

Review comment:
       This sentence should go into the `@return ....` line.

##########
File path: flink-clients/src/test/java/org/apache/flink/client/ClientUtilsTest.java
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.BlobServerOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.blob.BlobServer;
+import org.apache.flink.runtime.blob.VoidBlobStore;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.client.JobInitializationException;
+import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.DefaultDispatcherBootstrap;
+import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory;
+import org.apache.flink.runtime.dispatcher.Dispatcher;
+import org.apache.flink.runtime.dispatcher.DispatcherGateway;
+import org.apache.flink.runtime.dispatcher.DispatcherId;
+import org.apache.flink.runtime.dispatcher.DispatcherServices;
+import org.apache.flink.runtime.dispatcher.DispatcherTest;
+import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.NoOpJobGraphWriter;
+import org.apache.flink.runtime.dispatcher.TestingDispatcher;
+import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.JobGraphWriter;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcUtils;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Test for the ClientUtils.
+ */
+public class ClientUtilsTest extends TestLogger {
+
+	@ClassRule
+	public static TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+
+	private static TestingRpcService rpcService;
+
+	@Rule
+	public final TestingFatalErrorHandlerResource testingFatalErrorHandlerResource = new TestingFatalErrorHandlerResource();
+
+	final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 10000L);
+
+	final TestingLeaderElectionService jobMasterLeaderElectionService = new TestingLeaderElectionService();
+
+	final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
+
+	private BlobServer blobServer;
+	private DefaultDispatcherBootstrap dispatcherBootstrap;
+	private Configuration configuration;
+	private ResourceManagerGateway resourceManagerGateway;
+	private ArchivedExecutionGraphStore archivedExecutionGraphStore;
+	private JobGraphWriter jobGraphWriter;
+	private DefaultJobManagerRunnerFactory jobManagerRunnerFactory;
+
+	@Before
+	public void setUp() throws Exception {
+		haServices.setJobMasterLeaderElectionService(TEST_JOB_ID, jobMasterLeaderElectionService);
+		haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory());
+		haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService());
+
+		configuration = new Configuration();
+
+		configuration.setString(
+			BlobServerOptions.STORAGE_DIRECTORY,
+			temporaryFolder.newFolder().getAbsolutePath());
+
+		blobServer = new BlobServer(configuration, new VoidBlobStore());
+		resourceManagerGateway = new TestingResourceManagerGateway();
+		archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore();
+		dispatcherBootstrap = new DefaultDispatcherBootstrap(Collections.emptyList());
+
+		jobGraphWriter = NoOpJobGraphWriter.INSTANCE;
+		jobManagerRunnerFactory = DefaultJobManagerRunnerFactory.INSTANCE;
+	}
+
+	@BeforeClass
+	public static void setupClass() {
+		rpcService = new TestingRpcService();
+	}
+
+	@AfterClass
+	public static void teardownClass() throws Exception {
+		if (rpcService != null) {
+			RpcUtils.terminateRpcService(rpcService, TIMEOUT);
+
+			rpcService = null;
+		}
+	}
+
+	private TestingDispatcher createAndStartDispatcher() throws Exception {
+		final TestingDispatcher dispatcher =
+			new TestingDispatcher(
+				rpcService,
+				DispatcherId.generate(),
+				dispatcherBootstrap,
+				new DispatcherServices(
+					configuration,
+					haServices,
+					() -> CompletableFuture.completedFuture(resourceManagerGateway),
+					blobServer,
+					heartbeatServices,
+					archivedExecutionGraphStore,
+					testingFatalErrorHandlerResource.getFatalErrorHandler(),
+					VoidHistoryServerArchivist.INSTANCE,
+					null,
+					UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(),
+					jobGraphWriter,
+					jobManagerRunnerFactory));
+		dispatcher.start();
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		return dispatcher;
+	}
+
+	@Test
+	public void testWaitUntilJobInitializationFinished_throwsInitializationException() throws
+		Exception {
+		final JobVertex testVertex = new DispatcherTest.FailingInitializationJobVertex("testVertex");
+		testVertex.setInvokableClass(NoOpInvokable.class);
+
+		JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex);
+
+		Dispatcher dispatcher = createAndStartDispatcher();
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
+
+		CommonTestUtils.assertThrows("Could not instantiate JobManager", JobInitializationException.class, () -> {
+			ClientUtils.waitUntilJobInitializationFinished(jobGraph.getJobID(),
+				() -> dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(),
+				() -> dispatcherGateway.requestJobResult(jobGraph.getJobID(), TIMEOUT).get(),
+				ClassLoader.getSystemClassLoader());
+			return null;
+		});
+		CommonTestUtils.assertThrows("Could not instantiate JobManager", ExecutionException.class, () -> {

Review comment:
       Why does the shut down of the Dispatcher fail if the job submission failed? A submission failure is something which should not affect Flink itself because it can happen and is normal.

##########
File path: flink-clients/src/test/java/org/apache/flink/client/ClientUtilsTest.java
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.BlobServerOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.blob.BlobServer;
+import org.apache.flink.runtime.blob.VoidBlobStore;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.client.JobInitializationException;
+import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.DefaultDispatcherBootstrap;
+import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory;
+import org.apache.flink.runtime.dispatcher.Dispatcher;
+import org.apache.flink.runtime.dispatcher.DispatcherGateway;
+import org.apache.flink.runtime.dispatcher.DispatcherId;
+import org.apache.flink.runtime.dispatcher.DispatcherServices;
+import org.apache.flink.runtime.dispatcher.DispatcherTest;
+import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.NoOpJobGraphWriter;
+import org.apache.flink.runtime.dispatcher.TestingDispatcher;
+import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.JobGraphWriter;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcUtils;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Test for the ClientUtils.
+ */
+public class ClientUtilsTest extends TestLogger {
+
+	@ClassRule
+	public static TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+	private static final JobID TEST_JOB_ID = new JobID();
+
+	private static final Time TIMEOUT = Time.seconds(10L);
+
+	private static TestingRpcService rpcService;
+
+	@Rule
+	public final TestingFatalErrorHandlerResource testingFatalErrorHandlerResource = new TestingFatalErrorHandlerResource();
+
+	final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 10000L);
+
+	final TestingLeaderElectionService jobMasterLeaderElectionService = new TestingLeaderElectionService();
+
+	final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
+
+	private BlobServer blobServer;
+	private DefaultDispatcherBootstrap dispatcherBootstrap;
+	private Configuration configuration;
+	private ResourceManagerGateway resourceManagerGateway;
+	private ArchivedExecutionGraphStore archivedExecutionGraphStore;
+	private JobGraphWriter jobGraphWriter;
+	private DefaultJobManagerRunnerFactory jobManagerRunnerFactory;
+
+	@Before
+	public void setUp() throws Exception {
+		haServices.setJobMasterLeaderElectionService(TEST_JOB_ID, jobMasterLeaderElectionService);
+		haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory());
+		haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService());
+
+		configuration = new Configuration();
+
+		configuration.setString(
+			BlobServerOptions.STORAGE_DIRECTORY,
+			temporaryFolder.newFolder().getAbsolutePath());
+
+		blobServer = new BlobServer(configuration, new VoidBlobStore());
+		resourceManagerGateway = new TestingResourceManagerGateway();
+		archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore();
+		dispatcherBootstrap = new DefaultDispatcherBootstrap(Collections.emptyList());
+
+		jobGraphWriter = NoOpJobGraphWriter.INSTANCE;
+		jobManagerRunnerFactory = DefaultJobManagerRunnerFactory.INSTANCE;
+	}
+
+	@BeforeClass
+	public static void setupClass() {
+		rpcService = new TestingRpcService();
+	}
+
+	@AfterClass
+	public static void teardownClass() throws Exception {
+		if (rpcService != null) {
+			RpcUtils.terminateRpcService(rpcService, TIMEOUT);
+
+			rpcService = null;
+		}
+	}
+
+	private TestingDispatcher createAndStartDispatcher() throws Exception {
+		final TestingDispatcher dispatcher =
+			new TestingDispatcher(
+				rpcService,
+				DispatcherId.generate(),
+				dispatcherBootstrap,
+				new DispatcherServices(
+					configuration,
+					haServices,
+					() -> CompletableFuture.completedFuture(resourceManagerGateway),
+					blobServer,
+					heartbeatServices,
+					archivedExecutionGraphStore,
+					testingFatalErrorHandlerResource.getFatalErrorHandler(),
+					VoidHistoryServerArchivist.INSTANCE,
+					null,
+					UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(),
+					jobGraphWriter,
+					jobManagerRunnerFactory));
+		dispatcher.start();
+		jobMasterLeaderElectionService.isLeader(UUID.randomUUID());
+		return dispatcher;
+	}
+
+	@Test
+	public void testWaitUntilJobInitializationFinished_throwsInitializationException() throws
+		Exception {
+		final JobVertex testVertex = new DispatcherTest.FailingInitializationJobVertex("testVertex");
+		testVertex.setInvokableClass(NoOpInvokable.class);
+
+		JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex);
+
+		Dispatcher dispatcher = createAndStartDispatcher();
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
+
+		CommonTestUtils.assertThrows("Could not instantiate JobManager", JobInitializationException.class, () -> {
+			ClientUtils.waitUntilJobInitializationFinished(jobGraph.getJobID(),
+				() -> dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(),
+				() -> dispatcherGateway.requestJobResult(jobGraph.getJobID(), TIMEOUT).get(),
+				ClassLoader.getSystemClassLoader());
+			return null;
+		});
+		CommonTestUtils.assertThrows("Could not instantiate JobManager", ExecutionException.class, () -> {
+			dispatcher.closeAsync().get();
+			return null;
+		});
+	}
+
+	@Test
+	public void testWaitUntilJobInitializationFinished_doesNotThrowRuntimeException() throws Exception {
+		final JobVertex testVertex = new JobVertex("testVertex");
+		testVertex.setInvokableClass(NoOpInvokable.class);
+
+		JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex);
+
+		TestingDispatcher dispatcher = createAndStartDispatcher();
+		DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
+
+		dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
+
+		// we don't expect an exception here
+		ClientUtils.waitUntilJobInitializationFinished(jobGraph.getJobID(),
+			() -> dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(),
+			() -> dispatcherGateway.requestJobResult(jobGraph.getJobID(), TIMEOUT).get(),
+			ClassLoader.getSystemClassLoader());
+
+		// now "fail" the job
+		dispatcher.completeJobExecution(
+			ArchivedExecutionGraph.createFromInitializingJob(TEST_JOB_ID, "test", JobStatus.FAILED, new RuntimeException("yolo"), 1337));
+		// ensure it is failed
+		org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition(() -> {
+			JobStatus status = dispatcherGateway.requestJobStatus(
+				jobGraph.getJobID(),
+				TIMEOUT).get();
+			return status == JobStatus.FAILED;
+		}, Deadline.fromNow(

Review comment:
       nit
   
   ```suggestion
   		org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition(() ->
                          dispatcherGateway.requestJobStatus(
   				jobGraph.getJobID(),
   				TIMEOUT).get() == JobStatus.FAILED, 
   		       Deadline.fromNow(
   ```

##########
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,
+		// then we consider the JobManager to be initialized.
+		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 (jobStatus != DispatcherJobStatus.CANCELLING) {
+					jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+				}
+				if (throwable == null) {
+					// Forward result future
+					FutureUtils.forward(jobManagerRunner.getResultFuture(), jobResultFuture);
+				} 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) {
+		return requestJobStatus(timeout).thenApply(status -> {
+			int[] tasksPerState = new int[ExecutionState.values().length];
+			synchronized (lock) {
+				return new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					status,
+					0,
+					tasksPerState,
+					0);
+			}
+		});
+	}
+
+	/**
+	 * Cancel job.
+	 * A cancellation will be scheduled if the initialization is not completed.
+	 * The returned future will complete exceptionally if the JobManagerRunner initialization failed.
+	 */
+	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(jobMasterGateway -> jobMasterGateway.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());

Review comment:
       The returned future behaves differently depending on `isInitialized`. In this case here we wait for the job to have completely finished. In the if branch, we only wait for the cancel call to be acknowledged. I'd suggest to do the same here.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.concurrent.GuardedBy;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Abstraction used by the {@link Dispatcher} to manage JobManagers, in
+ * particular during initialization.
+ * While a job is initializing, the JobMasterGateway is not available. A small subset
+ * of the methods of the JobMasterGateway necessary during initialization are provided
+ * by this class (job details, cancel).
+ */
+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 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 JobStatus jobStatus = JobStatus.INITIALIZING;
+
+	private final CompletableFuture<Void> terminationFuture = new CompletableFuture<>();
+
+	private enum SubmissionType {
+		INITIAL, RECOVERY
+	}
+
+	static DispatcherJob createForSubmission(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.INITIAL);
+	}
+
+	static DispatcherJob createForRecovery(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp) {
+		return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp, SubmissionType.RECOVERY);
+	}
+
+	private DispatcherJob(
+		CompletableFuture<JobManagerRunner> jobManagerRunnerFuture,
+		JobID jobId,
+		String jobName,
+		long initializationTimestamp,
+		SubmissionType submissionType) {
+		this.jobManagerRunnerFuture = jobManagerRunnerFuture;
+		this.jobId = jobId;
+		this.jobName = jobName;
+		this.initializationTimestamp = initializationTimestamp;
+		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 == JobStatus.CANCELLING) {
+						log.info("Cancellation during initialization has been requested for job {}. Initialization completed, cancelling job.", jobId);
+
+						// cancel job
+						jobManagerRunner
+							.getJobMasterGateway()
+							.thenCompose(gw -> gw.cancel(RpcUtils.INF_TIMEOUT))
+							.whenComplete((ignored, cancelThrowable) -> {
+							if (cancelThrowable != null) {
+								log.warn("Cancellation of job {} failed", jobId, cancelThrowable);
+							}
+						});
+
+						// cancellation will eventually complete the jobResultFuture
+						jobResultFuture.whenComplete((archivedExecutionGraph, resultThrowable) -> {
+							synchronized (lock) {
+								if (resultThrowable == null) {
+									jobStatus = archivedExecutionGraph.getState();
+								} else {
+									jobStatus = JobStatus.FAILED;
+								}
+							}
+						});
+					} else {
+						jobStatus = JobStatus.RUNNING; // this status should never be exposed from the DispatcherJob. Only used internally for tracking running state
+					}
+				} else { // failure during initialization
+					if (submissionType == SubmissionType.RECOVERY) {
+						jobResultFuture.completeExceptionally(throwable);
+					} else {
+						jobResultFuture.complete(ArchivedExecutionGraph.createFromInitializingJob(
+							jobId,
+							jobName,
+							throwable,
+							JobStatus.FAILED,
+							initializationTimestamp));
+					}
+				}
+			}
+			return null;
+		}));
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+		return jobResultFuture;
+	}
+
+	public CompletableFuture<JobDetails> requestJobDetails(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobDetails(
+					timeout));
+			} else {
+				int[] tasksPerState = new int[ExecutionState.values().length];
+				return CompletableFuture.completedFuture(new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					jobStatus,
+					0,
+					tasksPerState,
+					0));
+			}
+		}
+	}
+
+	public CompletableFuture<Acknowledge> cancel(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				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);
+				jobStatus = JobStatus.CANCELLING;
+				return jobResultFuture.thenApply(ignored -> Acknowledge.get());
+			}
+		}
+	}
+
+	public CompletableFuture<JobStatus> requestJobStatus(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJobStatus(timeout));
+			} else {
+				return CompletableFuture.completedFuture(jobStatus);
+			}
+		}
+	}
+
+	public CompletableFuture<ArchivedExecutionGraph> requestJob(Time timeout) {
+		synchronized (lock) {
+			if (isRunning()) {
+				return getJobMasterGateway().thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout));
+			} else {
+				return CompletableFuture.supplyAsync(() -> ArchivedExecutionGraph.createFromInitializingJob(jobId, jobName, null, jobStatus, initializationTimestamp));
+			}
+		}
+	}
+
+	public boolean isRunning() {
+		synchronized (lock) {
+			return jobStatus == JobStatus.RUNNING;
+		}
+	}
+
+	public CompletableFuture<JobMasterGateway> getJobMasterGateway() {

Review comment:
       In JavaDocs on usually uses `@throw IllegalStateException is thrown if the job is not initialized` after the `@return` tag.

##########
File path: flink-clients/src/test/java/org/apache/flink/client/ClientUtilsTest.java
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.client;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.BlobServerOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.blob.BlobServer;
+import org.apache.flink.runtime.blob.VoidBlobStore;
+import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.client.JobInitializationException;
+import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.DefaultDispatcherBootstrap;
+import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory;
+import org.apache.flink.runtime.dispatcher.Dispatcher;
+import org.apache.flink.runtime.dispatcher.DispatcherGateway;
+import org.apache.flink.runtime.dispatcher.DispatcherId;
+import org.apache.flink.runtime.dispatcher.DispatcherServices;
+import org.apache.flink.runtime.dispatcher.DispatcherTest;
+import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore;
+import org.apache.flink.runtime.dispatcher.NoOpJobGraphWriter;
+import org.apache.flink.runtime.dispatcher.TestingDispatcher;
+import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist;
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.runtime.heartbeat.HeartbeatServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobmanager.JobGraphWriter;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcUtils;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Test for the ClientUtils.
+ */
+public class ClientUtilsTest extends TestLogger {
+
+	@ClassRule

Review comment:
       Yes, the setup is quite manual. There is `MiniClusterResource` which spawns a `MiniCluster` as an external resource.
   
   I actually believe that we don't really need to start a `MiniCluster` and submit a job. `ClientUtils. waitUntilJobInitializationFinished` is abstracted so well that it can be tested in isolation. Just provide the respective supplier which produce a failure or not to the utility and then test its behaviour.

##########
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,
+		// then we consider the JobManager to be initialized.
+		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 (jobStatus != DispatcherJobStatus.CANCELLING) {
+					jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+				}
+				if (throwable == null) {
+					// Forward result future
+					FutureUtils.forward(jobManagerRunner.getResultFuture(), jobResultFuture);
+				} 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) {
+		return requestJobStatus(timeout).thenApply(status -> {
+			int[] tasksPerState = new int[ExecutionState.values().length];
+			synchronized (lock) {
+				return new JobDetails(
+					jobId,
+					jobName,
+					initializationTimestamp,
+					0,
+					0,
+					status,
+					0,
+					tasksPerState,
+					0);
+			}
+		});
+	}
+
+	/**
+	 * Cancel job.
+	 * A cancellation will be scheduled if the initialization is not completed.
+	 * The returned future will complete exceptionally if the JobManagerRunner initialization failed.
+	 */
+	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(jobMasterGateway -> jobMasterGateway.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);
+	}
+
+	/**
+	 * Returns a future completing to the ArchivedExecutionGraph of the job.
+	 */
+	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(this.jobStatus == DispatcherJobStatus.INITIALIZING || jobStatus == DispatcherJobStatus.CANCELLING);
+				return CompletableFuture.completedFuture(
+					ArchivedExecutionGraph.createFromInitializingJob(
+						jobId,
+						jobName,
+						jobStatus.asJobStatus(),
+						null,
+						initializationTimestamp));
+			}
+		}
+	}
+
+	/**
+	 * The job is initialized once the JobManager runner has been initialized.
+	 * It is also initialized if the runner initialization failed, or of it has been
+	 * canceled (and the cancellation is complete).
+	 */
+	public boolean isInitialized() {
+		synchronized (lock) {
+			return jobStatus == DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED;
+		}
+	}
+
+	/**
+	 * Returns the {@link JobMasterGateway} from the JobManagerRunner.
+	 * This method will fail with an {@link IllegalStateException} if the job is initialized.

Review comment:
       ```suggestion
   	 * This method will fail with an {@link IllegalStateException} if the job is not initialized.
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678283618


   Note: I'm still investigating one last unstable test `FunctionITCase.testInvalidUseOfTableFunction()`


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85c9bbac2d9ca07435fbd80d76208fa2a51e5d37 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776) 
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 9127b879bec8dc1c05176c1acb89680c0e7baf04 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] tillrohrmann commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
tillrohrmann commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475565191



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -350,71 +341,42 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
 		}, getRpcService().getExecutor());
 	}
 
-	private CompletableFuture<Void> persistAndRunJob(JobGraph jobGraph) throws Exception {
+	private Void persistAndRunJob(JobGraph jobGraph) throws Exception {

Review comment:
       Hmm, I think the problem is that you are using `persistAndRunJob` still as a function even though it does not return anything. Changing `waitForTerminatingJobManager` to take a `ThrowingConsumer` should solve the problem.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13217:
URL: https://github.com/apache/flink/pull/13217#discussion_r475791446



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -350,71 +341,42 @@ private boolean isPartialResourceConfigured(JobGraph jobGraph) {
 		}, getRpcService().getExecutor());
 	}
 
-	private CompletableFuture<Void> persistAndRunJob(JobGraph jobGraph) throws Exception {
+	private Void persistAndRunJob(JobGraph jobGraph) throws Exception {

Review comment:
       Ah, very nice. Thanks a lot!




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13217: [FLINK-16866] Make job submission non-blocking

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13217:
URL: https://github.com/apache/flink/pull/13217#issuecomment-678285884


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5776",
       "triggerID" : "85c9bbac2d9ca07435fbd80d76208fa2a51e5d37",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3655fcea1966bfbcb85c86d6a159c354f20d6cc7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5807",
       "triggerID" : "9127b879bec8dc1c05176c1acb89680c0e7baf04",
       "triggerType" : "PUSH"
     }, {
       "hash" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832",
       "triggerID" : "52a49f0b0840aa9220de72d64c50a6b33f6adf92",
       "triggerType" : "PUSH"
     }, {
       "hash" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "805f3142619f652b96804ab3c65beca2ba50f5d4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3655fcea1966bfbcb85c86d6a159c354f20d6cc7 UNKNOWN
   * 52a49f0b0840aa9220de72d64c50a6b33f6adf92 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5832) 
   * 805f3142619f652b96804ab3c65beca2ba50f5d4 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org