-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-38403][tests] Fix the unexpected test that the second job does not restore from checkpoint #27254
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-38403][tests] Fix the unexpected test that the second job does not restore from checkpoint #27254
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
|
|
||
| import org.apache.flink.api.common.JobExecutionResult; | ||
| import org.apache.flink.api.common.JobID; | ||
| import org.apache.flink.api.common.JobStatus; | ||
| import org.apache.flink.api.common.JobSubmissionResult; | ||
| import org.apache.flink.api.common.accumulators.IntCounter; | ||
| import org.apache.flink.api.common.accumulators.LongCounter; | ||
|
|
@@ -57,13 +58,12 @@ | |
| import org.apache.flink.runtime.state.FunctionSnapshotContext; | ||
| import org.apache.flink.runtime.testutils.CommonTestUtils; | ||
| import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; | ||
| import org.apache.flink.runtime.throwable.ThrowableAnnotation; | ||
| import org.apache.flink.runtime.throwable.ThrowableType; | ||
| import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; | ||
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
| import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; | ||
| import org.apache.flink.streaming.api.functions.sink.legacy.RichSinkFunction; | ||
| import org.apache.flink.streaming.api.graph.StreamGraph; | ||
| import org.apache.flink.streaming.util.RestartStrategyUtils; | ||
| import org.apache.flink.test.util.MiniClusterWithClientResource; | ||
| import org.apache.flink.testutils.junit.FailsWithAdaptiveScheduler; | ||
| import org.apache.flink.util.Collector; | ||
|
|
@@ -73,6 +73,7 @@ | |
|
|
||
| import org.apache.flink.shaded.guava33.com.google.common.collect.Iterables; | ||
|
|
||
| import org.assertj.core.api.Fail; | ||
| import org.junit.AfterClass; | ||
| import org.junit.BeforeClass; | ||
| import org.junit.Rule; | ||
|
|
@@ -103,6 +104,7 @@ | |
|
|
||
| import static org.apache.flink.shaded.guava33.com.google.common.collect.Iterables.getOnlyElement; | ||
| import static org.apache.flink.util.Preconditions.checkState; | ||
| import static org.assertj.core.api.Assertions.assertThat; | ||
|
|
||
| /** | ||
| * Base class for tests related to unaligned checkpoints. | ||
|
|
@@ -150,7 +152,7 @@ public static void afterAll() { | |
| } | ||
|
|
||
| @Nullable | ||
| protected File execute(UnalignedSettings settings) throws Exception { | ||
| protected String execute(UnalignedSettings settings) throws Exception { | ||
| final File checkpointDir = temp.newFolder(); | ||
| Configuration conf = settings.getConfiguration(checkpointDir); | ||
|
|
||
|
|
@@ -179,32 +181,42 @@ protected File execute(UnalignedSettings settings) throws Exception { | |
| final StreamExecutionEnvironment env = | ||
| StreamExecutionEnvironment.getExecutionEnvironment(conf); | ||
| settings.configure(env); | ||
| JobID jobID = null; | ||
| try { | ||
| // print the test parameters to help debugging when the case is stuck | ||
| System.out.println( | ||
| "Starting " + getClass().getCanonicalName() + "#" + name.getMethodName() + "."); | ||
| final CompletableFuture<JobSubmissionResult> result = | ||
| miniCluster.getMiniCluster().submitJob(streamGraph.getJobGraph()); | ||
|
|
||
| final JobID jobID = result.get().getJobID(); | ||
| jobID = result.get().getJobID(); | ||
| checkCounters( | ||
| miniCluster | ||
| .getMiniCluster() | ||
| .requestJobResult(jobID) | ||
| .get() | ||
| .toJobExecutionResult(getClass().getClassLoader())); | ||
| if (settings.expectedFinalJobStatus != null) { | ||
| assertThat(miniCluster.getMiniCluster().getJobStatus(jobID)) | ||
| .succeedsWithin(Duration.ofMinutes(1)) | ||
| .isEqualTo(settings.expectedFinalJobStatus); | ||
| } | ||
| System.out.println( | ||
| "Finished " + getClass().getCanonicalName() + "#" + name.getMethodName() + "."); | ||
| if (settings.generateCheckpoint) { | ||
| return CommonTestUtils.getLatestCompletedCheckpointPath( | ||
| jobID, miniCluster.getMiniCluster()) | ||
| .map(File::new) | ||
| .orElseThrow(() -> new AssertionError("Could not generate checkpoint")); | ||
| .orElseGet(() -> Fail.fail("Could not generate checkpoint")); | ||
| } | ||
| } catch (Exception e) { | ||
| if (ExceptionUtils.findThrowable(e, TestException.class).isEmpty()) { | ||
| throw e; | ||
| } | ||
| if (settings.generateCheckpoint) { | ||
| return CommonTestUtils.getLatestCompletedCheckpointPath( | ||
| jobID, miniCluster.getMiniCluster()) | ||
| .orElseGet(() -> Fail.fail("Could not generate checkpoint")); | ||
| } | ||
| } finally { | ||
| miniCluster.after(); | ||
| } | ||
|
|
@@ -680,7 +692,7 @@ public String toString() { | |
| protected static class UnalignedSettings { | ||
| private int parallelism; | ||
| private final int minCheckpoints = 10; | ||
| @Nullable private File restoreCheckpoint; | ||
| @Nullable private String restoreCheckpoint; | ||
| private boolean generateCheckpoint = false; | ||
| int expectedFailures = 0; | ||
| int tolerableCheckpointFailures = 0; | ||
|
|
@@ -691,6 +703,7 @@ protected static class UnalignedSettings { | |
| private int failuresAfterSourceFinishes = 0; | ||
| private ChannelType channelType = ChannelType.MIXED; | ||
| private long sourceSleepMs = 0; | ||
| @Nullable private JobStatus expectedFinalJobStatus = null; | ||
|
|
||
| public UnalignedSettings(DagCreator dagCreator) { | ||
| this.dagCreator = dagCreator; | ||
|
|
@@ -701,7 +714,7 @@ public UnalignedSettings setParallelism(int parallelism) { | |
| return this; | ||
| } | ||
|
|
||
| public UnalignedSettings setRestoreCheckpoint(File restoreCheckpoint) { | ||
| public UnalignedSettings setRestoreCheckpoint(String restoreCheckpoint) { | ||
| this.restoreCheckpoint = restoreCheckpoint; | ||
| return this; | ||
| } | ||
|
|
@@ -746,6 +759,11 @@ public UnalignedSettings setSourceSleepMs(long sourceSleepMs) { | |
| return this; | ||
| } | ||
|
|
||
| public UnalignedSettings setExpectedFinalJobStatus(JobStatus expectedFinalJobStatus) { | ||
| this.expectedFinalJobStatus = expectedFinalJobStatus; | ||
| return this; | ||
| } | ||
|
|
||
| public void configure(StreamExecutionEnvironment env) { | ||
| env.enableCheckpointing(Math.max(100L, parallelism * 50L)); | ||
| env.getCheckpointConfig() | ||
|
|
@@ -754,6 +772,8 @@ public void configure(StreamExecutionEnvironment env) { | |
| env.getCheckpointConfig() | ||
| .setTolerableCheckpointFailureNumber(tolerableCheckpointFailures); | ||
| env.setParallelism(parallelism); | ||
| RestartStrategyUtils.configureFixedDelayRestartStrategy( | ||
| env, generateCheckpoint ? expectedFailures / 2 : expectedFailures, 100L); | ||
|
Comment on lines
+775
to
+776
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What's the reasoning behind the calculated number of expected failures?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It reverts the change in https://github.com/apache/flink/pull/27119/files#diff-ace775e80e66d4f4001bdaea6bcbaae1975bd5e9a5497532d8d7152e4090069aL752 The original intention is :
|
||
| env.getCheckpointConfig().enableUnalignedCheckpoints(true); | ||
| // for custom partitioner | ||
| env.getCheckpointConfig().setForceUnalignedCheckpoints(true); | ||
|
|
@@ -772,7 +792,7 @@ public Configuration getConfiguration(File checkpointDir) { | |
| conf.set(StateBackendOptions.STATE_BACKEND, "hashmap"); | ||
| conf.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); | ||
| if (restoreCheckpoint != null) { | ||
| conf.set(StateRecoveryOptions.SAVEPOINT_PATH, restoreCheckpoint.toURI().toString()); | ||
| conf.set(StateRecoveryOptions.SAVEPOINT_PATH, restoreCheckpoint); | ||
|
Comment on lines
-775
to
+795
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
| } | ||
|
|
||
| conf.set( | ||
|
|
@@ -1132,7 +1152,6 @@ protected static long checkHeader(long value) { | |
| return value; | ||
| } | ||
|
|
||
| @ThrowableAnnotation(ThrowableType.NonRecoverableError) | ||
| static class TestException extends Exception { | ||
| public TestException(String s) { | ||
| super(s); | ||
|
|
||

There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The second job expects failover once, and finishes after source generates all records. So removing
@ThrowableAnnotation(ThrowableType.NonRecoverableError)forTestException.Also, I introduced
ExpectedFinalJobStatusinUnalignedSettingsto check the finalJobStatus.