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

Aljoscha Krettek updated FLINK-4779:
------------------------------------
    Priority: Blocker  (was: Major)

> Restoring from savepoint fails when BucketingSink has not yet created folder
> ----------------------------------------------------------------------------
>
>                 Key: FLINK-4779
>                 URL: https://issues.apache.org/jira/browse/FLINK-4779
>             Project: Flink
>          Issue Type: Bug
>          Components: filesystem-connector
>    Affects Versions: 1.2.0
>            Reporter: static-max
>            Priority: Blocker
>
> When I restore from a savepoint, starting the job fails when the root-folder 
> used by the BucketingSink not yet exists. This may happen in my case, when 
> the source for my sink has not yet emitted any messages and I did not create 
> the folder by hand.
> The complete folder structure is not required by the BucketingSink as it will 
> create itermediate folders by itself when creating the bucket.
> I suggest that this does not prevent the job from being restarted.
> {code}
> 10/07/2016 22:50:53     Source: Kafka Consumer for X -> (Sink: HDFS for X, 
> Sink: X)(1/1) switched to FAILED
> java.lang.Exception: Failed to restore state to function: Error while 
> deleting old pending files.
>         at 
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:184)
>         at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreState(StreamTask.java:550)
>         at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:255)
>         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:609)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.RuntimeException: Error while deleting old pending files.
>         at 
> org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.restoreState(BucketingSink.java:805)
>         at 
> org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.restoreState(BucketingSink.java:139)
>         at 
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:182)
>         ... 4 more
> Caused by: java.io.FileNotFoundException: File 
> hdfs://server:8020/1/2/3/4/flink does not exist.
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$DirListingIterator.<init>(DistributedFileSystem.java:948)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$DirListingIterator.<init>(DistributedFileSystem.java:927)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$19.doCall(DistributedFileSystem.java:872)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$19.doCall(DistributedFileSystem.java:868)
>         at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem.listLocatedStatus(DistributedFileSystem.java:886)
>         at 
> org.apache.hadoop.fs.FileSystem.listLocatedStatus(FileSystem.java:1696)
>         at org.apache.hadoop.fs.FileSystem$6.<init>(FileSystem.java:1791)
>         at org.apache.hadoop.fs.FileSystem.listFiles(FileSystem.java:1787)
>         at 
> org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink.restoreState(BucketingSink.java:784)
>         ... 6 more
> {code}



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

Reply via email to