You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Piotr Nowojski (Jira)" <ji...@apache.org> on 2019/12/06 13:48:00 UTC

[jira] [Closed] (FLINK-14949) Task cancellation can be stuck against out-of-thread error

     [ https://issues.apache.org/jira/browse/FLINK-14949?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Piotr Nowojski closed FLINK-14949.
----------------------------------
    Fix Version/s: 1.9.2
       Resolution: Fixed

merged commit 0f47614 to master branch and asĀ 9a9548948563e7778a465d76bc4319c06a29fe7b to release-1.9


Back porting to 1.8 would require to rewrite the test.

> Task cancellation can be stuck against out-of-thread error
> ----------------------------------------------------------
>
>                 Key: FLINK-14949
>                 URL: https://issues.apache.org/jira/browse/FLINK-14949
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Task
>    Affects Versions: 1.8.2
>            Reporter: Hwanju Kim
>            Assignee: Hwanju Kim
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.10.0, 1.9.2
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> Task cancellation ([_cancelOrFailAndCancelInvokable_|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java#L991]) relies on multiple separate threads, which are _TaskCanceler_, _TaskInterrupter_, and _TaskCancelerWatchdog_. While TaskCanceler performs cancellation itself, TaskInterrupter periodically interrupts a non-reacting task and TaskCancelerWatchdog kills JVM if cancellation has never been finished within a certain amount of time (by default 3 min). Those all ensure that cancellation can be done or either aborted transitioning to a terminal state in finite time (FLINK-4715).
> However, if any asynchronous thread creation is failed such as by out-of-thread (_java.lang.OutOfMemoryError: unable to create new native thread_), the code transitions to CANCELING, but nothing could be performed for cancellation or watched by watchdog. Currently, jobmanager does [retry cancellation|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java#L1121] against any error returned, but a next retry [returns success once it sees CANCELING|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java#L997], assuming that it is in progress. This leads to complete stuck in CANCELING, which is non-terminal, so state machine is stuck after that.
> One solution would be that if a task has transitioned to CANCELLING but it gets fatal error or OOM (i.e., _isJvmFatalOrOutOfMemoryError_ is true) indicating that it could not reach spawning TaskCancelerWatchdog, it could immediately consider that as fatal error (not safely cancellable) calling _notifyFatalError_, just as TaskCancelerWatchdog does but eagerly and synchronously. That way, it can at least transition out of the non-terminal state and furthermore clear potentially leaked thread/memory by restarting JVM. The same method is also invoked by _failExternally_, but transitioning to FAILED seems less critical as it's already terminal state.
> How to reproduce is straightforward by running an application that keeps creating threads, each of which never finishes in a loop, and has multiple tasks so that one task triggers failure and then the others are attempted to be cancelled by full fail-over. In web UI dashboard, some tasks from a task manager where any of cancellation-related threads failed to be spawned are stuck in CANCELLING for good.



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