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

ASF GitHub Bot commented on FLINK-5218:
---------------------------------------

GitHub user StephanEwen opened a pull request:

    https://github.com/apache/flink/pull/2920

    [FLINK-5218] [state backends] Eagerly close checkpoint streams on 
cancellation

    When a task is canceled during a checkpoint operation, the operation needs 
to cancel fast.
    
    This is a forward fis from version 1.1, where checkpoints could get stuck 
when the state output streams did not handle interruptions correctly (HDFS has 
that problem).
    
    Most of this is already handled in version 1.2 via the *CloseableRegistry*.
    
    This adds a test to validate this case is handled correctly and adds minor 
changes to make it work reliably, like:
      - fail fast on `write()` on closed checkpoint streams
      - fail fast on `flush()` on closed checkpoint streams
      - slight optimization to save a flag in the checkpoint streams


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/StephanEwen/incubator-flink closing_validation

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2920.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2920
    
----
commit e592c098f25f97b223f07ff84cd2fd9233e36dc4
Author: Stephan Ewen <se...@apache.org>
Date:   2016-12-01T16:12:12Z

    [FLINK-5218] [state backends] Add test that validates that Checkpoint 
Streams are eagerly closed on cancellation.
    
    This is important for some stream implementations (such as HDFS) that do 
not properly
    handle thread interruption.

----


> Eagerly close checkpoint streams on cancellation
> ------------------------------------------------
>
>                 Key: FLINK-5218
>                 URL: https://issues.apache.org/jira/browse/FLINK-5218
>             Project: Flink
>          Issue Type: Bug
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.1.3
>            Reporter: Stephan Ewen
>            Assignee: Stephan Ewen
>            Priority: Critical
>             Fix For: 1.2.0, 1.1.4
>
>
> Some output streams perform blocking operations that cannot be properly 
> interrupted. This causes cancellations to take very long when happening 
> concurrently to large synchronous state snapshot operations.
> Closing the streams concurrently helps to abort these blocking operations.
> This might already be fixed in 1.2 by the {{CloseableRegistry}}.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to