[ https://issues.apache.org/jira/browse/FLINK-18592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17175575#comment-17175575 ]
Yu Li edited comment on FLINK-18592 at 8/11/20, 1:41 PM: --------------------------------------------------------- More details about the error stack please refer to the [original email|http://apache-flink.147419.n8.nabble.com/Flink-1-10-1-StreamingFileSink-HDFS-td5898.html] of user-zh ML [~ALVINWJ] I think your analysis makes sense that we should have tried to recover lease after the lease timeout passed but file still not closed in {{HadoopRecoverableFsDataOutputStream#waitUntilLeaseIsRevoked}}, and we could reuse the [RecoverLeaseFSUtils tool in HBase|https://github.com/apache/hbase/blob/a9a1b9524daa9e33541c655620b9c07d5a93d533/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/util/RecoverLeaseFSUtils.java] to implement this. Since you've done a good job of analyze the problem, would you like to submit a PR to fix this? Or just let us know if you prefer someone else to do the task. Thanks. was (Author: carp84): More details about the error stack please refer to the [original email|[http://apache-flink.147419.n8.nabble.com/Flink-1-10-1-StreamingFileSink-HDFS-td5898.html]] of user-zh ML [~ALVINWJ] I think your analysis makes sense that we should have tried to recover lease after the lease timeout passed but file still not closed in {{HadoopRecoverableFsDataOutputStream#waitUntilLeaseIsRevoked}}, and we could reuse the [RecoverLeaseFSUtils tool in HBase|https://github.com/apache/hbase/blob/a9a1b9524daa9e33541c655620b9c07d5a93d533/hbase-asyncfs/src/main/java/org/apache/hadoop/hbase/util/RecoverLeaseFSUtils.java] to implement this. Since you've done a good job of analyze the problem, would you like to submit a PR to fix this? Or just let us know if you prefer someone else to do the task. Thanks. > StreamingFileSink fails due to truncating HDFS file failure > ----------------------------------------------------------- > > Key: FLINK-18592 > URL: https://issues.apache.org/jira/browse/FLINK-18592 > Project: Flink > Issue Type: Bug > Components: Connectors / FileSystem > Affects Versions: 1.10.1 > Reporter: JIAN WANG > Priority: Major > > I meet the issue on flink-1.10.1. I use flink on YARN(3.0.0-cdh6.3.2) with > StreamingFileSink. > code part like this: > {code} > public static <IN> StreamingFileSink<IN> build(String dir, > BucketAssigner<IN, String> assigner, String prefix) { > return StreamingFileSink.forRowFormat(new Path(dir), new > SimpleStringEncoder<IN>()) > .withRollingPolicy( > DefaultRollingPolicy > .builder() > > .withRolloverInterval(TimeUnit.HOURS.toMillis(2)) > > .withInactivityInterval(TimeUnit.MINUTES.toMillis(10)) > .withMaxPartSize(1024L * 1024L * 1024L > * 50) // Max 50GB > .build()) > .withBucketAssigner(assigner) > > .withOutputFileConfig(OutputFileConfig.builder().withPartPrefix(prefix).build()) > .build(); > } > {code} > The error is > {noformat} > java.io.IOException: > Problem while truncating file: > hdfs:///business_log/hashtag/2020-06-25/.hashtag-122-37.inprogress.8e65f69c-b5ba-4466-a844-ccc0a5a93de2 > {noformat} > Due to this issue, it can not restart from the latest checkpoint and > savepoint. > Currently, my workaround is that we keep latest 3 checkpoint, and if it > fails, I manually restart from penult checkpoint. -- This message was sent by Atlassian Jira (v8.3.4#803005)