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

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

GitHub user aljoscha opened a pull request:

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

    [FLINK-2979] Fix RollingSink truncate for Hadoop 2.7

    The problem was, that truncate is asynchronous and the RollingSink was
    not taking this into account.
    
    Now it has a loop after the truncate call that waits until the file is
    actually truncated.
    
    This also changes the Hadoop 2.6 travis build to 2.7, instead.

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

    $ git pull https://github.com/aljoscha/flink rolling-sink-fix

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

    https://github.com/apache/flink/pull/1336.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 #1336
    
----
commit 05136da77617a577e62cca2dec469e2c2d14b91e
Author: Aljoscha Krettek <aljoscha.kret...@gmail.com>
Date:   2015-11-06T15:15:06Z

    [FLINK-2979] Fix RollingSink truncate for Hadoop 2.7
    
    The problem was, that truncate is asynchronous and the RollingSink was
    not taking this into account.
    
    Now it has a loop after the truncate call that waits until the file is
    actually truncated.
    
    This also changes the Hadoop 2.6 travis build to 2.7, instead.

----


> RollingSink does not work with Hadoop 2.7.1
> -------------------------------------------
>
>                 Key: FLINK-2979
>                 URL: https://issues.apache.org/jira/browse/FLINK-2979
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming Connectors
>    Affects Versions: 0.10
>            Reporter: Till Rohrmann
>            Assignee: Aljoscha Krettek
>
> When executing the {{RollingSinkFaultToleranceITCase}} with Hadoop 2.7.1, 
> then the test either does not finish because it's stuck in an endless restart 
> loop with the following exception
> {code}
> java.lang.Exception: Could not restore checkpointed state to operators and 
> functions
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreStateLazy(StreamTask.java:414)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:208)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.Exception: Failed to restore state to function: 
> In-Progress file hdfs://127.0.0.1:52884/string-non-rolling-out/part-0-1 was 
> neither moved to pending nor is still in progress.
>       at 
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:165)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreStateLazy(StreamTask.java:406)
>       ... 3 more
> Caused by: java.lang.RuntimeException: In-Progress file 
> hdfs://127.0.0.1:52884/string-non-rolling-out/part-0-1 was neither moved to 
> pending nor is still in progress.
>       at 
> org.apache.flink.streaming.connectors.fs.RollingSink.restoreState(RollingSink.java:670)
>       at 
> org.apache.flink.streaming.connectors.fs.RollingSink.restoreState(RollingSink.java:120)
>       at 
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:162)
>       ... 4 more
> {code}
> or it fails because the number of read strings differs from the exactly-once 
> result (some strings are read multiple times).



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

Reply via email to