-
Notifications
You must be signed in to change notification settings - Fork 9.2k
HADOOP-19303. VectorIO API: support pass-down of a release() operator #7418
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
steveloughran
merged 2 commits into
apache:trunk
from
steveloughran:s3/HADOOP-19303-vectorio-api
Mar 3, 2025
+427
−54
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -49,25 +49,29 @@ | |
| import java.util.concurrent.atomic.AtomicLong; | ||
| import java.util.List; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.function.Consumer; | ||
| import java.util.function.IntFunction; | ||
|
|
||
| import org.apache.hadoop.classification.InterfaceAudience; | ||
| import org.apache.hadoop.classification.InterfaceStability; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.impl.StoreImplementationUtils; | ||
| import org.apache.hadoop.fs.impl.VectorIOBufferPool; | ||
| import org.apache.hadoop.fs.permission.FsPermission; | ||
| import org.apache.hadoop.fs.statistics.IOStatistics; | ||
| import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; | ||
| import org.apache.hadoop.fs.statistics.IOStatisticsContext; | ||
| import org.apache.hadoop.fs.statistics.IOStatisticsSource; | ||
| import org.apache.hadoop.fs.statistics.BufferedIOStatisticsOutputStream; | ||
| import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; | ||
| import org.apache.hadoop.io.ByteBufferPool; | ||
| import org.apache.hadoop.io.IOUtils; | ||
| import org.apache.hadoop.io.nativeio.NativeIO; | ||
| import org.apache.hadoop.util.Progressable; | ||
| import org.apache.hadoop.util.Shell; | ||
| import org.apache.hadoop.util.StringUtils; | ||
|
|
||
| import static org.apache.hadoop.fs.VectoredReadUtils.LOG_BYTE_BUFFER_RELEASED; | ||
| import static org.apache.hadoop.fs.VectoredReadUtils.sortRangeList; | ||
| import static org.apache.hadoop.fs.VectoredReadUtils.validateRangeRequest; | ||
| import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; | ||
|
|
@@ -319,74 +323,131 @@ AsynchronousFileChannel getAsyncChannel() throws IOException { | |
| @Override | ||
| public void readVectored(List<? extends FileRange> ranges, | ||
| IntFunction<ByteBuffer> allocate) throws IOException { | ||
| readVectored(ranges, allocate, LOG_BYTE_BUFFER_RELEASED); | ||
| } | ||
|
|
||
| @Override | ||
| public void readVectored(final List<? extends FileRange> ranges, | ||
| final IntFunction<ByteBuffer> allocate, | ||
| final Consumer<ByteBuffer> release) throws IOException { | ||
|
|
||
| // Validate, but do not pass in a file length as it may change. | ||
| List<? extends FileRange> sortedRanges = sortRangeList(ranges); | ||
| // Set up all of the futures, so that we can use them if things fail | ||
| for(FileRange range: sortedRanges) { | ||
| // Set up all of the futures, so that the caller can await on | ||
| // their completion. | ||
| for (FileRange range: sortedRanges) { | ||
| validateRangeRequest(range); | ||
| range.setData(new CompletableFuture<>()); | ||
| } | ||
| try { | ||
| AsynchronousFileChannel channel = getAsyncChannel(); | ||
| ByteBuffer[] buffers = new ByteBuffer[sortedRanges.size()]; | ||
| AsyncHandler asyncHandler = new AsyncHandler(channel, sortedRanges, buffers); | ||
| for(int i = 0; i < sortedRanges.size(); ++i) { | ||
| FileRange range = sortedRanges.get(i); | ||
| buffers[i] = allocate.apply(range.getLength()); | ||
| channel.read(buffers[i], range.getOffset(), i, asyncHandler); | ||
| } | ||
| } catch (IOException ioe) { | ||
| LOG.debug("Exception occurred during vectored read ", ioe); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. seems like this was unnecessary. Any failure will automatically be caught in failed() method.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. there is no IOE to be raised any more |
||
| for(FileRange range: sortedRanges) { | ||
| range.getData().completeExceptionally(ioe); | ||
| } | ||
| } | ||
| final ByteBufferPool pool = new VectorIOBufferPool(allocate, release); | ||
| // Initiate the asynchronous reads. | ||
| new AsyncHandler(getAsyncChannel(), | ||
| sortedRanges, | ||
| pool) | ||
| .initiateRead(); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * A CompletionHandler that implements readFully and translates back | ||
| * into the form of CompletionHandler that our users expect. | ||
| * <p> | ||
| * All reads are started in {@link #initiateRead()}; | ||
| * the handler then receives callbacks on success | ||
| * {@link #completed(Integer, Integer)}, and on failure | ||
| * by {@link #failed(Throwable, Integer)}. | ||
| * These are mapped to the specific range in the read, and its | ||
| * outcome updated. | ||
| */ | ||
| static class AsyncHandler implements CompletionHandler<Integer, Integer> { | ||
| private static class AsyncHandler implements CompletionHandler<Integer, Integer> { | ||
| /** File channel to read from. */ | ||
| private final AsynchronousFileChannel channel; | ||
|
|
||
| /** Ranges to fetch. */ | ||
| private final List<? extends FileRange> ranges; | ||
|
|
||
| /** | ||
| * Pool providing allocate/release operations. | ||
| */ | ||
| private final ByteBufferPool allocateRelease; | ||
|
|
||
| /** Buffers being read. */ | ||
| private final ByteBuffer[] buffers; | ||
|
|
||
| AsyncHandler(AsynchronousFileChannel channel, | ||
| List<? extends FileRange> ranges, | ||
| ByteBuffer[] buffers) { | ||
| /** | ||
| * Instantiate. | ||
| * @param channel open channel. | ||
| * @param ranges ranges to read. | ||
| * @param allocateRelease pool for allocating buffers, and releasing on failure | ||
| */ | ||
| AsyncHandler( | ||
| final AsynchronousFileChannel channel, | ||
| final List<? extends FileRange> ranges, | ||
| final ByteBufferPool allocateRelease) { | ||
| this.channel = channel; | ||
| this.ranges = ranges; | ||
| this.buffers = buffers; | ||
| this.buffers = new ByteBuffer[ranges.size()]; | ||
| this.allocateRelease = allocateRelease; | ||
| } | ||
|
|
||
| /** | ||
| * Initiate the read operation. | ||
| * <p> | ||
| * Allocate all buffers, queue the read into the channel, | ||
| * providing this object as the handler. | ||
| */ | ||
| private void initiateRead() { | ||
| for(int i = 0; i < ranges.size(); ++i) { | ||
| FileRange range = ranges.get(i); | ||
| buffers[i] = allocateRelease.getBuffer(false, range.getLength()); | ||
| channel.read(buffers[i], range.getOffset(), i, this); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Callback for a completed full/partial read. | ||
| * <p> | ||
| * For an EOF the number of bytes may be -1. | ||
| * That is mapped to a {@link #failed(Throwable, Integer)} outcome. | ||
| * @param result The bytes read. | ||
| * @param rangeIndex range index within the range list. | ||
| */ | ||
| @Override | ||
| public void completed(Integer result, Integer r) { | ||
| FileRange range = ranges.get(r); | ||
| ByteBuffer buffer = buffers[r]; | ||
| public void completed(Integer result, Integer rangeIndex) { | ||
| FileRange range = ranges.get(rangeIndex); | ||
| ByteBuffer buffer = buffers[rangeIndex]; | ||
| if (result == -1) { | ||
| failed(new EOFException("Read past End of File"), r); | ||
| // no data was read back. | ||
| failed(new EOFException("Read past End of File"), rangeIndex); | ||
| } else { | ||
| if (buffer.remaining() > 0) { | ||
| // issue a read for the rest of the buffer | ||
| // QQ: What if this fails? It has the same handler. | ||
| channel.read(buffer, range.getOffset() + buffer.position(), r, this); | ||
| channel.read(buffer, range.getOffset() + buffer.position(), rangeIndex, this); | ||
| } else { | ||
| // QQ: Why is this required? I think because we don't want the | ||
| // user to read data beyond limit. | ||
| // Flip the buffer and declare success. | ||
| buffer.flip(); | ||
| range.getData().complete(buffer); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * The read of the range failed. | ||
| * <p> | ||
| * Release the buffer supplied for this range, then | ||
| * report to the future as {{completeExceptionally(exc)}} | ||
| * @param exc exception. | ||
| * @param rangeIndex range index within the range list. | ||
| */ | ||
| @Override | ||
| public void failed(Throwable exc, Integer r) { | ||
| LOG.debug("Failed while reading range {} ", r, exc); | ||
| ranges.get(r).getData().completeExceptionally(exc); | ||
| public void failed(Throwable exc, Integer rangeIndex) { | ||
| LOG.debug("Failed while reading range {} ", rangeIndex, exc); | ||
| // release the buffer | ||
| allocateRelease.putBuffer(buffers[rangeIndex]); | ||
| // report the failure. | ||
| ranges.get(rangeIndex).getData().completeExceptionally(exc); | ||
| } | ||
|
|
||
| } | ||
|
|
||
| @Override | ||
|
|
||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need coverage of this in the spec?
https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/filesystem/fsdatainputstream.html#void_readVectored.28List.3C.3F_extends_FileRange.3E_ranges.2C_IntFunction.3CByteBuffer.3E_allocate.29
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
will do
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
done