-
Notifications
You must be signed in to change notification settings - Fork 9.1k
HADOOP-19394. Integrate with AAL's readVectored(). #7720
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
base: trunk
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,9 +21,19 @@ | |
|
||
import java.io.EOFException; | ||
import java.io.IOException; | ||
import java.nio.ByteBuffer; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.Optional; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.function.Consumer; | ||
import java.util.function.IntFunction; | ||
|
||
import org.apache.hadoop.fs.FileRange; | ||
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. nit: import block |
||
import org.apache.hadoop.fs.VectoredReadUtils; | ||
import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory; | ||
import software.amazon.s3.analyticsaccelerator.S3SeekableInputStream; | ||
import software.amazon.s3.analyticsaccelerator.common.ObjectRange; | ||
import software.amazon.s3.analyticsaccelerator.request.ObjectMetadata; | ||
import software.amazon.s3.analyticsaccelerator.util.InputPolicy; | ||
import software.amazon.s3.analyticsaccelerator.util.OpenStreamInformation; | ||
|
@@ -38,6 +48,9 @@ | |
import org.apache.hadoop.fs.s3a.S3AInputPolicy; | ||
import org.apache.hadoop.fs.s3a.S3ObjectAttributes; | ||
|
||
import static org.apache.hadoop.fs.VectoredReadUtils.LOG_BYTE_BUFFER_RELEASED; | ||
|
||
|
||
/** | ||
* Analytics stream creates a stream using aws-analytics-accelerator-s3. This stream supports | ||
* parquet specific optimisations such as parquet-aware prefetching. For more details, see | ||
|
@@ -128,6 +141,48 @@ public int read(byte[] buf, int off, int len) throws IOException { | |
return bytesRead; | ||
} | ||
|
||
/** | ||
* {@inheritDoc} | ||
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. I'd put that at L148 and cut the params/IOE as superfluous |
||
* Pass to {@link #readVectored(List, IntFunction, Consumer)} | ||
* with the {@link VectoredReadUtils#LOG_BYTE_BUFFER_RELEASED} releaser. | ||
* @param ranges the byte ranges to read. | ||
* @param allocate the function to allocate ByteBuffer. | ||
* @throws IOException IOE if any. | ||
*/ | ||
@Override | ||
public void readVectored(List<? extends FileRange> ranges, | ||
IntFunction<ByteBuffer> allocate) throws IOException { | ||
readVectored(ranges, allocate, LOG_BYTE_BUFFER_RELEASED); | ||
} | ||
|
||
/** | ||
* {@inheritDoc} | ||
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. same |
||
* Pass to {@link #readVectored(List, IntFunction, Consumer)} | ||
* with the {@link VectoredReadUtils#LOG_BYTE_BUFFER_RELEASED} releaser. | ||
* @param ranges the byte ranges to read. | ||
* @param allocate the function to allocate ByteBuffer. | ||
* @throws IOException IOE if any. | ||
*/ | ||
@Override | ||
public void readVectored(final List<? extends FileRange> ranges, | ||
final IntFunction<ByteBuffer> allocate, | ||
final Consumer<ByteBuffer> release) throws IOException { | ||
LOG.debug("AAL: Starting vectored read on path {} for ranges {} ", getPathStr(), ranges); | ||
throwIfClosed(); | ||
|
||
List<ObjectRange> objectRanges = new ArrayList<>(); | ||
|
||
for (FileRange range : ranges) { | ||
CompletableFuture<ByteBuffer> result = new CompletableFuture<>(); | ||
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. nit: may be rename to data? |
||
ObjectRange objectRange = new ObjectRange(result, range.getOffset(), range.getLength()); | ||
objectRanges.add(objectRange); | ||
range.setData(result); | ||
} | ||
|
||
// AAL does not do any range coalescing, so input and combined ranges are the same. | ||
this.getS3AStreamStatistics().readVectoredOperationStarted(ranges.size(), ranges.size()); | ||
inputStream.readVectored(objectRanges, allocate, release); | ||
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. does this call release on errors? curious -and hopeful |
||
} | ||
|
||
@Override | ||
public boolean seekToNewSource(long l) throws IOException { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,12 +19,24 @@ | |
package org.apache.hadoop.fs.contract.s3a; | ||
|
||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.FSDataInputStream; | ||
import org.apache.hadoop.fs.FileRange; | ||
import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest; | ||
import org.apache.hadoop.fs.contract.AbstractFSContract; | ||
|
||
import org.junit.jupiter.api.BeforeEach; | ||
|
||
import org.apache.hadoop.fs.contract.ContractTestUtils; | ||
import org.apache.hadoop.fs.statistics.IOStatistics; | ||
import org.apache.hadoop.fs.statistics.StreamStatisticNames; | ||
|
||
import org.junit.Test; | ||
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. nit: ordering |
||
|
||
import java.util.List; | ||
|
||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.enableAnalyticsAccelerator; | ||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipForAnyEncryptionExceptSSES3; | ||
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; | ||
|
||
/** | ||
* S3A contract tests for vectored reads with the Analytics stream. | ||
|
@@ -63,12 +75,35 @@ protected Configuration createConfiguration() { | |
// This issue is tracked in: | ||
// https://github.com/awslabs/analytics-accelerator-s3/issues/218 | ||
skipForAnyEncryptionExceptSSES3(conf); | ||
conf.set("fs.contract.vector-io-early-eof-check", "false"); | ||
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. why this is removed? |
||
return conf; | ||
} | ||
|
||
@Override | ||
protected AbstractFSContract createContract(Configuration conf) { | ||
return new S3AContract(conf); | ||
} | ||
|
||
@Override | ||
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. add a javadoc explaining why the override |
||
public void testNegativeOffsetRange(String pBufferType) throws Exception { | ||
verifyExceptionalVectoredRead(ContractTestUtils.range(-1, 50), IllegalArgumentException.class); | ||
} | ||
|
||
@Test | ||
public void testReadVectoredWithAALStatsCollection() throws Exception { | ||
|
||
List<FileRange> fileRanges = createSampleNonOverlappingRanges(); | ||
try (FSDataInputStream in = openVectorFile()){ | ||
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. nit, space before { |
||
in.readVectored(fileRanges, getAllocate()); | ||
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. Are we not verifying the data after vectored read. |
||
|
||
IOStatistics st = in.getIOStatistics(); | ||
|
||
// Statistics such as GET requests will be added after IoStats support. | ||
verifyStatisticCounterValue(st, | ||
StreamStatisticNames.STREAM_READ_ANALYTICS_OPENED, 1); | ||
|
||
verifyStatisticCounterValue(st, | ||
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS, | ||
1); | ||
} | ||
} | ||
} |
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.
for the final merge, this should be pulled out to its own.
should we try to get into 3.4.2 now?