You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Ivan Yang <iv...@gmail.com> on 2020/09/01 00:06:47 UTC

Re: Exception on s3 committer

Hi Yun,

Thank you so much for you suggestion.

(1) The job couldn’t restore from the last checkpoint. The exception is in my original email.
(2) No, I didn’t change any multipart upload settings. 
(3) The file is gone. I have another batch process that reads Flink output s3 bucket and pushes objects to another bucket. Upon success read and write, The batch job will delete the file. What’s puzzling me is if Flink hasn’t successfully commit the multipart file, it should not be visible to the batch job. It looks the situation is while Flink tried to commit the multipart file, it crashed and restarted. The file is committed on s3 successfully, but not acknowledge recorded on Flink side. In between, the batch job consumed the file. I don’t know if that’s possible.

Thanks
Ivan

> On Aug 30, 2020, at 11:10 PM, Yun Gao <yu...@aliyun.com> wrote:
> 
> 
> Hi Ivan,
> 
>    I think there might be some points to check:
> 
>    1. Is the job restored from the latest successful checkpoint after restart ? 
>    2. Have you ever changed the timeout settings for uncompleted multipart upload ?
>    3. Does cbd/late-landing/event_date=2020-08-28/event_hour=16/part-5-264804 exist or not ?
> 
> Best,
>  Yun
> 
> ------------------Original Mail ------------------
> Sender:Ivan Yang <iv...@gmail.com>
> Send Date:Sat Aug 29 12:43:28 2020
> Recipients:user <us...@flink.apache.org>
> Subject:Exception on s3 committer
> Hi all,
> 
> We got this exception after a job restart. Does anyone know what may lead to this situation? and how to get pass this Checkpoint issue? Prior to this, the job failed due to “Checkpoint expired before completing.” We are s3 heavy, writing out 10K files to s3 every 10 minutes using StreamingFileSink/BulkFormat to various s3 prefixes. Thanks in advance. -Ivan
> 
> 2020-08-28 15:17:58
> java.io <http://java.io/>.IOException: Recovering commit failed for object cbd/late-landing/event_date=2020-08-28/event_hour=16/part-5-264804. Object does not exist and MultiPart Upload 3OnIJwYXCxm8fkHpphQOiCdjgfy3jTBqBcg8SbscYJFg0Etl4GoDpPiBms9HUfF_3f7AwL5CyQF4Ne.KDIOKk4aXecP2QRkTTlbbTT8_SnS3Dky.SF7zvDuuMZP9YWlFwtT79rWErOB9K4YPIzUnc4GhUQv4AQIPDF4Nav0ppiw- is not valid.
>     at org.apache.flink.fs.s3.common.writer.S3Committer.commitAfterRecovery(S3Committer.java:102)
>     at org.apache.flink.streaming.api.functions.sink.filesystem.OutputStreamBasedPartFileWriter$OutputStreamBasedPendingFile.commitAfterRecovery(OutputStreamBasedPartFileWriter.java:179)
>     at org.apache.flink.streaming.api.functions.sink.filesystem.Bucket.commitRecoveredPendingFiles(Bucket.java:148)
>     at org.apache.flink.streaming.api.functions.sink.filesystem.Bucket.<init>(Bucket.java:122)
>     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:473)
>     at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:92)
>     at org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:469)
>     at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:522)
>     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: java.io <http://java.io/>.FileNotFoundException: Completing multipart commit on cbd/late-landing/event_date=2020-08-28/event_hour=16/part-5-264804: com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; Request ID: 9A99AFAD80A8F202; S3 Extended Request ID: fjORHBv8K4a5nJ3yULudLjEVgc8vTVro04rYuXC26CQzWs3KMGhoKp/R33g9v4Qi6qN/DsVjENw=), S3 Extended Request ID: fjORHBv8K4a5nJ3yULudLjEVgc8vTVro04rYuXC26CQzWs3KMGhoKp/R33g9v4Qi6qN/DsVjENw=:NoSuchUpload
>     at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:225)
>     at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
>     at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:260)
>     at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:317)
>     at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:256)
>     at org.apache.hadoop.fs.s3a.WriteOperationHelper.finalizeMultipartUpload(WriteOperationHelper.java:222)
>     at org.apache.hadoop.fs.s3a.WriteOperationHelper.completeMPUwithRetries(WriteOperationHelper.java:267)
>     at org.apache.flink.fs.s3hadoop.HadoopS3AccessHelper.commitMultiPartUpload(HadoopS3AccessHelper.java:84)
>     at org.apache.flink.fs.s3.common.writer.S3Committer.commitAfterRecovery(S3Committer.java:85)
>     ... 23 more
> Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist. The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload; Request ID: 9A99AFAD80A8F202; S3 Extended Request ID: fjORHBv8K4a5nJ3yULudLjEVgc8vTVro04rYuXC26CQzWs3KMGhoKp/R33g9v4Qi6qN/DsVjENw=), S3 Extended Request ID: fjORHBv8K4a5nJ3yULudLjEVgc8vTVro04rYuXC26CQzWs3KMGhoKp/R33g9v4Qi6qN/DsVjENw=
>     at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1799)
>     at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1383)
>     at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1359)
>     at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1139)
>     at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:796)
>     at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:764)
>     at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:738)
>     at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:698)
>     at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:680)
>     at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:544)
>     at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:524)
>     at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5054)
>     at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5000)
>     at com.amazonaws.services.s3.AmazonS3Client.completeMultipartUpload(AmazonS3Client.java:3483)
>     at org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$finalizeMultipartUpload$1(WriteOperationHelper.java:229)
>     at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
>     ... 30 more