[
https://issues.apache.org/jira/browse/FLINK-27802?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17542767#comment-17542767
]
Yang Wang commented on FLINK-27802:
-----------------------------------
I believe you could find the root cause in the JobManager logs of first
attempt. This need to be fixed in the upstream Flink because we expect the
JobManager not exit when {{execution.shutdown-on-application-finish}} is false.
kubectl logs flink-example-statemachine-6d596bfdcd-jtmn9 --previous | less
{code:java}
2022-05-26 11:52:18,085 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - No checkpoint
found during restore.
2022-05-26 11:52:18,087 INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Starting job
00000000000000000000000000000000 from savepoint file:///non/exist/sp ()
2022-05-26 11:52:18,174 INFO
org.apache.flink.runtime.dispatcher.StandaloneDispatcher [] - Job
00000000000000000000000000000000 reached terminal state FAILED.
org.apache.flink.runtime.client.JobInitializationException: Could not start the
JobMaster.
at
org.apache.flink.runtime.jobmaster.DefaultJobMasterServiceProcess.lambda$new$0(DefaultJobMasterServiceProcess.java:97)
at
java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(Unknown Source)
at
java.base/java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(Unknown
Source)
at
java.base/java.util.concurrent.CompletableFuture.postComplete(Unknown Source)
at
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(Unknown Source)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown
Source)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown
Source)
at java.base/java.lang.Thread.run(Unknown Source)
Caused by: java.util.concurrent.CompletionException:
java.lang.RuntimeException: java.io.FileNotFoundException: Cannot find
checkpoint or savepoint file/directory 'file:///non/exist/sp' on file system
'file'.
at
java.base/java.util.concurrent.CompletableFuture.encodeThrowable(Unknown Source)
at
java.base/java.util.concurrent.CompletableFuture.completeThrowable(Unknown
Source)
... 4 more
Caused by: java.lang.RuntimeException: java.io.FileNotFoundException: Cannot
find checkpoint or savepoint file/directory 'file:///non/exist/sp' on file
system 'file'.
at org.apache.flink.util.ExceptionUtils.rethrow(ExceptionUtils.java:319)
at
org.apache.flink.util.function.FunctionUtils.lambda$uncheckedSupplier$4(FunctionUtils.java:114)
... 4 more
Caused by: java.io.FileNotFoundException: Cannot find checkpoint or savepoint
file/directory 'file:///non/exist/sp' on file system 'file'.
at
org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess.resolveCheckpointPointer(AbstractFsCheckpointStorageAccess.java:275)
at
org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess.resolveCheckpoint(AbstractFsCheckpointStorageAccess.java:136)
at
org.apache.flink.runtime.checkpoint.CheckpointCoordinator.restoreSavepoint(CheckpointCoordinator.java:1746)
at
org.apache.flink.runtime.scheduler.DefaultExecutionGraphFactory.tryRestoreExecutionGraphFromSavepoint(DefaultExecutionGraphFactory.java:206)
at
org.apache.flink.runtime.scheduler.DefaultExecutionGraphFactory.createAndRestoreExecutionGraph(DefaultExecutionGraphFactory.java:181)
at
org.apache.flink.runtime.scheduler.SchedulerBase.createAndRestoreExecutionGraph(SchedulerBase.java:363)
at
org.apache.flink.runtime.scheduler.SchedulerBase.<init>(SchedulerBase.java:208)
at
org.apache.flink.runtime.scheduler.DefaultScheduler.<init>(DefaultScheduler.java:191)
at
org.apache.flink.runtime.scheduler.DefaultScheduler.<init>(DefaultScheduler.java:139)
at
org.apache.flink.runtime.scheduler.DefaultSchedulerFactory.createInstance(DefaultSchedulerFactory.java:135)
at
org.apache.flink.runtime.jobmaster.DefaultSlotPoolServiceSchedulerFactory.createScheduler(DefaultSlotPoolServiceSchedulerFactory.java:115)
at
org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:345)
at
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:322)
at
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.internalCreateJobMasterService(DefaultJobMasterServiceFactory.java:106)
at
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.lambda$createJobMasterService$0(DefaultJobMasterServiceFactory.java:94)
at
org.apache.flink.util.function.FunctionUtils.lambda$uncheckedSupplier$4(FunctionUtils.java:112)
... 4 more {code}
> Savepoint restore errors are swallowed for Flink 1.15
> -----------------------------------------------------
>
> Key: FLINK-27802
> URL: https://issues.apache.org/jira/browse/FLINK-27802
> Project: Flink
> Issue Type: Improvement
> Components: Kubernetes Operator
> Affects Versions: kubernetes-operator-1.0.0
> Reporter: Gyula Fora
> Priority: Critical
>
> When a job is submitted with an incorrect savepoint path the error is
> swallowed by Flink due to the result store:
> 2022-05-26 12:34:43,497 WARN
> org.apache.flink.runtime.dispatcher.StandaloneDispatcher [] - Ignoring
> JobGraph submission 'State machine job' (00000000000000000000000000000000)
> because the job already reached a globally-terminal state (i.e. FAILED,
> CANCELED, FINISHED) in a previous execution.
> 2022-05-26 12:34:43,552 INFO
> org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap
> [] - Application completed SUCCESSFULLY
> The easiest way to reproduce this is to create a new deployment and set
> initialSavepointPath to a random missing path.
--
This message was sent by Atlassian Jira
(v8.20.7#820007)