[ https://issues.apache.org/jira/browse/HADOOP-19569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17953110#comment-17953110 ]
ASF GitHub Bot commented on HADOOP-19569: ----------------------------------------- steveloughran opened a new pull request, #7700: URL: https://github.com/apache/hadoop/pull/7700 HADOOP-19569. Executors in hadoop-common to - pick up shutdown of inner executor and shut themselves down. - semaphore executor to decrement counters in this process so that queue state is updated This stops callers being able to submit work when the inner executor has shut down. S3A code - StoreImpl to IllegalStateException on method invocation whene the service isn't running. Some methods are kept open as they do seem needed. - WriteOperationHelper callbacks to raise IllegalStateException when invoked when FS is closed. This is complex. TODO: - WriteOperationHelper MUST make all calls to the FS through its callback interface, rather than given a ref to S3AFS. This makes it easy to identify and lock down the methods. - What is the correct exception to raise in write/close() failures? IOE or illegal state? <!-- Thanks for sending a pull request! 1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute 2. Make sure your PR title starts with JIRA issue id, e.g., 'HADOOP-17799. Your PR title ...'. --> ### How was this patch tested? New ITests which close the FS while simple and multipart writes are in progress. S3 london. ### For code changes: - [X] Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')? - [X] Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation? - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? - [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, `NOTICE-binary` files? > S3A: stream write/close fails badly once FS is closed > ----------------------------------------------------- > > 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 > Assignee: 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} > * close() in small writes also fails, just with a different exception > * and on some large writes, the output stream hangs as it awaits the end of > the queued writes. This is a problem inside the semaphore executor -- 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