|
20 | 20 |
|
21 | 21 | import org.apache.flink.api.common.ExecutionConfig; |
22 | 22 | import org.apache.flink.api.common.JobExecutionResult; |
| 23 | +import org.apache.flink.api.common.JobStatus; |
23 | 24 | import org.apache.flink.api.common.restartstrategy.RestartStrategies; |
| 25 | +import org.apache.flink.api.common.time.Deadline; |
24 | 26 | import org.apache.flink.configuration.Configuration; |
25 | 27 | import org.apache.flink.configuration.JobManagerOptions; |
26 | 28 | import org.apache.flink.runtime.execution.Environment; |
|
33 | 35 | import org.apache.flink.runtime.jobmaster.JobResult; |
34 | 36 | import org.apache.flink.runtime.minicluster.MiniCluster; |
35 | 37 | import org.apache.flink.runtime.testtasks.NoOpInvokable; |
| 38 | +import org.apache.flink.runtime.testutils.CommonTestUtils; |
36 | 39 | import org.apache.flink.runtime.testutils.MiniClusterResource; |
37 | 40 | import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; |
38 | 41 | import org.apache.flink.util.FlinkRuntimeException; |
|
43 | 46 |
|
44 | 47 | import java.io.IOException; |
45 | 48 | import java.time.Duration; |
| 49 | +import java.time.temporal.ChronoUnit; |
46 | 50 |
|
47 | 51 | import static org.junit.Assert.assertTrue; |
48 | 52 |
|
@@ -104,6 +108,34 @@ private JobGraph createJobGraph() { |
104 | 108 | return JobGraphTestUtils.streamingJobGraph(source, sink); |
105 | 109 | } |
106 | 110 |
|
| 111 | + @Test |
| 112 | + public void testJobCancellationWhileRestartingSucceeds() throws Exception { |
| 113 | + final long timeInRestartingState = 10000L; |
| 114 | + |
| 115 | + final MiniCluster miniCluster = MINI_CLUSTER_RESOURCE.getMiniCluster(); |
| 116 | + final JobVertex alwaysFailingOperator = new JobVertex("Always failing operator"); |
| 117 | + alwaysFailingOperator.setInvokableClass(AlwaysFailingInvokable.class); |
| 118 | + alwaysFailingOperator.setParallelism(1); |
| 119 | + |
| 120 | + final JobGraph jobGraph = JobGraphTestUtils.streamingJobGraph(alwaysFailingOperator); |
| 121 | + ExecutionConfig executionConfig = new ExecutionConfig(); |
| 122 | + // configure a high delay between attempts: We'll stay in RESTARTING for 10 seconds. |
| 123 | + executionConfig.setRestartStrategy( |
| 124 | + RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, timeInRestartingState)); |
| 125 | + jobGraph.setExecutionConfig(executionConfig); |
| 126 | + |
| 127 | + miniCluster.submitJob(jobGraph).join(); |
| 128 | + |
| 129 | + // wait until we are in RESTARTING state |
| 130 | + CommonTestUtils.waitUntilCondition( |
| 131 | + () -> miniCluster.getJobStatus(jobGraph.getJobID()).get() == JobStatus.RESTARTING, |
| 132 | + Deadline.fromNow(Duration.of(timeInRestartingState, ChronoUnit.MILLIS)), |
| 133 | + 5); |
| 134 | + |
| 135 | + // now cancel while in RESTARTING state |
| 136 | + miniCluster.cancelJob(jobGraph.getJobID()).get(); |
| 137 | + } |
| 138 | + |
107 | 139 | @Test |
108 | 140 | public void testGlobalFailoverIfTaskFails() throws Throwable { |
109 | 141 | final MiniCluster miniCluster = MINI_CLUSTER_RESOURCE.getMiniCluster(); |
@@ -160,4 +192,15 @@ private static void reset() { |
160 | 192 | hasFailed = false; |
161 | 193 | } |
162 | 194 | } |
| 195 | + |
| 196 | + public static final class AlwaysFailingInvokable extends AbstractInvokable { |
| 197 | + public AlwaysFailingInvokable(Environment environment) { |
| 198 | + super(environment); |
| 199 | + } |
| 200 | + |
| 201 | + @Override |
| 202 | + public void invoke() throws Exception { |
| 203 | + throw new FlinkRuntimeException("Test failure."); |
| 204 | + } |
| 205 | + } |
163 | 206 | } |
0 commit comments