|
18 | 18 |
|
19 | 19 | package org.apache.flink.runtime.dispatcher.runner; |
20 | 20 |
|
| 21 | +import org.apache.flink.api.common.ApplicationID; |
21 | 22 | import org.apache.flink.api.common.JobID; |
22 | 23 | import org.apache.flink.core.testutils.FlinkAssertions; |
23 | 24 | import org.apache.flink.core.testutils.OneShotLatch; |
| 25 | +import org.apache.flink.runtime.application.SingleJobApplication; |
24 | 26 | import org.apache.flink.runtime.blob.BlobServer; |
25 | 27 | import org.apache.flink.runtime.client.DuplicateJobSubmissionException; |
26 | 28 | import org.apache.flink.runtime.client.JobSubmissionException; |
|
32 | 34 | import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; |
33 | 35 | import org.apache.flink.runtime.jobmaster.JobResult; |
34 | 36 | import org.apache.flink.runtime.messages.Acknowledge; |
| 37 | +import org.apache.flink.runtime.messages.FlinkApplicationNotFoundException; |
| 38 | +import org.apache.flink.runtime.testutils.TestingApplicationResultStore; |
| 39 | +import org.apache.flink.runtime.testutils.TestingApplicationStore; |
35 | 40 | import org.apache.flink.runtime.testutils.TestingExecutionPlanStore; |
36 | 41 | import org.apache.flink.runtime.testutils.TestingJobResultStore; |
37 | 42 | import org.apache.flink.runtime.util.TestingFatalErrorHandler; |
@@ -96,13 +101,16 @@ class SessionDispatcherLeaderProcessTest { |
96 | 101 | @BeforeAll |
97 | 102 | static void setupClass() { |
98 | 103 | ioExecutor = Executors.newSingleThreadExecutor(); |
| 104 | + JOB_GRAPH.setApplicationId(ApplicationID.fromHexString(JOB_GRAPH.getJobID().toHexString())); |
99 | 105 | } |
100 | 106 |
|
101 | 107 | @BeforeEach |
102 | 108 | void setup() { |
103 | 109 | fatalErrorHandler = new TestingFatalErrorHandler(); |
104 | 110 | executionPlanStore = TestingExecutionPlanStore.newBuilder().build(); |
105 | 111 | jobResultStore = TestingJobResultStore.builder().build(); |
| 112 | + applicationStore = TestingApplicationStore.newBuilder().build(); |
| 113 | + applicationResultStore = TestingApplicationResultStore.builder().build(); |
106 | 114 | dispatcherServiceFactory = |
107 | 115 | createFactoryBasedOnGenericSupplier( |
108 | 116 | () -> TestingDispatcherGatewayService.newBuilder().build()); |
@@ -183,6 +191,7 @@ void testRecoveryWithMultipleExecutionPlansAndOneMatchingDirtyJobResult() throws |
183 | 191 | final JobResult matchingDirtyJobResult = |
184 | 192 | TestingJobResultStore.createSuccessfulJobResult(JOB_GRAPH.getJobID()); |
185 | 193 | final ExecutionPlan otherExecutionPlan = JobGraphTestUtils.emptyJobGraph(); |
| 194 | + otherExecutionPlan.setApplicationId(JOB_GRAPH.getApplicationId().get()); |
186 | 195 |
|
187 | 196 | testJobRecovery( |
188 | 197 | Arrays.asList(otherExecutionPlan, JOB_GRAPH), |
@@ -566,11 +575,27 @@ void onAddedExecutionPlan_submitsRecoveredJob() throws Exception { |
566 | 575 | final CompletableFuture<ExecutionPlan> submittedJobFuture = new CompletableFuture<>(); |
567 | 576 | final TestingDispatcherGateway testingDispatcherGateway = |
568 | 577 | TestingDispatcherGateway.newBuilder() |
569 | | - .setSubmitFunction( |
570 | | - submittedJob -> { |
571 | | - submittedJobFuture.complete(submittedJob); |
572 | | - return CompletableFuture.completedFuture(Acknowledge.get()); |
| 578 | + .setRequestApplicationFunction( |
| 579 | + applicationId -> |
| 580 | + FutureUtils.completedExceptionally( |
| 581 | + new FlinkApplicationNotFoundException( |
| 582 | + applicationId))) |
| 583 | + .setSubmitApplicationFunction( |
| 584 | + application -> { |
| 585 | + if (application instanceof SingleJobApplication) { |
| 586 | + ExecutionPlan submittedJob = |
| 587 | + ((SingleJobApplication) application) |
| 588 | + .getExecutionPlan(); |
| 589 | + submittedJobFuture.complete(submittedJob); |
| 590 | + return CompletableFuture.completedFuture(Acknowledge.get()); |
| 591 | + } |
| 592 | + return FutureUtils.completedExceptionally( |
| 593 | + new UnsupportedOperationException()); |
573 | 594 | }) |
| 595 | + .setSubmitFunction( |
| 596 | + ignored -> |
| 597 | + FutureUtils.completedExceptionally( |
| 598 | + new UnsupportedOperationException())) |
574 | 599 | .build(); |
575 | 600 |
|
576 | 601 | dispatcherServiceFactory = |
@@ -715,11 +740,20 @@ private void runJobRecoveryFailureTest(FlinkException testException) throws Exce |
715 | 740 | void onAddedExecutionPlan_failingRecoveredJobSubmission_failsFatally() throws Exception { |
716 | 741 | final TestingDispatcherGateway dispatcherGateway = |
717 | 742 | TestingDispatcherGateway.newBuilder() |
718 | | - .setSubmitFunction( |
719 | | - jobGraph -> |
| 743 | + .setRequestApplicationFunction( |
| 744 | + applicationId -> |
| 745 | + FutureUtils.completedExceptionally( |
| 746 | + new FlinkApplicationNotFoundException( |
| 747 | + applicationId))) |
| 748 | + .setSubmitApplicationFunction( |
| 749 | + application -> |
720 | 750 | FutureUtils.completedExceptionally( |
721 | 751 | new JobSubmissionException( |
722 | | - jobGraph.getJobID(), "test exception"))) |
| 752 | + JOB_GRAPH.getJobID(), "test exception"))) |
| 753 | + .setSubmitFunction( |
| 754 | + ignored -> |
| 755 | + FutureUtils.completedExceptionally( |
| 756 | + new UnsupportedOperationException())) |
723 | 757 | .build(); |
724 | 758 |
|
725 | 759 | runOnAddedExecutionPlanTest( |
|
0 commit comments