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

Steve Loughran commented on HADOOP-19569:
-----------------------------------------

Can also seem to hang if close() is called after the fs is shut as the 
submission failure isn't picked up on
{code}
2025-05-20 20:56:53,212 [JUnit-testOutputClosed] DEBUG s3a.Invoker 
(DurationInfo.java:<init>(80)) - Starting: upload part request
2025-05-20 20:56:53,212 [JUnit-testOutputClosed] DEBUG impl.RequestFactoryImpl 
(RequestFactoryImpl.java:newUploadPartRequestBuilder(662)) - Creating part 
upload request for 
dNf9u4SxpkPi9UN5XHGVK2BiAzR7JqRFsLwowvY2TtIS0vAo1IE9wgBprGTYYfeGOVMe5G_nbHHtjk8l1K7hcwBelCeWr2fNR3mhSe7._fmKNA2kDThpzX_.aAMqSXPS
 #1 size 1
2025-05-20 20:56:53,216 [JUnit-testOutputClosed] DEBUG s3a.Invoker 
(DurationInfo.java:close(101)) - upload part request: duration 0:00.004s
2025-05-20 20:56:53,225 [JUnit-testOutputClosed] ERROR 
util.BlockingThreadPoolExecutorService 
(BlockingThreadPoolExecutorService.java:rejectedExecution(141)) - Could not 
submit task to executor 
java.util.concurrent.ThreadPoolExecutor@2c12bb5e[Terminated, pool size = 0, 
active threads = 0, queued tasks = 0, completed tasks = 0]
2025-05-20 20:56:53,225 [JUnit-testOutputClosed] DEBUG s3a.S3ABlockOutputStream 
(S3ABlockOutputStream.java:clearActiveBlock(340)) - Clearing active block
2025-05-20 20:56:53,225 [JUnit-testOutputClosed] DEBUG s3a.S3ABlockOutputStream 
(S3ABlockOutputStream.java:waitForAllPartUploads(1164)) - Waiting for 1 uploads 
to complete
{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

Reply via email to