HADOOP-18181. Move prefetch common classes to hadoop common (#4690)

contains
HADOOP-18187. Convert s3a prefetching to use JavaDoc for fields and enums.
HADOOP-18318. Update class names to be clear they belong to S3A prefetching

Contributed by Steve Loughran
This commit is contained in:
Steve Loughran 2022-08-17 09:50:49 +01:00 committed by GitHub
parent e23f70a03c
commit 36bbde2fda
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
52 changed files with 1565 additions and 1278 deletions

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.io.Closeable;
import java.io.IOException;

View File

@ -17,232 +17,234 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkPositiveInteger;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkWithinRange;
/**
* Holds information about blocks of data in a file.
*/
public class BlockData {
public final class BlockData {
// State of each block of data.
enum State {
// Data is not yet ready to be read from this block (still being prefetched).
/** Data is not yet ready to be read from this block (still being prefetched). */
NOT_READY,
// A read of this block has been enqueued in the prefetch queue.
/** A read of this block has been enqueued in the prefetch queue. */
QUEUED,
// This block is ready to be read. That is, it has been fully read.
/** A read of this block has been enqueued in the prefetch queue. */
READY,
// This block has been cached in the local disk cache.
/** This block has been cached in the local disk cache. */
CACHED
}
// State of all blocks in a file.
/**
* State of all blocks in a file.
*/
private State[] state;
// The size of a file.
/**
* The size of a file.
*/
private final long fileSize;
// The file is divided into blocks of this size.
/**
* The file is divided into blocks of this size.
*/
private final int blockSize;
// The file has these many blocks.
/**
* The file has these many blocks.
*/
private final int numBlocks;
/**
* Constructs an instance of {@link BlockData}.
*
* @param fileSize the size of a file.
* @param blockSize the file is divided into blocks of this size.
*
* @throws IllegalArgumentException if fileSize is negative.
* @throws IllegalArgumentException if blockSize is negative.
* @throws IllegalArgumentException if blockSize is zero or negative.
*/
public BlockData(long fileSize, int blockSize) {
Validate.checkNotNegative(fileSize, "fileSize");
checkNotNegative(fileSize, "fileSize");
if (fileSize == 0) {
Validate.checkNotNegative(blockSize, "blockSize");
checkNotNegative(blockSize, "blockSize");
} else {
Validate.checkPositiveInteger(blockSize, "blockSize");
checkPositiveInteger(blockSize, "blockSize");
}
this.fileSize = fileSize;
this.blockSize = blockSize;
this.numBlocks =
(fileSize == 0) ? 0 : ((int) (fileSize / blockSize)) + (fileSize % blockSize > 0 ? 1 : 0);
(fileSize == 0)
? 0
: ((int) (fileSize / blockSize)) + (fileSize % blockSize > 0
? 1
: 0);
this.state = new State[this.numBlocks];
for (int b = 0; b < this.numBlocks; b++) {
this.setState(b, State.NOT_READY);
setState(b, State.NOT_READY);
}
}
/**
* Gets the size of each block.
*
* @return the size of each block.
*/
public int getBlockSize() {
return this.blockSize;
return blockSize;
}
/**
* Gets the size of the associated file.
*
* @return the size of the associated file.
*/
public long getFileSize() {
return this.fileSize;
return fileSize;
}
/**
* Gets the number of blocks in the associated file.
*
* @return the number of blocks in the associated file.
*/
public int getNumBlocks() {
return this.numBlocks;
return numBlocks;
}
/**
* Indicates whether the given block is the last block in the associated file.
*
* @param blockNumber the id of the desired block.
* @return true if the given block is the last block in the associated file, false otherwise.
*
* @throws IllegalArgumentException if blockNumber is invalid.
*/
public boolean isLastBlock(int blockNumber) {
if (this.fileSize == 0) {
if (fileSize == 0) {
return false;
}
throwIfInvalidBlockNumber(blockNumber);
return blockNumber == (this.numBlocks - 1);
return blockNumber == (numBlocks - 1);
}
/**
* Gets the id of the block that contains the given absolute offset.
*
* @param offset the absolute offset to check.
* @return the id of the block that contains the given absolute offset.
*
* @throws IllegalArgumentException if offset is invalid.
*/
public int getBlockNumber(long offset) {
throwIfInvalidOffset(offset);
return (int) (offset / this.blockSize);
return (int) (offset / blockSize);
}
/**
* Gets the size of the given block.
*
* @param blockNumber the id of the desired block.
* @return the size of the given block.
*/
public int getSize(int blockNumber) {
if (this.fileSize == 0) {
if (fileSize == 0) {
return 0;
}
if (this.isLastBlock(blockNumber)) {
return (int) (this.fileSize - (((long) this.blockSize) * (this.numBlocks - 1)));
if (isLastBlock(blockNumber)) {
return (int) (fileSize - (((long) blockSize) * (numBlocks - 1)));
} else {
return this.blockSize;
return blockSize;
}
}
/**
* Indicates whether the given absolute offset is valid.
*
* @param offset absolute offset in the file..
* @return true if the given absolute offset is valid, false otherwise.
*/
public boolean isValidOffset(long offset) {
return (offset >= 0) && (offset < this.fileSize);
return (offset >= 0) && (offset < fileSize);
}
/**
* Gets the start offset of the given block.
* @param blockNumber the id of the given block.
* @return the start offset of the given block.
*
* @throws IllegalArgumentException if blockNumber is invalid.
*/
public long getStartOffset(int blockNumber) {
throwIfInvalidBlockNumber(blockNumber);
return blockNumber * (long) this.blockSize;
return blockNumber * (long) blockSize;
}
/**
* Gets the relative offset corresponding to the given block and the absolute offset.
*
* @param blockNumber the id of the given block.
* @param offset absolute offset in the file.
* @return the relative offset corresponding to the given block and the absolute offset.
*
* @throws IllegalArgumentException if either blockNumber or offset is invalid.
*/
public int getRelativeOffset(int blockNumber, long offset) {
throwIfInvalidOffset(offset);
return (int) (offset - this.getStartOffset(blockNumber));
return (int) (offset - getStartOffset(blockNumber));
}
/**
* Gets the state of the given block.
*
* @param blockNumber the id of the given block.
* @return the state of the given block.
*
* @throws IllegalArgumentException if blockNumber is invalid.
*/
public State getState(int blockNumber) {
throwIfInvalidBlockNumber(blockNumber);
return this.state[blockNumber];
return state[blockNumber];
}
/**
* Sets the state of the given block to the given value.
*
* @param blockNumber the id of the given block.
* @param blockState the target state.
*
* @throws IllegalArgumentException if blockNumber is invalid.
*/
public void setState(int blockNumber, State blockState) {
throwIfInvalidBlockNumber(blockNumber);
this.state[blockNumber] = blockState;
state[blockNumber] = blockState;
}
// Debug helper.
public String getStateString() {
StringBuilder sb = new StringBuilder();
int blockNumber = 0;
while (blockNumber < this.numBlocks) {
State tstate = this.getState(blockNumber);
while (blockNumber < numBlocks) {
State tstate = getState(blockNumber);
int endBlockNumber = blockNumber;
while ((endBlockNumber < this.numBlocks) && (this.getState(endBlockNumber) == tstate)) {
while ((endBlockNumber < numBlocks) && (getState(endBlockNumber)
== tstate)) {
endBlockNumber++;
}
sb.append(String.format("[%03d ~ %03d] %s%n", blockNumber, endBlockNumber - 1, tstate));
sb.append(
String.format("[%03d ~ %03d] %s%n", blockNumber, endBlockNumber - 1,
tstate));
blockNumber = endBlockNumber;
}
return sb.toString();
}
private void throwIfInvalidBlockNumber(int blockNumber) {
Validate.checkWithinRange(blockNumber, "blockNumber", 0, this.numBlocks - 1);
checkWithinRange(blockNumber, "blockNumber", 0, numBlocks - 1);
}
private void throwIfInvalidOffset(long offset) {
Validate.checkWithinRange(offset, "offset", 0, this.fileSize - 1);
checkWithinRange(offset, "offset", 0, fileSize - 1);
}
}

View File

@ -17,12 +17,15 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
/**
* Provides read access to the underlying file one block at a time.
*
@ -31,8 +34,10 @@ import java.nio.ByteBuffer;
*/
public abstract class BlockManager implements Closeable {
// Information about each block of the underlying file.
private BlockData blockData;
/**
* Information about each block of the underlying file.
*/
private final BlockData blockData;
/**
* Constructs an instance of {@code BlockManager}.
@ -42,7 +47,7 @@ public abstract class BlockManager implements Closeable {
* @throws IllegalArgumentException if blockData is null.
*/
public BlockManager(BlockData blockData) {
Validate.checkNotNull(blockData, "blockData");
checkNotNull(blockData, "blockData");
this.blockData = blockData;
}
@ -53,7 +58,7 @@ public abstract class BlockManager implements Closeable {
* @return instance of {@code BlockData}.
*/
public BlockData getBlockData() {
return this.blockData;
return blockData;
}
/**
@ -70,12 +75,12 @@ public abstract class BlockManager implements Closeable {
* @throws IllegalArgumentException if blockNumber is negative.
*/
public BufferData get(int blockNumber) throws IOException {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
int size = this.blockData.getSize(blockNumber);
int size = blockData.getSize(blockNumber);
ByteBuffer buffer = ByteBuffer.allocate(size);
long startOffset = this.blockData.getStartOffset(blockNumber);
this.read(buffer, startOffset, size);
long startOffset = blockData.getStartOffset(blockNumber);
read(buffer, startOffset, size);
buffer.flip();
return new BufferData(blockNumber, buffer);
}
@ -100,7 +105,7 @@ public abstract class BlockManager implements Closeable {
* @throws IllegalArgumentException if data is null.
*/
public void release(BufferData data) {
Validate.checkNotNull(data, "data");
checkNotNull(data, "data");
// Do nothing because we allocate a new buffer each time.
}
@ -113,7 +118,7 @@ public abstract class BlockManager implements Closeable {
* @throws IllegalArgumentException if blockNumber is negative.
*/
public void requestPrefetch(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
// Do nothing because we do not support prefetches.
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.util.ArrayList;
import java.util.Arrays;
@ -31,6 +31,8 @@ import java.util.regex.Pattern;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
/**
* Block level operations performed on a file.
* This class is meant to be used by {@code BlockManager}.
@ -39,9 +41,12 @@ import org.slf4j.LoggerFactory;
* This class is used for debugging/logging. Calls to this class
* can be safely removed without affecting the overall operation.
*/
public class BlockOperations {
public final class BlockOperations {
private static final Logger LOG = LoggerFactory.getLogger(BlockOperations.class);
/**
* Operation kind.
*/
public enum Kind {
UNKNOWN("??", "unknown", false),
CANCEL_PREFETCHES("CP", "cancelPrefetches", false),
@ -68,7 +73,7 @@ public class BlockOperations {
private static Map<String, Kind> shortNameToKind = new HashMap<>();
public static Kind fromShortName(String shortName) {
if (shortNameToKind.size() == 0) {
if (shortNameToKind.isEmpty()) {
for (Kind kind : Kind.values()) {
shortNameToKind.put(kind.shortName, kind);
}
@ -89,30 +94,30 @@ public class BlockOperations {
}
public Kind getKind() {
return this.kind;
return kind;
}
public int getBlockNumber() {
return this.blockNumber;
return blockNumber;
}
public long getTimestamp() {
return this.timestamp;
return timestamp;
}
public void getSummary(StringBuilder sb) {
if (this.kind.hasBlock) {
sb.append(String.format("%s(%d)", this.kind.shortName, this.blockNumber));
if (kind.hasBlock) {
sb.append(String.format("%s(%d)", kind.shortName, blockNumber));
} else {
sb.append(String.format("%s", this.kind.shortName));
sb.append(String.format("%s", kind.shortName));
}
}
public String getDebugInfo() {
if (this.kind.hasBlock) {
return String.format("--- %s(%d)", this.kind.name, this.blockNumber);
if (kind.hasBlock) {
return String.format("--- %s(%d)", kind.name, blockNumber);
} else {
return String.format("... %s()", this.kind.name);
return String.format("... %s()", kind.name);
}
}
}
@ -137,7 +142,7 @@ public class BlockOperations {
}
public double duration() {
return (this.getTimestamp() - this.op.getTimestamp()) / 1e9;
return (getTimestamp() - op.getTimestamp()) / 1e9;
}
}
@ -149,11 +154,11 @@ public class BlockOperations {
}
public synchronized void setDebug(boolean state) {
this.debugMode = state;
debugMode = state;
}
private synchronized Operation add(Operation op) {
if (this.debugMode) {
if (debugMode) {
LOG.info(op.getDebugInfo());
}
ops.add(op);
@ -161,63 +166,63 @@ public class BlockOperations {
}
public Operation getPrefetched(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
return this.add(new Operation(Kind.GET_PREFETCHED, blockNumber));
return add(new Operation(Kind.GET_PREFETCHED, blockNumber));
}
public Operation getCached(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
return this.add(new Operation(Kind.GET_CACHED, blockNumber));
return add(new Operation(Kind.GET_CACHED, blockNumber));
}
public Operation getRead(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
return this.add(new Operation(Kind.GET_READ, blockNumber));
return add(new Operation(Kind.GET_READ, blockNumber));
}
public Operation release(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
return this.add(new Operation(Kind.RELEASE, blockNumber));
return add(new Operation(Kind.RELEASE, blockNumber));
}
public Operation requestPrefetch(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
return this.add(new Operation(Kind.REQUEST_PREFETCH, blockNumber));
return add(new Operation(Kind.REQUEST_PREFETCH, blockNumber));
}
public Operation prefetch(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
return this.add(new Operation(Kind.PREFETCH, blockNumber));
return add(new Operation(Kind.PREFETCH, blockNumber));
}
public Operation cancelPrefetches() {
return this.add(new Operation(Kind.CANCEL_PREFETCHES, -1));
return add(new Operation(Kind.CANCEL_PREFETCHES, -1));
}
public Operation close() {
return this.add(new Operation(Kind.CLOSE, -1));
return add(new Operation(Kind.CLOSE, -1));
}
public Operation requestCaching(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
return this.add(new Operation(Kind.REQUEST_CACHING, blockNumber));
return add(new Operation(Kind.REQUEST_CACHING, blockNumber));
}
public Operation addToCache(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
return this.add(new Operation(Kind.CACHE_PUT, blockNumber));
return add(new Operation(Kind.CACHE_PUT, blockNumber));
}
public Operation end(Operation op) {
return this.add(new End(op));
return add(new End(op));
}
private static void append(StringBuilder sb, String format, Object... args) {
@ -226,7 +231,7 @@ public class BlockOperations {
public synchronized String getSummary(boolean showDebugInfo) {
StringBuilder sb = new StringBuilder();
for (Operation op : this.ops) {
for (Operation op : ops) {
if (op != null) {
if (showDebugInfo) {
sb.append(op.getDebugInfo());
@ -239,14 +244,14 @@ public class BlockOperations {
}
sb.append("\n");
this.getDurationInfo(sb);
getDurationInfo(sb);
return sb.toString();
}
public synchronized void getDurationInfo(StringBuilder sb) {
Map<Kind, DoubleSummaryStatistics> durations = new HashMap<>();
for (Operation op : this.ops) {
for (Operation op : ops) {
if (op instanceof End) {
End endOp = (End) op;
DoubleSummaryStatistics stats = durations.get(endOp.getKind());
@ -293,7 +298,7 @@ public class BlockOperations {
Map<Integer, List<Operation>> blockOps = new HashMap<>();
// Group-by block number.
for (Operation op : this.ops) {
for (Operation op : ops) {
if (op.blockNumber < 0) {
continue;
}
@ -352,11 +357,11 @@ public class BlockOperations {
}
}
if (prefetchedNotUsed.size() > 0) {
if (!prefetchedNotUsed.isEmpty()) {
append(sb, "Prefetched but not used: %s\n", getIntList(prefetchedNotUsed));
}
if (cachedNotUsed.size() > 0) {
if (!cachedNotUsed.isEmpty()) {
append(sb, "Cached but not used: %s\n", getIntList(cachedNotUsed));
}
}
@ -410,7 +415,7 @@ public class BlockOperations {
}
if (op == null) {
LOG.warn("Start op not found: %s(%d)", endKind, blockNumber);
LOG.warn("Start op not found: {}({})", endKind, blockNumber);
}
}
}

View File

@ -17,26 +17,34 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.util.Collections;
import java.util.IdentityHashMap;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
/**
* Manages a fixed pool of resources.
*
* Avoids creating a new resource if a previously created instance is already available.
*/
public abstract class BoundedResourcePool<T> extends ResourcePool<T> {
// The size of this pool. Fixed at creation time.
/**
* The size of this pool. Fixed at creation time.
*/
private final int size;
// Items currently available in the pool.
/**
* Items currently available in the pool.
*/
private ArrayBlockingQueue<T> items;
// Items that have been created so far (regardless of whether they are currently available).
/**
* Items that have been created so far (regardless of whether they are currently available).
*/
private Set<T> createdItems;
/**
@ -50,7 +58,7 @@ public abstract class BoundedResourcePool<T> extends ResourcePool<T> {
Validate.checkPositiveInteger(size, "size");
this.size = size;
this.items = new ArrayBlockingQueue<T>(size);
this.items = new ArrayBlockingQueue<>(size);
// The created items are identified based on their object reference.
this.createdItems = Collections.newSetFromMap(new IdentityHashMap<T, Boolean>());
@ -79,41 +87,40 @@ public abstract class BoundedResourcePool<T> extends ResourcePool<T> {
*/
@Override
public void release(T item) {
Validate.checkNotNull(item, "item");
checkNotNull(item, "item");
synchronized (this.createdItems) {
if (!this.createdItems.contains(item)) {
synchronized (createdItems) {
if (!createdItems.contains(item)) {
throw new IllegalArgumentException("This item is not a part of this pool");
}
}
// Return if this item was released earlier.
// We cannot use this.items.contains() because that check is not based on reference equality.
for (T entry : this.items) {
// We cannot use items.contains() because that check is not based on reference equality.
for (T entry : items) {
if (entry == item) {
return;
}
}
try {
this.items.put(item);
return;
items.put(item);
} catch (InterruptedException e) {
throw new IllegalStateException("release() should never block");
throw new IllegalStateException("release() should never block", e);
}
}
@Override
public synchronized void close() {
for (T item : this.createdItems) {
this.close(item);
for (T item : createdItems) {
close(item);
}
this.items.clear();
this.items = null;
items.clear();
items = null;
this.createdItems.clear();
this.createdItems = null;
createdItems.clear();
createdItems = null;
}
/**
@ -124,16 +131,22 @@ public abstract class BoundedResourcePool<T> extends ResourcePool<T> {
// Do nothing in this class. Allow overriding classes to take any cleanup action.
}
// Number of items created so far. Mostly for testing purposes.
/**
* Number of items created so far. Mostly for testing purposes.
* @return the count.
*/
public int numCreated() {
synchronized (this.createdItems) {
return this.createdItems.size();
synchronized (createdItems) {
return createdItems.size();
}
}
// Number of items available to be acquired. Mostly for testing purposes.
/**
* Number of items available to be acquired. Mostly for testing purposes.
* @return the number available.
*/
public synchronized int numAvailable() {
return (this.size - this.numCreated()) + this.items.size();
return (size - numCreated()) + items.size();
}
// For debugging purposes.
@ -141,7 +154,7 @@ public abstract class BoundedResourcePool<T> extends ResourcePool<T> {
public synchronized String toString() {
return String.format(
"size = %d, #created = %d, #in-queue = %d, #available = %d",
this.size, this.numCreated(), this.items.size(), this.numAvailable());
size, numCreated(), items.size(), numAvailable());
}
/**
@ -153,16 +166,16 @@ public abstract class BoundedResourcePool<T> extends ResourcePool<T> {
// Prefer reusing an item if one is available.
// That avoids unnecessarily creating new instances.
T result = this.items.poll();
T result = items.poll();
if (result != null) {
return result;
}
synchronized (this.createdItems) {
synchronized (createdItems) {
// Create a new instance if allowed by the capacity of this pool.
if (this.createdItems.size() < this.size) {
T item = this.createNew();
this.createdItems.add(item);
if (createdItems.size() < size) {
T item = createNew();
createdItems.add(item);
return item;
}
}
@ -170,7 +183,7 @@ public abstract class BoundedResourcePool<T> extends ResourcePool<T> {
if (canBlock) {
try {
// Block for an instance to be available.
return this.items.take();
return items.take();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return null;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@ -38,42 +38,65 @@ import org.slf4j.LoggerFactory;
* Holding all of the state associated with a block allows us to validate and control
* state transitions in a synchronized fashion.
*/
public class BufferData {
public final class BufferData {
private static final Logger LOG = LoggerFactory.getLogger(BufferData.class);
public enum State {
// Unknown / invalid state.
/**
* Unknown / invalid state.
*/
UNKNOWN,
// Buffer has been acquired but has no data.
/**
* Buffer has been acquired but has no data.
*/
BLANK,
// This block is being prefetched.
/**
* This block is being prefetched.
*/
PREFETCHING,
// This block is being added to the local cache.
/**
* This block is being added to the local cache.
*/
CACHING,
// This block has data and is ready to be read.
/**
* This block has data and is ready to be read.
*/
READY,
// This block is no longer in-use and should not be used once in this state.
/**
* This block is no longer in-use and should not be used once in this state.
*/
DONE
}
// Number of the block associated with this buffer.
/**
* Number of the block associated with this buffer.
*/
private final int blockNumber;
// The buffer associated with this block.
/**
* The buffer associated with this block.
*/
private ByteBuffer buffer;
// Current state of this block.
/**
* Current state of this block.
*/
private volatile State state;
// Future of the action being performed on this block (eg, prefetching or caching).
/**
* Future of the action being performed on this block (eg, prefetching or caching).
*/
private Future<Void> action;
// Checksum of the buffer contents once in READY state.
/**
* Checksum of the buffer contents once in READY state.
*/
private long checksum = 0;
/**
@ -210,7 +233,6 @@ public class BufferData {
/**
* Updates the current state to the specified value.
* Asserts that the current state is as expected.
*
* @param newState the state to transition to.
* @param expectedCurrentState the collection of states from which
* transition to {@code newState} is allowed.
@ -218,7 +240,8 @@ public class BufferData {
* @throws IllegalArgumentException if newState is null.
* @throws IllegalArgumentException if expectedCurrentState is null.
*/
public synchronized void updateState(State newState, State... expectedCurrentState) {
public synchronized void updateState(State newState,
State... expectedCurrentState) {
Validate.checkNotNull(newState, "newState");
Validate.checkNotNull(expectedCurrentState, "expectedCurrentState");
@ -246,7 +269,8 @@ public class BufferData {
}
String message = String.format(
"Expected buffer state to be '%s' but found: %s", String.join(" or ", statesStr), this);
"Expected buffer state to be '%s' but found: %s",
String.join(" or ", statesStr), this);
throw new IllegalStateException(message);
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.io.Closeable;
import java.nio.ByteBuffer;
@ -32,45 +32,64 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkState;
import static org.apache.hadoop.util.Preconditions.checkArgument;
import static org.apache.hadoop.util.Preconditions.checkNotNull;
/**
* Manages a fixed pool of {@code ByteBuffer} instances.
*
* <p>
* Avoids creating a new buffer if a previously created buffer is already available.
*/
public class BufferPool implements Closeable {
private static final Logger LOG = LoggerFactory.getLogger(BufferPool.class);
// Max number of buffers in this pool.
/**
* Max number of buffers in this pool.
*/
private final int size;
// Size in bytes of each buffer.
/**
* Size in bytes of each buffer.
*/
private final int bufferSize;
// Invariants for internal state.
// -- a buffer is either in this.pool or in this.allocated
// -- transition between this.pool <==> this.allocated must be atomic
// -- only one buffer allocated for a given blockNumber
/*
Invariants for internal state.
-- a buffer is either in this.pool or in this.allocated
-- transition between this.pool <==> this.allocated must be atomic
-- only one buffer allocated for a given blockNumber
*/
// Underlying bounded resource pool.
/**
* Underlying bounded resource pool.
*/
private BoundedResourcePool<ByteBuffer> pool;
// Allows associating metadata to each buffer in the pool.
/**
* Allows associating metadata to each buffer in the pool.
*/
private Map<BufferData, ByteBuffer> allocated;
/**
* Prefetching stats.
*/
private PrefetchingStatistics prefetchingStatistics;
/**
* Initializes a new instance of the {@code BufferPool} class.
*
* @param size number of buffer in this pool.
* @param bufferSize size in bytes of each buffer.
* @param prefetchingStatistics statistics for this stream.
*
* @throws IllegalArgumentException if size is zero or negative.
* @throws IllegalArgumentException if bufferSize is zero or negative.
*/
public BufferPool(int size, int bufferSize, PrefetchingStatistics prefetchingStatistics) {
public BufferPool(int size,
int bufferSize,
PrefetchingStatistics prefetchingStatistics) {
Validate.checkPositiveInteger(size, "size");
Validate.checkPositiveInteger(bufferSize, "bufferSize");
@ -79,29 +98,27 @@ public class BufferPool implements Closeable {
this.allocated = new IdentityHashMap<BufferData, ByteBuffer>();
this.prefetchingStatistics = requireNonNull(prefetchingStatistics);
this.pool = new BoundedResourcePool<ByteBuffer>(size) {
@Override
public ByteBuffer createNew() {
ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
prefetchingStatistics.memoryAllocated(bufferSize);
return buffer;
}
};
@Override
public ByteBuffer createNew() {
ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
prefetchingStatistics.memoryAllocated(bufferSize);
return buffer;
}
};
}
/**
* Gets a list of all blocks in this pool.
*
* @return a list of all blocks in this pool.
*/
public List<BufferData> getAll() {
synchronized (this.allocated) {
return Collections.unmodifiableList(new ArrayList<BufferData>(this.allocated.keySet()));
synchronized (allocated) {
return Collections.unmodifiableList(new ArrayList<>(allocated.keySet()));
}
}
/**
* Acquires a {@code ByteBuffer}; blocking if necessary until one becomes available.
*
* @param blockNumber the id of the block to acquire.
* @return the acquired block's {@code BufferData}.
*/
@ -113,43 +130,46 @@ public class BufferPool implements Closeable {
do {
if (retryer.updateStatus()) {
LOG.warn("waiting to acquire block: {}", blockNumber);
LOG.info("state = {}", this.toString());
this.releaseReadyBlock(blockNumber);
if (LOG.isDebugEnabled()) {
LOG.debug("waiting to acquire block: {}", blockNumber);
LOG.debug("state = {}", this);
}
releaseReadyBlock(blockNumber);
}
data = this.tryAcquire(blockNumber);
data = tryAcquire(blockNumber);
}
while ((data == null) && retryer.continueRetry());
if (data != null) {
return data;
} else {
String message = String.format("Wait failed for acquire(%d)", blockNumber);
String message =
String.format("Wait failed for acquire(%d)", blockNumber);
throw new IllegalStateException(message);
}
}
/**
* Acquires a buffer if one is immediately available. Otherwise returns null.
*
* @param blockNumber the id of the block to try acquire.
* @return the acquired block's {@code BufferData} or null.
*/
public synchronized BufferData tryAcquire(int blockNumber) {
return this.acquireHelper(blockNumber, false);
return acquireHelper(blockNumber, false);
}
private synchronized BufferData acquireHelper(int blockNumber, boolean canBlock) {
Validate.checkNotNegative(blockNumber, "blockNumber");
private synchronized BufferData acquireHelper(int blockNumber,
boolean canBlock) {
checkNotNegative(blockNumber, "blockNumber");
this.releaseDoneBlocks();
releaseDoneBlocks();
BufferData data = this.find(blockNumber);
BufferData data = find(blockNumber);
if (data != null) {
return data;
}
ByteBuffer buffer = canBlock ? this.pool.acquire() : this.pool.tryAcquire();
ByteBuffer buffer = canBlock ? pool.acquire() : pool.tryAcquire();
if (buffer == null) {
return null;
}
@ -157,10 +177,10 @@ public class BufferPool implements Closeable {
buffer.clear();
data = new BufferData(blockNumber, buffer.duplicate());
synchronized (this.allocated) {
Validate.checkState(this.find(blockNumber) == null, "buffer data already exists");
synchronized (allocated) {
checkState(find(blockNumber) == null, "buffer data already exists");
this.allocated.put(data, buffer);
allocated.put(data, buffer);
}
return data;
@ -170,9 +190,9 @@ public class BufferPool implements Closeable {
* Releases resources for any blocks marked as 'done'.
*/
private synchronized void releaseDoneBlocks() {
for (BufferData data : this.getAll()) {
for (BufferData data : getAll()) {
if (data.stateEqualsOneOf(BufferData.State.DONE)) {
this.release(data);
release(data);
}
}
}
@ -184,12 +204,13 @@ public class BufferPool implements Closeable {
*/
private synchronized void releaseReadyBlock(int blockNumber) {
BufferData releaseTarget = null;
for (BufferData data : this.getAll()) {
for (BufferData data : getAll()) {
if (data.stateEqualsOneOf(BufferData.State.READY)) {
if (releaseTarget == null) {
releaseTarget = data;
} else {
if (distance(data, blockNumber) > distance(releaseTarget, blockNumber)) {
if (distance(data, blockNumber) > distance(releaseTarget,
blockNumber)) {
releaseTarget = data;
}
}
@ -208,36 +229,34 @@ public class BufferPool implements Closeable {
/**
* Releases a previously acquired resource.
*
* @param data the {@code BufferData} instance to release.
*
* @throws IllegalArgumentException if data is null.
* @throws IllegalArgumentException if data cannot be released due to its state.
*/
public synchronized void release(BufferData data) {
Validate.checkNotNull(data, "data");
checkNotNull(data, "data");
synchronized (data) {
Validate.checkArgument(
this.canRelease(data),
checkArgument(
canRelease(data),
String.format("Unable to release buffer: %s", data));
ByteBuffer buffer = this.allocated.get(data);
ByteBuffer buffer = allocated.get(data);
if (buffer == null) {
// Likely released earlier.
return;
}
buffer.clear();
this.pool.release(buffer);
this.allocated.remove(data);
pool.release(buffer);
allocated.remove(data);
}
this.releaseDoneBlocks();
releaseDoneBlocks();
}
@Override
public synchronized void close() {
for (BufferData data : this.getAll()) {
for (BufferData data : getAll()) {
Future<Void> actionFuture = data.getActionFuture();
if (actionFuture != null) {
actionFuture.cancel(true);
@ -259,10 +278,11 @@ public class BufferPool implements Closeable {
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(this.pool.toString());
sb.append(pool.toString());
sb.append("\n");
List<BufferData> allData = new ArrayList<>(this.getAll());
Collections.sort(allData, (d1, d2) -> d1.getBlockNumber() - d2.getBlockNumber());
List<BufferData> allData = new ArrayList<>(getAll());
Collections.sort(allData,
(d1, d2) -> d1.getBlockNumber() - d2.getBlockNumber());
for (BufferData data : allData) {
sb.append(data.toString());
sb.append("\n");
@ -273,18 +293,18 @@ public class BufferPool implements Closeable {
// Number of ByteBuffers created so far.
public synchronized int numCreated() {
return this.pool.numCreated();
return pool.numCreated();
}
// Number of ByteBuffers available to be acquired.
public synchronized int numAvailable() {
this.releaseDoneBlocks();
return this.pool.numAvailable();
releaseDoneBlocks();
return pool.numAvailable();
}
private BufferData find(int blockNumber) {
synchronized (this.allocated) {
for (BufferData data : this.allocated.keySet()) {
synchronized (allocated) {
for (BufferData data : allocated.keySet()) {
if ((data.getBlockNumber() == blockNumber)
&& !data.stateEqualsOneOf(BufferData.State.DONE)) {
return data;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
@ -37,6 +37,7 @@ import org.apache.hadoop.fs.statistics.DurationTracker;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
@ -47,33 +48,49 @@ public abstract class CachingBlockManager extends BlockManager {
private static final Logger LOG = LoggerFactory.getLogger(CachingBlockManager.class);
private static final int TIMEOUT_MINUTES = 60;
// Asynchronous tasks are performed in this pool.
/**
* Asynchronous tasks are performed in this pool.
*/
private final ExecutorServiceFuturePool futurePool;
// Pool of shared ByteBuffer instances.
/**
* Pool of shared ByteBuffer instances.
*/
private BufferPool bufferPool;
// Size of the in-memory cache in terms of number of blocks.
// Total memory consumption is up to bufferPoolSize * blockSize.
/**
* Size of the in-memory cache in terms of number of blocks.
* Total memory consumption is up to bufferPoolSize * blockSize.
*/
private final int bufferPoolSize;
// Local block cache.
/**
* Local block cache.
*/
private BlockCache cache;
// Error counts. For testing purposes.
/**
* Error counts. For testing purposes.
*/
private final AtomicInteger numCachingErrors;
private final AtomicInteger numReadErrors;
// Operations performed by this block manager.
/**
* Operations performed by this block manager.
*/
private final BlockOperations ops;
private boolean closed;
// If a single caching operation takes more than this time (in seconds),
// we disable caching to prevent further perf degradation due to caching.
/**
* If a single caching operation takes more than this time (in seconds),
* we disable caching to prevent further perf degradation due to caching.
*/
private static final int SLOW_CACHING_THRESHOLD = 5;
// Once set to true, any further caching requests will be ignored.
/**
* Once set to true, any further caching requests will be ignored.
*/
private final AtomicBoolean cachingDisabled;
private final PrefetchingStatistics prefetchingStatistics;
@ -121,21 +138,21 @@ public abstract class CachingBlockManager extends BlockManager {
*/
@Override
public BufferData get(int blockNumber) throws IOException {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
BufferData data = null;
final int maxRetryDelayMs = this.bufferPoolSize * 120 * 1000;
BufferData data;
final int maxRetryDelayMs = bufferPoolSize * 120 * 1000;
final int statusUpdateDelayMs = 120 * 1000;
Retryer retryer = new Retryer(10, maxRetryDelayMs, statusUpdateDelayMs);
boolean done;
do {
if (this.closed) {
if (closed) {
throw new IOException("this stream is already closed");
}
data = this.bufferPool.acquire(blockNumber);
done = this.getInternal(data);
data = bufferPool.acquire(blockNumber);
done = getInternal(data);
if (retryer.updateStatus()) {
LOG.warn("waiting to get block: {}", blockNumber);
@ -174,13 +191,13 @@ public abstract class CachingBlockManager extends BlockManager {
int blockNumber = data.getBlockNumber();
if (data.getState() == BufferData.State.READY) {
BlockOperations.Operation op = this.ops.getPrefetched(blockNumber);
this.ops.end(op);
BlockOperations.Operation op = ops.getPrefetched(blockNumber);
ops.end(op);
return true;
}
data.throwIfStateIncorrect(BufferData.State.BLANK);
this.read(data);
read(data);
return true;
}
}
@ -192,37 +209,37 @@ public abstract class CachingBlockManager extends BlockManager {
*/
@Override
public void release(BufferData data) {
if (this.closed) {
if (closed) {
return;
}
Validate.checkNotNull(data, "data");
BlockOperations.Operation op = this.ops.release(data.getBlockNumber());
this.bufferPool.release(data);
this.ops.end(op);
BlockOperations.Operation op = ops.release(data.getBlockNumber());
bufferPool.release(data);
ops.end(op);
}
@Override
public synchronized void close() {
if (this.closed) {
if (closed) {
return;
}
this.closed = true;
closed = true;
final BlockOperations.Operation op = this.ops.close();
final BlockOperations.Operation op = ops.close();
// Cancel any prefetches in progress.
this.cancelPrefetches();
cancelPrefetches();
cleanupWithLogger(LOG, this.cache);
cleanupWithLogger(LOG, cache);
this.ops.end(op);
LOG.info(this.ops.getSummary(false));
ops.end(op);
LOG.info(ops.getSummary(false));
this.bufferPool.close();
this.bufferPool = null;
bufferPool.close();
bufferPool = null;
}
/**
@ -233,14 +250,14 @@ public abstract class CachingBlockManager extends BlockManager {
*/
@Override
public void requestPrefetch(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
checkNotNegative(blockNumber, "blockNumber");
if (this.closed) {
if (closed) {
return;
}
// We initiate a prefetch only if we can acquire a buffer from the shared pool.
BufferData data = this.bufferPool.tryAcquire(blockNumber);
BufferData data = bufferPool.tryAcquire(blockNumber);
if (data == null) {
return;
}
@ -258,11 +275,11 @@ public abstract class CachingBlockManager extends BlockManager {
return;
}
BlockOperations.Operation op = this.ops.requestPrefetch(blockNumber);
BlockOperations.Operation op = ops.requestPrefetch(blockNumber);
PrefetchTask prefetchTask = new PrefetchTask(data, this, Instant.now());
Future<Void> prefetchFuture = this.futurePool.executeFunction(prefetchTask);
Future<Void> prefetchFuture = futurePool.executeFunction(prefetchTask);
data.setPrefetch(prefetchFuture);
this.ops.end(op);
ops.end(op);
}
}
@ -271,21 +288,21 @@ public abstract class CachingBlockManager extends BlockManager {
*/
@Override
public void cancelPrefetches() {
BlockOperations.Operation op = this.ops.cancelPrefetches();
BlockOperations.Operation op = ops.cancelPrefetches();
for (BufferData data : this.bufferPool.getAll()) {
for (BufferData data : bufferPool.getAll()) {
// We add blocks being prefetched to the local cache so that the prefetch is not wasted.
if (data.stateEqualsOneOf(BufferData.State.PREFETCHING, BufferData.State.READY)) {
this.requestCaching(data);
requestCaching(data);
}
}
this.ops.end(op);
ops.end(op);
}
private void read(BufferData data) throws IOException {
synchronized (data) {
this.readBlock(data, false, BufferData.State.BLANK);
readBlock(data, false, BufferData.State.BLANK);
}
}
@ -293,7 +310,7 @@ public abstract class CachingBlockManager extends BlockManager {
synchronized (data) {
prefetchingStatistics.executorAcquired(
Duration.between(taskQueuedStartTime, Instant.now()));
this.readBlock(
readBlock(
data,
true,
BufferData.State.PREFETCHING,
@ -304,7 +321,7 @@ public abstract class CachingBlockManager extends BlockManager {
private void readBlock(BufferData data, boolean isPrefetch, BufferData.State... expectedState)
throws IOException {
if (this.closed) {
if (closed) {
return;
}
@ -323,25 +340,25 @@ public abstract class CachingBlockManager extends BlockManager {
int blockNumber = data.getBlockNumber();
// Prefer reading from cache over reading from network.
if (this.cache.containsBlock(blockNumber)) {
op = this.ops.getCached(blockNumber);
this.cache.get(blockNumber, data.getBuffer());
if (cache.containsBlock(blockNumber)) {
op = ops.getCached(blockNumber);
cache.get(blockNumber, data.getBuffer());
data.setReady(expectedState);
return;
}
if (isPrefetch) {
tracker = prefetchingStatistics.prefetchOperationStarted();
op = this.ops.prefetch(data.getBlockNumber());
op = ops.prefetch(data.getBlockNumber());
} else {
op = this.ops.getRead(data.getBlockNumber());
op = ops.getRead(data.getBlockNumber());
}
long offset = this.getBlockData().getStartOffset(data.getBlockNumber());
int size = this.getBlockData().getSize(data.getBlockNumber());
long offset = getBlockData().getStartOffset(data.getBlockNumber());
int size = getBlockData().getSize(data.getBlockNumber());
ByteBuffer buffer = data.getBuffer();
buffer.clear();
this.read(buffer, offset, size);
read(buffer, offset, size);
buffer.flip();
data.setReady(expectedState);
} catch (Exception e) {
@ -352,12 +369,12 @@ public abstract class CachingBlockManager extends BlockManager {
tracker.failed();
}
this.numReadErrors.incrementAndGet();
numReadErrors.incrementAndGet();
data.setDone();
throw e;
} finally {
if (op != null) {
this.ops.end(op);
ops.end(op);
}
if (isPrefetch) {
@ -387,7 +404,7 @@ public abstract class CachingBlockManager extends BlockManager {
@Override
public Void get() {
try {
this.blockManager.prefetch(data, taskQueuedStartTime);
blockManager.prefetch(data, taskQueuedStartTime);
} catch (Exception e) {
LOG.error("error during prefetch", e);
}
@ -409,11 +426,11 @@ public abstract class CachingBlockManager extends BlockManager {
*/
@Override
public void requestCaching(BufferData data) {
if (this.closed) {
if (closed) {
return;
}
if (this.cachingDisabled.get()) {
if (cachingDisabled.get()) {
data.setDone();
return;
}
@ -431,14 +448,14 @@ public abstract class CachingBlockManager extends BlockManager {
return;
}
if (this.cache.containsBlock(data.getBlockNumber())) {
if (cache.containsBlock(data.getBlockNumber())) {
data.setDone();
return;
}
BufferData.State state = data.getState();
BlockOperations.Operation op = this.ops.requestCaching(data.getBlockNumber());
BlockOperations.Operation op = ops.requestCaching(data.getBlockNumber());
Future<Void> blockFuture;
if (state == BufferData.State.PREFETCHING) {
blockFuture = data.getActionFuture();
@ -449,9 +466,9 @@ public abstract class CachingBlockManager extends BlockManager {
}
CachePutTask task = new CachePutTask(data, blockFuture, this, Instant.now());
Future<Void> actionFuture = this.futurePool.executeFunction(task);
Future<Void> actionFuture = futurePool.executeFunction(task);
data.setCaching(actionFuture);
this.ops.end(op);
ops.end(op);
}
}
@ -460,11 +477,11 @@ public abstract class CachingBlockManager extends BlockManager {
prefetchingStatistics.executorAcquired(
Duration.between(taskQueuedStartTime, Instant.now()));
if (this.closed) {
if (closed) {
return;
}
if (this.cachingDisabled.get()) {
if (cachingDisabled.get()) {
data.setDone();
return;
}
@ -481,7 +498,7 @@ public abstract class CachingBlockManager extends BlockManager {
return;
}
if (this.cachingDisabled.get()) {
if (cachingDisabled.get()) {
data.setDone();
return;
}
@ -494,27 +511,27 @@ public abstract class CachingBlockManager extends BlockManager {
return;
}
if (this.cache.containsBlock(data.getBlockNumber())) {
if (cache.containsBlock(data.getBlockNumber())) {
data.setDone();
return;
}
op = this.ops.addToCache(data.getBlockNumber());
op = ops.addToCache(data.getBlockNumber());
ByteBuffer buffer = data.getBuffer().duplicate();
buffer.rewind();
this.cachePut(data.getBlockNumber(), buffer);
cachePut(data.getBlockNumber(), buffer);
data.setDone();
} catch (Exception e) {
this.numCachingErrors.incrementAndGet();
numCachingErrors.incrementAndGet();
String message = String.format("error adding block to cache after wait: %s", data);
LOG.error(message, e);
data.setDone();
}
if (op != null) {
BlockOperations.End endOp = (BlockOperations.End) this.ops.end(op);
BlockOperations.End endOp = (BlockOperations.End) ops.end(op);
if (endOp.duration() > SLOW_CACHING_THRESHOLD) {
if (!this.cachingDisabled.getAndSet(true)) {
if (!cachingDisabled.getAndSet(true)) {
String message = String.format(
"Caching disabled because of slow operation (%.1f sec)", endOp.duration());
LOG.warn(message);
@ -529,11 +546,11 @@ public abstract class CachingBlockManager extends BlockManager {
}
protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException {
if (this.closed) {
if (closed) {
return;
}
this.cache.put(blockNumber, buffer);
cache.put(blockNumber, buffer);
}
private static class CachePutTask implements Supplier<Void> {
@ -560,7 +577,7 @@ public abstract class CachingBlockManager extends BlockManager {
@Override
public Void get() {
this.blockManager.addToCacheAndRelease(this.data, this.blockFuture, taskQueuedStartTime);
blockManager.addToCacheAndRelease(data, blockFuture, taskQueuedStartTime);
return null;
}
}
@ -571,7 +588,7 @@ public abstract class CachingBlockManager extends BlockManager {
* @return the number of available buffers.
*/
public int numAvailable() {
return this.bufferPool.numAvailable();
return bufferPool.numAvailable();
}
/**
@ -580,7 +597,7 @@ public abstract class CachingBlockManager extends BlockManager {
* @return the number of cached buffers.
*/
public int numCached() {
return this.cache.size();
return cache.size();
}
/**
@ -589,7 +606,7 @@ public abstract class CachingBlockManager extends BlockManager {
* @return the number of errors encountered when caching.
*/
public int numCachingErrors() {
return this.numCachingErrors.get();
return numCachingErrors.get();
}
/**
@ -598,11 +615,11 @@ public abstract class CachingBlockManager extends BlockManager {
* @return the number of errors encountered when reading.
*/
public int numReadErrors() {
return this.numReadErrors.get();
return numReadErrors.get();
}
BufferData getData(int blockNumber) {
return this.bufferPool.tryAcquire(blockNumber);
return bufferPool.tryAcquire(blockNumber);
}
@Override
@ -610,11 +627,11 @@ public abstract class CachingBlockManager extends BlockManager {
StringBuilder sb = new StringBuilder();
sb.append("cache(");
sb.append(this.cache.toString());
sb.append(cache.toString());
sb.append("); ");
sb.append("pool: ");
sb.append(this.bufferPool.toString());
sb.append(bufferPool.toString());
return sb.toString();
}

View File

@ -0,0 +1,80 @@
/*
* 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.prefetch;
import java.time.Duration;
import org.apache.hadoop.fs.statistics.DurationTracker;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker;
/**
* Empty implementation of the prefetching statistics interface.
*/
public final class EmptyPrefetchingStatistics
implements PrefetchingStatistics {
private static final EmptyPrefetchingStatistics
EMPTY_PREFETCHING_STATISTICS =
new EmptyPrefetchingStatistics();
private EmptyPrefetchingStatistics() {
}
public static EmptyPrefetchingStatistics getInstance() {
return EMPTY_PREFETCHING_STATISTICS;
}
@Override
public DurationTracker prefetchOperationStarted() {
return stubDurationTracker();
}
@Override
public void blockAddedToFileCache() {
}
@Override
public void blockRemovedFromFileCache() {
}
@Override
public void prefetchOperationCompleted() {
}
@Override
public void executorAcquired(Duration timeInQueue) {
}
@Override
public void memoryAllocated(int size) {
}
@Override
public void memoryFreed(int size) {
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.util.Locale;
import java.util.concurrent.ExecutorService;

View File

@ -17,10 +17,16 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.nio.ByteBuffer;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNegative;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkPositiveInteger;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkState;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkWithinRange;
/**
* Provides functionality related to tracking the position within a file.
*
@ -31,25 +37,38 @@ import java.nio.ByteBuffer;
* A file is made up of equal sized blocks. The last block may be of a smaller size.
* The size of a buffer associated with this file is typically the same as block size.
*/
public class FilePosition {
// Holds block based information about a file.
public final class FilePosition {
/**
* Holds block based information about a file.
*/
private BlockData blockData;
// Information about the buffer in use.
/**
* Information about the buffer in use.
*/
private BufferData data;
// Provides access to the underlying file.
/**
* Provides access to the underlying file.
*/
private ByteBuffer buffer;
// Start offset of the buffer relative to the start of a file.
/**
* Start offset of the buffer relative to the start of a file.
*/
private long bufferStartOffset;
// Offset where reading starts relative to the start of a file.
/**
* Offset where reading starts relative to the start of a file.
*/
private long readStartOffset;
// Read stats after a seek (mostly for debugging use).
private int numSingleByteReads;
private int numBytesRead;
private int numBufferReads;
/**
@ -62,11 +81,11 @@ public class FilePosition {
* @throws IllegalArgumentException if blockSize is zero or negative.
*/
public FilePosition(long fileSize, int blockSize) {
Validate.checkNotNegative(fileSize, "fileSize");
checkNotNegative(fileSize, "fileSize");
if (fileSize == 0) {
Validate.checkNotNegative(blockSize, "blockSize");
checkNotNegative(blockSize, "blockSize");
} else {
Validate.checkPositiveInteger(blockSize, "blockSize");
checkPositiveInteger(blockSize, "blockSize");
}
this.blockData = new BlockData(fileSize, blockSize);
@ -87,33 +106,35 @@ public class FilePosition {
* @throws IllegalArgumentException if readOffset is negative.
* @throws IllegalArgumentException if readOffset is outside the range [startOffset, buffer end].
*/
public void setData(BufferData bufferData, long startOffset, long readOffset) {
Validate.checkNotNull(bufferData, "bufferData");
Validate.checkNotNegative(startOffset, "startOffset");
Validate.checkNotNegative(readOffset, "readOffset");
Validate.checkWithinRange(
public void setData(BufferData bufferData,
long startOffset,
long readOffset) {
checkNotNull(bufferData, "bufferData");
checkNotNegative(startOffset, "startOffset");
checkNotNegative(readOffset, "readOffset");
checkWithinRange(
readOffset,
"readOffset",
startOffset,
startOffset + bufferData.getBuffer().limit() - 1);
this.data = bufferData;
this.buffer = bufferData.getBuffer().duplicate();
this.bufferStartOffset = startOffset;
this.readStartOffset = readOffset;
this.setAbsolute(readOffset);
data = bufferData;
buffer = bufferData.getBuffer().duplicate();
bufferStartOffset = startOffset;
readStartOffset = readOffset;
setAbsolute(readOffset);
this.resetReadStats();
resetReadStats();
}
public ByteBuffer buffer() {
throwIfInvalidBuffer();
return this.buffer;
return buffer;
}
public BufferData data() {
throwIfInvalidBuffer();
return this.data;
return data;
}
/**
@ -123,7 +144,7 @@ public class FilePosition {
*/
public long absolute() {
throwIfInvalidBuffer();
return this.bufferStartOffset + this.relative();
return bufferStartOffset + relative();
}
/**
@ -134,9 +155,9 @@ public class FilePosition {
* @return true if the given current position was updated, false otherwise.
*/
public boolean setAbsolute(long pos) {
if (this.isValid() && this.isWithinCurrentBuffer(pos)) {
int relativePos = (int) (pos - this.bufferStartOffset);
this.buffer.position(relativePos);
if (isValid() && isWithinCurrentBuffer(pos)) {
int relativePos = (int) (pos - bufferStartOffset);
buffer.position(relativePos);
return true;
} else {
return false;
@ -150,7 +171,7 @@ public class FilePosition {
*/
public int relative() {
throwIfInvalidBuffer();
return this.buffer.position();
return buffer.position();
}
/**
@ -161,8 +182,8 @@ public class FilePosition {
*/
public boolean isWithinCurrentBuffer(long pos) {
throwIfInvalidBuffer();
long bufferEndOffset = this.bufferStartOffset + this.buffer.limit() - 1;
return (pos >= this.bufferStartOffset) && (pos <= bufferEndOffset);
long bufferEndOffset = bufferStartOffset + buffer.limit() - 1;
return (pos >= bufferStartOffset) && (pos <= bufferEndOffset);
}
/**
@ -172,7 +193,7 @@ public class FilePosition {
*/
public int blockNumber() {
throwIfInvalidBuffer();
return this.blockData.getBlockNumber(this.bufferStartOffset);
return blockData.getBlockNumber(bufferStartOffset);
}
/**
@ -181,7 +202,7 @@ public class FilePosition {
* @return true if the current block is the last block in this file, false otherwise.
*/
public boolean isLastBlock() {
return this.blockData.isLastBlock(this.blockNumber());
return blockData.isLastBlock(blockNumber());
}
/**
@ -190,16 +211,16 @@ public class FilePosition {
* @return true if the current position is valid, false otherwise.
*/
public boolean isValid() {
return this.buffer != null;
return buffer != null;
}
/**
* Marks the current position as invalid.
*/
public void invalidate() {
this.buffer = null;
this.bufferStartOffset = -1;
this.data = null;
buffer = null;
bufferStartOffset = -1;
data = null;
}
/**
@ -209,7 +230,7 @@ public class FilePosition {
*/
public long bufferStartOffset() {
throwIfInvalidBuffer();
return this.bufferStartOffset;
return bufferStartOffset;
}
/**
@ -219,30 +240,30 @@ public class FilePosition {
*/
public boolean bufferFullyRead() {
throwIfInvalidBuffer();
return (this.bufferStartOffset == this.readStartOffset)
&& (this.relative() == this.buffer.limit())
&& (this.numBytesRead == this.buffer.limit());
return (bufferStartOffset == readStartOffset)
&& (relative() == buffer.limit())
&& (numBytesRead == buffer.limit());
}
public void incrementBytesRead(int n) {
this.numBytesRead += n;
numBytesRead += n;
if (n == 1) {
this.numSingleByteReads++;
numSingleByteReads++;
} else {
this.numBufferReads++;
numBufferReads++;
}
}
public int numBytesRead() {
return this.numBytesRead;
return numBytesRead;
}
public int numSingleByteReads() {
return this.numSingleByteReads;
return numSingleByteReads;
}
public int numBufferReads() {
return this.numBufferReads;
return numBufferReads;
}
private void resetReadStats() {
@ -253,21 +274,21 @@ public class FilePosition {
public String toString() {
StringBuilder sb = new StringBuilder();
if (this.buffer == null) {
if (buffer == null) {
sb.append("currentBuffer = null");
} else {
int pos = this.buffer.position();
int pos = buffer.position();
int val;
if (pos >= this.buffer.limit()) {
if (pos >= buffer.limit()) {
val = -1;
} else {
val = this.buffer.get(pos);
val = buffer.get(pos);
}
String currentBufferState =
String.format("%d at pos: %d, lim: %d", val, pos, this.buffer.limit());
String.format("%d at pos: %d, lim: %d", val, pos, buffer.limit());
sb.append(String.format(
"block: %d, pos: %d (CBuf: %s)%n",
this.blockNumber(), this.absolute(),
blockNumber(), absolute(),
currentBufferState));
sb.append("\n");
}
@ -275,8 +296,6 @@ public class FilePosition {
}
private void throwIfInvalidBuffer() {
if (!this.isValid()) {
Validate.checkState(buffer != null, "'buffer' must not be null");
}
checkState(buffer != null, "'buffer' must not be null");
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.time.Duration;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.io.Closeable;

View File

@ -17,22 +17,28 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkGreater;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkPositiveInteger;
/**
* Provides retry related functionality.
*/
public class Retryer {
// Maximum amount of delay (in ms) before retry fails.
/* Maximum amount of delay (in ms) before retry fails. */
private int maxDelay;
// Per retry delay (in ms).
/* Per retry delay (in ms). */
private int perRetryDelay;
// The time interval (in ms) at which status update would be made.
/**
* The time interval (in ms) at which status update would be made.
*/
private int statusUpdateInterval;
// Current delay.
/* Current delay. */
private int delay;
/**
@ -47,9 +53,9 @@ public class Retryer {
* @throws IllegalArgumentException if statusUpdateInterval is zero or negative.
*/
public Retryer(int perRetryDelay, int maxDelay, int statusUpdateInterval) {
Validate.checkPositiveInteger(perRetryDelay, "perRetryDelay");
Validate.checkGreater(maxDelay, "maxDelay", perRetryDelay, "perRetryDelay");
Validate.checkPositiveInteger(statusUpdateInterval, "statusUpdateInterval");
checkPositiveInteger(perRetryDelay, "perRetryDelay");
checkGreater(maxDelay, "maxDelay", perRetryDelay, "perRetryDelay");
checkPositiveInteger(statusUpdateInterval, "statusUpdateInterval");
this.perRetryDelay = perRetryDelay;
this.maxDelay = maxDelay;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.io.File;
import java.io.IOException;
@ -43,6 +43,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkNotNull;
/**
* Provides functionality necessary for caching blocks of data read from FileSystem.
@ -51,20 +52,26 @@ import static java.util.Objects.requireNonNull;
public class SingleFilePerBlockCache implements BlockCache {
private static final Logger LOG = LoggerFactory.getLogger(SingleFilePerBlockCache.class);
// Blocks stored in this cache.
private Map<Integer, Entry> blocks = new ConcurrentHashMap<>();
/**
* Blocks stored in this cache.
*/
private final Map<Integer, Entry> blocks = new ConcurrentHashMap<>();
// Number of times a block was read from this cache.
// Used for determining cache utilization factor.
/**
* Number of times a block was read from this cache.
* Used for determining cache utilization factor.
*/
private int numGets = 0;
private boolean closed;
private final PrefetchingStatistics prefetchingStatistics;
// Cache entry.
// Each block is stored as a separate file.
private static class Entry {
/**
* Cache entry.
* Each block is stored as a separate file.
*/
private static final class Entry {
private final int blockNumber;
private final Path path;
private final int size;
@ -81,7 +88,7 @@ public class SingleFilePerBlockCache implements BlockCache {
public String toString() {
return String.format(
"([%03d] %s: size = %d, checksum = %d)",
this.blockNumber, this.path, this.size, this.checksum);
blockNumber, path, size, checksum);
}
}
@ -99,7 +106,7 @@ public class SingleFilePerBlockCache implements BlockCache {
*/
@Override
public boolean containsBlock(int blockNumber) {
return this.blocks.containsKey(blockNumber);
return blocks.containsKey(blockNumber);
}
/**
@ -107,7 +114,7 @@ public class SingleFilePerBlockCache implements BlockCache {
*/
@Override
public Iterable<Integer> blocks() {
return Collections.unmodifiableList(new ArrayList<Integer>(this.blocks.keySet()));
return Collections.unmodifiableList(new ArrayList<>(blocks.keySet()));
}
/**
@ -115,7 +122,7 @@ public class SingleFilePerBlockCache implements BlockCache {
*/
@Override
public int size() {
return this.blocks.size();
return blocks.size();
}
/**
@ -125,15 +132,15 @@ public class SingleFilePerBlockCache implements BlockCache {
*/
@Override
public void get(int blockNumber, ByteBuffer buffer) throws IOException {
if (this.closed) {
if (closed) {
return;
}
Validate.checkNotNull(buffer, "buffer");
checkNotNull(buffer, "buffer");
Entry entry = this.getEntry(blockNumber);
Entry entry = getEntry(blockNumber);
buffer.clear();
this.readFile(entry.path, buffer);
readFile(entry.path, buffer);
buffer.rewind();
validateEntry(entry, buffer);
@ -154,11 +161,11 @@ public class SingleFilePerBlockCache implements BlockCache {
private Entry getEntry(int blockNumber) {
Validate.checkNotNegative(blockNumber, "blockNumber");
Entry entry = this.blocks.get(blockNumber);
Entry entry = blocks.get(blockNumber);
if (entry == null) {
throw new IllegalStateException(String.format("block %d not found in cache", blockNumber));
}
this.numGets++;
numGets++;
return entry;
}
@ -170,14 +177,14 @@ public class SingleFilePerBlockCache implements BlockCache {
*/
@Override
public void put(int blockNumber, ByteBuffer buffer) throws IOException {
if (this.closed) {
if (closed) {
return;
}
Validate.checkNotNull(buffer, "buffer");
checkNotNull(buffer, "buffer");
if (this.blocks.containsKey(blockNumber)) {
Entry entry = this.blocks.get(blockNumber);
if (blocks.containsKey(blockNumber)) {
Entry entry = blocks.get(blockNumber);
validateEntry(entry, buffer);
return;
}
@ -193,11 +200,11 @@ public class SingleFilePerBlockCache implements BlockCache {
throw new IllegalStateException(message);
}
this.writeFile(blockFilePath, buffer);
this.prefetchingStatistics.blockAddedToFileCache();
writeFile(blockFilePath, buffer);
prefetchingStatistics.blockAddedToFileCache();
long checksum = BufferData.getChecksum(buffer);
Entry entry = new Entry(blockNumber, blockFilePath, buffer.limit(), checksum);
this.blocks.put(blockNumber, entry);
blocks.put(blockNumber, entry);
}
private static final Set<? extends OpenOption> CREATE_OPTIONS =
@ -220,19 +227,19 @@ public class SingleFilePerBlockCache implements BlockCache {
@Override
public void close() throws IOException {
if (this.closed) {
if (closed) {
return;
}
this.closed = true;
closed = true;
LOG.info(this.getStats());
LOG.info(getStats());
int numFilesDeleted = 0;
for (Entry entry : this.blocks.values()) {
for (Entry entry : blocks.values()) {
try {
Files.deleteIfExists(entry.path);
this.prefetchingStatistics.blockRemovedFromFileCache();
prefetchingStatistics.blockRemovedFromFileCache();
numFilesDeleted++;
} catch (IOException e) {
// Ignore while closing so that we can delete as many cache files as possible.
@ -250,7 +257,7 @@ public class SingleFilePerBlockCache implements BlockCache {
sb.append("stats: ");
sb.append(getStats());
sb.append(", blocks:[");
sb.append(this.getIntList(this.blocks()));
sb.append(getIntList(blocks()));
sb.append("]");
return sb.toString();
}
@ -310,7 +317,7 @@ public class SingleFilePerBlockCache implements BlockCache {
StringBuilder sb = new StringBuilder();
sb.append(String.format(
"#entries = %d, #gets = %d",
this.blocks.size(), this.numGets));
blocks.size(), numGets));
return sb.toString();
}

View File

@ -17,25 +17,28 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Collection;
import static org.apache.hadoop.util.Preconditions.checkArgument;
/**
* A superset of Validate class in Apache commons lang3.
*
* <p>
* It provides consistent message strings for frequently encountered checks.
* That simplifies callers because they have to supply only the name of the argument
* that failed a check instead of having to supply the entire message.
*/
public final class Validate {
private Validate() {}
private Validate() {
}
/**
* Validates that the given reference argument is not null.
*
* @param obj the argument reference to validate.
* @param argName the name of the argument being validated.
*/
@ -45,7 +48,6 @@ public final class Validate {
/**
* Validates that the given integer argument is not zero or negative.
*
* @param value the argument value to validate
* @param argName the name of the argument being validated.
*/
@ -55,7 +57,6 @@ public final class Validate {
/**
* Validates that the given integer argument is not negative.
*
* @param value the argument value to validate
* @param argName the name of the argument being validated.
*/
@ -63,9 +64,8 @@ public final class Validate {
checkArgument(value >= 0, "'%s' must not be negative.", argName);
}
/*
/**
* Validates that the expression (that checks a required field is present) is true.
*
* @param isPresent indicates whether the given argument is present.
* @param argName the name of the argument being validated.
*/
@ -75,7 +75,6 @@ public final class Validate {
/**
* Validates that the expression (that checks a field is valid) is true.
*
* @param isValid indicates whether the given argument is valid.
* @param argName the name of the argument being validated.
*/
@ -85,101 +84,96 @@ public final class Validate {
/**
* Validates that the expression (that checks a field is valid) is true.
*
* @param isValid indicates whether the given argument is valid.
* @param argName the name of the argument being validated.
* @param validValues the list of values that are allowed.
*/
public static void checkValid(boolean isValid, String argName, String validValues) {
checkArgument(isValid, "'%s' is invalid. Valid values are: %s.", argName, validValues);
public static void checkValid(boolean isValid,
String argName,
String validValues) {
checkArgument(isValid, "'%s' is invalid. Valid values are: %s.", argName,
validValues);
}
/**
* Validates that the given string is not null and has non-zero length.
*
* @param arg the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(String arg, String argName) {
Validate.checkNotNull(arg, argName);
Validate.checkArgument(
arg.length() > 0,
checkNotNull(arg, argName);
checkArgument(
!arg.isEmpty(),
"'%s' must not be empty.",
argName);
}
/**
* Validates that the given array is not null and has at least one element.
*
* @param <T> the type of array's elements.
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static <T> void checkNotNullAndNotEmpty(T[] array, String argName) {
Validate.checkNotNull(array, argName);
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given array is not null and has at least one element.
*
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(byte[] array, String argName) {
Validate.checkNotNull(array, argName);
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given array is not null and has at least one element.
*
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(short[] array, String argName) {
Validate.checkNotNull(array, argName);
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given array is not null and has at least one element.
*
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(int[] array, String argName) {
Validate.checkNotNull(array, argName);
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given array is not null and has at least one element.
*
* @param array the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static void checkNotNullAndNotEmpty(long[] array, String argName) {
Validate.checkNotNull(array, argName);
checkNotNull(array, argName);
checkNotEmpty(array.length, argName);
}
/**
* Validates that the given buffer is not null and has non-zero capacity.
*
* @param <T> the type of iterable's elements.
* @param iter the argument reference to validate.
* @param argName the name of the argument being validated.
*/
public static <T> void checkNotNullAndNotEmpty(Iterable<T> iter, String argName) {
Validate.checkNotNull(iter, argName);
public static <T> void checkNotNullAndNotEmpty(Iterable<T> iter,
String argName) {
checkNotNull(iter, argName);
int minNumElements = iter.iterator().hasNext() ? 1 : 0;
checkNotEmpty(minNumElements, argName);
}
/**
* Validates that the given set is not null and has an exact number of items.
*
* @param <T> the type of collection's elements.
* @param collection the argument reference to validate.
* @param numElements the expected number of elements in the collection.
@ -187,7 +181,7 @@ public final class Validate {
*/
public static <T> void checkNotNullAndNumberOfElements(
Collection<T> collection, int numElements, String argName) {
Validate.checkNotNull(collection, argName);
checkNotNull(collection, argName);
checkArgument(
collection.size() == numElements,
"Number of elements in '%s' must be exactly %s, %s given.",
@ -199,7 +193,6 @@ public final class Validate {
/**
* Validates that the given two values are equal.
*
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
@ -221,7 +214,6 @@ public final class Validate {
/**
* Validates that the first value is an integer multiple of the second value.
*
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
@ -243,7 +235,6 @@ public final class Validate {
/**
* Validates that the first value is greater than the second value.
*
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
@ -265,7 +256,6 @@ public final class Validate {
/**
* Validates that the first value is greater than or equal to the second value.
*
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
@ -287,7 +277,6 @@ public final class Validate {
/**
* Validates that the first value is less than or equal to the second value.
*
* @param value1 the first value to check.
* @param value1Name the name of the first argument.
* @param value2 the second value to check.
@ -309,7 +298,6 @@ public final class Validate {
/**
* Validates that the given value is within the given range of values.
*
* @param value the value to check.
* @param valueName the name of the argument.
* @param minValueInclusive inclusive lower limit for the value.
@ -331,7 +319,6 @@ public final class Validate {
/**
* Validates that the given value is within the given range of values.
*
* @param value the value to check.
* @param valueName the name of the argument.
* @param minValueInclusive inclusive lower limit for the value.
@ -353,18 +340,17 @@ public final class Validate {
/**
* Validates that the given path exists.
*
* @param path the path to check.
* @param argName the name of the argument being validated.
*/
public static void checkPathExists(Path path, String argName) {
checkNotNull(path, argName);
checkArgument(Files.exists(path), "Path %s (%s) does not exist.", argName, path);
checkArgument(Files.exists(path), "Path %s (%s) does not exist.", argName,
path);
}
/**
* Validates that the given path exists and is a directory.
*
* @param path the path to check.
* @param argName the name of the argument being validated.
*/
@ -379,25 +365,33 @@ public final class Validate {
/**
* Validates that the given path exists and is a file.
*
* @param path the path to check.
* @param argName the name of the argument being validated.
*/
public static void checkPathExistsAsFile(Path path, String argName) {
checkPathExists(path, argName);
checkArgument(Files.isRegularFile(path), "Path %s (%s) must point to a file.", argName, path);
checkArgument(Files.isRegularFile(path),
"Path %s (%s) must point to a file.", argName, path);
}
public static void checkArgument(boolean expression, String format, Object... args) {
org.apache.commons.lang3.Validate.isTrue(expression, format, args);
}
public static void checkState(boolean expression, String format, Object... args) {
org.apache.commons.lang3.Validate.validState(expression, format, args);
/**
* Check state.
* @param expression expression which must hold.
* @param format format string
* @param args arguments for the error string
* @throws IllegalStateException if the state is not valid.
*/
public static void checkState(boolean expression,
String format,
Object... args) {
if (!expression) {
throw new IllegalStateException(String.format(format, args));
}
}
private static void checkNotEmpty(int arraySize, String argName) {
Validate.checkArgument(
checkArgument(
arraySize > 0,
"'%s' must have at least one element.",
argName);

View File

@ -19,9 +19,10 @@
/**
* block caching for use in object store clients.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -17,14 +17,16 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import org.apache.hadoop.test.LambdaTestUtils;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
public final class ExceptionAsserts {
private ExceptionAsserts() {}
private ExceptionAsserts() {
}
/**
@ -47,7 +49,6 @@ public final class ExceptionAsserts {
String partialMessage,
LambdaTestUtils.VoidCallable code) throws Exception {
intercept(expectedExceptionClass, partialMessage, code);
}
@ -55,6 +56,8 @@ public final class ExceptionAsserts {
public static <E extends Exception> void assertThrows(
Class<E> expectedExceptionClass,
LambdaTestUtils.VoidCallable code) throws Exception {
assertThrows(expectedExceptionClass, null, code);
intercept(expectedExceptionClass, code);
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.util.ArrayList;
import java.util.Arrays;
@ -27,31 +27,45 @@ import java.util.List;
* Frequently used test data items.
*/
public final class SampleDataForTests {
private SampleDataForTests() {}
private SampleDataForTests() {
}
// Array data.
public static final Object[] NULL_ARRAY = null;
public static final Object[] EMPTY_ARRAY = new Object[0];
public static final Object[] NULL_ARRAY = null;
public static final Object[] EMPTY_ARRAY = new Object[0];
public static final Object[] NON_EMPTY_ARRAY = new Object[1];
public static final byte[] NULL_BYTE_ARRAY = null;
public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
public static final byte[] NULL_BYTE_ARRAY = null;
public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
public static final byte[] NON_EMPTY_BYTE_ARRAY = new byte[1];
public static final short[] NULL_SHORT_ARRAY = null;
public static final short[] EMPTY_SHORT_ARRAY = new short[0];
public static final short[] NULL_SHORT_ARRAY = null;
public static final short[] EMPTY_SHORT_ARRAY = new short[0];
public static final short[] NON_EMPTY_SHORT_ARRAY = new short[1];
public static final int[] NULL_INT_ARRAY = null;
public static final int[] EMPTY_INT_ARRAY = new int[0];
public static final int[] NON_EMPTY_INT_ARRAY = new int[1];
public static final int[] NULL_INT_ARRAY = null;
public static final long[] NULL_LONG_ARRAY = null;
public static final long[] EMPTY_LONG_ARRAY = new long[0];
public static final long[] NON_EMPTY_LONG_ARRAY = new long[1];
public static final int[] EMPTY_INT_ARRAY = new int[0];
public static final int[] NON_EMPTY_INT_ARRAY = new int[1];
public static final long[] NULL_LONG_ARRAY = null;
public static final long[] EMPTY_LONG_ARRAY = new long[0];
public static final long[] NON_EMPTY_LONG_ARRAY = new long[1];
public static final List<Object> NULL_LIST = null;
public static final List<Object> NULL_LIST = null;
public static final List<Object> EMPTY_LIST = new ArrayList<Object>();
public static final List<Object> VALID_LIST = Arrays.asList(new Object[1]);
}

View File

@ -17,15 +17,15 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.nio.ByteBuffer;
import org.junit.Test;
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
@ -45,21 +45,20 @@ public class TestBlockCache extends AbstractHadoopTestBase {
ByteBuffer buffer = ByteBuffer.allocate(16);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'buffer' must not be null",
intercept(IllegalArgumentException.class, "'buffer' must not be null",
() -> cache.put(42, null));
ExceptionAsserts.assertThrows(
NullPointerException.class,
intercept(NullPointerException.class, null,
() -> new SingleFilePerBlockCache(null));
}
@Test
public void testPutAndGet() throws Exception {
BlockCache cache =
new SingleFilePerBlockCache(new EmptyS3AStatisticsContext().newInputStreamStatistics());
new SingleFilePerBlockCache(EmptyPrefetchingStatistics.getInstance());
ByteBuffer buffer1 = ByteBuffer.allocate(BUFFER_SIZE);
for (byte i = 0; i < BUFFER_SIZE; i++) {

View File

@ -17,12 +17,13 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@ -37,30 +38,29 @@ public class TestBlockData extends AbstractHadoopTestBase {
new BlockData(0, 10);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'fileSize' must not be negative",
intercept(IllegalArgumentException.class, "'fileSize' must not be negative",
() -> new BlockData(-1, 2));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockSize' must be a positive integer",
() -> new BlockData(10, 0));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockSize' must be a positive integer",
() -> new BlockData(10, -2));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' (-1) must be within the range [0, 3]",
() -> new BlockData(10, 3).isLastBlock(-1));
() -> new BlockData(10, 3).isLastBlock(
-1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' (11) must be within the range [0, 3]",
() -> new BlockData(10, 3).isLastBlock(11));
() -> new BlockData(10, 3).isLastBlock(
11));
}
@Test
@ -72,7 +72,8 @@ public class TestBlockData extends AbstractHadoopTestBase {
testComputedFieldsHelper(10, 3);
}
private void testComputedFieldsHelper(long fileSize, int blockSize) throws Exception {
private void testComputedFieldsHelper(long fileSize, int blockSize)
throws Exception {
BlockData bd = new BlockData(fileSize, blockSize);
if (fileSize == 0) {

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.lang.reflect.Method;
@ -25,6 +25,7 @@ import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertTrue;
public class TestBlockOperations extends AbstractHadoopTestBase {
@ -35,35 +36,31 @@ public class TestBlockOperations extends AbstractHadoopTestBase {
BlockOperations ops = new BlockOperations();
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.getPrefetched(-1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.getCached(-1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.getRead(-1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.release(-1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.requestPrefetch(-1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> ops.requestCaching(-1));
}
@Test
@ -81,11 +78,13 @@ public class TestBlockOperations extends AbstractHadoopTestBase {
verifySummaryNoArg("close", "CX");
}
private void verifySummary(String methodName, String shortName) throws Exception {
private void verifySummary(String methodName, String shortName)
throws Exception {
int blockNumber = 42;
BlockOperations ops = new BlockOperations();
Method method = ops.getClass().getDeclaredMethod(methodName, int.class);
BlockOperations.Operation op = (BlockOperations.Operation) method.invoke(ops, blockNumber);
BlockOperations.Operation op =
(BlockOperations.Operation) method.invoke(ops, blockNumber);
ops.end(op);
String summary = ops.getSummary(false);
String opSummary = String.format("%s(%d)", shortName, blockNumber);
@ -93,10 +92,12 @@ public class TestBlockOperations extends AbstractHadoopTestBase {
assertTrue(summary.startsWith(expectedSummary));
}
private void verifySummaryNoArg(String methodName, String shortName) throws Exception {
private void verifySummaryNoArg(String methodName, String shortName)
throws Exception {
BlockOperations ops = new BlockOperations();
Method method = ops.getClass().getDeclaredMethod(methodName);
BlockOperations.Operation op = (BlockOperations.Operation) method.invoke(ops);
BlockOperations.Operation op =
(BlockOperations.Operation) method.invoke(ops);
ops.end(op);
String summary = ops.getSummary(false);
String expectedSummary = String.format("%s;E%s;", shortName, shortName);

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.nio.ByteBuffer;
import java.util.Collections;
@ -28,6 +28,7 @@ import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
@ -37,6 +38,7 @@ import static org.junit.Assert.assertTrue;
public class TestBoundedResourcePool extends AbstractHadoopTestBase {
static class BufferPool extends BoundedResourcePool<ByteBuffer> {
BufferPool(int size) {
super(size);
}
@ -54,25 +56,22 @@ public class TestBoundedResourcePool extends AbstractHadoopTestBase {
BufferPool pool = new BufferPool(5);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'size' must be a positive integer",
() -> new BufferPool(-1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'size' must be a positive integer",
() -> new BufferPool(0));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'item' must not be null",
intercept(IllegalArgumentException.class, "'item' must not be null",
() -> pool.release(null));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"This item is not a part of this pool",
() -> pool.release(ByteBuffer.allocate(4)));
}
@Test

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.nio.ByteBuffer;
import java.nio.ReadOnlyBufferException;
@ -30,6 +30,7 @@ import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
@ -45,35 +46,28 @@ public class TestBufferData extends AbstractHadoopTestBase {
BufferData data = new BufferData(1, buffer);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> new BufferData(-1, buffer));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'buffer' must not be null",
intercept(IllegalArgumentException.class, "'buffer' must not be null",
() -> new BufferData(1, null));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'actionFuture' must not be null",
intercept(IllegalArgumentException.class, "'actionFuture' must not be null",
() -> data.setPrefetch(null));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'actionFuture' must not be null",
intercept(IllegalArgumentException.class, "'actionFuture' must not be null",
() -> data.setCaching(null));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'states' must not be null",
intercept(IllegalArgumentException.class, "'states' must not be null",
() -> data.throwIfStateIncorrect((BufferData.State[]) null));
ExceptionAsserts.assertThrows(
IllegalStateException.class,
intercept(IllegalStateException.class,
"Expected buffer state to be 'READY or CACHING' but found",
() -> data.throwIfStateIncorrect(BufferData.State.READY, BufferData.State.CACHING));
() -> data.throwIfStateIncorrect(BufferData.State.READY,
BufferData.State.CACHING));
}
@Test
@ -193,6 +187,7 @@ public class TestBufferData extends AbstractHadoopTestBase {
@FunctionalInterface
public interface StateChanger {
void run(BufferData data) throws Exception;
}

View File

@ -17,14 +17,13 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import org.junit.Test;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
@ -33,59 +32,54 @@ import static org.junit.Assert.assertSame;
public class TestBufferPool extends AbstractHadoopTestBase {
private static final int POOL_SIZE = 2;
private static final int BUFFER_SIZE = 10;
private final S3AInputStreamStatistics s3AInputStreamStatistics =
new EmptyS3AStatisticsContext().newInputStreamStatistics();
private final PrefetchingStatistics statistics =
EmptyPrefetchingStatistics.getInstance();
@Test
public void testArgChecks() throws Exception {
// Should not throw.
BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, s3AInputStreamStatistics);
BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, statistics);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'size' must be a positive integer",
() -> new BufferPool(0, 10, s3AInputStreamStatistics));
() -> new BufferPool(0, 10, statistics));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'size' must be a positive integer",
() -> new BufferPool(-1, 10, s3AInputStreamStatistics));
() -> new BufferPool(-1, 10, statistics));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'bufferSize' must be a positive integer",
() -> new BufferPool(10, 0, s3AInputStreamStatistics));
() -> new BufferPool(10, 0, statistics));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'bufferSize' must be a positive integer",
() -> new BufferPool(1, -10, s3AInputStreamStatistics));
() -> new BufferPool(1, -10, statistics));
ExceptionAsserts.assertThrows(
NullPointerException.class,
intercept(NullPointerException.class,
() -> new BufferPool(1, 10, null));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> pool.acquire(-1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> pool.tryAcquire(-1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'data' must not be null",
intercept(NullPointerException.class, "data",
() -> pool.release((BufferData) null));
}
@Test
public void testGetAndRelease() {
BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, s3AInputStreamStatistics);
BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, statistics);
assertInitialState(pool, POOL_SIZE);
int count = 0;
@ -129,20 +123,21 @@ public class TestBufferPool extends AbstractHadoopTestBase {
testReleaseHelper(BufferData.State.READY, false);
}
private void testReleaseHelper(BufferData.State stateBeforeRelease, boolean expectThrow)
private void testReleaseHelper(BufferData.State stateBeforeRelease,
boolean expectThrow)
throws Exception {
BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, s3AInputStreamStatistics);
BufferPool pool = new BufferPool(POOL_SIZE, BUFFER_SIZE, statistics);
assertInitialState(pool, POOL_SIZE);
BufferData data = this.acquire(pool, 1);
data.updateState(stateBeforeRelease, BufferData.State.BLANK);
if (expectThrow) {
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"Unable to release buffer",
intercept(IllegalArgumentException.class, "Unable to release buffer",
() -> pool.release(data));
} else {
pool.release(data);
}

View File

@ -17,9 +17,8 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
@ -31,8 +30,8 @@ import org.junit.Before;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import org.apache.hadoop.test.LambdaTestUtils;
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
import static org.junit.Assert.assertTrue;
public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase {
@ -53,16 +52,19 @@ public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase {
@Test
public void testRunnableSucceeds() throws Exception {
ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService);
ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(executorService);
final AtomicBoolean atomicBoolean = new AtomicBoolean(false);
Future<Void> future = futurePool.executeRunnable(() -> atomicBoolean.set(true));
Future<Void> future =
futurePool.executeRunnable(() -> atomicBoolean.set(true));
future.get(30, TimeUnit.SECONDS);
assertTrue("atomicBoolean set to true?", atomicBoolean.get());
}
@Test
public void testSupplierSucceeds() throws Exception {
ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService);
ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(executorService);
final AtomicBoolean atomicBoolean = new AtomicBoolean(false);
Future<Void> future = futurePool.executeFunction(() -> {
atomicBoolean.set(true);
@ -74,19 +76,23 @@ public class TestExecutorServiceFuturePool extends AbstractHadoopTestBase {
@Test
public void testRunnableFails() throws Exception {
ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService);
ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(executorService);
Future<Void> future = futurePool.executeRunnable(() -> {
throw new IllegalStateException("deliberate");
});
LambdaTestUtils.intercept(ExecutionException.class, () -> future.get(30, TimeUnit.SECONDS));
interceptFuture(IllegalStateException.class, "deliberate", 30,
TimeUnit.SECONDS, future);
}
@Test
public void testSupplierFails() throws Exception {
ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(executorService);
ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(executorService);
Future<Void> future = futurePool.executeFunction(() -> {
throw new IllegalStateException("deliberate");
});
LambdaTestUtils.intercept(ExecutionException.class, () -> future.get(30, TimeUnit.SECONDS));
interceptFuture(IllegalStateException.class, "deliberate", 30,
TimeUnit.SECONDS, future);
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.nio.ByteBuffer;
@ -25,6 +25,7 @@ import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@ -44,84 +45,62 @@ public class TestFilePosition extends AbstractHadoopTestBase {
new FilePosition(10, 5).setData(data, 3, 4);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'fileSize' must not be negative",
intercept(IllegalArgumentException.class, "'fileSize' must not be negative",
() -> new FilePosition(-1, 2));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockSize' must be a positive integer",
() -> new FilePosition(1, 0));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'blockSize' must be a positive integer",
() -> new FilePosition(1, -1));
FilePosition pos = new FilePosition(10, 3);
// Verify that we cannot obtain buffer properties without setting buffer.
ExceptionAsserts.assertThrows(
IllegalStateException.class,
"'buffer' must not be null",
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.buffer());
ExceptionAsserts.assertThrows(
IllegalStateException.class,
"'buffer' must not be null",
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.absolute());
ExceptionAsserts.assertThrows(
IllegalStateException.class,
"'buffer' must not be null",
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.isWithinCurrentBuffer(2));
ExceptionAsserts.assertThrows(
IllegalStateException.class,
"'buffer' must not be null",
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.blockNumber());
ExceptionAsserts.assertThrows(
IllegalStateException.class,
"'buffer' must not be null",
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.isLastBlock());
ExceptionAsserts.assertThrows(
IllegalStateException.class,
"'buffer' must not be null",
intercept(IllegalStateException.class, "'buffer' must not be null",
() -> pos.bufferFullyRead());
// Verify that we cannot set invalid buffer parameters.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'bufferData' must not be null",
intercept(IllegalArgumentException.class, "'bufferData' must not be null",
() -> pos.setData(null, 4, 4));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'startOffset' must not be negative",
() -> pos.setData(data, -4, 4));
intercept(IllegalArgumentException.class,
"'startOffset' must not be negative", () -> pos.setData(data, -4, 4));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'readOffset' must not be negative",
() -> pos.setData(data, 4, -4));
intercept(IllegalArgumentException.class,
"'readOffset' must not be negative", () -> pos.setData(data, 4, -4));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'readOffset' must not be negative",
() -> pos.setData(data, 4, -4));
intercept(IllegalArgumentException.class,
"'readOffset' must not be negative", () -> pos.setData(data, 4, -4));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'readOffset' (15) must be within the range [4, 13]",
() -> pos.setData(data, 4, 15));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'readOffset' (3) must be within the range [4, 13]",
() -> pos.setData(data, 4, 3));
}
@Test

View File

@ -17,12 +17,13 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@ -34,30 +35,27 @@ public class TestRetryer extends AbstractHadoopTestBase {
new Retryer(10, 50, 500);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'perRetryDelay' must be a positive integer",
() -> new Retryer(-1, 50, 500));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'perRetryDelay' must be a positive integer",
() -> new Retryer(0, 50, 500));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'maxDelay' (5) must be greater than 'perRetryDelay' (10)",
() -> new Retryer(10, 5, 500));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'statusUpdateInterval' must be a positive integer",
() -> new Retryer(10, 50, -1));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'statusUpdateInterval' must be a positive integer",
() -> new Retryer(10, 50, 0));
}
@Test
@ -66,7 +64,8 @@ public class TestRetryer extends AbstractHadoopTestBase {
int statusUpdateInterval = 3;
int maxDelay = 10;
Retryer retryer = new Retryer(perRetryDelay, maxDelay, statusUpdateInterval);
Retryer retryer =
new Retryer(perRetryDelay, maxDelay, statusUpdateInterval);
for (int t = 1; t <= maxDelay; t++) {
assertTrue(retryer.continueRetry());
if (t % statusUpdateInterval == 0) {

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.common;
package org.apache.hadoop.fs.impl.prefetch;
import java.nio.file.Files;
import java.nio.file.Path;
@ -28,7 +28,27 @@ import org.junit.Test;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_BYTE_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_INT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_LIST;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_LONG_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.EMPTY_SHORT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_BYTE_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_INT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_LONG_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NON_EMPTY_SHORT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_BYTE_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_INT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_LIST;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_LONG_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.NULL_SHORT_ARRAY;
import static org.apache.hadoop.fs.impl.prefetch.SampleDataForTests.VALID_LIST;
import static org.apache.hadoop.fs.impl.prefetch.Validate.checkPositiveInteger;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
public class TestValidate extends AbstractHadoopTestBase {
@Test
public void testCheckNotNull() throws Exception {
String nonNullArg = "nonNullArg";
@ -38,10 +58,10 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkNotNull(nonNullArg, "nonNullArg");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'nullArg' must not be null",
intercept(IllegalArgumentException.class, "'nullArg' must not be null",
() -> Validate.checkNotNull(nullArg, "nullArg"));
}
@Test
@ -51,17 +71,18 @@ public class TestValidate extends AbstractHadoopTestBase {
int negativeArg = -1;
// Should not throw.
Validate.checkPositiveInteger(positiveArg, "positiveArg");
checkPositiveInteger(positiveArg, "positiveArg");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'negativeArg' must be a positive integer",
() -> Validate.checkPositiveInteger(negativeArg, "negativeArg"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
() -> checkPositiveInteger(negativeArg, "negativeArg"));
intercept(IllegalArgumentException.class,
"'zero' must be a positive integer",
() -> Validate.checkPositiveInteger(zero, "zero"));
() -> checkPositiveInteger(zero, "zero"));
}
@Test
@ -75,10 +96,11 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkNotNegative(positiveArg, "positiveArg");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'negativeArg' must not be negative",
() -> Validate.checkNotNegative(negativeArg, "negativeArg"));
}
@Test
@ -87,10 +109,10 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkRequired(true, "arg");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'arg' is required",
intercept(IllegalArgumentException.class, "'arg' is required",
() -> Validate.checkRequired(false, "arg"));
}
@Test
@ -113,93 +135,92 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkValid(true, "arg", validValues);
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'arg' is invalid. Valid values are: foo, bar",
() -> Validate.checkValid(false, "arg", validValues));
}
@Test
public void testCheckNotNullAndNotEmpty() throws Exception {
// Should not throw.
Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_BYTE_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_SHORT_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_INT_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(SampleDataForTests.NON_EMPTY_LONG_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(NON_EMPTY_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(NON_EMPTY_BYTE_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(NON_EMPTY_SHORT_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(NON_EMPTY_INT_ARRAY, "array");
Validate.checkNotNullAndNotEmpty(NON_EMPTY_LONG_ARRAY, "array");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'string' must not be empty",
intercept(IllegalArgumentException.class, "'string' must not be empty",
() -> Validate.checkNotNullAndNotEmpty("", "string"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must not be null",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_ARRAY, "array"));
intercept(IllegalArgumentException.class, "'array' must not be null", () ->
Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_ARRAY,
"array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_ARRAY, "array"));
intercept(IllegalArgumentException.class,
"'array' must have at least one element", () ->
Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_ARRAY,
"array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must not be null",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_BYTE_ARRAY, "array"));
() -> Validate.checkNotNullAndNotEmpty(NULL_BYTE_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_BYTE_ARRAY, "array"));
() -> Validate.checkNotNullAndNotEmpty(EMPTY_BYTE_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must not be null",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_SHORT_ARRAY, "array"));
() -> Validate.checkNotNullAndNotEmpty(NULL_SHORT_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_SHORT_ARRAY, "array"));
() -> Validate.checkNotNullAndNotEmpty(EMPTY_SHORT_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must not be null",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_INT_ARRAY, "array"));
() -> Validate.checkNotNullAndNotEmpty(NULL_INT_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_INT_ARRAY, "array"));
() -> Validate.checkNotNullAndNotEmpty(EMPTY_INT_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must not be null",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_LONG_ARRAY, "array"));
() -> Validate.checkNotNullAndNotEmpty(NULL_LONG_ARRAY, "array"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'array' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_LONG_ARRAY, "array"));
() -> Validate.checkNotNullAndNotEmpty(EMPTY_LONG_ARRAY, "array"));
}
@Test
public void testCheckListNotNullAndNotEmpty() throws Exception {
// Should not throw.
Validate.checkNotNullAndNotEmpty(SampleDataForTests.VALID_LIST, "list");
Validate.checkNotNullAndNotEmpty(VALID_LIST, "list");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'list' must not be null",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.NULL_LIST, "list"));
() -> Validate.checkNotNullAndNotEmpty(NULL_LIST, "list"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'list' must have at least one element",
() -> Validate.checkNotNullAndNotEmpty(SampleDataForTests.EMPTY_LIST, "list"));
() -> Validate.checkNotNullAndNotEmpty(EMPTY_LIST, "list"));
}
@Test
@ -208,17 +229,16 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkNotNullAndNumberOfElements(Arrays.asList(1, 2, 3), 3, "arg");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'arg' must not be null",
() -> Validate.checkNotNullAndNumberOfElements(null, 3, "arg")
);
intercept(IllegalArgumentException.class, "'arg' must not be null",
() -> Validate.checkNotNullAndNumberOfElements(null, 3, "arg"));
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"Number of elements in 'arg' must be exactly 3, 2 given.",
() -> Validate.checkNotNullAndNumberOfElements(Arrays.asList(1, 2), 3, "arg")
() -> Validate.checkNotNullAndNumberOfElements(Arrays.asList(1, 2), 3,
"arg")
);
}
@ -228,10 +248,11 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkValuesEqual(1, "arg1", 1, "arg2");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'arg1' (1) must equal 'arg2' (2)",
() -> Validate.checkValuesEqual(1, "arg1", 2, "arg2"));
}
@Test
@ -240,10 +261,11 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkIntegerMultiple(10, "arg1", 5, "arg2");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'arg1' (10) must be an integer multiple of 'arg2' (3)",
() -> Validate.checkIntegerMultiple(10, "arg1", 3, "arg2"));
}
@Test
@ -252,10 +274,11 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkGreater(10, "arg1", 5, "arg2");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'arg1' (5) must be greater than 'arg2' (10)",
() -> Validate.checkGreater(5, "arg1", 10, "arg2"));
}
@Test
@ -264,10 +287,11 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkGreaterOrEqual(10, "arg1", 5, "arg2");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'arg1' (5) must be greater than or equal to 'arg2' (10)",
() -> Validate.checkGreaterOrEqual(5, "arg1", 10, "arg2"));
}
@Test
@ -277,21 +301,21 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkWithinRange(10.0, "arg", 5.0, 15.0);
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'arg' (5) must be within the range [10, 20]",
() -> Validate.checkWithinRange(5, "arg", 10, 20));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"'arg' (5.0) must be within the range [10.0, 20.0]",
() -> Validate.checkWithinRange(5.0, "arg", 10.0, 20.0));
}
@Test
public void testCheckPathExists() throws Exception {
Path tempFile = Files.createTempFile("foo", "bar");
Path tempDir = tempFile.getParent();
Path tempDir = tempFile.getParent();
Path notFound = Paths.get("<not-found>");
// Should not throw.
@ -299,24 +323,19 @@ public class TestValidate extends AbstractHadoopTestBase {
Validate.checkPathExists(tempDir, "tempDir");
// Verify it throws.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'nullArg' must not be null",
intercept(IllegalArgumentException.class, "'nullArg' must not be null",
() -> Validate.checkPathExists(null, "nullArg"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
intercept(IllegalArgumentException.class,
"Path notFound (<not-found>) does not exist",
() -> Validate.checkPathExists(notFound, "notFound"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"must point to a directory",
intercept(IllegalArgumentException.class, "must point to a directory",
() -> Validate.checkPathExistsAsDir(tempFile, "tempFile"));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"must point to a file",
intercept(IllegalArgumentException.class, "must point to a file",
() -> Validate.checkPathExistsAsFile(tempDir, "tempDir"));
}
}

View File

@ -79,7 +79,7 @@ import com.amazonaws.services.s3.transfer.model.CopyResult;
import com.amazonaws.services.s3.transfer.model.UploadResult;
import com.amazonaws.event.ProgressListener;
import org.apache.hadoop.fs.common.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -125,6 +125,7 @@ import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder;
import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum;
import org.apache.hadoop.fs.s3a.impl.StoreContext;
import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder;
import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperations;
import org.apache.hadoop.fs.s3a.tools.MarkerToolOperationsImpl;
import org.apache.hadoop.fs.statistics.DurationTracker;
@ -173,7 +174,6 @@ import org.apache.hadoop.fs.s3a.commit.CommitConstants;
import org.apache.hadoop.fs.s3a.commit.PutTracker;
import org.apache.hadoop.fs.s3a.commit.MagicCommitIntegration;
import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
import org.apache.hadoop.fs.s3a.read.S3PrefetchingInputStream;
import org.apache.hadoop.fs.s3a.select.SelectBinding;
import org.apache.hadoop.fs.s3a.select.SelectConstants;
import org.apache.hadoop.fs.s3a.s3guard.S3Guard;
@ -1521,7 +1521,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
if (this.prefetchEnabled) {
return new FSDataInputStream(
new S3PrefetchingInputStream(
new S3APrefetchingInputStream(
readContext.build(),
createObjectAttributes(path, fileStatus),
createInputStreamCallbacks(auditSpan),

View File

@ -21,7 +21,7 @@ package org.apache.hadoop.fs.s3a;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.common.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy;
import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext;
import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;

View File

@ -17,30 +17,30 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.fs.common.BlockData;
import org.apache.hadoop.fs.common.BlockManager;
import org.apache.hadoop.fs.common.Validate;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.BlockManager;
import org.apache.hadoop.fs.impl.prefetch.Validate;
/**
* Provides read access to S3 file one block at a time.
*
* A naive implementation of a {@code BlockManager} that provides no prefetching or caching.
* Useful baseline for comparing performance difference against {@code S3CachingBlockManager}.
* Useful baseline for comparing performance difference against {@code S3ACachingBlockManager}.
*/
public class S3BlockManager extends BlockManager {
public class S3ABlockManager extends BlockManager {
/**
* Reader that reads from S3 file.
*/
private final S3Reader reader;
private final S3ARemoteObjectReader reader;
/**
* Constructs an instance of {@code S3BlockManager}.
* Constructs an instance of {@code S3ABlockManager}.
*
* @param reader a reader that reads from S3 file.
* @param blockData information about each block of the S3 file.
@ -48,7 +48,7 @@ public class S3BlockManager extends BlockManager {
* @throws IllegalArgumentException if reader is null.
* @throws IllegalArgumentException if blockData is null.
*/
public S3BlockManager(S3Reader reader, BlockData blockData) {
public S3ABlockManager(S3ARemoteObjectReader reader, BlockData blockData) {
super(blockData);
Validate.checkNotNull(reader, "reader");
@ -66,12 +66,13 @@ public class S3BlockManager extends BlockManager {
* @return number of bytes read.
*/
@Override
public int read(ByteBuffer buffer, long startOffset, int size) throws IOException {
return this.reader.read(buffer, startOffset, size);
public int read(ByteBuffer buffer, long startOffset, int size)
throws IOException {
return reader.read(buffer, startOffset, size);
}
@Override
public void close() {
this.reader.close();
reader.close();
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
@ -25,25 +25,27 @@ import java.nio.ByteBuffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.common.BlockData;
import org.apache.hadoop.fs.common.CachingBlockManager;
import org.apache.hadoop.fs.common.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.common.Validate;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.CachingBlockManager;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
/**
* Provides access to S3 file one block at a time.
*/
public class S3CachingBlockManager extends CachingBlockManager {
private static final Logger LOG = LoggerFactory.getLogger(S3CachingBlockManager.class);
public class S3ACachingBlockManager extends CachingBlockManager {
private static final Logger LOG = LoggerFactory.getLogger(
S3ACachingBlockManager.class);
/**
* Reader that reads from S3 file.
*/
private final S3Reader reader;
private final S3ARemoteObjectReader reader;
/**
* Constructs an instance of a {@code S3CachingBlockManager}.
* Constructs an instance of a {@code S3ACachingBlockManager}.
*
* @param futurePool asynchronous tasks are performed in this pool.
* @param reader reader that reads from S3 file.
@ -53,9 +55,9 @@ public class S3CachingBlockManager extends CachingBlockManager {
*
* @throws IllegalArgumentException if reader is null.
*/
public S3CachingBlockManager(
public S3ACachingBlockManager(
ExecutorServiceFuturePool futurePool,
S3Reader reader,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize,
S3AInputStreamStatistics streamStatistics) {
@ -66,7 +68,7 @@ public class S3CachingBlockManager extends CachingBlockManager {
this.reader = reader;
}
protected S3Reader getReader() {
protected S3ARemoteObjectReader getReader() {
return this.reader;
}
@ -80,7 +82,8 @@ public class S3CachingBlockManager extends CachingBlockManager {
* @return number of bytes read.
*/
@Override
public int read(ByteBuffer buffer, long startOffset, int size) throws IOException {
public int read(ByteBuffer buffer, long startOffset, int size)
throws IOException {
return this.reader.read(buffer, startOffset, size);
}

View File

@ -17,17 +17,17 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.IOException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.common.BlockData;
import org.apache.hadoop.fs.common.BlockManager;
import org.apache.hadoop.fs.common.BufferData;
import org.apache.hadoop.fs.common.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.BlockManager;
import org.apache.hadoop.fs.impl.prefetch.BufferData;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
@ -41,8 +41,10 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTra
* Prefetched blocks are cached to local disk if a seek away from the
* current block is issued.
*/
public class S3CachingInputStream extends S3InputStream {
private static final Logger LOG = LoggerFactory.getLogger(S3CachingInputStream.class);
public class S3ACachingInputStream extends S3ARemoteInputStream {
private static final Logger LOG = LoggerFactory.getLogger(
S3ACachingInputStream.class);
/**
* Number of blocks queued for prefching.
@ -52,7 +54,7 @@ public class S3CachingInputStream extends S3InputStream {
private final BlockManager blockManager;
/**
* Initializes a new instance of the {@code S3CachingInputStream} class.
* Initializes a new instance of the {@code S3ACachingInputStream} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
@ -63,7 +65,7 @@ public class S3CachingInputStream extends S3InputStream {
* @throws IllegalArgumentException if s3Attributes is null.
* @throws IllegalArgumentException if client is null.
*/
public S3CachingInputStream(
public S3ACachingInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
@ -78,7 +80,8 @@ public class S3CachingInputStream extends S3InputStream {
this.getBlockData(),
bufferPoolSize);
int fileSize = (int) s3Attributes.getLen();
LOG.debug("Created caching input stream for {} (size = {})", this.getName(), fileSize);
LOG.debug("Created caching input stream for {} (size = {})", this.getName(),
fileSize);
}
/**
@ -90,18 +93,18 @@ public class S3CachingInputStream extends S3InputStream {
*/
@Override
public void seek(long pos) throws IOException {
this.throwIfClosed();
this.throwIfInvalidSeek(pos);
throwIfClosed();
throwIfInvalidSeek(pos);
// The call to setAbsolute() returns true if the target position is valid and
// within the current block. Therefore, no additional work is needed when we get back true.
if (!this.getFilePosition().setAbsolute(pos)) {
if (!getFilePosition().setAbsolute(pos)) {
LOG.info("seek({})", getOffsetStr(pos));
// We could be here in two cases:
// -- the target position is invalid:
// We ignore this case here as the next read will return an error.
// -- it is valid but outside of the current block.
if (this.getFilePosition().isValid()) {
if (getFilePosition().isValid()) {
// There are two cases to consider:
// -- the seek was issued after this buffer was fully read.
// In this case, it is very unlikely that this buffer will be needed again;
@ -109,15 +112,15 @@ public class S3CachingInputStream extends S3InputStream {
// -- if we are jumping out of the buffer before reading it completely then
// we will likely need this buffer again (as observed empirically for Parquet)
// therefore we issue an async request to cache this buffer.
if (!this.getFilePosition().bufferFullyRead()) {
this.blockManager.requestCaching(this.getFilePosition().data());
if (!getFilePosition().bufferFullyRead()) {
blockManager.requestCaching(getFilePosition().data());
} else {
this.blockManager.release(this.getFilePosition().data());
blockManager.release(getFilePosition().data());
}
this.getFilePosition().invalidate();
this.blockManager.cancelPrefetches();
getFilePosition().invalidate();
blockManager.cancelPrefetches();
}
this.setSeekTargetPos(pos);
setSeekTargetPos(pos);
}
}
@ -125,83 +128,87 @@ public class S3CachingInputStream extends S3InputStream {
public void close() throws IOException {
// Close the BlockManager first, cancelling active prefetches,
// deleting cached files and freeing memory used by buffer pool.
this.blockManager.close();
blockManager.close();
super.close();
LOG.info("closed: {}", this.getName());
LOG.info("closed: {}", getName());
}
@Override
protected boolean ensureCurrentBuffer() throws IOException {
if (this.isClosed()) {
if (isClosed()) {
return false;
}
if (this.getFilePosition().isValid() && this.getFilePosition().buffer().hasRemaining()) {
if (getFilePosition().isValid() && getFilePosition()
.buffer()
.hasRemaining()) {
return true;
}
long readPos;
int prefetchCount;
if (this.getFilePosition().isValid()) {
if (getFilePosition().isValid()) {
// A sequential read results in a prefetch.
readPos = this.getFilePosition().absolute();
prefetchCount = this.numBlocksToPrefetch;
readPos = getFilePosition().absolute();
prefetchCount = numBlocksToPrefetch;
} else {
// A seek invalidates the current position.
// We prefetch only 1 block immediately after a seek operation.
readPos = this.getSeekTargetPos();
readPos = getSeekTargetPos();
prefetchCount = 1;
}
if (!this.getBlockData().isValidOffset(readPos)) {
if (!getBlockData().isValidOffset(readPos)) {
return false;
}
if (this.getFilePosition().isValid()) {
if (this.getFilePosition().bufferFullyRead()) {
this.blockManager.release(this.getFilePosition().data());
if (getFilePosition().isValid()) {
if (getFilePosition().bufferFullyRead()) {
blockManager.release(getFilePosition().data());
} else {
this.blockManager.requestCaching(this.getFilePosition().data());
blockManager.requestCaching(getFilePosition().data());
}
}
int toBlockNumber = this.getBlockData().getBlockNumber(readPos);
long startOffset = this.getBlockData().getStartOffset(toBlockNumber);
int toBlockNumber = getBlockData().getBlockNumber(readPos);
long startOffset = getBlockData().getStartOffset(toBlockNumber);
for (int i = 1; i <= prefetchCount; i++) {
int b = toBlockNumber + i;
if (b < this.getBlockData().getNumBlocks()) {
this.blockManager.requestPrefetch(b);
if (b < getBlockData().getNumBlocks()) {
blockManager.requestPrefetch(b);
}
}
BufferData data = invokeTrackingDuration(
this.getS3AStreamStatistics().trackDuration(STREAM_READ_BLOCK_ACQUIRE_AND_READ),
() -> this.blockManager.get(toBlockNumber));
getS3AStreamStatistics()
.trackDuration(STREAM_READ_BLOCK_ACQUIRE_AND_READ),
() -> blockManager.get(toBlockNumber));
this.getFilePosition().setData(data, startOffset, readPos);
getFilePosition().setData(data, startOffset, readPos);
return true;
}
@Override
public String toString() {
if (this.isClosed()) {
if (isClosed()) {
return "closed";
}
StringBuilder sb = new StringBuilder();
sb.append(String.format("fpos = (%s)%n", this.getFilePosition()));
sb.append(this.blockManager.toString());
sb.append(String.format("fpos = (%s)%n", getFilePosition()));
sb.append(blockManager.toString());
return sb.toString();
}
protected BlockManager createBlockManager(
ExecutorServiceFuturePool futurePool,
S3Reader reader,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize) {
return new S3CachingBlockManager(futurePool, reader, blockData, bufferPoolSize,
this.getS3AStreamStatistics());
return new S3ACachingBlockManager(futurePool, reader, blockData,
bufferPoolSize,
getS3AStreamStatistics());
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
@ -25,7 +25,7 @@ import java.nio.ByteBuffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.common.BufferData;
import org.apache.hadoop.fs.impl.prefetch.BufferData;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
@ -38,13 +38,15 @@ import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
* Use of this class is recommended only for small files that can fit
* entirely in memory.
*/
public class S3InMemoryInputStream extends S3InputStream {
private static final Logger LOG = LoggerFactory.getLogger(S3InMemoryInputStream.class);
public class S3AInMemoryInputStream extends S3ARemoteInputStream {
private static final Logger LOG = LoggerFactory.getLogger(
S3AInMemoryInputStream.class);
private ByteBuffer buffer;
/**
* Initializes a new instance of the {@code S3InMemoryInputStream} class.
* Initializes a new instance of the {@code S3AInMemoryInputStream} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
@ -55,7 +57,7 @@ public class S3InMemoryInputStream extends S3InputStream {
* @throws IllegalArgumentException if s3Attributes is null.
* @throws IllegalArgumentException if client is null.
*/
public S3InMemoryInputStream(
public S3AInMemoryInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
@ -63,7 +65,8 @@ public class S3InMemoryInputStream extends S3InputStream {
super(context, s3Attributes, client, streamStatistics);
int fileSize = (int) s3Attributes.getLen();
this.buffer = ByteBuffer.allocate(fileSize);
LOG.debug("Created in-memory input stream for {} (size = {})", this.getName(), fileSize);
LOG.debug("Created in-memory input stream for {} (size = {})",
getName(), fileSize);
}
/**
@ -75,24 +78,25 @@ public class S3InMemoryInputStream extends S3InputStream {
*/
@Override
protected boolean ensureCurrentBuffer() throws IOException {
if (this.isClosed()) {
if (isClosed()) {
return false;
}
if (this.getBlockData().getFileSize() == 0) {
if (getBlockData().getFileSize() == 0) {
return false;
}
if (!this.getFilePosition().isValid()) {
this.buffer.clear();
int numBytesRead = this.getReader().read(buffer, 0, this.buffer.capacity());
if (!getFilePosition().isValid()) {
buffer.clear();
int numBytesRead =
getReader().read(buffer, 0, buffer.capacity());
if (numBytesRead <= 0) {
return false;
}
BufferData data = new BufferData(0, buffer);
this.getFilePosition().setData(data, 0, this.getSeekTargetPos());
getFilePosition().setData(data, 0, getSeekTargetPos());
}
return this.getFilePosition().buffer().hasRemaining();
return getFilePosition().buffer().hasRemaining();
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.IOException;
@ -30,7 +30,7 @@ import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.common.Validate;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
@ -44,16 +44,20 @@ import org.apache.hadoop.fs.statistics.IOStatisticsSource;
* This implementation provides improved read throughput by asynchronously prefetching
* blocks of configurable size from the underlying S3 file.
*/
public class S3PrefetchingInputStream
public class S3APrefetchingInputStream
extends FSInputStream
implements CanSetReadahead, StreamCapabilities, IOStatisticsSource {
private static final Logger LOG = LoggerFactory.getLogger(S3PrefetchingInputStream.class);
// Underlying input stream used for reading S3 file.
private S3InputStream inputStream;
private static final Logger LOG = LoggerFactory.getLogger(
S3APrefetchingInputStream.class);
/**
* Initializes a new instance of the {@code S3PrefetchingInputStream} class.
* Underlying input stream used for reading S3 file.
*/
private S3ARemoteInputStream inputStream;
/**
* Initializes a new instance of the {@code S3APrefetchingInputStream} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
@ -64,7 +68,7 @@ public class S3PrefetchingInputStream
* @throws IllegalArgumentException if s3Attributes is null.
* @throws IllegalArgumentException if client is null.
*/
public S3PrefetchingInputStream(
public S3APrefetchingInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
@ -72,17 +76,29 @@ public class S3PrefetchingInputStream
Validate.checkNotNull(context, "context");
Validate.checkNotNull(s3Attributes, "s3Attributes");
Validate.checkNotNullAndNotEmpty(s3Attributes.getBucket(), "s3Attributes.getBucket()");
Validate.checkNotNullAndNotEmpty(s3Attributes.getKey(), "s3Attributes.getKey()");
Validate.checkNotNullAndNotEmpty(s3Attributes.getBucket(),
"s3Attributes.getBucket()");
Validate.checkNotNullAndNotEmpty(s3Attributes.getKey(),
"s3Attributes.getKey()");
Validate.checkNotNegative(s3Attributes.getLen(), "s3Attributes.getLen()");
Validate.checkNotNull(client, "client");
Validate.checkNotNull(streamStatistics, "streamStatistics");
long fileSize = s3Attributes.getLen();
if (fileSize <= context.getPrefetchBlockSize()) {
this.inputStream = new S3InMemoryInputStream(context, s3Attributes, client, streamStatistics);
LOG.debug("Creating in memory input stream for {}", context.getPath());
this.inputStream = new S3AInMemoryInputStream(
context,
s3Attributes,
client,
streamStatistics);
} else {
this.inputStream = new S3CachingInputStream(context, s3Attributes, client, streamStatistics);
LOG.debug("Creating in caching input stream for {}", context.getPath());
this.inputStream = new S3ACachingInputStream(
context,
s3Attributes,
client,
streamStatistics);
}
}
@ -94,8 +110,8 @@ public class S3PrefetchingInputStream
*/
@Override
public synchronized int available() throws IOException {
this.throwIfClosed();
return this.inputStream.available();
throwIfClosed();
return inputStream.available();
}
/**
@ -106,7 +122,7 @@ public class S3PrefetchingInputStream
*/
@Override
public synchronized long getPos() throws IOException {
return this.isClosed() ? 0 : this.inputStream.getPos();
return isClosed() ? 0 : inputStream.getPos();
}
/**
@ -117,8 +133,8 @@ public class S3PrefetchingInputStream
*/
@Override
public synchronized int read() throws IOException {
this.throwIfClosed();
return this.inputStream.read();
throwIfClosed();
return inputStream.read();
}
/**
@ -133,9 +149,10 @@ public class S3PrefetchingInputStream
* @throws IOException if there is an IO error during this operation.
*/
@Override
public synchronized int read(byte[] buffer, int offset, int len) throws IOException {
this.throwIfClosed();
return this.inputStream.read(buffer, offset, len);
public synchronized int read(byte[] buffer, int offset, int len)
throws IOException {
throwIfClosed();
return inputStream.read(buffer, offset, len);
}
/**
@ -145,9 +162,9 @@ public class S3PrefetchingInputStream
*/
@Override
public synchronized void close() throws IOException {
if (this.inputStream != null) {
this.inputStream.close();
this.inputStream = null;
if (inputStream != null) {
inputStream.close();
inputStream = null;
super.close();
}
}
@ -160,8 +177,8 @@ public class S3PrefetchingInputStream
*/
@Override
public synchronized void seek(long pos) throws IOException {
this.throwIfClosed();
this.inputStream.seek(pos);
throwIfClosed();
inputStream.seek(pos);
}
/**
@ -171,8 +188,8 @@ public class S3PrefetchingInputStream
*/
@Override
public synchronized void setReadahead(Long readahead) {
if (!this.isClosed()) {
this.inputStream.setReadahead(readahead);
if (!isClosed()) {
inputStream.setReadahead(readahead);
}
}
@ -184,8 +201,8 @@ public class S3PrefetchingInputStream
*/
@Override
public boolean hasCapability(String capability) {
if (!this.isClosed()) {
return this.inputStream.hasCapability(capability);
if (!isClosed()) {
return inputStream.hasCapability(capability);
}
return false;
@ -199,10 +216,10 @@ public class S3PrefetchingInputStream
@InterfaceAudience.Private
@InterfaceStability.Unstable
public S3AInputStreamStatistics getS3AStreamStatistics() {
if (this.isClosed()) {
if (isClosed()) {
return null;
}
return this.inputStream.getS3AStreamStatistics();
return inputStream.getS3AStreamStatistics();
}
/**
@ -212,18 +229,18 @@ public class S3PrefetchingInputStream
*/
@Override
public IOStatistics getIOStatistics() {
if (this.isClosed()) {
if (isClosed()) {
return null;
}
return this.inputStream.getIOStatistics();
return inputStream.getIOStatistics();
}
protected boolean isClosed() {
return this.inputStream == null;
return inputStream == null;
}
protected void throwIfClosed() throws IOException {
if (this.isClosed()) {
if (isClosed()) {
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
}
}
@ -232,7 +249,7 @@ public class S3PrefetchingInputStream
@Override
public boolean seekToNewSource(long targetPos) throws IOException {
this.throwIfClosed();
throwIfClosed();
return false;
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.EOFException;
import java.io.IOException;
@ -32,9 +32,9 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.common.BlockData;
import org.apache.hadoop.fs.common.FilePosition;
import org.apache.hadoop.fs.common.Validate;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.FilePosition;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.S3AInputPolicy;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
@ -49,51 +49,64 @@ import static java.util.Objects.requireNonNull;
/**
* Provides an {@link InputStream} that allows reading from an S3 file.
*/
public abstract class S3InputStream
public abstract class S3ARemoteInputStream
extends InputStream
implements CanSetReadahead, StreamCapabilities, IOStatisticsSource {
private static final Logger LOG = LoggerFactory.getLogger(S3InputStream.class);
private static final Logger LOG = LoggerFactory.getLogger(
S3ARemoteInputStream.class);
// The S3 file read by this instance.
private S3File s3File;
/**
* The S3 file read by this instance.
*/
private S3ARemoteObject remoteObject;
// Reading of S3 file takes place through this reader.
private S3Reader reader;
/**
* Reading of S3 file takes place through this reader.
*/
private S3ARemoteObjectReader reader;
// Name of this stream. Used only for logging.
/**
* Name of this stream. Used only for logging.
*/
private final String name;
// Indicates whether the stream has been closed.
/**
* Indicates whether the stream has been closed.
*/
private volatile boolean closed;
// Current position within the file.
/**
* Current position within the file.
*/
private FilePosition fpos;
// The target of the most recent seek operation.
/** The target of the most recent seek operation. */
private long seekTargetPos;
// Information about each block of the mapped S3 file.
/** Information about each block of the mapped S3 file. */
private BlockData blockData;
// Read-specific operation context.
/** Read-specific operation context. */
private S3AReadOpContext context;
// Attributes of the S3 object being read.
/** Attributes of the S3 object being read. */
private S3ObjectAttributes s3Attributes;
// Callbacks used for interacting with the underlying S3 client.
/** Callbacks used for interacting with the underlying S3 client. */
private S3AInputStream.InputStreamCallbacks client;
// Used for reporting input stream access statistics.
/** Used for reporting input stream access statistics. */
private final S3AInputStreamStatistics streamStatistics;
private S3AInputPolicy inputPolicy;
private final ChangeTracker changeTracker;
private final IOStatistics ioStatistics;
/**
* Initializes a new instance of the {@code S3InputStream} class.
* Initializes a new instance of the {@code S3ARemoteInputStream} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
@ -104,7 +117,7 @@ public abstract class S3InputStream
* @throws IllegalArgumentException if s3Attributes is null.
* @throws IllegalArgumentException if client is null.
*/
public S3InputStream(
public S3ARemoteInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
@ -115,7 +128,7 @@ public abstract class S3InputStream
this.client = requireNonNull(client);
this.streamStatistics = requireNonNull(streamStatistics);
this.ioStatistics = streamStatistics.getIOStatistics();
this.name = S3File.getPath(s3Attributes);
this.name = S3ARemoteObject.getPath(s3Attributes);
this.changeTracker = new ChangeTracker(
this.name,
context.getChangeDetectionPolicy(),
@ -130,8 +143,8 @@ public abstract class S3InputStream
this.blockData = new BlockData(fileSize, bufferSize);
this.fpos = new FilePosition(fileSize, bufferSize);
this.s3File = this.getS3File();
this.reader = new S3Reader(this.s3File);
this.remoteObject = getS3File();
this.reader = new S3ARemoteObjectReader(remoteObject);
this.seekTargetPos = 0;
}
@ -143,7 +156,7 @@ public abstract class S3InputStream
*/
@Override
public IOStatistics getIOStatistics() {
return this.ioStatistics;
return ioStatistics;
}
/**
@ -154,7 +167,7 @@ public abstract class S3InputStream
@InterfaceAudience.Private
@InterfaceStability.Unstable
public S3AInputStreamStatistics getS3AStreamStatistics() {
return this.streamStatistics;
return streamStatistics;
}
/**
@ -197,13 +210,13 @@ public abstract class S3InputStream
*/
@Override
public int available() throws IOException {
this.throwIfClosed();
throwIfClosed();
if (!ensureCurrentBuffer()) {
return 0;
}
return this.fpos.buffer().remaining();
return fpos.buffer().remaining();
}
/**
@ -213,12 +226,12 @@ public abstract class S3InputStream
* @throws IOException if there is an IO error during this operation.
*/
public long getPos() throws IOException {
this.throwIfClosed();
throwIfClosed();
if (this.fpos.isValid()) {
return this.fpos.absolute();
if (fpos.isValid()) {
return fpos.absolute();
} else {
return this.seekTargetPos;
return seekTargetPos;
}
}
@ -231,12 +244,12 @@ public abstract class S3InputStream
* @throws IllegalArgumentException if pos is outside of the range [0, file size].
*/
public void seek(long pos) throws IOException {
this.throwIfClosed();
this.throwIfInvalidSeek(pos);
throwIfClosed();
throwIfInvalidSeek(pos);
if (!this.fpos.setAbsolute(pos)) {
this.fpos.invalidate();
this.seekTargetPos = pos;
if (!fpos.setAbsolute(pos)) {
fpos.invalidate();
seekTargetPos = pos;
}
}
@ -252,9 +265,10 @@ public abstract class S3InputStream
@Override
public int read() throws IOException {
this.throwIfClosed();
throwIfClosed();
if (this.s3File.size() == 0 || this.seekTargetPos >= this.s3File.size()) {
if (remoteObject.size() == 0
|| seekTargetPos >= remoteObject.size()) {
return -1;
}
@ -262,9 +276,9 @@ public abstract class S3InputStream
return -1;
}
this.incrementBytesRead(1);
incrementBytesRead(1);
return this.fpos.buffer().get() & 0xff;
return fpos.buffer().get() & 0xff;
}
/**
@ -278,7 +292,7 @@ public abstract class S3InputStream
*/
@Override
public int read(byte[] buffer) throws IOException {
return this.read(buffer, 0, buffer.length);
return read(buffer, 0, buffer.length);
}
/**
@ -294,13 +308,14 @@ public abstract class S3InputStream
*/
@Override
public int read(byte[] buffer, int offset, int len) throws IOException {
this.throwIfClosed();
throwIfClosed();
if (len == 0) {
return 0;
}
if (this.s3File.size() == 0 || this.seekTargetPos >= this.s3File.size()) {
if (remoteObject.size() == 0
|| seekTargetPos >= remoteObject.size()) {
return -1;
}
@ -316,10 +331,10 @@ public abstract class S3InputStream
break;
}
ByteBuffer buf = this.fpos.buffer();
ByteBuffer buf = fpos.buffer();
int bytesToRead = Math.min(numBytesRemaining, buf.remaining());
buf.get(buffer, offset, bytesToRead);
this.incrementBytesRead(bytesToRead);
incrementBytesRead(bytesToRead);
offset += bytesToRead;
numBytesRemaining -= bytesToRead;
numBytesRead += bytesToRead;
@ -328,71 +343,71 @@ public abstract class S3InputStream
return numBytesRead;
}
protected S3File getFile() {
return this.s3File;
protected S3ARemoteObject getFile() {
return remoteObject;
}
protected S3Reader getReader() {
return this.reader;
protected S3ARemoteObjectReader getReader() {
return reader;
}
protected S3ObjectAttributes getS3ObjectAttributes() {
return this.s3Attributes;
return s3Attributes;
}
protected FilePosition getFilePosition() {
return this.fpos;
return fpos;
}
protected String getName() {
return this.name;
return name;
}
protected boolean isClosed() {
return this.closed;
return closed;
}
protected long getSeekTargetPos() {
return this.seekTargetPos;
return seekTargetPos;
}
protected void setSeekTargetPos(long pos) {
this.seekTargetPos = pos;
seekTargetPos = pos;
}
protected BlockData getBlockData() {
return this.blockData;
return blockData;
}
protected S3AReadOpContext getContext() {
return this.context;
return context;
}
private void incrementBytesRead(int bytesRead) {
if (bytesRead > 0) {
this.streamStatistics.bytesRead(bytesRead);
if (this.getContext().getStats() != null) {
this.getContext().getStats().incrementBytesRead(bytesRead);
streamStatistics.bytesRead(bytesRead);
if (getContext().getStats() != null) {
getContext().getStats().incrementBytesRead(bytesRead);
}
this.fpos.incrementBytesRead(bytesRead);
fpos.incrementBytesRead(bytesRead);
}
}
protected S3File getS3File() {
return new S3File(
this.context,
this.s3Attributes,
this.client,
this.streamStatistics,
this.changeTracker
protected S3ARemoteObject getS3File() {
return new S3ARemoteObject(
context,
s3Attributes,
client,
streamStatistics,
changeTracker
);
}
protected String getOffsetStr(long offset) {
int blockNumber = -1;
if (this.blockData.isValidOffset(offset)) {
blockNumber = this.blockData.getBlockNumber(offset);
if (blockData.isValidOffset(offset)) {
blockNumber = blockData.getBlockNumber(offset);
}
return String.format("%d:%d", blockNumber, offset);
@ -405,22 +420,22 @@ public abstract class S3InputStream
*/
@Override
public void close() throws IOException {
if (this.closed) {
if (closed) {
return;
}
this.closed = true;
closed = true;
this.blockData = null;
this.reader.close();
this.reader = null;
this.s3File = null;
this.fpos.invalidate();
blockData = null;
reader.close();
reader = null;
remoteObject = null;
fpos.invalidate();
try {
this.client.close();
client.close();
} finally {
this.streamStatistics.close();
streamStatistics.close();
}
this.client = null;
client = null;
}
@Override
@ -429,8 +444,9 @@ public abstract class S3InputStream
}
protected void throwIfClosed() throws IOException {
if (this.closed) {
throw new IOException(this.name + ": " + FSExceptionMessages.STREAM_IS_CLOSED);
if (closed) {
throw new IOException(
name + ": " + FSExceptionMessages.STREAM_IS_CLOSED);
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.IOException;
@ -30,7 +30,7 @@ import com.amazonaws.services.s3.model.S3Object;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.common.Validate;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
@ -45,8 +45,10 @@ import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
* Encapsulates low level interactions with S3 object on AWS.
*/
public class S3File {
private static final Logger LOG = LoggerFactory.getLogger(S3File.class);
public class S3ARemoteObject {
private static final Logger LOG =
LoggerFactory.getLogger(S3ARemoteObject.class);
/**
* Read-specific operation context.
@ -90,7 +92,7 @@ public class S3File {
private static final int DRAIN_BUFFER_SIZE = 16384;
/**
* Initializes a new instance of the {@code S3File} class.
* Initializes a new instance of the {@code S3ARemoteObject} class.
*
* @param context read-specific operation context.
* @param s3Attributes attributes of the S3 object being read.
@ -104,7 +106,7 @@ public class S3File {
* @throws IllegalArgumentException if streamStatistics is null.
* @throws IllegalArgumentException if changeTracker is null.
*/
public S3File(
public S3ARemoteObject(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
@ -132,7 +134,7 @@ public class S3File {
* @return an instance of {@code Invoker} for interacting with S3 API.
*/
public Invoker getReadInvoker() {
return this.context.getReadInvoker();
return context.getReadInvoker();
}
/**
@ -141,7 +143,7 @@ public class S3File {
* @return an instance of {@code S3AInputStreamStatistics} used for reporting access metrics.
*/
public S3AInputStreamStatistics getStatistics() {
return this.streamStatistics;
return streamStatistics;
}
/**
@ -150,7 +152,7 @@ public class S3File {
* @return the path of this file.
*/
public String getPath() {
return getPath(this.s3Attributes);
return getPath(s3Attributes);
}
/**
@ -160,7 +162,8 @@ public class S3File {
* @return the path corresponding to the given s3Attributes.
*/
public static String getPath(S3ObjectAttributes s3Attributes) {
return String.format("s3a://%s/%s", s3Attributes.getBucket(), s3Attributes.getKey());
return String.format("s3a://%s/%s", s3Attributes.getBucket(),
s3Attributes.getKey());
}
/**
@ -170,7 +173,7 @@ public class S3File {
* @return the size of this file.
*/
public long size() {
return this.s3Attributes.getLen();
return s3Attributes.getLen();
}
/**
@ -191,9 +194,10 @@ public class S3File {
Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset");
streamStatistics.streamOpened();
final GetObjectRequest request = client.newGetRequest(this.s3Attributes.getKey())
.withRange(offset, offset + size - 1);
this.changeTracker.maybeApplyConstraint(request);
final GetObjectRequest request =
client.newGetRequest(s3Attributes.getKey())
.withRange(offset, offset + size - 1);
changeTracker.maybeApplyConstraint(request);
String operation = String.format(
"%s %s at %d", S3AInputStream.OPERATION_OPEN, uri, offset);
@ -202,7 +206,7 @@ public class S3File {
try {
object = Invoker.once(operation, uri, () -> client.getObject(request));
} catch(IOException e) {
} catch (IOException e) {
tracker.failed();
throw e;
} finally {
@ -211,8 +215,8 @@ public class S3File {
changeTracker.processResponse(object, operation, offset);
InputStream stream = object.getObjectContent();
synchronized (this.s3Objects) {
this.s3Objects.put(stream, object);
synchronized (s3Objects) {
s3Objects.put(stream, object);
}
return stream;
@ -220,22 +224,24 @@ public class S3File {
void close(InputStream inputStream, int numRemainingBytes) {
S3Object obj;
synchronized (this.s3Objects) {
obj = this.s3Objects.get(inputStream);
synchronized (s3Objects) {
obj = s3Objects.get(inputStream);
if (obj == null) {
throw new IllegalArgumentException("inputStream not found");
}
this.s3Objects.remove(inputStream);
s3Objects.remove(inputStream);
}
if (numRemainingBytes <= this.context.getAsyncDrainThreshold()) {
if (numRemainingBytes <= context.getAsyncDrainThreshold()) {
// don't bother with async io.
drain(false, "close() operation", numRemainingBytes, obj, inputStream);
} else {
LOG.debug("initiating asynchronous drain of {} bytes", numRemainingBytes);
// schedule an async drain/abort with references to the fields so they
// can be reused
client.submit(() -> drain(false, "close() operation", numRemainingBytes, obj, inputStream));
client.submit(
() -> drain(false, "close() operation", numRemainingBytes, obj,
inputStream));
}
}
@ -259,8 +265,10 @@ public class S3File {
final InputStream inputStream) {
try {
return invokeTrackingDuration(streamStatistics.initiateInnerStreamClose(shouldAbort),
() -> drainOrAbortHttpStream(shouldAbort, reason, remaining, requestObject, inputStream));
return invokeTrackingDuration(
streamStatistics.initiateInnerStreamClose(shouldAbort),
() -> drainOrAbortHttpStream(shouldAbort, reason, remaining,
requestObject, inputStream));
} catch (IOException e) {
// this is only here because invokeTrackingDuration() has it in its
// signature
@ -303,7 +311,8 @@ public class S3File {
LOG.debug("Drained stream of {} bytes", drained);
} catch (Exception e) {
// exception escalates to an abort
LOG.debug("When closing {} stream for {}, will abort the stream", uri, reason, e);
LOG.debug("When closing {} stream for {}, will abort the stream", uri,
reason, e);
shouldAbort = true;
}
}
@ -311,7 +320,8 @@ public class S3File {
cleanupWithLogger(LOG, requestObject);
streamStatistics.streamClose(shouldAbort, remaining);
LOG.debug("Stream {} {}: {}; remaining={}", uri, (shouldAbort ? "aborted" : "closed"), reason,
LOG.debug("Stream {} {}: {}; remaining={}", uri,
(shouldAbort ? "aborted" : "closed"), reason,
remaining);
return shouldAbort;
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.Closeable;
import java.io.EOFException;
@ -29,7 +29,7 @@ import java.nio.ByteBuffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.common.Validate;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
@ -39,32 +39,34 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDura
/**
* Provides functionality to read S3 file one block at a time.
*/
public class S3Reader implements Closeable {
private static final Logger LOG = LoggerFactory.getLogger(S3Reader.class);
public class S3ARemoteObjectReader implements Closeable {
// We read from the underlying input stream in blocks of this size.
private static final Logger LOG = LoggerFactory.getLogger(
S3ARemoteObjectReader.class);
/** We read from the underlying input stream in blocks of this size. */
private static final int READ_BUFFER_SIZE = 64 * 1024;
// The S3 file to read.
private final S3File s3File;
/** The S3 file to read. */
private final S3ARemoteObject remoteObject;
// Set to true by close().
/** Set to true by close(). */
private volatile boolean closed;
private final S3AInputStreamStatistics streamStatistics;
/**
* Constructs an instance of {@link S3Reader}.
* Constructs an instance of {@link S3ARemoteObjectReader}.
*
* @param s3File The S3 file to read.
* @param remoteObject The S3 file to read.
*
* @throws IllegalArgumentException if s3File is null.
* @throws IllegalArgumentException if remoteObject is null.
*/
public S3Reader(S3File s3File) {
Validate.checkNotNull(s3File, "s3File");
public S3ARemoteObjectReader(S3ARemoteObject remoteObject) {
Validate.checkNotNull(remoteObject, "remoteObject");
this.s3File = s3File;
this.streamStatistics = this.s3File.getStatistics();
this.remoteObject = remoteObject;
this.streamStatistics = this.remoteObject.getStatistics();
}
/**
@ -83,14 +85,14 @@ public class S3Reader implements Closeable {
*/
public int read(ByteBuffer buffer, long offset, int size) throws IOException {
Validate.checkNotNull(buffer, "buffer");
Validate.checkWithinRange(offset, "offset", 0, this.s3File.size());
Validate.checkWithinRange(offset, "offset", 0, this.remoteObject.size());
Validate.checkPositiveInteger(size, "size");
if (this.closed) {
return -1;
}
int reqSize = (int) Math.min(size, this.s3File.size() - offset);
int reqSize = (int) Math.min(size, this.remoteObject.size() - offset);
return readOneBlockWithRetries(buffer, offset, reqSize);
}
@ -103,27 +105,30 @@ public class S3Reader implements Closeable {
throws IOException {
this.streamStatistics.readOperationStarted(offset, size);
Invoker invoker = this.s3File.getReadInvoker();
Invoker invoker = this.remoteObject.getReadInvoker();
int invokerResponse = invoker.retry("read", this.s3File.getPath(), true,
trackDurationOfOperation(streamStatistics, STREAM_READ_REMOTE_BLOCK_READ, () -> {
try {
this.readOneBlock(buffer, offset, size);
} catch (EOFException e) {
// the base implementation swallows EOFs.
return -1;
} catch (SocketTimeoutException e) {
throw e;
} catch (IOException e) {
this.s3File.getStatistics().readException();
throw e;
}
return 0;
}));
int invokerResponse =
invoker.retry("read", this.remoteObject.getPath(), true,
trackDurationOfOperation(streamStatistics,
STREAM_READ_REMOTE_BLOCK_READ, () -> {
try {
this.readOneBlock(buffer, offset, size);
} catch (EOFException e) {
// the base implementation swallows EOFs.
return -1;
} catch (SocketTimeoutException e) {
throw e;
} catch (IOException e) {
this.remoteObject.getStatistics().readException();
throw e;
}
return 0;
}));
int numBytesRead = buffer.position();
buffer.limit(numBytesRead);
this.s3File.getStatistics().readOperationCompleted(size, numBytesRead);
this.remoteObject.getStatistics()
.readOperationCompleted(size, numBytesRead);
if (invokerResponse < 0) {
return invokerResponse;
@ -132,13 +137,14 @@ public class S3Reader implements Closeable {
}
}
private void readOneBlock(ByteBuffer buffer, long offset, int size) throws IOException {
private void readOneBlock(ByteBuffer buffer, long offset, int size)
throws IOException {
int readSize = Math.min(size, buffer.remaining());
if (readSize == 0) {
return;
}
InputStream inputStream = s3File.openForRead(offset, readSize);
InputStream inputStream = remoteObject.openForRead(offset, readSize);
int numRemainingBytes = readSize;
byte[] bytes = new byte[READ_BUFFER_SIZE];
@ -163,7 +169,7 @@ public class S3Reader implements Closeable {
}
while (!this.closed && (numRemainingBytes > 0));
} finally {
s3File.close(inputStream, numRemainingBytes);
remoteObject.close(inputStream, numRemainingBytes);
}
}
}

View File

@ -20,9 +20,10 @@
* High performance s3 input stream which reads in
* blocks and can cache blocks in the local filesystem.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.fs.s3a.statistics;
import org.apache.hadoop.fs.common.PrefetchingStatistics;
import org.apache.hadoop.fs.impl.prefetch.PrefetchingStatistics;
import org.apache.hadoop.fs.statistics.DurationTracker;
/**

View File

@ -1,76 +0,0 @@
/*
* 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.common;
import java.time.Duration;
import org.apache.hadoop.fs.statistics.DurationTracker;
import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker;
public final class EmptyPrefetchingStatistics implements PrefetchingStatistics {
private static final EmptyPrefetchingStatistics EMPTY_PREFETCHING_STATISTICS =
new EmptyPrefetchingStatistics();
private EmptyPrefetchingStatistics() {
}
public static EmptyPrefetchingStatistics getInstance() {
return EMPTY_PREFETCHING_STATISTICS;
}
@Override
public DurationTracker prefetchOperationStarted() {
return stubDurationTracker();
}
@Override
public void blockAddedToFileCache() {
}
@Override
public void blockRemovedFromFileCache() {
}
@Override
public void prefetchOperationCompleted() {
}
@Override
public void executorAcquired(Duration timeInQueue) {
}
@Override
public void memoryAllocated(int size) {
}
@Override
public void memoryFreed(int size) {
}
}

View File

@ -43,27 +43,27 @@ import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatis
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
* Test the prefetching input stream, validates that the underlying S3CachingInputStream and
* S3InMemoryInputStream are working as expected.
* Test the prefetching input stream, validates that the underlying S3ACachingInputStream and
* S3AInMemoryInputStream are working as expected.
*/
public class ITestS3PrefetchingInputStream extends AbstractS3ACostTest {
public class ITestS3APrefetchingInputStream extends AbstractS3ACostTest {
public ITestS3PrefetchingInputStream() {
public ITestS3APrefetchingInputStream() {
super(true);
}
private static final Logger LOG =
LoggerFactory.getLogger(ITestS3PrefetchingInputStream.class);
LoggerFactory.getLogger(ITestS3APrefetchingInputStream.class);
private static final int S_1K = 1024;
private static final int S_1M = S_1K * S_1K;
// Path for file which should have length > block size so S3CachingInputStream is used
// Path for file which should have length > block size so S3ACachingInputStream is used
private Path largeFile;
private FileSystem largeFileFS;
private int numBlocks;
private int blockSize;
private long largeFileSize;
// Size should be < block size so S3InMemoryInputStream is used
// Size should be < block size so S3AInMemoryInputStream is used
private static final int SMALL_FILE_SIZE = S_1K * 16;
@ -104,7 +104,7 @@ public class ITestS3PrefetchingInputStream extends AbstractS3ACostTest {
@Test
public void testReadLargeFileFully() throws Throwable {
describe("read a large file fully, uses S3CachingInputStream");
describe("read a large file fully, uses S3ACachingInputStream");
IOStatistics ioStats;
openFS();
@ -134,7 +134,7 @@ public class ITestS3PrefetchingInputStream extends AbstractS3ACostTest {
@Test
public void testRandomReadLargeFile() throws Throwable {
describe("random read on a large file, uses S3CachingInputStream");
describe("random read on a large file, uses S3ACachingInputStream");
IOStatistics ioStats;
openFS();
@ -163,7 +163,7 @@ public class ITestS3PrefetchingInputStream extends AbstractS3ACostTest {
@Test
public void testRandomReadSmallFile() throws Throwable {
describe("random read on a small file, uses S3InMemoryInputStream");
describe("random read on a small file, uses S3AInMemoryInputStream");
byte[] data = ContractTestUtils.dataset(SMALL_FILE_SIZE, 'a', 26);
Path smallFile = path("randomReadSmallFile");

View File

@ -81,7 +81,7 @@ public class ITestS3ARequesterPays extends AbstractS3ATestBase {
inputStream.readByte();
if (conf.getBoolean(PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT)) {
// For S3PrefetchingInputStream, verify a call was made
// For S3APrefetchingInputStream, verify a call was made
IOStatisticAssertions.assertThatStatisticCounter(inputStream.getIOStatistics(),
StreamStatisticNames.STREAM_READ_OPENED).isEqualTo(1);
} else {

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.ByteArrayInputStream;
import java.io.IOException;
@ -27,7 +27,7 @@ import java.util.concurrent.CompletableFuture;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.S3Object;
import org.apache.hadoop.fs.common.Validate;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
import org.apache.hadoop.util.functional.CallableRaisingIOE;
@ -35,7 +35,8 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE;
/**
* A mock s3 file with some fault injection.
*/
class MockS3File extends S3File {
class MockS3ARemoteObject extends S3ARemoteObject {
private byte[] contents;
// If true, throws IOException on open request just once.
@ -43,19 +44,20 @@ class MockS3File extends S3File {
private boolean throwExceptionOnOpen;
private static final String BUCKET = "bucket";
private static final String KEY = "key";
MockS3File(int size) {
MockS3ARemoteObject(int size) {
this(size, false);
}
MockS3File(int size, boolean throwExceptionOnOpen) {
MockS3ARemoteObject(int size, boolean throwExceptionOnOpen) {
super(
Fakes.createReadContext(null, KEY, size, 1, 1),
Fakes.createObjectAttributes(BUCKET, KEY, size),
Fakes.createInputStreamCallbacks(BUCKET, KEY),
new EmptyS3AStatisticsContext().EMPTY_INPUT_STREAM_STATISTICS,
Fakes.createChangeTracker(BUCKET, KEY, size)
S3APrefetchFakes.createReadContext(null, KEY, size, 1, 1),
S3APrefetchFakes.createObjectAttributes(BUCKET, KEY, size),
S3APrefetchFakes.createInputStreamCallbacks(BUCKET, KEY),
EmptyS3AStatisticsContext.EMPTY_INPUT_STREAM_STATISTICS,
S3APrefetchFakes.createChangeTracker(BUCKET, KEY, size)
);
this.throwExceptionOnOpen = throwExceptionOnOpen;
@ -70,8 +72,8 @@ class MockS3File extends S3File {
Validate.checkLessOrEqual(offset, "offset", size(), "size()");
Validate.checkLessOrEqual(size, "size", size() - offset, "size() - offset");
if (this.throwExceptionOnOpen) {
this.throwExceptionOnOpen = false;
if (throwExceptionOnOpen) {
throwExceptionOnOpen = false;
throw new IOException("Throwing because throwExceptionOnOpen is true ");
}
int bufSize = (int) Math.min(size, size() - offset);

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.ByteArrayInputStream;
import java.io.IOException;
@ -37,11 +37,11 @@ import com.amazonaws.services.s3.model.S3Object;
import com.amazonaws.services.s3.model.S3ObjectInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.common.BlockCache;
import org.apache.hadoop.fs.common.BlockData;
import org.apache.hadoop.fs.common.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.common.SingleFilePerBlockCache;
import org.apache.hadoop.fs.common.Validate;
import org.apache.hadoop.fs.impl.prefetch.BlockCache;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.SingleFilePerBlockCache;
import org.apache.hadoop.fs.impl.prefetch.Validate;
import org.apache.hadoop.fs.s3a.Invoker;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
import org.apache.hadoop.fs.s3a.S3AFileStatus;
@ -63,7 +63,7 @@ import org.apache.hadoop.util.functional.CallableRaisingIOE;
import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatisticsStore;
/**
* Provides 'fake' implementations of S3InputStream variants.
* Provides 'fake' implementations of S3ARemoteInputStream variants.
*
* These implementations avoid accessing the following real resources:
* -- S3 store
@ -73,14 +73,19 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStat
* implementations without accessing external resources. It also helps
* avoid test flakiness introduced by external factors.
*/
public final class Fakes {
public final class S3APrefetchFakes {
private Fakes() {}
private S3APrefetchFakes() {
}
public static final String E_TAG = "eTag";
public static final String OWNER = "owner";
public static final String VERSION_ID = "v1";
public static final long MODIFICATION_TIME = 0L;
public static final ChangeDetectionPolicy CHANGE_POLICY =
ChangeDetectionPolicy.createPolicy(
ChangeDetectionPolicy.Mode.None,
@ -125,7 +130,8 @@ public final class Fakes {
FileSystem.Statistics statistics = new FileSystem.Statistics("s3a");
S3AStatisticsContext statisticsContext = new EmptyS3AStatisticsContext();
RetryPolicy retryPolicy =
RetryPolicies.retryUpToMaximumCountWithFixedSleep(3, 10, TimeUnit.MILLISECONDS);
RetryPolicies.retryUpToMaximumCountWithFixedSleep(3, 10,
TimeUnit.MILLISECONDS);
return new S3AReadOpContext(
path,
@ -172,18 +178,18 @@ public final class Fakes {
String key) {
S3Object object = new S3Object() {
@Override
public S3ObjectInputStream getObjectContent() {
return createS3ObjectInputStream(new byte[8]);
}
@Override
public S3ObjectInputStream getObjectContent() {
return createS3ObjectInputStream(new byte[8]);
}
@Override
public ObjectMetadata getObjectMetadata() {
ObjectMetadata metadata = new ObjectMetadata();
metadata.setHeader("ETag", E_TAG);
return metadata;
}
};
@Override
public ObjectMetadata getObjectMetadata() {
ObjectMetadata metadata = new ObjectMetadata();
metadata.setHeader("ETag", E_TAG);
return metadata;
}
};
return new S3AInputStream.InputStreamCallbacks() {
@Override
@ -208,8 +214,8 @@ public final class Fakes {
}
public static S3InputStream createInputStream(
Class<? extends S3InputStream> clazz,
public static S3ARemoteInputStream createInputStream(
Class<? extends S3ARemoteInputStream> clazz,
ExecutorServiceFuturePool futurePool,
String bucket,
String key,
@ -220,7 +226,8 @@ public final class Fakes {
org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(key);
S3AFileStatus fileStatus = createFileStatus(key, fileSize);
S3ObjectAttributes s3ObjectAttributes = createObjectAttributes(bucket, key, fileSize);
S3ObjectAttributes s3ObjectAttributes =
createObjectAttributes(bucket, key, fileSize);
S3AReadOpContext s3AReadOpContext = createReadContext(
futurePool,
key,
@ -228,30 +235,34 @@ public final class Fakes {
prefetchBlockSize,
prefetchBlockCount);
S3AInputStream.InputStreamCallbacks callbacks = createInputStreamCallbacks(bucket, key);
S3AInputStream.InputStreamCallbacks callbacks =
createInputStreamCallbacks(bucket, key);
S3AInputStreamStatistics stats =
s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics();
if (clazz == TestS3InMemoryInputStream.class) {
return new TestS3InMemoryInputStream(s3AReadOpContext, s3ObjectAttributes, callbacks, stats);
} else if (clazz == TestS3CachingInputStream.class) {
return new TestS3CachingInputStream(s3AReadOpContext, s3ObjectAttributes, callbacks, stats);
if (clazz == FakeS3AInMemoryInputStream.class) {
return new FakeS3AInMemoryInputStream(s3AReadOpContext,
s3ObjectAttributes, callbacks, stats);
} else if (clazz == FakeS3ACachingInputStream.class) {
return new FakeS3ACachingInputStream(s3AReadOpContext, s3ObjectAttributes,
callbacks, stats);
}
throw new RuntimeException("Unsupported class: " + clazz);
}
public static TestS3InMemoryInputStream createS3InMemoryInputStream(
public static FakeS3AInMemoryInputStream createS3InMemoryInputStream(
ExecutorServiceFuturePool futurePool,
String bucket,
String key,
int fileSize) {
return (TestS3InMemoryInputStream) createInputStream(
TestS3InMemoryInputStream.class, futurePool, bucket, key, fileSize, 1, 1);
return (FakeS3AInMemoryInputStream) createInputStream(
FakeS3AInMemoryInputStream.class, futurePool, bucket, key, fileSize, 1,
1);
}
public static TestS3CachingInputStream createS3CachingInputStream(
public static FakeS3ACachingInputStream createS3CachingInputStream(
ExecutorServiceFuturePool futurePool,
String bucket,
String key,
@ -259,8 +270,8 @@ public final class Fakes {
int prefetchBlockSize,
int prefetchBlockCount) {
return (TestS3CachingInputStream) createInputStream(
TestS3CachingInputStream.class,
return (FakeS3ACachingInputStream) createInputStream(
FakeS3ACachingInputStream.class,
futurePool,
bucket,
key,
@ -269,8 +280,10 @@ public final class Fakes {
prefetchBlockCount);
}
public static class TestS3InMemoryInputStream extends S3InMemoryInputStream {
public TestS3InMemoryInputStream(
public static class FakeS3AInMemoryInputStream
extends S3AInMemoryInputStream {
public FakeS3AInMemoryInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
@ -279,18 +292,22 @@ public final class Fakes {
}
@Override
protected S3File getS3File() {
protected S3ARemoteObject getS3File() {
randomDelay(200);
return new MockS3File((int) this.getS3ObjectAttributes().getLen(), false);
return new MockS3ARemoteObject(
(int) this.getS3ObjectAttributes().getLen(), false);
}
}
public static class TestS3FilePerBlockCache extends SingleFilePerBlockCache {
public static class FakeS3FilePerBlockCache extends SingleFilePerBlockCache {
private final Map<Path, byte[]> files;
private final int readDelay;
private final int writeDelay;
public TestS3FilePerBlockCache(int readDelay, int writeDelay) {
public FakeS3FilePerBlockCache(int readDelay, int writeDelay) {
super(new EmptyS3AStatisticsContext().newInputStreamStatistics());
this.files = new ConcurrentHashMap<>();
this.readDelay = readDelay;
@ -339,10 +356,12 @@ public final class Fakes {
}
}
public static class TestS3CachingBlockManager extends S3CachingBlockManager {
public TestS3CachingBlockManager(
public static class FakeS3ACachingBlockManager
extends S3ACachingBlockManager {
public FakeS3ACachingBlockManager(
ExecutorServiceFuturePool futurePool,
S3Reader reader,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize) {
super(futurePool, reader, blockData, bufferPoolSize,
@ -350,7 +369,8 @@ public final class Fakes {
}
@Override
public int read(ByteBuffer buffer, long offset, int size) throws IOException {
public int read(ByteBuffer buffer, long offset, int size)
throws IOException {
randomDelay(100);
return this.getReader().read(buffer, offset, size);
}
@ -359,12 +379,13 @@ public final class Fakes {
protected BlockCache createCache() {
final int readDelayMs = 50;
final int writeDelayMs = 200;
return new TestS3FilePerBlockCache(readDelayMs, writeDelayMs);
return new FakeS3FilePerBlockCache(readDelayMs, writeDelayMs);
}
}
public static class TestS3CachingInputStream extends S3CachingInputStream {
public TestS3CachingInputStream(
public static class FakeS3ACachingInputStream extends S3ACachingInputStream {
public FakeS3ACachingInputStream(
S3AReadOpContext context,
S3ObjectAttributes s3Attributes,
S3AInputStream.InputStreamCallbacks client,
@ -373,18 +394,20 @@ public final class Fakes {
}
@Override
protected S3File getS3File() {
protected S3ARemoteObject getS3File() {
randomDelay(200);
return new MockS3File((int) this.getS3ObjectAttributes().getLen(), false);
return new MockS3ARemoteObject(
(int) this.getS3ObjectAttributes().getLen(), false);
}
@Override
protected S3CachingBlockManager createBlockManager(
protected S3ACachingBlockManager createBlockManager(
ExecutorServiceFuturePool futurePool,
S3Reader reader,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize) {
return new TestS3CachingBlockManager(futurePool, reader, blockData, bufferPoolSize);
return new FakeS3ACachingBlockManager(futurePool, reader, blockData,
bufferPoolSize);
}
}
}

View File

@ -17,62 +17,63 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.junit.Test;
import org.apache.hadoop.fs.common.BlockData;
import org.apache.hadoop.fs.common.BufferData;
import org.apache.hadoop.fs.common.ExceptionAsserts;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.BufferData;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
public class TestS3BlockManager extends AbstractHadoopTestBase {
public class TestS3ABlockManager extends AbstractHadoopTestBase {
static final int FILE_SIZE = 12;
static final int BLOCK_SIZE = 3;
@Test
public void testArgChecks() throws Exception {
BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE);
MockS3File s3File = new MockS3File(FILE_SIZE, false);
S3Reader reader = new S3Reader(s3File);
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
// Should not throw.
new S3BlockManager(reader, blockData);
new S3ABlockManager(reader, blockData);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'reader' must not be null",
() -> new S3BlockManager(null, blockData));
() -> new S3ABlockManager(null, blockData));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'blockData' must not be null",
() -> new S3BlockManager(reader, null));
() -> new S3ABlockManager(reader, null));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> new S3BlockManager(reader, blockData).get(-1));
() -> new S3ABlockManager(reader, blockData).get(-1));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'data' must not be null",
() -> new S3BlockManager(reader, blockData).release(null));
() -> new S3ABlockManager(reader, blockData).release(null));
}
@Test
public void testGet() throws IOException {
BlockData blockData = new BlockData(FILE_SIZE, BLOCK_SIZE);
MockS3File s3File = new MockS3File(FILE_SIZE, false);
S3Reader reader = new S3Reader(s3File);
S3BlockManager blockManager = new S3BlockManager(reader, blockData);
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
S3ABlockManager blockManager = new S3ABlockManager(reader, blockData);
for (int b = 0; b < blockData.getNumBlocks(); b++) {
BufferData data = blockManager.get(b);

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.IOException;
import java.nio.ByteBuffer;
@ -26,23 +26,29 @@ import java.util.concurrent.Executors;
import org.junit.Test;
import org.apache.hadoop.fs.common.BlockData;
import org.apache.hadoop.fs.common.BufferData;
import org.apache.hadoop.fs.common.ExceptionAsserts;
import org.apache.hadoop.fs.common.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.BlockData;
import org.apache.hadoop.fs.impl.prefetch.BufferData;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
public class TestS3ACachingBlockManager extends AbstractHadoopTestBase {
static final int FILE_SIZE = 15;
static final int BLOCK_SIZE = 2;
static final int POOL_SIZE = 3;
private final ExecutorService threadPool = Executors.newFixedThreadPool(4);
private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool);
private final ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(threadPool);
private final S3AInputStreamStatistics streamStatistics =
new EmptyS3AStatisticsContext().newInputStreamStatistics();
@ -50,69 +56,78 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
@Test
public void testArgChecks() throws Exception {
MockS3File s3File = new MockS3File(FILE_SIZE, false);
S3Reader reader = new S3Reader(s3File);
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
// Should not throw.
S3CachingBlockManager blockManager =
new S3CachingBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics);
S3ACachingBlockManager blockManager =
new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
intercept(
NullPointerException.class,
() -> new S3CachingBlockManager(null, reader, blockData, POOL_SIZE, streamStatistics));
() -> new S3ACachingBlockManager(null, reader, blockData, POOL_SIZE,
streamStatistics));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'reader' must not be null",
() -> new S3CachingBlockManager(futurePool, null, blockData, POOL_SIZE, streamStatistics));
() -> new S3ACachingBlockManager(futurePool, null, blockData, POOL_SIZE,
streamStatistics));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'blockData' must not be null",
() -> new S3CachingBlockManager(futurePool, reader, null, POOL_SIZE, streamStatistics));
() -> new S3ACachingBlockManager(futurePool, reader, null, POOL_SIZE,
streamStatistics));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'bufferPoolSize' must be a positive integer",
() -> new S3CachingBlockManager(futurePool, reader, blockData, 0, streamStatistics));
() -> new S3ACachingBlockManager(futurePool, reader, blockData, 0,
streamStatistics));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'bufferPoolSize' must be a positive integer",
() -> new S3CachingBlockManager(futurePool, reader, blockData, -1, streamStatistics));
() -> new S3ACachingBlockManager(futurePool, reader, blockData, -1,
streamStatistics));
ExceptionAsserts.assertThrows(NullPointerException.class,
() -> new S3CachingBlockManager(futurePool, reader, blockData, POOL_SIZE, null));
intercept(NullPointerException.class,
() -> new S3ACachingBlockManager(futurePool, reader, blockData,
POOL_SIZE, null));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> blockManager.get(-1));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'data' must not be null",
() -> blockManager.release(null));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'blockNumber' must not be negative",
() -> blockManager.requestPrefetch(-1));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'data' must not be null",
() -> blockManager.requestCaching(null));
}
/**
* Extends S3CachingBlockManager so that we can inject asynchronous failures.
* Extends S3ACachingBlockManager so that we can inject asynchronous failures.
*/
static class TestBlockManager extends S3CachingBlockManager {
TestBlockManager(
private static final class BlockManagerForTesting
extends S3ACachingBlockManager {
BlockManagerForTesting(
ExecutorServiceFuturePool futurePool,
S3Reader reader,
S3ARemoteObjectReader reader,
BlockData blockData,
int bufferPoolSize,
S3AInputStreamStatistics streamStatistics) {
@ -124,7 +139,8 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
private boolean forceNextReadToFail;
@Override
public int read(ByteBuffer buffer, long offset, int size) throws IOException {
public int read(ByteBuffer buffer, long offset, int size)
throws IOException {
if (forceNextReadToFail) {
forceNextReadToFail = false;
throw new RuntimeException("foo");
@ -138,7 +154,8 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
private boolean forceNextCachePutToFail;
@Override
protected void cachePut(int blockNumber, ByteBuffer buffer) throws IOException {
protected void cachePut(int blockNumber, ByteBuffer buffer)
throws IOException {
if (forceNextCachePutToFail) {
forceNextCachePutToFail = false;
throw new RuntimeException("bar");
@ -161,10 +178,11 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
}
private void testGetHelper(boolean forceReadFailure) throws Exception {
MockS3File s3File = new MockS3File(FILE_SIZE, true);
S3Reader reader = new S3Reader(s3File);
TestBlockManager blockManager =
new TestBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics);
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, true);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
BlockManagerForTesting blockManager =
new BlockManagerForTesting(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
for (int b = 0; b < blockData.getNumBlocks(); b++) {
// We simulate caching failure for all even numbered blocks.
@ -175,7 +193,7 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
if (forceFailure) {
blockManager.forceNextReadToFail = true;
ExceptionAsserts.assertThrows(
intercept(
RuntimeException.class,
"foo",
() -> blockManager.get(3));
@ -208,10 +226,11 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
private void testPrefetchHelper(boolean forcePrefetchFailure)
throws IOException, InterruptedException {
MockS3File s3File = new MockS3File(FILE_SIZE, false);
S3Reader reader = new S3Reader(s3File);
TestBlockManager blockManager =
new TestBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics);
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
BlockManagerForTesting blockManager =
new BlockManagerForTesting(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
assertInitialState(blockManager);
int expectedNumErrors = 0;
@ -239,11 +258,13 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
// @Ignore
@Test
public void testCachingOfPrefetched() throws IOException, InterruptedException {
MockS3File s3File = new MockS3File(FILE_SIZE, false);
S3Reader reader = new S3Reader(s3File);
S3CachingBlockManager blockManager =
new S3CachingBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics);
public void testCachingOfPrefetched()
throws IOException, InterruptedException {
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
S3ACachingBlockManager blockManager =
new S3ACachingBlockManager(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
assertInitialState(blockManager);
for (int b = 0; b < blockData.getNumBlocks(); b++) {
@ -265,16 +286,18 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
// @Ignore
@Test
public void testCachingFailureOfGet() throws IOException, InterruptedException {
public void testCachingFailureOfGet()
throws IOException, InterruptedException {
testCachingOfGetHelper(true);
}
public void testCachingOfGetHelper(boolean forceCachingFailure)
throws IOException, InterruptedException {
MockS3File s3File = new MockS3File(FILE_SIZE, false);
S3Reader reader = new S3Reader(s3File);
TestBlockManager blockManager =
new TestBlockManager(futurePool, reader, blockData, POOL_SIZE, streamStatistics);
MockS3ARemoteObject s3File = new MockS3ARemoteObject(FILE_SIZE, false);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(s3File);
BlockManagerForTesting blockManager =
new BlockManagerForTesting(futurePool, reader, blockData, POOL_SIZE,
streamStatistics);
assertInitialState(blockManager);
int expectedNumErrors = 0;
@ -307,9 +330,9 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
}
private void waitForCaching(
S3CachingBlockManager blockManager,
S3ACachingBlockManager blockManager,
int expectedCount)
throws InterruptedException {
throws InterruptedException {
// Wait for async cache operation to be over.
int numTrys = 0;
int count;
@ -320,18 +343,19 @@ public class TestS3CachingBlockManager extends AbstractHadoopTestBase {
if (numTrys > 600) {
String message = String.format(
"waitForCaching: expected: %d, actual: %d, read errors: %d, caching errors: %d",
expectedCount, count, blockManager.numReadErrors(), blockManager.numCachingErrors());
expectedCount, count, blockManager.numReadErrors(),
blockManager.numCachingErrors());
throw new IllegalStateException(message);
}
}
while (count < expectedCount);
}
private int totalErrors(S3CachingBlockManager blockManager) {
private int totalErrors(S3ACachingBlockManager blockManager) {
return blockManager.numCachingErrors() + blockManager.numReadErrors();
}
private void assertInitialState(S3CachingBlockManager blockManager) {
private void assertInitialState(S3ACachingBlockManager blockManager) {
assertEquals(0, blockManager.numCached());
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.io.EOFException;
import java.io.IOException;
@ -27,8 +27,8 @@ import java.util.concurrent.Executors;
import org.junit.Test;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.common.ExceptionAsserts;
import org.apache.hadoop.fs.common.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
@ -38,54 +38,64 @@ import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.junit.Assert.assertEquals;
/**
* Applies the same set of tests to both S3CachingInputStream and S3InMemoryInputStream.
* Applies the same set of tests to both S3ACachingInputStream and S3AInMemoryInputStream.
*/
public class TestS3InputStream extends AbstractHadoopTestBase {
public class TestS3ARemoteInputStream extends AbstractHadoopTestBase {
private static final int FILE_SIZE = 10;
private final ExecutorService threadPool = Executors.newFixedThreadPool(4);
private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool);
private final S3AInputStream.InputStreamCallbacks client = MockS3File.createClient("bucket");
private final ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(threadPool);
private final S3AInputStream.InputStreamCallbacks client =
MockS3ARemoteObject.createClient("bucket");
@Test
public void testArgChecks() throws Exception {
S3AReadOpContext readContext = Fakes.createReadContext(futurePool, "key", 10, 10, 1);
S3ObjectAttributes attrs = Fakes.createObjectAttributes("bucket", "key", 10);
S3AReadOpContext readContext =
S3APrefetchFakes.createReadContext(futurePool, "key", 10, 10, 1);
S3ObjectAttributes attrs =
S3APrefetchFakes.createObjectAttributes("bucket", "key", 10);
S3AInputStreamStatistics stats =
readContext.getS3AStatisticsContext().newInputStreamStatistics();
// Should not throw.
new S3CachingInputStream(readContext, attrs, client, stats);
new S3ACachingInputStream(readContext, attrs, client, stats);
ExceptionAsserts.assertThrows(
NullPointerException.class,
() -> new S3CachingInputStream(null, attrs, client, stats));
() -> new S3ACachingInputStream(null, attrs, client, stats));
ExceptionAsserts.assertThrows(
NullPointerException.class,
() -> new S3CachingInputStream(readContext, null, client, stats));
() -> new S3ACachingInputStream(readContext, null, client, stats));
ExceptionAsserts.assertThrows(
NullPointerException.class,
() -> new S3CachingInputStream(readContext, attrs, null, stats));
() -> new S3ACachingInputStream(readContext, attrs, null, stats));
ExceptionAsserts.assertThrows(
NullPointerException.class,
() -> new S3CachingInputStream(readContext, attrs, client, null));
() -> new S3ACachingInputStream(readContext, attrs, client, null));
}
@Test
public void testRead0SizedFile() throws Exception {
S3InputStream inputStream =
Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", 0);
S3ARemoteInputStream inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", 0);
testRead0SizedFileHelper(inputStream, 9);
inputStream = Fakes.createS3CachingInputStream(futurePool, "bucket", "key", 0, 5, 2);
inputStream =
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
0, 5, 2);
testRead0SizedFileHelper(inputStream, 5);
}
private void testRead0SizedFileHelper(S3InputStream inputStream, int bufferSize)
private void testRead0SizedFileHelper(S3ARemoteInputStream inputStream,
int bufferSize)
throws Exception {
assertEquals(0, inputStream.available());
assertEquals(-1, inputStream.read());
@ -98,16 +108,19 @@ public class TestS3InputStream extends AbstractHadoopTestBase {
@Test
public void testRead() throws Exception {
S3InputStream inputStream =
Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", FILE_SIZE);
S3ARemoteInputStream inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", FILE_SIZE);
testReadHelper(inputStream, FILE_SIZE);
inputStream =
Fakes.createS3CachingInputStream(futurePool, "bucket", "key", FILE_SIZE, 5, 2);
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
FILE_SIZE, 5, 2);
testReadHelper(inputStream, 5);
}
private void testReadHelper(S3InputStream inputStream, int bufferSize) throws Exception {
private void testReadHelper(S3ARemoteInputStream inputStream, int bufferSize)
throws Exception {
assertEquals(bufferSize, inputStream.available());
assertEquals(0, inputStream.read());
assertEquals(1, inputStream.read());
@ -141,15 +154,21 @@ public class TestS3InputStream extends AbstractHadoopTestBase {
@Test
public void testSeek() throws Exception {
S3InputStream inputStream;
inputStream = Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", 9);
S3ARemoteInputStream inputStream;
inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", 9);
testSeekHelper(inputStream, 9, 9);
inputStream = Fakes.createS3CachingInputStream(futurePool, "bucket", "key", 9, 5, 1);
inputStream =
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
9, 5, 1);
testSeekHelper(inputStream, 5, 9);
}
private void testSeekHelper(S3InputStream inputStream, int bufferSize, int fileSize)
private void testSeekHelper(S3ARemoteInputStream inputStream,
int bufferSize,
int fileSize)
throws Exception {
assertEquals(0, inputStream.getPos());
inputStream.seek(7);
@ -177,15 +196,21 @@ public class TestS3InputStream extends AbstractHadoopTestBase {
@Test
public void testRandomSeek() throws Exception {
S3InputStream inputStream;
inputStream = Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", 9);
S3ARemoteInputStream inputStream;
inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", 9);
testRandomSeekHelper(inputStream, 9, 9);
inputStream = Fakes.createS3CachingInputStream(futurePool, "bucket", "key", 9, 5, 1);
inputStream =
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
9, 5, 1);
testRandomSeekHelper(inputStream, 5, 9);
}
private void testRandomSeekHelper(S3InputStream inputStream, int bufferSize, int fileSize)
private void testRandomSeekHelper(S3ARemoteInputStream inputStream,
int bufferSize,
int fileSize)
throws Exception {
assertEquals(0, inputStream.getPos());
inputStream.seek(7);
@ -213,16 +238,19 @@ public class TestS3InputStream extends AbstractHadoopTestBase {
@Test
public void testClose() throws Exception {
S3InputStream inputStream =
Fakes.createS3InMemoryInputStream(futurePool, "bucket", "key", 9);
S3ARemoteInputStream inputStream =
S3APrefetchFakes.createS3InMemoryInputStream(futurePool, "bucket",
"key", 9);
testCloseHelper(inputStream, 9);
inputStream =
Fakes.createS3CachingInputStream(futurePool, "bucket", "key", 9, 5, 3);
S3APrefetchFakes.createS3CachingInputStream(futurePool, "bucket", "key",
9, 5, 3);
testCloseHelper(inputStream, 5);
}
private void testCloseHelper(S3InputStream inputStream, int bufferSize) throws Exception {
private void testCloseHelper(S3ARemoteInputStream inputStream, int bufferSize)
throws Exception {
assertEquals(bufferSize, inputStream.available());
assertEquals(0, inputStream.read());
assertEquals(1, inputStream.read());

View File

@ -17,15 +17,15 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.junit.Test;
import org.apache.hadoop.fs.common.ExceptionAsserts;
import org.apache.hadoop.fs.common.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.impl.prefetch.ExceptionAsserts;
import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3AReadOpContext;
import org.apache.hadoop.fs.s3a.S3ObjectAttributes;
@ -33,45 +33,56 @@ import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.apache.hadoop.test.AbstractHadoopTestBase;
public class TestS3File extends AbstractHadoopTestBase {
public class TestS3ARemoteObject extends AbstractHadoopTestBase {
private final ExecutorService threadPool = Executors.newFixedThreadPool(1);
private final ExecutorServiceFuturePool futurePool = new ExecutorServiceFuturePool(threadPool);
private final S3AInputStream.InputStreamCallbacks client = MockS3File.createClient("bucket");
private final ExecutorServiceFuturePool futurePool =
new ExecutorServiceFuturePool(threadPool);
private final S3AInputStream.InputStreamCallbacks client =
MockS3ARemoteObject.createClient("bucket");
@Test
public void testArgChecks() throws Exception {
S3AReadOpContext readContext = Fakes.createReadContext(futurePool, "key", 10, 10, 1);
S3ObjectAttributes attrs = Fakes.createObjectAttributes("bucket", "key", 10);
S3AReadOpContext readContext =
S3APrefetchFakes.createReadContext(futurePool, "key", 10, 10, 1);
S3ObjectAttributes attrs =
S3APrefetchFakes.createObjectAttributes("bucket", "key", 10);
S3AInputStreamStatistics stats =
readContext.getS3AStatisticsContext().newInputStreamStatistics();
ChangeTracker changeTracker = Fakes.createChangeTracker("bucket", "key", 10);
ChangeTracker changeTracker =
S3APrefetchFakes.createChangeTracker("bucket", "key", 10);
// Should not throw.
new S3File(readContext, attrs, client, stats, changeTracker);
new S3ARemoteObject(readContext, attrs, client, stats, changeTracker);
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'context' must not be null",
() -> new S3File(null, attrs, client, stats, changeTracker));
() -> new S3ARemoteObject(null, attrs, client, stats, changeTracker));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'s3Attributes' must not be null",
() -> new S3File(readContext, null, client, stats, changeTracker));
() -> new S3ARemoteObject(readContext, null, client, stats,
changeTracker));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'client' must not be null",
() -> new S3File(readContext, attrs, null, stats, changeTracker));
() -> new S3ARemoteObject(readContext, attrs, null, stats,
changeTracker));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'streamStatistics' must not be null",
() -> new S3File(readContext, attrs, client, null, changeTracker));
() -> new S3ARemoteObject(readContext, attrs, client, null,
changeTracker));
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'changeTracker' must not be null",
() -> new S3File(readContext, attrs, client, stats, null));
() -> new S3ARemoteObject(readContext, attrs, client, stats, null));
}
}

View File

@ -17,57 +17,61 @@
* under the License.
*/
package org.apache.hadoop.fs.s3a.read;
package org.apache.hadoop.fs.s3a.prefetch;
import java.nio.ByteBuffer;
import org.junit.Test;
import org.apache.hadoop.fs.common.ExceptionAsserts;
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.junit.Assert.assertEquals;
public class TestS3Reader extends AbstractHadoopTestBase {
public class TestS3ARemoteObjectReader extends AbstractHadoopTestBase {
private static final int FILE_SIZE = 9;
private static final int BUFFER_SIZE = 2;
private final S3File s3File = new MockS3File(FILE_SIZE, false);
private final S3ARemoteObject remoteObject =
new MockS3ARemoteObject(FILE_SIZE, false);
@Test
public void testArgChecks() throws Exception {
// Should not throw.
S3Reader reader = new S3Reader(s3File);
S3ARemoteObjectReader reader = new S3ARemoteObjectReader(remoteObject);
// Verify it throws correctly.
ExceptionAsserts.assertThrows(
IllegalArgumentException.class,
"'s3File' must not be null",
() -> new S3Reader(null));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'remoteObject' must not be null",
() -> new S3ARemoteObjectReader(null));
intercept(
IllegalArgumentException.class,
"'buffer' must not be null",
() -> reader.read(null, 10, 2));
ByteBuffer buffer = ByteBuffer.allocate(BUFFER_SIZE);
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'offset' (-1) must be within the range [0, 9]",
() -> reader.read(buffer, -1, 2));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'offset' (11) must be within the range [0, 9]",
() -> reader.read(buffer, 11, 2));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'size' must be a positive integer",
() -> reader.read(buffer, 1, 0));
ExceptionAsserts.assertThrows(
intercept(
IllegalArgumentException.class,
"'size' must be a positive integer",
() -> reader.read(buffer, 1, -1));
@ -85,7 +89,9 @@ public class TestS3Reader extends AbstractHadoopTestBase {
throws Exception {
int numBlocks = 0;
ByteBuffer buffer;
S3Reader reader = new S3Reader(new MockS3File(FILE_SIZE, testWithRetry));
S3ARemoteObjectReader reader =
new S3ARemoteObjectReader(
new MockS3ARemoteObject(FILE_SIZE, testWithRetry));
int remainingSize = FILE_SIZE - (int) startOffset;
for (int bufferSize = 0; bufferSize <= FILE_SIZE + 1; bufferSize++) {
buffer = ByteBuffer.allocate(bufferSize);
@ -97,7 +103,7 @@ public class TestS3Reader extends AbstractHadoopTestBase {
assertEquals(expectedNumBytesRead, numBytesRead);
byte[] bytes = buffer.array();
for (int i = 0; i< expectedNumBytesRead; i++) {
for (int i = 0; i < expectedNumBytesRead; i++) {
assertEquals(startOffset + i, bytes[i]);
}
}

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3ATestConstants;
import org.apache.hadoop.fs.s3a.Statistic;
import org.apache.hadoop.fs.s3a.read.S3PrefetchingInputStream;
import org.apache.hadoop.fs.s3a.prefetch.S3APrefetchingInputStream;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -166,10 +166,10 @@ public class S3AScaleTestBase extends AbstractS3ATestBase {
InputStream inner = in.getWrappedStream();
if (inner instanceof S3AInputStream) {
return ((S3AInputStream) inner).getS3AStreamStatistics();
} else if (inner instanceof S3PrefetchingInputStream) {
return ((S3PrefetchingInputStream) inner).getS3AStreamStatistics();
} else if (inner instanceof S3APrefetchingInputStream) {
return ((S3APrefetchingInputStream) inner).getS3AStreamStatistics();
} else {
throw new AssertionError("Not an S3AInputStream or S3PrefetchingInputStream: " + inner);
throw new AssertionError("Not an S3AInputStream or S3APrefetchingInputStream: " + inner);
}
}