10000 HADOOP-19394. Integrate with AAL's readVectored(). by ahmarsuhail · Pull Request #7720 · apache/hadoop · GitHub
[go: up one dir, main page]
More Web Proxy on the site http://driver.im/
Skip to content

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

Open
wants to merge 1 commit into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;
import org.apache.hadoop.fs.VectoredReadUtils;
import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory;
import software.amazon.s3.analyticsaccelerator.S3SeekableInputStream;
10000 import software.amazon.s3.analyticsaccelerator.common.ObjectRange;

Check failure on line 36 in hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java

View check run for this annotation

ASF Cloudbees Jenkins ci-hadoop / Apache Yetus

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java#L36

javadoc: error: cannot find symbol
import software.amazon.s3.analyticsaccelerator.request.ObjectMetadata;
import software.amazon.s3.analyticsaccelerator.util.InputPolicy;
import software.amazon.s3.analyticsaccelerator.util.OpenStreamInformation;
Expand All @@ -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
Expand Down Expand Up @@ -128,6 +141,48 @@
return bytesRead;
}

/**
* {@inheritDoc}
* 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}
* 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<>();

Choose a reason for hiding this comment

The 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);
}

@Override
public boolean seekToNewSource(long l) throws IOException {
Expand Down
8000
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,21 @@
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.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.StreamStatisticNames;

import org.junit.Test;
Copy link
Contributor

Choose a reason for hiding this comment

The 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.
Expand Down Expand Up @@ -57,12 +67,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");
return conf;
}

@Override
protected AbstractFSContract createContract(Configuration conf) {
return new S3AContract(conf);
}

@Override
public void testNegativeOffsetRange() throws Exception {
verifyExceptionalVectoredRead(ContractTestUtils.range(-1, 50), IllegalArgumentException.class);
}

@Test
public void testReadVectoredWithAALStatsCollection() throws Exception {

List<FileRange> fileRanges = createSampleNonOverlappingRanges();
try (FSDataInputStream in = openVectorFile()){
in.readVectored(fileRanges, getAllocate());

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);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ protected AbstractFSContract createContract(Configuration conf) {
public void setup() throws Exception {
super.setup();
skipIfAnalyticsAcceleratorEnabled(getContract().getConf(),
"Analytics Accelerator does not support vectored reads");
"AAL with readVectored() is tested in ITestS3AContractAnalyticsStreamVectoredRead");
}

/**
Expand Down
0