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

Roman Khachatryan commented on FLINK-20992:
-------------------------------------------

{quote}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.
{quote}
I think the current issue can be solved this way by adding a check *and 
synchronized* in scheduleTriggerRequest;
 But for cases like FLINK-18290 it's not possible as scheduling a callback is 
done by j.u.c. (e.g. handleAsync).

 

I've created FLINK-21015 to implement waiting for cleanup completion (and added 
your comment there).
 

> 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