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

Stephan Ewen commented on FLINK-8751:
-------------------------------------

Yes, that makes sense.

The reason here is that when Flink cancels the user code, it interrupts the 
task thread (to pull the code out of blocking operations). When the thread has 
the interrupted flag, and successive wait that is part of graceful shutdown 
immediately throws an InterruptedException.

I think the right fix is to clear the interrupted flag at the beginning of the 
graceful shutdown code (the beginning of the finally code).

> Canceling a job results in a InterruptedException in the TM
> -----------------------------------------------------------
>
>                 Key: FLINK-8751
>                 URL: https://issues.apache.org/jira/browse/FLINK-8751
>             Project: Flink
>          Issue Type: Bug
>          Components: TaskManager
>    Affects Versions: 1.4.1
>            Reporter: Elias Levy
>            Priority: Major
>
> Canceling a job results in the following exception reported by the TM: 
> {code:java}
> ERROR org.apache.flink.streaming.runtime.tasks.StreamTask           - Could 
> not shut down timer service java.lang.InterruptedException 
>   at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(Unknown
>  Source) at java.util.concurrent.ThreadPoolExecutor.awaitTermination(Unknown 
> Source) 
>   at 
> org.apache.flink.streaming.runtime.tasks.SystemProcessingTimeService.shutdownAndAwaitPending(SystemProcessingTimeService.java:197)
>  
>   at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:317)
>  at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718) 
>   at java.lang.Thread.run(Unknown Source){code}
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to