diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockCache.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java similarity index 98% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockCache.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java index 16354c7be5f..c18dc519188 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockCache.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockCache.java @@ -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; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java similarity index 71% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockData.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java index df226d09938..ecb8bc7243b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockData.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockData.java @@ -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); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java similarity index 85% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockManager.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java index f5672aa6920..45f0aabe7dc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockManager.java @@ -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. } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockOperations.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockOperations.java similarity index 83% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockOperations.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockOperations.java index 2b322856293..2744334a3bd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BlockOperations.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BlockOperations.java @@ -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 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 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> 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); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BoundedResourcePool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BoundedResourcePool.java similarity index 74% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BoundedResourcePool.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BoundedResourcePool.java index 40f0a1cee57..a871f823772 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BoundedResourcePool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BoundedResourcePool.java @@ -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 extends ResourcePool { - // 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 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 createdItems; /** @@ -50,7 +58,7 @@ public abstract class BoundedResourcePool extends ResourcePool { Validate.checkPositiveInteger(size, "size"); this.size = size; - this.items = new ArrayBlockingQueue(size); + this.items = new ArrayBlockingQueue<>(size); // The created items are identified based on their object reference. this.createdItems = Collections.newSetFromMap(new IdentityHashMap()); @@ -79,41 +87,40 @@ public abstract class BoundedResourcePool extends ResourcePool { */ @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 extends ResourcePool { // 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 extends ResourcePool { 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 extends ResourcePool { // 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 extends ResourcePool { 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; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java similarity index 88% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferData.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java index a855a1c2c39..de68269ab70 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferData.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferData.java @@ -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 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); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferPool.java similarity index 71% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferPool.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferPool.java index bd7da11ddd8..189357f6bd0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/BufferPool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/BufferPool.java @@ -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. - * + *

* 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 pool; - // Allows associating metadata to each buffer in the pool. + /** + * Allows associating metadata to each buffer in the pool. + */ private Map 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(); this.prefetchingStatistics = requireNonNull(prefetchingStatistics); this.pool = new BoundedResourcePool(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 getAll() { - synchronized (this.allocated) { - return Collections.unmodifiableList(new ArrayList(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 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 allData = new ArrayList<>(this.getAll()); - Collections.sort(allData, (d1, d2) -> d1.getBlockNumber() - d2.getBlockNumber()); + List 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; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/CachingBlockManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java similarity index 78% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/CachingBlockManager.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java index 1207d3d0318..31084c7bf26 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/CachingBlockManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/CachingBlockManager.java @@ -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 prefetchFuture = this.futurePool.executeFunction(prefetchTask); + Future 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 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 actionFuture = this.futurePool.executeFunction(task); + Future 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 { @@ -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(); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/EmptyPrefetchingStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/EmptyPrefetchingStatistics.java new file mode 100644 index 00000000000..177ff7abab8 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/EmptyPrefetchingStatistics.java @@ -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) { + + } +} + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ExecutorServiceFuturePool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ExecutorServiceFuturePool.java similarity index 98% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ExecutorServiceFuturePool.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ExecutorServiceFuturePool.java index 932a047c940..9ef50e50d7e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ExecutorServiceFuturePool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ExecutorServiceFuturePool.java @@ -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; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/FilePosition.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java similarity index 70% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/FilePosition.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java index 24c46bc4d1f..7cd3bb3de2b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/FilePosition.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/FilePosition.java @@ -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"); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/PrefetchingStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java similarity index 97% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/PrefetchingStatistics.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java index b1894f97696..9ce2dec5889 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/PrefetchingStatistics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/PrefetchingStatistics.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.hadoop.fs.common; +package org.apache.hadoop.fs.impl.prefetch; import java.time.Duration; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ResourcePool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ResourcePool.java similarity index 97% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ResourcePool.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ResourcePool.java index 11affb590f4..77e00972d08 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/ResourcePool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/ResourcePool.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.hadoop.fs.common; +package org.apache.hadoop.fs.impl.prefetch; import java.io.Closeable; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Retryer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Retryer.java similarity index 80% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Retryer.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Retryer.java index 5aea7897008..84c17ef9dde 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Retryer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Retryer.java @@ -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; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java similarity index 86% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java index 7252c294bee..c84335a763e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/SingleFilePerBlockCache.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/SingleFilePerBlockCache.java @@ -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 blocks = new ConcurrentHashMap<>(); + /** + * Blocks stored in this cache. + */ + private final Map 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 blocks() { - return Collections.unmodifiableList(new ArrayList(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 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(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Validate.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Validate.java similarity index 90% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Validate.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Validate.java index 18e77fd1d0e..17a668a0d3b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/Validate.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/Validate.java @@ -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. - * + *

* 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 the type of array's elements. * @param array the argument reference to validate. * @param argName the name of the argument being validated. */ public static 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 the type of iterable's elements. * @param iter the argument reference to validate. * @param argName the name of the argument being validated. */ - public static void checkNotNullAndNotEmpty(Iterable iter, String argName) { - Validate.checkNotNull(iter, argName); + public static void checkNotNullAndNotEmpty(Iterable 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 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 void checkNotNullAndNumberOfElements( Collection 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); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/package-info.java similarity index 95% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/package-info.java rename to hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/package-info.java index f6f7b6f9e01..1b26da85d95 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/common/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/prefetch/package-info.java @@ -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; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/ExceptionAsserts.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/ExceptionAsserts.java similarity index 93% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/ExceptionAsserts.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/ExceptionAsserts.java index 96b7acdcdb8..82348d97798 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/ExceptionAsserts.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/ExceptionAsserts.java @@ -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 void assertThrows( Class expectedExceptionClass, LambdaTestUtils.VoidCallable code) throws Exception { - assertThrows(expectedExceptionClass, null, code); + + intercept(expectedExceptionClass, code); + } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/SampleDataForTests.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/SampleDataForTests.java similarity index 60% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/SampleDataForTests.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/SampleDataForTests.java index 97c2e0b7659..b6f744582d3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/SampleDataForTests.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/SampleDataForTests.java @@ -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 NULL_LIST = null; - public static final List NULL_LIST = null; public static final List EMPTY_LIST = new ArrayList(); + public static final List VALID_LIST = Arrays.asList(new Object[1]); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockCache.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockCache.java similarity index 87% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockCache.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockCache.java index c402673a49d..2ea041283a7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockCache.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockCache.java @@ -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++) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockData.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockData.java similarity index 89% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockData.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockData.java index dd8c9fb3c7e..50ce220f652 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockData.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockData.java @@ -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) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockOperations.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockOperations.java similarity index 80% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockOperations.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockOperations.java index da46db1e550..703041379ab 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBlockOperations.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBlockOperations.java @@ -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); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBoundedResourcePool.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBoundedResourcePool.java similarity index 91% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBoundedResourcePool.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBoundedResourcePool.java index 154e8464a49..fc29e1b7254 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBoundedResourcePool.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBoundedResourcePool.java @@ -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 { + 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 diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferData.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java similarity index 91% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferData.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java index c4699d11540..ee5f95ca6bb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferData.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferData.java @@ -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; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferPool.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java similarity index 70% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferPool.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java index c9134f1e251..b8375fe66dc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestBufferPool.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestBufferPool.java @@ -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); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestExecutorServiceFuturePool.java similarity index 75% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestExecutorServiceFuturePool.java index 00055a9ea1e..3b8bc75f149 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestExecutorServiceFuturePool.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestExecutorServiceFuturePool.java @@ -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 future = futurePool.executeRunnable(() -> atomicBoolean.set(true)); + Future 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 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 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 future = futurePool.executeFunction(() -> { throw new IllegalStateException("deliberate"); }); - LambdaTestUtils.intercept(ExecutionException.class, () -> future.get(30, TimeUnit.SECONDS)); + interceptFuture(IllegalStateException.class, "deliberate", 30, + TimeUnit.SECONDS, future); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestFilePosition.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestFilePosition.java similarity index 74% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestFilePosition.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestFilePosition.java index a1b4ae610a1..e86c4be97b9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestFilePosition.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestFilePosition.java @@ -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 diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestRetryer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestRetryer.java similarity index 80% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestRetryer.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestRetryer.java index 7220a7ec8fc..50701c717a4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestRetryer.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestRetryer.java @@ -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) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestValidate.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestValidate.java similarity index 62% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestValidate.java rename to hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestValidate.java index bffbde417b5..a42462b3355 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/TestValidate.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/impl/prefetch/TestValidate.java @@ -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(""); // 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 () 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")); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 18c09180389..47284195a3a 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -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), diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java index afbd91a3694..b19a70dec91 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java @@ -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; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3BlockManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ABlockManager.java similarity index 77% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3BlockManager.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ABlockManager.java index f7c47ef7925..adbb06bf142 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3BlockManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ABlockManager.java @@ -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(); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingBlockManager.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingBlockManager.java similarity index 77% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingBlockManager.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingBlockManager.java index 1c058087f31..f82786659da 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingBlockManager.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingBlockManager.java @@ -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); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java similarity index 68% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingInputStream.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java index b00119ac4e1..0afd0712464 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3CachingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ACachingInputStream.java @@ -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()); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InMemoryInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java similarity index 79% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InMemoryInputStream.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java index c97cf38669b..322459a9589 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InMemoryInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3AInMemoryInputStream.java @@ -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(); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3PrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java similarity index 77% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3PrefetchingInputStream.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java index 66dd7c2a375..76ef942ed65 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3PrefetchingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchingInputStream.java @@ -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; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java similarity index 76% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InputStream.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java index bbc9008c73a..0f46a8ed5e5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3InputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteInputStream.java @@ -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); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3File.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java similarity index 87% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3File.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java index ac22976e5bc..d749e9df02f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3File.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObject.java @@ -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; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3Reader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java similarity index 65% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3Reader.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java index 19ab4f6961d..89ea77d6d0e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/S3Reader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/S3ARemoteObjectReader.java @@ -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); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/package-info.java similarity index 96% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/package-info.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/package-info.java index b255537b40a..f953d35dcca 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/read/package-info.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/prefetch/package-info.java @@ -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; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java index e74a6d59a86..ec44bbbcb0f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/S3AInputStreamStatistics.java @@ -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; /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/EmptyPrefetchingStatistics.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/EmptyPrefetchingStatistics.java deleted file mode 100644 index ce13d2d9929..00000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/common/EmptyPrefetchingStatistics.java +++ /dev/null @@ -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) { - - } - -} - diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3PrefetchingInputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java similarity index 91% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3PrefetchingInputStream.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java index f46e93e1084..36d049cedf1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3PrefetchingInputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingInputStream.java @@ -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"); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index 58404b2d784..d3925d35a99 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -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 { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/MockS3File.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java similarity index 82% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/MockS3File.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java index 82b7a10d40c..6e2f547a22e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/MockS3File.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/MockS3ARemoteObject.java @@ -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); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/Fakes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java similarity index 76% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/Fakes.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java index 0aec7041f45..bab07f4f9ec 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/Fakes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/S3APrefetchFakes.java @@ -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 clazz, + public static S3ARemoteInputStream createInputStream( + Class 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 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); } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3BlockManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java similarity index 67% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3BlockManager.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java index eb3b700f280..c1b59d6f2e1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3BlockManager.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ABlockManager.java @@ -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); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3CachingBlockManager.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java similarity index 68% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3CachingBlockManager.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java index a9ebae276f3..aecf8802beb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3CachingBlockManager.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ACachingBlockManager.java @@ -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()); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3InputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java similarity index 67% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3InputStream.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java index cf3ad400afe..4ab33ef6cd0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3InputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteInputStream.java @@ -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()); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3File.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java similarity index 62% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3File.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java index 9f63ea0a889..b3788aac808 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3File.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObject.java @@ -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)); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3Reader.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java similarity index 80% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3Reader.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java index 10e5e29da2d..db70c4f22bc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/read/TestS3Reader.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/prefetch/TestS3ARemoteObjectReader.java @@ -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]); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index 6092abb77c6..0ab610a004e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -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); } }