[ https://issues.apache.org/jira/browse/HADOOP-19569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17951429#comment-17951429 ]
Steve Loughran commented on HADOOP-19569: ----------------------------------------- {code} Block size 16,777,216 Writing data as 8 blocks each of size 16,777,216 bytes Starting: Opening s3a://stevel-usw2/bw for upload Duration of Opening s3a://stevel-usw2/bw for upload: 0:00:00.038 Write block 0 in 0.077 seconds Write block 1 in 0.065 seconds Write block 2 in 0.065 seconds Write block 3 in 1.715 seconds Write block 4 in 0.067 seconds Write block 5 in 0.066 seconds Write block 6 in 0.065 seconds Write block 7 in 0.074 seconds Starting: upload stream close() `cc^C^C^C^C^C^C^C2025-05-14 17:32:37,416 [shutdown-hook-0] INFO s3a.S3AFileSystem (S3AFileSystem.java:processDeleteOnExit(4331)) - Ignoring failure to deleteOnExit for path s3a://stevel-usw2/bw 2025-05-14 17:32:41,846 [s3a-transfer-stevel-usw2-bounded-pool1-t1] WARN s3a.S3ABlockOutputStream (S3ABlockOutputStream.java:progressChanged(1334)) - Transfer failure of block ByteArrayBlock{index=0, state=Upload, limit=67108864, dataSize=67108864} 2025-05-14 17:32:41,852 [s3a-transfer-stevel-usw2-bounded-pool1-t2] WARN s3a.S3ABlockOutputStream (S3ABlockOutputStream.java:progressChanged(1334)) - Transfer failure of block ByteArrayBlock{index=0, state=Closed, limit=67108864, dataSize=67108864} 2025-05-14 17:32:41,856 [main] INFO impl.LoggingAuditor (LoggingAuditor.java:requestCreated(510)) - Creating a request outside an audit span {multipart_upload_aborted 'bw' size=0, mutating=true} Duration of upload stream close(): 0:00:22.772 java.lang.NullPointerException at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$abortMultipartUpload$41(S3AFileSystem.java:5337) at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:547) at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:528) at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:449) at org.apache.hadoop.fs.s3a.S3AFileSystem.abortMultipartUpload(S3AFileSystem.java:5336) at org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$abortMultipartUpload$4(WriteOperationHelper.java:392) at org.apache.hadoop.fs.store.audit.AuditingFunctions.lambda$withinAuditSpan$1(AuditingFunctions.java:80) at org.apache.hadoop.fs.s3a.Invoker.lambda$once$0(Invoker.java:165) at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:122) at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:163) at org.apache.hadoop.fs.s3a.WriteOperationHelper.abortMultipartUpload(WriteOperationHelper.java:389) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.lambda$abort$4(S3ABlockOutputStream.java:1258) at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.measureDurationOfInvocation(IOStatisticsBinding.java:494) at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation(IOStatisticsBinding.java:465) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.abort(S3ABlockOutputStream.java:1255) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.waitForAllPartUploads(S3ABlockOutputStream.java:1185) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.access$1400(S3ABlockOutputStream.java:875) at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:534) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:77) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) at org.apache.hadoop.fs.store.commands.Bandwidth.run(Bandwidth.java:272) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:82) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:97) at org.apache.hadoop.fs.store.commands.Bandwidth.exec(Bandwidth.java:470) at org.apache.hadoop.fs.store.commands.Bandwidth.main(Bandwidth.java:479) at bandwidth.main(bandwidth.java:25) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.util.RunJar.run(RunJar.java:333) at org.apache.hadoop.util.RunJar.main(RunJar.java:254) 2025-05-14 17:32:41,865 [main] INFO util.ExitUtil (ExitUtil.java:terminate(241)) - Exiting with status -1: java.lang.NullPointerException {code} > S3A: NPE in S3AFileSystem.abortMultipartUpload if fs closed and multipart > upload in progress > -------------------------------------------------------------------------------------------- > > Key: HADOOP-19569 > URL: https://issues.apache.org/jira/browse/HADOOP-19569 > Project: Hadoop Common > Issue Type: Bug > Components: fs/s3 > Affects Versions: 3.5.0 > Reporter: Steve Loughran > Priority: Minor > > when closing a process during a large upload, and NPE is triggered in the > abort call. This is because the S3 client has already been released. > {code} > java.lang.NullPointerException > at > org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$abortMultipartUpload$41(S3AFileSystem.java:5337) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:547) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:528) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:449) > at > org.apache.hadoop.fs.s3a.S3AFileSystem.abortMultipartUpload(S3AFileSystem.java:5336) > at > org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$abortMultipartUpload$4(WriteOperationHelper.java:392) > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org