[ 
https://issues.apache.org/jira/browse/FLINK-20992?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17267436#comment-17267436
 ] 

Till Rohrmann commented on FLINK-20992:
---------------------------------------

Thanks for creating this PR [~roman_khachatryan].

I agree that there is a lifecycle problem of the {{CheckpointCoordinator}} and 
its services. However, I am not sure whether we should harden the exception 
handling logic or rather make the {{CheckpointCoordinator}} no longer enqueuing 
something into the {{Executor}} after it is shut down. To me it makes more 
sense that the {{CheckpointCoordinator}} which knows about its state is 
responsible for making sure that we only execute a {{Runnable}} if we are still 
running.

I think you are also right that we currently don't wait for the checkpoint 
cleanup to complete. I think the component responsible for the clean up tasks 
should make sure that they are completed before shutting down or hand them over 
to a new owner who is responsible for them. Hence, if the 
{{CheckpointCoordinator}} is responsible, then the 
{{CheckpointCoordinator.shutdown}} method should make sure that all checkpoints 
are cleaned up. Alternatively, {{CheckpointCoordinator.shutdown}} could return 
a {{CompletableFuture}} which is completed once everything is cleaned up. 
Consequently, I wouldn't make it the responsibility of the exeuctors to make 
sure that all checkpoints are properly cleaned up by waiting on the completion 
of all enqueued tasks.

Both things should be fixed imo.

> Checkpoint cleanup can kill JobMaster
> -------------------------------------
>
>                 Key: FLINK-20992
>                 URL: https://issues.apache.org/jira/browse/FLINK-20992
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Checkpointing
>    Affects Versions: 1.12.0
>            Reporter: Till Rohrmann
>            Assignee: Roman Khachatryan
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 1.13.0, 1.12.2
>
>
> A user reported that cancelling a job can lead to an uncaught exception which 
> kills the {{JobMaster}}. The problem seems to be that the 
> {{CheckpointsCleaner}} might trigger {{CheckpointCoordinator}} actions after 
> the job has reached a terminal state and, thus, is shut down. Apparently, we 
> do not properly manage the lifecycles of {{CheckpointCoordinator}} and 
> checkpoint post clean up actions.
> The uncaught exception is 
> {code}
> java.util.concurrent.RejectedExecutionException: Task 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@41554407 
> rejected from 
> java.util.concurrent.ScheduledThreadPoolExecutor@5d0ec6f7[Terminated, pool 
> size = 0, active threads = 0, queued tasks = 0, completed tasks = 25977] at 
> java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2063
>  at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:326
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:533
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor.execute(ScheduledThreadPoolExecutor.java:622
>  at 
> java.util.concurrent.Executors$DelegatedExecutorService.execute(Executors.java:668
>  at 
> org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter.execute(ScheduledExecutorServiceAdapter.java:62
>  at 
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.scheduleTriggerRequest(CheckpointCoordinator.java:1152
>  at 
> org.apache.flink.runtime.checkpoint.CheckpointsCleaner.lambda$cleanCheckpoint$0(CheckpointsCleaner.java:58
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624
>  at java.lang.Thread.run(Thread.java:748 undefined)
> {code}
> cc [~roman_khachatryan].
> https://lists.apache.org/thread.html/r75901008d88163560aabb8ab6fc458cd9d4f19076e03ae85e00f9a3a%40%3Cuser.flink.apache.org%3E



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to