Compare commits
5 Commits
trunk
...
feature-ve
Author | SHA1 | Date |
---|---|---|
Mukund Thakur | 645bcd6480 | |
Mukund Thakur | b5520ac778 | |
Mukund Thakur | 06407903ce | |
Mukund Thakur | 307bd14e1e | |
Mukund Thakur | f3f71434b5 |
|
@ -47,7 +47,7 @@ pipeline {
|
|||
|
||||
options {
|
||||
buildDiscarder(logRotator(numToKeepStr: '5'))
|
||||
timeout (time: 24, unit: 'HOURS')
|
||||
timeout (time: 48, unit: 'HOURS')
|
||||
timestamps()
|
||||
checkoutToSubdirectory('src')
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
|
@ -22,6 +22,9 @@ import java.io.EOFException;
|
|||
import java.io.FileDescriptor;
|
||||
import java.io.IOException;
|
||||
import java.util.StringJoiner;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -158,8 +161,24 @@ public class BufferedFSInputStream extends BufferedInputStream
|
|||
@Override
|
||||
public String toString() {
|
||||
return new StringJoiner(", ",
|
||||
BufferedFSInputStream.class.getSimpleName() + "[", "]")
|
||||
.add("in=" + in)
|
||||
.toString();
|
||||
BufferedFSInputStream.class.getSimpleName() + "[", "]")
|
||||
.add("in=" + in)
|
||||
.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int minSeekForVectorReads() {
|
||||
return ((PositionedReadable) in).minSeekForVectorReads();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxReadSizeForVectorReads() {
|
||||
return ((PositionedReadable) in).maxReadSizeForVectorReads();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readVectored(List<? extends FileRange> ranges,
|
||||
IntFunction<ByteBuffer> allocate) throws IOException {
|
||||
((PositionedReadable) in).readVectored(ranges, allocate);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,17 +22,24 @@ import java.io.EOFException;
|
|||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.IntBuffer;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionException;
|
||||
import java.util.function.IntFunction;
|
||||
import java.util.zip.CRC32;
|
||||
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
|
||||
import org.apache.hadoop.fs.impl.CombinedFileRange;
|
||||
import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
|
||||
import org.apache.hadoop.fs.impl.OpenFileParameters;
|
||||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
|
@ -47,6 +54,7 @@ import org.apache.hadoop.util.Progressable;
|
|||
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS;
|
||||
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
|
||||
import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable;
|
||||
import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges;
|
||||
|
||||
/****************************************************************
|
||||
* Abstract Checksumed FileSystem.
|
||||
|
@ -66,7 +74,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
public static double getApproxChkSumLength(long size) {
|
||||
return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size;
|
||||
}
|
||||
|
||||
|
||||
public ChecksumFileSystem(FileSystem fs) {
|
||||
super(fs);
|
||||
}
|
||||
|
@ -82,7 +90,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
bytesPerChecksum);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Set whether to verify checksum.
|
||||
*/
|
||||
|
@ -95,7 +103,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
public void setWriteChecksum(boolean writeChecksum) {
|
||||
this.writeChecksum = writeChecksum;
|
||||
}
|
||||
|
||||
|
||||
/** get the raw file system */
|
||||
@Override
|
||||
public FileSystem getRawFileSystem() {
|
||||
|
@ -158,22 +166,22 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
* It verifies that data matches checksums.
|
||||
*******************************************************/
|
||||
private static class ChecksumFSInputChecker extends FSInputChecker implements
|
||||
IOStatisticsSource {
|
||||
IOStatisticsSource, StreamCapabilities {
|
||||
private ChecksumFileSystem fs;
|
||||
private FSDataInputStream datas;
|
||||
private FSDataInputStream sums;
|
||||
|
||||
|
||||
private static final int HEADER_LENGTH = 8;
|
||||
|
||||
|
||||
private int bytesPerSum = 1;
|
||||
|
||||
|
||||
public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
|
||||
throws IOException {
|
||||
this(fs, file, fs.getConf().getInt(
|
||||
LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY,
|
||||
LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY,
|
||||
LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT));
|
||||
}
|
||||
|
||||
|
||||
public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
|
||||
throws IOException {
|
||||
super( file, fs.getFileStatus(file).getReplication() );
|
||||
|
@ -189,7 +197,8 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
if (!Arrays.equals(version, CHECKSUM_VERSION))
|
||||
throw new IOException("Not a checksum file: "+sumFile);
|
||||
this.bytesPerSum = sums.readInt();
|
||||
set(fs.verifyChecksum, DataChecksum.newCrc32(), bytesPerSum, 4);
|
||||
set(fs.verifyChecksum, DataChecksum.newCrc32(), bytesPerSum,
|
||||
FSInputChecker.CHECKSUM_SIZE);
|
||||
} catch (IOException e) {
|
||||
// mincing the message is terrible, but java throws permission
|
||||
// exceptions as FNF because that's all the method signatures allow!
|
||||
|
@ -201,21 +210,21 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
set(fs.verifyChecksum, null, 1, 0);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private long getChecksumFilePos( long dataPos ) {
|
||||
return HEADER_LENGTH + 4*(dataPos/bytesPerSum);
|
||||
return HEADER_LENGTH + FSInputChecker.CHECKSUM_SIZE*(dataPos/bytesPerSum);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected long getChunkPosition( long dataPos ) {
|
||||
return dataPos/bytesPerSum*bytesPerSum;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
return datas.available() + super.available();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int read(long position, byte[] b, int off, int len)
|
||||
throws IOException {
|
||||
|
@ -233,7 +242,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
}
|
||||
return nread;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
datas.close();
|
||||
|
@ -242,7 +251,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
}
|
||||
set(fs.verifyChecksum, null, 1, 0);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public boolean seekToNewSource(long targetPos) throws IOException {
|
||||
|
@ -265,7 +274,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
final int checksumsToRead = Math.min(
|
||||
len/bytesPerSum, // number of checksums based on len to read
|
||||
checksum.length / CHECKSUM_SIZE); // size of checksum buffer
|
||||
long checksumPos = getChecksumFilePos(pos);
|
||||
long checksumPos = getChecksumFilePos(pos);
|
||||
if(checksumPos != sums.getPos()) {
|
||||
sums.seek(checksumPos);
|
||||
}
|
||||
|
@ -305,8 +314,134 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
public IOStatistics getIOStatistics() {
|
||||
return IOStatisticsSupport.retrieveIOStatistics(datas);
|
||||
}
|
||||
|
||||
public static long findChecksumOffset(long dataOffset,
|
||||
int bytesPerSum) {
|
||||
return HEADER_LENGTH + (dataOffset/bytesPerSum) * FSInputChecker.CHECKSUM_SIZE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Find the checksum ranges that correspond to the given data ranges.
|
||||
* @param dataRanges the input data ranges, which are assumed to be sorted
|
||||
* and non-overlapping
|
||||
* @return a list of AsyncReaderUtils.CombinedFileRange that correspond to
|
||||
* the checksum ranges
|
||||
*/
|
||||
public static List<CombinedFileRange> findChecksumRanges(
|
||||
List<? extends FileRange> dataRanges,
|
||||
int bytesPerSum,
|
||||
int minSeek,
|
||||
int maxSize) {
|
||||
List<CombinedFileRange> result = new ArrayList<>();
|
||||
CombinedFileRange currentCrc = null;
|
||||
for(FileRange range: dataRanges) {
|
||||
long crcOffset = findChecksumOffset(range.getOffset(), bytesPerSum);
|
||||
long crcEnd = findChecksumOffset(range.getOffset() + range.getLength() +
|
||||
bytesPerSum - 1, bytesPerSum);
|
||||
if (currentCrc == null ||
|
||||
!currentCrc.merge(crcOffset, crcEnd, range, minSeek, maxSize)) {
|
||||
currentCrc = new CombinedFileRange(crcOffset, crcEnd, range);
|
||||
result.add(currentCrc);
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check the data against the checksums.
|
||||
* @param sumsBytes the checksum data
|
||||
* @param sumsOffset where from the checksum file this buffer started
|
||||
* @param data the file data
|
||||
* @param dataOffset where the file data started (must be a multiple of
|
||||
* bytesPerSum)
|
||||
* @param bytesPerSum how many bytes per a checksum
|
||||
* @param file the path of the filename
|
||||
* @return the data buffer
|
||||
* @throws CompletionException if the checksums don't match
|
||||
*/
|
||||
static ByteBuffer checkBytes(ByteBuffer sumsBytes,
|
||||
long sumsOffset,
|
||||
ByteBuffer data,
|
||||
long dataOffset,
|
||||
int bytesPerSum,
|
||||
Path file) {
|
||||
// determine how many bytes we need to skip at the start of the sums
|
||||
int offset =
|
||||
(int) (findChecksumOffset(dataOffset, bytesPerSum) - sumsOffset);
|
||||
IntBuffer sums = sumsBytes.asIntBuffer();
|
||||
sums.position(offset / FSInputChecker.CHECKSUM_SIZE);
|
||||
ByteBuffer current = data.duplicate();
|
||||
int numChunks = data.remaining() / bytesPerSum;
|
||||
CRC32 crc = new CRC32();
|
||||
// check each chunk to ensure they match
|
||||
for(int c = 0; c < numChunks; ++c) {
|
||||
// set the buffer position and the limit
|
||||
current.limit((c + 1) * bytesPerSum);
|
||||
current.position(c * bytesPerSum);
|
||||
// compute the crc
|
||||
crc.reset();
|
||||
crc.update(current);
|
||||
int expected = sums.get();
|
||||
int calculated = (int) crc.getValue();
|
||||
|
||||
if (calculated != expected) {
|
||||
// cast of c added to silence findbugs
|
||||
long errPosn = dataOffset + (long) c * bytesPerSum;
|
||||
throw new CompletionException(new ChecksumException(
|
||||
"Checksum error: " + file + " at " + errPosn +
|
||||
" exp: " + expected + " got: " + calculated, errPosn));
|
||||
}
|
||||
}
|
||||
// if everything matches, we return the data
|
||||
return data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readVectored(List<? extends FileRange> ranges,
|
||||
IntFunction<ByteBuffer> allocate) throws IOException {
|
||||
// If the stream doesn't have checksums, just delegate.
|
||||
VectoredReadUtils.validateVectoredReadRanges(ranges);
|
||||
if (sums == null) {
|
||||
datas.readVectored(ranges, allocate);
|
||||
return;
|
||||
}
|
||||
int minSeek = minSeekForVectorReads();
|
||||
int maxSize = maxReadSizeForVectorReads();
|
||||
List<CombinedFileRange> dataRanges =
|
||||
VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(ranges)), bytesPerSum,
|
||||
minSeek, maxReadSizeForVectorReads());
|
||||
List<CombinedFileRange> checksumRanges = findChecksumRanges(dataRanges,
|
||||
bytesPerSum, minSeek, maxSize);
|
||||
sums.readVectored(checksumRanges, allocate);
|
||||
datas.readVectored(dataRanges, allocate);
|
||||
// Data read is correct. I have verified content of dataRanges.
|
||||
// There is some bug below here as test (testVectoredReadMultipleRanges)
|
||||
// is failing, should be
|
||||
// somewhere while slicing the merged data into smaller user ranges.
|
||||
// Spend some time figuring out but it is a complex code.
|
||||
for(CombinedFileRange checksumRange: checksumRanges) {
|
||||
for(FileRange dataRange: checksumRange.getUnderlying()) {
|
||||
// when we have both the ranges, validate the checksum
|
||||
CompletableFuture<ByteBuffer> result =
|
||||
checksumRange.getData().thenCombineAsync(dataRange.getData(),
|
||||
(sumBuffer, dataBuffer) ->
|
||||
checkBytes(sumBuffer, checksumRange.getOffset(),
|
||||
dataBuffer, dataRange.getOffset(), bytesPerSum, file));
|
||||
// Now, slice the read data range to the user's ranges
|
||||
for(FileRange original: ((CombinedFileRange) dataRange).getUnderlying()) {
|
||||
original.setData(result.thenApply(
|
||||
(b) -> VectoredReadUtils.sliceTo(b, dataRange.getOffset(), original)));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasCapability(String capability) {
|
||||
return datas.hasCapability(capability);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static class FSDataBoundedInputStream extends FSDataInputStream {
|
||||
private FileSystem fs;
|
||||
private Path file;
|
||||
|
@ -317,12 +452,12 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
this.fs = fs;
|
||||
this.file = file;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean markSupported() {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
/* Return the file length */
|
||||
private long getFileLength() throws IOException {
|
||||
if( fileLen==-1L ) {
|
||||
|
@ -330,7 +465,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
}
|
||||
return fileLen;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Skips over and discards <code>n</code> bytes of data from the
|
||||
* input stream.
|
||||
|
@ -354,11 +489,11 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
}
|
||||
return super.skip(n);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Seek to the given position in the stream.
|
||||
* The next read() will be from that position.
|
||||
*
|
||||
*
|
||||
* <p>This method does not allow seek past the end of the file.
|
||||
* This produces IOException.
|
||||
*
|
||||
|
@ -424,22 +559,22 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
*/
|
||||
public static long getChecksumLength(long size, int bytesPerSum) {
|
||||
//the checksum length is equal to size passed divided by bytesPerSum +
|
||||
//bytes written in the beginning of the checksum file.
|
||||
return ((size + bytesPerSum - 1) / bytesPerSum) * 4 +
|
||||
CHECKSUM_VERSION.length + 4;
|
||||
//bytes written in the beginning of the checksum file.
|
||||
return ((size + bytesPerSum - 1) / bytesPerSum) * FSInputChecker.CHECKSUM_SIZE +
|
||||
ChecksumFSInputChecker.HEADER_LENGTH;
|
||||
}
|
||||
|
||||
/** This class provides an output stream for a checksummed file.
|
||||
* It generates checksums for data. */
|
||||
private static class ChecksumFSOutputSummer extends FSOutputSummer
|
||||
implements IOStatisticsSource, StreamCapabilities {
|
||||
private FSDataOutputStream datas;
|
||||
private FSDataOutputStream datas;
|
||||
private FSDataOutputStream sums;
|
||||
private static final float CHKSUM_AS_FRACTION = 0.01f;
|
||||
private boolean isClosed = false;
|
||||
|
||||
public ChecksumFSOutputSummer(ChecksumFileSystem fs,
|
||||
Path file,
|
||||
|
||||
ChecksumFSOutputSummer(ChecksumFileSystem fs,
|
||||
Path file,
|
||||
boolean overwrite,
|
||||
int bufferSize,
|
||||
short replication,
|
||||
|
@ -460,7 +595,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
|
||||
sums.writeInt(bytesPerSum);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
|
@ -471,7 +606,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
isClosed = true;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected void writeChunk(byte[] b, int offset, int len, byte[] checksum,
|
||||
int ckoff, int cklen)
|
||||
|
@ -727,7 +862,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
value = fs.rename(srcCheckFile, dstCheckFile);
|
||||
} else if (fs.exists(dstCheckFile)) {
|
||||
// no src checksum, so remove dst checksum
|
||||
value = fs.delete(dstCheckFile, true);
|
||||
value = fs.delete(dstCheckFile, true);
|
||||
}
|
||||
|
||||
return value;
|
||||
|
@ -759,7 +894,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
return fs.delete(f, true);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
final private static PathFilter DEFAULT_FILTER = new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path file) {
|
||||
|
@ -770,7 +905,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
/**
|
||||
* List the statuses of the files/directories in the given path if the path is
|
||||
* a directory.
|
||||
*
|
||||
*
|
||||
* @param f
|
||||
* given path
|
||||
* @return the statuses of the files/directories in the given path
|
||||
|
@ -791,7 +926,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
/**
|
||||
* List the statuses of the files/directories in the given path if the path is
|
||||
* a directory.
|
||||
*
|
||||
*
|
||||
* @param f
|
||||
* given path
|
||||
* @return the statuses of the files/directories in the given patch
|
||||
|
@ -802,7 +937,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
throws IOException {
|
||||
return fs.listLocatedStatus(f, DEFAULT_FILTER);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean mkdirs(Path f) throws IOException {
|
||||
return fs.mkdirs(f);
|
||||
|
@ -856,7 +991,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|||
} else {
|
||||
FileStatus[] srcs = listStatus(src);
|
||||
for (FileStatus srcFile : srcs) {
|
||||
copyToLocalFile(srcFile.getPath(),
|
||||
copyToLocalFile(srcFile.getPath(),
|
||||
new Path(dst, srcFile.getPath().getName()), copyCrc);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
|
@ -26,6 +26,8 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -51,7 +53,7 @@ public class FSDataInputStream extends DataInputStream
|
|||
*/
|
||||
private final IdentityHashStore<ByteBuffer, ByteBufferPool>
|
||||
extendedReadBuffers
|
||||
= new IdentityHashStore<ByteBuffer, ByteBufferPool>(0);
|
||||
= new IdentityHashStore<>(0);
|
||||
|
||||
public FSDataInputStream(InputStream in) {
|
||||
super(in);
|
||||
|
@ -279,4 +281,20 @@ public class FSDataInputStream extends DataInputStream
|
|||
public IOStatistics getIOStatistics() {
|
||||
return IOStatisticsSupport.retrieveIOStatistics(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int minSeekForVectorReads() {
|
||||
return ((PositionedReadable) in).minSeekForVectorReads();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int maxReadSizeForVectorReads() {
|
||||
return ((PositionedReadable) in).maxReadSizeForVectorReads();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readVectored(List<? extends FileRange> ranges,
|
||||
IntFunction<ByteBuffer> allocate) throws IOException {
|
||||
((PositionedReadable) in).readVectored(ranges, allocate);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
import org.apache.hadoop.fs.impl.FileRangeImpl;
|
||||
|
||||
/**
|
||||
* A byte range of a file.
|
||||
* This is used for the asynchronous gather read API of
|
||||
* {@link PositionedReadable#readVectored}.
|
||||
*/
|
||||
public interface FileRange {
|
||||
|
||||
/**
|
||||
* Get the starting offset of the range.
|
||||
* @return the byte offset of the start
|
||||
*/
|
||||
long getOffset();
|
||||
|
||||
/**
|
||||
* Get the length of the range.
|
||||
* @return the number of bytes in the range.
|
||||
*/
|
||||
int getLength();
|
||||
|
||||
/**
|
||||
* Get the future data for this range.
|
||||
* @return the future for the {@link ByteBuffer} that contains the data
|
||||
*/
|
||||
CompletableFuture<ByteBuffer> getData();
|
||||
|
||||
/**
|
||||
* Set a future for this range's data.
|
||||
* This method is called by {@link PositionedReadable#readVectored} to store the
|
||||
* data for the user to pick up later via {@link #getData}.
|
||||
* @param data the future of the ByteBuffer that will have the data
|
||||
*/
|
||||
void setData(CompletableFuture<ByteBuffer> data);
|
||||
|
||||
/**
|
||||
* Factory method to create a FileRange object.
|
||||
* @param offset starting offset of the range.
|
||||
* @param length length of the range.
|
||||
* @return a new instance of FileRangeImpl.
|
||||
*/
|
||||
static FileRange createFileRange(long offset, int length) {
|
||||
return new FileRangeImpl(offset, length);
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
|
@ -17,7 +17,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import java.io.*;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -85,4 +89,37 @@ public interface PositionedReadable {
|
|||
* the read operation completed
|
||||
*/
|
||||
void readFully(long position, byte[] buffer) throws IOException;
|
||||
|
||||
/**
|
||||
* What is the smallest reasonable seek?
|
||||
* @return the minimum number of bytes
|
||||
*/
|
||||
default int minSeekForVectorReads() {
|
||||
return 4 * 1024;
|
||||
}
|
||||
|
||||
/**
|
||||
* What is the largest size that we should group ranges together as?
|
||||
* @return the number of bytes to read at once
|
||||
*/
|
||||
default int maxReadSizeForVectorReads() {
|
||||
return 1024 * 1024;
|
||||
}
|
||||
|
||||
/**
|
||||
* Read fully a list of file ranges asynchronously from this file.
|
||||
* The default iterates through the ranges to read each synchronously, but
|
||||
* the intent is that FSDataInputStream subclasses can make more efficient
|
||||
* readers.
|
||||
* As a result of the call, each range will have FileRange.setData(CompletableFuture)
|
||||
* called with a future that when complete will have a ByteBuffer with the
|
||||
* data from the file's range.
|
||||
* @param ranges the byte ranges to read
|
||||
* @param allocate the function to allocate ByteBuffer
|
||||
* @throws IOException any IOE.
|
||||
*/
|
||||
default void readVectored(List<? extends FileRange> ranges,
|
||||
IntFunction<ByteBuffer> allocate) throws IOException {
|
||||
VectoredReadUtils.readVectored(this, ranges, allocate);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
|
@ -33,8 +33,11 @@ import java.io.OutputStream;
|
|||
import java.io.FileDescriptor;
|
||||
import java.net.URI;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.AsynchronousFileChannel;
|
||||
import java.nio.channels.CompletionHandler;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.NoSuchFileException;
|
||||
import java.nio.file.StandardOpenOption;
|
||||
import java.nio.file.attribute.BasicFileAttributes;
|
||||
import java.nio.file.attribute.BasicFileAttributeView;
|
||||
import java.nio.file.attribute.FileTime;
|
||||
|
@ -44,6 +47,9 @@ import java.util.Locale;
|
|||
import java.util.Optional;
|
||||
import java.util.StringTokenizer;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
@ -61,6 +67,7 @@ import org.apache.hadoop.util.Shell;
|
|||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
|
||||
import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges;
|
||||
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
|
||||
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS;
|
||||
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS;
|
||||
|
@ -130,7 +137,9 @@ public class RawLocalFileSystem extends FileSystem {
|
|||
class LocalFSFileInputStream extends FSInputStream implements
|
||||
HasFileDescriptor, IOStatisticsSource, StreamCapabilities {
|
||||
private FileInputStream fis;
|
||||
private final File name;
|
||||
private long position;
|
||||
private AsynchronousFileChannel asyncChannel = null;
|
||||
|
||||
/**
|
||||
* Minimal set of counters.
|
||||
|
@ -148,7 +157,8 @@ public class RawLocalFileSystem extends FileSystem {
|
|||
private final AtomicLong bytesRead;
|
||||
|
||||
public LocalFSFileInputStream(Path f) throws IOException {
|
||||
fis = new FileInputStream(pathToFile(f));
|
||||
name = pathToFile(f);
|
||||
fis = new FileInputStream(name);
|
||||
bytesRead = ioStatistics.getCounterReference(
|
||||
STREAM_READ_BYTES);
|
||||
}
|
||||
|
@ -179,10 +189,16 @@ public class RawLocalFileSystem extends FileSystem {
|
|||
@Override
|
||||
public int available() throws IOException { return fis.available(); }
|
||||
@Override
|
||||
public void close() throws IOException { fis.close(); }
|
||||
@Override
|
||||
public boolean markSupported() { return false; }
|
||||
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
fis.close();
|
||||
if (asyncChannel != null) {
|
||||
asyncChannel.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
try {
|
||||
|
@ -262,6 +278,7 @@ public class RawLocalFileSystem extends FileSystem {
|
|||
// new capabilities.
|
||||
switch (capability.toLowerCase(Locale.ENGLISH)) {
|
||||
case StreamCapabilities.IOSTATISTICS:
|
||||
case StreamCapabilities.VECTOREDIO:
|
||||
return true;
|
||||
default:
|
||||
return false;
|
||||
|
@ -272,8 +289,89 @@ public class RawLocalFileSystem extends FileSystem {
|
|||
public IOStatistics getIOStatistics() {
|
||||
return ioStatistics;
|
||||
}
|
||||
|
||||
AsynchronousFileChannel getAsyncChannel() throws IOException {
|
||||
if (asyncChannel == null) {
|
||||
synchronized (this) {
|
||||
asyncChannel = AsynchronousFileChannel.open(name.toPath(),
|
||||
StandardOpenOption.READ);
|
||||
}
|
||||
}
|
||||
return asyncChannel;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readVectored(List<? extends FileRange> ranges,
|
||||
IntFunction<ByteBuffer> allocate) throws IOException {
|
||||
|
||||
List<? extends FileRange> sortedRanges = Arrays.asList(sortRanges(ranges));
|
||||
// Set up all of the futures, so that we can use them if things fail
|
||||
for(FileRange range: sortedRanges) {
|
||||
VectoredReadUtils.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);
|
||||
for(FileRange range: sortedRanges) {
|
||||
range.getData().completeExceptionally(ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A CompletionHandler that implements readFully and translates back
|
||||
* into the form of CompletionHandler that our users expect.
|
||||
*/
|
||||
static class AsyncHandler implements CompletionHandler<Integer, Integer> {
|
||||
private final AsynchronousFileChannel channel;
|
||||
private final List<? extends FileRange> ranges;
|
||||
private final ByteBuffer[] buffers;
|
||||
|
||||
AsyncHandler(AsynchronousFileChannel channel,
|
||||
List<? extends FileRange> ranges,
|
||||
ByteBuffer[] buffers) {
|
||||
this.channel = channel;
|
||||
this.ranges = ranges;
|
||||
this.buffers = buffers;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completed(Integer result, Integer r) {
|
||||
FileRange range = ranges.get(r);
|
||||
ByteBuffer buffer = buffers[r];
|
||||
if (result == -1) {
|
||||
failed(new EOFException("Read past End of File"), r);
|
||||
} 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);
|
||||
} else {
|
||||
// QQ: Why is this required? I think because we don't want the
|
||||
// user to read data beyond limit.
|
||||
buffer.flip();
|
||||
range.getData().complete(buffer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failed(Throwable exc, Integer r) {
|
||||
LOG.debug("Failed while reading range {} ", r, exc);
|
||||
ranges.get(r).getData().completeExceptionally(exc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
|
||||
getFileStatus(f);
|
||||
|
|
|
@ -80,6 +80,12 @@ public interface StreamCapabilities {
|
|||
*/
|
||||
String IOSTATISTICS = "iostatistics";
|
||||
|
||||
/**
|
||||
* Support for vectored IO api.
|
||||
* See {@code PositionedReadable#readVectored(List, IntFunction)}.
|
||||
*/
|
||||
String VECTOREDIO = "readvectored";
|
||||
|
||||
/**
|
||||
* Stream abort() capability implemented by {@link Abortable#abort()}.
|
||||
* This matches the Path Capability
|
||||
|
|
|
@ -0,0 +1,292 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.apache.hadoop.fs.impl.CombinedFileRange;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
|
||||
/**
|
||||
* Utility class which implements helper methods used
|
||||
* in vectored IO implementation.
|
||||
*/
|
||||
public final class VectoredReadUtils {
|
||||
|
||||
/**
|
||||
* Validate a single range.
|
||||
* @param range file range.
|
||||
* @throws EOFException any EOF Exception.
|
||||
*/
|
||||
public static void validateRangeRequest(FileRange range)
|
||||
throws EOFException {
|
||||
|
||||
Preconditions.checkArgument(range.getLength() >= 0, "length is negative");
|
||||
if (range.getOffset() < 0) {
|
||||
throw new EOFException("position is negative");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate a list of vectored read ranges.
|
||||
* @param ranges list of ranges.
|
||||
* @throws EOFException any EOF exception.
|
||||
*/
|
||||
public static void validateVectoredReadRanges(List<? extends FileRange> ranges)
|
||||
throws EOFException {
|
||||
for (FileRange range : ranges) {
|
||||
validateRangeRequest(range);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* This is the default implementation which iterates through the ranges
|
||||
* to read each synchronously, but the intent is that subclasses
|
||||
* can make more efficient readers.
|
||||
* The data or exceptions are pushed into {@link FileRange#getData()}.
|
||||
* @param stream the stream to read the data from
|
||||
* @param ranges the byte ranges to read
|
||||
* @param allocate the byte buffer allocation
|
||||
*/
|
||||
public static void readVectored(PositionedReadable stream,
|
||||
List<? extends FileRange> ranges,
|
||||
IntFunction<ByteBuffer> allocate) {
|
||||
for (FileRange range: ranges) {
|
||||
range.setData(readRangeFrom(stream, range, allocate));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Synchronously reads a range from the stream dealing with the combinations
|
||||
* of ByteBuffers buffers and PositionedReadable streams.
|
||||
* @param stream the stream to read from
|
||||
* @param range the range to read
|
||||
* @param allocate the function to allocate ByteBuffers
|
||||
* @return the CompletableFuture that contains the read data
|
||||
*/
|
||||
public static CompletableFuture<ByteBuffer> readRangeFrom(PositionedReadable stream,
|
||||
FileRange range,
|
||||
IntFunction<ByteBuffer> allocate) {
|
||||
CompletableFuture<ByteBuffer> result = new CompletableFuture<>();
|
||||
try {
|
||||
ByteBuffer buffer = allocate.apply(range.getLength());
|
||||
if (stream instanceof ByteBufferPositionedReadable) {
|
||||
((ByteBufferPositionedReadable) stream).readFully(range.getOffset(),
|
||||
buffer);
|
||||
buffer.flip();
|
||||
} else {
|
||||
readNonByteBufferPositionedReadable(stream, range, buffer);
|
||||
}
|
||||
result.complete(buffer);
|
||||
} catch (IOException ioe) {
|
||||
result.completeExceptionally(ioe);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private static void readNonByteBufferPositionedReadable(PositionedReadable stream,
|
||||
FileRange range,
|
||||
ByteBuffer buffer) throws IOException {
|
||||
if (buffer.isDirect()) {
|
||||
buffer.put(readInDirectBuffer(stream, range));
|
||||
buffer.flip();
|
||||
} else {
|
||||
stream.readFully(range.getOffset(), buffer.array(),
|
||||
buffer.arrayOffset(), range.getLength());
|
||||
}
|
||||
}
|
||||
|
||||
private static byte[] readInDirectBuffer(PositionedReadable stream,
|
||||
FileRange range) throws IOException {
|
||||
// if we need to read data from a direct buffer and the stream doesn't
|
||||
// support it, we allocate a byte array to use.
|
||||
byte[] tmp = new byte[range.getLength()];
|
||||
stream.readFully(range.getOffset(), tmp, 0, tmp.length);
|
||||
return tmp;
|
||||
}
|
||||
|
||||
/**
|
||||
* Is the given input list.
|
||||
* <ul>
|
||||
* <li>already sorted by offset</li>
|
||||
* <li>each range is more than minimumSeek apart</li>
|
||||
* <li>the start and end of each range is a multiple of chunkSize</li>
|
||||
* </ul>
|
||||
*
|
||||
* @param input the list of input ranges.
|
||||
* @param chunkSize the size of the chunks that the offset and end must align to.
|
||||
* @param minimumSeek the minimum distance between ranges.
|
||||
* @return true if we can use the input list as is.
|
||||
*/
|
||||
public static boolean isOrderedDisjoint(List<? extends FileRange> input,
|
||||
int chunkSize,
|
||||
int minimumSeek) {
|
||||
long previous = -minimumSeek;
|
||||
for (FileRange range: input) {
|
||||
long offset = range.getOffset();
|
||||
long end = range.getOffset() + range.getLength();
|
||||
if (offset % chunkSize != 0 ||
|
||||
end % chunkSize != 0 ||
|
||||
(offset - previous < minimumSeek)) {
|
||||
return false;
|
||||
}
|
||||
previous = end;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculates floor value of offset based on chunk size.
|
||||
* @param offset file offset.
|
||||
* @param chunkSize file chunk size.
|
||||
* @return floor value.
|
||||
*/
|
||||
public static long roundDown(long offset, int chunkSize) {
|
||||
if (chunkSize > 1) {
|
||||
return offset - (offset % chunkSize);
|
||||
} else {
|
||||
return offset;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculates the ceil value of offset based on chunk size.
|
||||
* @param offset file offset.
|
||||
* @param chunkSize file chunk size.
|
||||
* @return ceil value.
|
||||
*/
|
||||
public static long roundUp(long offset, int chunkSize) {
|
||||
if (chunkSize > 1) {
|
||||
long next = offset + chunkSize - 1;
|
||||
return next - (next % chunkSize);
|
||||
} else {
|
||||
return offset;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the input ranges are overlapping in nature.
|
||||
* We call two ranges to be overlapping when start offset
|
||||
* of second is less than the end offset of first.
|
||||
* End offset is calculated as start offset + length.
|
||||
* @param input list if input ranges.
|
||||
* @return true/false based on logic explained above.
|
||||
*/
|
||||
public static List<? extends FileRange> validateNonOverlappingAndReturnSortedRanges(
|
||||
List<? extends FileRange> input) {
|
||||
|
||||
if (input.size() <= 1) {
|
||||
return input;
|
||||
}
|
||||
FileRange[] sortedRanges = sortRanges(input);
|
||||
FileRange prev = sortedRanges[0];
|
||||
for (int i=1; i<sortedRanges.length; i++) {
|
||||
if (sortedRanges[i].getOffset() < prev.getOffset() + prev.getLength()) {
|
||||
throw new UnsupportedOperationException("Overlapping ranges are not supported");
|
||||
}
|
||||
}
|
||||
return Arrays.asList(sortedRanges);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the input ranges by offset.
|
||||
* @param input input ranges.
|
||||
* @return sorted ranges.
|
||||
*/
|
||||
public static FileRange[] sortRanges(List<? extends FileRange> input) {
|
||||
FileRange[] sortedRanges = input.toArray(new FileRange[0]);
|
||||
Arrays.sort(sortedRanges, Comparator.comparingLong(FileRange::getOffset));
|
||||
return sortedRanges;
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge sorted ranges to optimize the access from the underlying file
|
||||
* system.
|
||||
* The motivations are that:
|
||||
* <ul>
|
||||
* <li>Upper layers want to pass down logical file ranges.</li>
|
||||
* <li>Fewer reads have better performance.</li>
|
||||
* <li>Applications want callbacks as ranges are read.</li>
|
||||
* <li>Some file systems want to round ranges to be at checksum boundaries.</li>
|
||||
* </ul>
|
||||
*
|
||||
* @param sortedRanges already sorted list of ranges based on offset.
|
||||
* @param chunkSize round the start and end points to multiples of chunkSize
|
||||
* @param minimumSeek the smallest gap that we should seek over in bytes
|
||||
* @param maxSize the largest combined file range in bytes
|
||||
* @return the list of sorted CombinedFileRanges that cover the input
|
||||
*/
|
||||
public static List<CombinedFileRange> mergeSortedRanges(List<? extends FileRange> sortedRanges,
|
||||
int chunkSize,
|
||||
int minimumSeek,
|
||||
int maxSize) {
|
||||
|
||||
CombinedFileRange current = null;
|
||||
List<CombinedFileRange> result = new ArrayList<>(sortedRanges.size());
|
||||
|
||||
// now merge together the ones that merge
|
||||
for (FileRange range: sortedRanges) {
|
||||
long start = roundDown(range.getOffset(), chunkSize);
|
||||
long end = roundUp(range.getOffset() + range.getLength(), chunkSize);
|
||||
if (current == null || !current.merge(start, end, range, minimumSeek, maxSize)) {
|
||||
current = new CombinedFileRange(start, end, range);
|
||||
result.add(current);
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Slice the data that was read to the user's request.
|
||||
* This function assumes that the user's request is completely subsumed by the
|
||||
* read data. This always creates a new buffer pointing to the same underlying
|
||||
* data but with its own mark and position fields such that reading one buffer
|
||||
* can't effect other's mark and position.
|
||||
* @param readData the buffer with the readData
|
||||
* @param readOffset the offset in the file for the readData
|
||||
* @param request the user's request
|
||||
* @return the readData buffer that is sliced to the user's request
|
||||
*/
|
||||
public static ByteBuffer sliceTo(ByteBuffer readData, long readOffset,
|
||||
FileRange request) {
|
||||
int offsetChange = (int) (request.getOffset() - readOffset);
|
||||
int requestLength = request.getLength();
|
||||
readData = readData.slice();
|
||||
readData.position(offsetChange);
|
||||
readData.limit(offsetChange + requestLength);
|
||||
return readData;
|
||||
}
|
||||
|
||||
/**
|
||||
* private constructor.
|
||||
*/
|
||||
private VectoredReadUtils() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,70 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.impl;
|
||||
|
||||
import org.apache.hadoop.fs.FileRange;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A file range that represents a set of underlying file ranges.
|
||||
* This is used when we combine the user's FileRange objects
|
||||
* together into a single read for efficiency.
|
||||
*/
|
||||
public class CombinedFileRange extends FileRangeImpl {
|
||||
private ArrayList<FileRange> underlying = new ArrayList<>();
|
||||
|
||||
public CombinedFileRange(long offset, long end, FileRange original) {
|
||||
super(offset, (int) (end - offset));
|
||||
this.underlying.add(original);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of ranges that were merged together to form this one.
|
||||
* @return the list of input ranges
|
||||
*/
|
||||
public List<FileRange> getUnderlying() {
|
||||
return underlying;
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge this input range into the current one, if it is compatible.
|
||||
* It is assumed that otherOffset is greater or equal the current offset,
|
||||
* which typically happens by sorting the input ranges on offset.
|
||||
* @param otherOffset the offset to consider merging
|
||||
* @param otherEnd the end to consider merging
|
||||
* @param other the underlying FileRange to add if we merge
|
||||
* @param minSeek the minimum distance that we'll seek without merging the
|
||||
* ranges together
|
||||
* @param maxSize the maximum size that we'll merge into a single range
|
||||
* @return true if we have merged the range into this one
|
||||
*/
|
||||
public boolean merge(long otherOffset, long otherEnd, FileRange other,
|
||||
int minSeek, int maxSize) {
|
||||
long end = this.getOffset() + this.getLength();
|
||||
long newEnd = Math.max(end, otherEnd);
|
||||
if (otherOffset - end >= minSeek || newEnd - this.getOffset() > maxSize) {
|
||||
return false;
|
||||
}
|
||||
this.setLength((int) (newEnd - this.getOffset()));
|
||||
underlying.add(other);
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.fs.impl;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.FileRange;
|
||||
|
||||
/**
|
||||
* A range of bytes from a file with an optional buffer to read those bytes
|
||||
* for zero copy. This shouldn't be created directly via constructor rather
|
||||
* factory defined in {@code FileRange#createFileRange} should be used.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class FileRangeImpl implements FileRange {
|
||||
private long offset;
|
||||
private int length;
|
||||
private CompletableFuture<ByteBuffer> reader;
|
||||
|
||||
public FileRangeImpl(long offset, int length) {
|
||||
this.offset = offset;
|
||||
this.length = length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "range[" + offset + "," + (offset + length) + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getOffset() {
|
||||
return offset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getLength() {
|
||||
return length;
|
||||
}
|
||||
|
||||
public void setOffset(long offset) {
|
||||
this.offset = offset;
|
||||
}
|
||||
|
||||
public void setLength(int length) {
|
||||
this.length = length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setData(CompletableFuture<ByteBuffer> pReader) {
|
||||
this.reader = pReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompletableFuture<ByteBuffer> getData() {
|
||||
return reader;
|
||||
}
|
||||
}
|
|
@ -45,4 +45,9 @@ public interface ByteBufferPool {
|
|||
* @param buffer a direct bytebuffer
|
||||
*/
|
||||
void putBuffer(ByteBuffer buffer);
|
||||
|
||||
/**
|
||||
* Clear the buffer pool thus releasing all the buffers.
|
||||
*/
|
||||
default void release() { }
|
||||
}
|
||||
|
|
|
@ -36,8 +36,8 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Stable
|
||||
public final class ElasticByteBufferPool implements ByteBufferPool {
|
||||
private static final class Key implements Comparable<Key> {
|
||||
public class ElasticByteBufferPool implements ByteBufferPool {
|
||||
protected static final class Key implements Comparable<Key> {
|
||||
private final int capacity;
|
||||
private final long insertionTime;
|
||||
|
||||
|
|
|
@ -0,0 +1,155 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.io;
|
||||
|
||||
import java.lang.ref.WeakReference;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Buffer pool implementation which uses weak references to store
|
||||
* buffers in the pool, such that they are garbage collected when
|
||||
* there are no references to the buffer during a gc run. This is
|
||||
* important as direct buffers don't get garbage collected automatically
|
||||
* during a gc run as they are not stored on heap memory.
|
||||
* Also the buffers are stored in a tree map which helps in returning
|
||||
* smallest buffer whose size is just greater than requested length.
|
||||
* This is a thread safe implementation.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public final class WeakReferencedElasticByteBufferPool extends ElasticByteBufferPool {
|
||||
|
||||
/**
|
||||
* Map to store direct byte buffers of different sizes in the pool.
|
||||
* Used tree map such that we can return next greater than capacity
|
||||
* buffer if buffer with exact capacity is unavailable.
|
||||
* This must be accessed in synchronized blocks.
|
||||
*/
|
||||
private final TreeMap<Key, WeakReference<ByteBuffer>> directBuffers =
|
||||
new TreeMap<>();
|
||||
|
||||
/**
|
||||
* Map to store heap based byte buffers of different sizes in the pool.
|
||||
* Used tree map such that we can return next greater than capacity
|
||||
* buffer if buffer with exact capacity is unavailable.
|
||||
* This must be accessed in synchronized blocks.
|
||||
*/
|
||||
private final TreeMap<Key, WeakReference<ByteBuffer>> heapBuffers =
|
||||
new TreeMap<>();
|
||||
|
||||
/**
|
||||
* Method to get desired buffer tree.
|
||||
* @param isDirect whether the buffer is heap based or direct.
|
||||
* @return corresponding buffer tree.
|
||||
*/
|
||||
private TreeMap<Key, WeakReference<ByteBuffer>> getBufferTree(boolean isDirect) {
|
||||
return isDirect
|
||||
? directBuffers
|
||||
: heapBuffers;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*
|
||||
* @param direct whether we want a direct byte buffer or a heap one.
|
||||
* @param length length of requested buffer.
|
||||
* @return returns equal or next greater than capacity buffer from
|
||||
* pool if already available and not garbage collected else creates
|
||||
* a new buffer and return it.
|
||||
*/
|
||||
@Override
|
||||
public synchronized ByteBuffer getBuffer(boolean direct, int length) {
|
||||
TreeMap<Key, WeakReference<ByteBuffer>> buffersTree = getBufferTree(direct);
|
||||
|
||||
// Scan the entire tree and remove all weak null references.
|
||||
buffersTree.entrySet().removeIf(next -> next.getValue().get() == null);
|
||||
|
||||
Map.Entry<Key, WeakReference<ByteBuffer>> entry =
|
||||
buffersTree.ceilingEntry(new Key(length, 0));
|
||||
// If there is no buffer present in the pool with desired size.
|
||||
if (entry == null) {
|
||||
return direct ? ByteBuffer.allocateDirect(length) :
|
||||
ByteBuffer.allocate(length);
|
||||
}
|
||||
// buffer is available in the pool and not garbage collected.
|
||||
WeakReference<ByteBuffer> bufferInPool = entry.getValue();
|
||||
buffersTree.remove(entry.getKey());
|
||||
ByteBuffer buffer = bufferInPool.get();
|
||||
if (buffer != null) {
|
||||
return buffer;
|
||||
}
|
||||
// buffer was in pool but already got garbage collected.
|
||||
return direct
|
||||
? ByteBuffer.allocateDirect(length)
|
||||
: ByteBuffer.allocate(length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return buffer to the pool.
|
||||
* @param buffer buffer to be returned.
|
||||
*/
|
||||
@Override
|
||||
public synchronized void putBuffer(ByteBuffer buffer) {
|
||||
buffer.clear();
|
||||
TreeMap<Key, WeakReference<ByteBuffer>> buffersTree = getBufferTree(buffer.isDirect());
|
||||
// Buffers are indexed by (capacity, time).
|
||||
// If our key is not unique on the first try, we try again, since the
|
||||
// time will be different. Since we use nanoseconds, it's pretty
|
||||
// unlikely that we'll loop even once, unless the system clock has a
|
||||
// poor granularity or multi-socket systems have clocks slightly out
|
||||
// of sync.
|
||||
while (true) {
|
||||
Key keyToInsert = new Key(buffer.capacity(), System.nanoTime());
|
||||
if (!buffersTree.containsKey(keyToInsert)) {
|
||||
buffersTree.put(keyToInsert, new WeakReference<>(buffer));
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear the buffer pool thus releasing all the buffers.
|
||||
* The caller must remove all references of
|
||||
* existing buffers before calling this method to avoid
|
||||
* memory leaks.
|
||||
*/
|
||||
@Override
|
||||
public synchronized void release() {
|
||||
heapBuffers.clear();
|
||||
directBuffers.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get current buffers count in the pool.
|
||||
* @param isDirect whether we want to count the heap or direct buffers.
|
||||
* @return count of buffers.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public synchronized int getCurrentBuffersCount(boolean isDirect) {
|
||||
return isDirect
|
||||
? directBuffers.size()
|
||||
: heapBuffers.size();
|
||||
}
|
||||
}
|
|
@ -443,6 +443,45 @@ The semantics of this are exactly equivalent to
|
|||
That is, the buffer is filled entirely with the contents of the input source
|
||||
from position `position`
|
||||
|
||||
### `default void readVectored(List<? extends FileRange> ranges, IntFunction<ByteBuffer> allocate)`
|
||||
|
||||
Read fully data for a list of ranges asynchronously. The default implementation
|
||||
iterates through the ranges, tries to coalesce the ranges based on values of
|
||||
`minSeekForVectorReads` and `maxReadSizeForVectorReads` and then read each merged
|
||||
ranges synchronously, but the intent is sub classes can implement efficient
|
||||
implementation. Reading in both direct and heap byte buffers are supported.
|
||||
Also, clients are encouraged to use `WeakReferencedElasticByteBufferPool` for
|
||||
allocating buffers such that even direct buffers are garbage collected when
|
||||
they are no longer referenced.
|
||||
|
||||
Note: Don't use direct buffers for reading from ChecksumFileSystem as that may
|
||||
lead to memory fragmentation explained in HADOOP-18296.
|
||||
|
||||
|
||||
#### Preconditions
|
||||
|
||||
For each requested range:
|
||||
|
||||
range.getOffset >= 0 else raise IllegalArgumentException
|
||||
range.getLength >= 0 else raise EOFException
|
||||
|
||||
#### Postconditions
|
||||
|
||||
For each requested range:
|
||||
|
||||
range.getData() returns CompletableFuture<ByteBuffer> which will have data
|
||||
from range.getOffset to range.getLength.
|
||||
|
||||
### `minSeekForVectorReads()`
|
||||
|
||||
The smallest reasonable seek. Two ranges won't be merged together if the difference between
|
||||
end of first and start of next range is more than this value.
|
||||
|
||||
### `maxReadSizeForVectorReads()`
|
||||
|
||||
Maximum number of bytes which can be read in one go after merging the ranges.
|
||||
Two ranges won't be merged if the combined data to be read is more than this value.
|
||||
Essentially setting this to 0 will disable the merging of ranges.
|
||||
|
||||
## Consistency
|
||||
|
||||
|
|
|
@ -0,0 +1,371 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.IntBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Test;
|
||||
import org.mockito.ArgumentMatchers;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import org.apache.hadoop.fs.impl.CombinedFileRange;
|
||||
import org.apache.hadoop.test.HadoopTestBase;
|
||||
|
||||
import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges;
|
||||
import static org.apache.hadoop.test.MoreAsserts.assertFutureCompletedSuccessfully;
|
||||
import static org.apache.hadoop.test.MoreAsserts.assertFutureFailedExceptionally;
|
||||
|
||||
/**
|
||||
* Test behavior of {@link VectoredReadUtils}.
|
||||
*/
|
||||
public class TestVectoredReadUtils extends HadoopTestBase {
|
||||
|
||||
@Test
|
||||
public void testSliceTo() {
|
||||
final int size = 64 * 1024;
|
||||
ByteBuffer buffer = ByteBuffer.allocate(size);
|
||||
// fill the buffer with data
|
||||
IntBuffer intBuffer = buffer.asIntBuffer();
|
||||
for(int i=0; i < size / Integer.BYTES; ++i) {
|
||||
intBuffer.put(i);
|
||||
}
|
||||
// ensure we don't make unnecessary slices
|
||||
ByteBuffer slice = VectoredReadUtils.sliceTo(buffer, 100,
|
||||
FileRange.createFileRange(100, size));
|
||||
Assertions.assertThat(buffer)
|
||||
.describedAs("Slicing on the same offset shouldn't " +
|
||||
"create a new buffer")
|
||||
.isEqualTo(slice);
|
||||
|
||||
// try slicing a range
|
||||
final int offset = 100;
|
||||
final int sliceStart = 1024;
|
||||
final int sliceLength = 16 * 1024;
|
||||
slice = VectoredReadUtils.sliceTo(buffer, offset,
|
||||
FileRange.createFileRange(offset + sliceStart, sliceLength));
|
||||
// make sure they aren't the same, but use the same backing data
|
||||
Assertions.assertThat(buffer)
|
||||
.describedAs("Slicing on new offset should " +
|
||||
"create a new buffer")
|
||||
.isNotEqualTo(slice);
|
||||
Assertions.assertThat(buffer.array())
|
||||
.describedAs("Slicing should use the same underlying " +
|
||||
"data")
|
||||
.isEqualTo(slice.array());
|
||||
// test the contents of the slice
|
||||
intBuffer = slice.asIntBuffer();
|
||||
for(int i=0; i < sliceLength / Integer.BYTES; ++i) {
|
||||
assertEquals("i = " + i, i + sliceStart / Integer.BYTES, intBuffer.get());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRounding() {
|
||||
for(int i=5; i < 10; ++i) {
|
||||
assertEquals("i = "+ i, 5, VectoredReadUtils.roundDown(i, 5));
|
||||
assertEquals("i = "+ i, 10, VectoredReadUtils.roundUp(i+1, 5));
|
||||
}
|
||||
assertEquals("Error while roundDown", 13, VectoredReadUtils.roundDown(13, 1));
|
||||
assertEquals("Error while roundUp", 13, VectoredReadUtils.roundUp(13, 1));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMerge() {
|
||||
FileRange base = FileRange.createFileRange(2000, 1000);
|
||||
CombinedFileRange mergeBase = new CombinedFileRange(2000, 3000, base);
|
||||
|
||||
// test when the gap between is too big
|
||||
assertFalse("Large gap ranges shouldn't get merged", mergeBase.merge(5000, 6000,
|
||||
FileRange.createFileRange(5000, 1000), 2000, 4000));
|
||||
assertEquals("Number of ranges in merged range shouldn't increase",
|
||||
1, mergeBase.getUnderlying().size());
|
||||
assertEquals("post merge offset", 2000, mergeBase.getOffset());
|
||||
assertEquals("post merge length", 1000, mergeBase.getLength());
|
||||
|
||||
// test when the total size gets exceeded
|
||||
assertFalse("Large size ranges shouldn't get merged", mergeBase.merge(5000, 6000,
|
||||
FileRange.createFileRange(5000, 1000), 2001, 3999));
|
||||
assertEquals("Number of ranges in merged range shouldn't increase",
|
||||
1, mergeBase.getUnderlying().size());
|
||||
assertEquals("post merge offset", 2000, mergeBase.getOffset());
|
||||
assertEquals("post merge length", 1000, mergeBase.getLength());
|
||||
|
||||
// test when the merge works
|
||||
assertTrue("ranges should get merged ", mergeBase.merge(5000, 6000,
|
||||
FileRange.createFileRange(5000, 1000), 2001, 4000));
|
||||
assertEquals("post merge size", 2, mergeBase.getUnderlying().size());
|
||||
assertEquals("post merge offset", 2000, mergeBase.getOffset());
|
||||
assertEquals("post merge length", 4000, mergeBase.getLength());
|
||||
|
||||
// reset the mergeBase and test with a 10:1 reduction
|
||||
mergeBase = new CombinedFileRange(200, 300, base);
|
||||
assertEquals(200, mergeBase.getOffset());
|
||||
assertEquals(100, mergeBase.getLength());
|
||||
assertTrue("ranges should get merged ", mergeBase.merge(500, 600,
|
||||
FileRange.createFileRange(5000, 1000), 201, 400));
|
||||
assertEquals("post merge size", 2, mergeBase.getUnderlying().size());
|
||||
assertEquals("post merge offset", 200, mergeBase.getOffset());
|
||||
assertEquals("post merge length", 400, mergeBase.getLength());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSortAndMerge() {
|
||||
List<FileRange> input = Arrays.asList(
|
||||
FileRange.createFileRange(3000, 100),
|
||||
FileRange.createFileRange(2100, 100),
|
||||
FileRange.createFileRange(1000, 100)
|
||||
);
|
||||
assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
|
||||
List<CombinedFileRange> outputList = VectoredReadUtils.mergeSortedRanges(
|
||||
Arrays.asList(sortRanges(input)), 100, 1001, 2500);
|
||||
Assertions.assertThat(outputList)
|
||||
.describedAs("merged range size")
|
||||
.hasSize(1);
|
||||
CombinedFileRange output = outputList.get(0);
|
||||
Assertions.assertThat(output.getUnderlying())
|
||||
.describedAs("merged range underlying size")
|
||||
.hasSize(3);
|
||||
assertEquals("range[1000,3100)", output.toString());
|
||||
assertTrue("merged output ranges are disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(outputList, 100, 800));
|
||||
|
||||
// the minSeek doesn't allow the first two to merge
|
||||
assertFalse("Ranges are non disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(input, 100, 1000));
|
||||
outputList = VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(input)),
|
||||
100, 1000, 2100);
|
||||
Assertions.assertThat(outputList)
|
||||
.describedAs("merged range size")
|
||||
.hasSize(2);
|
||||
assertEquals("range[1000,1100)", outputList.get(0).toString());
|
||||
assertEquals("range[2100,3100)", outputList.get(1).toString());
|
||||
assertTrue("merged output ranges are disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(outputList, 100, 1000));
|
||||
|
||||
// the maxSize doesn't allow the third range to merge
|
||||
assertFalse("Ranges are non disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
|
||||
outputList = VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(input)),
|
||||
100, 1001, 2099);
|
||||
Assertions.assertThat(outputList)
|
||||
.describedAs("merged range size")
|
||||
.hasSize(2);
|
||||
assertEquals("range[1000,2200)", outputList.get(0).toString());
|
||||
assertEquals("range[3000,3100)", outputList.get(1).toString());
|
||||
assertTrue("merged output ranges are disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(outputList, 100, 800));
|
||||
|
||||
// test the round up and round down (the maxSize doesn't allow any merges)
|
||||
assertFalse("Ranges are non disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(input, 16, 700));
|
||||
outputList = VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(input)),
|
||||
16, 1001, 100);
|
||||
Assertions.assertThat(outputList)
|
||||
.describedAs("merged range size")
|
||||
.hasSize(3);
|
||||
assertEquals("range[992,1104)", outputList.get(0).toString());
|
||||
assertEquals("range[2096,2208)", outputList.get(1).toString());
|
||||
assertEquals("range[2992,3104)", outputList.get(2).toString());
|
||||
assertTrue("merged output ranges are disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(outputList, 16, 700));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSortAndMergeMoreCases() throws Exception {
|
||||
List<FileRange> input = Arrays.asList(
|
||||
FileRange.createFileRange(3000, 110),
|
||||
FileRange.createFileRange(3000, 100),
|
||||
FileRange.createFileRange(2100, 100),
|
||||
FileRange.createFileRange(1000, 100)
|
||||
);
|
||||
assertFalse("Ranges are non disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(input, 100, 800));
|
||||
List<CombinedFileRange> outputList = VectoredReadUtils.mergeSortedRanges(
|
||||
Arrays.asList(sortRanges(input)), 1, 1001, 2500);
|
||||
Assertions.assertThat(outputList)
|
||||
.describedAs("merged range size")
|
||||
.hasSize(1);
|
||||
CombinedFileRange output = outputList.get(0);
|
||||
Assertions.assertThat(output.getUnderlying())
|
||||
.describedAs("merged range underlying size")
|
||||
.hasSize(4);
|
||||
assertEquals("range[1000,3110)", output.toString());
|
||||
assertTrue("merged output ranges are disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800));
|
||||
|
||||
outputList = VectoredReadUtils.mergeSortedRanges(
|
||||
Arrays.asList(sortRanges(input)), 100, 1001, 2500);
|
||||
Assertions.assertThat(outputList)
|
||||
.describedAs("merged range size")
|
||||
.hasSize(1);
|
||||
output = outputList.get(0);
|
||||
Assertions.assertThat(output.getUnderlying())
|
||||
.describedAs("merged range underlying size")
|
||||
.hasSize(4);
|
||||
assertEquals("range[1000,3200)", output.toString());
|
||||
assertTrue("merged output ranges are disjoint",
|
||||
VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaxSizeZeroDisablesMering() throws Exception {
|
||||
List<FileRange> randomRanges = Arrays.asList(
|
||||
FileRange.createFileRange(3000, 110),
|
||||
FileRange.createFileRange(3000, 100),
|
||||
FileRange.createFileRange(2100, 100)
|
||||
);
|
||||
assertEqualRangeCountsAfterMerging(randomRanges, 1, 1, 0);
|
||||
assertEqualRangeCountsAfterMerging(randomRanges, 1, 0, 0);
|
||||
assertEqualRangeCountsAfterMerging(randomRanges, 1, 100, 0);
|
||||
}
|
||||
|
||||
private void assertEqualRangeCountsAfterMerging(List<FileRange> inputRanges,
|
||||
int chunkSize,
|
||||
int minimumSeek,
|
||||
int maxSize) {
|
||||
List<CombinedFileRange> combinedFileRanges = VectoredReadUtils
|
||||
.mergeSortedRanges(inputRanges, chunkSize, minimumSeek, maxSize);
|
||||
Assertions.assertThat(combinedFileRanges)
|
||||
.describedAs("Mismatch in number of ranges post merging")
|
||||
.hasSize(inputRanges.size());
|
||||
}
|
||||
|
||||
interface Stream extends PositionedReadable, ByteBufferPositionedReadable {
|
||||
// nothing
|
||||
}
|
||||
|
||||
static void fillBuffer(ByteBuffer buffer) {
|
||||
byte b = 0;
|
||||
while (buffer.remaining() > 0) {
|
||||
buffer.put(b++);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadRangeFromByteBufferPositionedReadable() throws Exception {
|
||||
Stream stream = Mockito.mock(Stream.class);
|
||||
Mockito.doAnswer(invocation -> {
|
||||
fillBuffer(invocation.getArgument(1));
|
||||
return null;
|
||||
}).when(stream).readFully(ArgumentMatchers.anyLong(),
|
||||
ArgumentMatchers.any(ByteBuffer.class));
|
||||
CompletableFuture<ByteBuffer> result =
|
||||
VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100),
|
||||
ByteBuffer::allocate);
|
||||
assertFutureCompletedSuccessfully(result);
|
||||
ByteBuffer buffer = result.get();
|
||||
assertEquals("Size of result buffer", 100, buffer.remaining());
|
||||
byte b = 0;
|
||||
while (buffer.remaining() > 0) {
|
||||
assertEquals("remain = " + buffer.remaining(), b++, buffer.get());
|
||||
}
|
||||
|
||||
// test an IOException
|
||||
Mockito.reset(stream);
|
||||
Mockito.doThrow(new IOException("foo"))
|
||||
.when(stream).readFully(ArgumentMatchers.anyLong(),
|
||||
ArgumentMatchers.any(ByteBuffer.class));
|
||||
result =
|
||||
VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100),
|
||||
ByteBuffer::allocate);
|
||||
assertFutureFailedExceptionally(result);
|
||||
}
|
||||
|
||||
static void runReadRangeFromPositionedReadable(IntFunction<ByteBuffer> allocate)
|
||||
throws Exception {
|
||||
PositionedReadable stream = Mockito.mock(PositionedReadable.class);
|
||||
Mockito.doAnswer(invocation -> {
|
||||
byte b=0;
|
||||
byte[] buffer = invocation.getArgument(1);
|
||||
for(int i=0; i < buffer.length; ++i) {
|
||||
buffer[i] = b++;
|
||||
}
|
||||
return null;
|
||||
}).when(stream).readFully(ArgumentMatchers.anyLong(),
|
||||
ArgumentMatchers.any(), ArgumentMatchers.anyInt(),
|
||||
ArgumentMatchers.anyInt());
|
||||
CompletableFuture<ByteBuffer> result =
|
||||
VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100),
|
||||
allocate);
|
||||
assertFutureCompletedSuccessfully(result);
|
||||
ByteBuffer buffer = result.get();
|
||||
assertEquals("Size of result buffer", 100, buffer.remaining());
|
||||
byte b = 0;
|
||||
while (buffer.remaining() > 0) {
|
||||
assertEquals("remain = " + buffer.remaining(), b++, buffer.get());
|
||||
}
|
||||
|
||||
// test an IOException
|
||||
Mockito.reset(stream);
|
||||
Mockito.doThrow(new IOException("foo"))
|
||||
.when(stream).readFully(ArgumentMatchers.anyLong(),
|
||||
ArgumentMatchers.any(), ArgumentMatchers.anyInt(),
|
||||
ArgumentMatchers.anyInt());
|
||||
result =
|
||||
VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100),
|
||||
ByteBuffer::allocate);
|
||||
assertFutureFailedExceptionally(result);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadRangeArray() throws Exception {
|
||||
runReadRangeFromPositionedReadable(ByteBuffer::allocate);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadRangeDirect() throws Exception {
|
||||
runReadRangeFromPositionedReadable(ByteBuffer::allocateDirect);
|
||||
}
|
||||
|
||||
static void validateBuffer(String message, ByteBuffer buffer, int start) {
|
||||
byte expected = (byte) start;
|
||||
while (buffer.remaining() > 0) {
|
||||
assertEquals(message + " remain: " + buffer.remaining(), expected++,
|
||||
buffer.get());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadVectored() throws Exception {
|
||||
List<FileRange> input = Arrays.asList(FileRange.createFileRange(0, 100),
|
||||
FileRange.createFileRange(100_000, 100),
|
||||
FileRange.createFileRange(200_000, 100));
|
||||
Stream stream = Mockito.mock(Stream.class);
|
||||
Mockito.doAnswer(invocation -> {
|
||||
fillBuffer(invocation.getArgument(1));
|
||||
return null;
|
||||
}).when(stream).readFully(ArgumentMatchers.anyLong(),
|
||||
ArgumentMatchers.any(ByteBuffer.class));
|
||||
// should not merge the ranges
|
||||
VectoredReadUtils.readVectored(stream, input, ByteBuffer::allocate);
|
||||
Mockito.verify(stream, Mockito.times(3))
|
||||
.readFully(ArgumentMatchers.anyLong(), ArgumentMatchers.any(ByteBuffer.class));
|
||||
for(int b=0; b < input.size(); ++b) {
|
||||
validateBuffer("buffer " + b, input.get(b).getData().get(), 0);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,406 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.contract;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileRange;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.StreamCapabilities;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.impl.FutureIOSupport;
|
||||
import org.apache.hadoop.io.WeakReferencedElasticByteBufferPool;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertCapabilities;
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDatasetEquals;
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead;
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public abstract class AbstractContractVectoredReadTest extends AbstractFSContractTestBase {
|
||||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(AbstractContractVectoredReadTest.class);
|
||||
|
||||
public static final int DATASET_LEN = 64 * 1024;
|
||||
protected static final byte[] DATASET = ContractTestUtils.dataset(DATASET_LEN, 'a', 32);
|
||||
protected static final String VECTORED_READ_FILE_NAME = "vectored_file.txt";
|
||||
|
||||
private final IntFunction<ByteBuffer> allocate;
|
||||
|
||||
private final WeakReferencedElasticByteBufferPool pool =
|
||||
new WeakReferencedElasticByteBufferPool();
|
||||
|
||||
private final String bufferType;
|
||||
|
||||
@Parameterized.Parameters(name = "Buffer type : {0}")
|
||||
public static List<String> params() {
|
||||
return Arrays.asList("direct", "array");
|
||||
}
|
||||
|
||||
public AbstractContractVectoredReadTest(String bufferType) {
|
||||
this.bufferType = bufferType;
|
||||
this.allocate = value -> {
|
||||
boolean isDirect = !"array".equals(bufferType);
|
||||
return pool.getBuffer(isDirect, value);
|
||||
};
|
||||
}
|
||||
|
||||
public IntFunction<ByteBuffer> getAllocate() {
|
||||
return allocate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setup() throws Exception {
|
||||
super.setup();
|
||||
Path path = path(VECTORED_READ_FILE_NAME);
|
||||
FileSystem fs = getFileSystem();
|
||||
createFile(fs, path, true, DATASET);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void teardown() throws Exception {
|
||||
super.teardown();
|
||||
pool.release();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVectoredReadCapability() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
String[] vectoredReadCapability = new String[]{StreamCapabilities.VECTOREDIO};
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
assertCapabilities(in, vectoredReadCapability, null);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVectoredReadMultipleRanges() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
FileRange fileRange = FileRange.createFileRange(i * 100, 100);
|
||||
fileRanges.add(fileRange);
|
||||
}
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
CompletableFuture<?>[] completableFutures = new CompletableFuture<?>[fileRanges.size()];
|
||||
int i = 0;
|
||||
for (FileRange res : fileRanges) {
|
||||
completableFutures[i++] = res.getData();
|
||||
}
|
||||
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(completableFutures);
|
||||
combinedFuture.get();
|
||||
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVectoredReadAndReadFully() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(100, 100));
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
byte[] readFullRes = new byte[100];
|
||||
in.readFully(100, readFullRes);
|
||||
ByteBuffer vecRes = FutureIOSupport.awaitFuture(fileRanges.get(0).getData());
|
||||
Assertions.assertThat(vecRes)
|
||||
.describedAs("Result from vectored read and readFully must match")
|
||||
.isEqualByComparingTo(ByteBuffer.wrap(readFullRes));
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* As the minimum seek value is 4*1024,none of the below ranges
|
||||
* will get merged.
|
||||
*/
|
||||
@Test
|
||||
public void testDisjointRanges() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(0, 100));
|
||||
fileRanges.add(FileRange.createFileRange(4_000 + 101, 100));
|
||||
fileRanges.add(FileRange.createFileRange(16_000 + 101, 100));
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* As the minimum seek value is 4*1024, all the below ranges
|
||||
* will get merged into one.
|
||||
*/
|
||||
@Test
|
||||
public void testAllRangesMergedIntoOne() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(0, 100));
|
||||
fileRanges.add(FileRange.createFileRange(4_000 - 101, 100));
|
||||
fileRanges.add(FileRange.createFileRange(8_000 - 101, 100));
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* As the minimum seek value is 4*1024, the first three ranges will be
|
||||
* merged into and other two will remain as it is.
|
||||
*/
|
||||
@Test
|
||||
public void testSomeRangesMergedSomeUnmerged() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(8 * 1024, 100));
|
||||
fileRanges.add(FileRange.createFileRange(14 * 1024, 100));
|
||||
fileRanges.add(FileRange.createFileRange(10 * 1024, 100));
|
||||
fileRanges.add(FileRange.createFileRange(2 * 1024 - 101, 100));
|
||||
fileRanges.add(FileRange.createFileRange(40 * 1024, 1024));
|
||||
FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
|
||||
CompletableFuture<FSDataInputStream> builder =
|
||||
fs.openFile(path(VECTORED_READ_FILE_NAME))
|
||||
.withFileStatus(fileStatus)
|
||||
.build();
|
||||
try (FSDataInputStream in = builder.get()) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOverlappingRanges() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = getSampleOverlappingRanges();
|
||||
FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
|
||||
CompletableFuture<FSDataInputStream> builder =
|
||||
fs.openFile(path(VECTORED_READ_FILE_NAME))
|
||||
.withFileStatus(fileStatus)
|
||||
.build();
|
||||
try (FSDataInputStream in = builder.get()) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSameRanges() throws Exception {
|
||||
// Same ranges are special case of overlapping only.
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = getSampleSameRanges();
|
||||
CompletableFuture<FSDataInputStream> builder =
|
||||
fs.openFile(path(VECTORED_READ_FILE_NAME))
|
||||
.build();
|
||||
try (FSDataInputStream in = builder.get()) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSomeRandomNonOverlappingRanges() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(500, 100));
|
||||
fileRanges.add(FileRange.createFileRange(1000, 200));
|
||||
fileRanges.add(FileRange.createFileRange(50, 10));
|
||||
fileRanges.add(FileRange.createFileRange(10, 5));
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConsecutiveRanges() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(500, 100));
|
||||
fileRanges.add(FileRange.createFileRange(600, 200));
|
||||
fileRanges.add(FileRange.createFileRange(800, 100));
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEOFRanges() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(DATASET_LEN, 100));
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
for (FileRange res : fileRanges) {
|
||||
CompletableFuture<ByteBuffer> data = res.getData();
|
||||
try {
|
||||
ByteBuffer buffer = data.get();
|
||||
// Shouldn't reach here.
|
||||
Assert.fail("EOFException must be thrown while reading EOF");
|
||||
} catch (ExecutionException ex) {
|
||||
// ignore as expected.
|
||||
} catch (Exception ex) {
|
||||
LOG.error("Exception while running vectored read ", ex);
|
||||
Assert.fail("Exception while running vectored read " + ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNegativeLengthRange() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(0, -50));
|
||||
verifyExceptionalVectoredRead(fs, fileRanges, IllegalArgumentException.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNegativeOffsetRange() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(-1, 50));
|
||||
verifyExceptionalVectoredRead(fs, fileRanges, EOFException.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNormalReadAfterVectoredRead() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = createSampleNonOverlappingRanges();
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
in.readVectored(fileRanges, allocate);
|
||||
// read starting 200 bytes
|
||||
byte[] res = new byte[200];
|
||||
in.read(res, 0, 200);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(res);
|
||||
assertDatasetEquals(0, "normal_read", buffer, 200, DATASET);
|
||||
Assertions.assertThat(in.getPos())
|
||||
.describedAs("Vectored read shouldn't change file pointer.")
|
||||
.isEqualTo(200);
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVectoredReadAfterNormalRead() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = createSampleNonOverlappingRanges();
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
// read starting 200 bytes
|
||||
byte[] res = new byte[200];
|
||||
in.read(res, 0, 200);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(res);
|
||||
assertDatasetEquals(0, "normal_read", buffer, 200, DATASET);
|
||||
Assertions.assertThat(in.getPos())
|
||||
.describedAs("Vectored read shouldn't change file pointer.")
|
||||
.isEqualTo(200);
|
||||
in.readVectored(fileRanges, allocate);
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges, pool);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultipleVectoredReads() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges1 = createSampleNonOverlappingRanges();
|
||||
List<FileRange> fileRanges2 = createSampleNonOverlappingRanges();
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
in.readVectored(fileRanges1, allocate);
|
||||
in.readVectored(fileRanges2, allocate);
|
||||
validateVectoredReadResult(fileRanges2, DATASET);
|
||||
validateVectoredReadResult(fileRanges1, DATASET);
|
||||
returnBuffersToPoolPostRead(fileRanges1, pool);
|
||||
returnBuffersToPoolPostRead(fileRanges2, pool);
|
||||
}
|
||||
}
|
||||
|
||||
protected List<FileRange> createSampleNonOverlappingRanges() {
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(0, 100));
|
||||
fileRanges.add(FileRange.createFileRange(110, 50));
|
||||
return fileRanges;
|
||||
}
|
||||
|
||||
protected List<FileRange> getSampleSameRanges() {
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(8_000, 1000));
|
||||
fileRanges.add(FileRange.createFileRange(8_000, 1000));
|
||||
fileRanges.add(FileRange.createFileRange(8_000, 1000));
|
||||
return fileRanges;
|
||||
}
|
||||
|
||||
protected List<FileRange> getSampleOverlappingRanges() {
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(100, 500));
|
||||
fileRanges.add(FileRange.createFileRange(400, 500));
|
||||
return fileRanges;
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate that exceptions must be thrown during a vectored
|
||||
* read operation with specific input ranges.
|
||||
* @param fs FileSystem instance.
|
||||
* @param fileRanges input file ranges.
|
||||
* @param clazz type of exception expected.
|
||||
* @throws Exception any other IOE.
|
||||
*/
|
||||
protected <T extends Throwable> void verifyExceptionalVectoredRead(
|
||||
FileSystem fs,
|
||||
List<FileRange> fileRanges,
|
||||
Class<T> clazz) throws Exception {
|
||||
|
||||
CompletableFuture<FSDataInputStream> builder =
|
||||
fs.openFile(path(VECTORED_READ_FILE_NAME))
|
||||
.build();
|
||||
try (FSDataInputStream in = builder.get()) {
|
||||
LambdaTestUtils.intercept(clazz,
|
||||
() -> in.readVectored(fileRanges, allocate));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.fs.contract;
|
|||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileContext;
|
||||
import org.apache.hadoop.fs.FileRange;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
|
@ -28,8 +29,10 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.PathCapabilities;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.StreamCapabilities;
|
||||
import org.apache.hadoop.io.ByteBufferPool;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.util.functional.RemoteIterators;
|
||||
import org.apache.hadoop.util.functional.FutureIO;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.AssumptionViolatedException;
|
||||
|
@ -41,6 +44,7 @@ import java.io.FileNotFoundException;
|
|||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -51,6 +55,9 @@ import java.util.NoSuchElementException;
|
|||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
|
||||
|
@ -70,6 +77,11 @@ public class ContractTestUtils extends Assert {
|
|||
public static final String IO_CHUNK_MODULUS_SIZE = "io.chunk.modulus.size";
|
||||
public static final int DEFAULT_IO_CHUNK_MODULUS_SIZE = 128;
|
||||
|
||||
/**
|
||||
* Timeout in seconds for vectored read operation in tests : {@value}.
|
||||
*/
|
||||
public static final int VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS = 5 * 60;
|
||||
|
||||
/**
|
||||
* Assert that a property in the property set matches the expected value.
|
||||
* @param props property set
|
||||
|
@ -1097,6 +1109,78 @@ public class ContractTestUtils extends Assert {
|
|||
mismatch);
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to validate vectored read results.
|
||||
* @param fileRanges input ranges.
|
||||
* @param originalData original data.
|
||||
* @throws IOException any ioe.
|
||||
*/
|
||||
public static void validateVectoredReadResult(List<FileRange> fileRanges,
|
||||
byte[] originalData)
|
||||
throws IOException, TimeoutException {
|
||||
CompletableFuture<?>[] completableFutures = new CompletableFuture<?>[fileRanges.size()];
|
||||
int i = 0;
|
||||
for (FileRange res : fileRanges) {
|
||||
completableFutures[i++] = res.getData();
|
||||
}
|
||||
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(completableFutures);
|
||||
FutureIO.awaitFuture(combinedFuture,
|
||||
VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
|
||||
TimeUnit.SECONDS);
|
||||
|
||||
for (FileRange res : fileRanges) {
|
||||
CompletableFuture<ByteBuffer> data = res.getData();
|
||||
ByteBuffer buffer = FutureIO.awaitFuture(data,
|
||||
VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
|
||||
TimeUnit.SECONDS);
|
||||
assertDatasetEquals((int) res.getOffset(), "vecRead",
|
||||
buffer, res.getLength(), originalData);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to return buffers back to the pool once all
|
||||
* data has been read for each file range.
|
||||
* @param fileRanges list of file range.
|
||||
* @param pool buffer pool.
|
||||
* @throws IOException any IOE
|
||||
* @throws TimeoutException ideally this should never occur.
|
||||
*/
|
||||
public static void returnBuffersToPoolPostRead(List<FileRange> fileRanges,
|
||||
ByteBufferPool pool)
|
||||
throws IOException, TimeoutException {
|
||||
for (FileRange range : fileRanges) {
|
||||
ByteBuffer buffer = FutureIO.awaitFuture(range.getData(),
|
||||
VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
|
||||
TimeUnit.SECONDS);
|
||||
pool.putBuffer(buffer);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Assert that the data read matches the dataset at the given offset.
|
||||
* This helps verify that the seek process is moving the read pointer
|
||||
* to the correct location in the file.
|
||||
* @param readOffset the offset in the file where the read began.
|
||||
* @param operation operation name for the assertion.
|
||||
* @param data data read in.
|
||||
* @param length length of data to check.
|
||||
* @param originalData original data.
|
||||
*/
|
||||
public static void assertDatasetEquals(
|
||||
final int readOffset,
|
||||
final String operation,
|
||||
final ByteBuffer data,
|
||||
int length, byte[] originalData) {
|
||||
for (int i = 0; i < length; i++) {
|
||||
int o = readOffset + i;
|
||||
assertEquals(operation + " with read offset " + readOffset
|
||||
+ ": data[" + i + "] != DATASET[" + o + "]",
|
||||
originalData[o], data.get());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Receives test data from the given input file and checks the size of the
|
||||
* data as well as the pattern inside the received data.
|
||||
|
|
|
@ -0,0 +1,86 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.contract.localfs;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileRange;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||
|
||||
public class TestLocalFSContractVectoredRead extends AbstractContractVectoredReadTest {
|
||||
|
||||
public TestLocalFSContractVectoredRead(String bufferType) {
|
||||
super(bufferType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration conf) {
|
||||
return new LocalFSContract(conf);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChecksumValidationDuringVectoredRead() throws Exception {
|
||||
Path testPath = path("big_range_checksum");
|
||||
LocalFileSystem localFs = (LocalFileSystem) getFileSystem();
|
||||
final byte[] datasetCorrect = ContractTestUtils.dataset(DATASET_LEN, 'a', 32);
|
||||
try (FSDataOutputStream out = localFs.create(testPath, true)){
|
||||
out.write(datasetCorrect);
|
||||
}
|
||||
Path checksumPath = localFs.getChecksumFile(testPath);
|
||||
Assertions.assertThat(localFs.exists(checksumPath))
|
||||
.describedAs("Checksum file should be present")
|
||||
.isTrue();
|
||||
CompletableFuture<FSDataInputStream> fis = localFs.openFile(testPath).build();
|
||||
List<FileRange> someRandomRanges = new ArrayList<>();
|
||||
someRandomRanges.add(FileRange.createFileRange(10, 1024));
|
||||
someRandomRanges.add(FileRange.createFileRange(1025, 1024));
|
||||
try (FSDataInputStream in = fis.get()){
|
||||
in.readVectored(someRandomRanges, getAllocate());
|
||||
validateVectoredReadResult(someRandomRanges, datasetCorrect);
|
||||
}
|
||||
final byte[] datasetCorrupted = ContractTestUtils.dataset(DATASET_LEN, 'a', 64);
|
||||
try (FSDataOutputStream out = localFs.getRaw().create(testPath, true)){
|
||||
out.write(datasetCorrupted);
|
||||
}
|
||||
CompletableFuture<FSDataInputStream> fisN = localFs.openFile(testPath).build();
|
||||
try (FSDataInputStream in = fisN.get()){
|
||||
in.readVectored(someRandomRanges, getAllocate());
|
||||
// Expect checksum exception when data is updated directly through
|
||||
// raw local fs instance.
|
||||
intercept(ChecksumException.class,
|
||||
() -> validateVectoredReadResult(someRandomRanges, datasetCorrupted));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.contract.rawlocal;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
|
||||
public class TestRawLocalContractVectoredRead extends AbstractContractVectoredReadTest {
|
||||
|
||||
public TestRawLocalContractVectoredRead(String bufferType) {
|
||||
super(bufferType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration conf) {
|
||||
return new RawlocalFSContract(conf);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,97 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.io;
|
||||
|
||||
import java.nio.BufferOverflowException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.test.HadoopTestBase;
|
||||
|
||||
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||
|
||||
/**
|
||||
* Non parameterized tests for {@code WeakReferencedElasticByteBufferPool}.
|
||||
*/
|
||||
public class TestMoreWeakReferencedElasticByteBufferPool
|
||||
extends HadoopTestBase {
|
||||
|
||||
@Test
|
||||
public void testMixedBuffersInPool() {
|
||||
WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
|
||||
ByteBuffer buffer1 = pool.getBuffer(true, 5);
|
||||
ByteBuffer buffer2 = pool.getBuffer(true, 10);
|
||||
ByteBuffer buffer3 = pool.getBuffer(false, 5);
|
||||
ByteBuffer buffer4 = pool.getBuffer(false, 10);
|
||||
ByteBuffer buffer5 = pool.getBuffer(true, 15);
|
||||
|
||||
assertBufferCounts(pool, 0, 0);
|
||||
pool.putBuffer(buffer1);
|
||||
pool.putBuffer(buffer2);
|
||||
assertBufferCounts(pool, 2, 0);
|
||||
pool.putBuffer(buffer3);
|
||||
assertBufferCounts(pool, 2, 1);
|
||||
pool.putBuffer(buffer5);
|
||||
assertBufferCounts(pool, 3, 1);
|
||||
pool.putBuffer(buffer4);
|
||||
assertBufferCounts(pool, 3, 2);
|
||||
pool.release();
|
||||
assertBufferCounts(pool, 0, 0);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnexpectedBufferSizes() throws Exception {
|
||||
WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
|
||||
ByteBuffer buffer1 = pool.getBuffer(true, 0);
|
||||
|
||||
// try writing a random byte in a 0 length buffer.
|
||||
// Expected exception as buffer requested is of size 0.
|
||||
intercept(BufferOverflowException.class,
|
||||
() -> buffer1.put(new byte[1]));
|
||||
|
||||
// Expected IllegalArgumentException as negative length buffer is requested.
|
||||
intercept(IllegalArgumentException.class,
|
||||
() -> pool.getBuffer(true, -5));
|
||||
|
||||
// test returning null buffer to the pool.
|
||||
intercept(NullPointerException.class,
|
||||
() -> pool.putBuffer(null));
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method to assert counts of direct and heap buffers in
|
||||
* the given buffer pool.
|
||||
* @param pool buffer pool.
|
||||
* @param numDirectBuffersExpected expected number of direct buffers.
|
||||
* @param numHeapBuffersExpected expected number of heap buffers.
|
||||
*/
|
||||
private void assertBufferCounts(WeakReferencedElasticByteBufferPool pool,
|
||||
int numDirectBuffersExpected,
|
||||
int numHeapBuffersExpected) {
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(true))
|
||||
.describedAs("Number of direct buffers in pool")
|
||||
.isEqualTo(numDirectBuffersExpected);
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(false))
|
||||
.describedAs("Number of heap buffers in pool")
|
||||
.isEqualTo(numHeapBuffersExpected);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,232 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.io;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import org.apache.hadoop.test.HadoopTestBase;
|
||||
|
||||
/**
|
||||
* Unit tests for {@code WeakReferencedElasticByteBufferPool}.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestWeakReferencedElasticByteBufferPool
|
||||
extends HadoopTestBase {
|
||||
|
||||
private final boolean isDirect;
|
||||
|
||||
private final String type;
|
||||
|
||||
@Parameterized.Parameters(name = "Buffer type : {0}")
|
||||
public static List<String> params() {
|
||||
return Arrays.asList("direct", "array");
|
||||
}
|
||||
|
||||
public TestWeakReferencedElasticByteBufferPool(String type) {
|
||||
this.type = type;
|
||||
this.isDirect = !"array".equals(type);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAndPutBasic() {
|
||||
WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
|
||||
int bufferSize = 5;
|
||||
ByteBuffer buffer = pool.getBuffer(isDirect, bufferSize);
|
||||
Assertions.assertThat(buffer.isDirect())
|
||||
.describedAs("Buffered returned should be of correct type {}", type)
|
||||
.isEqualTo(isDirect);
|
||||
Assertions.assertThat(buffer.capacity())
|
||||
.describedAs("Initial capacity of returned buffer from pool")
|
||||
.isEqualTo(bufferSize);
|
||||
Assertions.assertThat(buffer.position())
|
||||
.describedAs("Initial position of returned buffer from pool")
|
||||
.isEqualTo(0);
|
||||
|
||||
byte[] arr = createByteArray(bufferSize);
|
||||
buffer.put(arr, 0, arr.length);
|
||||
buffer.flip();
|
||||
validateBufferContent(buffer, arr);
|
||||
Assertions.assertThat(buffer.position())
|
||||
.describedAs("Buffer's position after filling bytes in it")
|
||||
.isEqualTo(bufferSize);
|
||||
// releasing buffer to the pool.
|
||||
pool.putBuffer(buffer);
|
||||
Assertions.assertThat(buffer.position())
|
||||
.describedAs("Position should be reset to 0 after returning buffer to the pool")
|
||||
.isEqualTo(0);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPoolingWithDifferentSizes() {
|
||||
WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
|
||||
ByteBuffer buffer = pool.getBuffer(isDirect, 5);
|
||||
ByteBuffer buffer1 = pool.getBuffer(isDirect, 10);
|
||||
ByteBuffer buffer2 = pool.getBuffer(isDirect, 15);
|
||||
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(0);
|
||||
|
||||
pool.putBuffer(buffer1);
|
||||
pool.putBuffer(buffer2);
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(2);
|
||||
ByteBuffer buffer3 = pool.getBuffer(isDirect, 12);
|
||||
Assertions.assertThat(buffer3.capacity())
|
||||
.describedAs("Pooled buffer should have older capacity")
|
||||
.isEqualTo(15);
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(1);
|
||||
pool.putBuffer(buffer);
|
||||
ByteBuffer buffer4 = pool.getBuffer(isDirect, 6);
|
||||
Assertions.assertThat(buffer4.capacity())
|
||||
.describedAs("Pooled buffer should have older capacity")
|
||||
.isEqualTo(10);
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(1);
|
||||
|
||||
pool.release();
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool post release")
|
||||
.isEqualTo(0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPoolingWithDifferentInsertionTime() {
|
||||
WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
|
||||
ByteBuffer buffer = pool.getBuffer(isDirect, 10);
|
||||
ByteBuffer buffer1 = pool.getBuffer(isDirect, 10);
|
||||
ByteBuffer buffer2 = pool.getBuffer(isDirect, 10);
|
||||
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(0);
|
||||
|
||||
pool.putBuffer(buffer1);
|
||||
pool.putBuffer(buffer2);
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(2);
|
||||
ByteBuffer buffer3 = pool.getBuffer(isDirect, 10);
|
||||
// As buffer1 is returned to the pool before buffer2, it should
|
||||
// be returned when buffer of same size is asked again from
|
||||
// the pool. Memory references must match not just content
|
||||
// that is why {@code Assertions.isSameAs} is used here rather
|
||||
// than usual {@code Assertions.isEqualTo}.
|
||||
Assertions.assertThat(buffer3)
|
||||
.describedAs("Buffers should be returned in order of their " +
|
||||
"insertion time")
|
||||
.isSameAs(buffer1);
|
||||
pool.putBuffer(buffer);
|
||||
ByteBuffer buffer4 = pool.getBuffer(isDirect, 10);
|
||||
Assertions.assertThat(buffer4)
|
||||
.describedAs("Buffers should be returned in order of their " +
|
||||
"insertion time")
|
||||
.isSameAs(buffer2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGarbageCollection() {
|
||||
WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
|
||||
ByteBuffer buffer = pool.getBuffer(isDirect, 5);
|
||||
ByteBuffer buffer1 = pool.getBuffer(isDirect, 10);
|
||||
ByteBuffer buffer2 = pool.getBuffer(isDirect, 15);
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(0);
|
||||
pool.putBuffer(buffer1);
|
||||
pool.putBuffer(buffer2);
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(2);
|
||||
// Before GC.
|
||||
ByteBuffer buffer4 = pool.getBuffer(isDirect, 12);
|
||||
Assertions.assertThat(buffer4.capacity())
|
||||
.describedAs("Pooled buffer should have older capacity")
|
||||
.isEqualTo(15);
|
||||
pool.putBuffer(buffer4);
|
||||
// Removing the references
|
||||
buffer1 = null;
|
||||
buffer2 = null;
|
||||
buffer4 = null;
|
||||
System.gc();
|
||||
ByteBuffer buffer3 = pool.getBuffer(isDirect, 12);
|
||||
Assertions.assertThat(buffer3.capacity())
|
||||
.describedAs("After garbage collection new buffer should be " +
|
||||
"returned with fixed capacity")
|
||||
.isEqualTo(12);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWeakReferencesPruning() {
|
||||
WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool();
|
||||
ByteBuffer buffer1 = pool.getBuffer(isDirect, 5);
|
||||
ByteBuffer buffer2 = pool.getBuffer(isDirect, 10);
|
||||
ByteBuffer buffer3 = pool.getBuffer(isDirect, 15);
|
||||
|
||||
pool.putBuffer(buffer2);
|
||||
pool.putBuffer(buffer3);
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(2);
|
||||
|
||||
// marking only buffer2 to be garbage collected.
|
||||
buffer2 = null;
|
||||
System.gc();
|
||||
ByteBuffer buffer4 = pool.getBuffer(isDirect, 10);
|
||||
// Number of buffers in the pool is 0 as one got garbage
|
||||
// collected and other got returned in above call.
|
||||
Assertions.assertThat(pool.getCurrentBuffersCount(isDirect))
|
||||
.describedAs("Number of buffers in the pool")
|
||||
.isEqualTo(0);
|
||||
Assertions.assertThat(buffer4.capacity())
|
||||
.describedAs("After gc, pool should return next greater than " +
|
||||
"available buffer")
|
||||
.isEqualTo(15);
|
||||
|
||||
}
|
||||
|
||||
private void validateBufferContent(ByteBuffer buffer, byte[] arr) {
|
||||
for (int i=0; i<arr.length; i++) {
|
||||
Assertions.assertThat(buffer.get())
|
||||
.describedAs("Content of buffer at index {} should match " +
|
||||
"with content of byte array", i)
|
||||
.isEqualTo(arr[i]);
|
||||
}
|
||||
}
|
||||
|
||||
private byte[] createByteArray(int length) {
|
||||
byte[] arr = new byte[length];
|
||||
Random r = new Random();
|
||||
r.nextBytes(arr);
|
||||
return arr;
|
||||
}
|
||||
}
|
|
@ -19,6 +19,9 @@
|
|||
package org.apache.hadoop.test;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Assert;
|
||||
|
||||
/**
|
||||
|
@ -28,17 +31,18 @@ public class MoreAsserts {
|
|||
|
||||
/**
|
||||
* Assert equivalence for array and iterable
|
||||
* @param <T> the type of the elements
|
||||
* @param s the name/message for the collection
|
||||
* @param expected the expected array of elements
|
||||
* @param actual the actual iterable of elements
|
||||
*
|
||||
* @param <T> the type of the elements
|
||||
* @param s the name/message for the collection
|
||||
* @param expected the expected array of elements
|
||||
* @param actual the actual iterable of elements
|
||||
*/
|
||||
public static <T> void assertEquals(String s, T[] expected,
|
||||
Iterable<T> actual) {
|
||||
Iterator<T> it = actual.iterator();
|
||||
int i = 0;
|
||||
for (; i < expected.length && it.hasNext(); ++i) {
|
||||
Assert.assertEquals("Element "+ i +" for "+ s, expected[i], it.next());
|
||||
Assert.assertEquals("Element " + i + " for " + s, expected[i], it.next());
|
||||
}
|
||||
Assert.assertTrue("Expected more elements", i == expected.length);
|
||||
Assert.assertTrue("Expected less elements", !it.hasNext());
|
||||
|
@ -46,7 +50,8 @@ public class MoreAsserts {
|
|||
|
||||
/**
|
||||
* Assert equality for two iterables
|
||||
* @param <T> the type of the elements
|
||||
*
|
||||
* @param <T> the type of the elements
|
||||
* @param s
|
||||
* @param expected
|
||||
* @param actual
|
||||
|
@ -57,10 +62,40 @@ public class MoreAsserts {
|
|||
Iterator<T> ita = actual.iterator();
|
||||
int i = 0;
|
||||
while (ite.hasNext() && ita.hasNext()) {
|
||||
Assert.assertEquals("Element "+ i +" for "+s, ite.next(), ita.next());
|
||||
Assert.assertEquals("Element " + i + " for " + s, ite.next(), ita.next());
|
||||
}
|
||||
Assert.assertTrue("Expected more elements", !ite.hasNext());
|
||||
Assert.assertTrue("Expected less elements", !ita.hasNext());
|
||||
}
|
||||
|
||||
|
||||
public static <T> void assertFutureCompletedSuccessfully(CompletableFuture<T> future) {
|
||||
Assertions.assertThat(future.isDone())
|
||||
.describedAs("This future is supposed to be " +
|
||||
"completed successfully")
|
||||
.isTrue();
|
||||
Assertions.assertThat(future.isCompletedExceptionally())
|
||||
.describedAs("This future is supposed to be " +
|
||||
"completed successfully")
|
||||
.isFalse();
|
||||
}
|
||||
|
||||
public static <T> void assertFutureFailedExceptionally(CompletableFuture<T> future) {
|
||||
Assertions.assertThat(future.isCompletedExceptionally())
|
||||
.describedAs("This future is supposed to be " +
|
||||
"completed exceptionally")
|
||||
.isTrue();
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert two same type of values.
|
||||
* @param actual actual value.
|
||||
* @param expected expected value.
|
||||
* @param message error message to print in case of mismatch.
|
||||
*/
|
||||
public static <T> void assertEqual(T actual, T expected, String message) {
|
||||
Assertions.assertThat(actual)
|
||||
.describedAs("Mismatch in %s", message)
|
||||
.isEqualTo(expected);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -56,5 +56,4 @@
|
|||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
</project>
|
||||
|
|
|
@ -219,6 +219,7 @@
|
|||
<nodejs.version>v12.22.1</nodejs.version>
|
||||
<yarnpkg.version>v1.22.5</yarnpkg.version>
|
||||
<apache-ant.version>1.10.11</apache-ant.version>
|
||||
<jmh.version>1.20</jmh.version>
|
||||
</properties>
|
||||
|
||||
<dependencyManagement>
|
||||
|
@ -1589,6 +1590,16 @@
|
|||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.openjdk.jmh</groupId>
|
||||
<artifactId>jmh-core</artifactId>
|
||||
<version>${jmh.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.openjdk.jmh</groupId>
|
||||
<artifactId>jmh-generator-annprocess</artifactId>
|
||||
<version>${jmh.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.curator</groupId>
|
||||
<artifactId>curator-test</artifactId>
|
||||
|
|
|
@ -1177,4 +1177,30 @@ public final class Constants {
|
|||
*/
|
||||
public static final String FS_S3A_CREATE_HEADER = "fs.s3a.create.header";
|
||||
|
||||
/**
|
||||
* What is the smallest reasonable seek in bytes such
|
||||
* that we group ranges together during vectored read operation.
|
||||
* Value : {@value}.
|
||||
*/
|
||||
public static final String AWS_S3_VECTOR_READS_MIN_SEEK_SIZE =
|
||||
"fs.s3a.vectored.read.min.seek.size";
|
||||
|
||||
/**
|
||||
* What is the largest merged read size in bytes such
|
||||
* that we group ranges together during vectored read.
|
||||
* Setting this value to 0 will disable merging of ranges.
|
||||
* Value : {@value}.
|
||||
*/
|
||||
public static final String AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE =
|
||||
"fs.s3a.vectored.read.max.merged.size";
|
||||
|
||||
/**
|
||||
* Default minimum seek in bytes during vectored reads : {@value}.
|
||||
*/
|
||||
public static final int DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE = 4896; // 4K
|
||||
|
||||
/**
|
||||
* Default maximum read size in bytes during vectored reads : {@value}.
|
||||
*/
|
||||
public static final int DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE = 1253376; //1M
|
||||
}
|
||||
|
|
|
@ -313,6 +313,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|||
* {@code openFile()}.
|
||||
*/
|
||||
private S3AInputPolicy inputPolicy;
|
||||
/** Vectored IO context. */
|
||||
private VectoredIOContext vectoredIOContext;
|
||||
|
||||
private long readAhead;
|
||||
private ChangeDetectionPolicy changeDetectionPolicy;
|
||||
private final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
private volatile boolean isClosed = false;
|
||||
|
@ -584,6 +588,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|||
longBytesOption(conf, ASYNC_DRAIN_THRESHOLD,
|
||||
DEFAULT_ASYNC_DRAIN_THRESHOLD, 0),
|
||||
inputPolicy);
|
||||
vectoredIOContext = populateVectoredIOContext(conf);
|
||||
} catch (AmazonClientException e) {
|
||||
// amazon client exception: stop all services then throw the translation
|
||||
cleanupWithLogger(LOG, span);
|
||||
|
@ -597,6 +602,23 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Populates the configurations related to vectored IO operation
|
||||
* in the context which has to passed down to input streams.
|
||||
* @param conf configuration object.
|
||||
* @return VectoredIOContext.
|
||||
*/
|
||||
private VectoredIOContext populateVectoredIOContext(Configuration conf) {
|
||||
final int minSeekVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MIN_SEEK_SIZE,
|
||||
DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, 0);
|
||||
final int maxReadSizeVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE,
|
||||
DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, 0);
|
||||
return new VectoredIOContext()
|
||||
.setMinSeekForVectoredReads(minSeekVectored)
|
||||
.setMaxReadSizeForVectoredReads(maxReadSizeVectored)
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the client side encryption gauge to 0 or 1, indicating if CSE is
|
||||
* enabled through the gauge or not.
|
||||
|
@ -1463,11 +1485,12 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|||
fileInformation.applyOptions(readContext);
|
||||
LOG.debug("Opening '{}'", readContext);
|
||||
return new FSDataInputStream(
|
||||
new S3AInputStream(
|
||||
readContext.build(),
|
||||
createObjectAttributes(path, fileStatus),
|
||||
createInputStreamCallbacks(auditSpan),
|
||||
inputStreamStats));
|
||||
new S3AInputStream(
|
||||
readContext.build(),
|
||||
createObjectAttributes(path, fileStatus),
|
||||
createInputStreamCallbacks(auditSpan),
|
||||
inputStreamStats,
|
||||
unboundedThreadPool));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1551,7 +1574,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|||
invoker,
|
||||
statistics,
|
||||
statisticsContext,
|
||||
fileStatus)
|
||||
fileStatus,
|
||||
vectoredIOContext)
|
||||
.withAuditSpan(auditSpan);
|
||||
openFileHelper.applyDefaultOptions(roc);
|
||||
return roc.build();
|
||||
|
@ -4926,9 +4950,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
|
|||
/**
|
||||
* This is a proof of concept of a select API.
|
||||
* @param source path to source data
|
||||
* @param expression select expression
|
||||
* @param options request configuration from the builder.
|
||||
* @param providedStatus any passed in status
|
||||
* @param fileInformation any passed in information.
|
||||
* @return the stream of the results
|
||||
* @throws IOException IO failure
|
||||
*/
|
||||
|
|
|
@ -19,38 +19,50 @@
|
|||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import com.amazonaws.services.s3.model.GetObjectRequest;
|
||||
import com.amazonaws.services.s3.model.S3Object;
|
||||
import com.amazonaws.services.s3.model.S3ObjectInputStream;
|
||||
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.CanSetReadahead;
|
||||
import org.apache.hadoop.fs.CanUnbuffer;
|
||||
import org.apache.hadoop.fs.FSExceptionMessages;
|
||||
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
||||
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
|
||||
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||
import org.apache.hadoop.fs.PathIOException;
|
||||
import org.apache.hadoop.fs.StreamCapabilities;
|
||||
import org.apache.hadoop.fs.FSInputStream;
|
||||
import org.apache.hadoop.util.functional.CallableRaisingIOE;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.classification.VisibleForTesting;
|
||||
import org.apache.hadoop.fs.CanSetReadahead;
|
||||
import org.apache.hadoop.fs.CanUnbuffer;
|
||||
import org.apache.hadoop.fs.FSExceptionMessages;
|
||||
import org.apache.hadoop.fs.FSInputStream;
|
||||
import org.apache.hadoop.fs.FileRange;
|
||||
import org.apache.hadoop.fs.PathIOException;
|
||||
import org.apache.hadoop.fs.StreamCapabilities;
|
||||
import org.apache.hadoop.fs.impl.CombinedFileRange;
|
||||
import org.apache.hadoop.fs.VectoredReadUtils;
|
||||
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
|
||||
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
||||
import org.apache.hadoop.fs.statistics.DurationTracker;
|
||||
import org.apache.hadoop.fs.statistics.IOStatistics;
|
||||
import org.apache.hadoop.fs.statistics.IOStatisticsSource;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.util.Preconditions;
|
||||
import org.apache.hadoop.util.functional.CallableRaisingIOE;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
|
||||
import static org.apache.hadoop.fs.VectoredReadUtils.isOrderedDisjoint;
|
||||
import static org.apache.hadoop.fs.VectoredReadUtils.mergeSortedRanges;
|
||||
import static org.apache.hadoop.fs.VectoredReadUtils.validateNonOverlappingAndReturnSortedRanges;
|
||||
import static org.apache.hadoop.fs.s3a.Invoker.onceTrackingDuration;
|
||||
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
|
||||
import static org.apache.hadoop.util.StringUtils.toLowerCase;
|
||||
|
@ -88,6 +100,20 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
* size of a buffer to create when draining the stream.
|
||||
*/
|
||||
private static final int DRAIN_BUFFER_SIZE = 16384;
|
||||
/**
|
||||
* This is the maximum temporary buffer size we use while
|
||||
* populating the data in direct byte buffers during a vectored IO
|
||||
* operation. This is to ensure that when a big range of data is
|
||||
* requested in direct byte buffer doesn't leads to OOM errors.
|
||||
*/
|
||||
private static final int TMP_BUFFER_MAX_SIZE = 64 * 1024;
|
||||
|
||||
/**
|
||||
* Atomic boolean variable to stop all ongoing vectored read operation
|
||||
* for this input stream. This will be set to true when the stream is
|
||||
* closed or unbuffer is called.
|
||||
*/
|
||||
private final AtomicBoolean stopVectoredIOOperations = new AtomicBoolean(false);
|
||||
|
||||
/**
|
||||
* This is the public position; the one set in {@link #seek(long)}
|
||||
|
@ -111,6 +137,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
private S3ObjectInputStream wrappedStream;
|
||||
private final S3AReadOpContext context;
|
||||
private final InputStreamCallbacks client;
|
||||
|
||||
/**
|
||||
* Thread pool used for vectored IO operation.
|
||||
*/
|
||||
private final ThreadPoolExecutor unboundedThreadPool;
|
||||
private final String bucket;
|
||||
private final String key;
|
||||
private final String pathStr;
|
||||
|
@ -122,6 +153,9 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
private S3AInputPolicy inputPolicy;
|
||||
private long readahead = Constants.DEFAULT_READAHEAD_RANGE;
|
||||
|
||||
/** Vectored IO context. */
|
||||
private final VectoredIOContext vectoredIOContext;
|
||||
|
||||
/**
|
||||
* This is the actual position within the object, used by
|
||||
* lazy seek to decide whether to seek on the next read or not.
|
||||
|
@ -160,12 +194,14 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
* @param ctx operation context
|
||||
* @param s3Attributes object attributes
|
||||
* @param client S3 client to use
|
||||
* @param streamStatistics statistics for this stream
|
||||
* @param streamStatistics stream io stats.
|
||||
* @param unboundedThreadPool thread pool to use.
|
||||
*/
|
||||
public S3AInputStream(S3AReadOpContext ctx,
|
||||
S3ObjectAttributes s3Attributes,
|
||||
InputStreamCallbacks client,
|
||||
S3AInputStreamStatistics streamStatistics) {
|
||||
S3ObjectAttributes s3Attributes,
|
||||
InputStreamCallbacks client,
|
||||
S3AInputStreamStatistics streamStatistics,
|
||||
ThreadPoolExecutor unboundedThreadPool) {
|
||||
Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()),
|
||||
"No Bucket");
|
||||
Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key");
|
||||
|
@ -187,6 +223,8 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
setInputPolicy(ctx.getInputPolicy());
|
||||
setReadahead(ctx.getReadahead());
|
||||
this.asyncDrainThreshold = ctx.getAsyncDrainThreshold();
|
||||
this.unboundedThreadPool = unboundedThreadPool;
|
||||
this.vectoredIOContext = context.getVectoredIOContext();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -559,6 +597,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
if (!closed) {
|
||||
closed = true;
|
||||
try {
|
||||
stopVectoredIOOperations.set(true);
|
||||
// close or abort the stream; blocking
|
||||
awaitFuture(closeStream("close() operation", false, true));
|
||||
LOG.debug("Statistics of stream {}\n{}", key, streamStatistics);
|
||||
|
@ -834,6 +873,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
sb.append(" remainingInCurrentRequest=")
|
||||
.append(remainingInCurrentRequest());
|
||||
sb.append(" ").append(changeTracker);
|
||||
sb.append(" ").append(vectoredIOContext);
|
||||
sb.append('\n').append(s);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
|
@ -880,6 +920,313 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}.
|
||||
*/
|
||||
@Override
|
||||
public int minSeekForVectorReads() {
|
||||
return vectoredIOContext.getMinSeekForVectorReads();
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}.
|
||||
*/
|
||||
@Override
|
||||
public int maxReadSizeForVectorReads() {
|
||||
return vectoredIOContext.getMaxReadSizeForVectorReads();
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
* Vectored read implementation for S3AInputStream.
|
||||
* @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 {
|
||||
|
||||
LOG.debug("Starting vectored read on path {} for ranges {} ", pathStr, ranges);
|
||||
checkNotClosed();
|
||||
if (stopVectoredIOOperations.getAndSet(false)) {
|
||||
LOG.debug("Reinstating vectored read operation for path {} ", pathStr);
|
||||
}
|
||||
List<? extends FileRange> sortedRanges = validateNonOverlappingAndReturnSortedRanges(ranges);
|
||||
for (FileRange range : ranges) {
|
||||
validateRangeRequest(range);
|
||||
CompletableFuture<ByteBuffer> result = new CompletableFuture<>();
|
||||
range.setData(result);
|
||||
}
|
||||
|
||||
if (isOrderedDisjoint(sortedRanges, 1, minSeekForVectorReads())) {
|
||||
LOG.debug("Not merging the ranges as they are disjoint");
|
||||
for (FileRange range: sortedRanges) {
|
||||
ByteBuffer buffer = allocate.apply(range.getLength());
|
||||
unboundedThreadPool.submit(() -> readSingleRange(range, buffer));
|
||||
}
|
||||
} else {
|
||||
LOG.debug("Trying to merge the ranges as they are not disjoint");
|
||||
List<CombinedFileRange> combinedFileRanges = mergeSortedRanges(sortedRanges,
|
||||
1, minSeekForVectorReads(),
|
||||
maxReadSizeForVectorReads());
|
||||
LOG.debug("Number of original ranges size {} , Number of combined ranges {} ",
|
||||
ranges.size(), combinedFileRanges.size());
|
||||
for (CombinedFileRange combinedFileRange: combinedFileRanges) {
|
||||
unboundedThreadPool.submit(
|
||||
() -> readCombinedRangeAndUpdateChildren(combinedFileRange, allocate));
|
||||
}
|
||||
}
|
||||
LOG.debug("Finished submitting vectored read to threadpool" +
|
||||
" on path {} for ranges {} ", pathStr, ranges);
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the data from S3 for the bigger combined file range and update all the
|
||||
* underlying ranges.
|
||||
* @param combinedFileRange big combined file range.
|
||||
* @param allocate method to create byte buffers to hold result data.
|
||||
*/
|
||||
private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange,
|
||||
IntFunction<ByteBuffer> allocate) {
|
||||
LOG.debug("Start reading combined range {} from path {} ", combinedFileRange, pathStr);
|
||||
// This reference is must be kept till all buffers are populated as this is a
|
||||
// finalizable object which closes the internal stream when gc triggers.
|
||||
S3Object objectRange = null;
|
||||
S3ObjectInputStream objectContent = null;
|
||||
try {
|
||||
checkIfVectoredIOStopped();
|
||||
final String operationName = "readCombinedFileRange";
|
||||
objectRange = getS3Object(operationName,
|
||||
combinedFileRange.getOffset(),
|
||||
combinedFileRange.getLength());
|
||||
objectContent = objectRange.getObjectContent();
|
||||
if (objectContent == null) {
|
||||
throw new PathIOException(uri,
|
||||
"Null IO stream received during " + operationName);
|
||||
}
|
||||
populateChildBuffers(combinedFileRange, objectContent, allocate);
|
||||
} catch (Exception ex) {
|
||||
LOG.debug("Exception while reading a range {} from path {} ", combinedFileRange, pathStr, ex);
|
||||
for(FileRange child : combinedFileRange.getUnderlying()) {
|
||||
child.getData().completeExceptionally(ex);
|
||||
}
|
||||
} finally {
|
||||
IOUtils.cleanupWithLogger(LOG, objectRange, objectContent);
|
||||
}
|
||||
LOG.debug("Finished reading range {} from path {} ", combinedFileRange, pathStr);
|
||||
}
|
||||
|
||||
/**
|
||||
* Populate underlying buffers of the child ranges.
|
||||
* @param combinedFileRange big combined file range.
|
||||
* @param objectContent data from s3.
|
||||
* @param allocate method to allocate child byte buffers.
|
||||
* @throws IOException any IOE.
|
||||
*/
|
||||
private void populateChildBuffers(CombinedFileRange combinedFileRange,
|
||||
S3ObjectInputStream objectContent,
|
||||
IntFunction<ByteBuffer> allocate) throws IOException {
|
||||
// If the combined file range just contains a single child
|
||||
// range, we only have to fill that one child buffer else
|
||||
// we drain the intermediate data between consecutive ranges
|
||||
// and fill the buffers one by one.
|
||||
if (combinedFileRange.getUnderlying().size() == 1) {
|
||||
FileRange child = combinedFileRange.getUnderlying().get(0);
|
||||
ByteBuffer buffer = allocate.apply(child.getLength());
|
||||
populateBuffer(child.getLength(), buffer, objectContent);
|
||||
child.getData().complete(buffer);
|
||||
} else {
|
||||
FileRange prev = null;
|
||||
for (FileRange child : combinedFileRange.getUnderlying()) {
|
||||
if (prev != null && prev.getOffset() + prev.getLength() < child.getOffset()) {
|
||||
long drainQuantity = child.getOffset() - prev.getOffset() - prev.getLength();
|
||||
drainUnnecessaryData(objectContent, drainQuantity);
|
||||
}
|
||||
ByteBuffer buffer = allocate.apply(child.getLength());
|
||||
populateBuffer(child.getLength(), buffer, objectContent);
|
||||
child.getData().complete(buffer);
|
||||
prev = child;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Drain unnecessary data in between ranges.
|
||||
* @param objectContent s3 data stream.
|
||||
* @param drainQuantity how many bytes to drain.
|
||||
* @throws IOException any IOE.
|
||||
*/
|
||||
private void drainUnnecessaryData(S3ObjectInputStream objectContent, long drainQuantity)
|
||||
throws IOException {
|
||||
int drainBytes = 0;
|
||||
int readCount;
|
||||
while (drainBytes < drainQuantity) {
|
||||
if (drainBytes + DRAIN_BUFFER_SIZE <= drainQuantity) {
|
||||
byte[] drainBuffer = new byte[DRAIN_BUFFER_SIZE];
|
||||
readCount = objectContent.read(drainBuffer);
|
||||
} else {
|
||||
byte[] drainBuffer = new byte[(int) (drainQuantity - drainBytes)];
|
||||
readCount = objectContent.read(drainBuffer);
|
||||
}
|
||||
drainBytes += readCount;
|
||||
}
|
||||
LOG.debug("{} bytes drained from stream ", drainBytes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates range parameters.
|
||||
* In case of S3 we already have contentLength from the first GET request
|
||||
* during an open file operation so failing fast here.
|
||||
* @param range requested range.
|
||||
* @throws EOFException end of file exception.
|
||||
*/
|
||||
private void validateRangeRequest(FileRange range) throws EOFException {
|
||||
VectoredReadUtils.validateRangeRequest(range);
|
||||
if(range.getOffset() + range.getLength() > contentLength) {
|
||||
LOG.warn("Requested range [{}, {}) is beyond EOF for path {}",
|
||||
range.getOffset(), range.getLength(), pathStr);
|
||||
throw new EOFException("Requested range [" + range.getOffset() +", "
|
||||
+ range.getLength() + ") is beyond EOF for path " + pathStr);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Read data from S3 for this range and populate the buffer.
|
||||
* @param range range of data to read.
|
||||
* @param buffer buffer to fill.
|
||||
*/
|
||||
private void readSingleRange(FileRange range, ByteBuffer buffer) {
|
||||
LOG.debug("Start reading range {} from path {} ", range, pathStr);
|
||||
S3Object objectRange = null;
|
||||
S3ObjectInputStream objectContent = null;
|
||||
try {
|
||||
checkIfVectoredIOStopped();
|
||||
long position = range.getOffset();
|
||||
int length = range.getLength();
|
||||
final String operationName = "readRange";
|
||||
objectRange = getS3Object(operationName, position, length);
|
||||
objectContent = objectRange.getObjectContent();
|
||||
if (objectContent == null) {
|
||||
throw new PathIOException(uri,
|
||||
"Null IO stream received during " + operationName);
|
||||
}
|
||||
populateBuffer(length, buffer, objectContent);
|
||||
range.getData().complete(buffer);
|
||||
} catch (Exception ex) {
|
||||
LOG.warn("Exception while reading a range {} from path {} ", range, pathStr, ex);
|
||||
range.getData().completeExceptionally(ex);
|
||||
} finally {
|
||||
IOUtils.cleanupWithLogger(LOG, objectRange, objectContent);
|
||||
}
|
||||
LOG.debug("Finished reading range {} from path {} ", range, pathStr);
|
||||
}
|
||||
|
||||
/**
|
||||
* Populates the buffer with data from objectContent
|
||||
* till length. Handles both direct and heap byte buffers.
|
||||
* @param length length of data to populate.
|
||||
* @param buffer buffer to fill.
|
||||
* @param objectContent result retrieved from S3 store.
|
||||
* @throws IOException any IOE.
|
||||
*/
|
||||
private void populateBuffer(int length,
|
||||
ByteBuffer buffer,
|
||||
S3ObjectInputStream objectContent) throws IOException {
|
||||
if (buffer.isDirect()) {
|
||||
int readBytes = 0;
|
||||
int offset = 0;
|
||||
byte[] tmp = new byte[TMP_BUFFER_MAX_SIZE];
|
||||
while (readBytes < length) {
|
||||
checkIfVectoredIOStopped();
|
||||
int currentLength = readBytes + TMP_BUFFER_MAX_SIZE < length ?
|
||||
TMP_BUFFER_MAX_SIZE
|
||||
: length - readBytes;
|
||||
readByteArray(objectContent, tmp, 0, currentLength);
|
||||
buffer.put(tmp, 0, currentLength);
|
||||
offset = offset + currentLength;
|
||||
readBytes = readBytes + currentLength;
|
||||
}
|
||||
buffer.flip();
|
||||
} else {
|
||||
readByteArray(objectContent, buffer.array(), 0, length);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Read data into destination buffer from s3 object content.
|
||||
* @param objectContent result from S3.
|
||||
* @param dest destination buffer.
|
||||
* @param offset start offset of dest buffer.
|
||||
* @param length number of bytes to fill in dest.
|
||||
* @throws IOException any IOE.
|
||||
*/
|
||||
private void readByteArray(S3ObjectInputStream objectContent,
|
||||
byte[] dest,
|
||||
int offset,
|
||||
int length) throws IOException {
|
||||
int readBytes = 0;
|
||||
while (readBytes < length) {
|
||||
int readBytesCurr = objectContent.read(dest,
|
||||
offset + readBytes,
|
||||
length - readBytes);
|
||||
readBytes +=readBytesCurr;
|
||||
if (readBytesCurr < 0) {
|
||||
throw new EOFException(FSExceptionMessages.EOF_IN_READ_FULLY);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Read data from S3 using a http request with retries.
|
||||
* This also handles if file has been changed while the
|
||||
* http call is getting executed. If the file has been
|
||||
* changed RemoteFileChangedException is thrown.
|
||||
* @param operationName name of the operation for which get object on S3 is called.
|
||||
* @param position position of the object to be read from S3.
|
||||
* @param length length from position of the object to be read from S3.
|
||||
* @return S3Object result s3 object.
|
||||
* @throws IOException exception if any.
|
||||
*/
|
||||
private S3Object getS3Object(String operationName, long position,
|
||||
int length) throws IOException {
|
||||
final GetObjectRequest request = client.newGetRequest(key)
|
||||
.withRange(position, position + length - 1);
|
||||
changeTracker.maybeApplyConstraint(request);
|
||||
DurationTracker tracker = streamStatistics.initiateGetRequest();
|
||||
S3Object objectRange;
|
||||
Invoker invoker = context.getReadInvoker();
|
||||
try {
|
||||
objectRange = invoker.retry(operationName, pathStr, true,
|
||||
() -> {
|
||||
checkIfVectoredIOStopped();
|
||||
return client.getObject(request);
|
||||
});
|
||||
|
||||
} catch (IOException ex) {
|
||||
tracker.failed();
|
||||
throw ex;
|
||||
} finally {
|
||||
tracker.close();
|
||||
}
|
||||
changeTracker.processResponse(objectRange, operationName,
|
||||
position);
|
||||
return objectRange;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if vectored io operation has been stooped. This happens
|
||||
* when the stream is closed or unbuffer is called.
|
||||
* @throws InterruptedIOException throw InterruptedIOException such
|
||||
* that all running vectored io is
|
||||
* terminated thus releasing resources.
|
||||
*/
|
||||
private void checkIfVectoredIOStopped() throws InterruptedIOException {
|
||||
if (stopVectoredIOOperations.get()) {
|
||||
throw new InterruptedIOException("Stream closed or unbuffer is called");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Access the input stream statistics.
|
||||
* This is for internal testing and may be removed without warning.
|
||||
|
@ -965,10 +1312,15 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
/**
|
||||
* Closes the underlying S3 stream, and merges the {@link #streamStatistics}
|
||||
* instance associated with the stream.
|
||||
* Also sets the {@code stopVectoredIOOperations} flag to true such that
|
||||
* active vectored read operations are terminated. However termination of
|
||||
* old vectored reads are not guaranteed if a new vectored read operation
|
||||
* is initiated after unbuffer is called.
|
||||
*/
|
||||
@Override
|
||||
public synchronized void unbuffer() {
|
||||
try {
|
||||
stopVectoredIOOperations.set(true);
|
||||
closeStream("unbuffer()", false, false);
|
||||
} finally {
|
||||
streamStatistics.unbuffered();
|
||||
|
@ -981,6 +1333,7 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead,
|
|||
case StreamCapabilities.IOSTATISTICS:
|
||||
case StreamCapabilities.READAHEAD:
|
||||
case StreamCapabilities.UNBUFFER:
|
||||
case StreamCapabilities.VECTOREDIO:
|
||||
return true;
|
||||
default:
|
||||
return false;
|
||||
|
|
|
@ -64,6 +64,12 @@ public class S3AReadOpContext extends S3AOpContext {
|
|||
*/
|
||||
private long asyncDrainThreshold;
|
||||
|
||||
/**
|
||||
* Vectored IO context for vectored read api
|
||||
* in {@code S3AInputStream#readVectored(List, IntFunction)}.
|
||||
*/
|
||||
private final VectoredIOContext vectoredIOContext;
|
||||
|
||||
/**
|
||||
* Instantiate.
|
||||
* @param path path of read
|
||||
|
@ -71,17 +77,19 @@ public class S3AReadOpContext extends S3AOpContext {
|
|||
* @param stats Fileystem statistics (may be null)
|
||||
* @param instrumentation statistics context
|
||||
* @param dstFileStatus target file status
|
||||
* @param vectoredIOContext context for vectored read operation.
|
||||
*/
|
||||
public S3AReadOpContext(
|
||||
final Path path,
|
||||
Invoker invoker,
|
||||
@Nullable FileSystem.Statistics stats,
|
||||
S3AStatisticsContext instrumentation,
|
||||
FileStatus dstFileStatus) {
|
||||
|
||||
FileStatus dstFileStatus,
|
||||
VectoredIOContext vectoredIOContext) {
|
||||
super(invoker, stats, instrumentation,
|
||||
dstFileStatus);
|
||||
this.path = requireNonNull(path);
|
||||
this.vectoredIOContext = requireNonNull(vectoredIOContext, "vectoredIOContext");
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -199,6 +207,14 @@ public class S3AReadOpContext extends S3AOpContext {
|
|||
return asyncDrainThreshold;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Vectored IO context for this this read op.
|
||||
* @return vectored IO context.
|
||||
*/
|
||||
public VectoredIOContext getVectoredIOContext() {
|
||||
return vectoredIOContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder(
|
||||
|
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.s3a;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
/**
|
||||
* Context related to vectored IO operation.
|
||||
* See {@link S3AInputStream#readVectored(List, IntFunction)}.
|
||||
*/
|
||||
public class VectoredIOContext {
|
||||
|
||||
/**
|
||||
* What is the smallest reasonable seek that we should group
|
||||
* ranges together during vectored read operation.
|
||||
*/
|
||||
private int minSeekForVectorReads;
|
||||
|
||||
/**
|
||||
* What is the largest size that we should group ranges
|
||||
* together during vectored read operation.
|
||||
* Setting this value 0 will disable merging of ranges.
|
||||
*/
|
||||
private int maxReadSizeForVectorReads;
|
||||
|
||||
/**
|
||||
* Default no arg constructor.
|
||||
*/
|
||||
public VectoredIOContext() {
|
||||
}
|
||||
|
||||
public VectoredIOContext setMinSeekForVectoredReads(int minSeek) {
|
||||
this.minSeekForVectorReads = minSeek;
|
||||
return this;
|
||||
}
|
||||
|
||||
public VectoredIOContext setMaxReadSizeForVectoredReads(int maxSize) {
|
||||
this.maxReadSizeForVectorReads = maxSize;
|
||||
return this;
|
||||
}
|
||||
|
||||
public VectoredIOContext build() {
|
||||
return this;
|
||||
}
|
||||
|
||||
public int getMinSeekForVectorReads() {
|
||||
return minSeekForVectorReads;
|
||||
}
|
||||
|
||||
public int getMaxReadSizeForVectorReads() {
|
||||
return maxReadSizeForVectorReads;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "VectoredIOContext{" +
|
||||
"minSeekForVectorReads=" + minSeekForVectorReads +
|
||||
", maxReadSizeForVectorReads=" + maxReadSizeForVectorReads +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -220,8 +220,7 @@ public class GetContentSummaryOperation extends
|
|||
|
||||
/***
|
||||
* List all entries under a path.
|
||||
*
|
||||
* @param path
|
||||
* @param path path.
|
||||
* @param recursive if the subdirectories need to be traversed recursively
|
||||
* @return an iterator over the listing.
|
||||
* @throws IOException failure
|
||||
|
|
|
@ -55,6 +55,36 @@ it isn't, and some attempts to preserve the metaphor are "aggressively suboptima
|
|||
|
||||
To make most efficient use of S3, care is needed.
|
||||
|
||||
## <a name="vectoredIO"></a> Improving read performance using Vectored IO
|
||||
The S3A FileSystem supports implementation of vectored read api using which
|
||||
a client can provide a list of file ranges to read returning a future read
|
||||
object associated with each range. For full api specification please see
|
||||
[FSDataInputStream](../../hadoop-common-project/hadoop-common/filesystem/fsdatainputstream.html).
|
||||
|
||||
The following properties can be configured to optimise vectored reads based
|
||||
on the client requirements.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.s3a.vectored.read.min.seek.size</name>
|
||||
<value>4K</value>
|
||||
<description>
|
||||
What is the smallest reasonable seek in bytes such
|
||||
that we group ranges together during vectored
|
||||
read operation.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.s3a.vectored.read.max.merged.size</name>
|
||||
<value>1M</value>
|
||||
<description>
|
||||
What is the largest merged read size in bytes such
|
||||
that we group ranges together during vectored read.
|
||||
Setting this value to 0 will disable merging of ranges.
|
||||
</description>
|
||||
</property>
|
||||
```
|
||||
|
||||
## <a name="fadvise"></a> Improving data input performance through fadvise
|
||||
|
||||
The S3A Filesystem client supports the notion of input policies, similar
|
||||
|
|
|
@ -0,0 +1,159 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.fs.contract.s3a;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileRange;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
import org.apache.hadoop.fs.s3a.Constants;
|
||||
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
||||
import org.apache.hadoop.fs.s3a.S3ATestUtils;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
|
||||
import static org.apache.hadoop.test.MoreAsserts.assertEqual;
|
||||
|
||||
public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTest {
|
||||
|
||||
public ITestS3AContractVectoredRead(String bufferType) {
|
||||
super(bufferType);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration conf) {
|
||||
return new S3AContract(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Overriding in S3 vectored read api fails fast in case of EOF
|
||||
* requested range.
|
||||
*/
|
||||
@Override
|
||||
public void testEOFRanges() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = new ArrayList<>();
|
||||
fileRanges.add(FileRange.createFileRange(DATASET_LEN, 100));
|
||||
verifyExceptionalVectoredRead(fs, fileRanges, EOFException.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMinSeekAndMaxSizeConfigsPropagation() throws Exception {
|
||||
Configuration conf = getFileSystem().getConf();
|
||||
S3ATestUtils.removeBaseAndBucketOverrides(conf,
|
||||
Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE,
|
||||
Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE);
|
||||
S3ATestUtils.disableFilesystemCaching(conf);
|
||||
final int configuredMinSeek = 2 * 1024;
|
||||
final int configuredMaxSize = 10 * 1024 * 1024;
|
||||
conf.set(Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, "2K");
|
||||
conf.set(Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, "10M");
|
||||
try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
|
||||
try (FSDataInputStream fis = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
int newMinSeek = fis.minSeekForVectorReads();
|
||||
int newMaxSize = fis.maxReadSizeForVectorReads();
|
||||
assertEqual(newMinSeek, configuredMinSeek,
|
||||
"configured s3a min seek for vectored reads");
|
||||
assertEqual(newMaxSize, configuredMaxSize,
|
||||
"configured s3a max size for vectored reads");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMinSeekAndMaxSizeDefaultValues() throws Exception {
|
||||
Configuration conf = getFileSystem().getConf();
|
||||
S3ATestUtils.removeBaseAndBucketOverrides(conf,
|
||||
Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE,
|
||||
Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE);
|
||||
try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) {
|
||||
try (FSDataInputStream fis = fs.open(path(VECTORED_READ_FILE_NAME))) {
|
||||
int minSeek = fis.minSeekForVectorReads();
|
||||
int maxSize = fis.maxReadSizeForVectorReads();
|
||||
assertEqual(minSeek, Constants.DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE,
|
||||
"default s3a min seek for vectored reads");
|
||||
assertEqual(maxSize, Constants.DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE,
|
||||
"default s3a max read size for vectored reads");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStopVectoredIoOperationsCloseStream() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = createSampleNonOverlappingRanges();
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))){
|
||||
in.readVectored(fileRanges, getAllocate());
|
||||
in.close();
|
||||
LambdaTestUtils.intercept(InterruptedIOException.class,
|
||||
() -> validateVectoredReadResult(fileRanges, DATASET));
|
||||
}
|
||||
// reopening the stream should succeed.
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))){
|
||||
in.readVectored(fileRanges, getAllocate());
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStopVectoredIoOperationsUnbuffer() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = createSampleNonOverlappingRanges();
|
||||
try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))){
|
||||
in.readVectored(fileRanges, getAllocate());
|
||||
in.unbuffer();
|
||||
LambdaTestUtils.intercept(InterruptedIOException.class,
|
||||
() -> validateVectoredReadResult(fileRanges, DATASET));
|
||||
// re-initiating the vectored reads after unbuffer should succeed.
|
||||
in.readVectored(fileRanges, getAllocate());
|
||||
validateVectoredReadResult(fileRanges, DATASET);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* S3 vectored IO doesn't support overlapping ranges.
|
||||
*/
|
||||
@Override
|
||||
public void testOverlappingRanges() throws Exception {
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = getSampleOverlappingRanges();
|
||||
verifyExceptionalVectoredRead(fs, fileRanges, UnsupportedOperationException.class);
|
||||
}
|
||||
|
||||
/**
|
||||
* S3 vectored IO doesn't support overlapping ranges.
|
||||
*/
|
||||
@Override
|
||||
public void testSameRanges() throws Exception {
|
||||
// Same ranges are special case of overlapping only.
|
||||
FileSystem fs = getFileSystem();
|
||||
List<FileRange> fileRanges = getSampleSameRanges();
|
||||
verifyExceptionalVectoredRead(fs, fileRanges, UnsupportedOperationException.class);
|
||||
}
|
||||
}
|
|
@ -111,7 +111,8 @@ public class TestS3AInputStreamRetry extends AbstractS3AMockTest {
|
|||
s3AReadOpContext,
|
||||
s3ObjectAttributes,
|
||||
getMockedInputStreamCallback(),
|
||||
s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics());
|
||||
s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(),
|
||||
null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,8 +19,13 @@
|
|||
package org.apache.hadoop.fs.s3a.scale;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import com.amazonaws.event.ProgressEvent;
|
||||
import com.amazonaws.event.ProgressEventType;
|
||||
|
@ -35,7 +40,9 @@ import org.slf4j.LoggerFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileRange;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||
import org.apache.hadoop.fs.s3a.Constants;
|
||||
|
@ -47,6 +54,7 @@ import org.apache.hadoop.fs.statistics.IOStatistics;
|
|||
import org.apache.hadoop.util.Progressable;
|
||||
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
|
||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
|
||||
import static org.apache.hadoop.fs.s3a.Constants.*;
|
||||
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
||||
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING;
|
||||
|
@ -446,6 +454,30 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|||
toHuman(timer.nanosPerOperation(ops)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_045_vectoredIOHugeFile() throws Throwable {
|
||||
assumeHugeFileExists();
|
||||
List<FileRange> rangeList = new ArrayList<>();
|
||||
rangeList.add(FileRange.createFileRange(5856368, 116770));
|
||||
rangeList.add(FileRange.createFileRange(3520861, 116770));
|
||||
rangeList.add(FileRange.createFileRange(8191913, 116770));
|
||||
rangeList.add(FileRange.createFileRange(1520861, 116770));
|
||||
rangeList.add(FileRange.createFileRange(2520861, 116770));
|
||||
rangeList.add(FileRange.createFileRange(9191913, 116770));
|
||||
rangeList.add(FileRange.createFileRange(2820861, 156770));
|
||||
IntFunction<ByteBuffer> allocate = ByteBuffer::allocate;
|
||||
FileSystem fs = getFileSystem();
|
||||
CompletableFuture<FSDataInputStream> builder =
|
||||
fs.openFile(hugefile).build();
|
||||
try (FSDataInputStream in = builder.get()) {
|
||||
in.readVectored(rangeList, allocate);
|
||||
byte[] readFullRes = new byte[(int)filesize];
|
||||
in.readFully(0, readFullRes);
|
||||
// Comparing vectored read results with read fully.
|
||||
validateVectoredReadResult(rangeList, readFullRes);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Read in the entire file using read() calls.
|
||||
* @throws Throwable failure
|
||||
|
|
|
@ -52,7 +52,7 @@ log4j.logger.org.apache.hadoop.ipc.Server=WARN
|
|||
|
||||
# for debugging low level S3a operations, uncomment these lines
|
||||
# Log all S3A classes
|
||||
#log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
|
||||
log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
|
||||
#log4j.logger.org.apache.hadoop.fs.s3a.S3AUtils=INFO
|
||||
#log4j.logger.org.apache.hadoop.fs.s3a.Listing=INFO
|
||||
|
||||
|
|
|
@ -0,0 +1,94 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
|
||||
|
||||
-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
|
||||
https://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-project</artifactId>
|
||||
<version>3.4.0-SNAPSHOT</version>
|
||||
<relativePath>../../hadoop-project/pom.xml</relativePath>
|
||||
</parent>
|
||||
<artifactId>hadoop-benchmark</artifactId>
|
||||
<version>3.4.0-SNAPSHOT</version>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<name>Apache Hadoop Common Benchmark</name>
|
||||
<description>Apache Hadoop Common Benchmark</description>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.openjdk.jmh</groupId>
|
||||
<artifactId>jmh-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.openjdk.jmh</groupId>
|
||||
<artifactId>jmh-generator-annprocess</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<artifactId>maven-assembly-plugin</artifactId>
|
||||
<configuration>
|
||||
<archive>
|
||||
<manifest>
|
||||
<mainClass>org.apache.hadoop.benchmark.VectoredReadBenchmark</mainClass>
|
||||
</manifest>
|
||||
</archive>
|
||||
<descriptors>
|
||||
<descriptor>src/main/assembly/uber.xml</descriptor>
|
||||
</descriptors>
|
||||
</configuration>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>make-assembly</id> <!-- this is used for inheritance merges -->
|
||||
<phase>package</phase> <!-- bind to the packaging phase -->
|
||||
<goals>
|
||||
<goal>single</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>findbugs-maven-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludeFilterFile>${basedir}/src/main/findbugs/exclude.xml</excludeFilterFile>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>com.github.spotbugs</groupId>
|
||||
<artifactId>spotbugs-maven-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludeFilterFile>${basedir}/src/main/findbugs/exclude.xml</excludeFilterFile>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
|
@ -0,0 +1,33 @@
|
|||
<!--
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<assembly>
|
||||
<id>uber</id>
|
||||
<formats>
|
||||
<format>jar</format>
|
||||
</formats>
|
||||
<includeBaseDirectory>false</includeBaseDirectory>
|
||||
<dependencySets>
|
||||
<dependencySet>
|
||||
<outputDirectory>/</outputDirectory>
|
||||
<useProjectArtifact>true</useProjectArtifact>
|
||||
<unpack>true</unpack>
|
||||
<scope>runtime</scope>
|
||||
</dependencySet>
|
||||
</dependencySets>
|
||||
<containerDescriptorHandlers>
|
||||
<containerDescriptorHandler>
|
||||
<handlerName>metaInf-services</handlerName>
|
||||
</containerDescriptorHandler>
|
||||
</containerDescriptorHandlers>
|
||||
</assembly>
|
|
@ -0,0 +1,22 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<FindBugsFilter>
|
||||
<Match>
|
||||
<Class name="~org\.apache\.hadoop\.benchmark\.generated.*"/>
|
||||
</Match>
|
||||
<Match>
|
||||
<Class name="~org\.openjdk\.jmh\.infra\.generated.*"/>
|
||||
</Match>
|
||||
</FindBugsFilter>
|
|
@ -0,0 +1,245 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.benchmark;
|
||||
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Level;
|
||||
import org.openjdk.jmh.annotations.Mode;
|
||||
import org.openjdk.jmh.annotations.OutputTimeUnit;
|
||||
import org.openjdk.jmh.annotations.Param;
|
||||
import org.openjdk.jmh.annotations.Scope;
|
||||
import org.openjdk.jmh.annotations.Setup;
|
||||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
import org.openjdk.jmh.runner.Runner;
|
||||
import org.openjdk.jmh.runner.options.OptionsBuilder;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.AsynchronousFileChannel;
|
||||
import java.nio.channels.CompletionHandler;
|
||||
import java.nio.file.FileSystems;
|
||||
import java.nio.file.StandardOpenOption;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileRange;
|
||||
import org.apache.hadoop.fs.impl.FileRangeImpl;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public class VectoredReadBenchmark {
|
||||
|
||||
static final Path DATA_PATH = getTestDataPath();
|
||||
static final String DATA_PATH_PROPERTY = "bench.data";
|
||||
static final int READ_SIZE = 64 * 1024;
|
||||
static final long SEEK_SIZE = 1024L * 1024;
|
||||
|
||||
|
||||
static Path getTestDataPath() {
|
||||
String value = System.getProperty(DATA_PATH_PROPERTY);
|
||||
return new Path(value == null ? "/tmp/taxi.orc" : value);
|
||||
}
|
||||
|
||||
@State(Scope.Thread)
|
||||
public static class FileSystemChoice {
|
||||
|
||||
@Param({"local", "raw"})
|
||||
private String fileSystemKind;
|
||||
|
||||
private Configuration conf;
|
||||
private FileSystem fs;
|
||||
|
||||
@Setup(Level.Trial)
|
||||
public void setup() {
|
||||
conf = new Configuration();
|
||||
try {
|
||||
LocalFileSystem local = FileSystem.getLocal(conf);
|
||||
fs = "raw".equals(fileSystemKind) ? local.getRaw() : local;
|
||||
} catch (IOException e) {
|
||||
throw new IllegalArgumentException("Can't get filesystem", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@State(Scope.Thread)
|
||||
public static class BufferChoice {
|
||||
@Param({"direct", "array"})
|
||||
private String bufferKind;
|
||||
|
||||
private IntFunction<ByteBuffer> allocate;
|
||||
@Setup(Level.Trial)
|
||||
public void setup() {
|
||||
allocate = "array".equals(bufferKind)
|
||||
? ByteBuffer::allocate : ByteBuffer::allocateDirect;
|
||||
}
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
public void asyncRead(FileSystemChoice fsChoice,
|
||||
BufferChoice bufferChoice,
|
||||
Blackhole blackhole) throws Exception {
|
||||
FSDataInputStream stream = fsChoice.fs.open(DATA_PATH);
|
||||
List<FileRange> ranges = new ArrayList<>();
|
||||
for(int m=0; m < 100; ++m) {
|
||||
FileRange range = FileRange.createFileRange(m * SEEK_SIZE, READ_SIZE);
|
||||
ranges.add(range);
|
||||
}
|
||||
stream.readVectored(ranges, bufferChoice.allocate);
|
||||
for(FileRange range: ranges) {
|
||||
blackhole.consume(range.getData().get());
|
||||
}
|
||||
stream.close();
|
||||
}
|
||||
|
||||
static class Joiner implements CompletionHandler<ByteBuffer, FileRange> {
|
||||
private int remaining;
|
||||
private final ByteBuffer[] result;
|
||||
private Throwable exception = null;
|
||||
|
||||
Joiner(int total) {
|
||||
remaining = total;
|
||||
result = new ByteBuffer[total];
|
||||
}
|
||||
|
||||
synchronized void finish() {
|
||||
remaining -= 1;
|
||||
if (remaining == 0) {
|
||||
notify();
|
||||
}
|
||||
}
|
||||
|
||||
synchronized ByteBuffer[] join() throws InterruptedException, IOException {
|
||||
while (remaining > 0 && exception == null) {
|
||||
wait();
|
||||
}
|
||||
if (exception != null) {
|
||||
throw new IOException("problem reading", exception);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public synchronized void completed(ByteBuffer buffer, FileRange attachment) {
|
||||
result[--remaining] = buffer;
|
||||
if (remaining == 0) {
|
||||
notify();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void failed(Throwable exc, FileRange attachment) {
|
||||
this.exception = exc;
|
||||
notify();
|
||||
}
|
||||
}
|
||||
|
||||
static class FileRangeCallback extends FileRangeImpl implements
|
||||
CompletionHandler<Integer, FileRangeCallback> {
|
||||
private final AsynchronousFileChannel channel;
|
||||
private final ByteBuffer buffer;
|
||||
private int completed = 0;
|
||||
private final Joiner joiner;
|
||||
|
||||
FileRangeCallback(AsynchronousFileChannel channel, long offset,
|
||||
int length, Joiner joiner, ByteBuffer buffer) {
|
||||
super(offset, length);
|
||||
this.channel = channel;
|
||||
this.joiner = joiner;
|
||||
this.buffer = buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void completed(Integer result, FileRangeCallback attachment) {
|
||||
final int bytes = result;
|
||||
if (bytes == -1) {
|
||||
failed(new EOFException("Read past end of file"), this);
|
||||
}
|
||||
completed += bytes;
|
||||
if (completed < this.getLength()) {
|
||||
channel.read(buffer, this.getOffset() + completed, this, this);
|
||||
} else {
|
||||
buffer.flip();
|
||||
joiner.finish();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failed(Throwable exc, FileRangeCallback attachment) {
|
||||
joiner.failed(exc, this);
|
||||
}
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
public void asyncFileChanArray(BufferChoice bufferChoice,
|
||||
Blackhole blackhole) throws Exception {
|
||||
java.nio.file.Path path = FileSystems.getDefault().getPath(DATA_PATH.toString());
|
||||
AsynchronousFileChannel channel = AsynchronousFileChannel.open(path, StandardOpenOption.READ);
|
||||
List<FileRangeImpl> ranges = new ArrayList<>();
|
||||
Joiner joiner = new Joiner(100);
|
||||
for(int m=0; m < 100; ++m) {
|
||||
ByteBuffer buffer = bufferChoice.allocate.apply(READ_SIZE);
|
||||
FileRangeCallback range = new FileRangeCallback(channel, m * SEEK_SIZE,
|
||||
READ_SIZE, joiner, buffer);
|
||||
ranges.add(range);
|
||||
channel.read(buffer, range.getOffset(), range, range);
|
||||
}
|
||||
joiner.join();
|
||||
channel.close();
|
||||
blackhole.consume(ranges);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
public void syncRead(FileSystemChoice fsChoice,
|
||||
Blackhole blackhole) throws Exception {
|
||||
FSDataInputStream stream = fsChoice.fs.open(DATA_PATH);
|
||||
List<byte[]> result = new ArrayList<>();
|
||||
for(int m=0; m < 100; ++m) {
|
||||
byte[] buffer = new byte[READ_SIZE];
|
||||
stream.readFully(m * SEEK_SIZE, buffer);
|
||||
result.add(buffer);
|
||||
}
|
||||
blackhole.consume(result);
|
||||
stream.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Run the benchmarks.
|
||||
* @param args the pathname of a 100MB data file
|
||||
* @throws Exception any ex.
|
||||
*/
|
||||
public static void main(String[] args) throws Exception {
|
||||
OptionsBuilder opts = new OptionsBuilder();
|
||||
opts.include("VectoredReadBenchmark");
|
||||
opts.jvmArgs("-server", "-Xms256m", "-Xmx2g",
|
||||
"-D" + DATA_PATH_PROPERTY + "=" + args[0]);
|
||||
opts.forks(1);
|
||||
new Runner(opts.build()).run();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,22 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Benchmark for Vectored Read IO operations.
|
||||
*/
|
||||
package org.apache.hadoop.benchmark;
|
|
@ -51,6 +51,7 @@
|
|||
<module>hadoop-azure-datalake</module>
|
||||
<module>hadoop-aliyun</module>
|
||||
<module>hadoop-fs2img</module>
|
||||
<module>hadoop-benchmark</module>
|
||||
</modules>
|
||||
|
||||
<build>
|
||||
|
|
1
pom.xml
1
pom.xml
|
@ -550,6 +550,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x
|
|||
<exclude>licenses-binary/**</exclude>
|
||||
<exclude>dev-support/docker/pkg-resolver/packages.json</exclude>
|
||||
<exclude>dev-support/docker/pkg-resolver/platforms.json</exclude>
|
||||
<exclude>**/target/**</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
|
|
Loading…
Reference in New Issue