This is an automated email from the ASF dual-hosted git repository.
trohrmann pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new c559685 [FLINK-20677] Introduce proper JobManagerRunnerResult
c559685 is described below
commit c5596859317316dc800ccc7198cae934428d3415
Author: Till Rohrmann <[email protected]>
AuthorDate: Thu Dec 17 18:49:09 2020 +0100
[FLINK-20677] Introduce proper JobManagerRunnerResult
The JobManagerRunnerResult allows to express different termination
conditions for the JobManagerRunner:
1) Job finished successfully
2) Job did not finish
3) Job initialization failed
This closes #14430.
---
.../flink/runtime/dispatcher/DispatcherJob.java | 38 +++++--
.../flink/runtime/jobmaster/JobManagerRunner.java | 3 +-
.../runtime/jobmaster/JobManagerRunnerImpl.java | 10 +-
.../runtime/jobmaster/JobManagerRunnerResult.java | 109 ++++++++++++++++++++
.../runtime/dispatcher/DispatcherJobTest.java | 9 +-
.../jobmaster/JobManagerRunnerImplTest.java | 33 +++---
.../jobmaster/JobManagerRunnerResultTest.java | 113 +++++++++++++++++++++
.../runtime/jobmaster/TestingJobManagerRunner.java | 12 +--
8 files changed, 280 insertions(+), 47 deletions(-)
diff --git
a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
index f0e994d..f957bd3 100644
---
a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java
@@ -24,7 +24,9 @@ import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
import org.apache.flink.runtime.jobmaster.JobManagerRunner;
+import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult;
import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.jobmaster.JobNotFinishedException;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.messages.webmonitor.JobDetails;
import org.apache.flink.util.AutoCloseableAsync;
@@ -110,28 +112,42 @@ public final class DispatcherJob implements
AutoCloseableAsync {
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));
+
jobManagerRunner.getResultFuture().whenComplete((jobManagerRunnerResult,
resultThrowable) -> {
+ if (jobManagerRunnerResult !=
null) {
+
handleJobManagerRunnerResult(jobManagerRunnerResult);
} 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));
+
handleInitializationFailure(ExceptionUtils.stripCompletionException(throwable));
}
}
return null;
}));
}
+ private void handleJobManagerRunnerResult(JobManagerRunnerResult
jobManagerRunnerResult) {
+ if (jobManagerRunnerResult.isSuccess()) {
+
jobResultFuture.complete(DispatcherJobResult.forSuccess(jobManagerRunnerResult.getArchivedExecutionGraph()));
+ } else if (jobManagerRunnerResult.isJobNotFinished()) {
+ jobResultFuture.completeExceptionally(new
JobNotFinishedException(jobId));
+ } else if (jobManagerRunnerResult.isInitializationFailure()) {
+
handleInitializationFailure(jobManagerRunnerResult.getInitializationFailure());
+ }
+ }
+
+ private void handleInitializationFailure(Throwable
initializationFailure) {
+ ArchivedExecutionGraph archivedExecutionGraph =
ArchivedExecutionGraph.createFromInitializingJob(
+ jobId,
+ jobName,
+ JobStatus.FAILED,
+ initializationFailure,
+ initializationTimestamp);
+
jobResultFuture.complete(DispatcherJobResult.forInitializationFailure(archivedExecutionGraph,
+ initializationFailure));
+ }
+
public CompletableFuture<DispatcherJobResult> getResultFuture() {
return jobResultFuture;
}
diff --git
a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
index 4a1d1c7..c59a40d 100644
---
a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
@@ -19,7 +19,6 @@
package org.apache.flink.runtime.jobmaster;
import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
import org.apache.flink.util.AutoCloseableAsync;
import java.util.concurrent.CompletableFuture;
@@ -50,7 +49,7 @@ public interface JobManagerRunner extends AutoCloseableAsync {
*
* @return Future which is completed with the job result
*/
- CompletableFuture<ArchivedExecutionGraph> getResultFuture();
+ CompletableFuture<JobManagerRunnerResult> getResultFuture();
/**
* Get the job id of the executed job.
diff --git
a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java
index 34f1b26..2acf773 100644
---
a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java
@@ -81,7 +81,7 @@ public class JobManagerRunnerImpl implements LeaderContender,
OnCompletionAction
private final FatalErrorHandler fatalErrorHandler;
- private final CompletableFuture<ArchivedExecutionGraph> resultFuture;
+ private final CompletableFuture<JobManagerRunnerResult> resultFuture;
private final CompletableFuture<Void> terminationFuture;
@@ -151,7 +151,7 @@ public class JobManagerRunnerImpl implements
LeaderContender, OnCompletionAction
}
@Override
- public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+ public CompletableFuture<JobManagerRunnerResult> getResultFuture() {
return resultFuture;
}
@@ -195,7 +195,7 @@ public class JobManagerRunnerImpl implements
LeaderContender, OnCompletionAction
classLoaderLease.release();
-
resultFuture.completeExceptionally(new
JobNotFinishedException(jobGraph.getJobID()));
+
resultFuture.complete(JobManagerRunnerResult.forJobNotFinished());
if (throwable != null) {
terminationFuture.completeExceptionally(
@@ -221,7 +221,7 @@ public class JobManagerRunnerImpl implements
LeaderContender, OnCompletionAction
public void jobReachedGloballyTerminalState(ArchivedExecutionGraph
executionGraph) {
unregisterJobFromHighAvailability();
// complete the result future with the terminal execution graph
- resultFuture.complete(executionGraph);
+
resultFuture.complete(JobManagerRunnerResult.forSuccess(executionGraph));
}
/**
@@ -229,7 +229,7 @@ public class JobManagerRunnerImpl implements
LeaderContender, OnCompletionAction
*/
@Override
public void jobFinishedByOther() {
- resultFuture.completeExceptionally(new
JobNotFinishedException(jobGraph.getJobID()));
+
resultFuture.complete(JobManagerRunnerResult.forJobNotFinished());
}
@Override
diff --git
a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResult.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResult.java
new file mode 100644
index 0000000..94b5041
--- /dev/null
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResult.java
@@ -0,0 +1,109 @@
+/*
+ * 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.jobmaster;
+
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+/**
+ * The result of the {@link JobManagerRunner}.
+ */
+public final class JobManagerRunnerResult {
+
+ @Nullable
+ private final ArchivedExecutionGraph archivedExecutionGraph;
+
+ @Nullable
+ private final Throwable failure;
+
+ private JobManagerRunnerResult(
+ @Nullable ArchivedExecutionGraph archivedExecutionGraph,
+ @Nullable Throwable failure) {
+ this.archivedExecutionGraph = archivedExecutionGraph;
+ this.failure = failure;
+ }
+
+ public boolean isSuccess() {
+ return archivedExecutionGraph != null && failure == null;
+ }
+
+ public boolean isJobNotFinished() {
+ return archivedExecutionGraph == null && failure == null;
+ }
+
+ public boolean isInitializationFailure() {
+ return archivedExecutionGraph == null && failure != null;
+ }
+
+ /**
+ * This method returns the payload of the successful
JobManagerRunnerResult.
+ *
+ * @return the successful completed {@link ArchivedExecutionGraph}
+ * @throws IllegalStateException if the result is not a success
+ */
+ public ArchivedExecutionGraph getArchivedExecutionGraph() {
+ Preconditions.checkState(isSuccess());
+ return archivedExecutionGraph;
+ }
+
+ /**
+ * This method returns the initialization failure.
+ *
+ * @return the initialization failure
+ * @throws IllegalStateException if the result is not an initialization
failure
+ */
+ public Throwable getInitializationFailure() {
+ Preconditions.checkState(isInitializationFailure());
+ return failure;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ JobManagerRunnerResult that = (JobManagerRunnerResult) o;
+ return Objects.equals(archivedExecutionGraph,
that.archivedExecutionGraph)
+ && Objects.equals(failure, that.failure);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(archivedExecutionGraph, failure);
+ }
+
+ public static JobManagerRunnerResult forJobNotFinished() {
+ return new JobManagerRunnerResult(null, null);
+ }
+
+ public static JobManagerRunnerResult forSuccess(ArchivedExecutionGraph
archivedExecutionGraph) {
+ return new JobManagerRunnerResult(archivedExecutionGraph, null);
+ }
+
+ public static JobManagerRunnerResult forInitializationFailure(Throwable
failure) {
+ return new JobManagerRunnerResult(null, failure);
+ }
+}
diff --git
a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
index 0abe5e8..295422d 100644
---
a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
+++
b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java
@@ -26,6 +26,7 @@ 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.JobManagerRunnerResult;
import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner;
import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway;
import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder;
@@ -264,7 +265,7 @@ public class DispatcherJobTest extends TestLogger {
private final CompletableFuture<Acknowledge> cancellationFuture;
private JobStatus internalJobStatus = JobStatus.INITIALIZING;
- private CompletableFuture<ArchivedExecutionGraph> resultFuture
= new CompletableFuture<>();
+ private CompletableFuture<JobManagerRunnerResult> resultFuture
= new CompletableFuture<>();
public TestContext (
CompletableFuture<JobManagerRunner>
jobManagerRunnerCompletableFuture,
@@ -316,14 +317,16 @@ public class DispatcherJobTest extends TestLogger {
public void finishJob() {
internalJobStatus = JobStatus.FINISHED;
resultFuture.complete(
-
ArchivedExecutionGraph.createFromInitializingJob(getJobID(), "test",
JobStatus.FINISHED, null, 1337));
+ JobManagerRunnerResult.forSuccess(
+
ArchivedExecutionGraph.createFromInitializingJob(getJobID(), "test",
JobStatus.FINISHED, null, 1337)));
}
public void finishCancellation() {
jobManagerRunnerCompletableFuture.thenAccept(runner -> {
internalJobStatus = JobStatus.CANCELED;
runner.getResultFuture()
-
.complete(ArchivedExecutionGraph.createFromInitializingJob(getJobID(), "test",
JobStatus.CANCELED, null, 1337));
+ .complete(
+
JobManagerRunnerResult.forSuccess(ArchivedExecutionGraph.createFromInitializingJob(getJobID(),
"test", JobStatus.CANCELED, null, 1337)));
cancellationFuture.complete(Acknowledge.get());
});
}
diff --git
a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java
index a99ff9e..65f8fa5 100644
---
a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java
+++
b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java
@@ -36,9 +36,8 @@ import
org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraph
import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.runtime.testtasks.NoOpInvokable;
import org.apache.flink.runtime.util.TestingFatalErrorHandler;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.TestLogger;
import org.apache.flink.runtime.util.TestingUserCodeClassLoader;
+import org.apache.flink.util.TestLogger;
import org.junit.After;
import org.junit.Before;
@@ -51,13 +50,12 @@ import javax.annotation.Nonnull;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/**
@@ -117,13 +115,14 @@ public class JobManagerRunnerImplTest extends TestLogger {
try {
jobManagerRunner.start();
- final CompletableFuture<ArchivedExecutionGraph>
resultFuture = jobManagerRunner.getResultFuture();
+ final CompletableFuture<JobManagerRunnerResult>
resultFuture = jobManagerRunner.getResultFuture();
assertThat(resultFuture.isDone(), is(false));
jobManagerRunner.jobReachedGloballyTerminalState(archivedExecutionGraph);
- assertThat(resultFuture.get(),
is(archivedExecutionGraph));
+ final JobManagerRunnerResult jobManagerRunnerResult =
resultFuture.get();
+ assertThat(jobManagerRunnerResult,
is(JobManagerRunnerResult.forSuccess(archivedExecutionGraph)));
} finally {
jobManagerRunner.close();
}
@@ -136,18 +135,15 @@ public class JobManagerRunnerImplTest extends TestLogger {
try {
jobManagerRunner.start();
- final CompletableFuture<ArchivedExecutionGraph>
resultFuture = jobManagerRunner.getResultFuture();
+ final CompletableFuture<JobManagerRunnerResult>
resultFuture = jobManagerRunner.getResultFuture();
assertThat(resultFuture.isDone(), is(false));
jobManagerRunner.jobFinishedByOther();
- try {
- resultFuture.get();
- fail("Should have failed.");
- } catch (ExecutionException ee) {
-
assertThat(ExceptionUtils.stripExecutionException(ee),
instanceOf(JobNotFinishedException.class));
- }
+ final JobManagerRunnerResult jobManagerRunnerResult =
resultFuture.get();
+
+ assertTrue(jobManagerRunnerResult.isJobNotFinished());
} finally {
jobManagerRunner.close();
}
@@ -160,18 +156,15 @@ public class JobManagerRunnerImplTest extends TestLogger {
try {
jobManagerRunner.start();
- final CompletableFuture<ArchivedExecutionGraph>
resultFuture = jobManagerRunner.getResultFuture();
+ final CompletableFuture<JobManagerRunnerResult>
resultFuture = jobManagerRunner.getResultFuture();
assertThat(resultFuture.isDone(), is(false));
jobManagerRunner.closeAsync();
- try {
- resultFuture.get();
- fail("Should have failed.");
- } catch (ExecutionException ee) {
-
assertThat(ExceptionUtils.stripExecutionException(ee),
instanceOf(JobNotFinishedException.class));
- }
+ final JobManagerRunnerResult jobManagerRunnerResult =
resultFuture.join();
+
+ assertTrue(jobManagerRunnerResult.isJobNotFinished());
} finally {
jobManagerRunner.close();
}
diff --git
a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResultTest.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResultTest.java
new file mode 100644
index 0000000..cfa5802
--- /dev/null
+++
b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResultTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.jobmaster;
+
+import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
+import
org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link JobManagerRunnerResult}.
+ */
+public class JobManagerRunnerResultTest extends TestLogger {
+
+ private final ArchivedExecutionGraph archivedExecutionGraph = new
ArchivedExecutionGraphBuilder().build();
+ private final FlinkException testException = new FlinkException("test
exception");
+
+ @Test
+ public void testSuccessfulJobManagerResult() {
+ final JobManagerRunnerResult jobManagerRunnerResult =
JobManagerRunnerResult.forSuccess(
+ archivedExecutionGraph);
+
+ assertTrue(jobManagerRunnerResult.isSuccess());
+ assertFalse(jobManagerRunnerResult.isJobNotFinished());
+ assertFalse(jobManagerRunnerResult.isInitializationFailure());
+ }
+
+ @Test
+ public void testJobNotFinishedJobManagerResult() {
+ final JobManagerRunnerResult jobManagerRunnerResult =
JobManagerRunnerResult.forJobNotFinished();
+
+ assertTrue(jobManagerRunnerResult.isJobNotFinished());
+ assertFalse(jobManagerRunnerResult.isSuccess());
+ assertFalse(jobManagerRunnerResult.isInitializationFailure());
+ }
+
+ @Test
+ public void testInitializationFailureJobManagerResult() {
+ final JobManagerRunnerResult jobManagerRunnerResult =
JobManagerRunnerResult.forInitializationFailure(testException);
+
+ assertTrue(jobManagerRunnerResult.isInitializationFailure());
+ assertFalse(jobManagerRunnerResult.isSuccess());
+ assertFalse(jobManagerRunnerResult.isJobNotFinished());
+ }
+
+ @Test
+ public void
testGetArchivedExecutionGraphFromSuccessfulJobManagerResult() {
+ final JobManagerRunnerResult jobManagerRunnerResult =
JobManagerRunnerResult.forSuccess(
+ archivedExecutionGraph);
+
+ assertThat(jobManagerRunnerResult.getArchivedExecutionGraph(),
is(archivedExecutionGraph));
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testGetArchivedExecutionGraphFromJobNotFinishedFails() {
+ final JobManagerRunnerResult jobManagerRunnerResult =
JobManagerRunnerResult.forJobNotFinished();
+
+ jobManagerRunnerResult.getArchivedExecutionGraph();
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void
testGetArchivedExecutionGraphFromInitializationFailureFails() {
+ final JobManagerRunnerResult jobManagerRunnerResult =
JobManagerRunnerResult.forInitializationFailure(
+ testException);
+
+ jobManagerRunnerResult.getArchivedExecutionGraph();
+ }
+
+ @Test
+ public void testGetInitializationFailureFromFailedJobManagerResult() {
+ final JobManagerRunnerResult jobManagerRunnerResult =
JobManagerRunnerResult.forInitializationFailure(
+ testException);
+
+ assertThat(jobManagerRunnerResult.getInitializationFailure(),
is(testException));
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void testGetInitializationFailureFromJobNotFinished() {
+ final JobManagerRunnerResult jobManagerRunnerResult =
JobManagerRunnerResult.forJobNotFinished();
+
+ jobManagerRunnerResult.getInitializationFailure();
+ }
+
+ @Test(expected = IllegalStateException.class)
+ public void
testGetInitializationFailureFromSuccessfulJobManagerResult() {
+ final JobManagerRunnerResult jobManagerRunnerResult =
JobManagerRunnerResult.forSuccess(archivedExecutionGraph);
+
+ jobManagerRunnerResult.getInitializationFailure();
+ }
+}
diff --git
a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java
index 5e3edfd..d1ddb9e 100644
---
a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java
+++
b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java
@@ -38,14 +38,14 @@ public class TestingJobManagerRunner implements
JobManagerRunner {
private final CompletableFuture<JobMasterGateway>
jobMasterGatewayFuture;
- private final CompletableFuture<ArchivedExecutionGraph> resultFuture;
+ private final CompletableFuture<JobManagerRunnerResult> resultFuture;
private final OneShotLatch closeAsyncCalledLatch = new OneShotLatch();
private TestingJobManagerRunner(JobID jobId,
boolean blockingTermination,
CompletableFuture<JobMasterGateway>
jobMasterGatewayFuture,
- CompletableFuture<ArchivedExecutionGraph> resultFuture)
{
+ CompletableFuture<JobManagerRunnerResult> resultFuture)
{
this.jobId = jobId;
this.blockingTermination = blockingTermination;
this.jobMasterGatewayFuture = jobMasterGatewayFuture;
@@ -64,7 +64,7 @@ public class TestingJobManagerRunner implements
JobManagerRunner {
}
@Override
- public CompletableFuture<ArchivedExecutionGraph> getResultFuture() {
+ public CompletableFuture<JobManagerRunnerResult> getResultFuture() {
return resultFuture;
}
@@ -88,7 +88,7 @@ public class TestingJobManagerRunner implements
JobManagerRunner {
}
public void completeResultFuture(ArchivedExecutionGraph
archivedExecutionGraph) {
- resultFuture.complete(archivedExecutionGraph);
+
resultFuture.complete(JobManagerRunnerResult.forSuccess(archivedExecutionGraph));
}
public void completeResultFutureExceptionally(Exception e) {
@@ -111,7 +111,7 @@ public class TestingJobManagerRunner implements
JobManagerRunner {
private JobID jobId = null;
private boolean blockingTermination = false;
private CompletableFuture<JobMasterGateway>
jobMasterGatewayFuture = new CompletableFuture<>();
- private CompletableFuture<ArchivedExecutionGraph> resultFuture
= new CompletableFuture<>();
+ private CompletableFuture<JobManagerRunnerResult> resultFuture
= new CompletableFuture<>();
public Builder setJobId(JobID jobId) {
this.jobId = jobId;
@@ -129,7 +129,7 @@ public class TestingJobManagerRunner implements
JobManagerRunner {
return this;
}
- public Builder
setResultFuture(CompletableFuture<ArchivedExecutionGraph> resultFuture) {
+ public Builder
setResultFuture(CompletableFuture<JobManagerRunnerResult> resultFuture) {
Preconditions.checkNotNull(resultFuture);
this.resultFuture = resultFuture;
return this;