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

Gavin Lee commented on FLINK-18592:
-----------------------------------

Is there any follow-up on this issue? Met it on Flink 1.11.2 with following 
error msg:

2021-05-24 10:07:502021-05-24 10:07:50java.io.IOException: Problem while 
truncating file: 
hdfs://***/***/***/***/.part-2-20760.inprogress.88eb880a-3719-43cd-8e71-b6b356994645
 at 
org.apache.flink.runtime.fs.hdfs.HadoopRecoverableFsDataOutputStream.safelyTruncateFile(HadoopRecoverableFsDataOutputStream.java:167)
 at 
org.apache.flink.runtime.fs.hdfs.HadoopRecoverableFsDataOutputStream.<init>(HadoopRecoverableFsDataOutputStream.java:90)
 at 
org.apache.flink.runtime.fs.hdfs.HadoopRecoverableWriter.recover(HadoopRecoverableWriter.java:83)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.OutputStreamBasedPartFileWriter$OutputStreamBasedBucketWriter.resumeInProgressFileFrom(OutputStreamBasedPartFileWriter.java:91)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.Bucket.restoreInProgressFile(Bucket.java:134)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.Bucket.<init>(Bucket.java:121)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.Bucket.restore(Bucket.java:379)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.DefaultBucketFactoryImpl.restoreBucket(DefaultBucketFactoryImpl.java:63)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.handleRestoredBucketState(Buckets.java:176)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.initializeActiveBuckets(Buckets.java:164)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.Buckets.initializeState(Buckets.java:148)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSinkHelper.<init>(StreamingFileSinkHelper.java:74)
 at 
org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink.initializeState(StreamingFileSink.java:399)
 at 
org.apache.flink.streaming.util.functions.StreamingFunctionUtils.tryRestoreFunction(StreamingFunctionUtils.java:185)
 at 
org.apache.flink.streaming.util.functions.StreamingFunctionUtils.restoreFunctionState(StreamingFunctionUtils.java:167)
 at 
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.initializeState(AbstractUdfStreamOperator.java:96)
 at 
org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.initializeOperatorState(StreamOperatorStateHandler.java:106)
 at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:258)
 at 
org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:290)
 at 
org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:479)
 at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47)
 at 
org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:475)
 at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:528) 
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721) at 
org.apache.flink.runtime.taskmanager.Task.run(Task.java:546) at 
java.lang.Thread.run(Thread.java:748)Caused by: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.HadoopIllegalArgumentException):
 Cannot truncate to a larger file size. Current size: 0, truncate size: 3937. 
at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.truncateInternal(FSNamesystem.java:2264)
 at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.truncateInt(FSNamesystem.java:2202)
 at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.truncate(FSNamesystem.java:2172)
 at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.truncate(NameNodeRpcServer.java:1056)
 at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.truncate(ClientNamenodeProtocolServerSideTranslatorPB.java:622)
 at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
 at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:640)
 at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:982) at 
org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2351) at 
org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2347) at 
java.security.AccessController.doPrivileged(Native Method) at 
javax.security.auth.Subject.doAs(Subject.java:422) at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1866)
 at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2345)
 at org.apache.hadoop.ipc.Client.call(Client.java:1476) at 
org.apache.hadoop.ipc.Client.call(Client.java:1413) at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
 at com.sun.proxy.$Proxy35.truncate(Unknown Source) at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.truncate(ClientNamenodeProtocolTranslatorPB.java:313)
 at sun.reflect.GeneratedMethodAccessor144.invoke(Unknown Source) at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:498) at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:191)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
 at com.sun.proxy.$Proxy36.truncate(Unknown Source) at 
org.apache.hadoop.hdfs.DFSClient.truncate(DFSClient.java:2025) at 
org.apache.hadoop.hdfs.DistributedFileSystem$13.doCall(DistributedFileSystem.java:689)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem$13.doCall(DistributedFileSystem.java:685)
 at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
 at 
org.apache.hadoop.hdfs.DistributedFileSystem.truncate(DistributedFileSystem.java:696)
 at sun.reflect.GeneratedMethodAccessor143.invoke(Unknown Source) at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:498) at 
org.apache.flink.runtime.fs.hdfs.HadoopRecoverableFsDataOutputStream.truncate(HadoopRecoverableFsDataOutputStream.java:202)
 at 
org.apache.flink.runtime.fs.hdfs.HadoopRecoverableFsDataOutputStream.safelyTruncateFile(HadoopRecoverableFsDataOutputStream.java:165)
 ... 25 more

> 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
>              Labels: pull-request-available
>             Fix For: 1.10.4, 1.11.4, 1.14.0
>
>
> 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)

Reply via email to