[
https://issues.apache.org/jira/browse/HADOOP-18521?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17629937#comment-17629937
]
ASF GitHub Bot commented on HADOOP-18521:
-----------------------------------------
steveloughran opened a new pull request, #5117:
URL: https://github.com/apache/hadoop/pull/5117
Addresses the issue by not trying to cancel in-progress reads when a stream
is closed()...they are allowed to continue and then their data discarded.
To enable discarding, AbfsInputStreams export their `closed` state in
which is now AtomicBool internally so reader threads can probe it.
The shared buffers now have owner tracking, which will reject
* attempts to acquire an owned buffer
* attempts to return a buffer not owned
Plus
* Lots of other invariants added to validate the state
* useful to string values
Also adds path and stream capability probe for the fix;
cloudstore "pathcapability" probe can report this.
Hadoop 3.3.2 added the path capability
"fs.capability.paths.acls", so two probes can
determine if abfs is exposed:
not vulnerable
!hasPathCability("fs.capability.paths.acls")
|| hasPathCability("fs.azure.capability.prefetch.safe")
vulnerable
hasPathCability("fs.capability.paths.acls")
&& !hasPathCability("fs.azure.capability.prefetch.safe")
It can also be demanded in an openFile() call.
That block the code ever working on a version without
the race condition. Possibly a bit excessive.
### How was this patch tested?
needs more tests with multi GB csv files to validate the patch.
Unable to come up with good tests to recreate the failure condition.
### 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?
> ABFS ReadBufferManager buffer sharing across concurrent HTTP requests
> ---------------------------------------------------------------------
>
> Key: HADOOP-18521
> URL: https://issues.apache.org/jira/browse/HADOOP-18521
> Project: Hadoop Common
> Issue Type: Bug
> Components: fs/azure
> Affects Versions: 3.3.2, 3.3.3, 3.3.4
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Priority: Critical
>
> AbfsInputStream.close() can trigger the return of buffers used for active
> prefetch GET requests into the ReadBufferManager free buffer pool.
> A subsequent prefetch by a different stream in the same process may acquire
> this same buffer. This can lead to risk of corruption of its own prefetched
> data, data which may then be returned to that other thread.
> On releases without the fix for this (3.3.2 to 3.3.4), the bug can be avoided
> by disabling all prefetching
> {code}
> fs.azure.readaheadqueue.depth
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]